diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 95c5c0e..f594437 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -553,6 +553,40 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal " invalided by updates or partition drops before this. Default is one week."), METASTORE_HBASE_FILE_METADATA_THREADS("hive.metastore.hbase.file.metadata.threads", 1, "Number of threads to use to read file metadata in background to cache it."), + METASTORE_HBASE_LOCK_POLL_TIMEOUT("hive.metastore.hbase.lock.poll.timeout", "5000ms", + new TimeValidator(TimeUnit.MILLISECONDS), + "Time to wait for locks when polling. Once this is up the client will be returned a " + + "waiting status."), + METASTORE_HBASE_TXN_MGR_LOCK_POLL_TIMEOUT("hive.metastore.hbase.txn.mgr.lock.poll.timeout", + "5000ms", new TimeValidator(TimeUnit.MILLISECONDS), + "Time to wait for a set of Hive Locks to acquire before returning waiting status to caller"), + METASTORE_HBASE_TXN_MGR_TIMEOUT_CLEANER_FREQ("hive.metastore.hbase.txn.mgr.timeout.cleaner.freq", + "5000ms", new TimeValidator(TimeUnit.MILLISECONDS), + "How often to run the timeout cleaner thread"), + METASTORE_HBASE_TXN_MGR_TIMEOUT_CLEANER_INITIAL_WAIT("hive.metastore.hbase.txn.mgr.timeout.cleaner.initial.wait", + "60s", new TimeValidator(TimeUnit.SECONDS), + "How long to wait after recovery before running the timeout cleaner for the first time"), + METASTORE_HBASE_TXN_MGR_DEADLOCK_DETECTOR_FREQ("hive.metastore.hbase.txn.mgr.deadlock.detector.freq", + "2000ms", new TimeValidator(TimeUnit.MILLISECONDS), + "How often to run the deadlock detector thread"), + METASTORE_HBASE_TXN_MGR_LOCK_QUEUE_SHRINKER_FREQ("hive.metastore.hbase.txn.mgr.lock.queue.shrinker.freq", + "60000ms", new TimeValidator(TimeUnit.MILLISECONDS), + "How often to run the lock queue shrinker thread"), + METASTORE_HBASE_TXN_MGR_FULL_CHECKER_FREQ("hive.metastore.hbase.txn.mgr.full.checker.freq", + "2000ms", new TimeValidator(TimeUnit.MILLISECONDS), + "How often to run the full checker thread"), + METASTORE_HBASE_TXN_MGR_COMMITTED_TXN_CLEANER_FREQ("hive.metastore.hbase.txn.mgr.committed.txn.cleaner.freq", + "2000ms", new TimeValidator(TimeUnit.MILLISECONDS), + "How often to run the committed transaction thread"), + METASTORE_HBASE_TXN_MGR_MAX_OBJECTS("hive.metastore.hbase.txn.mgr.max.objects", + 1000000, "Maximum number of locks + transactions that can be active in memory at one time" + + ". Once this is exceeded the transaction manager will reject any new transactions until " + + "the count falls below 90% of this number."), + METASTORE_HBASE_TXN_MGR_THREAD_POOL_BASE_SIZE("hive.metastore.hbase.txn.mgr.thread.pool.base.size", + 5, "Base number of threads used by the HBase Txn Manager for background processes"), + METASTORE_HBASE_TXN_MGR_THREAD_POOL_MAX_SIZE("hive.metastore.hbase.txn.mgr.thread.pool.max" + + ".size", + 20, "Maximum number of threads used by the HBase Txn Manager for background processes"), METASTORETHRIFTCONNECTIONRETRIES("hive.metastore.connect.retries", 3, "Number of retries while opening a connection to metastore"), diff --git itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java index e5833b8..f12c02a 100644 --- itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java +++ itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java @@ -571,7 +571,7 @@ public void oneMondoTest() throws Exception { outStr = new ByteArrayOutputStream(); out = new PrintStream(outStr); tool.go(false, HBaseReadWrite.SECURITY_TABLE, null, "whatever", conf, out, err); - Assert.assertEquals("Master key 0: this be a key" + lsep + + Assert.assertEquals("Master key 1: this be a key" + lsep + "Delegation token tokenid: delegation token" + lsep, outStr.toString()); outStr = new ByteArrayOutputStream(); diff --git itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreIntegration.java itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreIntegration.java index 2cc1373..90e05b1 100644 --- itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreIntegration.java +++ itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreIntegration.java @@ -1778,19 +1778,19 @@ public void delegationToken() throws Exception { @Test public void masterKey() throws Exception { - Assert.assertEquals(0, store.addMasterKey("k1")); - Assert.assertEquals(1, store.addMasterKey("k2")); + Assert.assertEquals(1, store.addMasterKey("k1")); + Assert.assertEquals(2, store.addMasterKey("k2")); String[] keys = store.getMasterKeys(); Arrays.sort(keys); Assert.assertArrayEquals(new String[]{"k1", "k2"}, keys); - store.updateMasterKey(0, "k3"); + store.updateMasterKey(1, "k3"); keys = store.getMasterKeys(); Arrays.sort(keys); Assert.assertArrayEquals(new String[]{"k2", "k3"}, keys); - store.removeMasterKey(1); + store.removeMasterKey(2); keys = store.getMasterKeys(); Assert.assertArrayEquals(new String[]{"k3"}, keys); diff --git metastore/if/hive_metastore.thrift metastore/if/hive_metastore.thrift index 6a55962..5589962 100755 --- metastore/if/hive_metastore.thrift +++ metastore/if/hive_metastore.thrift @@ -94,6 +94,7 @@ enum LockType { SHARED_READ = 1, SHARED_WRITE = 2, EXCLUSIVE = 3, + INTENTION = 4, } enum CompactionType { diff --git metastore/pom.xml metastore/pom.xml index 18c1f9c..f4c7687 100644 --- metastore/pom.xml +++ metastore/pom.xml @@ -76,6 +76,21 @@ + org.apache.hbase + hbase-server + ${hbase.version} + + + org.slf4j + slf4j-log4j12 + + + commmons-logging + commons-logging + + + + com.jolbox bonecp ${bonecp.version} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java index c9fadad..c4ee79d 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java @@ -77,6 +77,7 @@ import org.apache.hadoop.hive.metastore.events.PreReadDatabaseEvent; import org.apache.hadoop.hive.metastore.events.PreReadTableEvent; import org.apache.hadoop.hive.metastore.filemeta.OrcFileMetadataHandler; +import org.apache.hadoop.hive.metastore.hbase.HBaseStore; import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; @@ -6626,8 +6627,12 @@ private static void startHouseKeeperService(HiveConf conf) throws Exception { if(!HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_INITIATOR_ON)) { return; } - startHouseKeeperService(conf, Class.forName("org.apache.hadoop.hive.ql.txn.AcidHouseKeeperService")); - startHouseKeeperService(conf, Class.forName("org.apache.hadoop.hive.ql.txn.AcidCompactionHistoryService")); + if (!conf.getVar(ConfVars.METASTORE_RAW_STORE_IMPL).equals(HBaseStore.class.getName())) { + startHouseKeeperService(conf, + Class.forName("org.apache.hadoop.hive.ql.txn.AcidHouseKeeperService")); + startHouseKeeperService(conf, + Class.forName("org.apache.hadoop.hive.ql.txn.AcidCompactionHistoryService")); + } } private static void startHouseKeeperService(HiveConf conf, Class c) throws Exception { //todo: when metastore adds orderly-shutdown logic, houseKeeper.stop() diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java index 7ed825f..0d6880a 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java @@ -20,11 +20,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterators; +import com.google.protobuf.Message; import org.apache.commons.codec.binary.Base64; import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -37,8 +35,10 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.filter.CompareFilter; import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.filter.RegexStringComparator; import org.apache.hadoop.hbase.filter.RowFilter; import org.apache.hadoop.hive.common.ObjectPair; @@ -58,12 +58,15 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.hbase.PartitionKeyComparator.Operator; +import org.apache.hadoop.hive.metastore.hbase.txn.txnmgr.TransactionCoprocessor; import org.apache.hive.common.util.BloomFilter; import org.apache.thrift.TBase; import org.apache.thrift.TException; import org.apache.thrift.protocol.TProtocol; import org.apache.thrift.protocol.TSimpleJSONProtocol; import org.apache.thrift.transport.TMemoryBuffer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.io.UnsupportedEncodingException; @@ -81,6 +84,7 @@ import java.util.Map; import java.util.NavigableMap; import java.util.Set; +import java.util.TreeMap; /** @@ -90,14 +94,17 @@ final static String AGGR_STATS_TABLE = "HBMS_AGGR_STATS"; final static String DB_TABLE = "HBMS_DBS"; + final static String COMPACTION_TABLE = "HBMS_COMPACTIONS"; final static String FUNC_TABLE = "HBMS_FUNCS"; final static String GLOBAL_PRIVS_TABLE = "HBMS_GLOBAL_PRIVS"; final static String PART_TABLE = "HBMS_PARTITIONS"; + final static String POTENTIAL_COMPACTION_TABLE = "HBMS_POTENTIAL_COMPACTIONS"; final static String ROLE_TABLE = "HBMS_ROLES"; final static String SD_TABLE = "HBMS_SDS"; final static String SECURITY_TABLE = "HBMS_SECURITY"; final static String SEQUENCES_TABLE = "HBMS_SEQUENCES"; final static String TABLE_TABLE = "HBMS_TBLS"; + final static String TXN_TABLE = "HBMS_TXNS"; final static String USER_TO_ROLE_TABLE = "HBMS_USER_TO_ROLE"; final static String FILE_METADATA_TABLE = "HBMS_FILE_METADATA"; final static byte[] CATALOG_CF = "c".getBytes(HBaseUtils.ENCODING); @@ -109,12 +116,15 @@ public final static String[] tableNames = { AGGR_STATS_TABLE, DB_TABLE, FUNC_TABLE, GLOBAL_PRIVS_TABLE, PART_TABLE, USER_TO_ROLE_TABLE, ROLE_TABLE, SD_TABLE, SECURITY_TABLE, SEQUENCES_TABLE, - TABLE_TABLE, FILE_METADATA_TABLE }; + TABLE_TABLE, FILE_METADATA_TABLE, TXN_TABLE, + COMPACTION_TABLE, POTENTIAL_COMPACTION_TABLE}; public final static Map> columnFamilies = new HashMap<> (tableNames.length); static { columnFamilies.put(AGGR_STATS_TABLE, Arrays.asList(CATALOG_CF)); columnFamilies.put(DB_TABLE, Arrays.asList(CATALOG_CF)); + columnFamilies.put(COMPACTION_TABLE, Arrays.asList(CATALOG_CF)); + columnFamilies.put(POTENTIAL_COMPACTION_TABLE, Arrays.asList(CATALOG_CF)); columnFamilies.put(FUNC_TABLE, Arrays.asList(CATALOG_CF)); columnFamilies.put(GLOBAL_PRIVS_TABLE, Arrays.asList(CATALOG_CF)); columnFamilies.put(PART_TABLE, Arrays.asList(CATALOG_CF, STATS_CF)); @@ -124,15 +134,11 @@ columnFamilies.put(SECURITY_TABLE, Arrays.asList(CATALOG_CF)); columnFamilies.put(SEQUENCES_TABLE, Arrays.asList(CATALOG_CF)); columnFamilies.put(TABLE_TABLE, Arrays.asList(CATALOG_CF, STATS_CF)); + columnFamilies.put(TXN_TABLE, Arrays.asList(CATALOG_CF)); // Stats CF will contain PPD stats. columnFamilies.put(FILE_METADATA_TABLE, Arrays.asList(CATALOG_CF, STATS_CF)); } - final static byte[] MASTER_KEY_SEQUENCE = "master_key".getBytes(HBaseUtils.ENCODING); - // The change version functionality uses the sequences table, but we don't want to give the - // caller complete control over the sequence name as they might inadvertently clash with one of - // our sequence keys, so add a prefix to their topic name. - final static String CHANGE_VERSION_SEQUENCE_PREFIX = "cv_"; final static byte[] AGGR_STATS_BLOOM_COL = "b".getBytes(HBaseUtils.ENCODING); private final static byte[] AGGR_STATS_STATS_COL = "s".getBytes(HBaseUtils.ENCODING); @@ -141,7 +147,18 @@ private final static byte[] REF_COUNT_COL = "ref".getBytes(HBaseUtils.ENCODING); private final static byte[] DELEGATION_TOKEN_COL = "dt".getBytes(HBaseUtils.ENCODING); private final static byte[] MASTER_KEY_COL = "mk".getBytes(HBaseUtils.ENCODING); + + final static byte[] MASTER_KEY_SEQUENCE = "master_key".getBytes(HBaseUtils.ENCODING); + public final static byte[] TXN_SEQUENCE = "txn_seq".getBytes(HBaseUtils.ENCODING); + public final static byte[] LOCK_SEQUENCE = "lock_seq".getBytes(HBaseUtils.ENCODING); + public final static byte[] COMPACTION_SEQUENCE = "lock_seq".getBytes(HBaseUtils.ENCODING); + // The change version functionality uses the sequences table, but we don't want to give the + // caller complete control over the sequence name as they might inadvertently clash with one of + // our sequence keys, so add a prefix to their topic name. + final static String CHANGE_VERSION_SEQUENCE_PREFIX = "cv_"; + private final static byte[] GLOBAL_PRIVS_KEY = "gp".getBytes(HBaseUtils.ENCODING); + private final static int TABLES_TO_CACHE = 10; // False positives are very bad here because they cause us to invalidate entries we shouldn't. // Space used and # of hash functions grows in proportion to ln of num bits so a 10x increase @@ -206,7 +223,7 @@ public static synchronized void setConf(Configuration configuration) { * {@link #setConf} has been called. Woe betide you if that's not the case. * @return thread's instance of HBaseReadWrite */ - static HBaseReadWrite getInstance() { + public static HBaseReadWrite getInstance() { if (staticConf == null) { throw new RuntimeException("Must set conf object before getting an instance"); } @@ -307,7 +324,7 @@ static synchronized void createTablesIfNotExist() throws IOException { /** * Begin a transaction */ - void begin() { + public void begin() { try { conn.beginTransaction(); } catch (IOException e) { @@ -318,7 +335,7 @@ void begin() { /** * Commit a transaction */ - void commit() { + public void commit() { try { conn.commitTransaction(); } catch (IOException e) { @@ -326,7 +343,7 @@ void commit() { } } - void rollback() { + public void rollback() { try { conn.rollbackTransaction(); } catch (IOException e) { @@ -1979,11 +1996,11 @@ ColumnStatistics getTableStatistics(String dbName, String tblName, List colNameBytes[i] = HBaseUtils.buildKey(colNames.get(i)); } + List partKeyTypes = + HBaseUtils.getPartitionKeyTypes(getTable(dbName, tblName).getPartitionKeys()); for (int i = 0; i < partNames.size(); i++) { valToPartMap.put(partVals.get(i), partNames.get(i)); - byte[] partKey = HBaseUtils.buildPartitionKey(dbName, tblName, - HBaseUtils.getPartitionKeyTypes(getTable(dbName, tblName).getPartitionKeys()), - partVals.get(i)); + byte[] partKey = HBaseUtils.buildPartitionKey(dbName, tblName, partKeyTypes, partVals.get(i)); Get get = new Get(partKey); for (byte[] colName : colNameBytes) { get.addColumn(STATS_CF, colName); @@ -2016,6 +2033,39 @@ ColumnStatistics getTableStatistics(String dbName, String tblName, List } /** + * Find out which columns have statistics. This does not return the statistics, just + * information on which columns for this table or partition have them. + * @param dbName database the table is in + * @param tblName table to fetch column info for + * @param partVals partition values for one partition. These can be null, in which case + * tblName will be assumed to be an unpartitioned table. + * @return list of columns for which stats are stored for this table or partition. + * @throws IOException + */ + public List getColumnsWithStatistics(String dbName, String tblName, List partVals) + throws IOException { + HTableInterface htable; + byte[] key; + if (partVals == null) { + htable = conn.getHBaseTable(TABLE_TABLE); + key = HBaseUtils.buildKey(dbName, tblName); + } else { + htable = conn.getHBaseTable(PART_TABLE); + List partKeyTypes = + HBaseUtils.getPartitionKeyTypes(getTable(dbName, tblName).getPartitionKeys()); + key = HBaseUtils.buildPartitionKey(dbName, tblName, partKeyTypes, partVals); + } + Get get = new Get(key); + get.addFamily(STATS_CF); + Result result = htable.get(get); + List cols = new ArrayList<>(); + for (byte[] colname : result.getFamilyMap(STATS_CF).keySet()) { + cols.add(new String(colname, HBaseUtils.ENCODING)); + } + return cols; + } + + /** * Get a reference to the stats cache. * @return the stats cache. */ @@ -2390,21 +2440,552 @@ void deleteMasterKey(Integer seqNo) throws IOException { } /********************************************************************************************** + * Transaction and Compaction related methods + *********************************************************************************************/ + + /** + * Get a transaction object + * @param txnId id of the transaction to get + * @return transaction object, or null if no such transaction + * @throws IOException + */ + public HbaseMetastoreProto.Transaction getTransaction(long txnId) throws IOException { + byte[] key = HBaseUtils.buildKey(Long.toString(txnId)); + byte[] serialized = read(TXN_TABLE, key, CATALOG_CF, CATALOG_COL); + if (serialized == null) return null; + return HBaseUtils.deserializeTransaction(serialized); + } + + public List getTransactions(Collection txnIds) + throws IOException { + List gets = new ArrayList<>(txnIds.size()); + for (long txnId : txnIds) { + byte[] key = HBaseUtils.buildKey(Long.toString(txnId)); + Get g = new Get(key); + g.addColumn(CATALOG_CF, CATALOG_COL); + gets.add(g); + } + + HTableInterface htab = conn.getHBaseTable(TXN_TABLE); + Result[] results = htab.get(gets); + List txns = new ArrayList<>(results.length); + for (Result result : results) { + txns.add(HBaseUtils.deserializeTransaction(result.getValue(CATALOG_CF, CATALOG_COL))); + } + return txns; + } + + /** + * Get an iterator to all of the transactions in the table. + * @return iterator + * @throws IOException + * @param filter + */ + public List scanTransactions(Filter filter) throws IOException { + Iterator iter = scan(TXN_TABLE, null, null, CATALOG_CF, CATALOG_COL, filter); + List txns = new ArrayList<>(); + while (iter.hasNext()) { + Result result = iter.next(); + txns.add(HBaseUtils.deserializeTransaction(result.getValue(CATALOG_CF, CATALOG_COL))); + } + return txns; + } + + /** + * Write a transaction to HBase. + * @param txn transaction to record. + * @throws IOException + */ + public void putTransaction(HbaseMetastoreProto.Transaction txn) throws IOException { + byte[][] serialized = HBaseUtils.serializeTransaction(txn); + store(TXN_TABLE, serialized[0], CATALOG_CF, CATALOG_COL, serialized[1]); + } + + /** + * Put a group of transactions into the table + * @param txns list of transaction to add + * @throws IOException + */ + public void putTransactions(List txns) throws IOException { + List puts = new ArrayList<>(txns.size()); + for (HbaseMetastoreProto.Transaction txn : txns) { + byte[][] serialized = HBaseUtils.serializeTransaction(txn); + Put p = new Put(serialized[0]); + p.add(CATALOG_CF, CATALOG_COL, serialized[1]); + puts.add(p); + } + HTableInterface htab = conn.getHBaseTable(TXN_TABLE); + htab.put(puts); + conn.flush(htab); + } + + /** + * Remove a transaction from HBase. + * @param txnId id of transaction to remove. + * @throws IOException + */ + public void deleteTransaction(long txnId) throws IOException { + byte[] key = HBaseUtils.buildKey(Long.toString(txnId)); + delete(TXN_TABLE, key, null, null); + } + + /** + * Delete a group of transactions. + * @param txnIds txns to delete + * @throws IOException + */ + public void deleteTransactions(Collection txnIds) throws IOException { + List deletes = new ArrayList<>(txnIds.size()); + for (Long txnId : txnIds) { + deletes.add(new Delete(HBaseUtils.buildKey(Long.toString(txnId)))); + } + HTableInterface htab = conn.getHBaseTable(TXN_TABLE); + htab.delete(deletes); + conn.flush(htab); + } + + /** + * Print out one transaction. + * @param txnId id of transaction to print + * @return string containing transaction information + * @throws IOException + */ + String printTransaction(long txnId) throws IOException { + return printTransaction(getTransaction(txnId)); + } + + /** + * Print all transactions. + * @return list of transactions + * @throws IOException + */ + public List printTransactions() throws IOException { + List lines = new ArrayList<>(); + for (HbaseMetastoreProto.Transaction txn : scanTransactions(null)) { + lines.add(printTransaction(txn)); + } + return lines; + } + + private String printTransaction(HbaseMetastoreProto.Transaction txn) { + StringBuilder buf = new StringBuilder("{id:") + .append(txn.getId()) + .append(", state:\"") + .append(txn.getTxnState().toString()) + .append("\", "); + if (txn.hasUser()) { + buf.append("user: \"") + .append(txn.getUser()) + .append("\", "); + } + if (txn.hasHostname()) { + buf.append("hostname: \"") + .append(txn.getHostname()) + .append("\", "); + } + if (txn.hasAgentInfo()) { + buf.append("agentInfo: \"") + .append(txn.getAgentInfo()) + .append("\", "); + } + if (txn.hasMetaInfo()) { + buf.append("metaInfo: \"") + .append(txn.getMetaInfo()) + .append("\", "); + } + buf.append("locks:["); + boolean first = true; + List locks = txn.getLocksList(); + for (HbaseMetastoreProto.Transaction.Lock lock : locks) { + if (first) first = false; + else buf.append(", "); + buf.append("{id: ") + .append(lock.getId()) + .append(", "); + buf.append("{state: \"") + .append(lock.getState().toString()) + .append("\", "); + buf.append("{type: \"") + .append(lock.getType().toString()) + .append("\""); + if (lock.hasDb()) { + buf.append(", ") + .append("db: \"") + .append(lock.getDb()) + .append("\""); + } + if (lock.hasTable()) { + buf.append(", ") + .append("table: \"") + .append(lock.getTable()) + .append("\""); + } + if (lock.hasPartition()) { + buf.append(", ") + .append("partition: \"") + .append(lock.getPartition()) + .append("\""); + } + if (lock.hasAcquiredAt()) { + buf.append(", ") + .append("acquiredAt: ") + .append(lock.getAcquiredAt()); + } + buf.append("}"); + } + buf.append("]}"); + return buf.toString(); + } + + /** + * Get a compaction object + * @param compactionId id of the compactions to get + * @return transaction object, or null if no such transaction + * @throws IOException + */ + public HbaseMetastoreProto.Compaction getCompaction(long compactionId) throws IOException { + byte[] key = HBaseUtils.buildKey(Long.toString(compactionId)); + byte[] serialized = read(COMPACTION_TABLE, key, CATALOG_CF, CATALOG_COL); + if (serialized == null) return null; + return HBaseUtils.deserializeCompaction(serialized); + } + + /** + * Get an iterator to all of the compactions in the table. + * @param state only return compactions in the given state. + * @return iterator + * @throws IOException + */ + public List scanCompactions(HbaseMetastoreProto.CompactionState state) + throws IOException { + Filter filter = null; + // TODO figure out how to set a filter that only pulls out entries that match the desired state. + Iterator iter = scan(COMPACTION_TABLE, null, null, CATALOG_CF, CATALOG_COL, filter); + List compactions = new ArrayList<>(); + while (iter.hasNext()) { + Result result = iter.next(); + compactions.add(HBaseUtils.deserializeCompaction(result.getValue(CATALOG_CF, CATALOG_COL))); + } + return compactions; + } + + /** + * Write a compaction to HBase. If you are changing the state of the compaction you MUST + * delete it and then put it, as you're also changing the key. + * @param compaction compaction to record. + * @throws IOException + */ + public void putCompaction(HbaseMetastoreProto.Compaction compaction) throws IOException { + byte[][] serialized = HBaseUtils.serializeCompaction(compaction); + store(COMPACTION_TABLE, serialized[0], CATALOG_CF, CATALOG_COL, serialized[1]); + } + + public void putCompactions(List compactions) throws IOException { + List puts = new ArrayList<>(compactions.size()); + for (HbaseMetastoreProto.Compaction compaction : compactions) { + byte[][] serialized = HBaseUtils.serializeCompaction(compaction); + Put p = new Put(serialized[0]); + p.add(CATALOG_CF, CATALOG_COL, serialized[1]); + puts.add(p); + } + HTableInterface htab = conn.getHBaseTable(COMPACTION_TABLE); + htab.put(puts); + conn.flush(htab); + } + + /** + * Remove a compaction from HBase. + * @param compactionId id of compaction to remove. + * @throws IOException + */ + public void deleteCompaction(long compactionId) throws IOException { + byte[] key = HBaseUtils.buildKey(Long.toString(compactionId)); + delete(COMPACTION_TABLE, key, null, null); + } + + public void deleteCompactions(List compactionIds) throws IOException { + List deletes = new ArrayList<>(compactionIds.size()); + for (long id : compactionIds) { + deletes.add(new Delete(HBaseUtils.buildKey(Long.toString(id)))); + } + HTableInterface htab = conn.getHBaseTable(COMPACTION_TABLE); + htab.delete(deletes); + conn.flush(htab); + } + + /** + * Print out one compaction. + * @param compactionId id of compaction to print + * @return string containing compaction information + * @throws IOException + */ + String printCompaction(long compactionId) throws IOException { + return printCompaction(getCompaction(compactionId)); + } + + /** + * Print all compactions. + * @return list of compactions. + * @throws IOException + */ + List printCompactions() throws IOException { + List lines = new ArrayList<>(); + for (HbaseMetastoreProto.Compaction compaction : scanCompactions(null)) { + lines.add(printCompaction(compaction)); + } + return lines; + } + + private String printCompaction(HbaseMetastoreProto.Compaction compaction) { + StringBuilder buf = new StringBuilder("{id:") + .append(compaction.getId()) + .append(", state:\"") + .append(compaction.getState().toString()) + .append("\", ") + .append(", type:\"") + .append(compaction.getType().toString()) + .append("\""); + if (compaction.hasDb()) { + buf.append(", db: \"") + .append(compaction.getDb()) + .append("\""); + } + if (compaction.hasTable()) { + buf.append(", table: \"") + .append(compaction.getDb()) + .append("\""); + } + if (compaction.hasPartition()) { + buf.append(", partition: \"") + .append(compaction.getPartition()) + .append("\""); + } + if (compaction.hasWorkerId()) { + buf.append(", workerId: \"") + .append(compaction.getWorkerId()) + .append("\""); + } + if (compaction.hasStartedWorkingAt()) { + buf.append(", startedWorkingAt: ") + .append(compaction.getStartedWorkingAt()); + } + if (compaction.hasRunAs()) { + buf.append(", runAs: \"") + .append(compaction.getRunAs()) + .append("\""); + } + if (compaction.hasHighestTxnId()) { + buf.append(", highestTxnId: ") + .append(compaction.getHighestTxnId()); + } + if (compaction.hasMetaInfo()) { + buf.append(", metaInfo: \"") + .append(compaction.getMetaInfo()) + .append("\""); + } + if (compaction.hasHadoopJobId()) { + buf.append(", hadoopJobId: \"") + .append(compaction.getHadoopJobId()) + .append("\""); + } + buf.append("}"); + return buf.toString(); + } + + /** + * Get a potential compaction object + * @param db database potential is in + * @param table table of potential + * @param partition partition of potential, may be null + * @return potential compaction object, or null if no such potential compaction + * @throws IOException + */ + public HbaseMetastoreProto.PotentialCompaction getPotentialCompaction(String db, String table, + String partition) + throws IOException { + byte[] key = partition == null ? HBaseUtils.buildKey(db, table) : + HBaseUtils.buildKey(db, table, partition); + byte[] serialized = read(POTENTIAL_COMPACTION_TABLE, key, CATALOG_CF, CATALOG_COL); + if (serialized == null) return null; + return HBaseUtils.deserializePotentialCompaction(serialized); + } + + /** + * Get an iterator to all of the potential compactions in the table. Compactions are ordered + * by those with the most transactions (and thus likely to have the most outstanding delta files). + * @return iterator + * @throws IOException + */ + public Iterator scanPotentialCompactions() + throws IOException { + Iterator iter = scan(POTENTIAL_COMPACTION_TABLE, null, null, CATALOG_CF, CATALOG_COL, + null); + NavigableMap potentials = new TreeMap<>(); + while (iter.hasNext()) { + Result result = iter.next(); + HbaseMetastoreProto.PotentialCompaction pc = + HBaseUtils.deserializePotentialCompaction(result.getValue(CATALOG_CF, CATALOG_COL)); + potentials.put(pc.getTxnIdsCount(), pc); + } + return potentials.descendingMap().values().iterator(); + } + + public void putPotentialCompaction(HbaseMetastoreProto.PotentialCompaction potential) + throws IOException { + byte[][] serialized = HBaseUtils.serializePotentialCompaction(potential); + store(POTENTIAL_COMPACTION_TABLE, serialized[0], CATALOG_CF, CATALOG_COL, serialized[1]); + } + + public static class PotentialCompactionEntity { + final public String db; + final public String table; + final public String partition; + + public PotentialCompactionEntity(String db, String table, String partition) { + this.db = db; + this.table = table; + this.partition = partition; + } + } + /** + * Write a potential compaction to HBase. + * @param txnid transaction id + * @param pces potential compaction entities s to record. This makes the assumption that there + * are no duplicates in the list. + * @throws IOException + */ + public void putPotentialCompactions(long txnid, List pces) + throws IOException { + List hbasePcs = new ArrayList<>(); + for (PotentialCompactionEntity pc : pces) { + // There may already be an entry for this compaction. If so, just add our transaction id + // to it. Otherwise build a new one. + HbaseMetastoreProto.PotentialCompaction hbasePc = + getPotentialCompaction(pc.db, pc.table, pc.partition); + if (hbasePc == null) { + HbaseMetastoreProto.PotentialCompaction.Builder bldr = + HbaseMetastoreProto.PotentialCompaction.newBuilder() + .setDb(pc.db) + .setTable(pc.table) + .addTxnIds(txnid); + + if (pc.partition != null) bldr.setPartition(pc.partition); + hbasePcs.add(bldr.build()); + } else { + hbasePcs.add(HbaseMetastoreProto.PotentialCompaction.newBuilder(hbasePc) + .addTxnIds(txnid) + .build()); + } + } + + List puts = new ArrayList<>(hbasePcs.size()); + for (HbaseMetastoreProto.PotentialCompaction potential : hbasePcs) { + byte[][] serialized = HBaseUtils.serializePotentialCompaction(potential); + Put p = new Put(serialized[0]); + p.add(CATALOG_CF, CATALOG_COL, serialized[1]); + puts.add(p); + } + HTableInterface htab = conn.getHBaseTable(POTENTIAL_COMPACTION_TABLE); + htab.put(puts); + conn.flush(htab); + } + + /** + * Remove a potential compaction from HBase. + * @param db database potential is in + * @param table table of potential + * @param partition partition of potential, may be null + * @throws IOException + */ + public void deletePotentialCompaction(String db, String table, String partition) throws IOException { + byte[] key = partition == null ? HBaseUtils.buildKey(db, table) : + HBaseUtils.buildKey(db, table, partition); + delete(POTENTIAL_COMPACTION_TABLE, key, null, null); + } + + /** + * Print out one potential compaction. + * @param db database potential is in + * @param table table of potential + * @param partition partition of potential, may be null + * @return string containing potential compaction information + * @throws IOException + */ + String printPotentialCompaction(String db, String table, String partition) throws IOException { + return printPotentialCompaction(getPotentialCompaction(db, table, partition)); + } + + /** + * Print all potential compactions. + * @return list of potential compactions + * @throws IOException + */ + List printPotentialCompactions() throws IOException { + List lines = new ArrayList<>(); + Iterator iter = scanPotentialCompactions(); + while (iter.hasNext()) { + lines.add(printPotentialCompaction(iter.next())); + } + return lines; + } + + private String printPotentialCompaction(HbaseMetastoreProto.PotentialCompaction potential) { + StringBuilder buf = new StringBuilder("{db: \"") + .append(potential.getDb()) + .append("\", table: \"") + .append(potential.getTable()) + .append("\""); + if (potential.hasPartition()) { + buf.append(", partition : \"") + .append(potential.getPartition()) + .append("\""); + } + boolean first = true; + List txns = potential.getTxnIdsList(); + buf.append(", txnIds:["); + for (Long txn : txns) { + if (first) first = false; + else buf.append(", "); + buf.append(txn); + } + buf.append("]}"); + return buf.toString(); + } + + /********************************************************************************************** * Sequence methods *********************************************************************************************/ - long peekAtSequence(byte[] sequence) throws IOException { + public long peekAtSequence(byte[] sequence) throws IOException { byte[] serialized = read(SEQUENCES_TABLE, sequence, CATALOG_CF, CATALOG_COL); return serialized == null ? 0 : Long.valueOf(new String(serialized, HBaseUtils.ENCODING)); } - long getNextSequence(byte[] sequence) throws IOException { + /** + * Increment a sequence by one. + * @param sequence sequence to increment + * @return Current sequence number + * @throws IOException + */ + public long getNextSequence(byte[] sequence) throws IOException { + return addToSequence(sequence, 1); + } + + /** + * Increment a sequence by a given amount. + * @param sequence sequence to add to + * @param toAdd amount to add + * @return Current sequence value + * @throws IOException + */ + public long addToSequence(byte[] sequence, long toAdd) throws IOException { byte[] serialized = read(SEQUENCES_TABLE, sequence, CATALOG_CF, CATALOG_COL); long val = 0; if (serialized != null) { val = Long.valueOf(new String(serialized, HBaseUtils.ENCODING)); } - byte[] incrSerialized = new Long(val + 1).toString().getBytes(HBaseUtils.ENCODING); + val += toAdd; + byte[] incrSerialized = Long.toString(val).getBytes(HBaseUtils.ENCODING); store(SEQUENCES_TABLE, sequence, CATALOG_CF, CATALOG_COL, incrSerialized); return val; } @@ -2415,7 +2996,6 @@ long getNextSequence(byte[] sequence) throws IOException { * @throws IOException */ List printSequences() throws IOException { - HTableInterface htab = conn.getHBaseTable(SEQUENCES_TABLE); Iterator iter = scan(SEQUENCES_TABLE, CATALOG_CF, CATALOG_COL, null); List sequences = new ArrayList<>(); @@ -2431,6 +3011,23 @@ long getNextSequence(byte[] sequence) throws IOException { } /********************************************************************************************** + * Co-processor methods + *********************************************************************************************/ + // TODO I don't know if this will work with Omid or Tephra. See what they do with table + // .coprocessorServer() + + public R callTransactionManager(Batch.Call call ) + throws Throwable { + HTableInterface htab = conn.getHBaseTable(TXN_TABLE); + Map result = htab.coprocessorService(TransactionCoprocessor.class, null, null, call); + if (result.size() > 1) { + throw new RuntimeException("Logic error! Got result from more than one co-processor"); + } + return result.values().iterator().next(); + } + + + /********************************************************************************************** * Cache methods *********************************************************************************************/ @@ -2512,31 +3109,6 @@ private void multiRead(String table, byte[] colFam, byte[] colName, } } - private void multiModify(String table, byte[][] keys, byte[] colFam, - byte[] colName, List values) throws IOException, InterruptedException { - assert values == null || keys.length == values.size(); - // HBase APIs are weird. To supply bytebuffer value, you have to also have bytebuffer - // column name, but not column family. So there. Perhaps we should add these to constants too. - ByteBuffer colNameBuf = ByteBuffer.wrap(colName); - @SuppressWarnings("deprecation") - HTableInterface htab = conn.getHBaseTable(table); - List actions = new ArrayList<>(keys.length); - for (int i = 0; i < keys.length; ++i) { - ByteBuffer value = (values != null) ? values.get(i) : null; - if (value == null) { - actions.add(new Delete(keys[i])); - } else { - Put p = new Put(keys[i]); - p.addColumn(colFam, colNameBuf, HConstants.LATEST_TIMESTAMP, value); - actions.add(p); - } - } - Object[] results = new Object[keys.length]; - htab.batch(actions, results); - // TODO: should we check results array? we don't care about partial results - conn.flush(htab); - } - private Result read(String table, byte[] key, byte[] colFam, byte[][] colNames) throws IOException { HTableInterface htab = conn.getHBaseTable(table); diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java index a73dbeb..f4b216d 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java @@ -2474,9 +2474,9 @@ static String buildExternalPartName(Table table, List partVals) { return FileUtils.makePartName(partCols, partVals); } - private static List partNameToVals(String name) { + public static List partNameToVals(String name) { if (name == null) return null; - List vals = new ArrayList(); + List vals = new ArrayList<>(); String[] kvp = name.split("/"); for (String kv : kvp) { vals.add(FileUtils.unescapePathName(kv.substring(kv.indexOf('=') + 1))); @@ -2512,7 +2512,7 @@ private void commitOrRoleBack(boolean commit) { } } - @VisibleForTesting HBaseReadWrite backdoor() { + @VisibleForTesting public HBaseReadWrite backdoor() { return getHBase(); } diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java index e0b449b..8e715e2 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.metastore.hbase; import java.io.IOException; +import java.io.InputStream; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.charset.Charset; @@ -27,6 +28,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Properties; @@ -38,12 +40,17 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; +import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions; import org.apache.hadoop.hive.metastore.api.AggrStats; import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData; import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData; +import org.apache.hadoop.hive.metastore.api.CheckLockRequest; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; +import org.apache.hadoop.hive.metastore.api.CommitTxnRequest; +import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.Decimal; import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData; @@ -51,7 +58,17 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Function; import org.apache.hadoop.hive.metastore.api.FunctionType; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; +import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeRequest; +import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse; +import org.apache.hadoop.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.LockState; +import org.apache.hadoop.hive.metastore.api.LockType; import org.apache.hadoop.hive.metastore.api.LongColumnStatsData; +import org.apache.hadoop.hive.metastore.api.OpenTxnRequest; +import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse; import org.apache.hadoop.hive.metastore.api.Order; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; @@ -61,10 +78,15 @@ import org.apache.hadoop.hive.metastore.api.ResourceUri; import org.apache.hadoop.hive.metastore.api.Role; import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement; +import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement; import org.apache.hadoop.hive.metastore.api.SkewedInfo; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.StringColumnStatsData; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.TxnInfo; +import org.apache.hadoop.hive.metastore.api.TxnState; +import org.apache.hadoop.hive.metastore.txn.CompactionInfo; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.ByteStream.Output; import org.apache.hadoop.hive.serde2.SerDeException; @@ -1397,6 +1419,316 @@ static String deserializeMasterKey(byte[] value) throws InvalidProtocolBufferExc } /** + * Serialize a transaction. + * @param txn transaction to serialize + * @return two byte arrays, first contains the key, the second the serialized value. + */ + static byte[][] serializeTransaction(HbaseMetastoreProto.Transaction txn) { + byte[][] result = new byte[2][]; + result[0] = buildKey(Long.toString(txn.getId())); + result[1] = txn.toByteArray(); + return result; + } + + /** + * Deserialize a transaction. + * @param value value fetched from hbase + * @return A transaction + * @throws InvalidProtocolBufferException + */ + static HbaseMetastoreProto.Transaction deserializeTransaction(byte[] value) + throws InvalidProtocolBufferException { + return HbaseMetastoreProto.Transaction.parseFrom(value); + } + + /** + * Deserialize a transaction. This option is useful when you need to read it directly from the + * byte buffer without making a copy, as you can put a ByteArrayInputStream around the byte + * @param is value fetched from hbase as an input stream + * @return A transaction + * @throws InvalidProtocolBufferException + */ + public static HbaseMetastoreProto.Transaction deserializeTransaction(InputStream is) + throws InvalidProtocolBufferException, IOException { + return HbaseMetastoreProto.Transaction.parseFrom(is); + } + + public static TxnInfo pbToThrift(HbaseMetastoreProto.Transaction pb) { + // Use empty constructor because requireds aren't the same between the two, and we need to + // fill in defaults for requireds in thrift that aren't in pb. + TxnInfo thrift = new TxnInfo(); + thrift.setId(pb.getId()); + thrift.setState(pbToThrift(pb.getTxnState())); + if (pb.hasUser()) thrift.setUser(pb.getUser()); + else thrift.setUser("unknown"); + if (pb.hasHostname()) thrift.setHostname(pb.getHostname()); + else thrift.setHostname("unknown"); + if (pb.hasAgentInfo()) thrift.setAgentInfo(pb.getAgentInfo()); + if (pb.hasMetaInfo()) thrift.setMetaInfo(pb.getMetaInfo()); + return thrift; + } + + private static TxnState pbToThrift(HbaseMetastoreProto.TxnState pb) { + switch (pb) { + case COMMITTED: return TxnState.COMMITTED; + case OPEN: return TxnState.OPEN; + case ABORTED: return TxnState.ABORTED; + default: throw new RuntimeException("Unknown txn type: " + pb.toString()); + } + } + + public static GetOpenTxnsResponse pbToThrift(HbaseMetastoreProto.GetOpenTxnsResponse pb) { + GetOpenTxnsResponse thrift = new GetOpenTxnsResponse(pb.getHighWaterMark(), + new HashSet<>(pb.getAbortedTransactionsList())); + thrift.getOpen_txns().addAll(pb.getOpenTransactionsList()); + return thrift; + } + + public static HbaseMetastoreProto.OpenTxnsRequest thriftToPb(OpenTxnRequest thrift) { + HbaseMetastoreProto.OpenTxnsRequest.Builder builder = + HbaseMetastoreProto.OpenTxnsRequest.newBuilder(); + builder.setNumTxns(thrift.getNum_txns()); + builder.setUser(thrift.getUser()); + builder.setHostname(thrift.getHostname()); + if (thrift.isSetAgentInfo()) builder.setAgentInfo(thrift.getAgentInfo()); + return builder.build(); + } + + public static OpenTxnsResponse pbToThrift(HbaseMetastoreProto.OpenTxnsResponse pb) { + return new OpenTxnsResponse(pb.getTxnIdsList()); + } + + public static HbaseMetastoreProto.TransactionId thriftToPb(AbortTxnRequest thrift) { + return HbaseMetastoreProto.TransactionId.newBuilder() + .setId(thrift.getTxnid()) + .build(); + } + + public static HbaseMetastoreProto.TransactionId thriftToPb(CommitTxnRequest thrift) { + return HbaseMetastoreProto.TransactionId.newBuilder() + .setId(thrift.getTxnid()) + .build(); + } + + public static HbaseMetastoreProto.LockRequest thriftToPb(LockRequest thrift) { + HbaseMetastoreProto.LockRequest.Builder lrBuilder = HbaseMetastoreProto.LockRequest.newBuilder(); + lrBuilder.setTxnId(thrift.getTxnid()); + for (LockComponent lc : thrift.getComponent()) { + HbaseMetastoreProto.LockComponent.Builder lcBuilder = + HbaseMetastoreProto.LockComponent.newBuilder(); + lcBuilder.setType(thriftToPb(lc.getType())); + lcBuilder.setDb(lc.getDbname()); + + // Add intention locks + // This is a strange place to do this but it's the only place at the moment since Thrift + // doesn't know about intention locks and there's no convenient way to add them to the PB + // later. + switch (lc.getLevel()) { + case PARTITION: + lcBuilder.setPartition(lc.getPartitionname()); + // Add an intention lock for the table + lrBuilder.addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setType(HbaseMetastoreProto.LockType.INTENTION) + .setDb(lc.getDbname()) + .setTable(lc.getTablename())); + // FALL THROUGH INTENTIONAL + + case TABLE: + lcBuilder.setTable(lc.getTablename()); + // Add an intention lock for the db + lrBuilder.addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setType(HbaseMetastoreProto.LockType.INTENTION) + .setDb(lc.getDbname())); + + // NO DB OR default INTENTIONAL + } + lrBuilder.addComponents(lcBuilder); + } + return lrBuilder.build(); + } + + private static HbaseMetastoreProto.LockType thriftToPb(LockType thrift) { + switch (thrift) { + case SHARED_READ: return HbaseMetastoreProto.LockType.SHARED_READ; + case SHARED_WRITE: return HbaseMetastoreProto.LockType.SHARED_WRITE; + case EXCLUSIVE: return HbaseMetastoreProto.LockType.EXCLUSIVE; + case INTENTION: return HbaseMetastoreProto.LockType.INTENTION; + default: throw new RuntimeException("Unknown lock type " + thrift.toString()); + } + } + + public static LockResponse pbToThrift(HbaseMetastoreProto.LockResponse pb) { + return new LockResponse(-1, pbToThrift(pb.getState())); + } + + private static LockState pbToThrift(HbaseMetastoreProto.LockState pb) { + switch (pb) { + case ACQUIRED: return LockState.ACQUIRED; + case WAITING: return LockState.WAITING; + case TXN_ABORTED: return LockState.ABORT; + case RELEASED: throw new RuntimeException("Need to fix this"); + default: throw new RuntimeException("Unknown lock state " + pb.toString()); + } + } + + private static LockType pbToThrift(HbaseMetastoreProto.LockType pb) { + switch (pb) { + case INTENTION: return LockType.INTENTION; + case SHARED_READ: return LockType.SHARED_READ; + case SHARED_WRITE: return LockType.SHARED_WRITE; + case EXCLUSIVE: return LockType.EXCLUSIVE; + default: throw new RuntimeException("Unknown lock type " + pb.toString()); + } + } + + public static HbaseMetastoreProto.TransactionId thriftToPb(CheckLockRequest thrift) { + return HbaseMetastoreProto.TransactionId.newBuilder() + .setId(thrift.getTxnid()) + .build(); + } + + public static HbaseMetastoreProto.HeartbeatTxnRangeRequest thriftToPb(HeartbeatTxnRangeRequest thrift) { + return HbaseMetastoreProto.HeartbeatTxnRangeRequest.newBuilder() + .setMinTxn(thrift.getMin()) + .setMaxTxn(thrift.getMax()) + .build(); + } + + public static HeartbeatTxnRangeResponse pbToThrift(HbaseMetastoreProto.HeartbeatTxnRangeResponse pb) { + return new HeartbeatTxnRangeResponse( + new HashSet<>(pb.getAbortedList()), + new HashSet<>(pb.getNoSuchList())); + } + + public static HbaseMetastoreProto.CompactionType thriftToPb(CompactionType thrift) { + switch (thrift) { + case MINOR: return HbaseMetastoreProto.CompactionType.MINOR; + case MAJOR: return HbaseMetastoreProto.CompactionType.MAJOR; + default: throw new RuntimeException("Unknown compaction type " + thrift.toString()); + } + } + + public static ShowCompactResponseElement pbToThrift(HbaseMetastoreProto.Compaction pb) { + ShowCompactResponseElement thrift = new ShowCompactResponseElement(); + thrift.setDbname(pb.getDb()); + thrift.setTablename(pb.getTable()); + if (pb.hasPartition()) thrift.setPartitionname(pb.getPartition()); + thrift.setType(pbToThrift(pb.getType())); + thrift.setState(pb.getState().toString()); + if (pb.hasWorkerId()) thrift.setWorkerid(pb.getWorkerId()); + if (pb.hasStartedWorkingAt()) thrift.setStart(pb.getStartedWorkingAt()); + if (pb.hasRunAs()) thrift.setRunAs(pb.getRunAs()); + if (pb.hasHighestTxnId()) thrift.setHightestTxnId(pb.getHighestTxnId()); + if (pb.hasMetaInfo()) thrift.setMetaInfo(pb.getMetaInfo()); + if (pb.hasEndTime()) thrift.setEndTime(pb.getEndTime()); + if (pb.hasHadoopJobId()) thrift.setHadoopJobId(pb.getHadoopJobId()); + return thrift; + } + + private static CompactionType pbToThrift(HbaseMetastoreProto.CompactionType pb) { + switch (pb) { + case MINOR: return CompactionType.MINOR; + case MAJOR: return CompactionType.MAJOR; + default: throw new RuntimeException("Unknown compaction type " + pb.toString()); + } + } + + public static HbaseMetastoreProto.AddDynamicPartitionsRequest thriftToPb(AddDynamicPartitions thrift) { + return HbaseMetastoreProto.AddDynamicPartitionsRequest.newBuilder() + .setTxnId(thrift.getTxnid()) + .setDb(thrift.getDbname()) + .setTable(thrift.getTablename()) + .addAllPartitions(thrift.getPartitionnames()) + .build(); + } + + public static ShowLocksResponseElement pbLockToThriftShowLock(HbaseMetastoreProto.Transaction txn, + HbaseMetastoreProto.Transaction.Lock lock) { + ShowLocksResponseElement thrift = new ShowLocksResponseElement(lock.getId(), lock.getDb(), + pbToThrift(lock.getState()), pbToThrift(lock.getType()), 0, txn.getUser(), txn.getHostname()); + if (lock.hasTable()) thrift.setTablename(lock.getTable()); + if (lock.hasPartition()) thrift.setPartname(lock.getPartition()); + if (lock.hasAcquiredAt()) thrift.setAcquiredat(lock.getAcquiredAt()); + if (txn.hasAgentInfo()) thrift.setAgentInfo(txn.getAgentInfo()); + return thrift; + } + + public static CompactionInfo pbToCompactor(HbaseMetastoreProto.PotentialCompaction pb) { + CompactionInfo ci = new CompactionInfo(); + ci.dbname = pb.getDb(); + ci.tableName = pb.getTable(); + if (pb.hasPartition()) ci.partName = pb.getPartition(); + ci.numTxns = pb.getTxnIdsCount(); + return ci; + } + + public static CompactionInfo pbToCompactor(HbaseMetastoreProto.Compaction pb) { + CompactionInfo ci = new CompactionInfo(); + ci.id = pb.getId(); + ci.dbname = pb.getDb(); + ci.tableName = pb.getTable(); + if (pb.hasPartition()) ci.partName = pb.getPartition(); + ci.type = pbToThrift(pb.getType()); + if (pb.hasWorkerId()) ci.workerId = pb.getWorkerId(); + if (pb.hasStartedWorkingAt()) ci.start = pb.getStartedWorkingAt(); + if (pb.hasRunAs()) ci.runAs = pb.getRunAs(); + if (pb.hasHighestTxnId()) ci.highestTxnId = pb.getHighestTxnId(); + if (pb.hasMetaInfo()) ci.metaInfo = pb.getMetaInfo().getBytes(); + if (pb.hasHadoopJobId()) ci.hadoopJobId = pb.getHadoopJobId(); + return ci; + } + + /** + * Serialize a compaction. + * @param compaction compaction to serialize + * @return two byte arrays, first contains the key, the second the serialized value. + */ + static byte[][] serializeCompaction(HbaseMetastoreProto.Compaction compaction) { + byte[][] result = new byte[2][]; + result[0] = buildKey(Long.toString(compaction.getId())); + result[1] = compaction.toByteArray(); + return result; + } + + /** + * Deserialize a compaction. + * @param value value fetched from hbase + * @return A compaction + * @throws InvalidProtocolBufferException + */ + static HbaseMetastoreProto.Compaction deserializeCompaction(byte[] value) + throws InvalidProtocolBufferException { + return HbaseMetastoreProto.Compaction.parseFrom(value); + } + + /** + * Serialize a potential compaction. + * @param potential potentialCompaction to serialize + * @return two byte arrays, first contains the key, the second the serialized value. + */ + static byte[][] serializePotentialCompaction(HbaseMetastoreProto.PotentialCompaction potential) { + byte[][] result = new byte[2][]; + if (potential.hasPartition()) { + result[0] = buildKey(potential.getDb(), potential.getTable(), potential.getPartition()); + } else { + result[0] = buildKey(potential.getDb(), potential.getTable()); + } + result[1] = potential.toByteArray(); + return result; + } + + /** + * Deserialize a potential compaction. + * @param value value fetched from hbase + * @return A potentialCompaction + * @throws InvalidProtocolBufferException + */ + static HbaseMetastoreProto.PotentialCompaction deserializePotentialCompaction(byte[] value) + throws InvalidProtocolBufferException { + return HbaseMetastoreProto.PotentialCompaction.parseFrom(value); + } + + /** * @param keyStart byte array representing the start prefix * @return byte array corresponding to the next possible prefix */ diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/HBaseTxnHandler.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/HBaseTxnHandler.java new file mode 100644 index 0000000..eafa277 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/HBaseTxnHandler.java @@ -0,0 +1,994 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; +import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions; +import org.apache.hadoop.hive.metastore.api.CheckLockRequest; +import org.apache.hadoop.hive.metastore.api.CommitTxnRequest; +import org.apache.hadoop.hive.metastore.api.CompactionRequest; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; +import org.apache.hadoop.hive.metastore.api.HeartbeatRequest; +import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeRequest; +import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse; +import org.apache.hadoop.hive.metastore.api.HiveObjectType; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchLockException; +import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; +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.ShowLocksRequest; +import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.TxnAbortedException; +import org.apache.hadoop.hive.metastore.api.TxnInfo; +import org.apache.hadoop.hive.metastore.api.TxnOpenException; +import org.apache.hadoop.hive.metastore.api.UnlockRequest; +import org.apache.hadoop.hive.metastore.hbase.HBaseReadWrite; +import org.apache.hadoop.hive.metastore.hbase.HBaseStore; +import org.apache.hadoop.hive.metastore.hbase.HBaseUtils; +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; +import org.apache.hadoop.hive.metastore.hbase.txn.txnmgr.TransactionCoprocessor; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class HBaseTxnHandler implements TxnStore { + static final private Logger LOG = LoggerFactory.getLogger(HBaseTxnHandler.class.getName()); + + private HBaseReadWrite hbase = null; + private HiveConf conf; + + @Override + public void setConf(HiveConf conf) { + this.conf = conf; + } + + @Override + public GetOpenTxnsInfoResponse getOpenTxnsInfo() throws MetaException { + // We have to go to the table to get this information because much of it isn't kept in memory. + try { + getHBase().begin(); + List txns = getHBase().scanTransactions(null); + long hwm = getHBase().peekAtSequence(HBaseReadWrite.TXN_SEQUENCE); + List openTxns = new ArrayList<>(txns.size()); + for (HbaseMetastoreProto.Transaction txn : txns) openTxns.add(HBaseUtils.pbToThrift(txn)); + return new GetOpenTxnsInfoResponse(hwm, openTxns); + } catch (IOException e) { + LOG.error("Failed to scan transactions", e); + throw new MetaException(e.getMessage()); + } finally { + // It's a read only operation, so always commit + getHBase().commit(); + } + } + + @Override + public GetOpenTxnsResponse getOpenTxns() throws MetaException { + // No HBase txn as we're relying on the co-processor here + Batch.Call call = + new Batch.Call() { + @Override + public HbaseMetastoreProto.GetOpenTxnsResponse call(TransactionCoprocessor txnMgr) throws IOException { + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback<>(); + txnMgr.getOpenTxns(null, HbaseMetastoreProto.Void.getDefaultInstance(), rpcCallback); + return rpcCallback.get(); + } + }; + try { + return HBaseUtils.pbToThrift(getHBase().callTransactionManager(call)); + } catch (Throwable e) { + LOG.error("Failed to get open transactions", e); + throw new MetaException(e.getMessage()); + } + } + + @Override + public OpenTxnsResponse openTxns(OpenTxnRequest rqst) throws MetaException { + // No HBase txn as we're relying on the co-processor here + final HbaseMetastoreProto.OpenTxnsRequest pbRqst = HBaseUtils.thriftToPb(rqst); + Batch.Call call = + new Batch.Call() { + @Override + public HbaseMetastoreProto.OpenTxnsResponse call(TransactionCoprocessor txnMgr) throws + IOException { + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback<>(); + txnMgr.openTxns(null, pbRqst, rpcCallback); + return rpcCallback.get(); + } + }; + try { + return HBaseUtils.pbToThrift(getHBase().callTransactionManager(call)); + } catch (Throwable e) { + LOG.error("Failed to open transactions", e); + throw new MetaException(e.getMessage()); + } + } + + @Override + public void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaException { + // No HBase txn as we're relying on the co-processor here + final HbaseMetastoreProto.TransactionId pbRqst = HBaseUtils.thriftToPb(rqst); + Batch.Call call = + new Batch.Call() { + @Override + public HbaseMetastoreProto.TransactionResult call(TransactionCoprocessor txnMgr) throws + IOException { + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback<>(); + txnMgr.abortTxn(null, pbRqst, rpcCallback); + return rpcCallback.get(); + } + }; + try { + // Thrift doesn't return a value for abort, even though the co-processor does. + getHBase().callTransactionManager(call); + } catch (Throwable e) { + LOG.error("Failed to abort transaction", e); + throw new MetaException(e.getMessage()); + } + } + + @Override + public void commitTxn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbortedException, + MetaException { + // No HBase txn as we're relying on the co-processor here + final HbaseMetastoreProto.TransactionId pbRqst = HBaseUtils.thriftToPb(rqst); + Batch.Call call = + new Batch.Call() { + @Override + public HbaseMetastoreProto.TransactionResult call(TransactionCoprocessor txnMgr) throws + IOException { + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback<>(); + txnMgr.commitTxn(null, pbRqst, rpcCallback); + return rpcCallback.get(); + } + }; + try { + // Thrift doesn't return a value for abort, even though the co-processor does. + getHBase().callTransactionManager(call); + } catch (Throwable e) { + LOG.error("Failed to abort transaction", e); + throw new MetaException(e.getMessage()); + } + } + + @Override + public LockResponse lock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, + MetaException { + // No HBase txn as we're relying on the co-processor here + if (!rqst.isSetTxnid()) { + throw new MetaException("You must now set a transaction id when requesting locks"); + } + + final HbaseMetastoreProto.LockRequest pbRqst = HBaseUtils.thriftToPb(rqst); + Batch.Call call = + new Batch.Call() { + @Override + public HbaseMetastoreProto.LockResponse call(TransactionCoprocessor txnMgr) throws + IOException { + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback<>(); + txnMgr.lock(null, pbRqst, rpcCallback); + return rpcCallback.get(); + } + }; + try { + return HBaseUtils.pbToThrift(getHBase().callTransactionManager(call)); + } catch (Throwable e) { + LOG.error("Failed to get locks", e); + throw new MetaException(e.getMessage()); + } + } + + @Override + public LockResponse checkLock(CheckLockRequest rqst) throws NoSuchTxnException, + NoSuchLockException, TxnAbortedException, MetaException { + // No HBase txn as we're relying on the co-processor here + if (!rqst.isSetTxnid()) { + throw new MetaException("You must now set a transaction id when requesting locks"); + } + final HbaseMetastoreProto.TransactionId pbRqst = HBaseUtils.thriftToPb(rqst); + Batch.Call call = + new Batch.Call() { + @Override + public HbaseMetastoreProto.LockResponse call(TransactionCoprocessor txnMgr) throws + IOException { + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback<>(); + txnMgr.checkLocks(null, pbRqst, rpcCallback); + return rpcCallback.get(); + } + }; + try { + return HBaseUtils.pbToThrift(getHBase().callTransactionManager(call)); + } catch (Throwable e) { + LOG.error("Failed to check locks", e); + throw new MetaException(e.getMessage()); + } + } + + @Override + public void unlock(UnlockRequest rqst) throws NoSuchLockException, TxnOpenException, + MetaException { + throw new TxnOpenException("All locks must now be part of a txn, unlocking not allowed."); + + } + + // HBase filters to only pick records with at least one matching lock + private abstract static class LockFilter extends FilterBase { + protected HbaseMetastoreProto.Transaction getTxn(Cell cell) throws IOException { + ByteArrayInputStream is = new ByteArrayInputStream(cell.getValueArray(), cell + .getValueOffset(), cell.getValueLength()); + return HBaseUtils.deserializeTransaction(is); + } + + } + + // Return any transaction with a lock + private static class AnyLockFilter extends LockFilter { + @Override + public ReturnCode filterKeyValue(Cell cell) throws IOException { + HbaseMetastoreProto.Transaction txn = getTxn(cell); + if (txn.getLocksCount() > 0) return ReturnCode.INCLUDE; + else return ReturnCode.NEXT_ROW; + } + } + + // Return only transactions with at least one lock with a matching database + private static class DbLockFilter extends LockFilter { + private final String db; + + public DbLockFilter(String db) { + this.db = db; + } + + @Override + public ReturnCode filterKeyValue(Cell cell) throws IOException { + // If we find any lock that matches return this transaction, then the lock selectors on the + // other end will filter out only the appropriate locks + HbaseMetastoreProto.Transaction txn = getTxn(cell); + if (txn.getLocksCount() > 0) { + for (HbaseMetastoreProto.Transaction.Lock lock : txn.getLocksList()) { + if (lock.getDb().equals(db)) return ReturnCode.INCLUDE; + } + } + return ReturnCode.NEXT_ROW; + } + } + + // Return only transactions with at least one lock with a matching database and table + private static class TableLockFilter extends LockFilter { + private final String db; + private final String table; + + public TableLockFilter(String db, String table) { + this.db = db; + this.table = table; + } + + @Override + public ReturnCode filterKeyValue(Cell cell) throws IOException { + // If we find any lock that matches return this transaction, then the lock selectors on the + // other end will filter out only the appropriate locks + HbaseMetastoreProto.Transaction txn = getTxn(cell); + if (txn.getLocksCount() > 0) { + for (HbaseMetastoreProto.Transaction.Lock lock : txn.getLocksList()) { + if (lock.getDb().equals(db) && lock.hasTable() && lock.getTable().equals(table)) { + return ReturnCode.INCLUDE; + } + } + } + return ReturnCode.NEXT_ROW; + } + } + + // Return only transactions with at least one lock with a matching database, table, and partition + private static class PartitionLockFilter extends LockFilter { + private final String db; + private final String table; + private final String partition; + + public PartitionLockFilter(String db, String table, String partition) { + this.db = db; + this.table = table; + this.partition = partition; + } + + @Override + public ReturnCode filterKeyValue(Cell cell) throws IOException { + // If we find any lock that matches return this transaction, then the lock selectors on the + // other end will filter out only the appropriate locks + HbaseMetastoreProto.Transaction txn = getTxn(cell); + if (txn.getLocksCount() > 0) { + for (HbaseMetastoreProto.Transaction.Lock lock : txn.getLocksList()) { + if (lock.getDb().equals(db) && lock.hasTable() && lock.getTable().equals(table) && + lock.hasPartition() && lock.getPartition().equals(partition)) { + return ReturnCode.INCLUDE; + } + } + } + return ReturnCode.NEXT_ROW; + } + } + + // This interface is used to work through locks once we're on the client side. + private interface LockSelector { + void filterLock(HbaseMetastoreProto.Transaction txn, + HbaseMetastoreProto.Transaction.Lock lock); + } + + private static class AllLockSelector implements LockSelector { + final ShowLocksResponse rsp; + + public AllLockSelector(ShowLocksResponse rsp) { + this.rsp = rsp; + } + + @Override + public void filterLock(HbaseMetastoreProto.Transaction txn, + HbaseMetastoreProto.Transaction.Lock lock) { + rsp.addToLocks(HBaseUtils.pbLockToThriftShowLock(txn, lock)); + } + } + + private static class DbLockSelector implements LockSelector { + final ShowLocksResponse rsp; + final ShowLocksRequest rqst; + + public DbLockSelector(ShowLocksResponse rsp, + ShowLocksRequest rqst) { + this.rsp = rsp; + this.rqst = rqst; + } + + @Override + public void filterLock(HbaseMetastoreProto.Transaction txn, + HbaseMetastoreProto.Transaction.Lock lock) { + if (lock.getDb().equals(rqst.getDbname())) { + rsp.addToLocks(HBaseUtils.pbLockToThriftShowLock(txn, lock)); + } + } + }; + + private static class TableLockSelector implements LockSelector { + final ShowLocksResponse rsp; + final ShowLocksRequest rqst; + + public TableLockSelector(ShowLocksResponse rsp, + ShowLocksRequest rqst) { + this.rsp = rsp; + this.rqst = rqst; + } + + @Override + public void filterLock(HbaseMetastoreProto.Transaction txn, + HbaseMetastoreProto.Transaction.Lock lock) { + if (lock.hasTable() && lock.getDb().equals(rqst.getDbname()) && + lock.getTable().equals(rqst.getTablename())) { + rsp.addToLocks(HBaseUtils.pbLockToThriftShowLock(txn, lock)); + } + } + }; + + + private static class PartitionLockSelector implements LockSelector { + final ShowLocksResponse rsp; + final ShowLocksRequest rqst; + + public PartitionLockSelector(ShowLocksResponse rsp, + ShowLocksRequest rqst) { + this.rsp = rsp; + this.rqst = rqst; + } + + @Override + public void filterLock(HbaseMetastoreProto.Transaction txn, + HbaseMetastoreProto.Transaction.Lock lock) { + if (lock.hasTable() && lock.hasPartition() && lock.getDb().equals(rqst.getDbname()) && + lock.getTable().equals(rqst.getTablename()) && + lock.getPartition().equals(rqst.getPartname())) { + rsp.addToLocks(HBaseUtils.pbLockToThriftShowLock(txn, lock)); + } + } + }; + + @Override + public ShowLocksResponse showLocks(ShowLocksRequest rqst) throws MetaException { + // Show locks filters at both the server side and the client side. On the server side we + // pass a filter to only return transactions with at least one matching lock. Then on the + // client side we pull out only the locks that match the criteria and send those in the + // response. + ShowLocksResponse rsp = new ShowLocksResponse(); + LockSelector selector; + Filter filter; + if (!rqst.isSetDbname()) { + selector = new AllLockSelector(rsp); + filter = new AnyLockFilter(); + } else if (!rqst.isSetTablename()) { + selector = new DbLockSelector(rsp, rqst); + filter = new DbLockFilter(rqst.getDbname()); + } else if (!rqst.isSetPartname()) { + selector = new TableLockSelector(rsp, rqst); + filter = new TableLockFilter(rqst.getDbname(), rqst.getTablename()); + } else { + selector = new PartitionLockSelector(rsp, rqst); + filter = new PartitionLockFilter(rqst.getDbname(), rqst.getTablename(), rqst.getPartname()); + } + + getHBase().begin(); + try { + List hbaseTxns = getHBase().scanTransactions(filter); + for (HbaseMetastoreProto.Transaction hbaseTxn : hbaseTxns) { + if (hbaseTxn.getTxnState() == HbaseMetastoreProto.TxnState.OPEN && + hbaseTxn.getLocksCount() > 0) { + for (HbaseMetastoreProto.Transaction.Lock hbaseLock : hbaseTxn.getLocksList()) { + selector.filterLock(hbaseTxn, hbaseLock); + } + } + } + } catch (IOException e) { + LOG.error("Failed to show locks", e); + throw new MetaException(e.getMessage()); + } finally { + getHBase().commit(); + } + return rsp; + } + + @Override + public void heartbeat(HeartbeatRequest ids) throws NoSuchTxnException, NoSuchLockException, + TxnAbortedException, MetaException { + if (!ids.isSetTxnid()) { + throw new NoSuchLockException("You must now set a transaction id when heartbeating"); + } + heartbeatTxnRange(new HeartbeatTxnRangeRequest(ids.getTxnid(), ids.getTxnid())); + } + + @Override + public HeartbeatTxnRangeResponse heartbeatTxnRange(HeartbeatTxnRangeRequest rqst) throws + MetaException { + // No HBase txn as we're relying on the co-processor here + final HbaseMetastoreProto.HeartbeatTxnRangeRequest pbRqst = HBaseUtils.thriftToPb(rqst); + Batch.Call call = + new Batch.Call() { + @Override + public HbaseMetastoreProto.HeartbeatTxnRangeResponse call(TransactionCoprocessor txnMgr) throws + IOException { + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback<>(); + txnMgr.heartbeat(null, pbRqst, rpcCallback); + return rpcCallback.get(); + } + }; + try { + return HBaseUtils.pbToThrift(getHBase().callTransactionManager(call)); + } catch (Throwable e) { + LOG.error("Failed to heartbeat", e); + throw new MetaException(e.getMessage()); + } + } + + @Override + public long compact(CompactionRequest rqst) throws MetaException { + boolean shouldCommit = false; + try { + getHBase().begin(); + HbaseMetastoreProto.Compaction.Builder builder = HbaseMetastoreProto.Compaction.newBuilder(); + long compactionId = getHBase().getNextSequence(HBaseReadWrite.COMPACTION_SEQUENCE); + builder.setId(compactionId); + builder.setDb(rqst.getDbname()); + builder.setTable(rqst.getTablename()); + if (rqst.isSetPartitionname()) builder.setPartition(rqst.getPartitionname()); + builder.setState(HbaseMetastoreProto.CompactionState.INITIATED); + builder.setType(HBaseUtils.thriftToPb(rqst.getType())); + getHBase().putCompaction(builder.build()); + shouldCommit = true; + return compactionId; + } catch (IOException e) { + LOG.error("Failed to request compaction", e); + throw new MetaException(e.getMessage()); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + @Override + public ShowCompactResponse showCompact(ShowCompactRequest rqst) throws MetaException { + try { + getHBase().begin(); + List compactions = getHBase().scanCompactions(null); + List elements = new ArrayList<>(compactions.size()); + for (HbaseMetastoreProto.Compaction compaction : compactions) { + elements.add(HBaseUtils.pbToThrift(compaction)); + } + return new ShowCompactResponse(elements); + } catch (IOException e) { + LOG.error("Failed to get compactions", e); + throw new MetaException(e.getMessage()); + } finally { + // It's a read only operation, so always commit + getHBase().commit(); + } + } + + @Override + public void addDynamicPartitions(AddDynamicPartitions rqst) throws NoSuchTxnException, + TxnAbortedException, MetaException { + // No HBase txn as we're relying on the co-processor here + final HbaseMetastoreProto.AddDynamicPartitionsRequest pbRqst = HBaseUtils.thriftToPb(rqst); + Batch.Call call = + new Batch.Call() { + @Override + public HbaseMetastoreProto.TransactionResult call(TransactionCoprocessor txnMgr) throws + IOException { + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback<>(); + txnMgr.addDynamicPartitions(null, pbRqst, rpcCallback); + return rpcCallback.get(); + } + }; + try { + getHBase().callTransactionManager(call); + } catch (Throwable e) { + LOG.error("Failed to heartbeat", e); + throw new MetaException(e.getMessage()); + } + } + + @Override + public void cleanupRecords(HiveObjectType type, Database db, Table table, + Iterator partitionIterator) throws MetaException { + // TODO - NOP for now, not sure what should be done here + } + + @Override + public void performTimeOuts() { + // NOP + } + + @Override + public Set findPotentialCompactions(int maxAborted) throws MetaException { + try { + getHBase().begin(); + Set cis = new HashSet<>(); + Iterator iter = + getHBase().scanPotentialCompactions(); + while (iter.hasNext()) cis.add(HBaseUtils.pbToCompactor(iter.next())); + return cis; + } catch (IOException e) { + LOG.error("Failed to find potential compactions", e); + throw new MetaException(e.getMessage()); + } finally { + // It's a read only operation, so always commit + getHBase().commit(); + } + } + + @Override + public void setRunAs(long cq_id, String user) throws MetaException { + boolean shouldCommit = false; + try { + getHBase().begin(); + HbaseMetastoreProto.Compaction compaction = getHBase().getCompaction(cq_id); + HbaseMetastoreProto.Compaction newCompaction = + HbaseMetastoreProto.Compaction.newBuilder(compaction) + .setRunAs(user) + .build(); + getHBase().putCompaction(newCompaction); + shouldCommit = true; + } catch (IOException e) { + LOG.error("Failed to set run as", e); + throw new MetaException(e.getMessage()); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + @Override + public CompactionInfo findNextToCompact(String workerId) throws MetaException { + boolean shouldCommit = false; + try { + getHBase().begin(); + List initiated = + getHBase().scanCompactions(HbaseMetastoreProto.CompactionState.INITIATED); + if (initiated.size() == 0) return null; + + // Pick the first one and set the worker id and the start time + HbaseMetastoreProto.Compaction toWorkOn = + HbaseMetastoreProto.Compaction.newBuilder(initiated.get(0)) + .setState(HbaseMetastoreProto.CompactionState.WORKING) + .setWorkerId(workerId) + .setStartedWorkingAt(System.currentTimeMillis()) + .build(); + getHBase().putCompaction(toWorkOn); + shouldCommit = true; + return HBaseUtils.pbToCompactor(toWorkOn); + } catch (IOException e) { + LOG.error("Failed to find next compaction to work on", e); + throw new MetaException(e.getMessage()); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + @Override + public void markCompacted(CompactionInfo info) throws MetaException { + boolean shouldCommit = false; + try { + getHBase().begin(); + final HbaseMetastoreProto.Compaction pbRqst = getHBase().getCompaction(info.id); + if (pbRqst == null) { + throw new MetaException("No such compaction " + info.id); + } + Batch.Call call = + new Batch.Call() { + @Override + public HbaseMetastoreProto.Void call(TransactionCoprocessor txnMgr) throws IOException { + BlockingRpcCallback rpcCallback = new BlockingRpcCallback<>(); + txnMgr.cleanupAfterCompaction(null, pbRqst, rpcCallback); + return rpcCallback.get(); + } + }; + getHBase().callTransactionManager(call); + + HbaseMetastoreProto.Compaction toWorkOn = + HbaseMetastoreProto.Compaction.newBuilder(pbRqst) + .setState(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING) + .build(); + getHBase().putCompaction(toWorkOn); + shouldCommit = true; + } catch (Throwable e) { + LOG.error("Failed to cleanup after compaction", e); + throw new MetaException(e.getMessage()); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + @Override + public List findReadyToClean() throws MetaException { + boolean shouldCommit = false; + try { + getHBase().begin(); + List compacted = + getHBase().scanCompactions(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING); + if (compacted.size() == 0) return null; + + // For each of these we also need to assure that all overlapping transactions have already + // completed. Otherwise we have to not clean yet because there could still be readers + // making use of the old files + final HbaseMetastoreProto.CompactionList pbRqst = HbaseMetastoreProto.CompactionList + .newBuilder() + .addAllCompactions(compacted) + .build(); + Batch.Call call = + new Batch.Call() { + @Override + public HbaseMetastoreProto.CompactionList call(TransactionCoprocessor txnMgr) throws IOException { + BlockingRpcCallback rpcCallback = new BlockingRpcCallback<>(); + txnMgr.verifyCompactionCanBeCleaned(null, pbRqst, rpcCallback); + return rpcCallback.get(); + } + }; + HbaseMetastoreProto.CompactionList cleaningList = getHBase().callTransactionManager(call); + + // Put each of these in the cleaning state and return them + List cleanable = + new ArrayList<>(cleaningList.getCompactionsCount()); + for (HbaseMetastoreProto.Compaction compaction : cleaningList.getCompactionsList()) { + cleanable.add(HbaseMetastoreProto.Compaction.newBuilder(compaction) + .setState(HbaseMetastoreProto.CompactionState.CLEANING) + .build()); + } + + getHBase().putCompactions(cleanable); + + List returns = new ArrayList<>(cleanable.size()); + for (HbaseMetastoreProto.Compaction compaction : cleanable) { + returns.add(HBaseUtils.pbToCompactor(compaction)); + } + shouldCommit = true; + return returns; + } catch (Throwable e) { + LOG.error("Failed to find next compaction to work on", e); + throw new MetaException(e.getMessage()); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + @Override + public void markCleaned(CompactionInfo info) throws MetaException { + changeCompactionState(info, HbaseMetastoreProto.CompactionState.SUCCEEDED); + } + + @Override + public void markFailed(CompactionInfo info) throws MetaException { + changeCompactionState(info, HbaseMetastoreProto.CompactionState.FAILED); + } + + private void changeCompactionState(CompactionInfo info, + HbaseMetastoreProto.CompactionState state) + throws MetaException { + boolean shouldCommit = false; + try { + getHBase().begin(); + final HbaseMetastoreProto.Compaction pbRqst = getHBase().getCompaction(info.id); + if (pbRqst == null) { + throw new MetaException("No such compaction " + info.id); + } + + HbaseMetastoreProto.Compaction toMarkCleaned = + HbaseMetastoreProto.Compaction.newBuilder(pbRqst) + .setState(state) + .build(); + getHBase().putCompaction(toMarkCleaned); + shouldCommit = true; + } catch (IOException e) { + LOG.error("Failed to change compaction state", e); + throw new MetaException(e.getMessage()); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + @Override + public void cleanEmptyAbortedTxns() throws MetaException { + // NOP + } + + @Override + public void revokeFromLocalWorkers(String hostname) throws MetaException { + boolean shouldCommit = false; + try { + getHBase().begin(); + List working = + getHBase().scanCompactions(HbaseMetastoreProto.CompactionState.WORKING); + if (working.size() == 0) return; + + List revokable = new ArrayList<>(working.size()); + for (HbaseMetastoreProto.Compaction compaction : working) { + if (compaction.hasWorkerId() && compaction.getWorkerId().startsWith(hostname)) { + revokable.add(compaction); + } + } + if (revokable.size() == 0) return; + + List newCompactions = new ArrayList<>(revokable.size()); + for (HbaseMetastoreProto.Compaction compaction : revokable) { + newCompactions.add(HbaseMetastoreProto.Compaction.newBuilder(compaction) + .setState(HbaseMetastoreProto.CompactionState.INITIATED) + .clearWorkerId() + .clearStartedWorkingAt() + .build()); + } + + getHBase().putCompactions(newCompactions); + shouldCommit = true; + } catch (Throwable e) { + LOG.error("Failed to find next compaction to work on", e); + throw new MetaException(e.getMessage()); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + @Override + public void revokeTimedoutWorkers(long timeout) throws MetaException { + boolean shouldCommit = false; + try { + getHBase().begin(); + List working = + getHBase().scanCompactions(HbaseMetastoreProto.CompactionState.WORKING); + if (working.size() == 0) return; + + long mustBeStartedBefore = System.currentTimeMillis() - timeout; + List revokable = new ArrayList<>(working.size()); + for (HbaseMetastoreProto.Compaction compaction : working) { + if (compaction.hasStartedWorkingAt() && + compaction.getStartedWorkingAt() < mustBeStartedBefore) { + revokable.add(compaction); + } + } + if (revokable.size() == 0) return; + + List newCompactions = new ArrayList<>(revokable.size()); + for (HbaseMetastoreProto.Compaction compaction : revokable) { + newCompactions.add(HbaseMetastoreProto.Compaction.newBuilder(compaction) + .setState(HbaseMetastoreProto.CompactionState.INITIATED) + .clearWorkerId() + .clearStartedWorkingAt() + .build()); + } + + getHBase().putCompactions(newCompactions); + shouldCommit = true; + } catch (Throwable e) { + LOG.error("Failed to find next compaction to work on", e); + throw new MetaException(e.getMessage()); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + @Override + public List findColumnsWithStats(CompactionInfo ci) throws MetaException { + getHBase().begin(); + try { + return getHBase().getColumnsWithStatistics(ci.dbname, ci.tableName, + HBaseStore.partNameToVals(ci.partName)); + } catch (IOException e) { + LOG.error("Failed to find which columns had stats", e); + throw new MetaException(e.getMessage()); + } finally { + getHBase().commit(); + } + } + + @Override + public void setCompactionHighestTxnId(CompactionInfo ci, long highestTxnId) throws MetaException { + boolean shouldCommit = false; + try { + getHBase().begin(); + HbaseMetastoreProto.Compaction compaction = getHBase().getCompaction(ci.id); + HbaseMetastoreProto.Compaction newCompaction = + HbaseMetastoreProto.Compaction.newBuilder(compaction) + .setHighestTxnId(highestTxnId) + .build(); + getHBase().putCompaction(newCompaction); + shouldCommit = true; + } catch (IOException e) { + LOG.error("Failed to set highest txn id", e); + throw new MetaException(e.getMessage()); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + @Override + public void purgeCompactionHistory() throws MetaException { + boolean shouldCommit = false; + try { + getHBase().begin(); + List compactions = getHBase().scanCompactions(null); + if (compactions.size() == 0) return; + + TxnStore.RetentionCounters rc = new RetentionCounters( + conf.getIntVar(HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_ATTEMPTED), + TxnUtils.getFailedCompactionRetention(conf), + conf.getIntVar(HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_SUCCEEDED)); + + List deleteSet = new ArrayList<>(); + Map lookups = new HashMap<>(compactions.size()); + for (HbaseMetastoreProto.Compaction compaction : compactions) { + if (compaction.getState() == HbaseMetastoreProto.CompactionState.FAILED || + compaction.getState() == HbaseMetastoreProto.CompactionState.SUCCEEDED) { + lookups.put(compaction.getId(), compaction); + CompactionInfo ci = HBaseUtils.pbToCompactor(compaction); + TxnUtils.checkForDeletion(deleteSet, ci, rc); + } + } + + if (deleteSet.size() > 0) { + getHBase().deleteCompactions(deleteSet); + } + shouldCommit = true; + } catch (IOException e) { + LOG.error("Failed to purge compaction history", e); + throw new MetaException(e.getMessage()); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + @Override + public boolean checkFailedCompactions(CompactionInfo ci) throws MetaException { + try { + getHBase().begin(); + List compactions = + getHBase().scanCompactions(HbaseMetastoreProto.CompactionState.FAILED); + if (compactions.size() == 0) return false; + + int failedThreshold = conf.getIntVar(HiveConf.ConfVars.COMPACTOR_INITIATOR_FAILED_THRESHOLD); + + int numFails = 0; + for (HbaseMetastoreProto.Compaction compaction : compactions) { + if (compaction.getDb().equals(ci.dbname) && + compaction.getTable().equals(ci.tableName) && + (compaction.hasPartition() && compaction.getPartition().equals(ci.partName) || + !compaction.hasPartition() && ci.partName == null)) { + numFails++; + } + } + + return numFails > failedThreshold; + } catch (IOException e) { + LOG.error("Failed to purge compaction history", e); + throw new MetaException(e.getMessage()); + } finally { + // It's a read only operation, so always commit + getHBase().commit(); + } + } + + @Override + public int numLocksInLockTable() throws SQLException, MetaException { + return 0; + } + + @Override + public long setTimeout(long milliseconds) { + return 0; + } + + @Override + public MutexAPI getMutexAPI() { + // TODO - not sure what to do here + return null; + } + + private HBaseReadWrite getHBase() { + if (hbase == null) { + HBaseReadWrite.setConf(conf); + hbase = HBaseReadWrite.getInstance(); + } + return hbase; + } + +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/AbortedHiveTransaction.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/AbortedHiveTransaction.java new file mode 100644 index 0000000..54c5541 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/AbortedHiveTransaction.java @@ -0,0 +1,86 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +class AbortedHiveTransaction extends HiveTransaction { + + private Map compactableLocks; + + /** + * For use when creating a new aborted transaction. + * @param openTxn open transaction moving into aborted state. + */ + AbortedHiveTransaction(OpenHiveTransaction openTxn) { + super(openTxn.getId()); + compactableLocks = new HashMap<>(); + if (openTxn.getHiveLocks() != null) { + for (HiveLock lock : openTxn.getHiveLocks()) { + if (lock.getType() == HbaseMetastoreProto.LockType.SHARED_WRITE) { + compactableLocks.put(lock.getEntityLocked(), lock); + } + } + } + } + + /** + * For use when recovering transactions from HBase. + * @param hbaseTxn transaction record from HBase. + * @param txnMgr ptr to the transaction manager + * @throws IOException + */ + AbortedHiveTransaction(HbaseMetastoreProto.Transaction hbaseTxn, TransactionManager txnMgr) + throws IOException { + super(hbaseTxn.getId()); + compactableLocks = new HashMap<>(); + for (HbaseMetastoreProto.Transaction.Lock hbaseLock : hbaseTxn.getLocksList()) { + HiveLock hiveLock = new HiveLock(id, hbaseLock, txnMgr); + compactableLocks.put(hiveLock.getEntityLocked(), hiveLock); + } + } + + @Override + HbaseMetastoreProto.TxnState getState() { + return HbaseMetastoreProto.TxnState.ABORTED; + } + + /** + * Note that a dtp a lock is associated with has been compacted, so we can forget about the lock + * @param key dtp lock is associated with + */ + HiveLock compactLock(TransactionManager.EntityKey key) { + return compactableLocks.remove(key); + } + + /** + * Determine whether all dtps written to by an aborted transaction have been compacted. + * @return true if all have been compacted, false otherwise. + */ + boolean fullyCompacted() { + return compactableLocks.size() == 0; + } + + Map getCompactableLocks() { + return compactableLocks; + } +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/CommittedHiveTransaction.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/CommittedHiveTransaction.java new file mode 100644 index 0000000..7ab4e33 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/CommittedHiveTransaction.java @@ -0,0 +1,57 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; + +import java.io.IOException; + +class CommittedHiveTransaction extends HiveTransaction { + + private final long commitId; + + /** + * For use when creating a newly committed transaction. + * @param openTxn open transaction that is moving to a committed state. + * @param commitId id assigned at commit time. + */ + CommittedHiveTransaction(OpenHiveTransaction openTxn, long commitId) { + super(openTxn.getId()); + this.commitId = commitId; + } + + /** + * For use when recovering transactions from HBase. + * @param hbaseTxn transaction record from HBase. + * @throws IOException + */ + CommittedHiveTransaction(HbaseMetastoreProto.Transaction hbaseTxn) + throws IOException { + super(hbaseTxn.getId()); + this.commitId = hbaseTxn.getCommitId(); + } + + HbaseMetastoreProto.TxnState getState() { + return HbaseMetastoreProto.TxnState.COMMITTED; + } + + long getCommitId() { + return commitId; + } + +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/HiveLock.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/HiveLock.java new file mode 100644 index 0000000..ef0756f --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/HiveLock.java @@ -0,0 +1,97 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; + +import java.io.IOException; + +class HiveLock { + + private final long id; + // Transaction we are part of, needed so that we can backtrack to the txns when tracing via + // db/table/part + private final long txnId; + // Lock list this lock is part of + private final TransactionManager.EntityKey entityLocked; + private final HbaseMetastoreProto.LockType type; + private HbaseMetastoreProto.LockState state; + + /** + * For use when creating a new lock. This puts the lock in waiting state. + * @param id id for this lock + * @param txnId txn id this lock is part of + * @param type lock type + * @param entityLocked DTP list this will go in. + */ + HiveLock(long id, long txnId, HbaseMetastoreProto.LockType type, + TransactionManager.EntityKey entityLocked) { + this.id = id; + this.txnId = txnId; + this.type = type; + this.entityLocked = entityLocked; + state = HbaseMetastoreProto.LockState.WAITING; + } + + /** + * For use when recovering locks from HBase + * @param txnId txn id this lock is a part of + * @param hbaseLock lock record from HBase + * @param txnMgr transaction manager + * @throws IOException + */ + HiveLock(long txnId, HbaseMetastoreProto.Transaction.Lock hbaseLock, TransactionManager txnMgr) + throws IOException { + id = hbaseLock.getId(); + this.txnId = txnId; + entityLocked = txnMgr.getLockQueue(hbaseLock).getFirst(); + type = hbaseLock.getType(); + state = hbaseLock.getState(); + } + + long getId() { + return id; + } + + long getTxnId() { + return txnId; + } + + HbaseMetastoreProto.LockType getType() { + return type; + } + + HbaseMetastoreProto.LockState getState() { + return state; + } + + void setState(HbaseMetastoreProto.LockState state) { + this.state = state; + } + + TransactionManager.EntityKey getEntityLocked() { + return entityLocked; + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof HiveLock)) return false; + HiveLock other = (HiveLock)o; + return txnId == other.txnId && id == other.id; + } +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/HiveTransaction.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/HiveTransaction.java new file mode 100644 index 0000000..c312e74 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/HiveTransaction.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; + +abstract class HiveTransaction { + protected final long id; + + protected HiveTransaction(long id) { + this.id = id; + } + + final long getId() { + return id; + } + + abstract HbaseMetastoreProto.TxnState getState(); + + @Override + public boolean equals(Object o) { + if (o == null || !(o instanceof HiveTransaction)) return false; + HiveTransaction other = (HiveTransaction)o; + return id == other.id; + } + + @Override + public int hashCode() { + return (int)id; + } + +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/OpenHiveTransaction.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/OpenHiveTransaction.java new file mode 100644 index 0000000..80a150d --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/OpenHiveTransaction.java @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +class OpenHiveTransaction extends HiveTransaction { + private long lastHeartbeat; + + // I chose an array over a list so that I could explicitly control growth. ArrayList is memory + // efficient (only 4 more bytes than an array I believe) and you can control the initial + // capacity, but when it grows you loose control of how. + private HiveLock[] hiveLocks; + + /** + * For use when creating a new transaction. This creates the transaction in an open state. + * @param id txn id + */ + OpenHiveTransaction(long id) { + super(id); + lastHeartbeat = System.currentTimeMillis(); + } + + /** + * For use when recovering transactions from HBase. + * @param hbaseTxn transaction record from HBase. + * @param txnMgr transaction manager. + * @throws IOException + */ + OpenHiveTransaction(HbaseMetastoreProto.Transaction hbaseTxn, TransactionManager txnMgr) + throws IOException { + super(hbaseTxn.getId()); + lastHeartbeat = System.currentTimeMillis(); + List hbaseLocks = hbaseTxn.getLocksList(); + hiveLocks = new HiveLock[hbaseLocks.size()]; + for (int i = 0; i < hbaseLocks.size(); i++) { + hiveLocks[i] = new HiveLock(id, hbaseLocks.get(i), txnMgr); + // Insert the locks into the lockQueues. Since lockQueues is sorted they'll come out in + // the right order. + if (hiveLocks[i].getState() == HbaseMetastoreProto.LockState.WAITING || + hiveLocks[i].getState() == HbaseMetastoreProto.LockState.ACQUIRED) { + txnMgr.getLockQueue(hbaseLocks.get(i)).getSecond().queue.put(hiveLocks[i].getId(), + hiveLocks[i]); + } + } + } + + HbaseMetastoreProto.TxnState getState() { + return HbaseMetastoreProto.TxnState.OPEN; + } + + long getLastHeartbeat() { + return lastHeartbeat; + } + + void setLastHeartbeat(long lastHeartbeat) { + this.lastHeartbeat = lastHeartbeat; + } + + HiveLock[] getHiveLocks() { + return hiveLocks; + } + + /** + * Add locks to the transaction. + * @param newLocks array of locks to add. This method assumes it can take ownership of this + * array, so don't plan to do anything else with it. All your locks are belong to + * us. + */ + void addLocks(HiveLock[] newLocks) { + if (hiveLocks == null) { + hiveLocks = newLocks; + } else { + int origSize = hiveLocks.length; + hiveLocks = Arrays.copyOf(hiveLocks, origSize + newLocks.length); + System.arraycopy(newLocks, 0, hiveLocks, origSize, newLocks.length); + } + } + + boolean hasWriteLocks() { + if (hiveLocks != null) { + for (HiveLock hiveLock : hiveLocks) { + if (hiveLock.getType() == HbaseMetastoreProto.LockType.SHARED_WRITE) { + return true; + } + } + } + return false; + } +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/SeverusPleaseException.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/SeverusPleaseException.java new file mode 100644 index 0000000..e60b2a9 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/SeverusPleaseException.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +/** + * An exception that indicates to the TransactionCoprocessor that the TransactionManager should + * be killed and restarted. + */ +public class SeverusPleaseException extends Exception { + public SeverusPleaseException() { + } + + public SeverusPleaseException(String message) { + super(message); + } + + public SeverusPleaseException(String message, Throwable cause) { + super(message, cause); + } + + public SeverusPleaseException(Throwable cause) { + super(cause); + } + + public SeverusPleaseException(String message, Throwable cause, boolean enableSuppression, + boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TransactionCoprocessor.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TransactionCoprocessor.java new file mode 100644 index 0000000..c159000 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TransactionCoprocessor.java @@ -0,0 +1,250 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.CoprocessorService; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; + +import java.io.IOException; + +/** + * This forwards the calls to + * {@link org.apache.hadoop.hive.metastore.hbase.txn.txnmgr.TransactionManager}. It is separate + * from it to enable unit testing without the Protocol Buffer framework. + */ +public class TransactionCoprocessor extends HbaseMetastoreProto.TxnMgr + implements Coprocessor, CoprocessorService { + // Don't ever grab this directly, always use getTxnMgr + private static TransactionManager txnMgr = null; + + private Configuration conf; + + private TransactionManager getTxnMgr() throws IOException { + return txnMgr; + } + + private void restart(Configuration conf) throws IOException { + synchronized (TransactionCoprocessor.class) { + if (txnMgr != null) txnMgr.shutdown(); + txnMgr = new TransactionManager(conf); + } + } + + private void shutdown() throws IOException { + synchronized (TransactionCoprocessor.class) { + if (txnMgr != null) txnMgr.shutdown(); + } + } + + @Override + public void start(CoprocessorEnvironment coprocessorEnvironment) throws IOException { + conf = coprocessorEnvironment.getConfiguration(); + synchronized (TransactionCoprocessor.class) { + txnMgr = new TransactionManager(conf); + } + } + + @Override + public void stop(CoprocessorEnvironment coprocessorEnvironment) throws IOException { + shutdown(); + } + + @Override + public Service getService() { + return this; + } + + @Override + public void openTxns(RpcController controller, HbaseMetastoreProto.OpenTxnsRequest request, + RpcCallback done) { + HbaseMetastoreProto.OpenTxnsResponse response = null; + try { + try { + response = getTxnMgr().openTxns(request); + } catch (SeverusPleaseException e) { + restart(conf); + } + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + } + done.run(response); + + } + + @Override + public void getOpenTxns(RpcController controller, HbaseMetastoreProto.Void request, + RpcCallback done) { + HbaseMetastoreProto.GetOpenTxnsResponse response = null; + try { + try { + response = getTxnMgr().getOpenTxns(request); + } catch (SeverusPleaseException e) { + restart(conf); + } + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + } + done.run(response); + } + + @Override + public void abortTxn(RpcController controller, HbaseMetastoreProto.TransactionId request, + RpcCallback done) { + HbaseMetastoreProto.TransactionResult response = null; + try { + try { + response = getTxnMgr().abortTxn(request); + } catch (SeverusPleaseException e) { + restart(conf); + } + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + } + done.run(response); + } + + @Override + public void commitTxn(RpcController controller, HbaseMetastoreProto.TransactionId request, + RpcCallback done) { + HbaseMetastoreProto.TransactionResult response = null; + try { + try { + response = getTxnMgr().commitTxn(request); + } catch (SeverusPleaseException e) { + restart(conf); + } + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + } + done.run(response); + } + + @Override + public void heartbeat(RpcController controller, + HbaseMetastoreProto.HeartbeatTxnRangeRequest request, + RpcCallback done) { + HbaseMetastoreProto.HeartbeatTxnRangeResponse response = null; + try { + try { + response = getTxnMgr().heartbeat(request); + } catch (SeverusPleaseException e) { + restart(conf); + } + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + } + done.run(response); + } + + @Override + public void lock(RpcController controller, HbaseMetastoreProto.LockRequest request, + RpcCallback done) { + HbaseMetastoreProto.LockResponse response = null; + try { + try { + response = getTxnMgr().lock(request); + } catch (SeverusPleaseException e) { + restart(conf); + } + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + } + done.run(response); + } + + @Override + public void checkLocks(RpcController controller, HbaseMetastoreProto.TransactionId request, + RpcCallback done) { + HbaseMetastoreProto.LockResponse response = null; + try { + try { + response = getTxnMgr().checkLocks(request); + } catch (SeverusPleaseException e) { + restart(conf); + } + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + } + done.run(response); + } + + @Override + public void addDynamicPartitions(RpcController controller, + HbaseMetastoreProto.AddDynamicPartitionsRequest request, + RpcCallback done) { + HbaseMetastoreProto.TransactionResult response = null; + try { + try { + response = getTxnMgr().addDynamicPartitions(request); + } catch (SeverusPleaseException e) { + restart(conf); + } + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + } + done.run(response); + } + + @Override + public void cleanupAfterCompaction(RpcController controller, + HbaseMetastoreProto.Compaction request, + RpcCallback done) { + HbaseMetastoreProto.Void response = null; + try { + try { + response = getTxnMgr().cleanupAfterCompaction(request); + } catch (SeverusPleaseException e) { + restart(conf); + } + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + } + done.run(response); + } + + @Override + public void verifyCompactionCanBeCleaned(RpcController controller, + HbaseMetastoreProto.CompactionList request, + RpcCallback done) { + HbaseMetastoreProto.CompactionList response = null; + try { + try { + response = getTxnMgr().verifyCompactionCanBeCleaned(request); + } catch (SeverusPleaseException e) { + restart(conf); + } + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + } + done.run(response); + } + + @VisibleForTesting + public TransactionManager backdoor() { + return txnMgr; + } + +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TransactionManager.java metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TransactionManager.java new file mode 100644 index 0000000..8260560 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TransactionManager.java @@ -0,0 +1,1724 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.common.ObjectPair; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.hbase.HBaseReadWrite; +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * A transaction and lock manager written to work inside an HBase co-processor. This class keeps + * a lot of state in memory. It remembers all open transactions as well as committed ones that + * might be needed to avoid lost updates and aborted ones that haven't been compacted out yet. + * + * Locks are a part of a transaction, so each txn has a list of locks. All transaction oriented + * operations are done via txnId. + * + * Locks are also kept in queues which are sorted trees keyed by db, table, partition that + * the lock is on. This allows the system to quickly evaluate which locks should be granted next. + * + * All write operations are written through to HBase. When the class first starts it recovers + * the current state from HBase. + * + * All reads and writes to the Transaction table in HBase must be done through this + * class. Access to the table and internal memory structures is controlled by a read/write lock. + * + * All writes to the PotentialCompactions table are done here, reads are done by + * {@link org.apache.hadoop.hive.metastore.hbase.txn.HBaseTxnHandler} + */ +class TransactionManager { + + // TODO Convert Initiator et al to a thread pool so they don't accumulate memory over time. Do + // in a separate JIRA. + // TODO error in addDynamicPartitions, need to make sure we don't have a write-write conflict + // like we do in lock acquisition. + // TODO in commit and abort, don't copy over shared write locks if they are only wating not + // acquired + // TODO add new object types in HBaseSchemaTool + // TODO Write some tests so you have some clue if this works + + // Someday - handle lock promotion + + static final private Logger LOG = LoggerFactory.getLogger(TransactionManager.class.getName()); + + // This is for testing purposes so that we don't automatically start the background threads. + static final String CONF_NO_AUTO_BACKGROUND_THREADS = "txn.mgr.auto.background.threads"; + + // Track what locks types are compatible. First array is holder, second is requester + private static boolean[][] lockCompatibilityTable; + + // This lock needs to be acquired in write mode only when modifying structures (opening, + // aborting, committing txns, adding locks). To modify structures (ie heartbeat) it is + // only needed in the read mode. Anything looking at this structure should acquire it in the + // read mode. + private ReadWriteLock masterLock; + + // A list of all active transactions. Obtain the globalLock before changing this list. You + // can read this list without obtaining the global lock. + private Map openTxns; + + // List of aborted transactions, kept in memory for efficient reading when readers need a valid + // transaction list. + private Map abortedTxns; + + // A set of all committed transactions. + private Set committedTxns; + + private Configuration conf; + + // Protected by synchronized code section on openTxn; + private long nextTxnId; + + // Protected by synchronized code section on getLockId; + private long nextLockId; + + // A structure to store the locks according to which database/table/partition they lock. + private Map lockQueues; + + private HBaseReadWrite hbase; + + // Lock queues that should be checked for whether a lock can be acquired. Do not write to or + // read from this variable unless you own the write lock. + private List lockQueuesToCheck; + + // Thread pool to do all our dirty work + private ScheduledThreadPoolExecutor threadPool; + + // Configuration values cached to avoid re-reading config all the time. + private long lockPollTimeout; + private long txnTimeout; + private int maxObjects; + + // If true, then we've filled the capacity of this thing and we don't want to accept any new + // open transactions until some have bled off. + private boolean full; + + TransactionManager(Configuration conf) throws IOException { + LOG.info("Starting transaction manager"); + masterLock = new ReentrantReadWriteLock(); + // Don't set the values here, as for efficiency in iteration we want the size to match as + // closely as possible to the actual number of entries. + openTxns = new HashMap<>(); + abortedTxns = new HashMap<>(); + committedTxns = new HashSet<>(); + lockQueues = new HashMap<>(); + lockQueuesToCheck = new ArrayList<>(); + this.conf = conf; + lockPollTimeout = HiveConf.getTimeVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_LOCK_POLL_TIMEOUT, + TimeUnit.MILLISECONDS); + txnTimeout = HiveConf.getTimeVar(conf, HiveConf.ConfVars.HIVE_TXN_TIMEOUT, TimeUnit.MILLISECONDS); + maxObjects = HiveConf.getIntVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_MAX_OBJECTS); + recover(); + + threadPool = new ScheduledThreadPoolExecutor(HiveConf.getIntVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_THREAD_POOL_BASE_SIZE)); + threadPool.setMaximumPoolSize(HiveConf.getIntVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_THREAD_POOL_MAX_SIZE)); + + String noAutoBackgroundString = conf.get(CONF_NO_AUTO_BACKGROUND_THREADS); + + // This gives us a way to turn off the background threads when running unit tests so they + // don't run around at random intervals messing up the tests. + if (noAutoBackgroundString == null || !Boolean.parseBoolean(noAutoBackgroundString)) { + // Schedule the lock queue shrinker + long period = HiveConf.getTimeVar(conf, + HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_LOCK_QUEUE_SHRINKER_FREQ, + TimeUnit.MILLISECONDS); + threadPool.scheduleAtFixedRate(lockQueueShrinker, 10, period, TimeUnit.MILLISECONDS); + + // Schedule full checker + period = HiveConf.getTimeVar(conf, + HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_FULL_CHECKER_FREQ, TimeUnit.MILLISECONDS); + threadPool.scheduleAtFixedRate(fullChecker, 20, period, TimeUnit.MILLISECONDS); + + // Schedule the committed transaction cleaner + period = HiveConf.getTimeVar(conf, + HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_COMMITTED_TXN_CLEANER_FREQ, + TimeUnit.MILLISECONDS); + threadPool.scheduleAtFixedRate(committedTxnCleaner, 30, period, TimeUnit.MILLISECONDS); + + // schedule the timeoutCleaner. This one has an initial delay to give any running clients a + // chance to heartbeat after recovery. + long intialDelay = HiveConf.getTimeVar(conf, + HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_TIMEOUT_CLEANER_INITIAL_WAIT, TimeUnit + .MILLISECONDS); + period = HiveConf.getTimeVar(conf, + HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_TIMEOUT_CLEANER_FREQ, TimeUnit.MILLISECONDS); + threadPool.scheduleAtFixedRate(timedOutCleaner, intialDelay, period, TimeUnit.MILLISECONDS); + + // Schedule the deadlock detector + period = HiveConf.getTimeVar(conf, + HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_DEADLOCK_DETECTOR_FREQ, TimeUnit.MILLISECONDS); + threadPool.scheduleAtFixedRate(deadlockDetector, 40, period, TimeUnit.MILLISECONDS); + } + + } + + void shutdown() throws IOException { + LOG.info("Shutting down transaction manager"); + threadPool.shutdown(); + LOG.info("Finished shut down of transaction manager co-processor service"); + } + + /** + * Open a group of transactions + * @param request number of transactions, plus where they're from + * @return transaction ids of new transactions + * @throws IOException + * @throws SeverusPleaseException + */ + HbaseMetastoreProto.OpenTxnsResponse openTxns(HbaseMetastoreProto.OpenTxnsRequest request) + throws IOException, SeverusPleaseException { + if (full) { + LOG.warn("Request for new transaction rejected because the transaction manager has used " + + "available memory and cannot accept new transactions until some existing ones are " + + "closed out."); + throw new IOException("Full, no new transactions being accepted"); + } + + if (LOG.isDebugEnabled()) LOG.debug("Opening " + request.getNumTxns() + " transactions"); + boolean shouldCommit = false; + getHBase().begin(); + try (LockKeeper lk = new LockKeeper(masterLock.writeLock())) { + List hbaseTxns = new ArrayList<>(); + HbaseMetastoreProto.OpenTxnsResponse.Builder rspBuilder = + HbaseMetastoreProto.OpenTxnsResponse.newBuilder(); + for (int i = 0; i < request.getNumTxns(); i++) { + OpenHiveTransaction txn = new OpenHiveTransaction(nextTxnId++); + openTxns.put(txn.getId(), txn); + + HbaseMetastoreProto.Transaction.Builder builder = HbaseMetastoreProto.Transaction + .newBuilder() + .setId(txn.getId()) + .setTxnState(txn.getState()) + .setUser(request.getUser()) + .setHostname(request.getHostname()); + if (request.hasAgentInfo()) { + builder.setAgentInfo(request.getAgentInfo()); + } + hbaseTxns.add(builder.build()); + rspBuilder.addTxnIds(txn.getId()); + } + long newTxnVal = getHBase().addToSequence(HBaseReadWrite.TXN_SEQUENCE, request.getNumTxns()); + if (LOG.isDebugEnabled()) { + LOG.debug("nextTxnId is " + nextTxnId + ", newTxnVal is " + newTxnVal); + assert newTxnVal == nextTxnId; + } + getHBase().putTransactions(hbaseTxns); + shouldCommit = true; + return rspBuilder.build(); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + /** + * Get list of open transactions. + * @param request a void, just here because there's no option for it not to be + * @return high water mark plus list of open and aborted transactions + * @throws IOException + * @throws SeverusPleaseException + */ + HbaseMetastoreProto.GetOpenTxnsResponse getOpenTxns(HbaseMetastoreProto.Void request) + throws IOException, SeverusPleaseException { + LOG.debug("Getting open transactions"); + try (LockKeeper lk = new LockKeeper(masterLock.readLock())) { + return HbaseMetastoreProto.GetOpenTxnsResponse.newBuilder() + .setHighWaterMark(nextTxnId) + .addAllOpenTransactions(openTxns.keySet()) + .addAllAbortedTransactions(abortedTxns.keySet()) + .build(); + } + } + + /** + * Abort a transaction. + * @param request transaction to abort. + * @return result with status information. + * @throws IOException + * @throws SeverusPleaseException + */ + HbaseMetastoreProto.TransactionResult abortTxn(HbaseMetastoreProto.TransactionId request) + throws IOException, SeverusPleaseException { + if (LOG.isDebugEnabled()) { + LOG.debug("Aborting txn " + request.getId()); + } + HbaseMetastoreProto.TransactionResult response = null; + try (LockKeeper lk = new LockKeeper(masterLock.writeLock())) { + OpenHiveTransaction txn = openTxns.get(request.getId()); + if (txn == null) { + LOG.warn("Unable to find transaction to abort " + request.getId()); + response = HbaseMetastoreProto.TransactionResult.newBuilder() + .setState(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN) + .build(); + } else { + if (txn.getState() == HbaseMetastoreProto.TxnState.COMMITTED || + txn.getState() == HbaseMetastoreProto.TxnState.ABORTED) { + throw new SeverusPleaseException("Found a committed or aborted txn in the open list"); + } + abortTxn(txn); + response = HbaseMetastoreProto.TransactionResult.newBuilder() + .setState(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS) + .build(); + } + } + return response; + } + + /** + * Abort a transaction. You MUST own the master write lock before entering this method. You + * MUST NOT be in an HBase transaction. + * @param txn transaction to abort + * @throws IOException + */ + private void abortTxn(final OpenHiveTransaction txn) + throws IOException { + HiveLock[] locks = txn.getHiveLocks(); + if (locks != null) { + for (HiveLock lock : locks) { + lock.setState(HbaseMetastoreProto.LockState.TXN_ABORTED); + lockQueues.get(lock.getEntityLocked()).queue.remove(lock.getId()); + // It's ok to put these in the queue now even though we're still removing things because + // we hold the master write lock. The lockChecker won't be able to run until we've + // released that lock anyway. + lockQueuesToCheck.add(lock.getEntityLocked()); + } + // Request a lockChecker run since we've released locks. + LOG.debug("Requesting lockChecker run"); + threadPool.execute(lockChecker); + } + + // Move the entry to the aborted txns list + openTxns.remove(txn.getId()); + if (txn.hasWriteLocks()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Transaction " + txn.getId() + " has write locks, creating aborted txn"); + } + + List pces = new ArrayList<>(); + + // This is where protocol buffers suck. Since they're read only we have to make a whole + // new copy + HbaseMetastoreProto.Transaction hbaseTxn = getHBase().getTransaction(txn.getId()); + HbaseMetastoreProto.Transaction.Builder txnBuilder = + HbaseMetastoreProto.Transaction.newBuilder(hbaseTxn); + txnBuilder.clearLocks(); + txnBuilder.setTxnState(HbaseMetastoreProto.TxnState.ABORTED); + List hbaseLocks = hbaseTxn.getLocksList(); + for (HbaseMetastoreProto.Transaction.Lock hbaseLock : hbaseLocks) { + // We only need to remember the shared_write locks, all the rest can be ignored as they + // aren't a part of compaction. + if (hbaseLock.getType() == HbaseMetastoreProto.LockType.SHARED_WRITE) { + txnBuilder.addLocks(HbaseMetastoreProto.Transaction.Lock.newBuilder(hbaseLock) + .setState(HbaseMetastoreProto.LockState.TXN_ABORTED)); + if (LOG.isDebugEnabled()) { + LOG.debug("Found potential compaction for " + hbaseLock.getDb() + "." + + hbaseLock.getTable() + + (hbaseLock.hasPartition() ? "." + hbaseLock.getPartition() : "")); + } + pces.add(new HBaseReadWrite.PotentialCompactionEntity(hbaseLock.getDb(), + hbaseLock.getTable(), hbaseLock.hasPartition() ? hbaseLock.getPartition() : null)); + } + } + boolean shouldCommit = false; + getHBase().begin(); + try { + getHBase().putTransaction(txnBuilder.build()); + getHBase().putPotentialCompactions(txn.getId(), pces); + shouldCommit = true; + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + + AbortedHiveTransaction abortedTxn = new AbortedHiveTransaction(txn); + abortedTxns.put(txn.getId(), abortedTxn); + } else { + // There's no reason to remember this txn anymore, it either had no locks or was read only. + if (LOG.isDebugEnabled()) { + LOG.debug("Forgetting aborted txn " + txn.getId() + " as it has no write locks"); + } + getHBase().deleteTransaction(txn.getId()); + } + } + + /** + * Commit a transaction. + * @param request id of transaction to request + * @return status of commit + * @throws IOException + * @throws SeverusPleaseException + */ + HbaseMetastoreProto.TransactionResult commitTxn(HbaseMetastoreProto.TransactionId request) + throws IOException, SeverusPleaseException { + if (LOG.isDebugEnabled()) { + LOG.debug("Committing txn " + request.getId()); + } + boolean shouldCommit = false; + getHBase().begin(); + try (LockKeeper lk = new LockKeeper(masterLock.writeLock())) { + OpenHiveTransaction txn = openTxns.get(request.getId()); + if (txn == null) { + LOG.info("Unable to find txn to commit " + request.getId()); + return HbaseMetastoreProto.TransactionResult.newBuilder() + .setState(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN) + .build(); + } else { + if (txn.getState() == HbaseMetastoreProto.TxnState.COMMITTED || + txn.getState() == HbaseMetastoreProto.TxnState.ABORTED) { + throw new SeverusPleaseException("Found a committed or aborted txn in the open list"); + } + } + HiveLock[] locks = txn.getHiveLocks(); + if (locks != null) { + for (HiveLock lock : locks) { + lock.setState(HbaseMetastoreProto.LockState.RELEASED); + lockQueues.get(lock.getEntityLocked()).queue.remove(lock.getId()); + lockQueuesToCheck.add(lock.getEntityLocked()); + } + // Request a lockChecker run since we've released locks. + LOG.debug("Requesting lockChecker run"); + threadPool.execute(lockChecker); + } + + openTxns.remove(txn.getId()); + // We only need to remember the transaction if it had write locks. If it's read only or + // DDL we can forget it. + if (txn.hasWriteLocks()) { + // There's no need to move the transaction counter ahead + CommittedHiveTransaction committedTxn = new CommittedHiveTransaction(txn, nextTxnId); + + if (LOG.isDebugEnabled()) { + LOG.debug("Created new committed transaction with txn id " + committedTxn.getId() + + " and commit id " + committedTxn.getCommitId()); + } + + committedTxns.add(committedTxn); + // Record all of the commit ids in the lockQueues so other transactions can quickly look + // it up when getting locks. + for (HiveLock lock : txn.getHiveLocks()) { + if (lock.getType() != HbaseMetastoreProto.LockType.SHARED_WRITE) continue; + LockQueue queue = lockQueues.get(lock.getEntityLocked()); + if (LOG.isDebugEnabled()) { + LOG.debug(lock.getEntityLocked().toString() + " has max commit id of " + + queue.maxCommitId); + } + queue.maxCommitId = Math.max(queue.maxCommitId, committedTxn.getCommitId()); + if (LOG.isDebugEnabled()) { + LOG.debug("Set " + lock.getEntityLocked().toString() + " max commit id to " + + queue.maxCommitId); + } + } + List pces = new ArrayList<>(); + + HbaseMetastoreProto.Transaction hbaseTxn = getHBase().getTransaction(committedTxn.getId()); + HbaseMetastoreProto.Transaction.Builder txnBuilder = + HbaseMetastoreProto.Transaction.newBuilder(hbaseTxn); + txnBuilder.setTxnState(HbaseMetastoreProto.TxnState.COMMITTED); + txnBuilder.setCommitId(committedTxn.getCommitId()); + txnBuilder.clearLocks(); + List hbaseLocks = hbaseTxn.getLocksList(); + for (HbaseMetastoreProto.Transaction.Lock hbaseLock : hbaseLocks) { + // We only need to remember the shared_write locks, all the rest can be ignored as they + // aren't of interest for building write sets + if (hbaseLock.getType() == HbaseMetastoreProto.LockType.SHARED_WRITE) { + HbaseMetastoreProto.Transaction.Lock.Builder lockBuilder = + HbaseMetastoreProto.Transaction.Lock.newBuilder(hbaseLock) + .setState(HbaseMetastoreProto.LockState.RELEASED); + txnBuilder.addLocks(lockBuilder); + if (LOG.isDebugEnabled()) { + LOG.debug("Adding new entry to the potential compactions " + hbaseLock.getDb() + + "." + hbaseLock.getTable() + + (hbaseLock.hasPartition() ? "." + hbaseLock.getPartition() : "")); + } + pces.add(new HBaseReadWrite.PotentialCompactionEntity(hbaseLock.getDb(), + hbaseLock.getTable(), hbaseLock.hasPartition() ? hbaseLock.getPartition() : null)); + } + } + getHBase().putTransaction(txnBuilder.build()); + getHBase().putPotentialCompactions(committedTxn.getId(), pces); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Forgetting transaction " + txn.getId() + " as it is committed and held no " + + "write locks"); + } + getHBase().deleteTransaction(txn.getId()); + } + shouldCommit = true; + return HbaseMetastoreProto.TransactionResult.newBuilder() + .setState(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS) + .build(); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + /** + * Send a heartbeat for a range of transactions. Transaction heartbeats are not kept in HBase, + * it's just too expensive to rewrite an entire transaction on every heartbeat. + * @param request transaction range to heartbeat, max is inclusive + * @return list of transactions in the range that were aborted or could not be found. Note + * that since we don't record aborted txns with no write locks in the memory those will be + * reported as no such rather than aborted. + * @throws IOException + * @throws SeverusPleaseException + */ + HbaseMetastoreProto.HeartbeatTxnRangeResponse heartbeat(HbaseMetastoreProto.HeartbeatTxnRangeRequest request) + throws IOException, SeverusPleaseException { + long now = System.currentTimeMillis(); + HbaseMetastoreProto.HeartbeatTxnRangeResponse.Builder builder = + HbaseMetastoreProto.HeartbeatTxnRangeResponse.newBuilder(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Heartbeating transactions from " + request.getMinTxn() + " to " + + request.getMaxTxn()); + } + + try (LockKeeper lk = new LockKeeper(masterLock.readLock())) { + for (long txnId = request.getMinTxn(); txnId <= request.getMaxTxn(); txnId++) { + OpenHiveTransaction txn = openTxns.get(txnId); + if (txn != null) { + txn.setLastHeartbeat(now); + // Don't set the value in HBase, we don't track it there for efficiency + } else { + AbortedHiveTransaction abortedTxn = abortedTxns.get(txnId); + if (abortedTxn == null) builder.addNoSuch(txnId); + else builder.addAborted(txnId); + } + } + } + return builder.build(); + } + + /** + * Request a set of locks. + * @param request entities to lock + * @return status, could be txn_aborted (which really means we couldn't find the txn in memory, + * this doesn't go to HBase to understand if the transaction is committed, aborted, or never + * existed), acquired (meaning all of the locks are held), or waiting (meaning all the locks + * are in waiting state). + * @throws IOException + * @throws SeverusPleaseException + */ + HbaseMetastoreProto.LockResponse lock(HbaseMetastoreProto.LockRequest request) + throws IOException, SeverusPleaseException { + if (LOG.isDebugEnabled()) { + LOG.debug("Requesting locks for transaction " + request.getTxnId()); + for (HbaseMetastoreProto.LockComponent component : request.getComponentsList()) { + LOG.debug("entity: " + component.getDb() + + (component.hasTable() ? component.getTable() : "") + + (component.hasPartition() ? component.getPartition() : "")); + } + } + List components = request.getComponentsList(); + HiveLock[] hiveLocks = new HiveLock[components.size()]; + OpenHiveTransaction txn; + Future lockCheckerRun; + try (LockKeeper lk = new LockKeeper(masterLock.writeLock())) { + txn = openTxns.get(request.getTxnId()); + if (txn == null) { + LOG.info("Asked to get locks for non-open transaction " + request.getTxnId()); + return HbaseMetastoreProto.LockResponse.newBuilder() + .setState(HbaseMetastoreProto.LockState.TXN_ABORTED) + .build(); + } + if (txn.getState() != HbaseMetastoreProto.TxnState.OPEN) { + throw new SeverusPleaseException("Non-open txn in open list"); + } + + List lockBuilders = new ArrayList<>(); + for (int i = 0; i < components.size(); i++) { + HbaseMetastoreProto.LockComponent component = components.get(i); + long lockId = nextLockId++; + hiveLocks[i] = new HiveLock(lockId, request.getTxnId(), component.getType(), + getLockQueue(component).getFirst()); + // Build the hbase lock so we have it later when we need to record the locks in HBase + HbaseMetastoreProto.Transaction.Lock.Builder builder = + HbaseMetastoreProto.Transaction.Lock + .newBuilder() + .setId(lockId) + .setState(HbaseMetastoreProto.LockState.WAITING) + .setType(component.getType()) + .setDb(component.getDb()); + if (component.hasTable()) builder.setTable(component.getTable()); + if (component.hasPartition()) builder.setPartition(component.getPartition()); + lockBuilders.add(builder.build()); + + // Add to the appropriate DTP queue + lockQueues.get(hiveLocks[i].getEntityLocked()).queue.put(hiveLocks[i].getId(), hiveLocks[i]); + lockQueuesToCheck.add(hiveLocks[i].getEntityLocked()); + } + // Run the lock checker to see if we can acquire these locks + lockCheckerRun = threadPool.submit(lockChecker); + txn.addLocks(hiveLocks); + + // Record changes in HBase + long lockVal = + getHBase().addToSequence(HBaseReadWrite.LOCK_SEQUENCE, request.getComponentsCount()); + assert lockVal == nextLockId; + + HbaseMetastoreProto.Transaction hbaseTxn = getHBase().getTransaction(request.getTxnId()); + assert hbaseTxn != null; + HbaseMetastoreProto.Transaction.Builder txnBuilder = + HbaseMetastoreProto.Transaction.newBuilder(hbaseTxn); + txnBuilder.addAllLocks(lockBuilders); + boolean shouldCommit = false; + getHBase().begin(); + try { + getHBase().putTransaction(txnBuilder.build()); + shouldCommit = true; + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + // First, see if our locks acquired immediately using the return from our submission to the + // thread queue. + try { + lockCheckerRun.get(lockPollTimeout, TimeUnit.MILLISECONDS); + if (checkMyLocks(hiveLocks) == HbaseMetastoreProto.LockState.ACQUIRED) { + LOG.debug("Locks acquired immediately, returning"); + return HbaseMetastoreProto.LockResponse.newBuilder() + .setState(HbaseMetastoreProto.LockState.ACQUIRED) + .build(); + } + } catch (ExecutionException e) { + throw new IOException(e); + } catch (InterruptedException | TimeoutException e) { + LOG.debug("Lost patience waiting for locks, returning WAITING"); + return HbaseMetastoreProto.LockResponse.newBuilder() + .setState(HbaseMetastoreProto.LockState.WAITING) + .build(); + } + + // We didn't acquire right away, so long poll. We won't wait forever, but we can wait a few + // seconds to avoid the clients banging away every few hundred milliseconds to see if their + // locks have acquired. + LOG.debug("Locks did not acquire immediately, waiting..."); + return waitForLocks(hiveLocks); + } + + + /** + * This checks that all locks in the transaction are acquired. This will look immediately at + * the set of locks. If they have not yet acquired it will long poll. + * @param request transaction id to check for + * @return status, could be txn_aborted (which really means we couldn't find the txn in memory, + * this doesn't go to HBase to understand if the transaction is committed, aborted, or never + * existed), acquired (meaning all of the locks are held), or waiting (meaning all the locks + * are in waiting state). + * @throws IOException + * @throws SeverusPleaseException + */ + HbaseMetastoreProto.LockResponse checkLocks(HbaseMetastoreProto.TransactionId request) + throws IOException, SeverusPleaseException { + if (LOG.isDebugEnabled()) { + LOG.debug("Checking locks for transaction " + request.getId()); + } + OpenHiveTransaction txn; + try (LockKeeper lk = new LockKeeper(masterLock.readLock())) { + txn = openTxns.get(request.getId()); + if (txn == null) { + LOG.info("Attempt to check locks for non-open transaction " + request.getId()); + return HbaseMetastoreProto.LockResponse.newBuilder() + .setState(HbaseMetastoreProto.LockState.TXN_ABORTED) + .build(); + } else { + // Check to see if the locks have acquired yet + if (checkMyLocks(txn.getHiveLocks()) == HbaseMetastoreProto.LockState.ACQUIRED) { + return HbaseMetastoreProto.LockResponse.newBuilder() + .setState(HbaseMetastoreProto.LockState.ACQUIRED) + .build(); + } + } + } + + // Nope, not yet, but let's wait a bit and see if they acquire before we return + return waitForLocks(txn.getHiveLocks()); + } + + /** + * Wait for a bit to see if a set of locks acquire. This will wait on the lockChecker object + * to signal that the queues should be checked. It will only wait for up to lockPollTimeout + * milliseconds. + * @param hiveLocks locks to wait for. + * @return The state of the locks, could be WAITING, ACQUIRED, or TXN_ABORTED + * @throws IOException + * @throws SeverusPleaseException + */ + private HbaseMetastoreProto.LockResponse waitForLocks(HiveLock[] hiveLocks) + throws IOException, SeverusPleaseException { + LOG.debug("Waiting for locks"); + synchronized (lockChecker) { + try { + lockChecker.wait(lockPollTimeout); + } catch (InterruptedException e) { + LOG.warn("Interupted while waiting for locks", e); + // Still go ahead and check our status and return it. + } + } + try (LockKeeper lk = new LockKeeper(masterLock.readLock())) { + return HbaseMetastoreProto.LockResponse.newBuilder() + .setState(checkMyLocks(hiveLocks)) + .build(); + } + } + + /** + * See if a set of locks have acquired. You MUST hold the read lock before entering this method. + * @param hiveLocks set of locks to check + * @return state of checked locks + */ + private HbaseMetastoreProto.LockState checkMyLocks(HiveLock[] hiveLocks) + throws SeverusPleaseException { + for (HiveLock lock : hiveLocks) { + if (lock.getState() == HbaseMetastoreProto.LockState.WAITING) { + LOG.debug("Some of our locks still in waiting state"); + return HbaseMetastoreProto.LockState.WAITING; + } + if (lock.getState() != HbaseMetastoreProto.LockState.ACQUIRED) { + LOG.error("Found a lock in an unexpected state " + lock.getState()); + throw new SeverusPleaseException("Lock not in waiting or acquired state, not sure what to do"); + } + } + LOG.debug("All requested locks acquired"); + return HbaseMetastoreProto.LockState.ACQUIRED; + } + + private boolean twoLocksCompatible(HbaseMetastoreProto.LockType holder, + HbaseMetastoreProto.LockType requester) { + if (lockCompatibilityTable == null) { + // TODO not at all sure I have intention locks correct in this table + // The values of the enums are 1 based rather than 0 based. + lockCompatibilityTable = new boolean[HbaseMetastoreProto.LockType.values().length + 1][HbaseMetastoreProto.LockType.values().length + 1]; + Arrays.fill(lockCompatibilityTable[HbaseMetastoreProto.LockType.EXCLUSIVE_VALUE], false); + lockCompatibilityTable[HbaseMetastoreProto.LockType.SHARED_WRITE_VALUE][HbaseMetastoreProto.LockType.EXCLUSIVE_VALUE] = false; + lockCompatibilityTable[HbaseMetastoreProto.LockType.SHARED_WRITE_VALUE][HbaseMetastoreProto.LockType.SHARED_WRITE_VALUE] = false; + lockCompatibilityTable[HbaseMetastoreProto.LockType.SHARED_WRITE_VALUE][HbaseMetastoreProto.LockType.SHARED_READ_VALUE] = true; + lockCompatibilityTable[HbaseMetastoreProto.LockType.SHARED_WRITE_VALUE][HbaseMetastoreProto.LockType.INTENTION_VALUE] = true; + lockCompatibilityTable[HbaseMetastoreProto.LockType.SHARED_READ_VALUE][HbaseMetastoreProto.LockType.EXCLUSIVE_VALUE] = false; + lockCompatibilityTable[HbaseMetastoreProto.LockType.SHARED_READ_VALUE][HbaseMetastoreProto.LockType.SHARED_WRITE_VALUE] = true; + lockCompatibilityTable[HbaseMetastoreProto.LockType.SHARED_READ_VALUE][HbaseMetastoreProto.LockType.SHARED_READ_VALUE] = true; + lockCompatibilityTable[HbaseMetastoreProto.LockType.SHARED_READ_VALUE][HbaseMetastoreProto.LockType.INTENTION_VALUE] = true; + lockCompatibilityTable[HbaseMetastoreProto.LockType.INTENTION_VALUE][HbaseMetastoreProto.LockType.EXCLUSIVE_VALUE] = false; + lockCompatibilityTable[HbaseMetastoreProto.LockType.INTENTION_VALUE][HbaseMetastoreProto.LockType.SHARED_WRITE_VALUE] = true; + lockCompatibilityTable[HbaseMetastoreProto.LockType.INTENTION_VALUE][HbaseMetastoreProto.LockType.SHARED_READ_VALUE] = true; + lockCompatibilityTable[HbaseMetastoreProto.LockType.INTENTION_VALUE][HbaseMetastoreProto.LockType.INTENTION_VALUE] = true; + } + return lockCompatibilityTable[holder.getNumber()][requester.getNumber()]; + } + + /** + * Add list of partitions that were part of a dynamic partitions insert. These are needed so + * we know where to look for compactions. We do this by adding after the fact locks. This + * will also help us find any write/write conflicts. + * @param request Dynamic partitions to add. + * @return transaction result, can be success or no such transaction (which just means the + * transaction isn't open). + * @throws IOException + * @throws SeverusPleaseException + */ + HbaseMetastoreProto.TransactionResult addDynamicPartitions(HbaseMetastoreProto.AddDynamicPartitionsRequest request) + throws IOException, SeverusPleaseException { + if (LOG.isDebugEnabled()) { + LOG.debug("Adding dynamic partitions for transaction " + request.getTxnId() + " table " + + request.getDb() + "." + request.getTable()); + for (String part : request.getPartitionsList()) { + LOG.debug("Partition: " + part); + } + } + boolean shouldCommit = false; + getHBase().begin(); + // This needs to acquire the write lock because it might add entries to dtps, which is + // unfortunate. + try (LockKeeper lk = new LockKeeper(masterLock.writeLock())) { + // Add the locks to the appropriate transaction so that we know what things to compact and + // so we know what partitions were touched by this change. Don't put the locks in the dtps + // because we're actually covered by the table lock. Do increment the counters in the dtps. + OpenHiveTransaction txn = openTxns.get(request.getTxnId()); + if (txn == null) { + LOG.info("Attempt to add dynamic partitions to non-open transaction " + request.getTxnId()); + return HbaseMetastoreProto.TransactionResult.newBuilder() + .setState(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN) + .build(); + } + if (txn.getState() != HbaseMetastoreProto.TxnState.OPEN) { + throw new SeverusPleaseException("Found non-open transaction in open transaction list"); + } + + HbaseMetastoreProto.Transaction hbaseTxn = getHBase().getTransaction(request.getTxnId()); + HbaseMetastoreProto.Transaction.Builder txnBuilder = + HbaseMetastoreProto.Transaction.newBuilder(hbaseTxn); + + List partitionNames = request.getPartitionsList(); + HiveLock[] partitionsWrittenTo = new HiveLock[partitionNames.size()]; + List pces = new ArrayList<>(); + for (int i = 0; i < partitionNames.size(); i++) { + partitionsWrittenTo[i] = new HiveLock(-1, request.getTxnId(), + HbaseMetastoreProto.LockType.SHARED_WRITE, + // None of these should be null, so no need to check + getLockQueue(request.getDb(), request.getTable(), partitionNames.get(i)).getFirst()); + partitionsWrittenTo[i].setState(HbaseMetastoreProto.LockState.ACQUIRED); + + txnBuilder.addLocks(HbaseMetastoreProto.Transaction.Lock.newBuilder() + .setId(-1) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE) + .setState(HbaseMetastoreProto.LockState.ACQUIRED) + .setDb(request.getDb()) + .setTable(request.getTable()) + .setPartition(partitionNames.get(i))); + if (LOG.isDebugEnabled()) { + LOG.debug("Adding potential compaction " + request.getDb() + "." + request.getTable() + + "." + partitionNames.get(i)); + } + pces.add(new HBaseReadWrite.PotentialCompactionEntity(request.getDb(), request.getTable(), + partitionNames.get(i))); + } + txn.addLocks(partitionsWrittenTo); + + + getHBase().putTransaction(txnBuilder.build()); + getHBase().putPotentialCompactions(txn.getId(), pces); + shouldCommit = true; + return HbaseMetastoreProto.TransactionResult.newBuilder() + .setState(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS) + .build(); + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + } + + /** + * Cleanup our structures after compaciton. This includes potentially removing or altering + * records in the PotentialCompactions table as well as determining when to forget aborted + * records. Called after a worker is done compacting a partition or table. + * @param request compaction information. + * @return nothing, only here because protocol buffer service requires it + * @throws IOException + * @throws SeverusPleaseException + */ + HbaseMetastoreProto.Void cleanupAfterCompaction(HbaseMetastoreProto.Compaction request) + throws IOException, SeverusPleaseException { + if (LOG.isDebugEnabled()) { + LOG.debug("Cleaning up after compaction of " + request.getDb() + "." + request.getTable() + + (request.hasPartition() ? "." + request.getPartition() : "")); + } + boolean shouldCommit = false; + getHBase().begin(); + try { + // First, go find the potential compaction that matches this compaction. That has the map + // to which transactions we now need to modify or forget. + HbaseMetastoreProto.PotentialCompaction potential = + getHBase().getPotentialCompaction(request.getDb(), request.getTable(), + request.hasPartition() ? request.getPartition() : null); + // It is possible for the potential to be null, as the user could have requested a compaction + // when there was nothing really to do. + if (potential != null) { + LOG.debug("Found appropriate potential compaction"); + List compactedTxns = new ArrayList<>(); // All txns that were compacted + // Txns in the potential that weren't compacted as part of this compaction. We need to + // track this so we can properly write the potential back. + List uncompactedTxns = new ArrayList<>(); + for (long txnId : potential.getTxnIdsList()) { + // We only need to worry about this transaction if it's writes were compacted (that + // is, it's <= the highestTxnId) + if (txnId <= request.getHighestTxnId()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Adding " + txnId + " to list of transactions that have been compacted"); + } + compactedTxns.add(txnId); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Adding " + txnId + " to list of transactions there were not compacted"); + } + uncompactedTxns.add(txnId); + } + } + if (compactedTxns.size() > 0) { + // Transactions that have had all the tables/partitions they touched compacted. + List fullyCompacted = new ArrayList<>(); + // Transactions that have had locks marked as compacted but are not themselves fully + // compacted yet. + Map modifiedTxns = new HashMap<>(); + // We need the write lock because we might remove entries from abortedTxns + try (LockKeeper lk = new LockKeeper(masterLock.writeLock())) { + for (long txnId : compactedTxns) { + AbortedHiveTransaction txn = abortedTxns.get(txnId); + // The transaction could be null if this was a completed transaction and not an + // aborted one. + if (txn != null) { + // Mark the lock on that partition in this aborted txn as compacted. + HiveLock compactedLock = txn.compactLock(new EntityKey(request.getDb(), + request.getTable(), request.hasPartition() ? request.getPartition() : null)); + if (LOG.isDebugEnabled()) { + LOG.debug("Found aborted txn " + txn.getId() + " that was in the compacted list" + + " removing lock for " + compactedLock.getEntityLocked().toString()); + } + if (txn.fullyCompacted()) { + LOG.debug("Transaction is now fully compacted"); + fullyCompacted.add(txnId); + } else { + LOG.debug("Transaction has remaining uncompacted locks"); + modifiedTxns.put(txnId, compactedLock); + } + } + } + + if (fullyCompacted.size() > 0) { + for (long txnId : fullyCompacted) { + if (LOG.isDebugEnabled()) { + LOG.debug("Forgeting fully compacted aborted transaction " + txnId); + } + abortedTxns.remove(txnId); + } + getHBase().deleteTransactions(fullyCompacted); + } + if (modifiedTxns.size() > 0) { + List toModify = + getHBase().getTransactions(modifiedTxns.keySet()); + // New transaction objects we're writing back to HBase + List toWrite = new ArrayList<>(modifiedTxns.size()); + for (HbaseMetastoreProto.Transaction hbaseTxn : toModify) { + if (LOG.isDebugEnabled()) { + LOG.debug("Writing back modified aborted transaction " + hbaseTxn.getId()); + } + HiveLock compactedLock = modifiedTxns.get(hbaseTxn.getId()); + HbaseMetastoreProto.Transaction.Builder txnBldr = + HbaseMetastoreProto.Transaction.newBuilder(hbaseTxn); + List hbaseLocks = hbaseTxn.getLocksList(); + txnBldr.clearLocks(); + for (HbaseMetastoreProto.Transaction.Lock hbaseLock : hbaseLocks) { + if (hbaseLock.getId() != compactedLock.getId()) { + txnBldr.addLocks(hbaseLock); + } + } + toWrite.add(txnBldr.build()); + } + getHBase().putTransactions(toWrite); + } + // rewrite potential compaction + if (uncompactedTxns.size() > 0) { + LOG.debug("Rewriting potential compaction minus compacted transactions"); + // Rewrite the potential compaction to remove txns that have been compacted out + HbaseMetastoreProto.PotentialCompaction.Builder potentialBldr = + HbaseMetastoreProto.PotentialCompaction.newBuilder(potential); + potentialBldr.clearTxnIds(); + potentialBldr.addAllTxnIds(uncompactedTxns); + getHBase().putPotentialCompaction(potentialBldr.build()); + } else { + LOG.debug("Forgetting potential compaction as all transactions compacted"); + // This was the last transaction in this potential compaction, so remove it + getHBase().deletePotentialCompaction(potential.getDb(), potential.getTable(), + potential.hasPartition() ? potential.getPartition() : null); + } + shouldCommit = true; + } + } + } else { + LOG.info("Unable to find requested potential compaction " + request.getDb() + "." + + request.getTable() + (request.hasPartition() ? request.getPartition() : "")); + } + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + return HbaseMetastoreProto.Void.getDefaultInstance(); + } + + /** + * Verify whether a set of partitions and/or tables can be cleaned after compactions have been + * done. + * @param request list of partitions/tables that have been compacted + * @return list of partitions/tables that can be cleaned. + * @throws IOException + * @throws SeverusPleaseException + */ + HbaseMetastoreProto.CompactionList verifyCompactionCanBeCleaned(HbaseMetastoreProto.CompactionList request) + throws IOException, SeverusPleaseException { + // TODO - I may be double doing this. Can I ever have a highestCompactionId higher than any + // open transactions? + // This takes a set of compactions that have been compacted and are ready to be cleaned. It + // returns only ones that have highestCompactionId > min(openTxns). Any for which this is not + // true, there may still be open transactions referencing the files that were compacted, thus + // they should not yet be cleaned. + if (LOG.isDebugEnabled()) { + LOG.debug("Received list of compactions cleans to be verified"); + for (HbaseMetastoreProto.Compaction compaction : request.getCompactionsList()) { + LOG.debug("Compaction id " + compaction.getId() + " for " + compaction.getDb() + "." + + compaction.getTable() + (compaction.hasPartition() ? "." + compaction.getPartition() + : "")); + } + } + long minOpenTxn; + HbaseMetastoreProto.CompactionList.Builder builder = + HbaseMetastoreProto.CompactionList.newBuilder(); + // We only need the read lock to look at the open transaction list + try (LockKeeper lk = new LockKeeper(masterLock.readLock())) { + minOpenTxn = findMinOpenTxn(); + } + + for (HbaseMetastoreProto.Compaction compaction : request.getCompactionsList()) { + if (compaction.getHighestTxnId() < minOpenTxn) { + if (LOG.isDebugEnabled()) { + LOG.debug("Approving cleaning id " + compaction.getId() + " " + compaction.getDb() + "." + + compaction.getTable() + (compaction.hasPartition() ? "." + compaction.getPartition() + : "")); + } + builder.addCompactions(compaction); + } + } + return builder.build(); + } + + /** + * Find the appropriate lock queue given entity information. If an appropriate entry is not in + * the lockQueues it will be added. This method does not acquire any locks, but you should + * assure that you are inside the write lock before calling it, or you're in recover and thus + * don't need any locks. + * @param hbaseLock hbase lock + * @return object pair, first element is EntityKey, second is LockQueue. + * @throws IOException + */ + ObjectPair getLockQueue(HbaseMetastoreProto.Transaction.Lock hbaseLock) + throws IOException { + return getLockQueue(hbaseLock.getDb(), + hbaseLock.hasTable() ? hbaseLock.getTable() : null, + hbaseLock.hasPartition() ? hbaseLock.getPartition() : null); + } + + /** + * Find the appropriate lock queue given entity information. If an appropriate entry is not in + * the lockQueues it will be added. This method does not acquire any locks, but you should + * assure that you are inside the write lock before calling it, or you're in recover and thus + * don't need any locks. + * @param component lock component to get queue from + * @return object pair, first element is EntityKey, second is LockQueue. + * @throws IOException + */ + private ObjectPair getLockQueue(HbaseMetastoreProto.LockComponent component) + throws IOException { + return getLockQueue(component.getDb(), + component.hasTable() ? component.getTable() : null, + component.hasPartition() ? component.getPartition() : null); + } + + /** + * Find the appropriate lock queue given entity information. If an appropriate entry is not in + * the lockQueues it will be added. This method does not acquire any locks, but you should + * assure that you are inside the write lock before calling it, or you're in recover and thus + * don't need any locks. + * @param db database + * @param table table name (may be null) + * @param part partition name (may be null) + * @return object pair, first element is EntityKey, second is LockQueue. + * @throws IOException + */ + private ObjectPair getLockQueue(String db, String table, String part) + throws IOException { + EntityKey key = new EntityKey(db, table, part); + LockQueue queue = lockQueues.get(key); + if (queue == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Creating new LockQueue for " + key.toString()); + } + queue = new LockQueue(); + lockQueues.put(key, queue); + } + return new ObjectPair<>(key, queue); + } + + // This method assumes you are holding the read lock. + private long findMinOpenTxn() { + long minOpenTxn = nextTxnId; + for (Long txnId : openTxns.keySet()) { + minOpenTxn = Math.min(txnId, minOpenTxn); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Found minimum open transaction of " + minOpenTxn); + } + return minOpenTxn; + } + + private boolean checkFull() throws IOException { + // Check to see if the transaction manager is full. If it isn't currently set as full but + // has too many objects this will switch the full flag on. If the full flag is on and we've + // fallen below 90% this will turn it off. + int objectCnt = countObjects(); + if (full) { + if (objectCnt < maxObjects * 0.9) { + LOG.info("Transaction manager has drained, switching off full flag"); + full = false; + } else { + LOG.debug("Still full..."); + } + } else { + if (objectCnt > maxObjects) { + // Try to shrink the lock queues and then check again + tryToShrinkLockQueues(); + if (countObjects() > maxObjects) { + LOG.error("Transation manager object count exceeds configured max object count " + + maxObjects + ", marking full and no longer accepting new transactions until some " + + "current objects have drained off"); + full = true; + } + } else { + LOG.debug("Still not full..."); + } + } + return full; + } + + private int countObjects() throws IOException { + + try (LockKeeper lk = new LockKeeper(masterLock.readLock())) { + // First count all the transactions, this is relatively easy. + int objectCnt = committedTxns.size() + abortedTxns.size() + openTxns.size() + + lockQueues.keySet().size(); + // Early out if this alone fills us up, as the next part gets harder. + if (objectCnt > maxObjects) return objectCnt; + + for (LockQueue queue : lockQueues.values()) { + objectCnt += queue.queue.size(); + if (objectCnt > maxObjects) return objectCnt; + } + return objectCnt; + } + } + + private void tryToShrinkLockQueues() { + LOG.debug("Seeing if we can shrink the lock queue"); + List empties = new ArrayList<>(); + // First get the read lock and find all currently empty keys. Then release the read + // lock, get the write lock and remove all those keys, checking again that they are + // truly empty. + try (LockKeeper lk = new LockKeeper(masterLock.readLock())) { + for (Map.Entry entry : lockQueues.entrySet()) { + if (entry.getValue().queue.size() == 0 && entry.getValue().maxCommitId == 0) { + empties.add(entry.getKey()); + } + } + } catch (IOException e) { + LOG.warn("Caught exception while examining lock queues", e); + } + + if (empties.size() > 0) { + try (LockKeeper lk = new LockKeeper(masterLock.writeLock())) { + for (EntityKey key : empties) { + LockQueue queue = lockQueues.get(key); + if (queue.queue.size() == 0 && queue.maxCommitId == 0) { + if (LOG.isDebugEnabled()) { + LOG.debug("Removing lockQueue " + key.toString()); + } + lockQueues.remove(key); + } + } + } catch (IOException e) { + LOG.warn("Caught exception while removing empty lock queue", e); + } + } else { + LOG.debug("No lock queues found to remove"); + } + } + + private void recover() throws IOException { + // No locking is required here because we're still in the constructor and we're guaranteed no + // one else is muddying the waters. + // Get existing transactions from HBase + LOG.info("Beginning recovery"); + try { + List hbaseTxns = getHBase().scanTransactions(null); + if (hbaseTxns != null) { + for (HbaseMetastoreProto.Transaction hbaseTxn : hbaseTxns) { + switch (hbaseTxn.getTxnState()) { + case ABORTED: + if (LOG.isDebugEnabled()) { + LOG.debug("Found aborted transaction " + hbaseTxn.getId()); + } + AbortedHiveTransaction abortedTxn = new AbortedHiveTransaction(hbaseTxn, this); + abortedTxns.put(abortedTxn.getId(), abortedTxn); + break; + + case OPEN: + if (LOG.isDebugEnabled()) { + LOG.debug("Found open transaction " + hbaseTxn.getId()); + } + OpenHiveTransaction openTxn = new OpenHiveTransaction(hbaseTxn, this); + openTxns.put(openTxn.getId(), openTxn); + break; + + case COMMITTED: + if (LOG.isDebugEnabled()) { + LOG.debug("Found committed transaction " + hbaseTxn.getId()); + } + for (HbaseMetastoreProto.Transaction.Lock hbaseLock : hbaseTxn.getLocksList()) { + LockQueue queue = getLockQueue(hbaseLock).getSecond(); + queue.maxCommitId = Math.max(queue.maxCommitId, hbaseTxn.getCommitId()); + } + committedTxns.add(new CommittedHiveTransaction(hbaseTxn)); + break; + } + } + } + nextTxnId = getHBase().peekAtSequence(HBaseReadWrite.TXN_SEQUENCE); + nextLockId = getHBase().peekAtSequence(HBaseReadWrite.LOCK_SEQUENCE); + if (LOG.isDebugEnabled()) { + LOG.debug("Set nextTxnId to " + nextTxnId + " and nextLockId to " + nextLockId); + } + checkFull(); + } finally { + // It's a read only operation, so always commit + getHBase().commit(); + } + LOG.info("Completed recovery"); + } + + private HBaseReadWrite getHBase() { + if (hbase == null) { + HBaseReadWrite.setConf(conf); + hbase = HBaseReadWrite.getInstance(); + } + return hbase; + } + + public static class LockKeeper implements Closeable { + final Lock lock; + + public LockKeeper(Lock lock) { + this.lock = lock; + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for Java lock " + lock.getClass().getName()); + } + this.lock.lock(); + if (LOG.isDebugEnabled()) { + LOG.debug("Acquired Java lock " + lock.getClass().getName()); + } + } + + @Override + public void close() throws IOException { + lock.unlock(); + if (LOG.isDebugEnabled()) { + LOG.debug("Released Java lock " + lock.getClass().getName()); + } + } + } + + static class EntityKey { + final String db; + final String table; + final String part; + + EntityKey(String db, String table, String part) { + this.db = db; + this.table = table; + this.part = part; + } + + @Override + public int hashCode() { + // db should never be null + int hashCode = db.hashCode(); + if (table != null) hashCode = hashCode * 31 + table.hashCode(); + if (part != null) hashCode = hashCode * 31 + part.hashCode(); + return hashCode; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof EntityKey)) return false; + EntityKey other = (EntityKey)obj; + // db should never be null + if (db.equals(other.db)) { + if (table == null && other.table == null || + table != null && table.equals(other.table)) { + if (part == null && other.part == null || + part != null && part.equals(other.part)) { + return true; + } + } + } + return false; + } + + @Override + public String toString() { + StringBuilder bldr = new StringBuilder(db); + if (table != null) { + bldr.append('.') + .append(table); + if (part != null) { + bldr.append('.') + .append(part); + } + } + return bldr.toString(); + } + } + + static class LockQueue { + final SortedMap queue; + private long maxCommitId; // commit id of highest transaction that wrote to this DTP + + private LockQueue() { + queue = new TreeMap<>(); + maxCommitId = 0; + } + + long getMaxCommitId() { + return maxCommitId; + } + } + + private Runnable timedOutCleaner = new Runnable() { + @Override + public void run() { + LOG.debug("Running timeout cleaner"); + // First get the read lock and find all of the potential timeouts. + List potentials = new ArrayList<>(); + long now = System.currentTimeMillis(); + try (LockKeeper lk = new LockKeeper(masterLock.readLock())) { + for (OpenHiveTransaction txn : openTxns.values()) { + if (txn.getLastHeartbeat() + txnTimeout < now) { + if (LOG.isDebugEnabled()) { + LOG.debug("Adding " + txn.getId() + " to list of potential timeouts"); + } + potentials.add(txn); + } + } + } catch (IOException e) { + LOG.warn("Caught exception in timeOutCleaner, most likely during locking, not much we " + + "can do about it", e); + } + + // Now go back through the potentials list, holding the write lock, and remove any that + // still haven't heartbeat + try (LockKeeper lk = new LockKeeper(masterLock.writeLock())) { + for (OpenHiveTransaction txn : potentials) { + if (txn.getLastHeartbeat() + txnTimeout < now) { + LOG.info("Aborting transaction " + txn.getId() + " due to heartbeat timeout"); + abortTxn(txn); + } + } + } catch (IOException e) { + LOG.warn("Caught exception aborting transaction", e); + } + } + }; + + private Runnable deadlockDetector = new Runnable() { + // Rather than follow the general pattern of go through all the entries and find potentials + // and then remove all potentials this thread kills a deadlock as soon as it sees it. + // Otherwise we'd likely be too aggressive and kill all participants in the deadlock. If a + // deadlock is detected it immediately schedules another run of itself so that it doesn't end + // up taking minutes to find many deadlocks + @Override + public void run() { + LOG.debug("Looking for deadlocks"); + + try { + OpenHiveTransaction deadlocked = lookForCycles(); + if (deadlocked != null) { + try (LockKeeper lk = new LockKeeper(masterLock.writeLock())) { + abortTxn(openTxns.get(deadlocked.getId())); + } + threadPool.submit(this); + } + } catch (IOException e) { + LOG.warn("Received exception in deadlock detector", e); + } + } + + private OpenHiveTransaction lookForCycles() throws IOException { + try (LockKeeper lk = new LockKeeper(masterLock.readLock())) { + // We're looking only for transactions that have 1+ acquired locks and 1+ waiting locks + for (OpenHiveTransaction txn : openTxns.values()) { + boolean sawAcquired = false, sawWaiting = false; + for (HiveLock lock : txn.getHiveLocks()) { + if (lock.getState() == HbaseMetastoreProto.LockState.WAITING) { + sawWaiting = true; + } else if (lock.getState() == HbaseMetastoreProto.LockState.ACQUIRED) { + sawAcquired = true; + } + } + // Only check if we have both acquired and waiting locks. A transaction might be in + // a cycle without that, but it won't be key to the cycle without it. + if (sawAcquired && sawWaiting) { + if (lookForDeadlock(txn.getId(), txn, true)) { + LOG.warn("Detected deadlock, aborting transaction " + txn.getId() + + " to resolve it"); + // It's easiest to always kill this one rather than try to figure out where in + // the graph we can remove something and break the cycle. Given that which txn + // we examine first is mostly random this should be ok (I hope). + return txn; + } + } + } + } + return null; + } + + /** + * This looks for cycles in the lock graph. It remembers the first transaction we started + * looking at, and if it gets back that transaction then it returns true. + * @param initialTxnId Starting transaction in the graph traversal. + * @param currentTxn Current transaction in the graph traversal. + * @param initial Whether this call is the first time this is called, so initialTxn should + * equals currentTxn + * @return true if a cycle is detected. + */ + private boolean lookForDeadlock(long initialTxnId, OpenHiveTransaction currentTxn, + boolean initial) { + if (!initial && initialTxnId == currentTxn.getId()) return true; + for (HiveLock lock : currentTxn.getHiveLocks()) { + if (lock.getState() == HbaseMetastoreProto.LockState.WAITING) { + // We need to look at all of the locks ahead of this lock in it's queue + for (HiveLock predecessor : + lockQueues.get(lock.getEntityLocked()).queue.headMap(lock.getId()).values()) { + if (lookForDeadlock(initialTxnId, openTxns.get(predecessor.getTxnId()), false)) { + return true; + } + } + } + } + return false; + } + }; + + private Runnable lockQueueShrinker = new Runnable() { + @Override + public void run() { + LOG.debug("Running lock queue shrinker"); + tryToShrinkLockQueues(); + } + }; + + private Runnable fullChecker = new Runnable() { + @Override + public void run() { + try { + LOG.debug("Running full checker"); + if (checkFull()) { + // Schedule another instance of ourself very soon because we want to run frequently + // while we're full + threadPool.schedule(this, 100, TimeUnit.MILLISECONDS); + } + } catch (IOException e) { + // Go back through the while loop, as this might mean it's time to quit + LOG.warn("Caught exception while checking to see if transaction manager is full", e); + } + } + }; + + private Runnable committedTxnCleaner = new Runnable() { + // This looks through the list of committed transactions and figures out what can be + // forgotten. + @Override + public void run() { + try { + LOG.debug("Running committed transaction cleaner"); + Set forgetableIds = new HashSet<>(); + Set forgetableCommitIds = new HashSet<>(); + List forgetableTxns = new ArrayList<>(); + Map forgetableDtps = new HashMap<>(); + long minOpenTxn; + try (LockKeeper lk = new LockKeeper(masterLock.readLock())) { + minOpenTxn = findMinOpenTxn(); + for (CommittedHiveTransaction txn : committedTxns) { + // Look to see if all open transactions have a txnId greater than this txn's commitId + if (txn.getCommitId() <= minOpenTxn) { + if (LOG.isDebugEnabled()) { + LOG.debug("Adding " + txn.getId() + " to list of forgetable transactions"); + } + forgetableIds.add(txn.getId()); + forgetableTxns.add(txn); + forgetableCommitIds.add(txn.getCommitId()); + } + } + // For any of these found transactions, see if we can remove them from the lock queues. + // This is important because it enables us eventually to shrink the lock queues + for (Map.Entry entry : lockQueues.entrySet()) { + LockQueue queue = entry.getValue(); + if (forgetableCommitIds.contains(queue.maxCommitId)) { + forgetableDtps.put(queue, queue.maxCommitId); + if (LOG.isDebugEnabled()) { + LOG.debug("Adding lock queue " + entry.getKey().toString() + + " to forgetable list"); + } + } + } + } // exiting read lock + if (forgetableIds.size() > 0) { + try (LockKeeper lk1 = new LockKeeper(masterLock.writeLock())) { + committedTxns.removeAll(forgetableTxns); + boolean shouldCommit = false; + getHBase().begin(); + try { + getHBase().deleteTransactions(forgetableIds); + shouldCommit = true; + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + for (Map.Entry entry : forgetableDtps.entrySet()) { + // Make sure no one else has changed the value in the meantime + if (entry.getKey().maxCommitId == entry.getValue()) { + entry.getKey().maxCommitId = 0; + } + } + } + } else { + LOG.debug("No forgettable committed transactions found"); + } + } catch (IOException e) { + LOG.warn("Caught exception cleaning committed transactions", e); + } + } + }; + + private final Runnable lockChecker = new Runnable() { + // Unlike the preceding list of runnables, this code is only run when scheduled by a commit, + // abort, or lock operation that has changed the lock queues. When it is done it signals on + // itself to notify anyone in waitForLocks to go look at their locks. When locking on this + // object to wait you MUST NOT hold the read lock or write lock, as there's nasty potential + // for deadlock. + @Override + public void run() { + try { + LOG.debug("Checking locks"); + List keys; + + // Get the write lock and then read out the values to look for. + try (LockKeeper lk = new LockKeeper(masterLock.writeLock())) { + if (lockQueuesToCheck.size() == 0) { + // This is very possible because checkLocks forces a run of the lockChecker + LOG.debug("No lockQueuesToCheck"); + return; + } + keys = lockQueuesToCheck; + lockQueuesToCheck = new ArrayList<>(); + } + + Map toAcquire = new HashMap<>(); + Set acquiringTxns = new HashSet<>(); + final Set writeConflicts = new HashSet<>(); + try (LockKeeper lk = new LockKeeper(masterLock.readLock())) { + // Many keys may have been added to the queue, grab them all so we can do this just + // once. + for (EntityKey key : keys) { + LockQueue queue = lockQueues.get(key); + HiveLock lastLock = null; + for (HiveLock lock : queue.queue.values()) { + if (lock.getState() == HbaseMetastoreProto.LockState.WAITING) { + // See if we can acquire this lock + if (lastLock == null || twoLocksCompatible(lastLock.getType(), lock.getType())) { + // Before deciding we can acquire it we have to assure that we don't have a + // lost update problem where another transaction not in the acquiring + // transaction's read set has written to the same entity. If so, abort the + // acquiring transaction. Only do this if this is also a write lock. It's + // ok if we're reading old information, as this isn't serializable. + if (lock.getType() == HbaseMetastoreProto.LockType.SHARED_WRITE && + lockQueues.get(lock.getEntityLocked()).maxCommitId > lock.getTxnId()) { + LOG.warn("Transaction " + lock.getTxnId() + + " attempted to obtain shared write lock for " + + lock.getEntityLocked().toString() + " but that entity more recently " + + "updated with transaction with commit id " + + lockQueues.get(lock.getEntityLocked()).maxCommitId + + " so later transaction will be aborted."); + writeConflicts.add(lock.getTxnId()); + } else { + acquiringTxns.add(openTxns.get(lock.getTxnId())); + toAcquire.put(lock.getId(), lock); + if (LOG.isDebugEnabled()) { + LOG.debug("Adding lock " + lock.getTxnId() + "." + lock.getId() + + " to list of locks to acquire"); + } + } + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Not acquiring lock " + lock.getTxnId() + "." + lock.getId() + + " of type " + lock.getType().toString() + " as lock " + lastLock.getTxnId() + + "." + lastLock.getId() + " of type " + lastLock.getType() + " is ahead of" + + " it in state " + lastLock.getState().toString()); + } + // If we can't acquire then nothing behind us can either + // TODO prove to yourself this is true + break; + } + + } + lastLock = lock; + } + } + } // Now outside read lock + try (LockKeeper lk = new LockKeeper(masterLock.writeLock())) { + // Abort the transactions that ran into write conflicts. We logged this above, so no + // need to log it here. Do it in a separate thread as this could take a bit and we + // have other things to do. + Future abortingWriteConflicts = null; + if (writeConflicts.size() > 0) { + final List exceptions = new ArrayList<>(1); + Runnable conflictAborter = new Runnable() { + @Override + public void run() { + for (long txnId : writeConflicts) { + try { + abortTxn(openTxns.get(txnId)); + } catch (IOException e) { + exceptions.add(e); + return; + } + } + } + }; + abortingWriteConflicts = threadPool.submit(conflictAborter); + } + + for (HiveLock lock : toAcquire.values()) { + lock.setState(HbaseMetastoreProto.LockState.ACQUIRED); + } + + List txnsToPut = new ArrayList<>(); + for (OpenHiveTransaction txn : acquiringTxns) { + HbaseMetastoreProto.Transaction hbaseTxn = getHBase().getTransaction(txn.getId()); + HbaseMetastoreProto.Transaction.Builder txnBuilder = + HbaseMetastoreProto.Transaction.newBuilder(hbaseTxn); + List hbaseLocks = hbaseTxn.getLocksList(); + txnBuilder.clearLocks(); + for (HbaseMetastoreProto.Transaction.Lock hbaseLock : hbaseLocks) { + if (toAcquire.containsKey(hbaseLock.getId())) { + HbaseMetastoreProto.Transaction.Lock.Builder lockBuilder = + HbaseMetastoreProto.Transaction.Lock.newBuilder(hbaseLock); + lockBuilder.setAcquiredAt(System.currentTimeMillis()); + lockBuilder.setState(HbaseMetastoreProto.LockState.ACQUIRED); + txnBuilder.addLocks(lockBuilder); + } else { + txnBuilder.addLocks(hbaseLock); + } + } + txnsToPut.add(txnBuilder.build()); + } + boolean shouldCommit = false; + getHBase().begin(); + try { + getHBase().putTransactions(txnsToPut); + shouldCommit = true; + } finally { + if (shouldCommit) getHBase().commit(); + else getHBase().rollback(); + } + if (abortingWriteConflicts != null) abortingWriteConflicts.get(); + } // out of the write lock + + // Notify any waiters to go look for their locks + synchronized (this) { + this.notifyAll(); + } + + } catch (Exception ie) { + LOG.warn("Received exception while checking for locks", ie); + } + } + }; + + // I REALLY don't want to open up the member variables in this class for test classes to look + // at since usage of them is so dependent on holding the right locks, matching the state in + // HBase, etc. It is dangerous to let those out. But in order for unit tests to be effective I need a + // way to see the internal state of the object. So we'll make copies of them so the tests can + // see them. + @VisibleForTesting + Map copyOpenTransactions() { + return new HashMap<>(openTxns); + } + + @VisibleForTesting + Map copyAbortedTransactions() { + return new HashMap<>(abortedTxns); + } + + @VisibleForTesting + Set copyCommittedTransactions() { + return new HashSet<>(committedTxns); + } + + @VisibleForTesting + Map copyLockQueues() { + return new HashMap<>(lockQueues); + } + + // The following functions force running the various background threads. These should only be + // used for testing. When writing tests that are going to call these you should set the + // initial delay to MAX_LONG on the background threads so none of them are running unexpectedly + // on you. + @VisibleForTesting + void forceFullChecker() throws ExecutionException, InterruptedException { + Future fullCheckerRun = threadPool.submit(fullChecker); + fullCheckerRun.get(); + } + + @VisibleForTesting + void forceDeadlockDetection() throws ExecutionException, InterruptedException { + Future run = threadPool.submit(deadlockDetector); + run.get(); + } + + @VisibleForTesting + void forceLockQueueShrinker() throws ExecutionException, InterruptedException { + Future run = threadPool.submit(lockQueueShrinker); + run.get(); + } + + @VisibleForTesting + void forceCommittedTxnCleaner() throws ExecutionException, InterruptedException { + Future run = threadPool.submit(committedTxnCleaner); + run.get(); + } + + @VisibleForTesting + void forceTimedOutCleaner() throws ExecutionException, InterruptedException { + Future run = threadPool.submit(timedOutCleaner); + run.get(); + } + +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java index bea1473..0a2bdb1 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java @@ -32,12 +32,14 @@ public String dbname; public String tableName; public String partName; - char state; + public char state; public CompactionType type; - String workerId; - long start; + public String workerId; + public long start; public String runAs; public boolean tooManyAborts = false; + public int numTxns = 0; + /** * {@code 0} means it wasn't set (e.g. in case of upgrades, since ResultSet.getLong() will return 0 if field is NULL) * See {@link TxnStore#setCompactionHighestTxnId(CompactionInfo, long)} for precise definition. @@ -45,8 +47,8 @@ * {@link ValidCompactorTxnList#highWatermark} */ public long highestTxnId; - byte[] metaInfo; - String hadoopJobId; + public byte[] metaInfo; + public String hadoopJobId; private String fullPartitionName = null; private String fullTableName = null; @@ -62,7 +64,7 @@ public CompactionInfo(String dbname, String tableName, String partName, Compacti this.id = id; this.state = state; } - CompactionInfo() {} + public CompactionInfo() {} public String getFullPartitionName() { if (fullPartitionName == null) { diff --git metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java index 67e661f..0980b41 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java @@ -657,37 +657,6 @@ public void setCompactionHighestTxnId(CompactionInfo ci, long highestTxnId) thro setCompactionHighestTxnId(ci, highestTxnId); } } - private static class RetentionCounters { - int attemptedRetention = 0; - int failedRetention = 0; - int succeededRetention = 0; - RetentionCounters(int attemptedRetention, int failedRetention, int succeededRetention) { - this.attemptedRetention = attemptedRetention; - this.failedRetention = failedRetention; - this.succeededRetention = succeededRetention; - } - } - private void checkForDeletion(List deleteSet, CompactionInfo ci, RetentionCounters rc) { - switch (ci.state) { - case ATTEMPTED_STATE: - if(--rc.attemptedRetention < 0) { - deleteSet.add(ci.id); - } - break; - case FAILED_STATE: - if(--rc.failedRetention < 0) { - deleteSet.add(ci.id); - } - break; - case SUCCEEDED_STATE: - if(--rc.succeededRetention < 0) { - deleteSet.add(ci.id); - } - break; - default: - //do nothing to hanlde future RU/D where we may want to add new state types - } - } /** * For any given compactable entity (partition, table if not partitioned) the history of compactions @@ -720,10 +689,10 @@ public void purgeCompactionHistory() throws MetaException { if(!ci.getFullPartitionName().equals(lastCompactedEntity)) { lastCompactedEntity = ci.getFullPartitionName(); rc = new RetentionCounters(conf.getIntVar(HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_ATTEMPTED), - getFailedCompactionRetention(), + TxnUtils.getFailedCompactionRetention(conf), conf.getIntVar(HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_SUCCEEDED)); } - checkForDeletion(deleteSet, ci, rc); + TxnUtils.checkForDeletion(deleteSet, ci, rc); } close(rs); @@ -755,22 +724,7 @@ public void purgeCompactionHistory() throws MetaException { purgeCompactionHistory(); } } - /** - * this ensures that the number of failed compaction entries retained is > than number of failed - * compaction threshold which prevents new compactions from being scheduled. - */ - public int getFailedCompactionRetention() { - int failedThreshold = conf.getIntVar(HiveConf.ConfVars.COMPACTOR_INITIATOR_FAILED_THRESHOLD); - int failedRetention = conf.getIntVar(HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_FAILED); - if(failedRetention < failedThreshold) { - LOG.warn("Invalid configuration " + HiveConf.ConfVars.COMPACTOR_INITIATOR_FAILED_THRESHOLD.varname + - "=" + failedRetention + " < " + HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_FAILED + "=" + - failedRetention + ". Will use " + HiveConf.ConfVars.COMPACTOR_INITIATOR_FAILED_THRESHOLD.varname + - "=" + failedRetention); - failedRetention = failedThreshold; - } - return failedRetention; - } + /** * Returns {@code true} if there already exists sufficient number of consecutive failures for * this table/partition so that no new automatic compactions will be scheduled. diff --git metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java index be3c6de..36047f9 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java @@ -91,13 +91,6 @@ @InterfaceStability.Evolving abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI { - static final protected char INITIATED_STATE = 'i'; - static final protected char WORKING_STATE = 'w'; - static final protected char READY_FOR_CLEANING = 'r'; - static final char FAILED_STATE = 'f'; - static final char SUCCEEDED_STATE = 's'; - static final char ATTEMPTED_STATE = 'a'; - // Compactor types static final protected char MAJOR_TYPE = 'a'; static final protected char MINOR_TYPE = 'i'; diff --git metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java index 927e9bc..c9e796f 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java @@ -56,6 +56,15 @@ static final public String SUCCEEDED_RESPONSE = "succeeded"; static final public String ATTEMPTED_RESPONSE = "attempted"; + // Compactor states (Should really be enum) + static final public char INITIATED_STATE = 'i'; + static final public char WORKING_STATE = 'w'; + static final public char READY_FOR_CLEANING = 'r'; + + static final public char FAILED_STATE = 'f'; + static final public char SUCCEEDED_STATE = 's'; + static final public char ATTEMPTED_STATE = 'a'; + public static final int TIMED_OUT_TXN_ABORT_BATCH_SIZE = 1000; public void setConf(HiveConf conf); @@ -330,6 +339,17 @@ public void cleanupRecords(HiveObjectType type, Database db, Table table, */ public boolean checkFailedCompactions(CompactionInfo ci) throws MetaException; + public static class RetentionCounters { + public int attemptedRetention = 0; + public int failedRetention = 0; + public int succeededRetention = 0; + public RetentionCounters(int attemptedRetention, int failedRetention, int succeededRetention) { + this.attemptedRetention = attemptedRetention; + this.failedRetention = failedRetention; + this.succeededRetention = succeededRetention; + } + } + @VisibleForTesting public int numLocksInLockTable() throws SQLException, MetaException; diff --git metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java index cc9e583..c835732 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java @@ -207,4 +207,44 @@ private static boolean needNewQuery(HiveConf conf, StringBuilder sb) { long sizeInBytes = 8 * (((sb.length() * 2) + 45) / 8); return sizeInBytes / 1024 > queryMemoryLimit; } + + /** + * this ensures that the number of failed compaction entries retained is > than number of failed + * compaction threshold which prevents new compactions from being scheduled. + */ + public static int getFailedCompactionRetention(HiveConf conf) { + int failedThreshold = conf.getIntVar(HiveConf.ConfVars.COMPACTOR_INITIATOR_FAILED_THRESHOLD); + int failedRetention = conf.getIntVar(HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_FAILED); + if(failedRetention < failedThreshold) { + LOG.warn("Invalid configuration " + HiveConf.ConfVars.COMPACTOR_INITIATOR_FAILED_THRESHOLD.varname + + "=" + failedRetention + " < " + HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_FAILED + "=" + + failedRetention + ". Will use " + HiveConf.ConfVars.COMPACTOR_INITIATOR_FAILED_THRESHOLD.varname + + "=" + failedRetention); + failedRetention = failedThreshold; + } + return failedRetention; + } + + public static void checkForDeletion(List deleteSet, CompactionInfo ci, TxnStore.RetentionCounters + rc) { + switch (ci.state) { + case TxnStore.ATTEMPTED_STATE: + if(--rc.attemptedRetention < 0) { + deleteSet.add(ci.id); + } + break; + case TxnStore.FAILED_STATE: + if(--rc.failedRetention < 0) { + deleteSet.add(ci.id); + } + break; + case TxnStore.SUCCEEDED_STATE: + if(--rc.succeededRetention < 0) { + deleteSet.add(ci.id); + } + break; + default: + //do nothing to hanlde future RU/D where we may want to add new state types + } + } } diff --git metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto index 466fdf9..e01a9d0 100644 --- metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto +++ metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto @@ -17,6 +17,11 @@ */ package org.apache.hadoop.hive.metastore.hbase; +option java_generic_services = true; +option optimize_for = SPEED; + +// Metadata catalog objects + enum PrincipalType { USER = 0; ROLE = 1; @@ -281,3 +286,172 @@ message PartitionKeyComparator { repeated Operator op = 3; repeated Range range = 4; } + +// Transaction related persisted objects +enum CompactionState { + INITIATED = 1; + WORKING = 2; + READY_FOR_CLEANING = 3; + CLEANING = 4; + FAILED = 5; + SUCCEEDED = 6; +} + +enum CompactionType { + MINOR = 1; + MAJOR = 2; +} + +enum LockState { + ACQUIRED = 1; + WAITING = 2; + TXN_ABORTED = 3; + RELEASED = 4; // This indicates a lock has been released but the record is being kept around for later use +} + +enum LockType { + SHARED_READ = 1; + SHARED_WRITE = 2; + EXCLUSIVE = 3; + INTENTION = 4; +} + +enum TxnState { + COMMITTED = 1; + ABORTED = 2; + OPEN = 3; +} + +enum TxnStateChangeResult { + SUCCESS = 1; + NO_SUCH_TXN = 2; +} + +message Compaction { + required int64 id = 1; + optional string db = 2; + optional string table = 3; + optional string partition = 4; + required CompactionState state = 5; + required CompactionType type = 6; + optional string worker_id = 7; + optional int64 started_working_at = 8; + optional string run_as = 9; + optional int64 highest_txn_id = 10; + optional string meta_info = 11; + optional string hadoop_job_id = 12; + optional int64 end_time = 13; +} + +message Transaction { + message Lock { + required int64 id = 1; + required LockState state = 2; + required LockType type = 3; + optional string db = 4; + optional string table = 5; + optional string partition = 6; + optional int64 acquired_at = 7; + } + + required int64 id = 1; + required TxnState txn_state = 2; + optional string user = 3; + optional string hostname = 4; + optional string agent_info = 5; + optional string meta_info = 6; + optional int64 commit_id = 7; + repeated Lock locks = 8; +} + +message PotentialCompaction { + required string db = 1; + required string table = 2; + optional string partition = 3; + repeated int64 txnIds = 4; +} + +// Co-processor transaction service related messages + +message AddDynamicPartitionsRequest { + required int64 txn_id = 1; + required string db = 2; + required string table = 3; + repeated string partitions = 4; +} + +message CompactionList { + repeated Compaction compactions = 1; +} + +message GetOpenTxnsResponse { + required int64 high_water_mark = 1; + repeated int64 open_transactions = 2; + repeated int64 aborted_transactions = 3; +} + +message HeartbeatTxnRangeRequest { + required int64 min_txn = 1; // inclusive + required int64 max_txn = 2; // inclusive +} + +message HeartbeatTxnRangeResponse { + repeated int64 aborted = 1; + repeated int64 no_such = 2; +} + +message LockComponent { + required LockType type = 1; + optional LockState state = 2; + required string db = 3; + optional string table = 4; + optional string partition = 5; +} + +message LockRequest { + required int64 txn_id = 1; + repeated LockComponent components = 2; +} + +message LockResponse { + required LockState state = 1; +} + +message OpenTxnsRequest { + required int32 num_txns = 1; + required string user = 2; + required string hostname = 3; + optional string agent_info = 4; +} + +message OpenTxnsResponse { + repeated int64 txn_ids = 1; +} + +message TransactionId { + required int64 id = 1; +} + +message TransactionResult { + optional TxnStateChangeResult state = 1; +} + +message Void { + +} + +service TxnMgr { + rpc OpenTxns(OpenTxnsRequest) returns (OpenTxnsResponse); + rpc GetOpenTxns(Void) returns (GetOpenTxnsResponse); + rpc AbortTxn(TransactionId) returns (TransactionResult); + rpc CommitTxn(TransactionId) returns (TransactionResult); + rpc Heartbeat(HeartbeatTxnRangeRequest) returns (HeartbeatTxnRangeResponse); + rpc Lock(LockRequest) returns (LockResponse); + rpc CheckLocks(TransactionId) returns (LockResponse); + rpc AddDynamicPartitions(AddDynamicPartitionsRequest) returns (TransactionResult); + rpc cleanupAfterCompaction(Compaction) returns (Void); + rpc verifyCompactionCanBeCleaned(CompactionList) returns (CompactionList); + +} + + diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/MockUtils.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/MockUtils.java index 784648a..51a837c 100644 --- metastore/src/test/org/apache/hadoop/hive/metastore/hbase/MockUtils.java +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/MockUtils.java @@ -18,10 +18,12 @@ */ package org.apache.hadoop.hive.metastore.hbase; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTableInterface; @@ -29,31 +31,44 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.FileFormatProxy; import org.apache.hadoop.hive.metastore.PartitionExpressionProxy; import org.apache.hadoop.hive.metastore.api.FileMetadataExprType; import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.hbase.txn.txnmgr.TransactionCoprocessor; import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.SortedMap; +import java.util.TreeMap; /** - * Mock utilities for HBaseStore testing + * Mock utilities for HBaseStore testing. Extend this class to use the mock utilities and call + * {@link #mockInit} once you have a config file. */ public class MockUtils { + static final private Logger LOG = LoggerFactory.getLogger(MockUtils.class.getName()); + + private Map tables = new HashMap<>(); + + protected TransactionCoprocessor txnCoProc; + /** * The default impl is in ql package and is not available in unit tests. */ @@ -87,9 +102,8 @@ public FileFormatProxy getFileFormatProxy(FileMetadataExprType type) { } } - static HBaseStore init(Configuration conf, HTableInterface htable, - final SortedMap rows) throws IOException { - ((HiveConf)conf).setVar(ConfVars.METASTORE_EXPRESSION_PROXY_CLASS, NOOPProxy.class.getName()); + private void initTable(HTableInterface htable) throws Throwable { + final SortedMap rows = new TreeMap<>(); Mockito.when(htable.get(Mockito.any(Get.class))).thenAnswer(new Answer() { @Override public Result answer(InvocationOnMock invocation) throws Throwable { @@ -123,53 +137,72 @@ public Result answer(InvocationOnMock invocation) throws Throwable { } }); - Mockito.when(htable.getScanner(Mockito.any(Scan.class))).thenAnswer(new Answer() { - @Override - public ResultScanner answer(InvocationOnMock invocation) throws Throwable { - Scan scan = (Scan)invocation.getArguments()[0]; - List results = new ArrayList(); - String start = new String(scan.getStartRow()); - String stop = new String(scan.getStopRow()); - SortedMap sub = rows.subMap(start, stop); - for (Map.Entry e : sub.entrySet()) { - results.add(Result.create(new Cell[]{e.getValue()})); - } + // Filters not currently supported in scan + Mockito.when(htable.getScanner(Mockito.any(Scan.class))).thenAnswer( + new Answer() { + @Override + public ResultScanner answer(InvocationOnMock invocation) throws Throwable { + Scan scan = (Scan) invocation.getArguments()[0]; + List results = new ArrayList<>(); + String start = new String(scan.getStartRow()); + String stop = new String(scan.getStopRow()); + if (start.length() == 0) { + start = new String(new char[]{Character.MIN_VALUE}); + } + if (stop.length() == 0) { + stop = new String(new char[]{Character.MAX_VALUE}); + } + SortedMap sub = rows.subMap(start, stop); + for (Map.Entry e : sub.entrySet()) { + results.add(Result.create(new Cell[]{e.getValue()})); + } - final Iterator iter = results.iterator(); + final Iterator iter = results.iterator(); - return new ResultScanner() { - @Override - public Result next() throws IOException { - return null; - } + return new ResultScanner() { + @Override + public Result next() throws IOException { + return null; + } - @Override - public Result[] next(int nbRows) throws IOException { - return new Result[0]; - } + @Override + public Result[] next(int nbRows) throws IOException { + return new Result[0]; + } - @Override - public void close() { + @Override + public void close() { - } + } - @Override - public Iterator iterator() { - return iter; + @Override + public Iterator iterator() { + return iter; + } + }; } - }; - } - }); + }); Mockito.doAnswer(new Answer() { @Override public Void answer(InvocationOnMock invocation) throws Throwable { - Put put = (Put)invocation.getArguments()[0]; + Put put = (Put) invocation.getArguments()[0]; rows.put(new String(put.getRow()), put.getFamilyCellMap().firstEntry().getValue().get(0)); return null; } }).when(htable).put(Mockito.any(Put.class)); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocationOnMock) throws Throwable { + List puts = (List) invocationOnMock.getArguments()[0]; + for (Put put : puts) { + rows.put(new String(put.getRow()), put.getFamilyCellMap().firstEntry().getValue().get(0)); + } + return null; + } + }).when(htable).put(Mockito.anyListOf(Put.class)); + Mockito.when(htable.checkAndPut(Mockito.any(byte[].class), Mockito.any(byte[].class), Mockito.any(byte[].class), Mockito.any(byte[].class), Mockito.any(Put.class))).thenAnswer( new Answer() { @@ -177,7 +210,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { @Override public Boolean answer(InvocationOnMock invocation) throws Throwable { // Always say it succeeded and overwrite - Put put = (Put)invocation.getArguments()[4]; + Put put = (Put) invocation.getArguments()[4]; rows.put(new String(put.getRow()), put.getFamilyCellMap().firstEntry().getValue().get(0)); return true; @@ -187,33 +220,140 @@ public Boolean answer(InvocationOnMock invocation) throws Throwable { Mockito.doAnswer(new Answer() { @Override public Void answer(InvocationOnMock invocation) throws Throwable { - Delete del = (Delete)invocation.getArguments()[0]; + Delete del = (Delete) invocation.getArguments()[0]; rows.remove(new String(del.getRow())); return null; } }).when(htable).delete(Mockito.any(Delete.class)); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocationOnMock) throws Throwable { + List deletes = (List) invocationOnMock.getArguments()[0]; + for (Delete delete : deletes) { + rows.remove(new String(delete.getRow())); + } + return null; + } + }).when(htable).delete(Mockito.anyListOf(Delete.class)); + Mockito.when(htable.checkAndDelete(Mockito.any(byte[].class), Mockito.any(byte[].class), - Mockito.any(byte[].class), Mockito.any(byte[].class), Mockito.any(Delete.class))).thenAnswer( + Mockito.any(byte[].class), Mockito.any(byte[].class), + Mockito.any(Delete.class))).thenAnswer( new Answer() { @Override public Boolean answer(InvocationOnMock invocation) throws Throwable { // Always say it succeeded - Delete del = (Delete)invocation.getArguments()[4]; + Delete del = (Delete) invocation.getArguments()[4]; rows.remove(new String(del.getRow())); return true; } }); + Mockito.when(htable.coprocessorService(Mockito.any(Class.class), Mockito.any(byte[].class), + Mockito.any(byte[].class), Mockito.any(Batch.Call.class))) + .thenAnswer( + new Answer() { + @Override + public Map answer(InvocationOnMock invocationOnMock) throws Throwable { + Class clazz = + (Class)invocationOnMock.getArguments()[0]; + if (!clazz.equals(TransactionCoprocessor.class)) { + throw new RuntimeException("Only know how to do TransactionCoprocessor"); + } + Batch.Call call = (Batch.Call)invocationOnMock.getArguments()[3]; + Object o = call.call(txnCoProc); + return Collections.singletonMap((byte[])null, o); + } + } + ); + } + + protected HBaseStore mockInit(Configuration conf) throws IOException { + ((HiveConf) conf).setVar(ConfVars.METASTORE_EXPRESSION_PROXY_CLASS, + NOOPProxy.class.getName()); // Mock connection HBaseConnection hconn = Mockito.mock(HBaseConnection.class); - Mockito.when(hconn.getHBaseTable(Mockito.anyString())).thenReturn(htable); - HiveConf.setVar(conf, HiveConf.ConfVars.METASTORE_HBASE_CONNECTION_CLASS, HBaseReadWrite.TEST_CONN); + Mockito.when(hconn.getHBaseTable(Mockito.anyString())).thenAnswer( + new Answer() { + @Override + public HTableInterface answer(InvocationOnMock invocationOnMock) throws Throwable { + String tableName = (String)invocationOnMock.getArguments()[0]; + HTableInterface table = tables.get(tableName); + if (table == null) { + table = Mockito.mock(HTableInterface.class); + initTable(table); + tables.put(tableName, table); + } + return table; + } + } + ); + // Setup HBase Store + HiveConf.setVar(conf, HiveConf.ConfVars.METASTORE_HBASE_CONNECTION_CLASS, + HBaseReadWrite.TEST_CONN); HBaseReadWrite.setTestConnection(hconn); HBaseReadWrite.setConf(conf); HBaseStore store = new HBaseStore(); store.setConf(conf); + + // Setup the txn coprocessor + txnCoProc = new TransactionCoprocessor(); + CoprocessorEnvironment env = Mockito.mock(CoprocessorEnvironment.class); + Mockito.when(env.getConfiguration()).thenReturn(conf); + txnCoProc.start(env); return store; } + + protected void mockShutdown() throws IOException { + CoprocessorEnvironment env = Mockito.mock(CoprocessorEnvironment.class); + txnCoProc.stop(env); + } + + protected RpcController getController() { + /* + RpcController controller = Mockito.mock(RpcController.class); + // TODO Going to have to figure out what setControllerException calls and then mock that + return controller; + */ + return new RpcController() { + private String err = null; + + @Override + public void reset() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean failed() { + return err != null; + } + + @Override + public String errorText() { + return err; + } + + @Override + public void startCancel() { + throw new UnsupportedOperationException(); + } + + @Override + public void setFailed(String s) { + err = s; + } + + @Override + public boolean isCanceled() { + return false; + } + + @Override + public void notifyOnCancel(RpcCallback rpcCallback) { + throw new UnsupportedOperationException(); + } + }; + } } diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCache.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCache.java index 6cd3a46..a3b610e 100644 --- metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCache.java +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCache.java @@ -19,10 +19,6 @@ package org.apache.hadoop.hive.metastore.hbase; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.AggrStats; import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData; @@ -39,30 +35,27 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.SortedMap; -import java.util.TreeMap; -public class TestHBaseAggregateStatsCache { +public class TestHBaseAggregateStatsCache extends MockUtils { private static final Logger LOG = LoggerFactory.getLogger(TestHBaseAggregateStatsCache.class.getName()); - @Mock HTableInterface htable; private HBaseStore store; - SortedMap rows = new TreeMap<>(); @Before public void before() throws IOException { MockitoAnnotations.initMocks(this); HiveConf conf = new HiveConf(); conf.setBoolean(HBaseReadWrite.NO_CACHE_CONF, true); - store = MockUtils.init(conf, htable, rows); + store = mockInit(conf); store.backdoor().getStatsCache().resetCounters(); } diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java index e0c4094..153352d 100644 --- metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java @@ -18,13 +18,8 @@ */ package org.apache.hadoop.hive.metastore.hbase; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.AggrStats; -import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc; @@ -38,32 +33,28 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.SortedMap; -import java.util.TreeMap; -public class TestHBaseAggregateStatsCacheWithBitVector { +public class TestHBaseAggregateStatsCacheWithBitVector extends MockUtils { private static final Logger LOG = LoggerFactory .getLogger(TestHBaseAggregateStatsCacheWithBitVector.class.getName()); - @Mock - HTableInterface htable; private HBaseStore store; - SortedMap rows = new TreeMap<>(); @Before public void before() throws IOException { MockitoAnnotations.initMocks(this); HiveConf conf = new HiveConf(); conf.setBoolean(HBaseReadWrite.NO_CACHE_CONF, true); - store = MockUtils.init(conf, htable, rows); + store = mockInit(conf); store.backdoor().getStatsCache().resetCounters(); } diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java index f4e55ed..c3a46d4 100644 --- metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java @@ -18,14 +18,9 @@ */ package org.apache.hadoop.hive.metastore.hbase; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.StatObjectConverter; import org.apache.hadoop.hive.metastore.api.AggrStats; -import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc; @@ -42,25 +37,21 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.SortedMap; -import java.util.TreeMap; -public class TestHBaseAggregateStatsExtrapolation { +public class TestHBaseAggregateStatsExtrapolation extends MockUtils { private static final Logger LOG = LoggerFactory .getLogger(TestHBaseAggregateStatsExtrapolation.class.getName()); - @Mock - HTableInterface htable; private HBaseStore store; - SortedMap rows = new TreeMap<>(); // NDV will be 3 for the bitVectors String bitVectors = "{0, 4, 5, 7}{0, 1}{0, 1, 2}{0, 1, 4}{0}{0, 2}{0, 3}{0, 2, 3, 4}{0, 1, 4}{0, 1}{0}{0, 1, 3, 8}{0, 2}{0, 2}{0, 9}{0, 1, 4}"; @@ -70,7 +61,7 @@ public void before() throws IOException { MockitoAnnotations.initMocks(this); HiveConf conf = new HiveConf(); conf.setBoolean(HBaseReadWrite.NO_CACHE_CONF, true); - store = MockUtils.init(conf, htable, rows); + store = mockInit(conf); store.backdoor().getStatsCache().resetCounters(); } diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java index 62918be..bb5f6d6 100644 --- metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java @@ -18,14 +18,9 @@ */ package org.apache.hadoop.hive.metastore.hbase; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.StatObjectConverter; import org.apache.hadoop.hive.metastore.api.AggrStats; -import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc; @@ -41,25 +36,21 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.SortedMap; -import java.util.TreeMap; -public class TestHBaseAggregateStatsNDVUniformDist { +public class TestHBaseAggregateStatsNDVUniformDist extends MockUtils { private static final Logger LOG = LoggerFactory .getLogger(TestHBaseAggregateStatsNDVUniformDist.class.getName()); - @Mock - HTableInterface htable; private HBaseStore store; - SortedMap rows = new TreeMap<>(); // NDV will be 3 for bitVectors[0] and 12 for bitVectors[1] String bitVectors[] = { @@ -72,7 +63,7 @@ public void before() throws IOException { HiveConf conf = new HiveConf(); conf.setBoolean(HBaseReadWrite.NO_CACHE_CONF, true); conf.setBoolean(HiveConf.ConfVars.HIVE_METASTORE_STATS_NDV_DENSITY_FUNCTION.varname, true); - store = MockUtils.init(conf, htable, rows); + store = mockInit(conf); store.backdoor().getStatsCache().resetCounters(); } diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java index e4723f6..905918b 100644 --- metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java @@ -25,11 +25,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData; import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData; @@ -64,7 +60,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; -import org.mockito.Mock; import org.mockito.MockitoAnnotations; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,7 +67,7 @@ /** * */ -public class TestHBaseStore { +public class TestHBaseStore extends MockUtils { private static final Logger LOG = LoggerFactory.getLogger(TestHBaseStore.class.getName()); static Map emptyParameters = new HashMap(); // Table with NUM_PART_KEYS partitioning keys and NUM_PARTITIONS values per key @@ -130,8 +125,6 @@ NUM_PARTITIONS); @Rule public ExpectedException thrown = ExpectedException.none(); - @Mock HTableInterface htable; - SortedMap rows = new TreeMap<>(); HBaseStore store; @@ -236,7 +229,7 @@ private static ColumnStatisticsObj mockStringStats(int i) { return colStatsObj; } - private static ColumnStatisticsObj mockBinaryStats(int i) {; + private static ColumnStatisticsObj mockBinaryStats(int i) { long maxLen = 123412987L + 10*i; double avgLen = 76.98 + i; long nulls = 976998797L + 10*i; @@ -288,7 +281,7 @@ public void init() throws IOException { MockitoAnnotations.initMocks(this); HiveConf conf = new HiveConf(); conf.setBoolean(HBaseReadWrite.NO_CACHE_CONF, true); - store = MockUtils.init(conf, htable, rows); + store = mockInit(conf); } @Test diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreBitVector.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreBitVector.java index b1dc542..6dfe24d 100644 --- metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreBitVector.java +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreBitVector.java @@ -24,11 +24,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; @@ -52,7 +48,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; -import org.mockito.Mock; import org.mockito.MockitoAnnotations; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,7 +55,7 @@ /** * */ -public class TestHBaseStoreBitVector { +public class TestHBaseStoreBitVector extends MockUtils { private static final Logger LOG = LoggerFactory.getLogger(TestHBaseStoreBitVector.class.getName()); static Map emptyParameters = new HashMap(); // Table with NUM_PART_KEYS partitioning keys and NUM_PARTITIONS values per key @@ -108,8 +103,6 @@ NUM_PARTITIONS); @Rule public ExpectedException thrown = ExpectedException.none(); - @Mock HTableInterface htable; - SortedMap rows = new TreeMap<>(); HBaseStore store; @@ -236,7 +229,7 @@ public void init() throws IOException { MockitoAnnotations.initMocks(this); HiveConf conf = new HiveConf(); conf.setBoolean(HBaseReadWrite.NO_CACHE_CONF, true); - store = MockUtils.init(conf, htable, rows); + store = mockInit(conf); } @Test diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreCached.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreCached.java index cfe9cd0..d8b5520 100644 --- metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreCached.java +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreCached.java @@ -18,77 +18,49 @@ */ package org.apache.hadoop.hive.metastore.hbase; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.HTableInterface; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData; import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.Decimal; -import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData; -import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData; import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.LongColumnStatsData; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.Role; import org.apache.hadoop.hive.metastore.api.SerDeInfo; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; -import org.apache.hadoop.hive.metastore.api.StringColumnStatsData; import org.apache.hadoop.hive.metastore.api.Table; import org.junit.Assert; import org.junit.Before; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; -import org.mockito.Mock; -import org.mockito.Mockito; import org.mockito.MockitoAnnotations; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; /** * */ -public class TestHBaseStoreCached { +public class TestHBaseStoreCached extends MockUtils { private static final Logger LOG = LoggerFactory.getLogger(TestHBaseStoreCached.class.getName()); static Map emptyParameters = new HashMap(); @Rule public ExpectedException thrown = ExpectedException.none(); - @Mock HTableInterface htable; - SortedMap rows = new TreeMap(); HBaseStore store; @Before public void init() throws IOException { MockitoAnnotations.initMocks(this); HiveConf conf = new HiveConf(); - store = MockUtils.init(conf, htable, rows); + store = mockInit(conf); } @Test diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestHBaseTxnHandler.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestHBaseTxnHandler.java new file mode 100644 index 0000000..4f91a52 --- /dev/null +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestHBaseTxnHandler.java @@ -0,0 +1,252 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; +import org.apache.hadoop.hive.metastore.api.CheckLockRequest; +import org.apache.hadoop.hive.metastore.api.CommitTxnRequest; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; +import org.apache.hadoop.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockLevel; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.LockState; +import org.apache.hadoop.hive.metastore.api.LockType; +import org.apache.hadoop.hive.metastore.api.OpenTxnRequest; +import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse; +import org.apache.hadoop.hive.metastore.api.TxnInfo; +import org.apache.hadoop.hive.metastore.hbase.HBaseReadWrite; +import org.apache.hadoop.hive.metastore.hbase.HBaseStore; +import org.apache.hadoop.hive.metastore.hbase.MockUtils; +import org.apache.hadoop.hive.metastore.hbase.txn.HBaseTxnHandler; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +/** + * Tests for HBaseTxnHandler. This is focussed on making sure the txn handler properly calls the + * coprocessor. Functionality that is handled completely by the transaction handler is also + * tested here. + */ +public class TestHBaseTxnHandler extends MockUtils { + HBaseStore store; + HBaseReadWrite hrw; + HBaseTxnHandler txnHandler; + + @Before + public void init() throws IOException { + MockitoAnnotations.initMocks(this); + HiveConf conf = new HiveConf(); + + // Set the wait on the background threads to max long so that they don't run and clean things + // up on us, since we're trying to check state. + conf.set(TransactionManager.CONF_NO_AUTO_BACKGROUND_THREADS, Boolean.toString(Boolean.TRUE)); + // Set this super low so the test doesn't take forever + HiveConf.setTimeVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_LOCK_POLL_TIMEOUT, + 100, TimeUnit.MILLISECONDS); + // Set this value lower so we can fill up the txn mgr without creating an insane number of + // objects. + HiveConf.setIntVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_MAX_OBJECTS, 100); + + store = mockInit(conf); + hrw = HBaseReadWrite.getInstance(); + txnHandler = new HBaseTxnHandler(); + txnHandler.setConf(conf); + } + + @After + public void cleanup() throws IOException { + mockShutdown(); + } + + @Test + public void openGetInfoAbort() throws Exception { + OpenTxnRequest txnRequest = new OpenTxnRequest(1, "me", "localhost"); + txnRequest.setAgentInfo("agent"); + OpenTxnsResponse txnsResponse = txnHandler.openTxns(txnRequest); + List txnIds = txnsResponse.getTxn_ids(); + Assert.assertEquals(1, txnIds.size()); + long txnId = txnIds.get(0); + + // Check that our transaction is open + GetOpenTxnsResponse openTxns = txnHandler.getOpenTxns(); + Set openTxnIds = openTxns.getOpen_txns(); + Assert.assertTrue(openTxnIds.contains(txnId)); + long hwm = openTxns.getTxn_high_water_mark(); + + // Check that we can get info about our transaction + GetOpenTxnsInfoResponse txnInfo = txnHandler.getOpenTxnsInfo(); + Assert.assertEquals(hwm, txnInfo.getTxn_high_water_mark()); + List infos = txnInfo.getOpen_txns(); + boolean foundOurs = false; + for (TxnInfo info : infos) { + if (info.getId() == txnId) { + foundOurs = true; + Assert.assertEquals("me", info.getUser()); + Assert.assertEquals("localhost", info.getHostname()); + Assert.assertEquals("agent", info.getAgentInfo()); + } + } + Assert.assertTrue(foundOurs); + + AbortTxnRequest abort = new AbortTxnRequest(txnId); + txnHandler.abortTxn(abort); + } + + @Test + public void openLockHeartbeatCommit() throws Exception { + String db = "olhc_db"; + String table = "olhc_t"; + + OpenTxnRequest txnRequest = new OpenTxnRequest(2, "me", "localhost"); + txnRequest.setAgentInfo("agent"); + OpenTxnsResponse txnsResponse = txnHandler.openTxns(txnRequest); + List txnIds = txnsResponse.getTxn_ids(); + Assert.assertEquals(2, txnIds.size()); + long firstTxn = txnIds.get(0); + long secondTxn = txnIds.get(1); + + // Check that we can get info about our transaction + GetOpenTxnsInfoResponse txnInfo = txnHandler.getOpenTxnsInfo(); + List infos = txnInfo.getOpen_txns(); + boolean foundFirst = false, foundSecond = false; + for (TxnInfo info : infos) { + if (info.getId() == firstTxn) { + foundFirst = true; + Assert.assertEquals("me", info.getUser()); + Assert.assertEquals("localhost", info.getHostname()); + Assert.assertEquals("agent", info.getAgentInfo()); + } else if (info.getId() == secondTxn) { + foundSecond = true; + Assert.assertEquals("me", info.getUser()); + Assert.assertEquals("localhost", info.getHostname()); + Assert.assertEquals("agent", info.getAgentInfo()); + } + } + Assert.assertTrue(foundFirst && foundSecond); + + // Lock from the first txn + List components = new ArrayList<>(2); + LockComponent component = new LockComponent(LockType.INTENTION, LockLevel.DB, db); + components.add(component); + component = new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, db); + component.setTablename(table); + components.add(component); + + LockRequest lock = new LockRequest(components, "me", "localhost"); + lock.setAgentInfo("agent"); + lock.setTxnid(firstTxn); + LockResponse lockResponse = txnHandler.lock(lock); + Assert.assertEquals(LockState.ACQUIRED, lockResponse.getState()); + + // Lock from the second txn + components = new ArrayList<>(2); + component = new LockComponent(LockType.INTENTION, LockLevel.DB, db); + components.add(component); + component = new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, db); + component.setTablename(table); + components.add(component); + + lock = new LockRequest(components, "me", "localhost"); + lock.setAgentInfo("second_agent"); + lock.setTxnid(secondTxn); + lockResponse = txnHandler.lock(lock); + Assert.assertEquals(LockState.WAITING, lockResponse.getState()); + + // Commit the first transaction + CommitTxnRequest commit = new CommitTxnRequest(firstTxn); + txnHandler.commitTxn(commit); + + CheckLockRequest checkLockRequest = new CheckLockRequest(-1); + checkLockRequest.setTxnid(secondTxn); + lockResponse = txnHandler.checkLock(checkLockRequest); + Assert.assertEquals(LockState.ACQUIRED, lockResponse.getState()); + + commit = new CommitTxnRequest(secondTxn); + txnHandler.commitTxn(commit); + } + + @Test + public void show() throws Exception { + // Test all the various options in show + String dbs[] = { "show_db1", "show_db2" }; + String tables[] = { "show_table1", "show_table2" }; + String parts[] = { "show_part1", "show_part2" }; + + OpenTxnRequest txnRequest = new OpenTxnRequest(2, "me", "localhost"); + txnRequest.setAgentInfo("agent"); + OpenTxnsResponse txnsResponse = txnHandler.openTxns(txnRequest); + List txnIds = txnsResponse.getTxn_ids(); + Assert.assertEquals(2, txnIds.size()); + long firstTxn = txnIds.get(0); + long secondTxn = txnIds.get(1); + + List components = new ArrayList<>(); + LockComponent component = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, dbs[0]); + components.add(component); + + component = new LockComponent(LockType.INTENTION, LockLevel.DB, dbs[1]); + components.add(component); + component = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, dbs[1]); + component.setTablename(tables[0]); + components.add(component); + + component = new LockComponent(LockType.INTENTION, LockLevel.DB, dbs[1]); + components.add(component); + component = new LockComponent(LockType.INTENTION, LockLevel.TABLE, dbs[1]); + component.setTablename(tables[1]); + component = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, dbs[1]); + component.setTablename(tables[1]); + component.setPartitionname(parts[0]); + components.add(component); + + LockRequest lock = new LockRequest(components, "me", "localhost"); + lock.setAgentInfo("agent"); + lock.setTxnid(firstTxn); + LockResponse lockResponse = txnHandler.lock(lock); + Assert.assertEquals(LockState.ACQUIRED, lockResponse.getState()); + + components = new ArrayList<>(); + component = new LockComponent(LockType.INTENTION, LockLevel.DB, dbs[1]); + components.add(component); + component = new LockComponent(LockType.INTENTION, LockLevel.TABLE, dbs[1]); + component.setTablename(tables[1]); + component = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, dbs[1]); + component.setTablename(tables[1]); + component.setPartitionname(parts[1]); + components.add(component); + + lock = new LockRequest(components, "me", "localhost"); + lock.setAgentInfo("agent"); + lock.setTxnid(firstTxn); + lockResponse = txnHandler.lock(lock); + Assert.assertEquals(LockState.ACQUIRED, lockResponse.getState()); + + //ShowLocksResponse locksResponse = txnHandler.showLocks(); + } +} diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionCoprocessor.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionCoprocessor.java new file mode 100644 index 0000000..cfe8746 --- /dev/null +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionCoprocessor.java @@ -0,0 +1,337 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import com.google.protobuf.RpcController; +import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.hbase.HBaseReadWrite; +import org.apache.hadoop.hive.metastore.hbase.HBaseStore; +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; +import org.apache.hadoop.hive.metastore.hbase.MockUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.util.Arrays; +import java.util.concurrent.TimeUnit; + +/** + * This just tests that we can communicate through the coprocessor. Actual functionality of the + * transaction manager is tested in the TestTransactionManger* classes. This should call every + * RPC call in the transaction coprocessor. + */ +public class TestTransactionCoprocessor extends MockUtils { + HBaseStore store; + HBaseReadWrite hrw; + + @Before + public void init() throws IOException { + MockitoAnnotations.initMocks(this); + HiveConf conf = new HiveConf(); + + // Set the wait on the background threads to max long so that they don't run and clean things + // up on us, since we're trying to check state. + conf.set(TransactionManager.CONF_NO_AUTO_BACKGROUND_THREADS, Boolean.toString(Boolean.TRUE)); + // Set this super low so the test doesn't take forever + HiveConf.setTimeVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_LOCK_POLL_TIMEOUT, + 100, TimeUnit.MILLISECONDS); + // Set this value lower so we can fill up the txn mgr without creating an insane number of + // objects. + HiveConf.setIntVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_MAX_OBJECTS, 100); + + store = mockInit(conf); + hrw = HBaseReadWrite.getInstance(); + } + + @After + public void cleanup() throws IOException { + mockShutdown(); + } + + @Test + public void openAndAbort() throws Exception { + + // Open a single transaction + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + + BlockingRpcCallback openRpcRsp = + new BlockingRpcCallback<>(); + txnCoProc.openTxns(getController(), rqst, openRpcRsp); + + HbaseMetastoreProto.OpenTxnsResponse openRsp = openRpcRsp.get(); + Assert.assertEquals(1, openRsp.getTxnIdsCount()); + long txnId = openRsp.getTxnIds(0); + + BlockingRpcCallback txnRpcRsp = + new BlockingRpcCallback<>(); + txnCoProc.getOpenTxns(getController(), HbaseMetastoreProto.Void.getDefaultInstance(), txnRpcRsp); + + HbaseMetastoreProto.GetOpenTxnsResponse validTxns = txnRpcRsp.get(); + Assert.assertNotNull(validTxns); + + // Make sure there's at least one open txn + Assert.assertTrue(validTxns.getOpenTransactionsCount() > 0); + + // Make sure our transaction is open + boolean sawOurTxn = false; + for (long txn : validTxns.getOpenTransactionsList()) { + if (txn == txnId) { + sawOurTxn = true; + break; + } + } + Assert.assertTrue(sawOurTxn); + + // Abort this transaction. It should promptly be forgotten as it has no locks + HbaseMetastoreProto.TransactionId toAbort = HbaseMetastoreProto.TransactionId.newBuilder() + .setId(txnId) + .build(); + + BlockingRpcCallback abortRpcRsp = + new BlockingRpcCallback<>(); + + txnCoProc.abortTxn(getController(), toAbort, abortRpcRsp); + HbaseMetastoreProto.TransactionResult abortRsp = abortRpcRsp.get(); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, abortRsp.getState()); + + + // Our txn won't get put in the abort list because it didn't have any locks + txnRpcRsp = new BlockingRpcCallback<>(); + txnCoProc.getOpenTxns(getController(), HbaseMetastoreProto.Void.getDefaultInstance(), txnRpcRsp); + + validTxns = txnRpcRsp.get(); + Assert.assertNotNull(validTxns); + + } + + public void openAndCommit() throws Exception { + String db = "oac_db"; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(2) + .setUser("me") + .setHostname("localhost") + .build(); + + BlockingRpcCallback openRpcRsp = + new BlockingRpcCallback<>(); + txnCoProc.openTxns(getController(), rqst, openRpcRsp); + + HbaseMetastoreProto.OpenTxnsResponse openRsp = openRpcRsp.get(); + Assert.assertEquals(2, openRsp.getTxnIdsCount()); + long firstTxnId = openRsp.getTxnIds(0); + long secondTxnId = openRsp.getTxnIds(1); + + // Send a heartbeat for these transactions + HbaseMetastoreProto.HeartbeatTxnRangeRequest heartbeat = + HbaseMetastoreProto.HeartbeatTxnRangeRequest.newBuilder() + .setMinTxn(firstTxnId) + .setMaxTxn(secondTxnId) + .build(); + BlockingRpcCallback heartbeatRpcRsp = + new BlockingRpcCallback<>(); + txnCoProc.heartbeat(getController(), heartbeat, heartbeatRpcRsp); + HbaseMetastoreProto.HeartbeatTxnRangeResponse heartbeatRsp = heartbeatRpcRsp.get(); + Assert.assertEquals(0, heartbeatRsp.getAbortedCount()); + Assert.assertEquals(0, heartbeatRsp.getNoSuchCount()); + + // Lock in one transaction + HbaseMetastoreProto.LockRequest lockRequest = HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxnId) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build(); + BlockingRpcCallback lockRpcRsp = new BlockingRpcCallback<>(); + txnCoProc.lock(getController(), lockRequest, lockRpcRsp); + HbaseMetastoreProto.LockResponse lockRsp = lockRpcRsp.get(); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lockRsp.getState()); + + // Lock in the other transaction + lockRequest = HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxnId) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build(); + lockRpcRsp = new BlockingRpcCallback<>(); + txnCoProc.lock(getController(), lockRequest, lockRpcRsp); + lockRsp = lockRpcRsp.get(); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lockRsp.getState()); + + // Commit the first transaction + HbaseMetastoreProto.TransactionId toCommit = HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxnId) + .build(); + + BlockingRpcCallback commitRpcRsp = + new BlockingRpcCallback<>(); + + txnCoProc.commitTxn(getController(), toCommit, commitRpcRsp); + HbaseMetastoreProto.TransactionResult commitRsp = commitRpcRsp.get(); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commitRsp.getState()); + + // Check the locks on the second txn + HbaseMetastoreProto.TransactionId toCheck = HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxnId) + .build(); + lockRpcRsp = new BlockingRpcCallback<>(); + txnCoProc.checkLocks(getController(), toCheck, lockRpcRsp); + lockRsp = lockRpcRsp.get(); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lockRsp.getState()); + + // Commit the second txn just to keep things clean + toCommit = HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxnId) + .build(); + + commitRpcRsp = new BlockingRpcCallback<>(); + + txnCoProc.commitTxn(getController(), toCommit, commitRpcRsp); + commitRsp = commitRpcRsp.get(); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commitRsp.getState()); + } + + public void dynamicPartitionsAndCompaction() throws Exception { + String db = "dpac_db"; + String table = "dpac_t"; + String partbase = "dpac_p"; + + // Open a single transaction + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + + BlockingRpcCallback openRpcRsp = + new BlockingRpcCallback<>(); + txnCoProc.openTxns(getController(), rqst, openRpcRsp); + + HbaseMetastoreProto.OpenTxnsResponse openRsp = openRpcRsp.get(); + Assert.assertEquals(1, openRsp.getTxnIdsCount()); + long txnId = openRsp.getTxnIds(0); + + // Add dynamic partitions + HbaseMetastoreProto.AddDynamicPartitionsRequest dParts = + HbaseMetastoreProto.AddDynamicPartitionsRequest.newBuilder() + .setDb(db) + .setTable(table) + .setTxnId(txnId) + .addAllPartitions(Arrays.asList(partbase + "1", partbase + "2")) + .build(); + BlockingRpcCallback dPartsRpcRsp = + new BlockingRpcCallback<>(); + txnCoProc.addDynamicPartitions(getController(), dParts, dPartsRpcRsp); + HbaseMetastoreProto.TransactionResult dPartsRsp = dPartsRpcRsp.get(); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, dPartsRsp.getState()); + + // Commit this transaction. + HbaseMetastoreProto.TransactionId toCommit = HbaseMetastoreProto.TransactionId.newBuilder() + .setId(txnId) + .build(); + + BlockingRpcCallback commitRpcRsp = + new BlockingRpcCallback<>(); + + txnCoProc.commitTxn(getController(), toCommit, commitRpcRsp); + HbaseMetastoreProto.TransactionResult commitRsp = commitRpcRsp.get(); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commitRsp.getState()); + + // Cleanupt after a compaction + HbaseMetastoreProto.Compaction compaction = HbaseMetastoreProto.Compaction.newBuilder() + .setDb(db) + .setTable(table) + .setPartition(partbase + "1") + .setId(1) + .build(); + BlockingRpcCallback compactionRpcRsp = new BlockingRpcCallback<>(); + txnCoProc.cleanupAfterCompaction(getController(), compaction, compactionRpcRsp); + + // Verify compaction can be cleaned up + HbaseMetastoreProto.CompactionList clist = HbaseMetastoreProto.CompactionList.newBuilder() + .addCompactions(HbaseMetastoreProto.Compaction.newBuilder() + .setDb(db) + .setTable(table) + .setPartition(partbase + "1") + .setId(1)) + .build(); + BlockingRpcCallback clistRpcRsp = + new BlockingRpcCallback<>(); + txnCoProc.verifyCompactionCanBeCleaned(getController(), clist, clistRpcRsp); + HbaseMetastoreProto.CompactionList clistRsp = clistRpcRsp.get(); + // Just make sure the response contains something + Assert.assertEquals(1, clistRsp.getCompactionsCount()); + + } + + @Test + public void full() throws Exception { + // This tests what happens when the TransactionManager throws an exception back to the + // co-processor. Currently that happens when the txnmgr is full and more txns are opened. + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(101) + .setUser("me") + .setHostname("localhost") + .build(); + + BlockingRpcCallback openRpcRsp = + new BlockingRpcCallback<>(); + txnCoProc.openTxns(getController(), rqst, openRpcRsp); + + HbaseMetastoreProto.OpenTxnsResponse openRsp = openRpcRsp.get(); + Assert.assertEquals(101, openRsp.getTxnIdsCount()); + long firstTxn = openRsp.getTxnIds(0); + + txnCoProc.backdoor().forceFullChecker(); + + // Open another transaction, this should fail. + RpcController controller = getController(); + + openRpcRsp = new BlockingRpcCallback<>(); + txnCoProc.openTxns(controller, rqst, openRpcRsp); + Assert.assertTrue(controller.failed()); + Assert.assertTrue(controller.errorText().contains("Full, no new transactions being accepted")); + + // Abort all the transactions so we don't leave it full + for (int i = 0; i < 101; i++) { + // Abort this transaction. It should promptly be forgotten as it has no locks + HbaseMetastoreProto.TransactionId toAbort = HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn + i) + .build(); + + BlockingRpcCallback abortRpcRsp = + new BlockingRpcCallback<>(); + + txnCoProc.abortTxn(getController(), toAbort, abortRpcRsp); + HbaseMetastoreProto.TransactionResult abortRsp = abortRpcRsp.get(); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, abortRsp.getState()); + } + txnCoProc.backdoor().forceFullChecker(); + } + // TODO write wrapper around TestTxnHandler to re-route its calls to HBase metastore +} + diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManager.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManager.java new file mode 100644 index 0000000..0f9ca88 --- /dev/null +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManager.java @@ -0,0 +1,1466 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.hbase.HBaseReadWrite; +import org.apache.hadoop.hive.metastore.hbase.HBaseStore; +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; +import org.apache.hadoop.hive.metastore.hbase.MockUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.Test; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Map; +import java.util.Set; + +/** + * Most positive functionality for TestTransactionManager, including most of the background + * threads. Try to keep timing related tests out of here as we will need to exclude those from + * Jenkins runs since they end up being flaky. + */ +public class TestTransactionManager extends MockUtils { + + HBaseStore store; + TransactionManager txnMgr; + HBaseReadWrite hrw; + + @Before + public void init() throws IOException { + MockitoAnnotations.initMocks(this); + HiveConf conf = new HiveConf(); + + // Set the wait on the background threads to max long so that they don't run and clean things + // up on us, since we're trying to check state. + conf.set(TransactionManager.CONF_NO_AUTO_BACKGROUND_THREADS, Boolean.toString(Boolean.TRUE)); + + store = mockInit(conf); + txnMgr = txnCoProc.backdoor(); + hrw = HBaseReadWrite.getInstance(); + } + + @After + public void cleanup() throws IOException { + txnMgr.shutdown(); + } + + @Test + public void openAndAbort() throws Exception { + + HbaseMetastoreProto.GetOpenTxnsResponse before = + txnMgr.getOpenTxns(HbaseMetastoreProto.Void.getDefaultInstance()); + Map memoryBefore = txnMgr.copyOpenTransactions(); + + // Open a single transaction + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(1, rsp.getTxnIdsCount()); + long txnId = rsp.getTxnIds(0); + + HbaseMetastoreProto.GetOpenTxnsResponse after = + txnMgr.getOpenTxns(HbaseMetastoreProto.Void.getDefaultInstance()); + Map memoryAfter = txnMgr.copyOpenTransactions(); + Map abortedMemoryAfter = txnMgr.copyAbortedTransactions(); + + // Check that our transaction id is the same as the previous high water mark + Assert.assertEquals(before.getHighWaterMark(), txnId); + + // Check that the highwater mark went up by one + Assert.assertEquals(before.getHighWaterMark() + 1, after.getHighWaterMark()); + + // Check that we have one more open transaction and the same number of aborted transactions + Assert.assertEquals(before.getOpenTransactionsCount() + 1, after.getOpenTransactionsCount()); + Assert.assertEquals(before.getAbortedTransactionsCount(), after.getAbortedTransactionsCount()); + + // Make sure only one value got added to the in memory structure + Assert.assertEquals(memoryBefore.size() + 1, memoryAfter.size()); + + // Make sure our entry is in memory + OpenHiveTransaction newTxn = memoryAfter.get(txnId); + Assert.assertNotNull(newTxn); + + // Make sure our entry is in the open state + Assert.assertEquals(HbaseMetastoreProto.TxnState.OPEN, newTxn.getState()); + + // Make sure the heartbeat is set + Assert.assertTrue(newTxn.getLastHeartbeat() > 0); + + // Make sure it doesn't have any locks + Assert.assertNull(newTxn.getHiveLocks()); + + // Check the transaction in HBase + HbaseMetastoreProto.Transaction hbaseTxn = hrw.getTransaction(txnId); + Assert.assertNotNull(hbaseTxn); + Assert.assertEquals(txnId, hbaseTxn.getId()); + Assert.assertEquals(0, hbaseTxn.getLocksCount()); + Assert.assertEquals("me", hbaseTxn.getUser()); + Assert.assertEquals("localhost", hbaseTxn.getHostname()); + Assert.assertEquals(HbaseMetastoreProto.TxnState.OPEN, hbaseTxn.getTxnState()); + + // Abort this transaction. It should promptly be forgotten as it has no locks + HbaseMetastoreProto.TransactionResult abort = + txnMgr.abortTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, abort.getState()); + + HbaseMetastoreProto.GetOpenTxnsResponse afterAbort = + txnMgr.getOpenTxns(HbaseMetastoreProto.Void.getDefaultInstance()); + Map memoryAfterAbort = txnMgr.copyOpenTransactions(); + Map abortedMemoryAfterAbort = txnMgr.copyAbortedTransactions(); + + // We should have the same number of aborts as before in the open txns response, since this + // one should have been immediately forgotten + Assert.assertEquals(before.getAbortedTransactionsCount(), afterAbort.getAbortedTransactionsCount()); + Assert.assertEquals(memoryBefore.size(), memoryAfterAbort.size()); + + // Nothing should have been added to the aborted list because we didn't have any write locks + Assert.assertEquals(abortedMemoryAfter.size(), abortedMemoryAfterAbort.size()); + Assert.assertNull(abortedMemoryAfterAbort.get(txnId)); + + // Check that the transaction was forgotten in HBase as well + hbaseTxn = hrw.getTransaction(txnId); + Assert.assertNull(hbaseTxn); + } + + @Test + public void openAndCommit() throws Exception { + + HbaseMetastoreProto.GetOpenTxnsResponse before = + txnMgr.getOpenTxns(HbaseMetastoreProto.Void.getDefaultInstance()); + Map memoryBefore = txnMgr.copyOpenTransactions(); + + // Open a single transaction + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(1, rsp.getTxnIdsCount()); + long txnId = rsp.getTxnIds(0); + + Set committedMemoryAfter = txnMgr.copyCommittedTransactions(); + + // Commit this transaction. It should promptly be forgotten as it has no locks + HbaseMetastoreProto.TransactionResult commit = + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + Map memoryAfterCommit = txnMgr.copyOpenTransactions(); + Set committedMemoryAfterCommit = txnMgr.copyCommittedTransactions(); + + // We should have the same number of aborts as before in the open txns response, since this + // one should have been immediately forgotten + Assert.assertEquals(memoryBefore.size(), memoryAfterCommit.size()); + + // Nothing should have been added to the committed list because we didn't have any write locks + Assert.assertEquals(committedMemoryAfter.size(), committedMemoryAfterCommit.size()); + for (CommittedHiveTransaction committed : committedMemoryAfterCommit) { + Assert.assertNotEquals(txnId, committed.getId()); + } + + // Check that the transaction was forgotten in HBase as well + HbaseMetastoreProto.Transaction hbaseTxn = hrw.getTransaction(txnId); + Assert.assertNull(hbaseTxn); + } + + @Test + public void openLockAbort() throws Exception { + + String db1 = "ola_db1"; + String db2 = "ola_db2"; + String t2 = "ola_t2"; + String db3 = "ola_db3"; + String t3 = "ola_t3"; + String p3 = "ola_p3"; + + HbaseMetastoreProto.GetOpenTxnsResponse before = + txnMgr.getOpenTxns(HbaseMetastoreProto.Void.getDefaultInstance()); + Map memoryBefore = txnMgr.copyOpenTransactions(); + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(2) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(2, rsp.getTxnIdsCount()); + long firstTxnId = rsp.getTxnIds(0); + long secondsTxnId = rsp.getTxnIds(1); + + HbaseMetastoreProto.GetOpenTxnsResponse after = + txnMgr.getOpenTxns(HbaseMetastoreProto.Void.getDefaultInstance()); + Map memoryAfter = txnMgr.copyOpenTransactions(); + Map abortedMemoryAfter = txnMgr.copyAbortedTransactions(); + + // Check that the highwater mark went up by two + Assert.assertEquals(before.getHighWaterMark() + 2, after.getHighWaterMark()); + + // Check that we have two more open transaction and the same number of aborted transactions + Assert.assertEquals(before.getOpenTransactionsCount() + 2, after.getOpenTransactionsCount()); + Assert.assertEquals(before.getAbortedTransactionsCount(), after.getAbortedTransactionsCount()); + + // Make sure two values got added to the in memory structure + Assert.assertEquals(memoryBefore.size() + 2, memoryAfter.size()); + + // Make both entries are in memory + for (long i = firstTxnId; i <= secondsTxnId; i++) { + OpenHiveTransaction newTxn = memoryAfter.get(i); + Assert.assertNotNull(newTxn); + + // Make sure our entry is in the open state + Assert.assertEquals(HbaseMetastoreProto.TxnState.OPEN, newTxn.getState()); + + // Make sure the heartbeat is set + Assert.assertTrue(newTxn.getLastHeartbeat() > 0); + + // Make sure it doesn't have any locks + Assert.assertNull(newTxn.getHiveLocks()); + + // Check the transaction in HBase + HbaseMetastoreProto.Transaction hbaseTxn = hrw.getTransaction(i); + Assert.assertNotNull(hbaseTxn); + Assert.assertEquals(i, hbaseTxn.getId()); + Assert.assertEquals(0, hbaseTxn.getLocksCount()); + Assert.assertEquals("me", hbaseTxn.getUser()); + Assert.assertEquals("localhost", hbaseTxn.getHostname()); + Assert.assertEquals(HbaseMetastoreProto.TxnState.OPEN, hbaseTxn.getTxnState()); + } + + // Get some locks. We'll do X, SW, and I on one and SR and I on the other + // I use unique entity names to guarantee I'm creating new queues in the lockQueues + Map locksBefore = + txnMgr.copyLockQueues(); + + HbaseMetastoreProto.LockResponse firstLock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxnId) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db1) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db2) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db2) + .setTable(t2) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, firstLock.getState()); + + HbaseMetastoreProto.LockResponse secondLock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondsTxnId) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db3) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db3) + .setTable(t3) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db3) + .setTable(t3) + .setPartition(p3) + .setType(HbaseMetastoreProto.LockType.SHARED_READ)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, secondLock.getState()); + + // Make sure the in memory transactions have the right locks. We don't have to get a new + // copy of the map because only the map is copied, not the contents. + HiveLock[] locks = memoryAfter.get(firstTxnId).getHiveLocks(); + Assert.assertEquals(3, locks.length); + locks = memoryAfter.get(secondsTxnId).getHiveLocks(); + Assert.assertEquals(3, locks.length); + + // We'll check the actual contents of the locks when checking the lock queues + + Map locksAfter = + txnMgr.copyLockQueues(); + Assert.assertEquals(locksBefore.size() + 6, locksAfter.size()); + + TransactionManager.EntityKey key = new TransactionManager.EntityKey(db1, null, null); + TransactionManager.LockQueue queue = locksAfter.get(key); + Assert.assertNotNull(queue); + Assert.assertEquals(1, queue.queue.size()); + Assert.assertEquals(0L, queue.getMaxCommitId()); + HiveLock lock = queue.queue.values().iterator().next(); + Assert.assertEquals(firstTxnId, lock.getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.EXCLUSIVE, lock.getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + Assert.assertEquals(key, lock.getEntityLocked()); + + key = new TransactionManager.EntityKey(db2, null, null); + queue = locksAfter.get(key); + Assert.assertNotNull(queue); + Assert.assertEquals(1, queue.queue.size()); + Assert.assertEquals(0L, queue.getMaxCommitId()); + lock = queue.queue.values().iterator().next(); + Assert.assertEquals(firstTxnId, lock.getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.INTENTION, lock.getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + Assert.assertEquals(key, lock.getEntityLocked()); + + key = new TransactionManager.EntityKey(db2, t2, null); + queue = locksAfter.get(key); + Assert.assertNotNull(queue); + Assert.assertEquals(1, queue.queue.size()); + Assert.assertEquals(0L, queue.getMaxCommitId()); + lock = queue.queue.values().iterator().next(); + Assert.assertEquals(firstTxnId, lock.getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.SHARED_WRITE, lock.getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + Assert.assertEquals(key, lock.getEntityLocked()); + + key = new TransactionManager.EntityKey(db3, null, null); + queue = locksAfter.get(key); + Assert.assertNotNull(queue); + Assert.assertEquals(1, queue.queue.size()); + Assert.assertEquals(0L, queue.getMaxCommitId()); + lock = queue.queue.values().iterator().next(); + Assert.assertEquals(secondsTxnId, lock.getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.INTENTION, lock.getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + Assert.assertEquals(key, lock.getEntityLocked()); + + key = new TransactionManager.EntityKey(db3, t3, null); + queue = locksAfter.get(key); + Assert.assertNotNull(queue); + Assert.assertEquals(1, queue.queue.size()); + Assert.assertEquals(0L, queue.getMaxCommitId()); + lock = queue.queue.values().iterator().next(); + Assert.assertEquals(secondsTxnId, lock.getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.INTENTION, lock.getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + Assert.assertEquals(key, lock.getEntityLocked()); + + key = new TransactionManager.EntityKey(db3, t3, p3); + queue = locksAfter.get(key); + Assert.assertNotNull(queue); + Assert.assertEquals(1, queue.queue.size()); + Assert.assertEquals(0L, queue.getMaxCommitId()); + lock = queue.queue.values().iterator().next(); + Assert.assertEquals(secondsTxnId, lock.getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.SHARED_READ, lock.getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + Assert.assertEquals(key, lock.getEntityLocked()); + + // Check the records in HBase + HbaseMetastoreProto.Transaction hbaseTxn = hrw.getTransaction(firstTxnId); + Assert.assertNotNull(hbaseTxn); + Assert.assertEquals(3, hbaseTxn.getLocksCount()); + Assert.assertEquals(HbaseMetastoreProto.TxnState.OPEN, hbaseTxn.getTxnState()); + + boolean sawOne, sawTwo, sawThree; + sawOne = sawTwo = sawThree = false; + for (HbaseMetastoreProto.Transaction.Lock hbaseLock : hbaseTxn.getLocksList()) { + if (hbaseLock.getDb().equals(db1)) { + sawOne = true; + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, hbaseLock.getState()); + Assert.assertEquals(HbaseMetastoreProto.LockType.EXCLUSIVE, hbaseLock.getType()); + Assert.assertFalse(hbaseLock.hasTable()); + Assert.assertFalse(hbaseLock.hasPartition()); + Assert.assertTrue(hbaseLock.getAcquiredAt() > 0); + } else if (hbaseLock.getDb().equals(db2) && !hbaseLock.hasTable()) { + sawTwo = true; + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, hbaseLock.getState()); + Assert.assertEquals(HbaseMetastoreProto.LockType.INTENTION, hbaseLock.getType()); + Assert.assertFalse(hbaseLock.hasTable()); + Assert.assertFalse(hbaseLock.hasPartition()); + Assert.assertTrue(hbaseLock.getAcquiredAt() > 0); + } else if (hbaseLock.getDb().equals(db2) && hbaseLock.hasTable()) { + sawThree = true; + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, hbaseLock.getState()); + Assert.assertEquals(HbaseMetastoreProto.LockType.SHARED_WRITE, hbaseLock.getType()); + Assert.assertEquals(t2, hbaseLock.getTable()); + Assert.assertFalse(hbaseLock.hasPartition()); + Assert.assertTrue(hbaseLock.getAcquiredAt() > 0); + } else { + Assert.fail(); + } + } + + Assert.assertTrue(sawOne); + Assert.assertTrue(sawTwo); + Assert.assertTrue(sawThree); + + hbaseTxn = hrw.getTransaction(secondsTxnId); + Assert.assertNotNull(hbaseTxn); + Assert.assertEquals(3, hbaseTxn.getLocksCount()); + Assert.assertEquals(HbaseMetastoreProto.TxnState.OPEN, hbaseTxn.getTxnState()); + + sawOne = sawTwo = sawThree = false; + for (HbaseMetastoreProto.Transaction.Lock hbaseLock : hbaseTxn.getLocksList()) { + if (hbaseLock.getDb().equals(db3) && !hbaseLock.hasTable() && !hbaseLock.hasPartition()) { + sawOne = true; + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, hbaseLock.getState()); + Assert.assertEquals(HbaseMetastoreProto.LockType.INTENTION, hbaseLock.getType()); + Assert.assertTrue(hbaseLock.getAcquiredAt() > 0); + } else if (hbaseLock.getDb().equals(db3) && hbaseLock.hasTable() && !hbaseLock.hasPartition()) { + sawTwo = true; + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, hbaseLock.getState()); + Assert.assertEquals(HbaseMetastoreProto.LockType.INTENTION, hbaseLock.getType()); + Assert.assertEquals(t3, hbaseLock.getTable()); + Assert.assertTrue(hbaseLock.getAcquiredAt() > 0); + } else if (hbaseLock.getDb().equals(db3) && hbaseLock.hasTable() && hbaseLock.hasPartition()) { + sawThree = true; + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, hbaseLock.getState()); + Assert.assertEquals(HbaseMetastoreProto.LockType.SHARED_READ, hbaseLock.getType()); + Assert.assertEquals(t3, hbaseLock.getTable()); + Assert.assertEquals(p3, hbaseLock.getPartition()); + Assert.assertTrue(hbaseLock.getAcquiredAt() > 0); + } else { + Assert.fail(); + } + } + + Assert.assertTrue(sawOne); + Assert.assertTrue(sawTwo); + Assert.assertTrue(sawThree); + + // abort both transactions + HbaseMetastoreProto.GetOpenTxnsResponse beforeAbort = + txnMgr.getOpenTxns(HbaseMetastoreProto.Void.getDefaultInstance()); + Map memoryBeforeAbort = txnMgr.copyOpenTransactions(); + Map abortedMemoryBeforeAbort = txnMgr.copyAbortedTransactions(); + + HbaseMetastoreProto.TransactionResult abort = + txnMgr.abortTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, abort.getState()); + abort = txnMgr.abortTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondsTxnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, abort.getState()); + + HbaseMetastoreProto.GetOpenTxnsResponse afterAbort = + txnMgr.getOpenTxns(HbaseMetastoreProto.Void.getDefaultInstance()); + Map memoryAfterAbort = txnMgr.copyOpenTransactions(); + Map abortedMemoryAfterAbort = txnMgr.copyAbortedTransactions(); + Map locksAfterAbort = + txnMgr.copyLockQueues(); + + // We should have remembered one of the aborts and not the otehr + // one should have been immediately forgotten + Assert.assertEquals(beforeAbort.getAbortedTransactionsCount() + 1, afterAbort.getAbortedTransactionsCount()); + Assert.assertEquals(memoryBeforeAbort.size() - 2, memoryAfterAbort.size()); + + // The first entry should have been added to the aborted list, but not the second + Assert.assertEquals(abortedMemoryBeforeAbort.size() + 1, abortedMemoryAfterAbort.size()); + Assert.assertNull(abortedMemoryAfterAbort.get(secondsTxnId)); + AbortedHiveTransaction firstAbortedTxn = abortedMemoryAfterAbort.get(firstTxnId); + Assert.assertNotNull(firstAbortedTxn); + Assert.assertEquals(1, firstAbortedTxn.getCompactableLocks().size()); + + key = new TransactionManager.EntityKey(db2, t2, null); + queue = locksAfterAbort.get(key); + Assert.assertEquals(0, queue.queue.size()); + + lock = firstAbortedTxn.getCompactableLocks().get(key); + Assert.assertNotNull(lock); + Assert.assertEquals(firstTxnId, lock.getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.SHARED_WRITE, lock.getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, lock.getState()); + Assert.assertEquals(key, lock.getEntityLocked()); + + // The second transaction should have been forgotten altogether + hbaseTxn = hrw.getTransaction(secondsTxnId); + Assert.assertNull(hbaseTxn); + + // The first transaction should still be there + hbaseTxn = hrw.getTransaction(firstTxnId); + Assert.assertNotNull(hbaseTxn); + Assert.assertEquals(HbaseMetastoreProto.TxnState.ABORTED, hbaseTxn.getTxnState()); + + Assert.assertEquals(1, hbaseTxn.getLocksCount()); + + HbaseMetastoreProto.Transaction.Lock hbaseLock = hbaseTxn.getLocks(0); + Assert.assertEquals(db2, hbaseLock.getDb()); + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, hbaseLock.getState()); + Assert.assertEquals(HbaseMetastoreProto.LockType.SHARED_WRITE, hbaseLock.getType()); + Assert.assertEquals(t2, hbaseLock.getTable()); + Assert.assertFalse(hbaseLock.hasPartition()); + + // We should have a potential compaction for the second one but not for any of the others + HbaseMetastoreProto.PotentialCompaction pc = + hrw.getPotentialCompaction(db2, t2, null); + Assert.assertNotNull(pc); + Assert.assertEquals(1, pc.getTxnIdsCount()); + Assert.assertEquals(firstTxnId, pc.getTxnIds(0)); + Assert.assertEquals(db2, pc.getDb()); + Assert.assertEquals(t2, pc.getTable()); + Assert.assertFalse(pc.hasPartition()); + + pc = hrw.getPotentialCompaction(db3, t3, p3); + Assert.assertNull(pc); + } + + @Test + public void openLockCommit() throws Exception { + + String db1 = "olc_db1"; + String db2 = "olc_db2"; + String t2 = "olc_t2"; + String db3 = "olc_db3"; + String t3 = "olc_t3"; + String p3 = "olc_p3"; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(2) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(2, rsp.getTxnIdsCount()); + long firstTxnId = rsp.getTxnIds(0); + long secondsTxnId = rsp.getTxnIds(1); + + HbaseMetastoreProto.GetOpenTxnsResponse after = + txnMgr.getOpenTxns(HbaseMetastoreProto.Void.getDefaultInstance()); + + // Get some locks. We'll do X, SW, and I on one and SR and I on the other + // I use unique entity names to guarantee I'm creating new queues in the lockQueues + Map locksBefore = + txnMgr.copyLockQueues(); + + HbaseMetastoreProto.LockResponse firstLock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxnId) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db1) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db2) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db2) + .setTable(t2) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, firstLock.getState()); + + HbaseMetastoreProto.LockResponse secondLock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondsTxnId) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db3) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db3) + .setTable(t3) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db3) + .setTable(t3) + .setPartition(p3) + .setType(HbaseMetastoreProto.LockType.SHARED_READ)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, secondLock.getState()); + + // commit both transactions + HbaseMetastoreProto.GetOpenTxnsResponse beforeCommit = + txnMgr.getOpenTxns(HbaseMetastoreProto.Void.getDefaultInstance()); + Map memoryBeforeCommit = txnMgr.copyOpenTransactions(); + Set committedMemoryBeforeCommit = txnMgr.copyCommittedTransactions(); + + HbaseMetastoreProto.TransactionResult commit = + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + commit = txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondsTxnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + HbaseMetastoreProto.GetOpenTxnsResponse afterCommit = + txnMgr.getOpenTxns(HbaseMetastoreProto.Void.getDefaultInstance()); + Map memoryAfterCommit = txnMgr.copyOpenTransactions(); + Set committedMemoryAfterCommit = txnMgr.copyCommittedTransactions(); + Map locksAfterCommit = + txnMgr.copyLockQueues(); + + Assert.assertEquals(memoryBeforeCommit.size() - 2, memoryAfterCommit.size()); + + // The first entry should have been added to the commit list, but not the second + Assert.assertEquals(committedMemoryBeforeCommit.size() + 1, committedMemoryAfterCommit.size()); + + CommittedHiveTransaction firstCommittedTxn = null; + for (CommittedHiveTransaction ct : committedMemoryAfterCommit) { + if (ct.getId() == firstTxnId) { + firstCommittedTxn = ct; + break; + } + } + Assert.assertNotNull(firstCommittedTxn); + Assert.assertEquals(after.getHighWaterMark(), firstCommittedTxn.getCommitId()); + + TransactionManager.EntityKey key = new TransactionManager.EntityKey(db2, t2, null); + TransactionManager.LockQueue queue = locksAfterCommit.get(key); + Assert.assertEquals(0, queue.queue.size()); + Assert.assertEquals(firstCommittedTxn.getCommitId(), queue.getMaxCommitId()); + + // The second transaction should have been forgotten altogether + HbaseMetastoreProto.Transaction hbaseTxn = hrw.getTransaction(secondsTxnId); + Assert.assertNull(hbaseTxn); + + // The first transaction should still be there + hbaseTxn = hrw.getTransaction(firstTxnId); + Assert.assertNotNull(hbaseTxn); + Assert.assertEquals(HbaseMetastoreProto.TxnState.COMMITTED, hbaseTxn.getTxnState()); + Assert.assertEquals(firstCommittedTxn.getCommitId(), hbaseTxn.getCommitId()); + + Assert.assertEquals(1, hbaseTxn.getLocksCount()); + + HbaseMetastoreProto.Transaction.Lock hbaseLock = hbaseTxn.getLocks(0); + Assert.assertEquals(db2, hbaseLock.getDb()); + Assert.assertEquals(HbaseMetastoreProto.LockState.RELEASED, hbaseLock.getState()); + Assert.assertEquals(HbaseMetastoreProto.LockType.SHARED_WRITE, hbaseLock.getType()); + Assert.assertEquals(t2, hbaseLock.getTable()); + Assert.assertFalse(hbaseLock.hasPartition()); + + // We should have a potential compaction for the second one but not for any of the others + HbaseMetastoreProto.PotentialCompaction pc = + hrw.getPotentialCompaction(db2, t2, null); + Assert.assertNotNull(pc); + Assert.assertEquals(1, pc.getTxnIdsCount()); + Assert.assertEquals(firstTxnId, pc.getTxnIds(0)); + Assert.assertEquals(db2, pc.getDb()); + Assert.assertEquals(t2, pc.getTable()); + Assert.assertFalse(pc.hasPartition()); + + pc = hrw.getPotentialCompaction(db3, t3, p3); + Assert.assertNull(pc); + } + + @Test + public void addLocks() throws Exception { + + String db1 = "al_db1"; + String db2 = "al_db2"; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(1, rsp.getTxnIdsCount()); + long txnId = rsp.getTxnIds(0); + + HbaseMetastoreProto.LockResponse lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(txnId) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db1) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(txnId) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db2) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + Map memoryAfterLocks = txnMgr.copyOpenTransactions(); + OpenHiveTransaction openTxn = memoryAfterLocks.get(txnId); + HiveLock[] locks = openTxn.getHiveLocks(); + Assert.assertEquals(2, locks.length); + Assert.assertEquals(txnId, locks[0].getTxnId()); + Assert.assertEquals(db1, locks[0].getEntityLocked().db); + Assert.assertEquals(txnId, locks[1].getTxnId()); + Assert.assertEquals(db2, locks[1].getEntityLocked().db); + + HbaseMetastoreProto.Transaction hbaseTxn = hrw.getTransaction(txnId); + Assert.assertEquals(2, hbaseTxn.getLocksCount()); + Assert.assertEquals(db1, hbaseTxn.getLocks(0).getDb()); + Assert.assertEquals(db2, hbaseTxn.getLocks(1).getDb()); + + // commit the transactions + HbaseMetastoreProto.TransactionResult commit = + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + } + + @Test + public void heartbeat() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Figure out what the last heartbeat was + Map openTxns = txnMgr.copyOpenTransactions(); + OpenHiveTransaction openTxn = openTxns.get(txnId); + long lastHeartbeat = openTxn.getLastHeartbeat(); + long now = System.currentTimeMillis(); + Assert.assertTrue(now >= lastHeartbeat); + + Thread.sleep(10); + + HbaseMetastoreProto.HeartbeatTxnRangeResponse heartbeats = + txnMgr.heartbeat(HbaseMetastoreProto.HeartbeatTxnRangeRequest.newBuilder() + .setMinTxn(txnId) + .setMaxTxn(txnId) + .build()); + Assert.assertEquals(0, heartbeats.getAbortedCount()); + Assert.assertEquals(0, heartbeats.getNoSuchCount()); + + // No need to re-fetch the map as we still have the pointer to the actual transaction + long latestHeartbeat = openTxn.getLastHeartbeat(); + Assert.assertTrue(latestHeartbeat > lastHeartbeat); + } + + @Test + public void heartbeatAbortedAndNoSuch() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Have to get a write lock so that the aborted transaction is remembered + txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(txnId) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t4") + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE) + .build()) + .build()); + txnMgr.abortTxn(HbaseMetastoreProto.TransactionId.newBuilder().setId(txnId).build()); + + HbaseMetastoreProto.HeartbeatTxnRangeResponse heartbeats = + txnMgr.heartbeat(HbaseMetastoreProto.HeartbeatTxnRangeRequest.newBuilder() + .setMinTxn(txnId) + .setMaxTxn(txnId + 1) + .build()); + Assert.assertEquals(1, heartbeats.getAbortedCount()); + Assert.assertEquals(1, heartbeats.getNoSuchCount()); + Assert.assertEquals(txnId, heartbeats.getAborted(0)); + Assert.assertEquals(txnId + 1, heartbeats.getNoSuch(0)); + } + + @Test + public void cleanupNoPotential() throws Exception { + // Test that everything's ok when we call cleanupAfterCompaction when there was no + // PotentialCompaction + String db = "cnp_db"; + String t = "cnp_t"; + String p = "cnp_p"; + + HbaseMetastoreProto.PotentialCompaction pc = hrw.getPotentialCompaction(db, t, p); + Assert.assertNull(pc); + + txnMgr.cleanupAfterCompaction(HbaseMetastoreProto.Compaction.newBuilder() + .setId(1) + .setDb(db) + .setTable(t) + .setPartition(p) + .setHighestTxnId(17) + .setState(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING) + .setType(HbaseMetastoreProto.CompactionType.MINOR) + .build()); + + pc = hrw.getPotentialCompaction(db, t, p); + Assert.assertNull(pc); + } + + @Test + public void cleanupCommit() throws Exception { + // Test that when the highestTxnId is lower than some of the transactions in the potential + // the potential is kept but earlier txn ids are trimmed out. + // Have to open and commit the transactions serially to avoid write/write conflict + String db = "cc_db"; + String t = "cc_t"; + String p = "cc_p"; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long firstTxn = rsp.getTxnIds(0); + + HbaseMetastoreProto.LockResponse lock = + txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setTable(t) + .setPartition(p) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setTable(t) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + HbaseMetastoreProto.TransactionResult result = + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + HbaseMetastoreProto.PotentialCompaction pc = hrw.getPotentialCompaction(db, t, p); + Assert.assertNotNull(pc); + Assert.assertEquals(db, pc.getDb()); + Assert.assertEquals(t, pc.getTable()); + Assert.assertEquals(p, pc.getPartition()); + Assert.assertEquals(1, pc.getTxnIdsCount()); + Assert.assertEquals(firstTxn, pc.getTxnIds(0)); + + rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + rsp = txnMgr.openTxns(rqst); + long secondTxn = rsp.getTxnIds(0); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setTable(t) + .setPartition(p) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setTable(t) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + result = txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + pc = hrw.getPotentialCompaction(db, t, p); + Assert.assertNotNull(pc); + Assert.assertEquals(db, pc.getDb()); + Assert.assertEquals(t, pc.getTable()); + Assert.assertEquals(p, pc.getPartition()); + Assert.assertEquals(2, pc.getTxnIdsCount()); + Assert.assertEquals(firstTxn, pc.getTxnIds(0)); + Assert.assertEquals(secondTxn, pc.getTxnIds(1)); + + txnMgr.cleanupAfterCompaction(HbaseMetastoreProto.Compaction.newBuilder() + .setId(100) + .setDb(db) + .setTable(t) + .setPartition(p) + .setHighestTxnId(firstTxn) + .setState(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING) + .setType(HbaseMetastoreProto.CompactionType.MINOR) + .build()); + + pc = hrw.getPotentialCompaction(db, t, p); + Assert.assertNotNull(pc); + Assert.assertEquals(db, pc.getDb()); + Assert.assertEquals(t, pc.getTable()); + Assert.assertEquals(p, pc.getPartition()); + Assert.assertEquals(1, pc.getTxnIdsCount()); + Assert.assertEquals(secondTxn, pc.getTxnIds(0)); + + txnMgr.cleanupAfterCompaction(HbaseMetastoreProto.Compaction.newBuilder() + .setId(101) + .setDb(db) + .setTable(t) + .setPartition(p) + .setHighestTxnId(secondTxn) + .setState(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING) + .setType(HbaseMetastoreProto.CompactionType.MINOR) + .build()); + + pc = hrw.getPotentialCompaction(db, t, p); + Assert.assertNull(pc); + } + + @Test + public void cleanupAbort() throws Exception { + // Test that when the highestTxnId is lower than some of the transactions in the potential + // the potential is kept but earlier txn ids are trimmed out. + // Have to open and commit the transactions serially to avoid write/write conflict + String db = "ca_db"; + String t = "ca_t"; + String p = "ca_p"; + + String db2 = "ca_db2"; + String t2 = "ca_t2"; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long firstTxn = rsp.getTxnIds(0); + + HbaseMetastoreProto.LockResponse lock = + txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setTable(t) + .setPartition(p) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setTable(t) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + HbaseMetastoreProto.TransactionResult result = + txnMgr.abortTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + HbaseMetastoreProto.PotentialCompaction pc = hrw.getPotentialCompaction(db, t, p); + Assert.assertNotNull(pc); + Assert.assertEquals(db, pc.getDb()); + Assert.assertEquals(t, pc.getTable()); + Assert.assertEquals(p, pc.getPartition()); + Assert.assertEquals(1, pc.getTxnIdsCount()); + Assert.assertEquals(firstTxn, pc.getTxnIds(0)); + + rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + rsp = txnMgr.openTxns(rqst); + long secondTxn = rsp.getTxnIds(0); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setTable(t) + .setPartition(p) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setTable(t) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db2) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db2) + .setTable(t2) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + result = txnMgr.abortTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + pc = hrw.getPotentialCompaction(db, t, p); + Assert.assertNotNull(pc); + Assert.assertEquals(db, pc.getDb()); + Assert.assertEquals(t, pc.getTable()); + Assert.assertEquals(p, pc.getPartition()); + Assert.assertEquals(2, pc.getTxnIdsCount()); + Assert.assertEquals(firstTxn, pc.getTxnIds(0)); + Assert.assertEquals(secondTxn, pc.getTxnIds(1)); + + // Check that our aborted txns have the locks properly listed + Map abortedTxns = txnMgr.copyAbortedTransactions(); + AbortedHiveTransaction firstAbortedTxn = abortedTxns.get(firstTxn); + Assert.assertFalse(firstAbortedTxn.fullyCompacted()); + Assert.assertEquals(1, firstAbortedTxn.getCompactableLocks().size()); + AbortedHiveTransaction secondAbortedTxn = abortedTxns.get(secondTxn); + Assert.assertFalse(secondAbortedTxn.fullyCompacted()); + Assert.assertEquals(2, secondAbortedTxn.getCompactableLocks().size()); + + HbaseMetastoreProto.Transaction hbaseTxn = hrw.getTransaction(firstTxn); + Assert.assertEquals(1, hbaseTxn.getLocksCount()); + hbaseTxn = hrw.getTransaction(secondTxn); + Assert.assertEquals(2, hbaseTxn.getLocksCount()); + + txnMgr.cleanupAfterCompaction(HbaseMetastoreProto.Compaction.newBuilder() + .setId(100) + .setDb(db) + .setTable(t) + .setPartition(p) + .setHighestTxnId(firstTxn) + .setState(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING) + .setType(HbaseMetastoreProto.CompactionType.MINOR) + .build()); + + pc = hrw.getPotentialCompaction(db, t, p); + Assert.assertNotNull(pc); + Assert.assertEquals(db, pc.getDb()); + Assert.assertEquals(t, pc.getTable()); + Assert.assertEquals(p, pc.getPartition()); + Assert.assertEquals(1, pc.getTxnIdsCount()); + Assert.assertEquals(secondTxn, pc.getTxnIds(0)); + + Assert.assertTrue(firstAbortedTxn.fullyCompacted()); + Assert.assertFalse(secondAbortedTxn.fullyCompacted()); + Assert.assertEquals(2, secondAbortedTxn.getCompactableLocks().size()); + + hbaseTxn = hrw.getTransaction(firstTxn); + Assert.assertNull(hbaseTxn); + hbaseTxn = hrw.getTransaction(secondTxn); + Assert.assertEquals(2, hbaseTxn.getLocksCount()); + + txnMgr.cleanupAfterCompaction(HbaseMetastoreProto.Compaction.newBuilder() + .setId(110) + .setDb(db) + .setTable(t) + .setPartition(p) + .setHighestTxnId(secondTxn) + .setState(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING) + .setType(HbaseMetastoreProto.CompactionType.MINOR) + .build()); + + pc = hrw.getPotentialCompaction(db, t, p); + Assert.assertNull(pc); + + Assert.assertFalse(secondAbortedTxn.fullyCompacted()); + Assert.assertEquals(1, secondAbortedTxn.getCompactableLocks().size()); + + abortedTxns = txnMgr.copyAbortedTransactions(); + Assert.assertNull(abortedTxns.get(firstTxn)); + Assert.assertNotNull(abortedTxns.get(secondTxn)); + + hbaseTxn = hrw.getTransaction(secondTxn); + Assert.assertEquals(1, hbaseTxn.getLocksCount()); + + txnMgr.cleanupAfterCompaction(HbaseMetastoreProto.Compaction.newBuilder() + .setId(120) + .setDb(db2) + .setTable(t2) + .setHighestTxnId(secondTxn) + .setState(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING) + .setType(HbaseMetastoreProto.CompactionType.MINOR) + .build()); + + pc = hrw.getPotentialCompaction(db2, t2, null); + Assert.assertNull(pc); + Assert.assertTrue(secondAbortedTxn.fullyCompacted()); + + abortedTxns = txnMgr.copyAbortedTransactions(); + Assert.assertNull(abortedTxns.get(secondTxn)); + + hbaseTxn = hrw.getTransaction(secondTxn); + Assert.assertNull(hbaseTxn); + } + + @Test + public void addDynamicPartitionsAbort() throws Exception { + String db = "adpa_db"; + String t = "adpa_t"; + String pbase = "adpa_p"; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(1, rsp.getTxnIdsCount()); + long txnId = rsp.getTxnIds(0); + + txnMgr.addDynamicPartitions(HbaseMetastoreProto.AddDynamicPartitionsRequest.newBuilder() + .setTxnId(txnId) + .setDb(db) + .setTable(t) + .addAllPartitions(Arrays.asList(pbase + "1", pbase + "2")) + .build()); + + Map memoryBefore = txnMgr.copyOpenTransactions(); + OpenHiveTransaction openTxn = memoryBefore.get(txnId); + Assert.assertEquals(2, openTxn.getHiveLocks().length); + for (HiveLock lock : openTxn.getHiveLocks()) { + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + } + + HbaseMetastoreProto.Transaction hbaseTxn = hrw.getTransaction(txnId); + Assert.assertEquals(HbaseMetastoreProto.TxnState.OPEN, hbaseTxn.getTxnState()); + Assert.assertEquals(2, hbaseTxn.getLocksCount()); + for (HbaseMetastoreProto.Transaction.Lock hbaseLock : hbaseTxn.getLocksList()) { + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, hbaseLock.getState()); + } + + HbaseMetastoreProto.TransactionResult abort = + txnMgr.abortTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, abort.getState()); + + Map memoryAfter = txnMgr.copyAbortedTransactions(); + AbortedHiveTransaction abortedTxn = memoryAfter.get(txnId); + Assert.assertEquals(2, abortedTxn.getCompactableLocks().size()); + for (HiveLock lock : abortedTxn.getCompactableLocks().values()) { + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, lock.getState()); + } + + for (int i = 1; i <= 2; i++) { + HbaseMetastoreProto.PotentialCompaction pc = hrw.getPotentialCompaction(db, t, pbase + i); + Assert.assertNotNull(pc); + } + } + + @Test + public void addDynamicPartitionsCommit() throws Exception { + String db = "adpc_db"; + String t = "adpc_t"; + String pbase = "adpc_p"; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(1, rsp.getTxnIdsCount()); + long txnId = rsp.getTxnIds(0); + + txnMgr.addDynamicPartitions(HbaseMetastoreProto.AddDynamicPartitionsRequest.newBuilder() + .setTxnId(txnId) + .setDb(db) + .setTable(t) + .addAllPartitions(Arrays.asList(pbase + "1", pbase + "2")) + .build()); + + Map memoryBefore = txnMgr.copyOpenTransactions(); + OpenHiveTransaction openTxn = memoryBefore.get(txnId); + Assert.assertEquals(2, openTxn.getHiveLocks().length); + for (HiveLock lock : openTxn.getHiveLocks()) { + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + } + + HbaseMetastoreProto.Transaction hbaseTxn = hrw.getTransaction(txnId); + Assert.assertEquals(HbaseMetastoreProto.TxnState.OPEN, hbaseTxn.getTxnState()); + Assert.assertEquals(2, hbaseTxn.getLocksCount()); + for (HbaseMetastoreProto.Transaction.Lock hbaseLock : hbaseTxn.getLocksList()) { + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, hbaseLock.getState()); + } + + HbaseMetastoreProto.TransactionResult commit = + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + Set memoryAfter = txnMgr.copyCommittedTransactions(); + CommittedHiveTransaction committedTxn = null; + for (CommittedHiveTransaction c : memoryAfter) { + if (c.getId() == txnId) { + committedTxn = c; + break; + } + } + Assert.assertNotNull(committedTxn); + Map queues = txnMgr.copyLockQueues(); + Assert.assertEquals(committedTxn.getCommitId(), + queues.get(new TransactionManager.EntityKey(db, t, pbase + "1")).getMaxCommitId()); + Assert.assertEquals(committedTxn.getCommitId(), + queues.get(new TransactionManager.EntityKey(db, t, pbase + "2")).getMaxCommitId()); + + for (int i = 1; i <= 2; i++) { + HbaseMetastoreProto.PotentialCompaction pc = hrw.getPotentialCompaction(db, t, pbase + i); + Assert.assertNotNull(pc); + } + } + + @Test + public void txnsCanBeCleaned() throws Exception { + // Open 3 transactions, commit the first, then the third and mark both as compacted. We + // should be told the first can be cleaned but not the second. + String db[] = {"tcbc_db1", "tcbc_db2", "tcbc_db3"}; + String t[] = {"tcbc_t1", "tcbc_t2", "tcbc_t3"}; + String p[] = {"tcbc_p1", "tcbc_p2", "tcbc_p3"}; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(3) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(3, rsp.getTxnIdsCount()); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + long thirdTxn = rsp.getTxnIds(2); + + for (int i = 0; i < 3; i++) { + HbaseMetastoreProto.LockResponse lock = txnMgr.lock(HbaseMetastoreProto.LockRequest + .newBuilder() + .setTxnId(i + firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[i]) + .setTable(t[i]) + .setPartition(p[i]) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[i]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[i]) + .setTable(t[i]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + } + + HbaseMetastoreProto.TransactionResult commit = + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + commit = txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(thirdTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + txnMgr.cleanupAfterCompaction(HbaseMetastoreProto.Compaction.newBuilder() + .setId(220) + .setDb(db[0]) + .setTable(t[0]) + .setPartition(p[0]) + .setHighestTxnId(firstTxn) + .setState(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING) + .setType(HbaseMetastoreProto.CompactionType.MINOR) + .build()); + + txnMgr.cleanupAfterCompaction(HbaseMetastoreProto.Compaction.newBuilder() + .setId(221) + .setDb(db[2]) + .setTable(t[2]) + .setPartition(p[2]) + .setHighestTxnId(thirdTxn) + .setState(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING) + .setType(HbaseMetastoreProto.CompactionType.MINOR) + .build()); + + HbaseMetastoreProto.CompactionList cleanable = txnMgr.verifyCompactionCanBeCleaned( + HbaseMetastoreProto.CompactionList.newBuilder() + .addCompactions( + HbaseMetastoreProto.Compaction.newBuilder() + .setDb(db[0]) + .setTable(t[0]) + .setPartition(p[0]) + .setHighestTxnId(firstTxn) + .setId(220) + .setType(HbaseMetastoreProto.CompactionType.MINOR) + .setState(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING)) + .addCompactions( + HbaseMetastoreProto.Compaction.newBuilder() + .setDb(db[2]) + .setTable(t[2]) + .setPartition(p[2]) + .setHighestTxnId(thirdTxn) + .setId(221) + .setType(HbaseMetastoreProto.CompactionType.MINOR) + .setState(HbaseMetastoreProto.CompactionState.READY_FOR_CLEANING)) + .build()); + Assert.assertEquals(1, cleanable.getCompactionsCount()); + Assert.assertEquals(db[0], cleanable.getCompactions(0).getDb()); + + } + + @Test + public void lockQueueShrinkerAndCommittedTxnCleaner() throws Exception { + // We test these two threads together because we have to clean the committed txns in order + // for the lockQueueShrinker to shrink the lock queues. + // First shrink the lock queues so we start from a (hopefully) clean state + txnMgr.forceCommittedTxnCleaner(); + txnMgr.forceLockQueueShrinker(); + + Map lockQueues = + txnMgr.copyLockQueues(); + Set committedTxns = txnMgr.copyCommittedTransactions(); + Assume.assumeTrue("Expected to find no lock queue entries, other tests should clean up after " + + "themselves", lockQueues.size() == 0); + Assume.assumeTrue("Expected to find no committed transactions, other tests should clean up " + + "after themselves", committedTxns.size() == 0); + + + // Open three transactions, acquire read locks with the first two and write locks with the + // third. Then commit the first and the third. Then clean committed txns and shrink the + // lock queues. We should still have a record of the committed write (because a previous + // transaction is still open) and lock queues entries from 2 (because it's open) and 3 + // (because we can't forget the committed txn yet). Then commit the second transaction and + // clean committed and shrink lock queues again. At that point we should have forgotten + // everything. + String db[] = {"lqsactc_db1", "lqsactc_db2", "lqsactc_db3"}; + String t[] = {"lqsactc_t1", "lqsactc_t2", "lqsactc_t3"}; + String p[] = {"lqsactc_p1", "lqsactc_p2", "lqsactc_p3"}; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(3) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(3, rsp.getTxnIdsCount()); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + long thirdTxn = rsp.getTxnIds(2); + + HbaseMetastoreProto.LockResponse lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[0]) + .setTable(t[0]) + .setPartition(p[0]) + .setType(HbaseMetastoreProto.LockType.SHARED_READ)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[0]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[0]) + .setTable(t[0]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + HbaseMetastoreProto.TransactionResult commit = + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[1]) + .setTable(t[1]) + .setPartition(p[1]) + .setType(HbaseMetastoreProto.LockType.SHARED_READ)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[1]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[1]) + .setTable(t[1]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(thirdTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[2]) + .setTable(t[2]) + .setPartition(p[2]) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[2]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[2]) + .setTable(t[2]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + commit = txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(thirdTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + // Open one more transaction to force the counter forward + rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + rsp = txnMgr.openTxns(rqst); + long forthTxn = rsp.getTxnIds(0); + try { + + lockQueues = txnMgr.copyLockQueues(); + Assert.assertEquals(9, lockQueues.size()); + + txnMgr.forceCommittedTxnCleaner(); + txnMgr.forceLockQueueShrinker(); + + lockQueues = txnMgr.copyLockQueues(); + Assert.assertEquals(4, lockQueues.size()); + + Assert.assertNotNull(lockQueues.get(new TransactionManager.EntityKey(db[1], t[1], p[1]))); + Assert.assertNotNull(lockQueues.get(new TransactionManager.EntityKey(db[2], t[2], p[2]))); + + committedTxns = txnMgr.copyCommittedTransactions(); + Assert.assertEquals(1, committedTxns.size()); + Assert.assertEquals(thirdTxn, committedTxns.iterator().next().getId()); + + commit = txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + txnMgr.forceCommittedTxnCleaner(); + txnMgr.forceLockQueueShrinker(); + + committedTxns = txnMgr.copyCommittedTransactions(); + Assert.assertEquals(0, committedTxns.size()); + lockQueues = txnMgr.copyLockQueues(); + Assert.assertEquals(0, lockQueues.size()); + } finally { + commit = txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(forthTxn) + .build()); + } + } + + // TODO Test multiple threads all reading and writing + +} diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManagerErrors.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManagerErrors.java new file mode 100644 index 0000000..f416d5b --- /dev/null +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManagerErrors.java @@ -0,0 +1,621 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.hbase.HBaseStore; +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; +import org.apache.hadoop.hive.metastore.hbase.MockUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Tests that test error conditions in the transaction manager. The full tests and deadlock + * tests are in here too since those are sort of an error condition. + */ +public class TestTransactionManagerErrors extends MockUtils { + HBaseStore store; + TransactionManager txnMgr; + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Before + public void init() throws IOException { + MockitoAnnotations.initMocks(this); + HiveConf conf = new HiveConf(); + + // Set the wait on the background threads to max long so that they don't run and clean things + // up on us, since we're trying to check state. + conf.set(TransactionManager.CONF_NO_AUTO_BACKGROUND_THREADS, Boolean.toString(Boolean.TRUE)); + // Set this value lower so we can fill up the txn mgr without creating an insane number of + // objects. + HiveConf.setIntVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_MAX_OBJECTS, 100); + // Set poll timeout low so we don't wait forever for our locks to come back and tell us to wait. + HiveConf.setTimeVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_LOCK_POLL_TIMEOUT, 500, + TimeUnit.MILLISECONDS); + // Set timeout low for timeout testing. This should not affect other tests because in + // general the background threads aren't running. + HiveConf.setTimeVar(conf, HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 100, TimeUnit.MILLISECONDS); + store = mockInit(conf); + txnMgr = txnCoProc.backdoor(); + } + + @After + public void cleanup() throws IOException { + txnMgr.shutdown(); + } + + @Test + public void rejectTxnsWhenFull() throws Exception { + // Test that we properly reject new transactions when we're full. Note that this test does + // not cover all conditions that can create fullness, just that once we are full the right + // thing happens. + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(100) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(100, rsp.getTxnIdsCount()); + + long txnBase = rsp.getTxnIds(0); + + // One more should still work because the full checker hasn't run yet. + assertNotFull(); + + txnMgr.forceFullChecker(); + assertFull(); + + // Close just a couple to make sure we have to go below 90% before it opens back up. + closeTxns(txnBase, txnBase + 5); + txnMgr.forceFullChecker(); + assertFull(); + + // Close enough to get us under the limit + closeTxns(txnBase + 5, txnBase + 12); + txnMgr.forceFullChecker(); + assertNotFull(); + + closeTxns(txnBase + 12, txnBase + 101); + txnMgr.forceFullChecker(); + assertNotFull(); + } + + private void assertFull() throws SeverusPleaseException { + boolean sawException = false; + try { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + txnMgr.openTxns(rqst); + } catch (IOException e) { + sawException = true; + Assert.assertEquals("Full, no new transactions being accepted", e.getMessage()); + } + Assert.assertTrue(sawException); + } + + private void assertNotFull() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(1, rsp.getTxnIdsCount()); + } + + private void closeTxns(long min, long max) throws IOException, SeverusPleaseException { + for (long i = min; i < max; i++) { + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(i) + .build()); + } + } + + @Test + public void abortNoSuchTxn() throws Exception { + HbaseMetastoreProto.TransactionResult result = txnMgr.abortTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(10000) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN, result.getState()); + } + + @Test + public void abortAbortedTxn() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Abort it once, this should work + HbaseMetastoreProto.TransactionResult result = txnMgr.abortTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + // Abort it again, this should bork + result = txnMgr.abortTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN, result.getState()); + } + + @Test + public void abortCommittedTxn() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Commit it, this should work + HbaseMetastoreProto.TransactionResult result = txnMgr.commitTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + // Abort it, this should bork + result = txnMgr.abortTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN, result.getState()); + } + + @Test + public void commitNoSuchTxn() throws Exception { + HbaseMetastoreProto.TransactionResult result = txnMgr.commitTxn( + HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(10000) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN, result.getState()); + } + + @Test + public void commitAbortedTxn() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Abort it, this should work + HbaseMetastoreProto.TransactionResult result = txnMgr.abortTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + // Commit it, this should bork + result = txnMgr.commitTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN, result.getState()); + } + + @Test + public void commitCommittedTxn() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Commit it, this should work + HbaseMetastoreProto.TransactionResult result = txnMgr.commitTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + // Commit it again, this should bork + result = txnMgr.commitTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN, result.getState()); + } + + @Test + public void lockNoSuchTxn() throws Exception { + HbaseMetastoreProto.LockResponse result = txnMgr.lock(HbaseMetastoreProto.LockRequest + .newBuilder() + .setTxnId(10000) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("x") + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, result.getState()); + } + + @Test + public void lockAbortedTxn() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Abort it, this should work + HbaseMetastoreProto.TransactionResult result = txnMgr.abortTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + // lock it, this should bork + HbaseMetastoreProto.LockResponse lock = txnMgr.lock(HbaseMetastoreProto.LockRequest + .newBuilder() + .setTxnId(txnId) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("x") + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, lock.getState()); + } + + @Test + public void lockCommittedTxn() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Commit it, this should work + HbaseMetastoreProto.TransactionResult result = txnMgr.commitTxn( + HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + // lock it, this should bork + HbaseMetastoreProto.LockResponse lock = txnMgr.lock(HbaseMetastoreProto.LockRequest + .newBuilder() + .setTxnId(txnId) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("x") + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, lock.getState()); + } + + @Test + public void checkLocksNoSuchTxn() throws Exception { + HbaseMetastoreProto.LockResponse result = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(10000) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, result.getState()); + } + + @Test + public void checkLocksAbortedTxn() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Abort it, this should work + HbaseMetastoreProto.TransactionResult result = txnMgr.abortTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + // checkLocks it, this should bork + HbaseMetastoreProto.LockResponse lock = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, lock.getState()); + } + + @Test + public void checkLocksCommittedTxn() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Commit it, this should work + HbaseMetastoreProto.TransactionResult result = txnMgr.commitTxn( + HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + // lock it, this should bork + HbaseMetastoreProto.LockResponse lock = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, lock.getState()); + } + + @Test + public void dynamicPartitionsNoSuchTxn() throws Exception { + HbaseMetastoreProto.TransactionResult result = + txnMgr.addDynamicPartitions(HbaseMetastoreProto.AddDynamicPartitionsRequest + .newBuilder() + .setTxnId(10000) + .setDb("a") + .setTable("b") + .addAllPartitions(Arrays.asList("p1", "p2")) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN, result.getState()); + } + + @Test + public void dynamicPartitionsAbortedTxn() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Abort it, this should work + HbaseMetastoreProto.TransactionResult result = txnMgr.abortTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + // add dynamic partitions, this should bork + result = txnMgr.addDynamicPartitions(HbaseMetastoreProto.AddDynamicPartitionsRequest + .newBuilder() + .setTxnId(txnId) + .setDb("a") + .setTable("b") + .addAllPartitions(Arrays.asList("p1", "p2")) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN, result.getState()); + } + + @Test + public void addDynamicPartitionsCommittedTxn() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(1) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long txnId = rsp.getTxnIds(0); + + // Commit it, this should work + HbaseMetastoreProto.TransactionResult result = txnMgr.commitTxn( + HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(txnId) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, result.getState()); + + // add dynamic partitions, this should bork + result = txnMgr.addDynamicPartitions(HbaseMetastoreProto.AddDynamicPartitionsRequest + .newBuilder() + .setTxnId(txnId) + .setDb("a") + .setTable("b") + .addAllPartitions(Arrays.asList("p1", "p2")) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.NO_SUCH_TXN, result.getState()); + } + + @Test + public void runDeadlockDetectorNoDeadlocks() throws Exception { + String db = "rddnd_db"; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(2) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + + HbaseMetastoreProto.LockResponse lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + // This should not change anything. + txnMgr.forceDeadlockDetection(); + + HbaseMetastoreProto.LockResponse firstLock = + txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + HbaseMetastoreProto.LockResponse secondLock = + txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxn) + .build()); + + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, firstLock.getState()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, secondLock.getState()); + } + + @Test + public void simpleDeadlock() throws Exception { + String db1 = "sdl_db1"; + String db2 = "sdl_db2"; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(2) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + + HbaseMetastoreProto.LockResponse lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db1) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db1) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db2) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db2) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + lock = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + lock = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + txnMgr.forceDeadlockDetection(); + + HbaseMetastoreProto.LockResponse firstLock = + txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + HbaseMetastoreProto.LockResponse secondLock = + txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxn) + .build()); + if (firstLock.getState() == HbaseMetastoreProto.LockState.ACQUIRED) { + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, secondLock.getState()); + } else if (firstLock.getState() == HbaseMetastoreProto.LockState.TXN_ABORTED) { + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, secondLock.getState()); + } else { + Assert.fail("Completely unexpected state " + firstLock.getState()); + } + + // One of them should be dead, and the other acquired, but which is which doesn't matter. + } + + @Test + public void timeout() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(2) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + + Thread.sleep(100); + HbaseMetastoreProto.HeartbeatTxnRangeResponse heartbeats = + txnMgr.heartbeat(HbaseMetastoreProto.HeartbeatTxnRangeRequest.newBuilder() + .setMinTxn(firstTxn) + .setMaxTxn(secondTxn) + .build()); + txnMgr.forceTimedOutCleaner(); + + Map openTxns = txnMgr.copyOpenTransactions(); + Assert.assertNotNull(openTxns.get(firstTxn)); + Assert.assertNotNull(openTxns.get(secondTxn)); + + Thread.sleep(100); + heartbeats = txnMgr.heartbeat(HbaseMetastoreProto.HeartbeatTxnRangeRequest.newBuilder() + .setMinTxn(secondTxn) + .setMaxTxn(secondTxn) + .build()); + txnMgr.forceTimedOutCleaner(); + + openTxns = txnMgr.copyOpenTransactions(); + Assert.assertNull(openTxns.get(firstTxn)); + Assert.assertNotNull(openTxns.get(secondTxn)); + + Thread.sleep(100); + txnMgr.forceTimedOutCleaner(); + openTxns = txnMgr.copyOpenTransactions(); + Assert.assertNull(openTxns.get(secondTxn)); + } + + // TODO test deadlock with another txn in the middle + + // TODO test all conditions that can lead to fullness +} diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManagerLocks.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManagerLocks.java new file mode 100644 index 0000000..ca49fb8 --- /dev/null +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManagerLocks.java @@ -0,0 +1,564 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.hbase.HBaseStore; +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; +import org.apache.hadoop.hive.metastore.hbase.MockUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +/** + * Many of these tests involve timing, so they're kept in a separate test class so they can be + * excluded from the Jenkins runs. These should test all of the lock combinations. + */ +public class TestTransactionManagerLocks extends MockUtils { + HBaseStore store; + TransactionManager txnMgr; + + @Before + public void init() throws IOException { + MockitoAnnotations.initMocks(this); + HiveConf conf = new HiveConf(); + + // Set the wait on the background threads to max long so that they don't run and clean things + // up on us, since we're trying to check state. + conf.set(TransactionManager.CONF_NO_AUTO_BACKGROUND_THREADS, Boolean.toString(Boolean.TRUE)); + + // Set this super low so the test doesn't take forever + HiveConf.setTimeVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_LOCK_POLL_TIMEOUT, + 100, TimeUnit.MILLISECONDS); + + store = mockInit(conf); + txnMgr = txnCoProc.backdoor(); + } + + @After + public void cleanup() throws IOException { + txnMgr.shutdown(); + } + + @Test + public void exclusiveExclusive() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.EXCLUSIVE, HbaseMetastoreProto.LockType.EXCLUSIVE, + HbaseMetastoreProto.LockState.WAITING); + } + + @Test + public void exclusiveSharedWrite() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.EXCLUSIVE, HbaseMetastoreProto.LockType.SHARED_WRITE, + HbaseMetastoreProto.LockState.WAITING); + } + + @Test + public void exclusiveSharedRead() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.EXCLUSIVE, HbaseMetastoreProto.LockType.SHARED_READ, + HbaseMetastoreProto.LockState.WAITING); + } + + @Test + public void exclusiveIntention() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.EXCLUSIVE, HbaseMetastoreProto.LockType.INTENTION, + HbaseMetastoreProto.LockState.WAITING); + } + + @Test + public void sharedWriteExclusive() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.SHARED_WRITE, HbaseMetastoreProto.LockType.EXCLUSIVE, + HbaseMetastoreProto.LockState.WAITING); + } + + @Test + public void sharedWriteSharedWrite() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.SHARED_WRITE, HbaseMetastoreProto.LockType.SHARED_WRITE, + HbaseMetastoreProto.LockState.WAITING); + } + + @Test + public void sharedWriteSharedRead() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.SHARED_WRITE, HbaseMetastoreProto.LockType.SHARED_READ, + HbaseMetastoreProto.LockState.ACQUIRED); + } + + @Test + public void sharedWriteIntention() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.SHARED_WRITE, HbaseMetastoreProto.LockType.INTENTION, + HbaseMetastoreProto.LockState.ACQUIRED); + } + + @Test + public void sharedReadExclusive() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.SHARED_READ, HbaseMetastoreProto.LockType.EXCLUSIVE, + HbaseMetastoreProto.LockState.WAITING); + } + + @Test + public void sharedReadSharedWrite() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.SHARED_READ, HbaseMetastoreProto.LockType.SHARED_WRITE, + HbaseMetastoreProto.LockState.ACQUIRED); + } + + @Test + public void sharedReadSharedRead() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.SHARED_READ, HbaseMetastoreProto.LockType.SHARED_READ, + HbaseMetastoreProto.LockState.ACQUIRED); + } + + @Test + public void sharedReadIntention() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.SHARED_READ, HbaseMetastoreProto.LockType.INTENTION, + HbaseMetastoreProto.LockState.ACQUIRED); + } + + @Test + public void intentionExclusive() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.INTENTION, HbaseMetastoreProto.LockType.EXCLUSIVE, + HbaseMetastoreProto.LockState.WAITING); + } + + @Test + public void intentionSharedWrite() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.INTENTION, HbaseMetastoreProto.LockType.SHARED_WRITE, + HbaseMetastoreProto.LockState.ACQUIRED); + } + + @Test + public void intentionSharedRead() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.INTENTION, HbaseMetastoreProto.LockType.SHARED_READ, + HbaseMetastoreProto.LockState.ACQUIRED); + } + + @Test + public void intentionIntention() throws Exception { + lockCombo(HbaseMetastoreProto.LockType.INTENTION, HbaseMetastoreProto.LockType.INTENTION, + HbaseMetastoreProto.LockState.ACQUIRED); + } + + private void lockCombo(HbaseMetastoreProto.LockType first, HbaseMetastoreProto.LockType second, + HbaseMetastoreProto.LockState expectedState) throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(2) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + try { + + HbaseMetastoreProto.LockResponse lock = + txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(first) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(second) + .build()) + .build()); + Assert.assertEquals(expectedState, lock.getState()); + } finally { + for (long i = firstTxn; i <= secondTxn; i++) { + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(i) + .build()); + } + } + } + + @Test + public void acquireAfterAbort() throws Exception { + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(2) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + try { + + HbaseMetastoreProto.LockResponse lock = + txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + HbaseMetastoreProto.TransactionResult abort = + txnMgr.abortTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, abort.getState()); + + // Give it a bit to run and check the locks + Thread.sleep(100); + + lock = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + } finally { + for (long i = firstTxn; i <= secondTxn; i++) { + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(i) + .build()); + } + } + } + + @Test + public void lostUpdateDetection() throws Exception { + // Detect that if a partition has potentially changed underneath a writer their transaction + // is aborted. + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(2) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + try { + + HbaseMetastoreProto.LockResponse lock = + txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + HbaseMetastoreProto.TransactionResult commit = + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + // Give it a bit to run and check the locks + Thread.sleep(100); + + lock = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, lock.getState()); + } finally { + for (long i = firstTxn; i <= secondTxn; i++) { + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(i) + .build()); + } + } + } + + @Test + public void acquireAfterCommit() throws Exception { + // Detect that if a partition has potentially changed underneath a writer their transaction + // is aborted. + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(2) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + try { + + HbaseMetastoreProto.LockResponse lock = + txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + HbaseMetastoreProto.TransactionResult commit = + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + // Give it a bit to run and check the locks + Thread.sleep(100); + + lock = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + } finally { + for (long i = firstTxn; i <= secondTxn; i++) { + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(i) + .build()); + } + } + } + + @Test + public void acquireSharedAfterAbort() throws Exception { + // Detect that if a partition has potentially changed underneath a writer their transaction + // is aborted. + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(3) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + long thirdTxn = rsp.getTxnIds(2); + try { + + HbaseMetastoreProto.LockResponse lock = + txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.SHARED_READ) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(thirdTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.SHARED_READ) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + HbaseMetastoreProto.TransactionResult abort = + txnMgr.abortTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, abort.getState()); + + // Give it a bit to run and check the locks + Thread.sleep(100); + + lock = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + lock = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(thirdTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + } finally { + for (long i = firstTxn; i <= thirdTxn; i++) { + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(i) + .build()); + } + } + } + + @Test + public void acquireSharedAfterCommit() throws Exception { + // Detect that if a partition has potentially changed underneath a writer their transaction + // is aborted. + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(3) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + long thirdTxn = rsp.getTxnIds(2); + try { + + HbaseMetastoreProto.LockResponse lock = + txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.SHARED_READ) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(thirdTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setTable("t") + .setType(HbaseMetastoreProto.LockType.SHARED_READ) + .build()) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb("d") + .setType(HbaseMetastoreProto.LockType.INTENTION) + .build()) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + HbaseMetastoreProto.TransactionResult commit = + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + // Give it a bit to run and check the locks + Thread.sleep(100); + + lock = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(secondTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + lock = txnMgr.checkLocks(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(thirdTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + } finally { + for (long i = firstTxn; i <= thirdTxn; i++) { + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId + .newBuilder() + .setId(i) + .build()); + } + } + } +} diff --git metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManagerRecovery.java metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManagerRecovery.java new file mode 100644 index 0000000..77148b3 --- /dev/null +++ metastore/src/test/org/apache/hadoop/hive/metastore/hbase/txn/txnmgr/TestTransactionManagerRecovery.java @@ -0,0 +1,276 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.hbase.txn.txnmgr; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.hbase.HBaseReadWrite; +import org.apache.hadoop.hive.metastore.hbase.HBaseStore; +import org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto; +import org.apache.hadoop.hive.metastore.hbase.MockUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +/** + * Test recovery of the TransactionManager from HBase. + */ +public class TestTransactionManagerRecovery extends MockUtils { + HBaseStore store; + TransactionManager txnMgr; + HBaseReadWrite hrw; + HiveConf conf; + + @Before + public void init() throws IOException { + MockitoAnnotations.initMocks(this); + conf = new HiveConf(); + + // Set the wait on the background threads to max long so that they don't run and clean things + // up on us, since we're trying to check state. + conf.set(TransactionManager.CONF_NO_AUTO_BACKGROUND_THREADS, Boolean.toString(Boolean.TRUE)); + // Set poll timeout low so we don't wait forever for our locks to come back and tell us to wait. + HiveConf.setTimeVar(conf, HiveConf.ConfVars.METASTORE_HBASE_TXN_MGR_LOCK_POLL_TIMEOUT, 500, + TimeUnit.MILLISECONDS); + + store = mockInit(conf); + txnMgr = txnCoProc.backdoor(); + hrw = HBaseReadWrite.getInstance(); + } + + @After + public void cleanup() throws IOException { + txnMgr.shutdown(); + } + + @Test + public void recover() throws Exception { + String db[] = {"lqsactc_db1", "lqsactc_db2", "lqsactc_db3"}; + String t[] = {"lqsactc_t1", "lqsactc_t2", "lqsactc_t3"}; + String p[] = {"lqsactc_p1", "lqsactc_p2", "lqsactc_p3"}; + + HbaseMetastoreProto.OpenTxnsRequest rqst = HbaseMetastoreProto.OpenTxnsRequest.newBuilder() + .setNumTxns(5) + .setUser("me") + .setHostname("localhost") + .build(); + HbaseMetastoreProto.OpenTxnsResponse rsp = txnMgr.openTxns(rqst); + Assert.assertEquals(5, rsp.getTxnIdsCount()); + long firstTxn = rsp.getTxnIds(0); + long secondTxn = rsp.getTxnIds(1); + long thirdTxn = rsp.getTxnIds(2); + long fourthTxn = rsp.getTxnIds(3); + long fifthTxn = rsp.getTxnIds(4); + + HbaseMetastoreProto.LockResponse lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(firstTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[0]) + .setTable(t[0]) + .setPartition(p[0]) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[0]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[0]) + .setTable(t[0]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + HbaseMetastoreProto.TransactionResult abort = + txnMgr.abortTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(firstTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, abort.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(secondTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[1]) + .setTable(t[1]) + .setPartition(p[1]) + .setType(HbaseMetastoreProto.LockType.SHARED_READ)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[1]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[1]) + .setTable(t[1]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(thirdTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[2]) + .setTable(t[2]) + .setPartition(p[2]) + .setType(HbaseMetastoreProto.LockType.SHARED_WRITE)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[2]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[2]) + .setTable(t[2]) + .setType(HbaseMetastoreProto.LockType.INTENTION)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, lock.getState()); + + HbaseMetastoreProto.TransactionResult commit = + txnMgr.commitTxn(HbaseMetastoreProto.TransactionId.newBuilder() + .setId(thirdTxn) + .build()); + Assert.assertEquals(HbaseMetastoreProto.TxnStateChangeResult.SUCCESS, commit.getState()); + + lock = txnMgr.lock(HbaseMetastoreProto.LockRequest.newBuilder() + .setTxnId(fourthTxn) + .addComponents(HbaseMetastoreProto.LockComponent.newBuilder() + .setDb(db[1]) + .setType(HbaseMetastoreProto.LockType.EXCLUSIVE)) + .build()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, lock.getState()); + + // Now, shut it down and start a new one to see if recovery works properly + txnMgr.shutdown(); + txnMgr = new TransactionManager(conf); + + // We should have one aborted transaction with a single lock + Map abortedTxns = txnMgr.copyAbortedTransactions(); + Assert.assertEquals(1, abortedTxns.size()); + AbortedHiveTransaction aborted = abortedTxns.get(firstTxn); + Assert.assertNotNull(aborted); + Assert.assertFalse(aborted.fullyCompacted()); + Assert.assertEquals(1, aborted.getCompactableLocks().size()); + HiveLock abortedLock = aborted.getCompactableLocks().values().iterator().next(); + Assert.assertEquals(new TransactionManager.EntityKey(db[0], t[0], p[0]), + abortedLock.getEntityLocked()); + Assert.assertEquals(firstTxn, abortedLock.getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.SHARED_WRITE, abortedLock.getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.TXN_ABORTED, abortedLock.getState()); + + // We should have 3 open transactions. The second overall transaction should have 3 locks, + // and the fourth 1 + Map openTxns = txnMgr.copyOpenTransactions(); + Assert.assertEquals(3, openTxns.size()); + OpenHiveTransaction open = openTxns.get(secondTxn); + Assert.assertNotNull(open); + HiveLock[] locks = open.getHiveLocks(); + Assert.assertEquals(3, locks.length); + boolean sawDbLock = false, sawTableLock = false, sawPartLock = false; + for (HiveLock openLock : locks) { + if (openLock.getEntityLocked().part == null) { + if (openLock.getEntityLocked().table == null) { + Assert.assertEquals(new TransactionManager.EntityKey(db[1], null, null), + openLock.getEntityLocked()); + Assert.assertEquals(secondTxn, openLock.getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.INTENTION, openLock.getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, openLock.getState()); + sawDbLock = true; + } else { + Assert.assertEquals(new TransactionManager.EntityKey(db[1], t[1], null), + openLock.getEntityLocked()); + Assert.assertEquals(secondTxn, openLock.getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.INTENTION, openLock.getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, openLock.getState()); + sawTableLock = true; + } + } else { + Assert.assertEquals(new TransactionManager.EntityKey(db[1], t[1], p[1]), + openLock.getEntityLocked()); + Assert.assertEquals(secondTxn, openLock.getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.SHARED_READ, openLock.getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.ACQUIRED, openLock.getState()); + sawPartLock = true; + } + } + Assert.assertTrue(sawDbLock); + Assert.assertTrue(sawTableLock); + Assert.assertTrue(sawPartLock); + + open = openTxns.get(fourthTxn); + Assert.assertNotNull(open); + locks = open.getHiveLocks(); + Assert.assertEquals(1, locks.length); + Assert.assertEquals(new TransactionManager.EntityKey(db[1], null, null), + locks[0].getEntityLocked()); + Assert.assertEquals(fourthTxn, locks[0].getTxnId()); + Assert.assertEquals(HbaseMetastoreProto.LockType.EXCLUSIVE, locks[0].getType()); + Assert.assertEquals(HbaseMetastoreProto.LockState.WAITING, locks[0].getState()); + + // We should have 1 committed txn + Set committedTxns = txnMgr.copyCommittedTransactions(); + Assert.assertEquals(1, committedTxns.size()); + CommittedHiveTransaction committedTxn = committedTxns.iterator().next(); + Assert.assertEquals(thirdTxn, committedTxn.getId()); + Assert.assertEquals(fifthTxn + 1, committedTxn.getCommitId()); + + // We should have 5 lock queues (3 for the open txn locks, one for the committed txn lock, + // and one for the aborted txn. + Map lockQueues = + txnMgr.copyLockQueues(); + Assert.assertEquals(5, lockQueues.size()); + + // The aborted queue will be empty + TransactionManager.LockQueue queue = + lockQueues.get(new TransactionManager.EntityKey(db[0], t[0], p[0])); + Assert.assertNotNull(queue); + Assert.assertEquals(0, queue.getMaxCommitId()); + Assert.assertEquals(0, queue.queue.size()); + + queue = lockQueues.get(new TransactionManager.EntityKey(db[1], t[1], p[1])); + Assert.assertNotNull(queue); + Assert.assertEquals(0, queue.getMaxCommitId()); + Assert.assertEquals(1, queue.queue.size()); + // We've already checked the contents of the lock above, just check that this is the right lock + Assert.assertEquals(secondTxn, queue.queue.values().iterator().next().getTxnId()); + + queue = lockQueues.get(new TransactionManager.EntityKey(db[1], t[1], null)); + Assert.assertNotNull(queue); + Assert.assertEquals(0, queue.getMaxCommitId()); + Assert.assertEquals(1, queue.queue.size()); + // We've already checked the contents of the lock above, just check that this is the right lock + Assert.assertEquals(secondTxn, queue.queue.values().iterator().next().getTxnId()); + + queue = lockQueues.get(new TransactionManager.EntityKey(db[1], null, null)); + Assert.assertNotNull(queue); + Assert.assertEquals(0, queue.getMaxCommitId()); + Assert.assertEquals(2, queue.queue.size()); + // We've already checked the contents of the lock above, just check that this is the right lock + // Order matters on this one + Iterator iter = queue.queue.values().iterator(); + Assert.assertEquals(secondTxn, iter.next().getTxnId()); + Assert.assertEquals(fourthTxn, iter.next().getTxnId()); + + // The queue will be empty for the committed one, but the commit id should be set. + queue = lockQueues.get(new TransactionManager.EntityKey(db[2], t[2], p[2])); + Assert.assertNotNull(queue); + Assert.assertEquals(fifthTxn + 1, queue.getMaxCommitId()); + Assert.assertEquals(0, queue.queue.size()); + } + +}