diff --git hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java index 137323c3a4..b290a40734 100644 --- hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java +++ hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -46,6 +47,8 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hive.cli.CliSessionState; import org.apache.hadoop.hive.common.JavaUtils; +import org.apache.hadoop.hive.common.TableName; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.Validator; @@ -61,11 +64,13 @@ import org.apache.hadoop.hive.metastore.api.ShowLocksRequest; import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement; +import org.apache.hadoop.hive.metastore.api.TableValidWriteIds; import org.apache.hadoop.hive.metastore.api.TxnAbortedException; import org.apache.hadoop.hive.metastore.api.TxnInfo; import org.apache.hadoop.hive.metastore.api.TxnState; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; @@ -407,13 +412,13 @@ public void testNoBuckets() throws Exception { rs = queryTable(driver,"select ROW__ID, a, b, INPUT__FILE__NAME from default.streamingnobuckets order by ROW__ID"); Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\tfoo\tbar")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/base_0000005/bucket_00000")); + Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\ta3\tb4")); - Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/base_0000005/bucket_00000")); + Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\ta5\tb6")); - Assert.assertTrue(rs.get(2), rs.get(2).endsWith("streamingnobuckets/base_0000005/bucket_00000")); + Assert.assertTrue(rs.get(2), rs.get(2).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":4,\"bucketid\":536870912,\"rowid\":0}\t0\t0")); - Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/base_0000005/bucket_00000")); + Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); } /** @@ -553,7 +558,7 @@ public void testTableValidation() throws Exception { @Deprecated private void checkDataWritten(Path partitionPath, long minTxn, long maxTxn, int buckets, int numExpectedFiles, String... records) throws Exception { - ValidWriteIdList writeIds = msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName)); + ValidWriteIdList writeIds = getTransactionContext(conf); AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, writeIds); Assert.assertEquals(0, dir.getObsolete().size()); Assert.assertEquals(0, dir.getOriginalFiles().size()); @@ -587,6 +592,7 @@ private void checkDataWritten(Path partitionPath, long minTxn, long maxTxn, int AcidUtils.setAcidOperationalProperties(job, true, null); job.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); job.set(ValidWriteIdList.VALID_WRITEIDS_KEY, writeIds.toString()); + job.set(ValidTxnList.VALID_TXNS_KEY, conf.get(ValidTxnList.VALID_TXNS_KEY)); InputSplit[] splits = inf.getSplits(job, buckets); Assert.assertEquals(numExpectedFiles, splits.length); org.apache.hadoop.mapred.RecordReader rr = @@ -606,8 +612,7 @@ private void checkDataWritten(Path partitionPath, long minTxn, long maxTxn, int */ private void checkDataWritten2(Path partitionPath, long minTxn, long maxTxn, int numExpectedFiles, String validationQuery, boolean vectorize, String... records) throws Exception { - ValidWriteIdList txns = msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName)); - AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, txns); + AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, getTransactionContext(conf)); Assert.assertEquals(0, dir.getObsolete().size()); Assert.assertEquals(0, dir.getOriginalFiles().size()); List current = dir.getCurrentDirectories(); @@ -649,9 +654,15 @@ private void checkDataWritten2(Path partitionPath, long minTxn, long maxTxn, int conf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorizationEnabled); } + private ValidWriteIdList getTransactionContext(Configuration conf) throws Exception { + ValidTxnList validTxnList = msClient.getValidTxns(); + conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); + List v = msClient.getValidWriteIds(Collections + .singletonList(TableName.getDbTable(dbName, tblName)), validTxnList.writeToString()); + return TxnCommonUtils.createValidReaderWriteIdList(v.get(0)); + } private void checkNothingWritten(Path partitionPath) throws Exception { - ValidWriteIdList writeIds = msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName)); - AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, writeIds); + AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, getTransactionContext(conf)); Assert.assertEquals(0, dir.getObsolete().size()); Assert.assertEquals(0, dir.getOriginalFiles().size()); List current = dir.getCurrentDirectories(); @@ -1234,8 +1245,7 @@ public void testInterleavedTransactionBatchCommits() throws Exception { /*now both batches have committed (but not closed) so we for each primary file we expect a side file to exist and indicate the true length of primary file*/ FileSystem fs = partLoc.getFileSystem(conf); - AcidUtils.Directory dir = AcidUtils.getAcidState(partLoc, conf, - msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName))); + AcidUtils.Directory dir = AcidUtils.getAcidState(partLoc, conf, getTransactionContext(conf)); for(AcidUtils.ParsedDelta pd : dir.getCurrentDirectories()) { for(FileStatus stat : fs.listStatus(pd.getPath(), AcidUtils.bucketFileFilter)) { Path lengthFile = OrcAcidUtils.getSideFile(stat.getPath()); @@ -1260,7 +1270,7 @@ public void testInterleavedTransactionBatchCommits() throws Exception { //so each of 2 deltas has 1 bucket0 and 1 bucket0_flush_length. Furthermore, each bucket0 //has now received more data(logically - it's buffered) but it is not yet committed. //lets check that side files exist, etc - dir = AcidUtils.getAcidState(partLoc, conf, msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName))); + dir = AcidUtils.getAcidState(partLoc, conf, getTransactionContext(conf)); for(AcidUtils.ParsedDelta pd : dir.getCurrentDirectories()) { for(FileStatus stat : fs.listStatus(pd.getPath(), AcidUtils.bucketFileFilter)) { Path lengthFile = OrcAcidUtils.getSideFile(stat.getPath()); diff --git hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java index 0edf1cd7fe..59bba16ffa 100644 --- hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java +++ hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java @@ -25,6 +25,8 @@ import java.util.List; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.TableName; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; @@ -32,7 +34,9 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.TableValidWriteIds; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; import org.apache.hadoop.hive.ql.io.AcidInputFormat.AcidRecordReader; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.io.IOConstants; @@ -72,6 +76,7 @@ public StreamingAssert newStreamingAssert(Table table, List partition) t private IMetaStoreClient metaStoreClient; private Directory dir; private ValidWriteIdList writeIds; + private ValidTxnList validTxnList; private List currentDeltas; private long min; private long max; @@ -83,7 +88,14 @@ public StreamingAssert newStreamingAssert(Table table, List partition) t this.table = table; this.partition = partition; - writeIds = metaStoreClient.getValidWriteIds(AcidUtils.getFullTableName(table.getDbName(), table.getTableName())); + + validTxnList = metaStoreClient.getValidTxns(); + conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); + List v = metaStoreClient.getValidWriteIds(Collections + .singletonList(TableName.getDbTable(table.getDbName(), table.getTableName())), validTxnList.writeToString()); + writeIds = TxnCommonUtils.createValidReaderWriteIdList(v.get(0)); + +// writeIds = metaStoreClient.getValidWriteIds(AcidUtils.getFullTableName(table.getDbName(), table.getTableName())); partitionLocation = getPartitionLocation(); dir = AcidUtils.getAcidState(partitionLocation, conf, writeIds); assertEquals(0, dir.getObsolete().size()); @@ -146,6 +158,7 @@ public void assertMaxWriteId(long expectedMaxWriteId) { AcidUtils.setAcidOperationalProperties(job, true, null); job.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); job.set(ValidWriteIdList.VALID_WRITEIDS_KEY, writeIds.toString()); + job.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); InputSplit[] splits = inputFormat.getSplits(job, 1); assertEquals(numSplitsExpected, splits.length); diff --git itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java index 40dd992455..c630d5dddb 100644 --- itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java +++ itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java @@ -608,11 +608,11 @@ public void testInsertWithRemoveUnion() throws Exception { LOG.warn(s); } String[][] expected2 = { - {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "warehouse/t/base_-9223372036854775808/bucket_00000"}, - {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "warehouse/t/base_-9223372036854775808/bucket_00000"}, - {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":2}\t5\t6", "warehouse/t/base_-9223372036854775808/bucket_00000"}, - {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":3}\t7\t8", "warehouse/t/base_-9223372036854775808/bucket_00000"}, - {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":4}\t9\t10", "warehouse/t/base_-9223372036854775808/bucket_00000"} + {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "warehouse/t/base_-9223372036854775808_v0000024/bucket_00000"}, + {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "warehouse/t/base_-9223372036854775808_v0000024/bucket_00000"}, + {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":2}\t5\t6", "warehouse/t/base_-9223372036854775808_v0000024/bucket_00000"}, + {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":3}\t7\t8", "warehouse/t/base_-9223372036854775808_v0000024/bucket_00000"}, + {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":4}\t9\t10", "warehouse/t/base_-9223372036854775808_v0000024/bucket_00000"} }; Assert.assertEquals("Unexpected row count after major compact", expected2.length, rs.size()); for(int i = 0; i < expected2.length; i++) { diff --git itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java index 9648645edb..222597abac 100644 --- itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java +++ itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java @@ -122,6 +122,8 @@ public TestCompactor(boolean newStreamingAPI) { this.newStreamingAPI = newStreamingAPI; } + + @Rule public TemporaryFolder stagingFolder = new TemporaryFolder(); @@ -713,15 +715,15 @@ public void minorCompactWhileStreaming() throws Exception { Path resultFile = null; for (int i = 0; i < names.length; i++) { names[i] = stat[i].getPath().getName(); - if (names[i].equals("delta_0000001_0000004")) { + if (names[i].equals("delta_0000001_0000004_v0000009")) { resultFile = stat[i].getPath(); } } Arrays.sort(names); String[] expected = new String[]{"delta_0000001_0000002", - "delta_0000001_0000004", "delta_0000003_0000004", "delta_0000005_0000006"}; + "delta_0000001_0000004_v0000009", "delta_0000003_0000004", "delta_0000005_0000006"}; if (!Arrays.deepEquals(expected, names)) { - Assert.fail("Expected: " + Arrays.toString(expected) + ", found: " + Arrays.toString(names)); + Assert.fail("Expected: " + Arrays.toString(expected) + ", found: " + Arrays.toString(names) + ",stat=" + toString(stat)); } checkExpectedTxnsPresent(null, new Path[]{resultFile}, columnNamesProperty, columnTypesProperty, 0, 1L, 4L, 1); @@ -767,7 +769,7 @@ public void majorCompactWhileStreaming() throws Exception { Assert.fail("Expecting 1 file \"base_0000004\" and found " + stat.length + " files " + Arrays.toString(stat)); } String name = stat[0].getPath().getName(); - Assert.assertEquals(name, "base_0000004"); + Assert.assertEquals("base_0000004_v0000009", name); checkExpectedTxnsPresent(stat[0].getPath(), null, columnNamesProperty, columnTypesProperty, 0, 1L, 4L, 1); } finally { if (connection != null) { @@ -858,13 +860,13 @@ public void minorCompactAfterAbort() throws Exception { Path resultDelta = null; for (int i = 0; i < names.length; i++) { names[i] = stat[i].getPath().getName(); - if (names[i].equals("delta_0000001_0000004")) { + if (names[i].equals("delta_0000001_0000004_v0000009")) { resultDelta = stat[i].getPath(); } } Arrays.sort(names); String[] expected = new String[]{"delta_0000001_0000002", - "delta_0000001_0000004", "delta_0000003_0000004"}; + "delta_0000001_0000004_v0000009", "delta_0000003_0000004"}; if (!Arrays.deepEquals(expected, names)) { Assert.fail("Expected: " + Arrays.toString(expected) + ", found: " + Arrays.toString(names)); } @@ -947,7 +949,7 @@ public void majorCompactAfterAbort() throws Exception { Assert.fail("Expecting 1 file \"base_0000004\" and found " + stat.length + " files " + Arrays.toString(stat)); } String name = stat[0].getPath().getName(); - if (!name.equals("base_0000004")) { + if (!name.equals("base_0000004_v0000009")) { Assert.fail("majorCompactAfterAbort name " + name + " not equals to base_0000004"); } checkExpectedTxnsPresent(stat[0].getPath(), null, columnNamesProperty, columnTypesProperty, 0, 1L, 4L, 1); @@ -1312,7 +1314,7 @@ public void majorCompactWhileStreamingForSplitUpdate() throws Exception { Assert.fail("Expecting 1 file \"base_0000004\" and found " + stat.length + " files " + Arrays.toString(stat)); } String name = stat[0].getPath().getName(); - Assert.assertEquals(name, "base_0000004"); + Assert.assertEquals("base_0000004_v0000009", name); checkExpectedTxnsPresent(stat[0].getPath(), null, columnNamesProperty, columnTypesProperty, 1, 1L, 4L, 2); if (connection1 != null) { connection1.close(); @@ -1368,12 +1370,12 @@ public void testMinorCompactionForSplitUpdateWithInsertsAndDeletes() throws Exce Path minorCompactedDelta = null; for (int i = 0; i < deltas.length; i++) { deltas[i] = stat[i].getPath().getName(); - if (deltas[i].equals("delta_0000001_0000003")) { + if (deltas[i].equals("delta_0000001_0000003_v0000006")) { minorCompactedDelta = stat[i].getPath(); } } Arrays.sort(deltas); - String[] expectedDeltas = new String[]{"delta_0000001_0000001_0000", "delta_0000001_0000003", + String[] expectedDeltas = new String[]{"delta_0000001_0000001_0000", "delta_0000001_0000003_v0000006", "delta_0000002_0000002_0000"}; if (!Arrays.deepEquals(expectedDeltas, deltas)) { Assert.fail("Expected: " + Arrays.toString(expectedDeltas) + ", found: " + Arrays.toString(deltas)); @@ -1388,12 +1390,12 @@ public void testMinorCompactionForSplitUpdateWithInsertsAndDeletes() throws Exce Path minorCompactedDeleteDelta = null; for (int i = 0; i < deleteDeltas.length; i++) { deleteDeltas[i] = deleteDeltaStat[i].getPath().getName(); - if (deleteDeltas[i].equals("delete_delta_0000001_0000003")) { + if (deleteDeltas[i].equals("delete_delta_0000001_0000003_v0000006")) { minorCompactedDeleteDelta = deleteDeltaStat[i].getPath(); } } Arrays.sort(deleteDeltas); - String[] expectedDeleteDeltas = new String[]{"delete_delta_0000001_0000003", "delete_delta_0000003_0000003_0000"}; + String[] expectedDeleteDeltas = new String[]{"delete_delta_0000001_0000003_v0000006", "delete_delta_0000003_0000003_0000"}; if (!Arrays.deepEquals(expectedDeleteDeltas, deleteDeltas)) { Assert.fail("Expected: " + Arrays.toString(expectedDeleteDeltas) + ", found: " + Arrays.toString(deleteDeltas)); } @@ -1446,12 +1448,12 @@ public void testMinorCompactionForSplitUpdateWithOnlyInserts() throws Exception Path minorCompactedDelta = null; for (int i = 0; i < deltas.length; i++) { deltas[i] = stat[i].getPath().getName(); - if (deltas[i].equals("delta_0000001_0000002")) { + if (deltas[i].equals("delta_0000001_0000002_v0000005")) { minorCompactedDelta = stat[i].getPath(); } } Arrays.sort(deltas); - String[] expectedDeltas = new String[]{"delta_0000001_0000001_0000", "delta_0000001_0000002", + String[] expectedDeltas = new String[]{"delta_0000001_0000001_0000", "delta_0000001_0000002_v0000005", "delta_0000002_0000002_0000"}; if (!Arrays.deepEquals(expectedDeltas, deltas)) { Assert.fail("Expected: " + Arrays.toString(expectedDeltas) + ", found: " + Arrays.toString(deltas)); @@ -1466,12 +1468,12 @@ public void testMinorCompactionForSplitUpdateWithOnlyInserts() throws Exception Path minorCompactedDeleteDelta = null; for (int i = 0; i < deleteDeltas.length; i++) { deleteDeltas[i] = deleteDeltaStat[i].getPath().getName(); - if (deleteDeltas[i].equals("delete_delta_0000001_0000002")) { + if (deleteDeltas[i].equals("delete_delta_0000001_0000002_v0000005")) { minorCompactedDeleteDelta = deleteDeltaStat[i].getPath(); } } Arrays.sort(deleteDeltas); - String[] expectedDeleteDeltas = new String[]{"delete_delta_0000001_0000002"}; + String[] expectedDeleteDeltas = new String[]{"delete_delta_0000001_0000002_v0000005"}; if (!Arrays.deepEquals(expectedDeleteDeltas, deleteDeltas)) { Assert.fail("Expected: " + Arrays.toString(expectedDeleteDeltas) + ", found: " + Arrays.toString(deleteDeltas)); } @@ -1537,13 +1539,13 @@ public void minorCompactWhileStreamingWithSplitUpdate() throws Exception { Path resultFile = null; for (int i = 0; i < names.length; i++) { names[i] = stat[i].getPath().getName(); - if (names[i].equals("delta_0000001_0000004")) { + if (names[i].equals("delta_0000001_0000004_v0000009")) { resultFile = stat[i].getPath(); } } Arrays.sort(names); String[] expected = new String[]{"delta_0000001_0000002", - "delta_0000001_0000004", "delta_0000003_0000004", "delta_0000005_0000006"}; + "delta_0000001_0000004_v0000009", "delta_0000003_0000004", "delta_0000005_0000006"}; if (!Arrays.deepEquals(expected, names)) { Assert.fail("Expected: " + Arrays.toString(expected) + ", found: " + Arrays.toString(names)); } @@ -1557,12 +1559,12 @@ public void minorCompactWhileStreamingWithSplitUpdate() throws Exception { Path minorCompactedDeleteDelta = null; for (int i = 0; i < deleteDeltas.length; i++) { deleteDeltas[i] = deleteDeltaStat[i].getPath().getName(); - if (deleteDeltas[i].equals("delete_delta_0000001_0000004")) { + if (deleteDeltas[i].equals("delete_delta_0000001_0000004_v0000009")) { minorCompactedDeleteDelta = deleteDeltaStat[i].getPath(); } } Arrays.sort(deleteDeltas); - String[] expectedDeleteDeltas = new String[]{"delete_delta_0000001_0000004"}; + String[] expectedDeleteDeltas = new String[]{"delete_delta_0000001_0000004_v0000009"}; if (!Arrays.deepEquals(expectedDeleteDeltas, deleteDeltas)) { Assert.fail("Expected: " + Arrays.toString(expectedDeleteDeltas) + ", found: " + Arrays.toString(deleteDeltas)); } @@ -1968,4 +1970,15 @@ static void runCleaner(HiveConf hiveConf) throws MetaException { t.init(stop, looped); t.run(); } + private static String toString(FileStatus[] stat) { + StringBuilder sb = new StringBuilder("stat{"); + if(stat == null) { + return sb.toString(); + } + for(FileStatus f : stat) { + sb.append(f.getPath()).append(","); + } + sb.setCharAt(sb.length() - 1, '}'); + return sb.toString(); + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java index 28d61f94dd..5d6ae7fe3a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java @@ -302,6 +302,7 @@ void dumpTable(String dbName, String tblName, String validTxnList, Path dbRoot, String distCpDoAsUser = conf.getVar(HiveConf.ConfVars.HIVE_DISTCP_DOAS_USER); tuple.replicationSpec.setIsReplace(true); // by default for all other objects this is false if (AcidUtils.isTransactionalTable(tableSpec.tableHandle)) { + tuple.replicationSpec.setValidTxnList(validTxnList); tuple.replicationSpec.setValidWriteIdList(getValidWriteIdList(dbName, tblName, validTxnList)); // For transactional table, data would be valid snapshot for current txn and doesn't include data diff --git ql/src/java/org/apache/hadoop/hive/ql/io/AcidInputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/AcidInputFormat.java index 41007e2df0..bba3960102 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/AcidInputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/AcidInputFormat.java @@ -112,21 +112,26 @@ private long minWriteId; private long maxWriteId; private List stmtIds; - //would be useful to have enum for Type: insert/delete/load data + /** + * {@link AcidUtils#?} + */ + private long visibilityTxnId; public DeltaMetaData() { - this(0,0,new ArrayList()); + this(0,0,new ArrayList(), 0); } /** * @param stmtIds delta dir suffixes when a single txn writes > 1 delta in the same partition + * @param visibilityTxnId maybe 0, if the dir name didn't have it. txnid:0 is always visible */ - DeltaMetaData(long minWriteId, long maxWriteId, List stmtIds) { + DeltaMetaData(long minWriteId, long maxWriteId, List stmtIds, long visibilityTxnId) { this.minWriteId = minWriteId; this.maxWriteId = maxWriteId; if (stmtIds == null) { throw new IllegalArgumentException("stmtIds == null"); } this.stmtIds = stmtIds; + this.visibilityTxnId = visibilityTxnId; } long getMinWriteId() { return minWriteId; @@ -137,6 +142,9 @@ long getMaxWriteId() { List getStmtIds() { return stmtIds; } + long getVisibilityTxnId() { + return visibilityTxnId; + } @Override public void write(DataOutput out) throws IOException { out.writeLong(minWriteId); @@ -145,6 +153,7 @@ public void write(DataOutput out) throws IOException { for(Integer id : stmtIds) { out.writeInt(id); } + out.writeLong(visibilityTxnId); } @Override public void readFields(DataInput in) throws IOException { @@ -155,11 +164,21 @@ public void readFields(DataInput in) throws IOException { for(int i = 0; i < numStatements; i++) { stmtIds.add(in.readInt()); } + visibilityTxnId = in.readLong(); + } + String getName() { + assert stmtIds.isEmpty() : "use getName(int)"; + return AcidUtils.addVisibilitySuffix(AcidUtils + .deleteDeltaSubdir(minWriteId, maxWriteId), visibilityTxnId); + } + String getName(int stmtId) { + assert !stmtIds.isEmpty() : "use getName()"; + return AcidUtils.addVisibilitySuffix(AcidUtils + .deleteDeltaSubdir(minWriteId, maxWriteId, stmtId), visibilityTxnId); } @Override public String toString() { - //? is Type - when implemented - return "Delta(?," + minWriteId + "," + maxWriteId + "," + stmtIds + ")"; + return "Delta(?," + minWriteId + "," + maxWriteId + "," + stmtIds + "," + visibilityTxnId + ")"; } } /** diff --git ql/src/java/org/apache/hadoop/hive/ql/io/AcidOutputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/AcidOutputFormat.java index 05beafe1d0..be4c85d4b1 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/AcidOutputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/AcidOutputFormat.java @@ -63,6 +63,10 @@ //unique within a transaction private int statementId = 0; private Path finalDestination; + /** + * + */ + private long visibilityTxnId = 0; /** * Create the options object. * @param conf Use the given configuration @@ -252,6 +256,10 @@ public Options finalDestination(Path p) { this.finalDestination = p; return this; } + public Options visibilityTxnId(long visibilityTxnId) { + this.visibilityTxnId = visibilityTxnId; + return this; + } public Configuration getConfiguration() { return configuration; @@ -317,6 +325,9 @@ public int getCopyNumber() { public Path getFinalDestination() { return finalDestination; } + public long getVisibilityTxnId() { + return visibilityTxnId; + } } /** diff --git ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java index 71e51315d3..89ff796d1d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java @@ -32,7 +32,7 @@ import java.util.Set; import java.util.regex.Pattern; -import org.apache.avro.generic.GenericData; +import com.google.common.base.Strings; import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -46,7 +46,6 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.LockComponentBuilder; import org.apache.hadoop.hive.metastore.TransactionalValidationListener; -import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.DataOperationType; import org.apache.hadoop.hive.metastore.api.LockComponent; import org.apache.hadoop.hive.metastore.api.LockType; @@ -57,7 +56,6 @@ import org.apache.hadoop.hive.ql.hooks.Entity; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; -import org.apache.hadoop.hive.ql.io.AcidUtils.ParsedDelta; import org.apache.hadoop.hive.ql.io.orc.OrcFile; import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; import org.apache.hadoop.hive.ql.io.orc.OrcRecordUpdater; @@ -181,6 +179,8 @@ public boolean accept(Path p){ return !name.startsWith("_") && !name.startsWith("."); } }; + public static final String VISIBILITY_PREFIX = "_v"; + public static final Pattern VISIBILITY_PATTERN = Pattern.compile(VISIBILITY_PREFIX + "\\d+"); private static final HadoopShims SHIMS = ShimLoader.getHadoopShims(); @@ -294,8 +294,16 @@ public static Path createFilename(Path directory, options.getMaximumWriteId(), options.getStatementId()); } + subdir = addVisibilitySuffix(subdir, options.getVisibilityTxnId()); return createBucketFile(new Path(directory, subdir), options.getBucketId()); } + static String addVisibilitySuffix(String baseOrDeltaDir, long visibilityTxnId) { + if(visibilityTxnId == 0) { + return baseOrDeltaDir; + } + return baseOrDeltaDir + VISIBILITY_PREFIX + + String.format(DELTA_DIGITS, visibilityTxnId); + } /** * Represents bucketId and copy_N suffix */ @@ -339,20 +347,6 @@ private BucketMetaData(int bucketId, int copyNumber) { this.copyNumber = copyNumber; } } - /** - * Get the write id from a base directory name. - * @param path the base directory name - * @return the maximum write id that is included - */ - public static long parseBase(Path path) { - String filename = path.getName(); - if (filename.startsWith(BASE_PREFIX)) { - return Long.parseLong(filename.substring(BASE_PREFIX.length())); - } - throw new IllegalArgumentException(filename + " does not start with " + - BASE_PREFIX); - } - /** * Get the bucket id from the file path * @param bucketFile - bucket file path @@ -405,7 +399,7 @@ else if (filename.startsWith(BUCKET_PREFIX)) { result .setOldStyle(false) .minimumWriteId(0) - .maximumWriteId(parseBase(bucketFile.getParent())) + .maximumWriteId(ParsedBase.parseBase(bucketFile.getParent()).getWriteId()) .bucket(bucket) .writingBase(true); } else if (bucketFile.getParent().getName().startsWith(DELTA_PREFIX)) { @@ -443,11 +437,12 @@ else if (filename.startsWith(BUCKET_PREFIX)) { public DirectoryImpl(List abortedDirectories, boolean isBaseInRawFormat, List original, List obsolete, List deltas, Path base) { - this.abortedDirectories = abortedDirectories; + this.abortedDirectories = abortedDirectories == null ? + Collections.emptyList() : abortedDirectories; this.isBaseInRawFormat = isBaseInRawFormat; - this.original = original; - this.obsolete = obsolete; - this.deltas = deltas; + this.original = original == null ? Collections.emptyList() : original; + this.obsolete = obsolete == null ? Collections.emptyList() : obsolete; + this.deltas = deltas == null ? Collections.emptyList() : deltas; this.base = base; } @@ -757,7 +752,46 @@ public String toString() { */ List getAbortedDirectories(); } + /** + * Since version 3 but prior to version 4, format of a base is "base_X" where X is a writeId. + * If this base was produced by a compactor, X is the highest writeId that the compactor included. + * If this base is produced by Insert Overwrite stmt, X is a writeId of the transaction that + * executed the insert. + * Since Hive Version 4.0, the format of a base produced by a compactor is + * base_X_vY. X is like before, i.e. the highest writeId compactor included and Y is the visibilityTxnId + * of the transaction in which the compactor ran. (v(isibility) is a literal to help parsing). + */ + public static final class ParsedBase { + private final long writeId; + private final long visibilityTxnId; + ParsedBase(long writeId) { + this(writeId, 0); + } + ParsedBase(long writeId, long visibilityTxnId) { + this.writeId = writeId; + this.visibilityTxnId = visibilityTxnId; + } + public long getWriteId() { + return writeId; + } + public long getVisibilityTxnId() { + return visibilityTxnId; + } + public static ParsedBase parseBase(Path path) { + String filename = path.getName(); + if (filename.startsWith(BASE_PREFIX)) { + int idxOfv = filename.indexOf(VISIBILITY_PREFIX); + if(idxOfv < 0) { + return new ParsedBase(Long.parseLong(filename.substring(BASE_PREFIX.length()))); + } + return new ParsedBase(Long.parseLong(filename.substring(BASE_PREFIX.length(), idxOfv)), + Long.parseLong(filename.substring(idxOfv + VISIBILITY_PREFIX.length()))); + } + throw new IllegalArgumentException(filename + " does not start with " + + BASE_PREFIX); + } + } /** * Immutable */ @@ -770,16 +804,20 @@ public String toString() { private final int statementId; private final boolean isDeleteDelta; // records whether delta dir is of type 'delete_delta_x_y...' private final boolean isRawFormat; - + /** + * transaction Id of txn which created this delta. This dir should be considered + * invisible unless this txn is committed + */ + private final long visibilityTxnId; /** * for pre 1.3.x delta files */ private ParsedDelta(long min, long max, FileStatus path, boolean isDeleteDelta, - boolean isRawFormat) { - this(min, max, path, -1, isDeleteDelta, isRawFormat); + boolean isRawFormat, long visibilityTxnId) { + this(min, max, path, -1, isDeleteDelta, isRawFormat, visibilityTxnId); } private ParsedDelta(long min, long max, FileStatus path, int statementId, - boolean isDeleteDelta, boolean isRawFormat) { + boolean isDeleteDelta, boolean isRawFormat, long visibilityTxnId) { this.minWriteId = min; this.maxWriteId = max; this.path = path; @@ -787,6 +825,7 @@ private ParsedDelta(long min, long max, FileStatus path, int statementId, this.isDeleteDelta = isDeleteDelta; this.isRawFormat = isRawFormat; assert !isDeleteDelta || !isRawFormat : " deleteDelta should not be raw format"; + this.visibilityTxnId = visibilityTxnId; } public long getMinWriteId() { @@ -814,6 +853,9 @@ public boolean isDeleteDelta() { public boolean isRawFormat() { return isRawFormat; } + public long getVisibilityTxnId() { + return visibilityTxnId; + } /** * Compactions (Major/Minor) merge deltas/bases but delete of old files * happens in a different process; thus it's possible to have bases/deltas with @@ -886,7 +928,7 @@ else if(statementId != parsedDelta.statementId) { continue; } last = new AcidInputFormat.DeltaMetaData(parsedDelta.getMinWriteId(), - parsedDelta.getMaxWriteId(), new ArrayList()); + parsedDelta.getMaxWriteId(), new ArrayList<>(), parsedDelta.getVisibilityTxnId()); result.add(last); if (parsedDelta.statementId >= 0) { last.getStmtIds().add(parsedDelta.getStatementId()); @@ -905,14 +947,14 @@ else if(statementId != parsedDelta.statementId) { * @return the list of delta paths */ public static Path[] deserializeDeleteDeltas(Path root, final List deleteDeltas) throws IOException { - List results = new ArrayList(deleteDeltas.size()); + List results = new ArrayList<>(deleteDeltas.size()); for(AcidInputFormat.DeltaMetaData dmd : deleteDeltas) { if(dmd.getStmtIds().isEmpty()) { - results.add(new Path(root, deleteDeltaSubdir(dmd.getMinWriteId(), dmd.getMaxWriteId()))); + results.add(new Path(root, dmd.getName())); continue; } for(Integer stmtId : dmd.getStmtIds()) { - results.add(new Path(root, deleteDeltaSubdir(dmd.getMinWriteId(), dmd.getMaxWriteId(), stmtId))); + results.add(new Path(root, dmd.getName(stmtId))); } } return results.toArray(new Path[results.size()]); @@ -936,7 +978,7 @@ private static ParsedDelta parseDelta(FileStatus path, String deltaPrefix, FileS ParsedDelta p = parsedDelta(path.getPath(), deltaPrefix, fs); boolean isDeleteDelta = deltaPrefix.equals(DELETE_DELTA_PREFIX); return new ParsedDelta(p.getMinWriteId(), - p.getMaxWriteId(), path, p.statementId, isDeleteDelta, p.isRawFormat()); + p.getMaxWriteId(), path, p.statementId, isDeleteDelta, p.isRawFormat(), p.visibilityTxnId); } public static ParsedDelta parsedDelta(Path deltaDir, String deltaPrefix, FileSystem fs) @@ -959,6 +1001,12 @@ public static ParsedDelta parsedDelta(Path deltaDir, String deltaPrefix, FileSys */ public static ParsedDelta parsedDelta(Path deltaDir, boolean isRawFormat) { String filename = deltaDir.getName(); + int idxOfVis = filename.indexOf(VISIBILITY_PREFIX); + long visibilityTxnId = 0;//visibilityTxnId:0 is always visible + if(idxOfVis >= 0) { + visibilityTxnId = Long.parseLong(filename.substring(idxOfVis + VISIBILITY_PREFIX.length())); + filename = filename.substring(0, idxOfVis); + } boolean isDeleteDelta = filename.startsWith(DELETE_DELTA_PREFIX); //make sure it's null for delete delta no matter what was passed in - this //doesn't apply to delete deltas @@ -966,18 +1014,18 @@ public static ParsedDelta parsedDelta(Path deltaDir, boolean isRawFormat) { String rest = filename.substring((isDeleteDelta ? DELETE_DELTA_PREFIX : DELTA_PREFIX).length()); int split = rest.indexOf('_'); - //may be -1 if no statementId + //split2 may be -1 if no statementId int split2 = rest.indexOf('_', split + 1); long min = Long.parseLong(rest.substring(0, split)); long max = split2 == -1 ? Long.parseLong(rest.substring(split + 1)) : Long.parseLong(rest.substring(split + 1, split2)); if(split2 == -1) { - return new ParsedDelta(min, max, null, isDeleteDelta, isRawFormat); + return new ParsedDelta(min, max, null, isDeleteDelta, isRawFormat, visibilityTxnId); } int statementId = Integer.parseInt(rest.substring(split2 + 1)); return new ParsedDelta(min, max, null, statementId, isDeleteDelta, - isRawFormat); + isRawFormat, visibilityTxnId); } @@ -1046,6 +1094,23 @@ public static Directory getAcidState(Path directory, Ref useFileIds, boolean ignoreEmptyFiles, Map tblproperties) throws IOException { + ValidTxnList validTxnList = null; + String s = conf.get(ValidTxnList.VALID_TXNS_KEY); + if(!Strings.isNullOrEmpty(s)) { + /** + * getAcidState() is sometimes called on non-transactional tables, e.g. + * OrcInputFileFormat.FileGenerator.callInternal(). e.g. orc_merge3.q In that case + * writeIdList is bogus - doesn't even have a table name. + * see https://issues.apache.org/jira/browse/HIVE-20856. + * + * For now, assert that ValidTxnList.VALID_TXNS_KEY is set only if this is really a read + * of a transactional table. + * see {@link #getChildState(FileStatus, HdfsFileStatusWithId, ValidWriteIdList, List, List, List, List, TxnBase, boolean, List, Map, FileSystem, ValidTxnList)} + */ + validTxnList = new ValidReadTxnList(); + validTxnList.readFromString(s); + } + FileSystem fs = directory.getFileSystem(conf); // The following 'deltas' includes all kinds of delta files including insert & delete deltas. final List deltas = new ArrayList(); @@ -1073,13 +1138,13 @@ public static Directory getAcidState(Path directory, if (childrenWithId != null) { for (HdfsFileStatusWithId child : childrenWithId) { getChildState(child.getFileStatus(), child, writeIdList, working, originalDirectories, original, - obsolete, bestBase, ignoreEmptyFiles, abortedDirectories, tblproperties, fs); + obsolete, bestBase, ignoreEmptyFiles, abortedDirectories, tblproperties, fs, validTxnList); } } else { List children = HdfsUtils.listLocatedStatus(fs, directory, hiddenFileFilter); for (FileStatus child : children) { getChildState(child, null, writeIdList, working, originalDirectories, original, obsolete, - bestBase, ignoreEmptyFiles, abortedDirectories, tblproperties, fs); + bestBase, ignoreEmptyFiles, abortedDirectories, tblproperties, fs, validTxnList); } } @@ -1219,7 +1284,7 @@ private static void getChildState(FileStatus child, HdfsFileStatusWithId childWi ValidWriteIdList writeIdList, List working, List originalDirectories, List original, List obsolete, TxnBase bestBase, boolean ignoreEmptyFiles, List aborted, Map tblproperties, - FileSystem fs) throws IOException { + FileSystem fs, ValidTxnList validTxnList) throws IOException { Path p = child.getPath(); String fn = p.getName(); if (!child.isDirectory()) { @@ -1229,7 +1294,11 @@ private static void getChildState(FileStatus child, HdfsFileStatusWithId childWi return; } if (fn.startsWith(BASE_PREFIX)) { - long writeId = parseBase(p); + ParsedBase parsedBase = ParsedBase.parseBase(p); + if(!isDirUsable(child, parsedBase.getVisibilityTxnId(), aborted, validTxnList)) { + return; + } + long writeId = parsedBase.getWriteId(); if(bestBase.oldestBaseWriteId > writeId) { //keep track for error reporting bestBase.oldestBase = p; @@ -1252,13 +1321,14 @@ private static void getChildState(FileStatus child, HdfsFileStatusWithId childWi } else if (fn.startsWith(DELTA_PREFIX) || fn.startsWith(DELETE_DELTA_PREFIX)) { String deltaPrefix = fn.startsWith(DELTA_PREFIX) ? DELTA_PREFIX : DELETE_DELTA_PREFIX; ParsedDelta delta = parseDelta(child, deltaPrefix, fs); - // Handle aborted deltas. Currently this can only happen for MM tables. - if (tblproperties != null && isTransactionalTable(tblproperties) && - ValidWriteIdList.RangeResponse.ALL == writeIdList.isWriteIdRangeAborted( - delta.minWriteId, delta.maxWriteId)) { + if(!isDirUsable(child, delta.getVisibilityTxnId(), aborted, validTxnList)) { + return; + } + if(ValidWriteIdList.RangeResponse.ALL == + writeIdList.isWriteIdRangeAborted(delta.minWriteId, delta.maxWriteId)) { aborted.add(child); } - if (writeIdList.isWriteIdRangeValid( + else if (writeIdList.isWriteIdRangeValid( delta.minWriteId, delta.maxWriteId) != ValidWriteIdList.RangeResponse.NONE) { working.add(delta); } @@ -1270,7 +1340,20 @@ private static void getChildState(FileStatus child, HdfsFileStatusWithId childWi originalDirectories.add(child); } } - + private static boolean isDirUsable(FileStatus child, long visibilityTxnId, List aborted, ValidTxnList validTxnList) { + if(validTxnList == null) { + throw new IllegalArgumentException("No ValidTxnList for " + child.getPath()); + } + if(!validTxnList.isTxnValid(visibilityTxnId)) { + boolean isAborted = validTxnList.isTxnAborted(visibilityTxnId); + if(isAborted) { + aborted.add(child); + } + LOG.debug("getChildState() ignoring(" + aborted + ") " + child); + return false; + } + return true; + } public static HdfsFileStatusWithId createOriginalObj( HdfsFileStatusWithId childWithId, FileStatus child) { return childWithId != null ? childWithId : new HdfsFileStatusWithoutId(child); @@ -1818,6 +1901,7 @@ public static ValidWriteIdList getTableValidWriteIdListWithTxnList( } public static String getFullTableName(String dbName, String tableName) { + //todo: use TableName.getDbTable( return dbName.toLowerCase() + "." + tableName.toLowerCase(); } diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java index 8cabf960db..fbb931cbcd 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java @@ -1035,7 +1035,8 @@ public Options clone() { Options readerPairOptions = mergerOptions; if(mergerOptions.getBaseDir().getName().startsWith(AcidUtils.BASE_PREFIX)) { readerPairOptions = modifyForNonAcidSchemaRead(mergerOptions, - AcidUtils.parseBase(mergerOptions.getBaseDir()), mergerOptions.getBaseDir()); + AcidUtils.ParsedBase.parseBase(mergerOptions.getBaseDir()).getWriteId(), + mergerOptions.getBaseDir()); } pair = new OriginalReaderPairToCompact(baseKey, bucket, options, readerPairOptions, conf, validWriteIdList, @@ -1223,7 +1224,8 @@ static TransactionMetaData findWriteIDForSynthetcRowIDs(Path splitPath, Path roo boolean isDelta = parent.getName().startsWith(AcidUtils.DELTA_PREFIX); if(isBase || isDelta) { if(isBase) { - return new TransactionMetaData(AcidUtils.parseBase(parent), parent); + return new TransactionMetaData(AcidUtils.ParsedBase.parseBase(parent).getWriteId(), + parent); } else { AcidUtils.ParsedDelta pd = AcidUtils.parsedDelta(parent, AcidUtils.DELTA_PREFIX, diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java index 374e973243..00cdaaff59 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java @@ -403,13 +403,13 @@ LockState acquireLocks(QueryPlan plan, Context ctx, String username, boolean isB // Make sure we need locks. It's possible there's nothing to lock in // this operation. if(plan.getInputs().isEmpty() && plan.getOutputs().isEmpty()) { - LOG.debug("No locks needed for queryId" + queryId); + LOG.debug("No locks needed for queryId=" + queryId); return null; } List lockComponents = AcidUtils.makeLockComponents(plan.getOutputs(), plan.getInputs(), conf); //It's possible there's nothing to lock even if we have w/r entities. if(lockComponents.isEmpty()) { - LOG.debug("No locks needed for queryId" + queryId); + LOG.debug("No locks needed for queryId=" + queryId); return null; } rqstBuilder.addLockComponents(lockComponents); @@ -538,6 +538,9 @@ public void replTableWriteIdState(String validWriteIdList, String dbName, String } } + //todo: this won't work if there is a txn but no locks - won't do anything + //this can be a problem for multi-stmt txns or perhaps even for single stmt + //if a query takes a long time to compile @Override public void heartbeat() throws LockException { List locks; diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSpec.java ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSpec.java index 7d901f977b..b4eeb9cbd5 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSpec.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSpec.java @@ -44,6 +44,7 @@ private boolean isLazy = false; // lazy mode => we only list files, and expect that the eventual copy will pull data in. private boolean isReplace = true; // default is that the import mode is insert overwrite private String validWriteIdList = null; // WriteIds snapshot for replicating ACID/MM tables. + private String validTxnList = null;//TxnIds snapshot private Type specType = Type.DEFAULT; // DEFAULT means REPL_LOAD or BOOTSTRAP_DUMP or EXPORT // Key definitions related to replication @@ -54,7 +55,8 @@ NOOP("repl.noop"), LAZY("repl.lazy"), IS_REPLACE("repl.is.replace"), - VALID_WRITEID_LIST("repl.valid.writeid.list") + VALID_WRITEID_LIST("repl.valid.writeid.list"), + VALID_TXN_LIST("repl.valid.txnid.list") ; private final String keyName; @@ -143,6 +145,7 @@ public ReplicationSpec(Function keyFetcher) { this.isLazy = Boolean.parseBoolean(keyFetcher.apply(ReplicationSpec.KEY.LAZY.toString())); this.isReplace = Boolean.parseBoolean(keyFetcher.apply(ReplicationSpec.KEY.IS_REPLACE.toString())); this.validWriteIdList = keyFetcher.apply(ReplicationSpec.KEY.VALID_WRITEID_LIST.toString()); + this.validTxnList = keyFetcher.apply(KEY.VALID_TXN_LIST.toString()); } /** @@ -342,6 +345,15 @@ public void setValidWriteIdList(String validWriteIdList) { this.validWriteIdList = validWriteIdList; } + public String getValidTxnList() { + return validTxnList; + } + + public void setValidTxnList(String validTxnList) { + this.validTxnList = validTxnList; + } + + /** * @return whether the current replication dumped object related to ACID/Mm table */ @@ -372,6 +384,8 @@ public String get(KEY key) { return String.valueOf(isReplace()); case VALID_WRITEID_LIST: return getValidWriteIdList(); + case VALID_TXN_LIST: + return getValidTxnList(); } return null; } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java index 59ffb90328..83a9642248 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java @@ -19,6 +19,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.NotificationEvent; @@ -209,6 +210,7 @@ public static boolean shouldReplicate(NotificationEvent tableForEvent, throws IOException { if (replicationSpec.isTransactionalTableDump()) { try { + conf.set(ValidTxnList.VALID_TXNS_KEY, replicationSpec.getValidTxnList()); return AcidUtils.getValidDataPaths(fromPath, conf, replicationSpec.getValidWriteIdList()); } catch (FileNotFoundException e) { throw new IOException(ErrorMsg.FILE_NOT_FOUND.format(e.getMessage()), e); diff --git ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java index 3bc1f8a563..f9061dd441 100644 --- ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java +++ ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java @@ -17,10 +17,17 @@ */ package org.apache.hadoop.hive.ql.txn.compactor; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.common.TableName; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.metastore.ReplChangeManager; -import org.apache.hadoop.hive.metastore.txn.TxnStore; +import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest; +import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; +import org.apache.hadoop.hive.metastore.txn.TxnStore; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.FileStatus; @@ -31,9 +38,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.ShowLocksRequest; -import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; -import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.Database; @@ -45,15 +49,8 @@ import java.io.IOException; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; -import java.util.BitSet; import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -68,9 +65,6 @@ private long cleanerCheckInterval = 0; private ReplChangeManager replChangeManager; - // List of compactions to clean. - private Map> compactId2LockMap = new HashMap<>(); - private Map compactId2CompactInfoMap = new HashMap<>(); @Override public void init(AtomicBoolean stop, AtomicBoolean looped) throws MetaException { @@ -97,95 +91,9 @@ public void run() { try { handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.Cleaner.name()); startedAt = System.currentTimeMillis(); - // First look for all the compactions that are waiting to be cleaned. If we have not - // seen an entry before, look for all the locks held on that table or partition and - // record them. We will then only clean the partition once all of those locks have been - // released. This way we avoid removing the files while they are in use, - // while at the same time avoiding starving the cleaner as new readers come along. - // This works because we know that any reader who comes along after the worker thread has - // done the compaction will read the more up to date version of the data (either in a - // newer delta or in a newer base). - List toClean = txnHandler.findReadyToClean(); - { - /** - * Since there may be more than 1 instance of Cleaner running we may have state info - * for items which were cleaned by instances. Here we remove them. - * - * In the long run if we add end_time to compaction_queue, then we can check that - * hive_locks.acquired_at > compaction_queue.end_time + safety_buffer in which case - * we know the lock owner is reading files created by this compaction or later. - * The advantage is that we don't have to store the locks. - */ - Set currentToCleanSet = new HashSet<>(); - for (CompactionInfo ci : toClean) { - currentToCleanSet.add(ci.id); - } - Set cleanPerformedByOthers = new HashSet<>(); - for (long id : compactId2CompactInfoMap.keySet()) { - if (!currentToCleanSet.contains(id)) { - cleanPerformedByOthers.add(id); - } - } - for (long id : cleanPerformedByOthers) { - compactId2CompactInfoMap.remove(id); - compactId2LockMap.remove(id); - } - } - if (toClean.size() > 0 || compactId2LockMap.size() > 0) { - ShowLocksResponse locksResponse = txnHandler.showLocks(new ShowLocksRequest()); - if(LOG.isDebugEnabled()) { - dumpLockState(locksResponse); - } - for (CompactionInfo ci : toClean) { - // Check to see if we have seen this request before. If so, ignore it. If not, - // add it to our queue. - if (!compactId2LockMap.containsKey(ci.id)) { - compactId2LockMap.put(ci.id, findRelatedLocks(ci, locksResponse)); - compactId2CompactInfoMap.put(ci.id, ci); - } - } - - // Now, for each entry in the queue, see if all of the associated locks are clear so we - // can clean - Set currentLocks = buildCurrentLockSet(locksResponse); - List expiredLocks = new ArrayList(); - List compactionsCleaned = new ArrayList(); - try { - for (Map.Entry> queueEntry : compactId2LockMap.entrySet()) { - boolean sawLock = false; - for (Long lockId : queueEntry.getValue()) { - if (currentLocks.contains(lockId)) { - sawLock = true; - break; - } else { - expiredLocks.add(lockId); - } - } - - if (!sawLock) { - // Remember to remove this when we're out of the loop, - // we can't do it in the loop or we'll get a concurrent modification exception. - compactionsCleaned.add(queueEntry.getKey()); - //Future thought: this may be expensive so consider having a thread pool run in parallel - clean(compactId2CompactInfoMap.get(queueEntry.getKey())); - } else { - // Remove the locks we didn't see so we don't look for them again next time - for (Long lockId : expiredLocks) { - queueEntry.getValue().remove(lockId); - } - LOG.info("Skipping cleaning of " + - idWatermark(compactId2CompactInfoMap.get(queueEntry.getKey())) + - " due to reader present: " + queueEntry.getValue()); - } - } - } finally { - if (compactionsCleaned.size() > 0) { - for (Long compactId : compactionsCleaned) { - compactId2LockMap.remove(compactId); - compactId2CompactInfoMap.remove(compactId); - } - } - } + long minOpenTxnGLB = txnHandler.findMinOpenTxnGLB(); + for(CompactionInfo compactionInfo : txnHandler.findReadyToClean()) { + clean(compactionInfo, minOpenTxnGLB); } } catch (Throwable t) { LOG.error("Caught an exception in the main loop of compactor cleaner, " + @@ -213,41 +121,7 @@ public void run() { } while (!stop.get()); } - private Set findRelatedLocks(CompactionInfo ci, ShowLocksResponse locksResponse) { - Set relatedLocks = new HashSet(); - for (ShowLocksResponseElement lock : locksResponse.getLocks()) { - /** - * Hive QL is not case sensitive wrt db/table/column names - * Partition names get - * normalized (as far as I can tell) by lower casing column name but not partition value. - * {@link org.apache.hadoop.hive.metastore.Warehouse#makePartName(List, List, String)} - * {@link org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer#getPartSpec(ASTNode)} - * Since user input may start out in any case, compare here case-insensitive for db/table - * but leave partition name as is. - */ - if (ci.dbname.equalsIgnoreCase(lock.getDbname())) { - if ((ci.tableName == null && lock.getTablename() == null) || - (ci.tableName != null && ci.tableName.equalsIgnoreCase(lock.getTablename()))) { - if ((ci.partName == null && lock.getPartname() == null) || - (ci.partName != null && ci.partName.equals(lock.getPartname()))) { - relatedLocks.add(lock.getLockid()); - } - } - } - } - - return relatedLocks; - } - - private Set buildCurrentLockSet(ShowLocksResponse locksResponse) { - Set currentLocks = new HashSet(locksResponse.getLocks().size()); - for (ShowLocksResponseElement lock : locksResponse.getLocks()) { - currentLocks.add(lock.getLockid()); - } - return currentLocks; - } - - private void clean(CompactionInfo ci) throws MetaException { + private void clean(CompactionInfo ci, long minOpenTxnGLB) throws MetaException { LOG.info("Starting cleaning for " + ci); try { Table t = resolveTable(ci); @@ -271,27 +145,48 @@ private void clean(CompactionInfo ci) throws MetaException { } StorageDescriptor sd = resolveStorageDescriptor(t, p); final String location = sd.getLocation(); - /** - * Each Compaction only compacts as far as the highest txn id such that all txns below it - * are resolved (i.e. not opened). This is what "highestWriteId" tracks. This is only tracked - * since Hive 1.3.0/2.0 - thus may be 0. See ValidCompactorWriteIdList and uses for more info. + * todo: save this Use Case somewhere + * there are no open txns, then txn 7 starts, compactor makes base_N_c8 + * so if we make HWM=0 below, 8 will be visible but 7 will be reading + * below 8. so we should get current NEXT_TXN_ID, then check minOpen, + * and if there is nothing open, use NEXT_TXN_ID-1=HWM. + * + * todo: it's perhaps unwise to go above COMPACTION_QUEUE.CQ_HIGHEST_WRITE_ID + * TestTxnCommands2#writeBetweenWorkerAndCleaner - the last delta (3) is aborted (but present) + * It's returned by getAcidState() in Directory.abortedList + * so the new delta is delta_1_2 not 1_3 and it's not deleted either. * - * We only want to clean up to the highestWriteId - otherwise we risk deleting deltas from - * under an active reader. + * TestTxnCommands2#testCleanerForTxnToWriteId + */ +// ValidTxnList validTxnList = TxnCommonUtils. + //Visibility is capped at largest committed ID below which there are no open txns +// createValidReadTxnList(txnHandler.getOpenTxns(), minOpenTxnGLB - 1); + ValidTxnList validTxnList = TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), minOpenTxnGLB); + //save it so that getAcidState() sees it + conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); + /** + * {@code validTxnList} is capped by minOpenTxnGLB so if + * {@link AcidUtils#getAcidState(Path, Configuration, ValidWriteIdList)} sees a base/delta + * produced by a compactor, that means every reader that could be active right now see it + * as well. That means if this base/delta shadows some earlier base/delta, the it will be + * used in favor of any files that it shadows. Thus the shadowed files are safe to delete. * - * Suppose we have deltas D2 D3 for table T, i.e. the last compaction created D3 so now there is a - * clean request for D2. - * Cleaner checks existing locks and finds none. - * Between that check and removeFiles() a query starts (it will be reading D3) and another compaction - * completes which creates D4. - * Now removeFiles() (more specifically AcidUtils.getAcidState()) will declare D3 to be obsolete - * unless ValidWriteIdList is "capped" at highestWriteId. + * todo: given the above, COMPACTION_QUEUE.CQ_HIGHEST_WRITE_ID is not needed any more + * Repurpose it to store compactor txn id? */ - final ValidWriteIdList validWriteIdList = (ci.highestWriteId > 0) - ? new ValidReaderWriteIdList(ci.getFullTableName(), new long[0], new BitSet(), - ci.highestWriteId) - : new ValidReaderWriteIdList(); + List tblNames = Collections.singletonList( + TableName.getDbTable(t.getDbName(), t.getTableName())); + GetValidWriteIdsRequest rqst = new GetValidWriteIdsRequest(tblNames); + rqst.setValidTxnList(validTxnList.writeToString()); + GetValidWriteIdsResponse rsp = txnHandler.getValidWriteIds(rqst); + //we could have no write IDs for a table if it was never written to but + // since we are in the Cleaner phase of compactions, there must have + // been some delta/base dirs + assert rsp != null && rsp.getTblValidWriteIdsSize() == 1; + //Creating 'reader' list since we are interested in the set of 'obsolete' files + ValidReaderWriteIdList validWriteIdList = + TxnCommonUtils.createValidReaderWriteIdList(rsp.getTblValidWriteIds().get(0)); if (runJobAsSelf(ci.runAs)) { removeFiles(location, validWriteIdList, ci); @@ -328,7 +223,16 @@ private void removeFiles(String location, ValidWriteIdList writeIdList, Compacti Path locPath = new Path(location); AcidUtils.Directory dir = AcidUtils.getAcidState(locPath, conf, writeIdList); List obsoleteDirs = dir.getObsolete(); - List filesToDelete = new ArrayList(obsoleteDirs.size()); + /** + * add anything dir that only has data from aborted transactions - no one should be trying to + * read anything in that dir (except getAcidState() that only reads the name of this dir itself) + * So this may run ahead of {@link CompactionInfo#highestWriteId} but it's ok. The key is + * to not delete metadata about aborted txns with + * write IDs > {@link CompactionInfo#highestWriteId}. + * See {@link TxnStore#markCleaned(CompactionInfo)} + */ + obsoleteDirs.addAll(dir.getAbortedDirectories()); + List filesToDelete = new ArrayList<>(obsoleteDirs.size()); StringBuilder extraDebugInfo = new StringBuilder("["); for (FileStatus stat : obsoleteDirs) { filesToDelete.add(stat.getPath()); @@ -338,9 +242,6 @@ private void removeFiles(String location, ValidWriteIdList writeIdList, Compacti } } extraDebugInfo.setCharAt(extraDebugInfo.length() - 1, ']'); - List compactIds = new ArrayList<>(compactId2CompactInfoMap.keySet()); - Collections.sort(compactIds); - extraDebugInfo.append("compactId2CompactInfoMap.keySet(").append(compactIds).append(")"); LOG.info(idWatermark(ci) + " About to remove " + filesToDelete.size() + " obsolete directories from " + location + ". " + extraDebugInfo.toString()); if (filesToDelete.size() < 1) { @@ -361,63 +262,4 @@ private void removeFiles(String location, ValidWriteIdList writeIdList, Compacti fs.delete(dead, true); } } - private static class LockComparator implements Comparator { - //sort ascending by resource, nulls first - @Override - public int compare(ShowLocksResponseElement o1, ShowLocksResponseElement o2) { - if(o1 == o2) { - return 0; - } - if(o1 == null) { - return -1; - } - if(o2 == null) { - return 1; - } - int v = o1.getDbname().compareToIgnoreCase(o2.getDbname()); - if(v != 0) { - return v; - } - if(o1.getTablename() == null) { - return -1; - } - if(o2.getTablename() == null) { - return 1; - } - v = o1.getTablename().compareToIgnoreCase(o2.getTablename()); - if(v != 0) { - return v; - } - if(o1.getPartname() == null) { - return -1; - } - if(o2.getPartname() == null) { - return 1; - } - v = o1.getPartname().compareToIgnoreCase(o2.getPartname()); - if(v != 0) { - return v; - } - //if still equal, compare by lock ids - v = Long.compare(o1.getLockid(), o2.getLockid()); - if(v != 0) { - return v; - } - return Long.compare(o1.getLockIdInternal(), o2.getLockIdInternal()); - - } - } - private void dumpLockState(ShowLocksResponse slr) { - Iterator l = slr.getLocksIterator(); - List sortedList = new ArrayList<>(); - while(l.hasNext()) { - sortedList.add(l.next()); - } - //sort for readability - sortedList.sort(new LockComparator()); - LOG.info("dumping locks"); - for(ShowLocksResponseElement lock : sortedList) { - LOG.info(lock.toString()); - } - } } diff --git ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java index 92c74e1d06..99981d559d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java +++ ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java @@ -42,14 +42,14 @@ import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.common.StringableMap; import org.apache.hadoop.hive.common.ValidCompactorWriteIdList; -import org.apache.hadoop.hive.common.ValidReaderWriteIdList; +import org.apache.hadoop.hive.common.ValidReadTxnList; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Order; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.SerDeInfo; @@ -59,10 +59,7 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; 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.Driver; import org.apache.hadoop.hive.ql.DriverUtils; -import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.DDLTask; import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter; import org.apache.hadoop.hive.ql.io.AcidInputFormat; @@ -74,8 +71,6 @@ import org.apache.hadoop.hive.ql.io.RecordIdentifier; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.shims.HadoopShims.HdfsFileStatusWithId; @@ -100,6 +95,7 @@ import org.apache.hadoop.util.StringUtils; import org.apache.hive.common.util.HiveStringUtils; import org.apache.hive.common.util.Ref; +import org.apache.parquet.Strings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -351,6 +347,8 @@ private void runMmCompaction(HiveConf conf, Table t, Partition p, // Set up the session for driver. conf = new HiveConf(conf); conf.set(ConfVars.HIVE_QUOTEDID_SUPPORT.varname, "column"); + //todo: move into DriverUtils.runOnDriver()? have to clone conf in there then + conf.unset(ValidTxnList.VALID_TXNS_KEY); String user = UserGroupInformation.getCurrentUser().getShortUserName(); SessionState sessionState = DriverUtils.setUpSessionState(conf, user, false); @@ -599,7 +597,6 @@ private void launchCompactionJob(JobConf job, Path baseDir, CompactionType compa * to use. * @param job the job to update * @param cols the columns of the table - * @param map */ private void setColumnTypes(JobConf job, List cols) { StringBuilder colNames = new StringBuilder(); @@ -1006,7 +1003,16 @@ public void close() throws IOException { deleteEventWriter.close(false); } } - + private long getCompactorTxnId() { + String snapshot = jobConf.get(ValidTxnList.VALID_TXNS_KEY); + if(Strings.isNullOrEmpty(snapshot)) { + throw new IllegalStateException(ValidTxnList.VALID_TXNS_KEY + " not found for writing to " + + jobConf.get(FINAL_LOCATION)); + } + ValidTxnList validTxnList = new ValidReadTxnList(); + validTxnList.readFromString(snapshot); + return validTxnList.getHighWatermark(); + } private void getWriter(Reporter reporter, ObjectInspector inspector, int bucket) throws IOException { if (writer == null) { @@ -1019,7 +1025,8 @@ private void getWriter(Reporter reporter, ObjectInspector inspector, .minimumWriteId(jobConf.getLong(MIN_TXN, Long.MAX_VALUE)) .maximumWriteId(jobConf.getLong(MAX_TXN, Long.MIN_VALUE)) .bucket(bucket) - .statementId(-1);//setting statementId == -1 makes compacted delta files use + .statementId(-1)//setting statementId == -1 makes compacted delta files use + .visibilityTxnId(getCompactorTxnId()); //delta_xxxx_yyyy format // Instantiate the underlying output format @@ -1044,8 +1051,9 @@ private void getDeleteEventWriter(Reporter reporter, ObjectInspector inspector, .minimumWriteId(jobConf.getLong(MIN_TXN, Long.MAX_VALUE)) .maximumWriteId(jobConf.getLong(MAX_TXN, Long.MIN_VALUE)) .bucket(bucket) - .statementId(-1);//setting statementId == -1 makes compacted delta files use - //delta_xxxx_yyyy format + .statementId(-1)//setting statementId == -1 makes compacted delta files use + // delta_xxxx_yyyy format + .visibilityTxnId(getCompactorTxnId()); // Instantiate the underlying output format @SuppressWarnings("unchecked")//since there is no way to parametrize instance of Class @@ -1179,6 +1187,7 @@ public void commitJob(JobContext context) throws IOException { //we have MIN_TXN, MAX_TXN and IS_MAJOR in JobConf so we could figure out exactly what the dir //name is that we want to rename; leave it for another day // TODO: if we expect one dir why don't we enforce it? + //may actually have delta_x_y and delete_delta_x_y for (FileStatus fileStatus : contents) { //newPath is the base/delta dir Path newPath = new Path(finalLocation, fileStatus.getPath().getName()); @@ -1218,7 +1227,7 @@ private void commitMmCompaction(String from, String to, Configuration conf, ValidWriteIdList actualWriteIds) throws IOException { Path fromPath = new Path(from), toPath = new Path(to); FileSystem fs = fromPath.getFileSystem(conf); - // Assume the high watermark can be used as maximum transaction ID. + // Assume the high watermark can be used as maximum transaction ID. todo: is that true? can it be aborted? does it matter for compaction? long maxTxn = actualWriteIds.getHighWatermark(); AcidOutputFormat.Options options = new AcidOutputFormat.Options(conf) .writingBase(true).isCompressed(false).maximumWriteId(maxTxn).bucket(0).statementId(-1); diff --git ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java index dd0929f2b9..f5b901d6e8 100644 --- ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java +++ ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java @@ -51,7 +51,7 @@ /** * Superclass for all threads in the compactor. */ -abstract class CompactorThread extends Thread implements MetaStoreThread { +public abstract class CompactorThread extends Thread implements MetaStoreThread { static final private String CLASS_NAME = CompactorThread.class.getName(); static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME); @@ -67,8 +67,9 @@ public void setConf(Configuration configuration) { // TODO MS-SPLIT for now, keep a copy of HiveConf around as we need to call other methods with // it. This should be changed to Configuration once everything that this calls that requires // HiveConf is moved to the standalone metastore. - conf = (configuration instanceof HiveConf) ? (HiveConf)configuration : - new HiveConf(configuration, HiveConf.class); + //clone the conf - compactor needs to set properties in it which we don't + // want to bleed into the caller + conf = new HiveConf(configuration, HiveConf.class); } @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java index beb6902d7d..6886bd715d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java +++ ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.CompactionRequest; @@ -36,6 +37,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.ql.io.AcidUtils; @@ -143,13 +145,15 @@ public void run() { ", assuming it has been dropped and moving on."); continue; } - - // Compaction doesn't work under a transaction and hence pass null for validTxnList + //todo: this code is very similar to what is in Worker - refactor to catch comments etc + ValidTxnList validTxnList = TxnCommonUtils.createValidReadTxnList(txnHandler.getOpenTxns(), 0); + conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); // The response will have one entry per table and hence we get only one ValidWriteIdList String fullTableName = TxnUtils.getFullTableName(t.getDbName(), t.getTableName()); GetValidWriteIdsRequest rqst = new GetValidWriteIdsRequest(Collections.singletonList(fullTableName)); - ValidWriteIdList tblValidWriteIds = TxnUtils.createValidCompactWriteIdList( + rqst.setValidTxnList(validTxnList.writeToString()); + final ValidWriteIdList tblValidWriteIds = TxnUtils.createValidCompactWriteIdList( txnHandler.getValidWriteIds(rqst).getTblValidWriteIds().get(0)); StorageDescriptor sd = resolveStorageDescriptor(t, p); diff --git ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java index d9f186cd03..48f7d0df9c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java +++ ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java @@ -18,15 +18,22 @@ package org.apache.hadoop.hive.ql.txn.compactor; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hive.common.ValidCompactorWriteIdList; +import org.apache.hadoop.hive.common.ValidTxnList; +import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; +import org.apache.hadoop.hive.metastore.api.CommitTxnRequest; import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest; +import org.apache.hadoop.hive.metastore.api.HeartbeatRequest; import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.OpenTxnRequest; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; +import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager; import org.apache.hadoop.mapred.JobConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.txn.CompactionInfo; @@ -58,7 +65,7 @@ static final private long SLEEP_TIME = 5000; static final private int baseThreadNum = 10002; - private String name; + private String workerName; private JobConf mrJob; // the MR job for compaction /** @@ -83,7 +90,7 @@ public void run() { // Make sure nothing escapes this run method and kills the metastore at large, // so wrap it in a big catch Throwable statement. try { - final CompactionInfo ci = txnHandler.findNextToCompact(name); + final CompactionInfo ci = txnHandler.findNextToCompact(workerName); LOG.debug("Processing compaction request " + ci); if (ci == null && !stop.get()) { @@ -144,14 +151,6 @@ public void run() { // Compaction doesn't work under a transaction and hence pass 0 for current txn Id // The response will have one entry per table and hence we get only one OpenWriteIds String fullTableName = TxnUtils.getFullTableName(t.getDbName(), t.getTableName()); - GetValidWriteIdsRequest rqst = new GetValidWriteIdsRequest(Collections.singletonList(fullTableName)); - final ValidWriteIdList tblValidWriteIds = - TxnUtils.createValidCompactWriteIdList(txnHandler.getValidWriteIds(rqst).getTblValidWriteIds().get(0)); - LOG.debug("ValidCompactWriteIdList: " + tblValidWriteIds.writeToString()); - txnHandler.setCompactionHighestWriteId(ci, tblValidWriteIds.getHighWatermark()); - final StringBuilder jobName = new StringBuilder(name); - jobName.append("-compactor-"); - jobName.append(ci.getFullPartitionName()); // Determine who to run as String runAs; @@ -162,9 +161,59 @@ public void run() { runAs = ci.runAs; } - LOG.info("Starting " + ci.type.toString() + " compaction for " + - ci.getFullPartitionName()); + /** + * We need a transaction. could call txnHandler directly but then we'd have to set up a hearbeat + * but using {@link HiveTxnManager} creates a Thrift connection to the HMS + * will this cause security checks that could fail? + * on the other hand we run SQL via Driver which certainly uses {@link HiveTxnManager} + final HiveTxnManager txnMgr = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf); + * openTxn requires Context() which is set up based on query parse/plan.... + long txnid = txnMgr.openTxn(null, null); + */ + OpenTxnRequest otReq = new OpenTxnRequest(1, runAs, hostname()); + otReq.setAgentInfo(getName());//ThreadName + long compactorTxnId = txnHandler.openTxns(otReq).getTxn_ids().get(0); + //todo: now we can update compaction_queue entry with this id + //also make sure to write to TXN_COMPONENTS so that if txn aborts, we don't delete the metadata about it from TXNS!!!! + + HeartbeatRequest heartbeatRequest = new HeartbeatRequest(); + heartbeatRequest.setTxnid(compactorTxnId); + heartbeatRequest.setLockid(0); + /** + * todo: now set up a thread to do the heartbeat + */ + txnHandler.heartbeat(heartbeatRequest); + + //todo: should minOpenTxnId from this validTxnList be stored in MIN_HISTORY table? + ValidTxnList validTxnList = TxnCommonUtils.createValidReadTxnList(txnHandler.getOpenTxns(), compactorTxnId); + GetValidWriteIdsRequest rqst = new GetValidWriteIdsRequest(Collections.singletonList(fullTableName)); + //with this ValidWriteIdList is capped at whatever HWM validTxnList has + rqst.setValidTxnList(validTxnList.writeToString()); + final ValidCompactorWriteIdList tblValidWriteIds = + TxnUtils.createValidCompactWriteIdList(txnHandler.getValidWriteIds(rqst).getTblValidWriteIds().get(0)); + LOG.debug("ValidCompactWriteIdList: " + tblValidWriteIds.writeToString()); + /** + * we could set it in 'conf' but then the queries that compactor runs on the Driver see it and think + * that a txn is already opened (normally it's open txn then lock the snapshot) + * Changing this logic in the driver won't help since some statements compactor runs are create + * table which cannot be used in a multi stmt txn anyway. Also, locking a snapshot before opening txn + * doesn't make much sense - the snapshot should be capped at the current txn. + * So pass it in ValidWriteIdList and use it from compactor code which from here is pass by ref + * Alternatively, set it on conf here, but clone conf and remove it before calling Driver... + * Both are ugly. + * + * it's better to set it in conf and clear before calling Driver - this way every call to getAcidState() will see it + * whether set by Driver or here + */ + conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); + + //todo: this is a RDBMS call - so is setRunAs() above - could combine into 1 + txnHandler.setCompactionHighestWriteId(ci, tblValidWriteIds.getHighWatermark()); + final StringBuilder jobName = new StringBuilder(workerName); + jobName.append("-compactor-"); + jobName.append(ci.getFullPartitionName()); + LOG.info("Starting " + ci.type.toString() + " compaction for " + ci.getFullPartitionName()); final StatsUpdater su = StatsUpdater.init(ci, txnHandler.findColumnsWithStats(ci), conf, runJobAsSelf(runAs) ? runAs : t.getOwner()); final CompactorMR mr = new CompactorMR(); @@ -191,6 +240,7 @@ public Object run() throws Exception { } } txnHandler.markCompacted(ci); + txnHandler.commitTxn(new CommitTxnRequest(compactorTxnId)); if (conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST)) { mrJob = mr.getMrJob(); } @@ -198,9 +248,10 @@ public Object run() throws Exception { LOG.error("Caught exception while trying to compact " + ci + ". Marking failed to avoid repeated failures, " + StringUtils.stringifyException(e)); txnHandler.markFailed(ci); + txnHandler.abortTxn(new AbortTxnRequest(compactorTxnId)); } } catch (Throwable t) { - LOG.error("Caught an exception in the main loop of compactor worker " + name + ", " + + LOG.error("Caught an exception in the main loop of compactor worker " + workerName + ", " + StringUtils.stringifyException(t)); } @@ -223,7 +274,7 @@ public void init(AtomicBoolean stop, AtomicBoolean looped) throws MetaException StringBuilder name = new StringBuilder(hostname()); name.append("-"); name.append(getId()); - this.name = name.toString(); + this.workerName = name.toString(); setName(name.toString()); } @@ -250,7 +301,9 @@ public static StatsUpdater init(CompactionInfo ci, List columnListForSta private StatsUpdater(CompactionInfo ci, List columnListForStats, HiveConf conf, String userName) { - this.conf = conf; + this.conf = new HiveConf(conf); + //so that Driver doesn't think it's arleady in a transaction + this.conf.unset(ValidTxnList.VALID_TXNS_KEY); this.userName = userName; this.ci = ci; if (!ci.isMajorCompaction() || columnListForStats == null || columnListForStats.isEmpty()) { @@ -298,6 +351,8 @@ void gatherStats() { sb.setLength(sb.length() - 1); //remove trailing , LOG.info(ci + ": running '" + sb.toString() + "'"); conf.setVar(HiveConf.ConfVars.METASTOREURIS,""); + + //todo: use DriverUtils.runOnDriver() here Driver d = new Driver(new QueryState.Builder().withGenerateNewQueryId(true).withHiveConf(conf).build(), userName); SessionState localSession = null; try { diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java index 0cda871262..c2b931a26b 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java @@ -1129,54 +1129,34 @@ public void testNonAcidToAcidConversion01() throws Exception { //create a delta directory runStatementOnDriver("insert into " + Table.NONACIDORCTBL + "(a,b) values(1,17)"); - //make sure we assign correct Ids - List rs = runStatementOnDriver("select ROW__ID, a, b, INPUT__FILE__NAME from " + Table.NONACIDORCTBL + " order by ROW__ID"); - LOG.warn("before compact"); - for(String s : rs) { - LOG.warn(s); - } + boolean isVectorized = hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED); + String query = "select ROW__ID, a, b" + (isVectorized ? " from " : ", INPUT__FILE__NAME from ") + Table.NONACIDORCTBL + " order by ROW__ID"; + String[][] expected = new String[][] { + {"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":0}\t1\t2", "nonacidorctbl/000001_0"}, + {"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":1}\t1\t5", "nonacidorctbl/000001_0_copy_1"}, + {"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":2}\t0\t12", "nonacidorctbl/000001_0_copy_1"}, + {"{\"writeid\":10000001,\"bucketid\":536936448,\"rowid\":0}\t1\t17", "nonacidorctbl/delta_10000001_10000001_0000/bucket_00001"} + }; + checkResult(expected, query, isVectorized, "before compact", LOG); + Assert.assertEquals(536870912, - BucketCodec.V1.encode(new AcidOutputFormat.Options(hiveConf).bucket(0))); + BucketCodec.V1.encode(new AcidOutputFormat.Options(hiveConf).bucket(0))); Assert.assertEquals(536936448, - BucketCodec.V1.encode(new AcidOutputFormat.Options(hiveConf).bucket(1))); - Assert.assertEquals("", 4, rs.size()); - Assert.assertTrue(rs.get(0), - rs.get(0).startsWith("{\"writeid\":0,\"bucketid\":536936448,\"rowid\":0}\t1\t2")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("nonacidorctbl/000001_0")); - Assert.assertTrue(rs.get(1), - rs.get(1).startsWith("{\"writeid\":0,\"bucketid\":536936448,\"rowid\":1}\t1\t5")); - Assert.assertTrue(rs.get(1), rs.get(1).endsWith("nonacidorctbl/000001_0_copy_1")); - Assert.assertTrue(rs.get(2), - rs.get(2).startsWith("{\"writeid\":0,\"bucketid\":536936448,\"rowid\":2}\t0\t12")); - Assert.assertTrue(rs.get(2), rs.get(2).endsWith("nonacidorctbl/000001_0_copy_1")); - Assert.assertTrue(rs.get(3), - rs.get(3).startsWith("{\"writeid\":10000001,\"bucketid\":536936448,\"rowid\":0}\t1\t17")); - Assert.assertTrue(rs.get(3), rs.get(3) - .endsWith("nonacidorctbl/delta_10000001_10000001_0000/bucket_00001")); + BucketCodec.V1.encode(new AcidOutputFormat.Options(hiveConf).bucket(1))); + //run Compaction runStatementOnDriver("alter table "+ TestTxnCommands2.Table.NONACIDORCTBL +" compact 'major'"); TestTxnCommands2.runWorker(hiveConf); - rs = runStatementOnDriver("select ROW__ID, a, b, INPUT__FILE__NAME from " + - Table.NONACIDORCTBL + " order by ROW__ID"); - LOG.warn("after compact"); - for(String s : rs) { - LOG.warn(s); - } - Assert.assertEquals("", 4, rs.size()); - Assert.assertTrue(rs.get(0), - rs.get(0).startsWith("{\"writeid\":0,\"bucketid\":536936448,\"rowid\":0}\t1\t2")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("nonacidorctbl/base_10000001/bucket_00001")); - Assert.assertTrue(rs.get(1), - rs.get(1).startsWith("{\"writeid\":0,\"bucketid\":536936448,\"rowid\":1}\t1\t5")); - Assert.assertTrue(rs.get(1), rs.get(1).endsWith("nonacidorctbl/base_10000001/bucket_00001")); - Assert.assertTrue(rs.get(2), - rs.get(2).startsWith("{\"writeid\":0,\"bucketid\":536936448,\"rowid\":2}\t0\t12")); - Assert.assertTrue(rs.get(2), rs.get(2).endsWith("nonacidorctbl/base_10000001/bucket_00001")); - Assert.assertTrue(rs.get(3), - rs.get(3) - .startsWith("{\"writeid\":10000001,\"bucketid\":536936448,\"rowid\":0}\t1\t17")); - Assert.assertTrue(rs.get(3), rs.get(3).endsWith("nonacidorctbl/base_10000001/bucket_00001")); + query = "select ROW__ID, a, b" + (isVectorized ? "" : ", INPUT__FILE__NAME") + " from " + + Table.NONACIDORCTBL + " order by ROW__ID"; + String[][] expected2 = new String[][] { + {"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":0}\t1\t2", "nonacidorctbl/base_10000001_v0000020/bucket_00001"}, + {"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":1}\t1\t5", "nonacidorctbl/base_10000001_v0000020/bucket_00001"}, + {"{\"writeid\":0,\"bucketid\":536936448,\"rowid\":2}\t0\t12", "nonacidorctbl/base_10000001_v0000020/bucket_00001"}, + {"{\"writeid\":10000001,\"bucketid\":536936448,\"rowid\":0}\t1\t17", "nonacidorctbl/base_10000001_v0000020/bucket_00001"} + }; + checkResult(expected2, query, isVectorized, "after major compact", LOG); //make sure they are the same before and after compaction } //@Ignore("see bucket_num_reducers_acid.q") diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java index 057fd7704c..b9ce60bc46 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java @@ -24,8 +24,10 @@ import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -59,8 +61,10 @@ import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.metastore.txn.AcidOpenTxnsCounterService; import org.apache.hadoop.hive.ql.txn.compactor.Cleaner; +import org.apache.hadoop.hive.ql.txn.compactor.CompactorThread; import org.apache.hadoop.hive.ql.txn.compactor.Initiator; import org.apache.hadoop.hive.ql.txn.compactor.Worker; +import org.apache.hadoop.mapred.JobConf; import org.apache.orc.OrcFile; import org.apache.orc.Reader; import org.apache.orc.TypeDescription; @@ -246,14 +250,7 @@ private void testOrcPPD(boolean enablePPD) throws Exception { List rs0 = runStatementOnDriver(query); Assert.assertEquals("Read failed", 0, rs0.size()); runStatementOnDriver("alter table " + Table.ACIDTBL + " compact 'MAJOR'"); - Worker t = new Worker(); - t.setThreadId((int) t.getId()); - t.setConf(hiveConf); - AtomicBoolean stop = new AtomicBoolean(); - AtomicBoolean looped = new AtomicBoolean(); - stop.set(true); - t.init(stop, looped); - t.run(); + runWorker(hiveConf); //now we have base_0001 file int[][] tableData2 = {{1, 7}, {5, 6}, {7, 8}, {9, 10}}; runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData2)); @@ -293,14 +290,7 @@ public void testAlterTable() throws Exception { int[][] tableData = {{1,2}}; runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData)); runStatementOnDriver("alter table "+ Table.ACIDTBL + " compact 'MAJOR'"); - Worker t = new Worker(); - t.setThreadId((int) t.getId()); - t.setConf(hiveConf); - AtomicBoolean stop = new AtomicBoolean(); - AtomicBoolean looped = new AtomicBoolean(); - stop.set(true); - t.init(stop, looped); - t.run(); + runWorker(hiveConf); int[][] tableData2 = {{5,6}}; runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData2)); List rs1 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " where b > 0 order by a,b"); @@ -746,7 +736,9 @@ public void testNonAcidToAcidConversion3() throws Exception { boolean sawNewBase = false; for (int i = 0; i < status.length; i++) { if (status[i].getPath().getName().matches("base_.*")) { - Assert.assertEquals("base_-9223372036854775808", status[i].getPath().getName()); + //should be base_-9223372036854775808_v0000022 but 22 is a txn id not write id so it makes + //the tests fragile + Assert.assertTrue(status[i].getPath().getName().startsWith("base_-9223372036854775808_v0000022")); sawNewBase = true; FileStatus[] buckets = fs.listStatus(status[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER); Assert.assertEquals(BUCKET_COUNT - 1, buckets.length); @@ -798,7 +790,7 @@ public void testNonAcidToAcidConversion3() throws Exception { Assert.assertEquals("bucket_00001", buckets[0].getPath().getName()); } } else if (status[i].getPath().getName().matches("base_.*")) { - Assert.assertEquals("base_-9223372036854775808", status[i].getPath().getName()); + Assert.assertTrue("base_-9223372036854775808", status[i].getPath().getName().startsWith("base_-9223372036854775808_v0000022"));//_v0000022 sawNewBase = true; FileStatus[] buckets = fs.listStatus(status[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER); Assert.assertEquals(BUCKET_COUNT - 1, buckets.length); @@ -835,12 +827,12 @@ public void testNonAcidToAcidConversion3() throws Exception { FileStatus[] buckets = fs.listStatus(status[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER); Arrays.sort(buckets); if (numBase == 1) { - Assert.assertEquals("base_-9223372036854775808", status[i].getPath().getName()); + Assert.assertEquals("base_-9223372036854775808_v0000022", status[i].getPath().getName()); Assert.assertEquals(BUCKET_COUNT - 1, buckets.length); Assert.assertEquals("bucket_00001", buckets[0].getPath().getName()); } else if (numBase == 2) { // The new base dir now has two bucket files, since the delta dir has two bucket files - Assert.assertEquals("base_10000002", status[i].getPath().getName()); + Assert.assertEquals("base_10000002_v0000030", status[i].getPath().getName()); Assert.assertEquals(2, buckets.length); Assert.assertEquals("bucket_00000", buckets[0].getPath().getName()); } @@ -866,7 +858,7 @@ public void testNonAcidToAcidConversion3() throws Exception { status = fs.listStatus(new Path(TEST_WAREHOUSE_DIR + "/" + (Table.NONACIDORCTBL).toString().toLowerCase()), FileUtils.HIDDEN_FILES_PATH_FILTER); Assert.assertEquals(1, status.length); - Assert.assertEquals("base_10000002", status[0].getPath().getName()); + Assert.assertEquals("base_10000002_v0000030", status[0].getPath().getName()); FileStatus[] buckets = fs.listStatus(status[0].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER); Arrays.sort(buckets); Assert.assertEquals(2, buckets.length); @@ -1198,17 +1190,32 @@ else if(TxnStore.ATTEMPTED_RESPONSE.equals(compact.getState())) { return compactionsByState; } public static void runWorker(HiveConf hiveConf) throws MetaException { - AtomicBoolean stop = new AtomicBoolean(true); - Worker t = new Worker(); - t.setThreadId((int) t.getId()); - t.setConf(hiveConf); - AtomicBoolean looped = new AtomicBoolean(); - t.init(stop, looped); - t.run(); + runCompactorThread(hiveConf, CompactorThreadType.WORKER); } public static void runCleaner(HiveConf hiveConf) throws MetaException { + runCompactorThread(hiveConf, CompactorThreadType.CLEANER); + } + public static void runInitiator(HiveConf hiveConf) throws MetaException { + runCompactorThread(hiveConf, CompactorThreadType.INITIATOR); + } + private enum CompactorThreadType {INITIATOR, WORKER, CLEANER} + private static void runCompactorThread(HiveConf hiveConf, CompactorThreadType type) + throws MetaException { AtomicBoolean stop = new AtomicBoolean(true); - Cleaner t = new Cleaner(); + CompactorThread t = null; + switch (type) { + case INITIATOR: + t = new Initiator(); + break; + case WORKER: + t = new Worker(); + break; + case CLEANER: + t = new Cleaner(); + break; + default: + throw new IllegalArgumentException("Unknown type: " + type); + } t.setThreadId((int) t.getId()); t.setConf(hiveConf); AtomicBoolean looped = new AtomicBoolean(); @@ -1225,7 +1232,7 @@ public void writeBetweenWorkerAndCleaner() throws Exception { writeBetweenWorkerAndCleanerForVariousTblProperties("'transactional'='true'"); } - protected void writeBetweenWorkerAndCleanerForVariousTblProperties(String tblProperties) throws Exception { + private void writeBetweenWorkerAndCleanerForVariousTblProperties(String tblProperties) throws Exception { String tblName = "hive12352"; runStatementOnDriver("drop table if exists " + tblName); runStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " + @@ -1239,13 +1246,7 @@ protected void writeBetweenWorkerAndCleanerForVariousTblProperties(String tblPro //run Worker to execute compaction TxnStore txnHandler = TxnUtils.getTxnStore(hiveConf); txnHandler.compact(new CompactionRequest("default", tblName, CompactionType.MINOR)); - Worker t = new Worker(); - t.setThreadId((int) t.getId()); - t.setConf(hiveConf); - AtomicBoolean stop = new AtomicBoolean(true); - AtomicBoolean looped = new AtomicBoolean(); - t.init(stop, looped); - t.run(); + runWorker(hiveConf); //delete something, but make sure txn is rolled back hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); @@ -1259,21 +1260,23 @@ protected void writeBetweenWorkerAndCleanerForVariousTblProperties(String tblPro Assert.assertEquals("", expected, runStatementOnDriver("select a,b from " + tblName + " order by a")); - //run Cleaner - Cleaner c = new Cleaner(); - c.setThreadId((int)c.getId()); - c.setConf(hiveConf); - c.init(stop, new AtomicBoolean()); - c.run(); - - //this seems odd, but we wan to make sure that to run CompactionTxnHandler.cleanEmptyAbortedTxns() - Initiator i = new Initiator(); - i.setThreadId((int)i.getId()); - i.setConf(hiveConf); - i.init(stop, new AtomicBoolean()); - i.run(); - - //check that aborted operation didn't become committed + runCleaner(hiveConf); + + FileSystem fs = FileSystem.get(hiveConf); + FileStatus[] status = fs.listStatus(new Path(TEST_WAREHOUSE_DIR + "/" + tblName.toLowerCase()), + FileUtils.HIDDEN_FILES_PATH_FILTER); + Set expectedDeltas = new HashSet<>(); + expectedDeltas.add("delete_delta_0000001_0000002_v0000019"); + expectedDeltas.add("delta_0000001_0000002_v0000019"); + Set actualDeltas = new HashSet<>(); + for(FileStatus file : status) { + actualDeltas.add(file.getPath().getName()); + } + Assert.assertEquals(expectedDeltas, actualDeltas); + + //this seems odd, but we want to make sure that run CompactionTxnHandler.cleanEmptyAbortedTxns() + runInitiator(hiveConf); + //and check that aborted operation didn't become committed Assert.assertEquals("", expected, runStatementOnDriver("select a,b from " + tblName + " order by a")); } @@ -1338,18 +1341,11 @@ public void testCompactWithDelete() throws Exception { int[][] tableData = {{1,2},{3,4}}; runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData)); runStatementOnDriver("alter table "+ Table.ACIDTBL + " compact 'MAJOR'"); - Worker t = new Worker(); - t.setThreadId((int) t.getId()); - t.setConf(hiveConf); - AtomicBoolean stop = new AtomicBoolean(); - AtomicBoolean looped = new AtomicBoolean(); - stop.set(true); - t.init(stop, looped); - t.run(); + runWorker(hiveConf); runStatementOnDriver("delete from " + Table.ACIDTBL + " where b = 4"); runStatementOnDriver("update " + Table.ACIDTBL + " set b = -2 where b = 2"); runStatementOnDriver("alter table "+ Table.ACIDTBL + " compact 'MINOR'"); - t.run(); + runWorker(hiveConf); TxnStore txnHandler = TxnUtils.getTxnStore(hiveConf); ShowCompactResponse resp = txnHandler.showCompact(new ShowCompactRequest()); Assert.assertEquals("Unexpected number of compactions in history", 2, resp.getCompactsSize()); diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java index 833e63745f..b816a65ddd 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java @@ -1,22 +1,26 @@ /* - * 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. - */ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.hadoop.hive.ql; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.FileUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; @@ -25,13 +29,19 @@ import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.ql.io.orc.TestVectorizedOrcAcidRowBatchReader; +import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager; +import org.apache.hadoop.hive.ql.lockmgr.TxnManagerFactory; import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; +import java.util.HashSet; import java.util.List; +import java.util.Set; + +import static org.apache.hadoop.hive.ql.lockmgr.TestDbTxnManager2.swapTxnManager; public class TestTxnCommands3 extends TxnCommandsBaseForTests { static final private Logger LOG = LoggerFactory.getLogger(TestTxnCommands3.class); @@ -283,4 +293,168 @@ private void testSdpoBucketed(boolean isVectorized, boolean isSdpo, int bucketin "warehouse/acid_uap/ds=tomorrow/delta_0000003_0000003_0000/bucket_00000"}}; checkResult(expected2, testQuery, isVectorized, "after update", LOG); } + @Test + public void testCleaner2() throws Exception { + MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.CREATE_TABLES_AS_ACID, true); + dropTable(new String[] {"T"}); + //note: transaction names T1, T2, etc below, are logical, the actualy txnid will be different + runStatementOnDriver("create table T (a int, b int) stored as orc"); + runStatementOnDriver("insert into T values(0,2)");//makes delta_1_1 in T1 + runStatementOnDriver("insert into T values(1,4)");//makes delta_2_2 in T2 + + Driver driver2 = new Driver(new QueryState.Builder().withHiveConf(hiveConf).build(), null); + driver2.setMaxRows(10000); + + HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(hiveConf); + HiveTxnManager txnMgr1 = swapTxnManager(txnMgr2); + Driver driver1 = swapDrivers(driver2); + runStatementOnDriver("start transaction");//T3 + /* this select sees + target/warehouse/t/ + ├── delta_0000001_0000001_0000 + │   ├── _orc_acid_version + │   └── bucket_00000 + └── delta_0000002_0000002_0000 + ├── _orc_acid_version + └── bucket_00000*/ + String testQuery = "select ROW__ID, a, b, INPUT__FILE__NAME from T"; + String[][] expected = new String[][] { + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t0\t2", + "t/delta_0000001_0000001_0000/bucket_00000"}, + {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t4", + "t/delta_0000002_0000002_0000/bucket_00000"}}; + checkResult(expected, testQuery, false, "check data", LOG); + + + txnMgr2 = swapTxnManager(txnMgr1); + driver2 = swapDrivers(driver1); + runStatementOnDriver("alter table T compact 'minor'");//T4 + TestTxnCommands2.runWorker(hiveConf);//makes delta_1_2 & delete_delta_1_2 + /* Now we should have + target/warehouse/t/ + ├── delete_delta_0000001_0000002 + │   ├── _orc_acid_version + │   └── bucket_00000 + ├── delta_0000001_0000001_0000 + │   ├── _orc_acid_version + │   └── bucket_00000 + ├── delta_0000001_0000002 + │   ├── _orc_acid_version + │   └── bucket_00000 + └── delta_0000002_0000002_0000 + ├── _orc_acid_version + └── bucket_00000*/ + FileSystem fs = FileSystem.get(hiveConf); + Path warehousePath = new Path(getWarehouseDir()); + FileStatus[] actualList = fs.listStatus(new Path(warehousePath + "/t"), + FileUtils.HIDDEN_FILES_PATH_FILTER); + + String[] expectedList = new String[] { + "/t/delete_delta_0000001_0000002_v0000019",//todo: hardcoded txnids will make tests unstable + "/t/delta_0000001_0000002_v0000019", + "/t/delta_0000001_0000001_0000", + "/t/delta_0000002_0000002_0000", + }; + checkExpectedFiles(actualList, expectedList, warehousePath.toString()); + /* Now the state should be + target/warehouse/t/ + ├── delete_delta_0000001_0000002 + │   ├── _orc_acid_version + │   └── bucket_00000 + ├── delta_0000001_0000002 + │   ├── _orc_acid_version + │   └── bucket_00000 + └── delta_0000002_0000002_0000 + ├── _orc_acid_version + └── bucket_00000 + The cleaner should not delete delta_2_2 since T3 may still be + reading this file (assuming this select statement is still running) + Also, T3, can't see files produced by compactor in T19 so cleaner should not + delete anything*/ + + TestTxnCommands2.runCleaner(hiveConf); + expectedList = new String[] { + "/t/delete_delta_0000001_0000002_v0000019", + "/t/delta_0000001_0000002_v0000019", + "/t/delta_0000001_0000001_0000", + "/t/delta_0000002_0000002_0000", + }; + actualList = fs.listStatus(new Path(warehousePath + "/t"), + FileUtils.HIDDEN_FILES_PATH_FILTER); + checkExpectedFiles(actualList, expectedList, warehousePath.toString()); + + txnMgr1 = swapTxnManager(txnMgr2); + driver1 = swapDrivers(driver2); + runStatementOnDriver("commit");//commits T3 + //so now cleaner should be able to delete delta_0000001_0000001_0000 & delta_0000002_0000002_0000 + + //so that compactor makes a new delta (due to HIVE-20901) + runStatementOnDriver("insert into T values(2,5)");//makes delta_3_3 in T2 + + runStatementOnDriver("alter table T compact 'minor'"); + TestTxnCommands2.runWorker(hiveConf); + /* + target/tmp/org.apache.hadoop.hive.ql.TestTxnCommands3-1541810844849/warehouse/t/ +├── delete_delta_0000001_0000002_v0000019 +│   ├── _orc_acid_version +│   └── bucket_00000 +├── delete_delta_0000001_0000002_v0000021 +│   ├── _orc_acid_version +│   └── bucket_00000 +├── delta_0000001_0000001_0000 +│   ├── _orc_acid_version +│   └── bucket_00000 +├── delta_0000001_0000002_v0000019 +│   ├── _orc_acid_version +│   └── bucket_00000 +├── delta_0000001_0000002_v0000021 +│   ├── _orc_acid_version +│   └── bucket_00000 +└── delta_0000002_0000002_0000 + ├── _orc_acid_version + └── bucket_00000 + */ + TestTxnCommands2.runCleaner(hiveConf);//so this deletes d1 and d2 but now we have 2 different sets of deltas with exactly the same writeID range + //which should not happen. This was perhaps a bug all along but previously we'd create dirs with the same name and overwrite... + //all 4 dirs delta_1_2 are determined to be valid - makes sense since the later one doesn't shadow the earlier one. + + expectedList = new String[] { + "/t/delete_delta_0000001_0000003_v0000022", + "/t/delta_0000001_0000003_v0000022" + }; + actualList = fs.listStatus(new Path(warehousePath + "/t"), + FileUtils.HIDDEN_FILES_PATH_FILTER); + checkExpectedFiles(actualList, expectedList, warehousePath.toString()); + + //todo: these comments are bogus, i think + //ok, so this doesn't work - d2 is removed - this actually makes sense since if delta_1_2 is available the assumption is that + //it should be used in favor of delta_1_1 delta_2_2 + //relying on state of LM won't help here either: + /*1. start txn do a select * from T as above + * now the txn does the same select * again and while it's running compactor makes d_1_2 and cleaner runs while the 2nd select * is reading d2_2 + * The new locks will tell the cleaner that it's ok to clean.... + * One would argue it should then not get new locks if it already has the same type of lock on this resource.... + * */ + + + + + //alternatively, make the 2nd txn be a delete and verify that the reader doesn't see it in which case you'd have to do a major compaction + + + } + private static void checkExpectedFiles(FileStatus[] actualList, String[] expectedList, String filePrefix) throws Exception { + Set expectedSet = new HashSet<>(); + Set unexpectedSet = new HashSet<>(); + for(String f : expectedList) { + expectedSet.add(f); + } + for(FileStatus fs : actualList) { + String endOfPath = fs.getPath().toString().substring(fs.getPath().toString().indexOf(filePrefix) + filePrefix.length()); + if(!expectedSet.remove(endOfPath)) { + unexpectedSet.add(endOfPath); + } + } + Assert.assertTrue("not found set: " + expectedSet + " unexpected set: " + unexpectedSet, expectedSet.isEmpty() && unexpectedSet.isEmpty()); + } } diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java index 89ab043313..de2d89929b 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java @@ -580,7 +580,17 @@ public void testSnapshotIsolationWithAbortedTxnOnMmTable() throws Exception { hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); runStatementOnDriver("insert into " + TableExtended.MMTBL + " values (5, 6)"); hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); - // There should be 3 delta directories. The new one is the aborted one. + /** + * There should be 3 delta directories. The new one is the aborted one. + * + * target/tmp/org.apache.hadoop.hive.ql.TestTxnCommands-1541637725613/warehouse/mmtbl/ + ├── delta_0000001_0000001_0000 + │   └── 000000_0 + ├── delta_0000002_0000002_0000 + │   └── 000000_0 + └── delta_0000003_0000003_0000 + └── 000000_0 + */ verifyDirAndResult(3); // Execute SELECT statement and verify the result set (should be two rows). @@ -589,14 +599,11 @@ public void testSnapshotIsolationWithAbortedTxnOnMmTable() throws Exception { Assert.assertEquals(stringifyValues(expected), rs); // Run Cleaner. - // This run doesn't do anything for the above aborted transaction since - // the current compaction request entry in the compaction queue is updated - // to have highest_write_id when the worker is run before the aborted - // transaction. Specifically the id is 2 for the entry but the aborted - // transaction has 3 as writeId. This run does transition the entry - // "successful". + // delta_0000003_0000003_0000 produced by the aborted txn is removed even though it is + // above COMPACTION_QUEUE.CQ_HIGHEST_WRITE_ID since all data in it is aborted + // This run does transition the entry "successful". runCleaner(hiveConf); - verifyDirAndResult(3); + verifyDirAndResult(2); // Execute SELECT and verify that aborted operation is not counted for MM table. rs = runStatementOnDriver("select a,b from " + TableExtended.MMTBL + " order by a,b"); @@ -605,15 +612,10 @@ public void testSnapshotIsolationWithAbortedTxnOnMmTable() throws Exception { // Run initiator to execute CompactionTxnHandler.cleanEmptyAbortedTxns() Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXNS"), 1, TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXNS")); - Initiator i = new Initiator(); - i.setThreadId((int)i.getId()); - i.setConf(hiveConf); - AtomicBoolean stop = new AtomicBoolean(true); - i.init(stop, new AtomicBoolean()); - i.run(); + TestTxnCommands2.runInitiator(hiveConf); // This run of Initiator doesn't add any compaction_queue entry // since we only have one MM table with data - we don't compact MM tables. - verifyDirAndResult(3); + verifyDirAndResult(2); Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXNS"), 1, TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXNS")); @@ -638,7 +640,7 @@ public void testSnapshotIsolationWithAbortedTxnOnMmTable() throws Exception { runCleaner(hiveConf); // Run initiator to clean the row fro the aborted transaction from TXNS. - i.run(); + TestTxnCommands2.runInitiator(hiveConf); Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXNS"), 0, TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXNS")); Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXN_COMPONENTS"), diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java index e882c94cfe..eb0bfd9531 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnConcatenate.java @@ -81,13 +81,13 @@ public void testConcatenate() throws Exception { Assert.assertEquals(TxnStore.CLEANING_RESPONSE, rsp.getCompacts().get(0).getState()); String[][] expected2 = new String[][] { {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":1}\t1\t4", - "acidtbl/base_0000003/bucket_00001"}, + "acidtbl/base_0000003_v0000019/bucket_00001"}, {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t4\t4", - "acidtbl/base_0000003/bucket_00001"}, + "acidtbl/base_0000003_v0000019/bucket_00001"}, {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":1}\t5\t6", - "acidtbl/base_0000003/bucket_00001"}, + "acidtbl/base_0000003_v0000019/bucket_00001"}, {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t8\t8", - "acidtbl/base_0000003/bucket_00001"}}; + "acidtbl/base_0000003_v0000019/bucket_00001"}}; checkResult(expected2, testQuery, false, "check data after concatenate", LOG); } @Test @@ -122,11 +122,11 @@ public void testConcatenatePart() throws Exception { Assert.assertEquals(TxnStore.CLEANING_RESPONSE, rsp.getCompacts().get(0).getState()); String[][] expected2 = new String[][] { {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\t4", - "acidtblpart/p=p1/base_0000003/bucket_00001"}, + "acidtblpart/p=p1/base_0000003_v0000019/bucket_00001"}, {"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t4\t5", "acidtblpart/p=p2/delta_0000001_0000001_0000/bucket_00001"}, {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t5\t6", - "acidtblpart/p=p1/base_0000003/bucket_00001"}, + "acidtblpart/p=p1/base_0000003_v0000019/bucket_00001"}, {"{\"writeid\":3,\"bucketid\":536936448,\"rowid\":0}\t8\t8", "acidtblpart/p=p2/delta_0000003_0000003_0000/bucket_00001"}}; diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java index 333126b6a5..06e3ef2ba0 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java @@ -351,11 +351,11 @@ private void testImport(boolean isVectorized, boolean existingTarget) throws Exc TestTxnCommands2.runWorker(hiveConf); String[][] expected3 = new String[][] { {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2", - "t/delta_0000001_0000002/bucket_00000"}, + ".*t/delta_0000001_0000002_v000002[5-6]/bucket_00000"}, {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", - "t/delta_0000001_0000002/bucket_00000"}, + ".*t/delta_0000001_0000002_v000002[5-6]/bucket_00000"}, {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t0\t6", - "t/delta_0000001_0000002/bucket_00000"}}; + ".*t/delta_0000001_0000002_v000002[5-6]/bucket_00000"}}; checkResult(expected3, testQuery, isVectorized, "minor compact imported table"); } diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java index f7d39e5be3..3231a97009 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnLoadData.java @@ -128,8 +128,8 @@ private void loadDataUpdate(boolean isVectorized) throws Exception { runStatementOnDriver("alter table T compact 'minor'"); TestTxnCommands2.runWorker(hiveConf); String[][] expected3 = new String[][] { - {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/delta_0000001_0000004/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000001_0000004/bucket_00000"} + {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/delta_0000001_0000004_v0000029/bucket_00000"}, + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/delta_0000001_0000004_v0000029/bucket_00000"} }; checkResult(expected3, testQuery, isVectorized, "delete compact minor"); @@ -155,9 +155,9 @@ private void loadDataUpdate(boolean isVectorized) throws Exception { runStatementOnDriver("alter table T compact 'major'"); TestTxnCommands2.runWorker(hiveConf); String[][] expected6 = new String[][]{ - {"{\"writeid\":7,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/base_0000009/bucket_00000"}, - {"{\"writeid\":7,\"bucketid\":536870912,\"rowid\":1}\t1\t17", "t/base_0000009/bucket_00000"}, - {"{\"writeid\":9,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/base_0000009/bucket_00000"} + {"{\"writeid\":7,\"bucketid\":536870912,\"rowid\":0}\t1\t17", "t/base_0000009_v0000042/bucket_00000"}, + {"{\"writeid\":7,\"bucketid\":536870912,\"rowid\":1}\t1\t17", "t/base_0000009_v0000042/bucket_00000"}, + {"{\"writeid\":9,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/base_0000009_v0000042/bucket_00000"} }; checkResult(expected6, testQuery, isVectorized, "load data inpath compact major"); } @@ -192,10 +192,10 @@ private void loadData(boolean isVectorized) throws Exception { runStatementOnDriver("alter table T compact 'minor'"); TestTxnCommands2.runWorker(hiveConf); String[][] expected1 = new String[][] { - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t0\t2", "t/delta_0000001_0000002/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t0\t4", "t/delta_0000001_0000002/bucket_00000"}, - {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/delta_0000001_0000002/bucket_00000"}, - {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000001_0000002/bucket_00000"} + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t0\t2", "t/delta_0000001_0000002_v0000025/bucket_00000"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t0\t4", "t/delta_0000001_0000002_v0000025/bucket_00000"}, + {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/delta_0000001_0000002_v0000025/bucket_00000"}, + {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/delta_0000001_0000002_v0000025/bucket_00000"} }; checkResult(expected1, testQuery, isVectorized, "load data inpath (minor)"); @@ -204,11 +204,11 @@ private void loadData(boolean isVectorized) throws Exception { runStatementOnDriver("alter table T compact 'major'"); TestTxnCommands2.runWorker(hiveConf); String[][] expected2 = new String[][] { - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t0\t2", "t/base_0000003/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t0\t4", "t/base_0000003/bucket_00000"}, - {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/base_0000003/bucket_00000"}, - {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/base_0000003/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/base_0000003/bucket_00000"} + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t0\t2", "t/base_0000003_v0000030/bucket_00000"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t0\t4", "t/base_0000003_v0000030/bucket_00000"}, + {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/base_0000003_v0000030/bucket_00000"}, + {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/base_0000003_v0000030/bucket_00000"}, + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t2\t2", "t/base_0000003_v0000030/bucket_00000"} }; checkResult(expected2, testQuery, isVectorized, "load data inpath (major)"); @@ -226,9 +226,9 @@ private void loadData(boolean isVectorized) throws Exception { runStatementOnDriver("alter table T compact 'major'"); TestTxnCommands2.runWorker(hiveConf); String[][] expected4 = new String[][] { - {"{\"writeid\":4,\"bucketid\":536870912,\"rowid\":0}\t5\t6", "t/base_0000005/bucket_00000"}, - {"{\"writeid\":4,\"bucketid\":536870912,\"rowid\":1}\t7\t8", "t/base_0000005/bucket_00000"}, - {"{\"writeid\":5,\"bucketid\":536870912,\"rowid\":0}\t6\t6", "t/base_0000005/bucket_00000"}}; + {"{\"writeid\":4,\"bucketid\":536870912,\"rowid\":0}\t5\t6", "t/base_0000005_v0000040/bucket_00000"}, + {"{\"writeid\":4,\"bucketid\":536870912,\"rowid\":1}\t7\t8", "t/base_0000005_v0000040/bucket_00000"}, + {"{\"writeid\":5,\"bucketid\":536870912,\"rowid\":0}\t6\t6", "t/base_0000005_v0000040/bucket_00000"}}; checkResult(expected4, testQuery, isVectorized, "load data inpath overwrite (major)"); } /** @@ -307,13 +307,13 @@ private void loadDataNonAcid2AcidConversion(boolean isVectorized) throws Excepti String[][] expected3 = new String[][] { {"{\"writeid\":10000002,\"bucketid\":536870912,\"rowid\":0}\t5\t6", - "t/base_10000003/bucket_00000"}, + "t/base_10000003_v0000035/bucket_00000"}, {"{\"writeid\":10000002,\"bucketid\":536870912,\"rowid\":1}\t7\t8", - "t/base_10000003/bucket_00000"}, + "t/base_10000003_v0000035/bucket_00000"}, {"{\"writeid\":10000002,\"bucketid\":536936448,\"rowid\":0}\t8\t8", - "t/base_10000003/bucket_00001"}, + "t/base_10000003_v0000035/bucket_00001"}, {"{\"writeid\":10000003,\"bucketid\":536870912,\"rowid\":0}\t9\t9", - "t/base_10000003/bucket_00000"} + "t/base_10000003_v0000035/bucket_00000"} }; checkResult(expected3, testQuery, isVectorized, "load data inpath overwrite (major)"); } @@ -436,10 +436,10 @@ private void testMultiStatement(boolean isVectorized) throws Exception { runStatementOnDriver("alter table T compact 'major'"); TestTxnCommands2.runWorker(hiveConf); String[][] expected2 = new String[][] { - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/base_0000001/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/base_0000001/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":0}\t5\t5", "t/base_0000001/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":1}\t6\t6", "t/base_0000001/bucket_00000"} + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2", "t/base_0000001_v0000023/bucket_00000"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t4", "t/base_0000001_v0000023/bucket_00000"}, + {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":0}\t5\t5", "t/base_0000001_v0000023/bucket_00000"}, + {"{\"writeid\":1,\"bucketid\":536870913,\"rowid\":1}\t6\t6", "t/base_0000001_v0000023/bucket_00000"} }; checkResult(expected2, testQuery, isVectorized, "load data inpath (major)"); //at lest for now, Load Data w/Overwrite is not allowed in a txn: HIVE-18154 diff --git ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java index c74a707e4b..84d7c4e52a 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java +++ ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java @@ -149,7 +149,7 @@ public void testNoBuckets() throws Exception { LOG.warn(s); } /* -├── base_0000002 +├── base_0000002_v0000025 │   ├── bucket_00000 │   └── bucket_00001 ├── delete_delta_0000002_0000002_0000 @@ -161,24 +161,28 @@ public void testNoBuckets() throws Exception { └── delta_0000002_0000002_0000 └── bucket_00000 */ - Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t3\t3\t")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("nobuckets/base_0000002/bucket_00000")); - Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t0\t0\t17\t")); - Assert.assertTrue(rs.get(1), rs.get(1).endsWith("nobuckets/base_0000002/bucket_00000")); - Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":1,\"bucketid\":536936448,\"rowid\":1}\t2\t2\t2\t")); - Assert.assertTrue(rs.get(2), rs.get(2).endsWith("nobuckets/base_0000002/bucket_00001")); - Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\t1\t17\t")); - Assert.assertTrue(rs.get(3), rs.get(3).endsWith("nobuckets/base_0000002/bucket_00001")); + + String expected[][] = { + {"{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t0\t0\t17", "nobuckets/base_0000002_v0000025/bucket_00000"}, + {"{\"writeid\":2,\"bucketid\":536936448,\"rowid\":0}\t1\t1\t17", "nobuckets/base_0000002_v0000025/bucket_00001"}, + {"{\"writeid\":1,\"bucketid\":536936448,\"rowid\":1}\t2\t2\t2", "nobuckets/base_0000002_v0000025/bucket_00001"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t3\t3\t3", "nobuckets/base_0000002_v0000025/bucket_00000"} + }; + checkResult(expected, + "select ROW__ID, c1, c2, c3" + (shouldVectorize() ? "" : ", INPUT__FILE__NAME") + + " from nobuckets order by c1, c2, c3", + shouldVectorize(), + "After Major Compaction", LOG); expectedFiles.clear(); - expectedFiles.add("delete_delta_0000002_0000002_0000/bucket_00000"); - expectedFiles.add("delete_delta_0000002_0000002_0000/bucket_00001"); - expectedFiles.add("uckets/delta_0000001_0000001_0000/bucket_00000"); - expectedFiles.add("uckets/delta_0000001_0000001_0000/bucket_00001"); - expectedFiles.add("uckets/delta_0000002_0000002_0000/bucket_00000"); - expectedFiles.add("uckets/delta_0000002_0000002_0000/bucket_00001"); - expectedFiles.add("/warehouse/nobuckets/base_0000002/bucket_00000"); - expectedFiles.add("/warehouse/nobuckets/base_0000002/bucket_00001"); + expectedFiles.add("obuckets/delete_delta_0000002_0000002_0000/bucket_00000"); + expectedFiles.add("obuckets/delete_delta_0000002_0000002_0000/bucket_00001"); + expectedFiles.add("house/nobuckets/delta_0000001_0000001_0000/bucket_00000"); + expectedFiles.add("house/nobuckets/delta_0000001_0000001_0000/bucket_00001"); + expectedFiles.add("house/nobuckets/delta_0000002_0000002_0000/bucket_00000"); + expectedFiles.add("house/nobuckets/delta_0000002_0000002_0000/bucket_00001"); + expectedFiles.add("/warehouse/nobuckets/base_0000002_v0000025/bucket_00000"); + expectedFiles.add("/warehouse/nobuckets/base_0000002_v0000025/bucket_00001"); assertExpectedFileSet(expectedFiles, getWarehouseDir() + "/nobuckets"); TestTxnCommands2.runCleaner(hiveConf); @@ -187,8 +191,8 @@ public void testNoBuckets() throws Exception { Assert.assertEquals("Unexpected result after clean", stringifyValues(result), rs); expectedFiles.clear(); - expectedFiles.add("nobuckets/base_0000002/bucket_00000"); - expectedFiles.add("nobuckets/base_0000002/bucket_00001"); + expectedFiles.add("/warehouse/nobuckets/base_0000002_v0000025/bucket_00000"); + expectedFiles.add("/warehouse/nobuckets/base_0000002_v0000025/bucket_00001"); assertExpectedFileSet(expectedFiles, getWarehouseDir() + "/nobuckets"); } @@ -469,23 +473,23 @@ logical bucket (tranche) /*Compaction preserves location of rows wrt buckets/tranches (for now)*/ String expected4[][] = { {"{\"writeid\":0,\"bucketid\":537001984,\"rowid\":0}\t1\t2", - "warehouse/t/base_10000002/bucket_00002"}, + "warehouse/t/base_10000002_v0000029/bucket_00002"}, {"{\"writeid\":0,\"bucketid\":537001984,\"rowid\":1}\t2\t4", - "warehouse/t/base_10000002/bucket_00002"}, + "warehouse/t/base_10000002_v0000029/bucket_00002"}, {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":1}\t5\t6", - "warehouse/t/base_10000002/bucket_00000"}, + "warehouse/t/base_10000002_v0000029/bucket_00000"}, {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":0}\t9\t10", - "warehouse/t/base_10000002/bucket_00000"}, + "warehouse/t/base_10000002_v0000029/bucket_00000"}, {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":3}\t10\t20", - "warehouse/t/base_10000002/bucket_00000"}, + "warehouse/t/base_10000002_v0000029/bucket_00000"}, {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":2}\t12\t12", - "warehouse/t/base_10000002/bucket_00000"}, + "warehouse/t/base_10000002_v0000029/bucket_00000"}, {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":4}\t20\t40", - "warehouse/t/base_10000002/bucket_00000"}, + "warehouse/t/base_10000002_v0000029/bucket_00000"}, {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":5}\t50\t60", - "warehouse/t/base_10000002/bucket_00000"}, + "warehouse/t/base_10000002_v0000029/bucket_00000"}, {"{\"writeid\":10000001,\"bucketid\":536936448,\"rowid\":0}\t60\t88", - "warehouse/t/base_10000002/bucket_00001"}, + "warehouse/t/base_10000002_v0000029/bucket_00001"}, }; checkExpected(rs, expected4,"after major compact"); } @@ -741,15 +745,15 @@ public void testNonAcidToAcidVectorzied() throws Exception { rs = runStatementOnDriver(query); String[][] expected5 = {//the row__ids are the same after compaction {"{\"writeid\":10000001,\"bucketid\":536870912,\"rowid\":0}\t1\t17", - "warehouse/t/base_10000001/bucket_00000"}, + "warehouse/t/base_10000001_v0000029/bucket_00000"}, {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":1}\t2\t4", - "warehouse/t/base_10000001/bucket_00000"}, + "warehouse/t/base_10000001_v0000029/bucket_00000"}, {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":2}\t5\t6", - "warehouse/t/base_10000001/bucket_00000"}, + "warehouse/t/base_10000001_v0000029/bucket_00000"}, {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":3}\t6\t8", - "warehouse/t/base_10000001/bucket_00000"}, + "warehouse/t/base_10000001_v0000029/bucket_00000"}, {"{\"writeid\":0,\"bucketid\":536870912,\"rowid\":4}\t9\t10", - "warehouse/t/base_10000001/bucket_00000"} + "warehouse/t/base_10000001_v0000029/bucket_00000"} }; checkExpected(rs, expected5, "After major compaction"); //vectorized because there is INPUT__FILE__NAME @@ -811,10 +815,10 @@ public void testCompactStatsGather() throws Exception { {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t1\t1\t4\t3", "t/p=1/q=1/delta_0000001_0000001_0000/bucket_00000"}, {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t1\t1\t5\t1", "t/p=1/q=1/delta_0000003_0000003_0000/bucket_00000"}, {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":1}\t1\t1\t5\t3", "t/p=1/q=1/delta_0000003_0000003_0000/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2\t4\t2", "t/p=1/q=2/base_0000003/bucket_00000"}, - {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t1\t2\t4\t4", "t/p=1/q=2/base_0000003/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t1\t2\t5\t2", "t/p=1/q=2/base_0000003/bucket_00000"}, - {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":1}\t1\t2\t5\t4", "t/p=1/q=2/base_0000003/bucket_00000"} + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\t1\t2\t4\t2", "t/p=1/q=2/base_0000003_v0000020/bucket_00000"}, + {"{\"writeid\":1,\"bucketid\":536870912,\"rowid\":1}\t1\t2\t4\t4", "t/p=1/q=2/base_0000003_v0000020/bucket_00000"}, + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t1\t2\t5\t2", "t/p=1/q=2/base_0000003_v0000020/bucket_00000"}, + {"{\"writeid\":3,\"bucketid\":536870912,\"rowid\":1}\t1\t2\t5\t4", "t/p=1/q=2/base_0000003_v0000020/bucket_00000"} }; checkExpected(rs, expected2, "after major compaction"); diff --git ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java index dab0d982c9..0a53d94c39 100644 --- ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java +++ ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; +import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.io.HiveInputFormat; import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; import org.apache.hadoop.hive.ql.session.SessionState; @@ -42,6 +43,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.regex.Matcher; public abstract class TxnCommandsBaseForTests { private static final Logger LOG = LoggerFactory.getLogger(TxnCommandsBaseForTests.class); @@ -196,7 +198,7 @@ void assertVectorized(boolean vectorized, String query) throws Exception { /** * Will assert that actual files match expected. * @param expectedFiles - suffixes of expected Paths. Must be the same length - * @param rootPath - table or patition root where to start looking for actual files, recursively + * @param rootPath - table or partition root where to start looking for actual files, recursively */ void assertExpectedFileSet(Set expectedFiles, String rootPath) throws Exception { int suffixLength = 0; @@ -227,10 +229,27 @@ void checkExpected(List rs, String[][] expected, String msg, Logger LOG, for(int i = 0; i < expected.length; i++) { Assert.assertTrue("Actual line (data) " + i + " data: " + rs.get(i) + "; expected " + expected[i][0], rs.get(i).startsWith(expected[i][0])); if(checkFileName) { - Assert.assertTrue("Actual line(file) " + i + " file: " + rs.get(i), rs.get(i).endsWith(expected[i][1])); +// Assert.assertTrue("Actual line(file) " + i + " file: " + rs.get(i), +// normalizeBaseDelta(rs.get(i)).endsWith(normalizeBaseDelta(expected[i][1]))); + Assert.assertTrue("Actual line(file) " + i + " file: " + rs.get(i), rs.get(i).endsWith(expected[i][1]) || rs.get(i).matches(expected[i][1])); } } } + private static String normalizeBaseDelta(String path) { + /*Matcher m = AcidUtils.VISIBILITY_PATTERN.matcher(path); + if(!m.find()) { + return path; + } + String match = m.group(); + String newPath = path.replace(match, "");*/ + return path; + /* + * the "_vXXXXXXX" changes if more statements are executed before the one that reads + * the filenames making UTs sensitive to modifications. + * If this proves a pain, use comparison w/o specific txn below. + */ + //return path.replaceAll(AcidUtils.VISIBILITY_PATTERN.toString(), AcidUtils.VISIBILITY_PREFIX); + } void logResult(Logger LOG, List rs) { StringBuilder sb = new StringBuilder(); for(String s : rs) { @@ -254,4 +273,9 @@ void dropTable(String[] tabs) throws Exception { d.run("drop table if exists " + tab); } } + Driver swapDrivers(Driver otherDriver) { + Driver tmp = d; + d = otherDriver; + return tmp; + } } diff --git ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidInputFormat.java ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidInputFormat.java index a1eb39dd95..919cf4bbc9 100644 --- ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidInputFormat.java +++ ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidInputFormat.java @@ -72,7 +72,7 @@ public void testDeltaMetaDataReadFieldsWithStatementIds() throws Exception { @Test public void testDeltaMetaConstructWithState() throws Exception { - DeltaMetaData deltaMetaData = new AcidInputFormat.DeltaMetaData(2000L, 2001L, Arrays.asList(97, 98, 99)); + DeltaMetaData deltaMetaData = new AcidInputFormat.DeltaMetaData(2000L, 2001L, Arrays.asList(97, 98, 99), 0); assertThat(deltaMetaData.getMinWriteId(), is(2000L)); assertThat(deltaMetaData.getMaxWriteId(), is(2001L)); @@ -91,7 +91,7 @@ public void testDeltaMetaDataReadFieldsWithStatementIdsResetsState() throws Exce statementIds.add(97); statementIds.add(98); statementIds.add(99); - DeltaMetaData deltaMetaData = new AcidInputFormat.DeltaMetaData(2000L, 2001L, statementIds); + DeltaMetaData deltaMetaData = new AcidInputFormat.DeltaMetaData(2000L, 2001L, statementIds, 0); deltaMetaData.readFields(mockDataInput); verify(mockDataInput, times(3)).readInt(); diff --git ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java index c51ffa87f0..88a0553829 100644 --- ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java +++ ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; +import java.util.BitSet; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -28,7 +29,9 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.ValidCompactorWriteIdList; +import org.apache.hadoop.hive.common.ValidReadTxnList; import org.apache.hadoop.hive.common.ValidReaderWriteIdList; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.exec.Utilities; @@ -110,7 +113,10 @@ public void testParsing() throws Exception { new MockFile("mock:/tmp/delta_000005_000006/bucket_00001", 500, new byte[0]), new MockFile("mock:/tmp/delete_delta_000005_000006/bucket_00001", 500, new byte[0])); - assertEquals(123, AcidUtils.parseBase(new Path("/tmp/base_000123"))); + assertEquals(123, + AcidUtils.ParsedBase.parseBase(new Path("/tmp/base_000123")).getWriteId()); + assertEquals(0, + AcidUtils.ParsedBase.parseBase(new Path("/tmp/base_000123")).getVisibilityTxnId()); Path dir = new Path("/tmp/tbl"); AcidOutputFormat.Options opts = AcidUtils.parseBaseOrDeltaBucketFilename(new Path(dir, "base_567/bucket_123"), @@ -192,6 +198,8 @@ public void testOriginalDeltas() throws Exception { new MockFile("mock:/tbl/part1/delta_025_030/bucket_0", 0, new byte[0]), new MockFile("mock:/tbl/part1/delta_050_100/bucket_0", 0, new byte[0]), new MockFile("mock:/tbl/part1/delta_101_101/bucket_0", 0, new byte[0])); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(new MockPath(fs, "mock:/tbl/part1"), conf, new ValidReaderWriteIdList("tbl:100:" + Long.MAX_VALUE + ":")); @@ -234,6 +242,8 @@ public void testBaseDeltas() throws Exception { new MockFile("mock:/tbl/part1/delta_025_030/bucket_0", 0, new byte[0]), new MockFile("mock:/tbl/part1/delta_050_105/bucket_0", 0, new byte[0]), new MockFile("mock:/tbl/part1/delta_90_120/bucket_0", 0, new byte[0])); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(new MockPath(fs, "mock:/tbl/part1"), conf, new ValidReaderWriteIdList("tbl:100:" + Long.MAX_VALUE + ":")); @@ -263,6 +273,8 @@ public void testObsoleteOriginals() throws Exception { new MockFile("mock:/tbl/part1/000000_0", 500, new byte[0]), new MockFile("mock:/tbl/part1/000001_1", 500, new byte[0])); Path part = new MockPath(fs, "/tbl/part1"); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(part, conf, new ValidReaderWriteIdList("tbl:150:" + Long.MAX_VALUE + ":")); // Obsolete list should include the two original bucket files, and the old base dir @@ -284,6 +296,8 @@ public void testOverlapingDelta() throws Exception { new MockFile("mock:/tbl/part1/delta_052_55/bucket_0", 500, new byte[0]), new MockFile("mock:/tbl/part1/base_50/bucket_0", 500, new byte[0])); Path part = new MockPath(fs, "mock:/tbl/part1"); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(part, conf, new ValidReaderWriteIdList("tbl:100:" + Long.MAX_VALUE + ":")); assertEquals("mock:/tbl/part1/base_50", dir.getBaseDirectory().toString()); @@ -319,6 +333,8 @@ public void testOverlapingDelta2() throws Exception { new MockFile("mock:/tbl/part1/delta_058_58/bucket_0", 500, new byte[0]), new MockFile("mock:/tbl/part1/base_50/bucket_0", 500, new byte[0])); Path part = new MockPath(fs, "mock:/tbl/part1"); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(part, conf, new ValidReaderWriteIdList("tbl:100:" + Long.MAX_VALUE + ":")); assertEquals("mock:/tbl/part1/base_50", dir.getBaseDirectory().toString()); @@ -345,6 +361,9 @@ public void deltasWithOpenTxnInRead() throws Exception { new MockFile("mock:/tbl/part1/delta_1_1/bucket_0", 500, new byte[0]), new MockFile("mock:/tbl/part1/delta_2_5/bucket_0", 500, new byte[0])); Path part = new MockPath(fs, "mock:/tbl/part1"); + //hypothetically, txn 50 is open and writing write ID 4 + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[] {50}, new BitSet(), 1000, 55).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(part, conf, new ValidReaderWriteIdList("tbl:100:4:4")); List delts = dir.getCurrentDirectories(); assertEquals(2, delts.size()); @@ -366,6 +385,9 @@ public void deltasWithOpenTxnInRead2() throws Exception { new MockFile("mock:/tbl/part1/delta_4_4_3/bucket_0", 500, new byte[0]), new MockFile("mock:/tbl/part1/delta_101_101_1/bucket_0", 500, new byte[0])); Path part = new MockPath(fs, "mock:/tbl/part1"); + //hypothetically, txn 50 is open and writing write ID 4 + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[] {50}, new BitSet(), 1000, 55).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(part, conf, new ValidReaderWriteIdList("tbl:100:4:4")); List delts = dir.getCurrentDirectories(); assertEquals(2, delts.size()); @@ -380,6 +402,8 @@ public void deltasWithOpenTxnsNotInCompact() throws Exception { new MockFile("mock:/tbl/part1/delta_1_1/bucket_0", 500, new byte[0]), new MockFile("mock:/tbl/part1/delta_2_5/bucket_0", 500, new byte[0])); Path part = new MockPath(fs, "mock:/tbl/part1"); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(part, conf, new ValidCompactorWriteIdList("tbl:4:" + Long.MAX_VALUE)); List delts = dir.getCurrentDirectories(); @@ -397,6 +421,8 @@ public void deltasWithOpenTxnsNotInCompact2() throws Exception { new byte[0]), new MockFile("mock:/tbl/part1/delta_6_10/bucket_0", 500, new byte[0])); Path part = new MockPath(fs, "mock:/tbl/part1"); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(part, conf, new ValidCompactorWriteIdList("tbl:3:" + Long.MAX_VALUE)); List delts = dir.getCurrentDirectories(); @@ -421,6 +447,8 @@ public void testBaseWithDeleteDeltas() throws Exception { new MockFile("mock:/tbl/part1/delta_050_105/bucket_0", 0, new byte[0]), new MockFile("mock:/tbl/part1/delete_delta_050_105/bucket_0", 0, new byte[0]), new MockFile("mock:/tbl/part1/delete_delta_110_110/bucket_0", 0, new byte[0])); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(new MockPath(fs, "mock:/tbl/part1"), conf, new ValidReaderWriteIdList("tbl:100:" + Long.MAX_VALUE + ":")); @@ -459,6 +487,8 @@ public void testOverlapingDeltaAndDeleteDelta() throws Exception { new MockFile("mock:/tbl/part1/delete_delta_052_55/bucket_0", 500, new byte[0]), new MockFile("mock:/tbl/part1/base_50/bucket_0", 500, new byte[0])); Path part = new MockPath(fs, "mock:/tbl/part1"); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(part, conf, new ValidReaderWriteIdList("tbl:100:" + Long.MAX_VALUE + ":")); assertEquals("mock:/tbl/part1/base_50", dir.getBaseDirectory().toString()); @@ -488,6 +518,8 @@ public void testMinorCompactedDeltaMakesInBetweenDelteDeltaObsolete() throws Exc new MockFile("mock:/tbl/part1/delta_40_60/bucket_0", 500, new byte[0]), new MockFile("mock:/tbl/part1/delete_delta_50_50/bucket_0", 500, new byte[0])); Path part = new MockPath(fs, "mock:/tbl/part1"); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(part, conf, new ValidReaderWriteIdList("tbl:100:" + Long.MAX_VALUE + ":")); List obsolete = dir.getObsolete(); @@ -515,6 +547,8 @@ public void deltasAndDeleteDeltasWithOpenTxnsNotInCompact() throws Exception { new MockFile("mock:/tbl/part1/delete_delta_7_7/bucket_0", 500, new byte[0]), new MockFile("mock:/tbl/part1/delta_6_10/bucket_0", 500, new byte[0])); Path part = new MockPath(fs, "mock:/tbl/part1"); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(part, conf, new ValidCompactorWriteIdList("tbl:4:" + Long.MAX_VALUE + ":")); List delts = dir.getCurrentDirectories(); @@ -537,6 +571,9 @@ public void deleteDeltasWithOpenTxnInRead() throws Exception { new MockFile("mock:/tbl/part1/delta_4_4_3/bucket_0", 500, new byte[0]), new MockFile("mock:/tbl/part1/delta_101_101_1/bucket_0", 500, new byte[0])); Path part = new MockPath(fs, "mock:/tbl/part1"); + //hypothetically, txn 50 is open and writing write ID 4 + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[] {50}, new BitSet(), 1000, 55).writeToString()); AcidUtils.Directory dir = AcidUtils.getAcidState(part, conf, new ValidReaderWriteIdList("tbl:100:4:4")); List delts = dir.getCurrentDirectories(); assertEquals(3, delts.size()); diff --git ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java index 9123d725ab..3a83408430 100644 --- ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java +++ ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java @@ -31,6 +31,7 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; +import java.util.BitSet; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -46,6 +47,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.common.ValidReadTxnList; import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.common.type.Date; @@ -598,6 +600,8 @@ public void testACIDSplitStrategy() throws Exception { conf.set("bucket_count", "2"); conf.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); OrcInputFormat.Context context = new OrcInputFormat.Context(conf); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); MockFileSystem fs = new MockFileSystem(conf, new MockFile("mock:/a/delta_000_001/bucket_000000", 1000, new byte[1], new MockBlock("host1")), new MockFile("mock:/a/delta_000_001/bucket_000001", 1000, new byte[1], new MockBlock("host1")), @@ -621,6 +625,8 @@ public void testACIDSplitStrategyForSplitUpdate() throws Exception { conf.set("bucket_count", "2"); conf.set(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, "true"); conf.set(hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES, "default"); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); OrcInputFormat.Context context = new OrcInputFormat.Context(conf); // Case 1: Test with just originals => Single split strategy with two splits. @@ -762,6 +768,8 @@ public void testFSCallsVectorizedOrcAcidRowBatchReader() throws IOException { //set up props for read conf.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); AcidUtils.setAcidOperationalProperties(conf, true, null); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); conf.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS, TestVectorizedOrcAcidRowBatchReader.DummyRow.getColumnNamesProperty()); conf.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS_TYPES, @@ -908,6 +916,8 @@ public void testEtlCombinedStrategy() throws Exception { AcidUtils.setAcidOperationalProperties(conf, true, null); conf.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); conf.set(hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES, "default"); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); OrcInputFormat.Context context = new OrcInputFormat.Context(conf); MockFileSystem fs = new MockFileSystem(conf, @@ -2337,6 +2347,8 @@ public void testVectorizationWithAcid() throws Exception { StructObjectInspector inspector = new BigRowInspector(); JobConf conf = createMockExecutionEnvironment(workDir, new Path("mock:///"), "vectorizationAcid", inspector, true, 1); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); // write the orc file to the mock file system Path partDir = new Path(conf.get("mapred.input.dir")); @@ -2536,6 +2548,8 @@ public void testCombinationInputFormatWithAcid() throws Exception { // call getsplits conf.setInt(hive_metastoreConstants.BUCKET_COUNT, BUCKETS); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); HiveInputFormat inputFormat = new CombineHiveInputFormat(); InputSplit[] splits = inputFormat.getSplits(conf, 1); @@ -3644,6 +3658,8 @@ public void testACIDReaderNoFooterSerializeWithDeltas() throws Exception { //set up props for read conf.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); AcidUtils.setAcidOperationalProperties(conf, true, null); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); OrcInputFormat orcInputFormat = new OrcInputFormat(); @@ -3724,6 +3740,8 @@ public void testACIDReaderFooterSerializeWithDeltas() throws Exception { //set up props for read conf.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); AcidUtils.setAcidOperationalProperties(conf, true, null); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); OrcInputFormat orcInputFormat = new OrcInputFormat(); InputSplit[] splits = orcInputFormat.getSplits(conf, 2); diff --git ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java index bc03620e0e..e0dfeab985 100644 --- ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java +++ ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java @@ -20,6 +20,8 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.hive.common.ValidReadTxnList; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.io.BucketCodec; import org.apache.orc.CompressionKind; @@ -68,11 +70,11 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.BitSet; import java.util.List; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; public class TestOrcRawRecordMerger { @@ -595,6 +597,8 @@ public void testEmpty() throws Exception { AcidUtils.baseDir(100)), BUCKET), wo); w.close(); } + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); ValidWriteIdList writeIdList = new ValidReaderWriteIdList("testEmpty:200:" + Long.MAX_VALUE); AcidUtils.Directory directory = AcidUtils.getAcidState(root, conf, writeIdList); @@ -664,6 +668,8 @@ private void testNewBaseAndDelta(boolean use130Format) throws Exception { ru.delete(200, new MyRow("", 8, 0, BUCKET_PROPERTY)); ru.close(false); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); ValidWriteIdList writeIdList = new ValidReaderWriteIdList("testNewBaseAndDelta:200:" + Long.MAX_VALUE); AcidUtils.Directory directory = AcidUtils.getAcidState(root, conf, writeIdList); @@ -1149,6 +1155,8 @@ public synchronized void addedRow(int rows) throws IOException { InputFormat inf = new OrcInputFormat(); JobConf job = new JobConf(); + job.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS, BigRow.getColumnNamesProperty()); job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS_TYPES, BigRow.getColumnTypesProperty()); AcidUtils.setAcidOperationalProperties(job, true, null); @@ -1282,6 +1290,8 @@ public synchronized void addedRow(int rows) throws IOException { InputFormat inf = new OrcInputFormat(); JobConf job = new JobConf(); + job.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); job.set("mapred.min.split.size", "1"); job.set("mapred.max.split.size", "2"); job.set("mapred.input.dir", root.toString()); @@ -1379,6 +1389,8 @@ public void testRecordReaderDelta() throws Exception { InputFormat inf = new OrcInputFormat(); JobConf job = new JobConf(); + job.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); job.set("mapred.min.split.size", "1"); job.set("mapred.max.split.size", "2"); job.set("mapred.input.dir", root.toString()); @@ -1456,6 +1468,8 @@ private void testRecordReaderIncompleteDelta(boolean use130Format) throws Except } InputFormat inf = new OrcInputFormat(); JobConf job = new JobConf(); + job.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); job.set("mapred.input.dir", root.toString()); job.set("bucket_count", "2"); job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS, MyRow.getColumnNamesProperty()); diff --git ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedOrcAcidRowBatchReader.java ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedOrcAcidRowBatchReader.java index 99317cc3bf..3382288988 100644 --- ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedOrcAcidRowBatchReader.java +++ ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedOrcAcidRowBatchReader.java @@ -19,12 +19,15 @@ import java.io.File; import java.util.ArrayList; +import java.util.BitSet; import java.util.List; import java.util.Properties; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.ValidReadTxnList; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; @@ -272,6 +275,8 @@ private void testDeleteEventFiltering() throws Exception { new DummyRow(-1, 2, 3, bucket)); updater.close(false); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); //HWM is not important - just make sure deltas created above are read as // if committed conf.set(ValidWriteIdList.VALID_WRITEIDS_KEY, @@ -410,6 +415,8 @@ private void testDeleteEventFiltering2() throws Exception { new DummyRow(-1, 5, 10000003, bucket)); updater.close(false); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); //HWM is not important - just make sure deltas created above are read as // if committed conf.set(ValidWriteIdList.VALID_WRITEIDS_KEY, @@ -695,6 +702,8 @@ public void testDeleteEventOriginalFiltering() throws Exception { writer.close(); conf.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); int bucket = 0; @@ -946,6 +955,8 @@ private void testDeleteEventOriginalFiltering2() throws Exception { @Test public void testVectorizedOrcAcidRowBatchReader() throws Exception { conf.set("bucket_count", "1"); + conf.set(ValidTxnList.VALID_TXNS_KEY, + new ValidReadTxnList(new long[0], new BitSet(), 1000, Long.MAX_VALUE).writeToString()); int bucket = 0; AcidOutputFormat.Options options = new AcidOutputFormat.Options(conf) diff --git ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java index 77fe73687a..1990899128 100644 --- ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java +++ ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hive.metastore.txn.AcidWriteSetService; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; +import org.apache.hadoop.hive.ql.TestTxnCommands2; import org.junit.After; import org.junit.Assert; import org.apache.hadoop.hive.common.FileUtils; @@ -87,7 +88,7 @@ private static HiveConf conf = new HiveConf(Driver.class); private HiveTxnManager txnMgr; private Context ctx; - private Driver driver; + private Driver driver, driver2; private TxnStore txnHandler; public TestDbTxnManager2() throws Exception { @@ -103,6 +104,7 @@ public void setUp() throws Exception { SessionState.start(conf); ctx = new Context(conf); driver = new Driver(new QueryState.Builder().withHiveConf(conf).nonIsolated().build(), null); + driver2 = new Driver(new QueryState.Builder().withHiveConf(conf).build(), null); TxnDbUtil.cleanDb(conf); TxnDbUtil.prepDb(conf); SessionState ss = SessionState.get(); @@ -115,6 +117,7 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { driver.close(); + driver2.close(); if (txnMgr != null) { txnMgr.closeTxnManager(); } @@ -548,10 +551,10 @@ public void testMetastoreTablesCleanup() throws Exception { checkCmdOnDriver(cpr); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE where CQ_DATABASE='temp' and CQ_TABLE='t11' and CQ_STATE='i' and CQ_TYPE='i'"); Assert.assertEquals(1, count); - org.apache.hadoop.hive.ql.TestTxnCommands2.runWorker(conf); + TestTxnCommands2.runWorker(conf); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE where CQ_DATABASE='temp' and CQ_TABLE='t11' and CQ_STATE='r' and CQ_TYPE='i'"); Assert.assertEquals(1, count); - org.apache.hadoop.hive.ql.TestTxnCommands2.runCleaner(conf); + TestTxnCommands2.runCleaner(conf); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE where CQ_DATABASE='temp' and CQ_TABLE='t11'"); Assert.assertEquals(0, count); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPLETED_COMPACTIONS where CC_DATABASE='temp' and CC_TABLE='t11' and CC_STATE='s' and CC_TYPE='i'"); @@ -561,10 +564,10 @@ public void testMetastoreTablesCleanup() throws Exception { checkCmdOnDriver(cpr); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE where CQ_DATABASE='temp' and CQ_TABLE='t12p' and CQ_PARTITION='ds=tomorrow/hour=2' and CQ_STATE='i' and CQ_TYPE='i'"); Assert.assertEquals(1, count); - org.apache.hadoop.hive.ql.TestTxnCommands2.runWorker(conf); + TestTxnCommands2.runWorker(conf); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE where CQ_DATABASE='temp' and CQ_TABLE='t12p' and CQ_PARTITION='ds=tomorrow/hour=2' and CQ_STATE='r' and CQ_TYPE='i'"); Assert.assertEquals(1, count); - org.apache.hadoop.hive.ql.TestTxnCommands2.runCleaner(conf); + TestTxnCommands2.runCleaner(conf); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE where CQ_DATABASE='temp' and CQ_TABLE='t12p'"); Assert.assertEquals(0, count); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPLETED_COMPACTIONS where CC_DATABASE='temp' and CC_TABLE='t12p' and CC_STATE='s' and CC_TYPE='i'"); @@ -576,7 +579,7 @@ public void testMetastoreTablesCleanup() throws Exception { checkCmdOnDriver(cpr); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE where CQ_DATABASE='temp' and CQ_TABLE='t11' and CQ_STATE='i' and CQ_TYPE='a'"); Assert.assertEquals(1, count); - org.apache.hadoop.hive.ql.TestTxnCommands2.runWorker(conf); // will fail + TestTxnCommands2.runWorker(conf); // will fail count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE where CQ_DATABASE='temp' and CQ_TABLE='t11' and CQ_STATE='i' and CQ_TYPE='a'"); Assert.assertEquals(0, count); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPLETED_COMPACTIONS where CC_DATABASE='temp' and CC_TABLE='t11' and CC_STATE='f' and CC_TYPE='a'"); @@ -586,7 +589,7 @@ public void testMetastoreTablesCleanup() throws Exception { checkCmdOnDriver(cpr); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE where CQ_DATABASE='temp' and CQ_TABLE='t12p' and CQ_PARTITION='ds=tomorrow/hour=2' and CQ_STATE='i' and CQ_TYPE='a'"); Assert.assertEquals(1, count); - org.apache.hadoop.hive.ql.TestTxnCommands2.runWorker(conf); // will fail + TestTxnCommands2.runWorker(conf); // will fail count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE where CQ_DATABASE='temp' and CQ_TABLE='t12p' and CQ_PARTITION='ds=tomorrow/hour=2' and CQ_STATE='i' and CQ_TYPE='a'"); Assert.assertEquals(0, count); count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPLETED_COMPACTIONS where CC_DATABASE='temp' and CC_TABLE='t12p' and CC_STATE='f' and CC_TYPE='a'"); @@ -824,7 +827,7 @@ public static ShowLocksResponseElement checkLock(LockType expectedType, LockStat * the TxnManager instance in the session (hacky but nothing is actually threading so it allows us * to write good tests) */ - private static HiveTxnManager swapTxnManager(HiveTxnManager txnMgr) { + public static HiveTxnManager swapTxnManager(HiveTxnManager txnMgr) { return SessionState.get().setTxnMgr(txnMgr); } @Test @@ -2556,4 +2559,122 @@ public void testTruncate() throws Exception { Assert.assertEquals("Unexpected lock count", 1, locks.size()); checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "default", "T", null, locks); } + @Test + public void testCleaner() throws Exception { + dropTable(new String[] {"T"}); + CommandProcessorResponse cpr = driver.run("create table T (a int, b int) stored as" + + " orc tblproperties('transactional'='true')"); + checkCmdOnDriver(cpr); + checkCmdOnDriver(driver.run("insert into T values(0,2),(1,4)")); + checkCmdOnDriver(driver.run("start transaction")); + checkCmdOnDriver(driver.run("insert into T values(0,2),(1,4)")); + + HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf); + swapTxnManager(txnMgr2); + checkCmdOnDriver(driver2.run("start transaction")); + checkCmdOnDriver(driver2.run("select * from T")); + + swapTxnManager(txnMgr); + checkCmdOnDriver(driver.run("commit")); + checkCmdOnDriver(driver.run("alter table T compact 'minor'")); + TestTxnCommands2.runWorker(conf); + + //so now we should have d1,d2,d1_2 + /* Now we should have d1_d2 from minor compaction and d1 from 1st txn by 'driver' and + d2 from 2nd txn by 'driver' - both committed at this point. + At this point, txn doint 'select * from T' from 'driver2' is still open and it sees the txn + that created d2 as open, so cleaner should not remove d2 + [ + target/warehouse/t/ + ├── delete_delta_0000001_0000002 + │   ├── _orc_acid_version + │   └── bucket_00000 + ├── delta_0000001_0000001_0000 + │   ├── _orc_acid_version + │   └── bucket_00000 + ├── delta_0000001_0000002 + │   ├── _orc_acid_version + │   └── bucket_00000 + └── delta_0000002_0000002_0000 + ├── _orc_acid_version + └── bucket_00000 + */ + TestTxnCommands2.runCleaner(conf); +/* +target/warehouse/t/ +├── delete_delta_0000001_0000002 +│   ├── _orc_acid_version +│   └── bucket_00000 +├── delta_0000001_0000002 +│   ├── _orc_acid_version +│   └── bucket_00000 +└── delta_0000002_0000002_0000 + ├── _orc_acid_version + └── bucket_00000 + */ + System.currentTimeMillis(); + //todo: the state is correct - add checks that the right files are there + + //alternatively, make the 2nd txn be a delete and verify that the reader doesn't see it in which case you'd have to do a major compaction + } + @Test + public void testCleaner2() throws Exception { + dropTable(new String[] {"T"}); + CommandProcessorResponse cpr = driver.run("create table T (a int, b int) stored as" + + " orc tblproperties('transactional'='true')"); + checkCmdOnDriver(cpr); + checkCmdOnDriver(driver.run("insert into T values(0,2)"));//makes delta_1_1 in T1 + checkCmdOnDriver(driver.run("insert into T values(1,4)"));//makes delta_2_2 in T2 + + HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf); + swapTxnManager(txnMgr2); + checkCmdOnDriver(driver2.run("start transaction"));//T3 + checkCmdOnDriver(driver2.run("select INPUT__FILE__NAME from T")); + List rs = new ArrayList<>(); + driver2.getResults(rs); + /* this select sees + target/warehouse/t/ + ├── delta_0000001_0000002 + │   ├── _orc_acid_version + │   └── bucket_00000 + └── delta_0000002_0000002_0000 + ├── _orc_acid_version + └── bucket_00000*/ + swapTxnManager(txnMgr); + checkCmdOnDriver(driver.run("alter table T compact 'minor'"));//T4 + TestTxnCommands2.runWorker(conf);//makes delta_1_2 & delete_delta_1_2 + /* Now we should have + target/warehouse/t/ + ├── delete_delta_0000001_0000002 + │   ├── _orc_acid_version + │   └── bucket_00000 + ├── delta_0000001_0000001_0000 + │   ├── _orc_acid_version + │   └── bucket_00000 + ├── delta_0000001_0000002 + │   ├── _orc_acid_version + │   └── bucket_00000 + └── delta_0000002_0000002_0000 + ├── _orc_acid_version + └── bucket_00000 + */ + TestTxnCommands2.runCleaner(conf); + /* Now the state should be + target/warehouse/t/ + ├── delete_delta_0000001_0000002 + │   ├── _orc_acid_version + │   └── bucket_00000 + ├── delta_0000001_0000002 + │   ├── _orc_acid_version + │   └── bucket_00000 + └── delta_0000002_0000002_0000 + ├── _orc_acid_version + └── bucket_00000 + The cleaner should not delete delta_2_2 since the txn running 'select * from T' may still be + reading this file (assuming this select statement is still running)*/ + System.currentTimeMillis(); + //todo: the state is correct - add checks that the right files are there + + //alternatively, make the 2nd txn be a delete and verify that the reader doesn't see it in which case you'd have to do a major compaction + } } diff --git ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java index ce574b4ac4..467851a590 100644 --- ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java +++ ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java @@ -18,34 +18,20 @@ package org.apache.hadoop.hive.ql.txn.compactor; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.CompactionRequest; import org.apache.hadoop.hive.metastore.api.CompactionType; -import org.apache.hadoop.hive.metastore.api.DataOperationType; -import org.apache.hadoop.hive.metastore.api.LockComponent; -import org.apache.hadoop.hive.metastore.api.LockLevel; -import org.apache.hadoop.hive.metastore.api.LockRequest; -import org.apache.hadoop.hive.metastore.api.LockResponse; -import org.apache.hadoop.hive.metastore.api.LockType; -import org.apache.hadoop.hive.metastore.api.OpenTxnRequest; -import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse; 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.api.UnlockRequest; import org.apache.hadoop.hive.metastore.txn.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.junit.After; import org.junit.Assert; import org.junit.Test; -import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; /** * Tests for the compactor Cleaner thread @@ -197,231 +183,6 @@ public void cleanupAfterMinorPartitionCompaction() throws Exception { Assert.assertTrue(sawDelta); } - @Test - public void blockedByLockTable() throws Exception { - Table t = newTable("default", "bblt", false); - - addBaseFile(t, null, 20L, 20); - addDeltaFile(t, null, 21L, 22L, 2); - addDeltaFile(t, null, 23L, 24L, 2); - addDeltaFile(t, null, 21L, 24L, 4); - - burnThroughTransactions("default", "bblt", 25); - - CompactionRequest rqst = new CompactionRequest("default", "bblt", CompactionType.MINOR); - txnHandler.compact(rqst); - CompactionInfo ci = txnHandler.findNextToCompact("fred"); - txnHandler.markCompacted(ci); - txnHandler.setRunAs(ci.id, System.getProperty("user.name")); - - LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "default"); - comp.setTablename("bblt"); - comp.setOperationType(DataOperationType.SELECT); - List components = new ArrayList(1); - components.add(comp); - LockRequest req = new LockRequest(components, "me", "localhost"); - LockResponse res = txnHandler.lock(req); - - startCleaner(); - - // Check there are no compactions requests left. - ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); - List compacts = rsp.getCompacts(); - Assert.assertEquals(1, compacts.size()); - Assert.assertEquals("ready for cleaning", compacts.get(0).getState()); - Assert.assertEquals("bblt", compacts.get(0).getTablename()); - Assert.assertEquals(CompactionType.MINOR, compacts.get(0).getType()); - } - - @Test - public void blockedByLockPartition() throws Exception { - Table t = newTable("default", "bblp", true); - Partition p = newPartition(t, "today"); - - addBaseFile(t, p, 20L, 20); - addDeltaFile(t, p, 21L, 22L, 2); - addDeltaFile(t, p, 23L, 24L, 2); - addDeltaFile(t, p, 21L, 24L, 4); - - burnThroughTransactions("default", "bblp", 25); - - CompactionRequest rqst = new CompactionRequest("default", "bblp", CompactionType.MINOR); - rqst.setPartitionname("ds=today"); - txnHandler.compact(rqst); - CompactionInfo ci = txnHandler.findNextToCompact("fred"); - txnHandler.markCompacted(ci); - txnHandler.setRunAs(ci.id, System.getProperty("user.name")); - - LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "default"); - comp.setTablename("bblp"); - comp.setPartitionname("ds=today"); - comp.setOperationType(DataOperationType.DELETE); - List components = new ArrayList(1); - components.add(comp); - LockRequest req = new LockRequest(components, "me", "localhost"); - OpenTxnsResponse resp = txnHandler.openTxns(new OpenTxnRequest(1, "Dracula", "Transylvania")); - req.setTxnid(resp.getTxn_ids().get(0)); - LockResponse res = txnHandler.lock(req); - - startCleaner(); - - // Check there are no compactions requests left. - ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); - List compacts = rsp.getCompacts(); - Assert.assertEquals(1, compacts.size()); - Assert.assertEquals("ready for cleaning", compacts.get(0).getState()); - Assert.assertEquals("bblp", compacts.get(0).getTablename()); - Assert.assertEquals("ds=today", compacts.get(0).getPartitionname()); - Assert.assertEquals(CompactionType.MINOR, compacts.get(0).getType()); - } - - @Test - public void notBlockedBySubsequentLock() throws Exception { - Table t = newTable("default", "bblt", false); - - // Set the run frequency low on this test so it doesn't take long - conf.setTimeVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL, 100, - TimeUnit.MILLISECONDS); - - addBaseFile(t, null, 20L, 20); - addDeltaFile(t, null, 21L, 22L, 2); - addDeltaFile(t, null, 23L, 24L, 2); - addDeltaFile(t, null, 21L, 24L, 4); - - burnThroughTransactions("default", "bblt", 25); - - CompactionRequest rqst = new CompactionRequest("default", "bblt", CompactionType.MINOR); - txnHandler.compact(rqst); - CompactionInfo ci = txnHandler.findNextToCompact("fred"); - txnHandler.markCompacted(ci); - txnHandler.setRunAs(ci.id, System.getProperty("user.name")); - - LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "default"); - comp.setTablename("bblt"); - comp.setOperationType(DataOperationType.INSERT); - List components = new ArrayList(1); - components.add(comp); - LockRequest req = new LockRequest(components, "me", "localhost"); - LockResponse res = txnHandler.lock(req); - - AtomicBoolean looped = new AtomicBoolean(); - looped.set(false); - startCleaner(looped); - - // Make sure the compactor has a chance to run once - while (!looped.get()) { - Thread.currentThread().sleep(100); - } - - // There should still be one request, as the locks still held. - ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); - List compacts = rsp.getCompacts(); - Assert.assertEquals(1, compacts.size()); - - // obtain a second lock. This shouldn't block cleaner as it was acquired after the initial - // clean request - LockComponent comp2 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "default"); - comp2.setTablename("bblt"); - comp.setOperationType(DataOperationType.SELECT); - List components2 = new ArrayList(1); - components2.add(comp2); - LockRequest req2 = new LockRequest(components, "me", "localhost"); - LockResponse res2 = txnHandler.lock(req2); - - // Unlock the previous lock - txnHandler.unlock(new UnlockRequest(res.getLockid())); - looped.set(false); - - while (!looped.get()) { - Thread.currentThread().sleep(100); - } - stopThread(); - Thread.currentThread().sleep(200); - - - // Check there are no compactions requests left. - rsp = txnHandler.showCompact(new ShowCompactRequest()); - compacts = rsp.getCompacts(); - Assert.assertEquals(1, compacts.size()); - Assert.assertTrue(TxnStore.SUCCEEDED_RESPONSE.equals(rsp.getCompacts().get(0).getState())); - } - - @Test - public void partitionNotBlockedBySubsequentLock() throws Exception { - Table t = newTable("default", "bblt", true); - Partition p = newPartition(t, "today"); - - // Set the run frequency low on this test so it doesn't take long - conf.setTimeVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL, 100, - TimeUnit.MILLISECONDS); - - addBaseFile(t, p, 20L, 20); - addDeltaFile(t, p, 21L, 22L, 2); - addDeltaFile(t, p, 23L, 24L, 2); - addDeltaFile(t, p, 21L, 24L, 4); - - burnThroughTransactions("default", "bblt", 25); - - CompactionRequest rqst = new CompactionRequest("default", "bblt", CompactionType.MINOR); - rqst.setPartitionname("ds=today"); - txnHandler.compact(rqst); - CompactionInfo ci = txnHandler.findNextToCompact("fred"); - txnHandler.markCompacted(ci); - txnHandler.setRunAs(ci.id, System.getProperty("user.name")); - - LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.PARTITION, "default"); - comp.setTablename("bblt"); - comp.setPartitionname("ds=today"); - comp.setOperationType(DataOperationType.INSERT); - List components = new ArrayList(1); - components.add(comp); - LockRequest req = new LockRequest(components, "me", "localhost"); - LockResponse res = txnHandler.lock(req); - - AtomicBoolean looped = new AtomicBoolean(); - looped.set(false); - startCleaner(looped); - - // Make sure the compactor has a chance to run once - while (!looped.get()) { - Thread.currentThread().sleep(100); - } - - // There should still be one request, as the locks still held. - ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); - List compacts = rsp.getCompacts(); - Assert.assertEquals(1, compacts.size()); - - - // obtain a second lock. This shouldn't block cleaner as it was acquired after the initial - // clean request - LockComponent comp2 = new LockComponent(LockType.SHARED_READ, LockLevel.PARTITION, "default"); - comp2.setTablename("bblt"); - comp2.setPartitionname("ds=today"); - comp.setOperationType(DataOperationType.SELECT); - List components2 = new ArrayList(1); - components2.add(comp2); - LockRequest req2 = new LockRequest(components, "me", "localhost"); - LockResponse res2 = txnHandler.lock(req2); - - // Unlock the previous lock - txnHandler.unlock(new UnlockRequest(res.getLockid())); - looped.set(false); - - while (!looped.get()) { - Thread.currentThread().sleep(100); - } - stopThread(); - Thread.currentThread().sleep(200); - - - // Check there are no compactions requests left. - rsp = txnHandler.showCompact(new ShowCompactRequest()); - compacts = rsp.getCompacts(); - Assert.assertEquals(1, compacts.size()); - Assert.assertTrue(TxnStore.SUCCEEDED_RESPONSE.equals(rsp.getCompacts().get(0).getState())); - } - @Test public void cleanupAfterMajorPartitionCompactionNoBase() throws Exception { Table t = newTable("default", "campcnb", true); diff --git ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java index 488cd903c8..1f0906424b 100644 --- ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java +++ ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java @@ -300,7 +300,7 @@ public void minorTableWithBase() throws Exception { // Find the new delta file and make sure it has the right contents boolean sawNewDelta = false; for (int i = 0; i < stat.length; i++) { - if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(21, 24))) { + if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(21, 24) + "_v0000026")) { sawNewDelta = true; FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter); Assert.assertEquals(2, buckets.length); @@ -309,7 +309,7 @@ public void minorTableWithBase() throws Exception { Assert.assertEquals(104L, buckets[0].getLen()); Assert.assertEquals(104L, buckets[1].getLen()); } - if (stat[i].getPath().getName().equals(makeDeleteDeltaDirNameCompacted(21, 24))) { + if (stat[i].getPath().getName().equals(makeDeleteDeltaDirNameCompacted(21, 24) + "_v0000026")) { sawNewDelta = true; FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter); Assert.assertEquals(2, buckets.length); @@ -322,7 +322,7 @@ public void minorTableWithBase() throws Exception { LOG.debug("This is not the delta file you are looking for " + stat[i].getPath().getName()); } } - Assert.assertTrue(sawNewDelta); + Assert.assertTrue(toString(stat), sawNewDelta); } /** @@ -354,7 +354,7 @@ public void minorWithOpenInMiddle() throws Exception { // There should still now be 5 directories in the location FileSystem fs = FileSystem.get(conf); FileStatus[] stat = fs.listStatus(new Path(t.getSd().getLocation())); - Assert.assertEquals(5, stat.length); + Assert.assertEquals(toString(stat),5 , stat.length); // Find the new delta file and make sure it has the right contents Arrays.sort(stat); @@ -395,9 +395,9 @@ public void minorWithAborted() throws Exception { // Find the new delta file and make sure it has the right contents Arrays.sort(stat); Assert.assertEquals("base_20", stat[0].getPath().getName()); - Assert.assertEquals(makeDeleteDeltaDirNameCompacted(21, 27), stat[1].getPath().getName()); + Assert.assertEquals(makeDeleteDeltaDirNameCompacted(21, 27) + "_v0000028", stat[1].getPath().getName()); Assert.assertEquals(makeDeltaDirName(21, 22), stat[2].getPath().getName()); - Assert.assertEquals(makeDeltaDirNameCompacted(21, 27), stat[3].getPath().getName()); + Assert.assertEquals(makeDeltaDirNameCompacted(21, 27) + "_v0000028", stat[3].getPath().getName()); Assert.assertEquals(makeDeltaDirName(23, 25), stat[4].getPath().getName()); Assert.assertEquals(makeDeltaDirName(26, 27), stat[5].getPath().getName()); } @@ -432,7 +432,7 @@ public void minorPartitionWithBase() throws Exception { // Find the new delta file and make sure it has the right contents boolean sawNewDelta = false; for (int i = 0; i < stat.length; i++) { - if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(21, 24))) { + if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(21, 24) + "_v0000026")) { sawNewDelta = true; FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter); Assert.assertEquals(2, buckets.length); @@ -453,7 +453,7 @@ public void minorPartitionWithBase() throws Exception { LOG.debug("This is not the delta file you are looking for " + stat[i].getPath().getName()); } } - Assert.assertTrue(sawNewDelta); + Assert.assertTrue(toString(stat), sawNewDelta); } @Test @@ -484,7 +484,7 @@ public void minorTableNoBase() throws Exception { // Find the new delta file and make sure it has the right contents boolean sawNewDelta = false; for (int i = 0; i < stat.length; i++) { - if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(1, 4))) { + if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(1, 4) + "_v0000006")) { sawNewDelta = true; FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter); Assert.assertEquals(2, buckets.length); @@ -493,7 +493,7 @@ public void minorTableNoBase() throws Exception { Assert.assertEquals(104L, buckets[0].getLen()); Assert.assertEquals(104L, buckets[1].getLen()); } - if (stat[i].getPath().getName().equals(makeDeleteDeltaDirNameCompacted(1, 4))) { + if (stat[i].getPath().getName().equals(makeDeleteDeltaDirNameCompacted(1, 4) + "_v0000006")) { sawNewDelta = true; FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter); Assert.assertEquals(2, buckets.length); @@ -505,7 +505,7 @@ public void minorTableNoBase() throws Exception { LOG.debug("This is not the delta file you are looking for " + stat[i].getPath().getName()); } } - Assert.assertTrue(sawNewDelta); + Assert.assertTrue(toString(stat), sawNewDelta); } @Test @@ -537,7 +537,7 @@ public void majorTableWithBase() throws Exception { // Find the new delta file and make sure it has the right contents boolean sawNewBase = false; for (int i = 0; i < stat.length; i++) { - if (stat[i].getPath().getName().equals("base_0000024")) { + if (stat[i].getPath().getName().equals("base_0000024_v0000026")) { sawNewBase = true; FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER); Assert.assertEquals(2, buckets.length); @@ -549,7 +549,7 @@ public void majorTableWithBase() throws Exception { LOG.debug("This is not the file you are looking for " + stat[i].getPath().getName()); } } - Assert.assertTrue(sawNewBase); + Assert.assertTrue(toString(stat), sawNewBase); } @Test @@ -612,68 +612,37 @@ private void compactNoBaseLotsOfDeltas(CompactionType type) throws Exception { Assert.assertEquals(numFilesExpected, stat.length); // Find the new delta file and make sure it has the right contents - BitSet matchesFound = new BitSet(numFilesExpected); - for (int i = 0, j = 0; i < stat.length; i++) { - if(stat[i].getPath().getName().equals(makeDeleteDeltaDirNameCompacted(21,23))) { - matchesFound.set(j++); - } - if(stat[i].getPath().getName().equals(makeDeleteDeltaDirNameCompacted(25,33))) { - matchesFound.set(j++); - } - if(stat[i].getPath().getName().equals(makeDeltaDirName(21,21))) { - matchesFound.set(j++); - } - else if(stat[i].getPath().getName().equals(makeDeltaDirName(23, 23))) { - matchesFound.set(j++); - } - else if(stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(25, 29))) { - matchesFound.set(j++); - } - else if(stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(31, 32))) { - matchesFound.set(j++); - } - else if(stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(31, 33))) { - matchesFound.set(j++); - } - else if(stat[i].getPath().getName().equals(makeDeltaDirName(35, 35))) { - matchesFound.set(j++); - } - else if(stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(21,23))) { - matchesFound.set(j++); - } - else if(stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(25,33))) { - matchesFound.set(j++); - } - switch (type) { - case MINOR: - if(stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(21,35))) { - matchesFound.set(j++); - } - if (stat[i].getPath().getName().equals(makeDeleteDeltaDirNameCompacted(21, 35))) { - matchesFound.set(j++); - } - break; - case MAJOR: - if(stat[i].getPath().getName().equals(AcidUtils.baseDir(35))) { - matchesFound.set(j++); - } - break; - default: - throw new IllegalStateException(); - } + List matchesNotFound = new ArrayList<>(numFilesExpected); + matchesNotFound.add(makeDeleteDeltaDirNameCompacted(21,23) + "_v\\d+"); + matchesNotFound.add(makeDeleteDeltaDirNameCompacted(25,33) + "_v\\d+"); + matchesNotFound.add(makeDeltaDirName(21,21)); + matchesNotFound.add(makeDeltaDirName(23, 23)); + matchesNotFound.add(makeDeltaDirNameCompacted(25, 29));//streaming ingest + matchesNotFound.add(makeDeltaDirNameCompacted(31, 32));//streaming ingest + //todo: this should have some _vXXXX suffix but addDeltaFile() doesn't support it + matchesNotFound.add(makeDeltaDirNameCompacted(31, 33)); + matchesNotFound.add(makeDeltaDirName(35, 35)); + matchesNotFound.add(makeDeltaDirNameCompacted(21,23) + "_v\\d+"); + matchesNotFound.add(makeDeltaDirNameCompacted(25,33) + "_v\\d+"); + if(type == CompactionType.MINOR) { + matchesNotFound.add(makeDeltaDirNameCompacted(21,35) + "_v\\d+"); + matchesNotFound.add(makeDeleteDeltaDirNameCompacted(21, 35) + "_v\\d+"); } - StringBuilder sb = null; - for(int i = 0; i < stat.length; i++) { - if(!matchesFound.get(i)) { - if(sb == null) { - sb = new StringBuilder("Some files are missing at index: "); + if(type == CompactionType.MAJOR) { + matchesNotFound.add(AcidUtils.baseDir(35) + "_v\\d+"); + } + for(FileStatus f : stat) { + for(int j = 0; j < matchesNotFound.size(); j++) { + if (f.getPath().getName().matches(matchesNotFound.get(j))) { + matchesNotFound.remove(j); + break; } - sb.append(i).append(","); } } - if (sb != null) { - Assert.assertTrue(sb.toString(), false); + if(matchesNotFound.size() == 0) { + return; } + Assert.assertTrue("Files remaining: " + matchesNotFound + "; " + toString(stat), false); } @Test public void majorPartitionWithBase() throws Exception { @@ -706,7 +675,7 @@ public void majorPartitionWithBase() throws Exception { // Find the new delta file and make sure it has the right contents boolean sawNewBase = false; for (int i = 0; i < stat.length; i++) { - if (stat[i].getPath().getName().equals("base_0000024")) { + if (stat[i].getPath().getName().equals("base_0000024_v0000026")) { sawNewBase = true; FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER); Assert.assertEquals(2, buckets.length); @@ -718,7 +687,7 @@ public void majorPartitionWithBase() throws Exception { LOG.debug("This is not the file you are looking for " + stat[i].getPath().getName()); } } - Assert.assertTrue(sawNewBase); + Assert.assertTrue(toString(stat), sawNewBase); } @Test @@ -749,7 +718,7 @@ public void majorTableNoBase() throws Exception { // Find the new delta file and make sure it has the right contents boolean sawNewBase = false; for (int i = 0; i < stat.length; i++) { - if (stat[i].getPath().getName().equals("base_0000004")) { + if (stat[i].getPath().getName().equals("base_0000004_v0000005")) { sawNewBase = true; FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER); Assert.assertEquals(2, buckets.length); @@ -761,9 +730,20 @@ public void majorTableNoBase() throws Exception { LOG.debug("This is not the file you are looking for " + stat[i].getPath().getName()); } } - Assert.assertTrue(sawNewBase); + Assert.assertTrue(toString(stat), sawNewBase); } + private static String toString(FileStatus[] stat) { + StringBuilder sb = new StringBuilder("stat{"); + if(stat == null) { + return sb.toString(); + } + for(FileStatus f : stat) { + sb.append(f.getPath()).append(","); + } + sb.setCharAt(sb.length() - 1, '}'); + return sb.toString(); + } @Test public void majorTableLegacy() throws Exception { LOG.debug("Starting majorTableLegacy"); @@ -793,7 +773,7 @@ public void majorTableLegacy() throws Exception { // Find the new delta file and make sure it has the right contents boolean sawNewBase = false; for (int i = 0; i < stat.length; i++) { - if (stat[i].getPath().getName().equals("base_0000024")) { + if (stat[i].getPath().getName().equals("base_0000024_v0000026")) { sawNewBase = true; FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER); Assert.assertEquals(2, buckets.length); @@ -805,7 +785,7 @@ public void majorTableLegacy() throws Exception { LOG.debug("This is not the file you are looking for " + stat[i].getPath().getName()); } } - Assert.assertTrue(sawNewBase); + Assert.assertTrue(toString(stat), sawNewBase); } @Test @@ -836,7 +816,7 @@ public void minorTableLegacy() throws Exception { // Find the new delta file and make sure it has the right contents boolean sawNewDelta = false; for (int i = 0; i < stat.length; i++) { - if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(21, 24))) { + if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(21, 24) + "_v0000026")) { sawNewDelta = true; FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter); Assert.assertEquals(2, buckets.length); @@ -846,7 +826,7 @@ public void minorTableLegacy() throws Exception { LOG.debug("This is not the file you are looking for " + stat[i].getPath().getName()); } } - Assert.assertTrue(sawNewDelta); + Assert.assertTrue(toString(stat), sawNewDelta); } @Test @@ -881,7 +861,7 @@ public void majorPartitionWithBaseMissingBuckets() throws Exception { // Find the new delta file and make sure it has the right contents boolean sawNewBase = false; for (int i = 0; i < stat.length; i++) { - if (stat[i].getPath().getName().equals("base_0000026")) { + if (stat[i].getPath().getName().equals("base_0000026_v0000028")) { sawNewBase = true; FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER); Assert.assertEquals(2, buckets.length); @@ -901,7 +881,7 @@ public void majorPartitionWithBaseMissingBuckets() throws Exception { LOG.debug("This is not the file you are looking for " + stat[i].getPath().getName()); } } - Assert.assertTrue(sawNewBase); + Assert.assertTrue(toString(stat), sawNewBase); } @Test @@ -933,7 +913,7 @@ public void majorWithOpenInMiddle() throws Exception { // Find the new delta file and make sure it has the right contents Arrays.sort(stat); - Assert.assertEquals("base_0000022", stat[0].getPath().getName()); + Assert.assertEquals("base_0000022_v0000028", stat[0].getPath().getName()); Assert.assertEquals("base_20", stat[1].getPath().getName()); Assert.assertEquals(makeDeltaDirName(21, 22), stat[2].getPath().getName()); Assert.assertEquals(makeDeltaDirName(23, 25), stat[3].getPath().getName()); @@ -969,7 +949,7 @@ public void majorWithAborted() throws Exception { // Find the new delta file and make sure it has the right contents Arrays.sort(stat); - Assert.assertEquals("base_0000027", stat[0].getPath().getName()); + Assert.assertEquals("base_0000027_v0000028", stat[0].getPath().getName()); Assert.assertEquals("base_20", stat[1].getPath().getName()); Assert.assertEquals(makeDeltaDirName(21, 22), stat[2].getPath().getName()); Assert.assertEquals(makeDeltaDirName(23, 25), stat[3].getPath().getName()); diff --git standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnCommonUtils.java standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnCommonUtils.java index c61a997612..43cc805943 100644 --- standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnCommonUtils.java +++ standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnCommonUtils.java @@ -89,7 +89,7 @@ public static ValidTxnList createValidReadTxnList(GetOpenTxnsResponse txns, long * {@link org.apache.hadoop.hive.common.ValidTxnWriteIdList}. This assumes that the caller intends to * read the files, and thus treats both open and aborted transactions as invalid. * @param currentTxnId current txn ID for which we get the valid write ids list - * @param list valid write ids list from the metastore + * @param validIds valid write ids list from the metastore * @return a valid write IDs list for the whole transaction. */ public static ValidTxnWriteIdList createValidTxnWriteIdList(Long currentTxnId, diff --git standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java index cbb76d51d6..f33c299dcc 100644 --- standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java +++ standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hive.metastore.txn; -import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.common.classification.RetrySemantics; import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.MetaException; @@ -324,6 +323,54 @@ public void markCompacted(CompactionInfo info) throws MetaException { return findReadyToClean(); } } + @Override + public long findMinOpenTxnGLB() throws MetaException { + Connection dbConn = null; + Statement stmt = null; + ResultSet rs = null; + try { + try { + dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); + stmt = dbConn.createStatement(); + return findMinOpenTxnGLB(stmt); + } catch (SQLException e) { + LOG.error("Unable to findMinOpenTxnGLB() due to:" + e.getMessage()); + rollbackDBConn(dbConn); + checkRetryable(dbConn, e, "findMinOpenTxnGLB"); + throw new MetaException("Unable to execute findMinOpenTxnGLB() " + + StringUtils.stringifyException(e)); + } finally { + close(rs, stmt, dbConn); + } + } catch (RetryException e) { + return findMinOpenTxnGLB(); + } + } + + /** + * See doc at {@link TxnStore#findMinOpenTxnGLB()} + */ + private long findMinOpenTxnGLB(Statement stmt) throws MetaException, SQLException { + String s = "select ntxn_next from NEXT_TXN_ID"; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + if (!rs.next()) { + throw new MetaException("Transaction tables not properly " + + "initialized, no record found in next_txn_id"); + } + long hwm = rs.getLong(1); + s = "select min(mhl_min_open_txnid) from MIN_HISTORY_LEVEL"; + LOG.debug("Going to execute query <" + s + ">"); + rs = stmt.executeQuery(s); + rs.next(); + long minOpenTxnId = rs.getLong(1); + if(rs.wasNull()) { + return hwm; + } + //since generating new txnid uses select for update on single row in NEXT_TXN_ID + assert hwm >= minOpenTxnId : "(hwm, minOpenTxnId)=(" + hwm + "," + minOpenTxnId + ")"; + return minOpenTxnId; + } /** * This will remove an entry from the queue after @@ -387,11 +434,16 @@ public void markCleaned(CompactionInfo info) throws MetaException { pStmt.setLong(paramCount++, info.highestWriteId); } LOG.debug("Going to execute update <" + s + ">"); - if (pStmt.executeUpdate() < 1) { + if ((updCount = pStmt.executeUpdate()) < 1) { LOG.error("Expected to remove at least one row from completed_txn_components when " + "marking compaction entry as clean!"); } - + /** + * 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 + * aborted TXN_COMPONENTS above tc_writeid (and consequently about aborted txns). + * See {@link ql.txn.compactor.Cleaner.removeFiles()} + */ s = "select distinct txn_id from TXNS, TXN_COMPONENTS where txn_id = tc_txnid and txn_state = '" + TXN_ABORTED + "' and tc_database = ? and tc_table = ?"; if (info.highestWriteId != 0) s += " and tc_writeid <= ?"; @@ -480,7 +532,6 @@ public void markCleaned(CompactionInfo info) throws MetaException { markCleaned(info); } } - /** * Clean up entries from TXN_TO_WRITE_ID table less than min_uncommited_txnid as found by * min(NEXT_TXN_ID.ntxn_next, min(MIN_HISTORY_LEVEL.mhl_min_open_txnid), min(Aborted TXNS.txn_id)). @@ -502,30 +553,11 @@ public void cleanTxnToWriteIdTable() throws MetaException { // First need to find the min_uncommitted_txnid which is currently seen by any open transactions. // If there are no txns which are currently open or aborted in the system, then current value of // NEXT_TXN_ID.ntxn_next could be min_uncommitted_txnid. - String s = "select ntxn_next from NEXT_TXN_ID"; - LOG.debug("Going to execute query <" + s + ">"); - rs = stmt.executeQuery(s); - if (!rs.next()) { - throw new MetaException("Transaction tables not properly " + - "initialized, no record found in next_txn_id"); - } - long minUncommittedTxnId = rs.getLong(1); - - // If there are any open txns, then the minimum of min_open_txnid from MIN_HISTORY_LEVEL table - // could be the min_uncommitted_txnid if lesser than NEXT_TXN_ID.ntxn_next. - s = "select min(mhl_min_open_txnid) from MIN_HISTORY_LEVEL"; - LOG.debug("Going to execute query <" + s + ">"); - rs = stmt.executeQuery(s); - if (rs.next()) { - long minOpenTxnId = rs.getLong(1); - if (minOpenTxnId > 0) { - minUncommittedTxnId = Math.min(minOpenTxnId, minUncommittedTxnId); - } - } + long minUncommittedTxnId = findMinOpenTxnGLB(stmt); // If there are aborted txns, then the minimum aborted txnid could be the min_uncommitted_txnid // if lesser than both NEXT_TXN_ID.ntxn_next and min(MIN_HISTORY_LEVEL .mhl_min_open_txnid). - s = "select min(txn_id) from TXNS where txn_state = " + quoteChar(TXN_ABORTED); + String s = "select min(txn_id) from TXNS where txn_state = " + quoteChar(TXN_ABORTED); LOG.debug("Going to execute query <" + s + ">"); rs = stmt.executeQuery(s); if (rs.next()) { @@ -534,7 +566,6 @@ public void cleanTxnToWriteIdTable() throws MetaException { minUncommittedTxnId = Math.min(minAbortedTxnId, minUncommittedTxnId); } } - // As all txns below min_uncommitted_txnid are either committed or empty_aborted, we are allowed // to cleanup the entries less than min_uncommitted_txnid from the TXN_TO_WRITE_ID table. s = "delete from TXN_TO_WRITE_ID where t2w_txnid < " + minUncommittedTxnId; diff --git standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java index 22ce007278..4af0473fb4 100644 --- standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java +++ standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java @@ -356,6 +356,16 @@ void onRename(String oldCatName, String oldDbName, String oldTabName, String old @RetrySemantics.ReadOnly List findReadyToClean() throws MetaException; + /** + * Returns the smallest txnid that could be seen in open state across all active transactions in + * the system or {@code NEXT_TXN_ID.NTXN_NEXT} if there are no active transactions, i.e. greatest + * lower bound of open transaction IDs. Even if a transaction is opened concurrently with this + * call it cannot have an id less than what this method returns. + * @return transaction ID + */ + @RetrySemantics.ReadOnly + long findMinOpenTxnGLB() throws MetaException; + /** * This will remove an entry from the queue after * it has been compacted. diff --git standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java index 3bb1f0c62c..653c474a0a 100644 --- standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java +++ standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java @@ -19,8 +19,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.common.ValidCompactorWriteIdList; +import org.apache.hadoop.hive.common.ValidReadTxnList; import org.apache.hadoop.hive.common.ValidReaderWriteIdList; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.metastore.TransactionalValidationListener; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.TableValidWriteIds; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; @@ -39,12 +42,28 @@ public class TxnUtils { private static final Logger LOG = LoggerFactory.getLogger(TxnUtils.class); - // Transactional stats states - static final public char STAT_OPEN = 'o'; - static final public char STAT_INVALID = 'i'; - static final public char STAT_COMMITTED = 'c'; - static final public char STAT_OBSOLETE = 's'; - + public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns, long minOpenTxnGLB) { + long highWaterMark = minOpenTxnGLB - 1; + long[] abortedTxns = new long[txns.getOpen_txnsSize()]; + BitSet abortedBits = BitSet.valueOf(txns.getAbortedBits()); + int i = 0; + for(long txnId : txns.getOpen_txns()) { + if(txnId > highWaterMark) { + break; + } + if(abortedBits.get(i)) { + abortedTxns[i] = txnId; + } + else { + assert false : JavaUtils.txnIdToString(txnId) + " is open and <= hwm:" + highWaterMark; + } + ++i; + } + abortedTxns = Arrays.copyOf(abortedTxns, i); + BitSet bitSet = new BitSet(abortedTxns.length); + bitSet.set(0, abortedTxns.length); + return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);//todo: add ValidCleanerTxnList? - could be problematic for all the places that read it from string as they'd have to know which object to instantiate + } /** * Transform a {@link org.apache.hadoop.hive.metastore.api.TableValidWriteIds} to a * {@link org.apache.hadoop.hive.common.ValidCompactorWriteIdList}. This assumes that the caller intends to @@ -83,17 +102,6 @@ public static ValidCompactorWriteIdList createValidCompactWriteIdList(TableValid } } - public static ValidReaderWriteIdList updateForCompactionQuery(ValidReaderWriteIdList ids) { - // This is based on the existing valid write ID list that was built for a select query; - // therefore we assume all the aborted txns, etc. were already accounted for. - // All we do is adjust the high watermark to only include contiguous txns. - Long minOpenWriteId = ids.getMinOpenWriteId(); - if (minOpenWriteId != null && minOpenWriteId != Long.MAX_VALUE) { - return ids.updateHighWatermark(ids.getMinOpenWriteId() - 1); - } - return ids; - } - /** * Get an instance of the TxnStore that is appropriate for this store * @param conf configuration diff --git standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java index 2c9d98b5fd..2a70ec3e55 100644 --- standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java +++ standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java @@ -186,63 +186,6 @@ public void testTxNWithKeyWrongPrefix() throws Exception { Assert.assertTrue(validTxns.isTxnValid(1)); } - @Test - public void testTxnRange() throws Exception { - ValidTxnList validTxns = client.getValidTxns(); - Assert.assertEquals(ValidTxnList.RangeResponse.NONE, - validTxns.isTxnRangeValid(1L, 3L)); - List tids = client.openTxns("me", 5).getTxn_ids(); - - HeartbeatTxnRangeResponse rsp = client.heartbeatTxnRange(1, 5); - Assert.assertEquals(0, rsp.getNosuch().size()); - Assert.assertEquals(0, rsp.getAborted().size()); - - client.rollbackTxn(1L); - client.commitTxn(2L); - client.commitTxn(3L); - client.commitTxn(4L); - validTxns = client.getValidTxns(); - System.out.println("validTxns = " + validTxns); - Assert.assertEquals(ValidTxnList.RangeResponse.ALL, - validTxns.isTxnRangeValid(2L, 2L)); - Assert.assertEquals(ValidTxnList.RangeResponse.ALL, - validTxns.isTxnRangeValid(2L, 3L)); - Assert.assertEquals(ValidTxnList.RangeResponse.ALL, - validTxns.isTxnRangeValid(2L, 4L)); - Assert.assertEquals(ValidTxnList.RangeResponse.ALL, - validTxns.isTxnRangeValid(3L, 4L)); - - Assert.assertEquals(ValidTxnList.RangeResponse.SOME, - validTxns.isTxnRangeValid(1L, 4L)); - Assert.assertEquals(ValidTxnList.RangeResponse.SOME, - validTxns.isTxnRangeValid(2L, 5L)); - Assert.assertEquals(ValidTxnList.RangeResponse.SOME, - validTxns.isTxnRangeValid(1L, 2L)); - Assert.assertEquals(ValidTxnList.RangeResponse.SOME, - validTxns.isTxnRangeValid(4L, 5L)); - - Assert.assertEquals(ValidTxnList.RangeResponse.NONE, - validTxns.isTxnRangeValid(1L, 1L)); - Assert.assertEquals(ValidTxnList.RangeResponse.NONE, - validTxns.isTxnRangeValid(5L, 10L)); - - validTxns = new ValidReadTxnList("10:5:4,5,6:"); - Assert.assertEquals(ValidTxnList.RangeResponse.NONE, - validTxns.isTxnRangeValid(4,6)); - Assert.assertEquals(ValidTxnList.RangeResponse.ALL, - validTxns.isTxnRangeValid(7, 10)); - Assert.assertEquals(ValidTxnList.RangeResponse.SOME, - validTxns.isTxnRangeValid(7, 11)); - Assert.assertEquals(ValidTxnList.RangeResponse.SOME, - validTxns.isTxnRangeValid(3, 6)); - Assert.assertEquals(ValidTxnList.RangeResponse.SOME, - validTxns.isTxnRangeValid(4, 7)); - Assert.assertEquals(ValidTxnList.RangeResponse.SOME, - validTxns.isTxnRangeValid(1, 12)); - Assert.assertEquals(ValidTxnList.RangeResponse.ALL, - validTxns.isTxnRangeValid(1, 3)); - } - @Test public void testLocks() throws Exception { LockRequestBuilder rqstBuilder = new LockRequestBuilder(); diff --git storage-api/src/java/org/apache/hadoop/hive/common/ValidCompactorWriteIdList.java storage-api/src/java/org/apache/hadoop/hive/common/ValidCompactorWriteIdList.java index a849264b8e..5d74241815 100644 --- storage-api/src/java/org/apache/hadoop/hive/common/ValidCompactorWriteIdList.java +++ storage-api/src/java/org/apache/hadoop/hive/common/ValidCompactorWriteIdList.java @@ -80,6 +80,9 @@ public ValidCompactorWriteIdList(String value) { /** * Returns org.apache.hadoop.hive.common.ValidWriteIdList.RangeResponse.ALL if all write ids in * the range are resolved and RangeResponse.NONE otherwise + * Streaming ingest may create something like delta_11_20. Compactor cannot include such delta in + * compaction until all transactions that write to it terminate. (Otherwise compactor + * will produce delta that doesn't satisfy (D1 intersect D2 is empty or D1 intersect D2 = D2). */ @Override public RangeResponse isWriteIdRangeValid(long minWriteId, long maxWriteId) { diff --git storage-api/src/java/org/apache/hadoop/hive/common/ValidReaderWriteIdList.java storage-api/src/java/org/apache/hadoop/hive/common/ValidReaderWriteIdList.java index 95a0b56e39..bc8ac0d61b 100644 --- storage-api/src/java/org/apache/hadoop/hive/common/ValidReaderWriteIdList.java +++ storage-api/src/java/org/apache/hadoop/hive/common/ValidReaderWriteIdList.java @@ -35,6 +35,12 @@ private long minOpenWriteId = Long.MAX_VALUE; protected long highWatermark; + /** + * This seems like a bad c'tor. It doesn't even have a table name in it and it's used every time + * ValidWriteIdList.VALID_WRITEIDS_KEY is not found in Configuration. + * But, if anything, that would indicate a bug if was done for an acid read since it + * considers everything valid - this should not be assumed. + */ public ValidReaderWriteIdList() { this(null, new long[0], new BitSet(), Long.MAX_VALUE, Long.MAX_VALUE); } diff --git streaming/src/test/org/apache/hive/streaming/TestStreaming.java streaming/src/test/org/apache/hive/streaming/TestStreaming.java index 50433b6243..19daf65b6f 100644 --- streaming/src/test/org/apache/hive/streaming/TestStreaming.java +++ streaming/src/test/org/apache/hive/streaming/TestStreaming.java @@ -34,6 +34,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -52,12 +53,16 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hive.cli.CliSessionState; import org.apache.hadoop.hive.common.JavaUtils; +import org.apache.hadoop.hive.common.TableName; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.Validator; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse; +import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest; import org.apache.hadoop.hive.metastore.api.LockState; import org.apache.hadoop.hive.metastore.api.LockType; import org.apache.hadoop.hive.metastore.api.MetaException; @@ -66,10 +71,12 @@ import org.apache.hadoop.hive.metastore.api.ShowLocksRequest; import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement; +import org.apache.hadoop.hive.metastore.api.TableValidWriteIds; import org.apache.hadoop.hive.metastore.api.TxnAbortedException; import org.apache.hadoop.hive.metastore.api.TxnInfo; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; @@ -412,13 +419,13 @@ public void testNoBuckets() throws Exception { rs = queryTable(driver, "select ROW__ID, a, b, INPUT__FILE__NAME from default.streamingnobuckets order by ROW__ID"); Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\tfoo\tbar")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/base_0000005/bucket_00000")); + Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\ta3\tb4")); - Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/base_0000005/bucket_00000")); + Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\ta5\tb6")); - Assert.assertTrue(rs.get(2), rs.get(2).endsWith("streamingnobuckets/base_0000005/bucket_00000")); + Assert.assertTrue(rs.get(2), rs.get(2).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":4,\"bucketid\":536870912,\"rowid\":0}\t0\t0")); - Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/base_0000005/bucket_00000")); + Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); } @Test @@ -762,17 +769,17 @@ public void testAutoRollTransactionBatch() throws Exception { rs = queryTable(driver, "select ROW__ID, a, b, INPUT__FILE__NAME from default.streamingnobuckets order by ROW__ID"); Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\tfoo\tbar")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/base_0000009/bucket_00000")); + Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/base_0000009_v0000029/bucket_00000")); Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\ta3\tb4")); - Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/base_0000009/bucket_00000")); + Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/base_0000009_v0000029/bucket_00000")); Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\ta5\tb6")); - Assert.assertTrue(rs.get(2), rs.get(2).endsWith("streamingnobuckets/base_0000009/bucket_00000")); + Assert.assertTrue(rs.get(2), rs.get(2).endsWith("streamingnobuckets/base_0000009_v0000029/bucket_00000")); Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":4,\"bucketid\":536870912,\"rowid\":1}\ta11\tb12")); - Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/base_0000009/bucket_00000")); + Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/base_0000009_v0000029/bucket_00000")); Assert.assertTrue(rs.get(4), rs.get(4).startsWith("{\"writeid\":5,\"bucketid\":536870912,\"rowid\":0}\ta13\tb14")); - Assert.assertTrue(rs.get(4), rs.get(4).endsWith("streamingnobuckets/base_0000009/bucket_00000")); + Assert.assertTrue(rs.get(4), rs.get(4).endsWith("streamingnobuckets/base_0000009_v0000029/bucket_00000")); Assert.assertTrue(rs.get(5), rs.get(5).startsWith("{\"writeid\":6,\"bucketid\":536870912,\"rowid\":0}\t0\t0")); - Assert.assertTrue(rs.get(5), rs.get(5).endsWith("streamingnobuckets/base_0000009/bucket_00000")); + Assert.assertTrue(rs.get(5), rs.get(5).endsWith("streamingnobuckets/base_0000009_v0000029/bucket_00000")); } /** @@ -940,7 +947,7 @@ public void testTableValidation() throws Exception { @Deprecated private void checkDataWritten(Path partitionPath, long minTxn, long maxTxn, int buckets, int numExpectedFiles, String... records) throws Exception { - ValidWriteIdList writeIds = msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName)); + ValidWriteIdList writeIds = getTransactionContext(conf); AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, writeIds); Assert.assertEquals(0, dir.getObsolete().size()); Assert.assertEquals(0, dir.getOriginalFiles().size()); @@ -973,7 +980,8 @@ private void checkDataWritten(Path partitionPath, long minTxn, long maxTxn, int job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS_TYPES, "bigint:string"); AcidUtils.setAcidOperationalProperties(job, true, null); job.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); - job.set(ValidWriteIdList.VALID_WRITEIDS_KEY, writeIds.toString()); + job.set(ValidWriteIdList.VALID_WRITEIDS_KEY, writeIds.writeToString()); + job.set(ValidTxnList.VALID_TXNS_KEY, conf.get(ValidTxnList.VALID_TXNS_KEY)); InputSplit[] splits = inf.getSplits(job, buckets); Assert.assertEquals(numExpectedFiles, splits.length); org.apache.hadoop.mapred.RecordReader rr = @@ -994,8 +1002,7 @@ private void checkDataWritten(Path partitionPath, long minTxn, long maxTxn, int */ private void checkDataWritten2(Path partitionPath, long minTxn, long maxTxn, int numExpectedFiles, String validationQuery, boolean vectorize, String... records) throws Exception { - ValidWriteIdList txns = msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName)); - AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, txns); + AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, getTransactionContext(conf)); Assert.assertEquals(0, dir.getObsolete().size()); Assert.assertEquals(0, dir.getOriginalFiles().size()); List current = dir.getCurrentDirectories(); @@ -1038,9 +1045,15 @@ private void checkDataWritten2(Path partitionPath, long minTxn, long maxTxn, int conf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorizationEnabled); } + private ValidWriteIdList getTransactionContext(Configuration conf) throws Exception { + ValidTxnList validTxnList = msClient.getValidTxns(); + conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); + List v = msClient.getValidWriteIds(Collections + .singletonList(TableName.getDbTable(dbName, tblName)), validTxnList.writeToString()); + return TxnCommonUtils.createValidReaderWriteIdList(v.get(0)); + } private void checkNothingWritten(Path partitionPath) throws Exception { - ValidWriteIdList writeIds = msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName)); - AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, writeIds); + AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, getTransactionContext(conf)); Assert.assertEquals(0, dir.getObsolete().size()); Assert.assertEquals(0, dir.getOriginalFiles().size()); List current = dir.getCurrentDirectories(); @@ -1937,8 +1950,7 @@ public void testInterleavedTransactionBatchCommits() throws Exception { /*now both batches have committed (but not closed) so we for each primary file we expect a side file to exist and indicate the true length of primary file*/ FileSystem fs = partLoc.getFileSystem(conf); - AcidUtils.Directory dir = AcidUtils.getAcidState(partLoc, conf, - msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName))); + AcidUtils.Directory dir = AcidUtils.getAcidState(partLoc, conf, getTransactionContext(conf)); for (AcidUtils.ParsedDelta pd : dir.getCurrentDirectories()) { for (FileStatus stat : fs.listStatus(pd.getPath(), AcidUtils.bucketFileFilter)) { Path lengthFile = OrcAcidUtils.getSideFile(stat.getPath()); @@ -1963,7 +1975,7 @@ public void testInterleavedTransactionBatchCommits() throws Exception { //so each of 2 deltas has 1 bucket0 and 1 bucket0_flush_length. Furthermore, each bucket0 //has now received more data(logically - it's buffered) but it is not yet committed. //lets check that side files exist, etc - dir = AcidUtils.getAcidState(partLoc, conf, msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName))); + dir = AcidUtils.getAcidState(partLoc, conf, getTransactionContext(conf)); for (AcidUtils.ParsedDelta pd : dir.getCurrentDirectories()) { for (FileStatus stat : fs.listStatus(pd.getPath(), AcidUtils.bucketFileFilter)) { Path lengthFile = OrcAcidUtils.getSideFile(stat.getPath());