diff --git a/hbase-client/src/main/java/org/apache/omid/transaction/AttributeSetSnapshotFilter.java b/hbase-client/src/main/java/org/apache/omid/transaction/AttributeSetSnapshotFilter.java index 6fdcd4472..c3412d538 100644 --- a/hbase-client/src/main/java/org/apache/omid/transaction/AttributeSetSnapshotFilter.java +++ b/hbase-client/src/main/java/org/apache/omid/transaction/AttributeSetSnapshotFilter.java @@ -47,6 +47,8 @@ public Result get(Get get, HBaseTransaction transaction) throws IOException { get.setAttribute(CellUtils.TRANSACTION_ATTRIBUTE, getBuilder(transaction).build().toByteArray()); get.setAttribute(CellUtils.CLIENT_GET_ATTRIBUTE, Bytes.toBytes(true)); get.setAttribute(CellUtils.LL_ATTRIBUTE, Bytes.toBytes(transaction.isLowLatency())); + get.setAttribute(CellUtils.ROW_LEVEL_CONFLICTS_ATTRIBUTE, + Bytes.toBytes(transaction.getConflictDetectionLevel() == HBaseTransactionManager.ConflictDetectionLevel.ROW)); return table.get(get); } @@ -55,6 +57,8 @@ public Result get(Get get, HBaseTransaction transaction) throws IOException { public ResultScanner getScanner(Scan scan, HBaseTransaction transaction) throws IOException { scan.setAttribute(CellUtils.TRANSACTION_ATTRIBUTE, getBuilder(transaction).build().toByteArray()); scan.setAttribute(CellUtils.LL_ATTRIBUTE, Bytes.toBytes(transaction.isLowLatency())); + scan.setAttribute(CellUtils.ROW_LEVEL_CONFLICTS_ATTRIBUTE, + Bytes.toBytes(transaction.getConflictDetectionLevel() == HBaseTransactionManager.ConflictDetectionLevel.ROW)); return table.getScanner(scan); } } diff --git a/hbase-client/src/main/java/org/apache/omid/transaction/HBaseCellId.java b/hbase-client/src/main/java/org/apache/omid/transaction/HBaseCellId.java index a70cfefc5..4b390fa4c 100644 --- a/hbase-client/src/main/java/org/apache/omid/transaction/HBaseCellId.java +++ b/hbase-client/src/main/java/org/apache/omid/transaction/HBaseCellId.java @@ -18,21 +18,29 @@ package org.apache.omid.transaction; import static com.google.common.base.Charsets.UTF_8; +import static org.apache.omid.transaction.CellUtils.SHARED_FAMILY_QUALIFIER; +import com.google.common.base.Charsets; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.omid.tso.client.CellId; import com.google.common.hash.Hasher; import com.google.common.hash.Hashing; +import java.util.Arrays; +import java.util.Objects; + public class HBaseCellId implements CellId { + + private final TTable table; private final byte[] row; private final byte[] family; private final byte[] qualifier; private long timestamp; - public HBaseCellId(TTable table, byte[] row, byte[] family, byte[] qualifier, long timestamp) { + private HBaseCellId(TTable table, byte[] row, byte[] family, byte[] qualifier, long timestamp) { this.timestamp = timestamp; this.table = table; this.row = row; @@ -62,7 +70,7 @@ public long getTimestamp() { @Override public String toString() { - return new String(table.getTableName(), UTF_8) + return (table == null?"NO_TABLE":new String(table.getTableName(), UTF_8)) + ":" + new String(row, UTF_8) + ":" + new String(family, UTF_8) + ":" + new String(qualifier, UTF_8) @@ -97,4 +105,66 @@ public long getRowId() { public static Hasher getHasher() { return Hashing.murmur3_128().newHasher(); } + + @Override + public int hashCode() { + return Objects.hash(Arrays.hashCode(table.getTableName()), + Arrays.hashCode(row), + Arrays.hashCode(family), + Arrays.hashCode(qualifier)); + } + + @Override + public boolean equals(Object o) { + if (o == this) return true; + assert(o instanceof HBaseCellId); + HBaseCellId other = (HBaseCellId) o; + + return Arrays.equals(table.getTableName(), other.getTable().getTableName()) && + Arrays.equals(row, other.getRow()) && + Arrays.equals(family, other.getFamily()) && + Arrays.equals(qualifier, other.getQualifier()); + } + + + public int compareTo(CellId o) { + assert(o instanceof HBaseCellId); + HBaseCellId other = (HBaseCellId) o; + + int tableCompare = Bytes.compareTo(table.getTableName(), other.getTable().getTableName()); + if (tableCompare != 0) { + return tableCompare; + } else { + int rowCompare = Bytes.compareTo(row, other.getRow()); + if (rowCompare != 0) { + return rowCompare; + } else { + int familyCompare = Bytes.compareTo(family, other.getFamily()); + if (familyCompare != 0) { + return familyCompare; + } else { + return Bytes.compareTo(qualifier, other.getQualifier()); + } + } + } + } + public static HBaseCellId valueOf(Transaction tx, TTable table, byte[] row, byte[] family, byte[] qualifier, long timestamp) { + if (((HBaseTransaction)tx).getConflictDetectionLevel() == HBaseTransactionManager.ConflictDetectionLevel.ROW) { + return new HBaseCellId(table, row, family, SHARED_FAMILY_QUALIFIER, timestamp); + } else { + return new HBaseCellId(table, row, family, qualifier, timestamp); + } + } + + public static HBaseCellId valueOf(HBaseTransactionManager.ConflictDetectionLevel conflictDetectionLevel, + TTable table, byte[] row, byte[] family, byte[] qualifier, long timestamp) { + if (conflictDetectionLevel == HBaseTransactionManager.ConflictDetectionLevel.ROW) { + return new HBaseCellId(table, row, family, SHARED_FAMILY_QUALIFIER, timestamp); + } else { + return new HBaseCellId(table, row, family, qualifier, timestamp); + } + } + + + } diff --git a/hbase-client/src/main/java/org/apache/omid/transaction/HBaseOmidClientConfiguration.java b/hbase-client/src/main/java/org/apache/omid/transaction/HBaseOmidClientConfiguration.java index 2800fb6b4..fb031d991 100644 --- a/hbase-client/src/main/java/org/apache/omid/transaction/HBaseOmidClientConfiguration.java +++ b/hbase-client/src/main/java/org/apache/omid/transaction/HBaseOmidClientConfiguration.java @@ -24,7 +24,7 @@ import org.apache.omid.YAMLUtils; import org.apache.omid.metrics.MetricsRegistry; import org.apache.omid.tools.hbase.SecureHBaseConfig; -import org.apache.omid.tso.client.OmidClientConfiguration.ConflictDetectionLevel; + import org.apache.omid.tso.client.OmidClientConfiguration.PostCommitMode; import org.apache.omid.tso.client.OmidClientConfiguration; import org.apache.hadoop.conf.Configuration; @@ -91,14 +91,6 @@ public void setPostCommitMode(PostCommitMode postCommitMode) { omidClientConfiguration.setPostCommitMode(postCommitMode); } - public ConflictDetectionLevel getConflictAnalysisLevel() { - return omidClientConfiguration.getConflictAnalysisLevel(); - } - - public void setConflictAnalysisLevel(ConflictDetectionLevel conflictAnalysisLevel) { - omidClientConfiguration.setConflictAnalysisLevel(conflictAnalysisLevel); - } - public String getCommitTableName() { return commitTableName; } diff --git a/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransaction.java b/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransaction.java index 62ef93676..dafa5f75a 100644 --- a/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransaction.java +++ b/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransaction.java @@ -21,37 +21,76 @@ import java.util.HashSet; import java.util.Set; +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.Result; +import org.apache.hadoop.hbase.util.Bytes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.omid.transaction.HBaseTransactionManager.ConflictDetectionLevel.ROW; + public class HBaseTransaction extends AbstractTransaction { private static final Logger LOG = LoggerFactory.getLogger(HBaseTransaction.class); + private final HBaseTransactionManager.ConflictDetectionLevel conflictDetectionLevel; public HBaseTransaction(long transactionId, long epoch, Set writeSet, Set conflictFreeWriteSet, AbstractTransactionManager tm, boolean isLowLatency) { super(transactionId, epoch, writeSet, conflictFreeWriteSet, tm, isLowLatency); + this.conflictDetectionLevel = ((HBaseTransactionManager)tm).getConflictDetectionLevel(); } public HBaseTransaction(long transactionId, long epoch, Set writeSet, Set conflictFreeWriteSet, AbstractTransactionManager tm, long readTimestamp, long writeTimestamp, boolean isLowLatency) { super(transactionId, epoch, writeSet, conflictFreeWriteSet, tm, readTimestamp, writeTimestamp, isLowLatency); + this.conflictDetectionLevel = ((HBaseTransactionManager)tm).getConflictDetectionLevel(); } public HBaseTransaction(long transactionId, long readTimestamp, VisibilityLevel visibilityLevel, long epoch, Set writeSet, Set conflictFreeWriteSet, AbstractTransactionManager tm, boolean isLowLatency) { super(transactionId, readTimestamp, visibilityLevel, epoch, writeSet, conflictFreeWriteSet, tm, isLowLatency); + this.conflictDetectionLevel = ((HBaseTransactionManager)tm).getConflictDetectionLevel(); + } + + public HBaseTransaction(long transactionId, long readTimestamp, VisibilityLevel visibilityLevel, long epoch, + Set writeSet, Set conflictFreeWriteSet, + HBaseTransactionManager.ConflictDetectionLevel conflictDetectionLevel, boolean isLowLatency) { + super(transactionId, readTimestamp, visibilityLevel, epoch, writeSet, conflictFreeWriteSet, null, isLowLatency); + this.conflictDetectionLevel = conflictDetectionLevel; } + private void deleteCell(HBaseCellId cell) { - Delete delete = new Delete(cell.getRow()); - delete.addColumn(cell.getFamily(), cell.getQualifier(), cell.getTimestamp()); - try { - cell.getTable().getHTable().delete(delete); - } catch (IOException e) { - LOG.warn("Failed cleanup cell {} for Tx {}. This issue has been ignored", cell, getTransactionId(), e); + + if (conflictDetectionLevel == ROW) { + //cell only has family info so we need to query the table to get list of cells. + Get get = new Get(cell.getRow()); + get.addFamily(cell.getFamily()); + try { + get.setTimeStamp(cell.getTimestamp()); + Result res = cell.getTable().getHTable().get(get); + Delete delete = new Delete(cell.getRow()); + for (Cell rawCell: res.listCells()) { + delete.addColumn(cell.getFamily(), + Bytes.copy(rawCell.getQualifierArray(),rawCell.getQualifierOffset(),rawCell.getQualifierLength()), + cell.getTimestamp()); + } + cell.getTable().getHTable().delete(delete); + } catch (IOException e) { + LOG.warn("Failed cleanup cells in family {} for Tx {}. This issue has been ignored", + cell.getFamily(), getTransactionId(), e); + } + } else { + Delete delete = new Delete(cell.getRow()); + delete.addColumn(cell.getFamily(), cell.getQualifier(), cell.getTimestamp()); + try { + cell.getTable().getHTable().delete(delete); + } catch (IOException e) { + LOG.warn("Failed cleanup cell {} for Tx {}. This issue has been ignored", cell, getTransactionId(), e); + } } } @Override @@ -99,4 +138,8 @@ private Set getWrittenTables() { return tables; } + public HBaseTransactionManager.ConflictDetectionLevel getConflictDetectionLevel() { + return conflictDetectionLevel; + } + } diff --git a/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransactionManager.java b/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransactionManager.java index c66b9b2a2..dc089ed33 100644 --- a/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransactionManager.java +++ b/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransactionManager.java @@ -32,7 +32,7 @@ import org.apache.omid.committable.hbase.HBaseCommitTableConfig; import org.apache.omid.tools.hbase.HBaseLogin; import org.apache.omid.tso.client.CellId; -import org.apache.omid.tso.client.OmidClientConfiguration.ConflictDetectionLevel; + import org.apache.omid.tso.client.TSOClient; import org.apache.omid.tso.client.TSOProtocol; import org.slf4j.Logger; @@ -47,6 +47,11 @@ public class HBaseTransactionManager extends AbstractTransactionManager implemen private static final Logger LOG = LoggerFactory.getLogger(HBaseTransactionManager.class); + + public enum ConflictDetectionLevel {CELL, ROW} + + private final ConflictDetectionLevel conflictLevel; + private static class HBaseTransactionFactory implements TransactionFactory { @Override @@ -84,6 +89,7 @@ public static class Builder { private Optional commitTableClient = Optional.absent(); private Optional commitTableWriter = Optional.absent(); private Optional postCommitter = Optional.absent(); + private ConflictDetectionLevel conflictDetectionLevel = ConflictDetectionLevel.CELL; public Builder(HBaseOmidClientConfiguration hbaseOmidClientConf) { this.hbaseOmidClientConf = hbaseOmidClientConf; @@ -109,6 +115,11 @@ Builder postCommitter(PostCommitActions postCommitter) { return this; } + Builder conflictDetectionLevel(ConflictDetectionLevel conflictDetectionLevel) { + this.conflictDetectionLevel = conflictDetectionLevel; + return this; + } + public HBaseTransactionManager build() throws IOException, InterruptedException { CommitTable.Client commitTableClient = this.commitTableClient.or(buildCommitTableClient()).get(); @@ -121,7 +132,8 @@ public HBaseTransactionManager build() throws IOException, InterruptedException tsoClient, commitTableClient, commitTableWriter, - new HBaseTransactionFactory()); + new HBaseTransactionFactory(), + conflictDetectionLevel); } private Optional buildTSOClient() throws IOException, InterruptedException { @@ -175,7 +187,8 @@ private HBaseTransactionManager(HBaseOmidClientConfiguration hBaseOmidClientConf TSOProtocol tsoClient, CommitTable.Client commitTableClient, CommitTable.Writer commitTableWriter, - HBaseTransactionFactory hBaseTransactionFactory) { + HBaseTransactionFactory hBaseTransactionFactory, + ConflictDetectionLevel conflictLevel) { super(hBaseOmidClientConfiguration.getMetrics(), postCommitter, @@ -183,6 +196,7 @@ private HBaseTransactionManager(HBaseOmidClientConfiguration hBaseOmidClientConf commitTableClient, commitTableWriter, hBaseTransactionFactory); + this.conflictLevel = conflictLevel; } // ---------------------------------------------------------------------------------------------------------------- @@ -243,12 +257,9 @@ static HBaseTransaction enforceHBaseTransactionAsParam(AbstractTransaction tryToLocateCellCommitTimestamp(long epoch, - Cell cell, - Map commitCache, - boolean isLowLatency) + private Optional tryToLocateCellCommitTimestamp(long epoch, + Cell cell, + Map commitCache, + boolean isLowLatency, + HBaseTransactionManager.ConflictDetectionLevel conflictLevel) throws IOException { CommitTimestamp tentativeCommitTimestamp = @@ -255,7 +264,7 @@ public Optional tryToLocateCellCommitTimestamp(long epoch, cell.getTimestamp(), epoch, new CommitTimestampLocatorImpl( - new HBaseCellId(null, + HBaseCellId.valueOf(conflictLevel,null, CellUtil.cloneRow(cell), CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), @@ -276,7 +285,8 @@ public Optional tryToLocateCellCommitTimestamp(long epoch, // commit phase of the client probably failed, so we heal the shadow // cell with the right commit timestamp for avoiding further reads to // hit the storage - healShadowCell(cell, tentativeCommitTimestamp.getValue()); + healShadowCell(cell, tentativeCommitTimestamp.getValue(), conflictLevel); + commitCache.put(cell.getTimestamp(), tentativeCommitTimestamp.getValue()); return Optional.of(tentativeCommitTimestamp.getValue()); case CACHE: case SHADOW_CELL: @@ -305,23 +315,59 @@ private Optional getCommitTimestamp(Cell kv, HBaseTransaction transaction, } return tryToLocateCellCommitTimestamp(transaction.getEpoch(), kv, - commitCache, transaction.isLowLatency()); + commitCache, transaction.isLowLatency(), transaction.getConflictDetectionLevel()); } - - private Map buildCommitCache(List rawCells) { - Map commitCache = new HashMap<>(); + // In case of ROW conflict detection, the call to filterCellsForSnapshot may contain a cell that its sc didnt arrive + // because its stacked under a more recent shadow cell. This will cause a get from commit table so we do this get from sc before. + // TODO YONIGO - do we really need this? is access to SC faster than commit table? + private void getMissedShadowCell(List rawCells, Map commitCache) throws IOException { + if (rawCells.size() == 0) + return; + + Map familyShahowCells = new HashMap<>(); + for (Cell cell : rawCells) { + if (CellUtils.isShadowCell(cell)) { + String key = Bytes.toString(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); + familyShahowCells.put(key, cell.getTimestamp()); + } + } + + Get get = new Get(CellUtil.cloneRow(rawCells.get(0))); for (Cell cell : rawCells) { + if (!CellUtils.isShadowCell(cell)) { + String key = Bytes.toString(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); + Long ts = familyShahowCells.get(key); + if (ts != null && cell.getTimestamp() < ts) { + get.addColumn(CellUtil.cloneFamily(cell),CellUtils.addShadowCellSuffixPrefix(CellUtils.SHARED_FAMILY_QUALIFIER)); + get.setTimeStamp(cell.getTimestamp()); + } + } + } + Result result = tableAccessWrapper.get(get); + + + for (Cell cell : result.listCells()) { if (CellUtils.isShadowCell(cell)) { commitCache.put(cell.getTimestamp(), Bytes.toLong(CellUtil.cloneValue(cell))); } } + } + + private Map buildCommitCache(List rawCells) { + Map commitCache = new HashMap<>(); + for (Cell cell : rawCells) { + if (CellUtils.isShadowCell(cell)) { + commitCache.put(cell.getTimestamp(), Bytes.toLong(CellUtil.cloneValue(cell))); + } + } return commitCache; } + // We must find the first commit timestamp (if exists) of any family delete private void buildFamilyDeletionCache(HBaseTransaction transaction, List rawCells, Map familyDeletionCache, Map commitCache, Map attributeMap) throws IOException { for (Cell cell : rawCells) { if (CellUtils.isFamilyDeleteCell(cell)) { @@ -344,15 +390,19 @@ private void buildFamilyDeletionCache(HBaseTransaction transaction, List r boolean foundCommittedFamilyDeletion = false; while (!foundCommittedFamilyDeletion) { - Get g = createPendingGet(lastCell, 3); + Get g = createPendingGet(lastCell, 3, transaction.getConflictDetectionLevel()); Result result = tableAccessWrapper.get(g); List resultCells = result.listCells(); - if (resultCells == null) { + if (!hasFamilyDelete(resultCells)) { break; } cmtCache = buildCommitCache(resultCells); + if (transaction.getConflictDetectionLevel() == ROW) { + getMissedShadowCell(rawCells, cmtCache); + } + for (Cell c : resultCells) { if (CellUtils.isFamilyDeleteCell(c)) { commitTimeStamp = getTSIfInSnapshot(c, transaction, cmtCache); @@ -370,6 +420,17 @@ private void buildFamilyDeletionCache(HBaseTransaction transaction, List r } } + private boolean hasFamilyDelete(List resultCells) { + if (resultCells == null) { + return false; + } + for (Cell cell: resultCells) { + if (CellUtils.isFamilyDeleteCell(cell)) { + return true; + } + } + return false; + } public Optional getTSIfInTransaction(Cell kv, HBaseTransaction transaction) { long startTimestamp = transaction.getStartTimestamp(); @@ -397,19 +458,24 @@ public Optional getTSIfInSnapshot(Cell kv, HBaseTransaction transaction, M return Optional.absent(); } - private Get createPendingGet(Cell cell, int versionCount) throws IOException { - + private Get createPendingGet(Cell cell, int versionCount, HBaseTransactionManager.ConflictDetectionLevel cfLevel) + throws IOException { Get pendingGet = new Get(CellUtil.cloneRow(cell)); pendingGet.addColumn(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell)); - pendingGet.addColumn(CellUtil.cloneFamily(cell), CellUtils.addShadowCellSuffixPrefix(cell.getQualifierArray(), - cell.getQualifierOffset(), - cell.getQualifierLength())); + if (cfLevel == ROW) { + pendingGet.addColumn(CellUtil.cloneFamily(cell), CellUtils.addShadowCellSuffixPrefix(CellUtils.SHARED_FAMILY_QUALIFIER)); + }else { + pendingGet.addColumn(CellUtil.cloneFamily(cell), CellUtils.addShadowCellSuffixPrefix(cell.getQualifierArray(), + cell.getQualifierOffset(), + cell.getQualifierLength())); + } pendingGet.setMaxVersions(versionCount); pendingGet.setTimeRange(0, cell.getTimestamp()); return pendingGet; } + /** * Filters the raw results returned from HBase and returns only those belonging to the current snapshot, as defined * by the transaction object. If the raw results don't contain enough information for a particular qualifier, it @@ -435,6 +501,9 @@ public List filterCellsForSnapshot(List rawCells, HBaseTransaction t } Map commitCache = buildCommitCache(rawCells); + if (transaction.getConflictDetectionLevel() == ROW) { + getMissedShadowCell(rawCells, commitCache); + } buildFamilyDeletionCache(transaction, rawCells, familyDeletionCache, commitCache, attributeMap); ImmutableList> filteredCells; @@ -474,7 +543,7 @@ public List filterCellsForSnapshot(List rawCells, HBaseTransaction t } if (!snapshotValueFound) { assert (oldestCell != null); - Get pendingGet = createPendingGet(oldestCell, numberOfVersionsToFetch); + Get pendingGet = createPendingGet(oldestCell, numberOfVersionsToFetch, transaction.getConflictDetectionLevel()); for (Map.Entry entry : attributeMap.entrySet()) { pendingGet.setAttribute(entry.getKey(), entry.getValue()); } diff --git a/hbase-client/src/main/java/org/apache/omid/transaction/TTable.java b/hbase-client/src/main/java/org/apache/omid/transaction/TTable.java index 44f0708ef..77aaafe96 100644 --- a/hbase-client/src/main/java/org/apache/omid/transaction/TTable.java +++ b/hbase-client/src/main/java/org/apache/omid/transaction/TTable.java @@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.util.Bytes; import org.apache.omid.committable.CommitTable; -import org.apache.omid.tso.client.OmidClientConfiguration.ConflictDetectionLevel; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -193,13 +193,18 @@ public Result get(Transaction tx, final Get get) throws IOException { Map> kvs = get.getFamilyMap(); for (Map.Entry> entry : kvs.entrySet()) { byte[] family = entry.getKey(); + if (((HBaseTransaction)tx).getConflictDetectionLevel() == HBaseTransactionManager.ConflictDetectionLevel.ROW) { + tsget.addColumn(family, CellUtils.addShadowCellSuffixPrefix(CellUtils.SHARED_FAMILY_QUALIFIER)); + } NavigableSet qualifiers = entry.getValue(); if (qualifiers == null || qualifiers.isEmpty()) { tsget.addFamily(family); } else { for (byte[] qualifier : qualifiers) { tsget.addColumn(family, qualifier); - tsget.addColumn(family, CellUtils.addShadowCellSuffixPrefix(qualifier)); + if (((HBaseTransaction)tx).getConflictDetectionLevel() == HBaseTransactionManager.ConflictDetectionLevel.CELL) { + tsget.addColumn(family, CellUtils.addShadowCellSuffixPrefix(qualifier)); + } } tsget.addColumn(family, CellUtils.FAMILY_DELETE_QUALIFIER); tsget.addColumn(family, CellUtils.addShadowCellSuffixPrefix(CellUtils.FAMILY_DELETE_QUALIFIER)); @@ -226,12 +231,12 @@ private void familyQualifierBasedDeletion(HBaseTransaction tx, Put deleteP, Get byte[] family = entryF.getKey(); for (Entry> entryQ : entryF.getValue().entrySet()) { byte[] qualifier = entryQ.getKey(); - addWriteSetElement(tx, new HBaseCellId(this, deleteP.getRow(), family, qualifier, + addWriteSetElement(tx, HBaseCellId.valueOf(tx,this, deleteP.getRow(), family, qualifier, tx.getWriteTimestamp())); } deleteP.addColumn(family, CellUtils.FAMILY_DELETE_QUALIFIER, tx.getWriteTimestamp(), HConstants.EMPTY_BYTE_ARRAY); - addWriteSetElement(tx, new HBaseCellId(this, deleteP.getRow(), family, CellUtils.FAMILY_DELETE_QUALIFIER, + addWriteSetElement(tx, HBaseCellId.valueOf(tx,this, deleteP.getRow(), family, CellUtils.FAMILY_DELETE_QUALIFIER, tx.getWriteTimestamp())); } } @@ -243,7 +248,7 @@ private void familyQualifierBasedDeletionWithOutRead(HBaseTransaction tx, Put d for (byte[] family : fset) { deleteP.addColumn(family, CellUtils.FAMILY_DELETE_QUALIFIER, tx.getWriteTimestamp(), HConstants.EMPTY_BYTE_ARRAY); - addWriteSetElement(tx, new HBaseCellId(this, deleteP.getRow(), family, CellUtils.FAMILY_DELETE_QUALIFIER, + addWriteSetElement(tx, HBaseCellId.valueOf(tx,this, deleteP.getRow(), family, CellUtils.FAMILY_DELETE_QUALIFIER, tx.getWriteTimestamp())); } @@ -291,7 +296,7 @@ private Put deleteInternal(Transaction tx, Delete delete) throws IOException { writeTimestamp, CellUtils.DELETE_TOMBSTONE); addWriteSetElement(transaction, - new HBaseCellId(this, + HBaseCellId.valueOf(tx,this, delete.getRow(), CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), @@ -308,7 +313,7 @@ private Put deleteInternal(Transaction tx, Delete delete) throws IOException { writeTimestamp, CellUtils.DELETE_TOMBSTONE); addWriteSetElement(transaction, - new HBaseCellId(this, + HBaseCellId.valueOf(tx,this, delete.getRow(), CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), @@ -324,8 +329,7 @@ private Put deleteInternal(Transaction tx, Delete delete) throws IOException { } } if (deleteFamily) { - if (enforceHBaseTransactionManagerAsParam(transaction.getTransactionManager()). - getConflictDetectionLevel() == ConflictDetectionLevel.ROW) { + if (transaction.getConflictDetectionLevel() == HBaseTransactionManager.ConflictDetectionLevel.ROW) { familyQualifierBasedDeletionWithOutRead(transaction, deleteP, deleteG); } else { familyQualifierBasedDeletion(transaction, deleteP, deleteG); @@ -419,7 +423,7 @@ private Put putInternal(Transaction tx, Put put, boolean addShadowCell) throws I kv.getTimestamp(), Bytes.toBytes(kv.getTimestamp())); } else { - HBaseCellId cellId = new HBaseCellId(this, + HBaseCellId cellId = HBaseCellId.valueOf(tx,this, CellUtil.cloneRow(kv), CellUtil.cloneFamily(kv), CellUtil.cloneQualifier(kv), @@ -480,11 +484,18 @@ public ResultScanner getScanner(Transaction tx, Scan scan) throws IOException { if (qualifiers == null) { continue; } - for (byte[] qualifier : qualifiers) { - tsscan.addColumn(family, CellUtils.addShadowCellSuffixPrefix(qualifier)); + if (transaction.getConflictDetectionLevel() == HBaseTransactionManager.ConflictDetectionLevel.ROW) { + tsscan.addColumn(family, CellUtils.addShadowCellSuffixPrefix(CellUtils.SHARED_FAMILY_QUALIFIER)); + } else { + for (byte[] qualifier : qualifiers) { + tsscan.addColumn(family, CellUtils.addShadowCellSuffixPrefix(qualifier)); + } } if (!qualifiers.isEmpty()) { tsscan.addColumn(entry.getKey(), CellUtils.FAMILY_DELETE_QUALIFIER); + if (transaction.getConflictDetectionLevel() == HBaseTransactionManager.ConflictDetectionLevel.CELL) { + tsscan.addColumn(family, CellUtils.addShadowCellSuffixPrefix(CellUtils.FAMILY_DELETE_QUALIFIER)); + } } } @@ -598,7 +609,7 @@ public ResultScanner getScanner(Transaction transaction, byte[] family, byte[] q * * @param transaction an instance of transaction to be used * @param puts List of puts - * @param addShadowCell denotes whether to add the shadow cell + * @param addShadowCells denotes whether to add the shadow cell * @throws IOException if a remote or network exception occurs */ public void put(Transaction transaction, List puts, boolean addShadowCells) throws IOException { @@ -618,7 +629,7 @@ public void put(Transaction transaction, List puts) throws IOException { * * @param transaction an instance of transaction to be used * @param rows List of rows that must be instances of Put or Delete - * @param addShadowCell denotes whether to add the shadow cell + * @param addShadowCells denotes whether to add the shadow cell * @throws IOException if a remote or network exception occurs */ public void batch(Transaction transaction, List rows, boolean addShadowCells) throws IOException { diff --git a/hbase-client/src/test/java/org/apache/omid/transaction/OmidTestBase.java b/hbase-client/src/test/java/org/apache/omid/transaction/OmidTestBase.java index 0e7969db5..8628eb0d0 100644 --- a/hbase-client/src/test/java/org/apache/omid/transaction/OmidTestBase.java +++ b/hbase-client/src/test/java/org/apache/omid/transaction/OmidTestBase.java @@ -22,6 +22,8 @@ import java.io.File; import java.io.IOException; import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; @@ -137,14 +139,24 @@ public void beforeGroups(ITestContext context) throws Exception { } protected void createTestTable() throws IOException { + createTable(TEST_TABLE); + } + + protected void createTable(String tableName) throws IOException { + List families = new ArrayList<>(); + families.add(TEST_FAMILY2); + families.add(TEST_FAMILY); + createTable(tableName, families); + } + + protected void createTable(String tableName, List families) throws IOException { HBaseAdmin admin = hBaseUtils.getHBaseAdmin(); - HTableDescriptor test_table_desc = new HTableDescriptor(TableName.valueOf(TEST_TABLE)); - HColumnDescriptor datafam = new HColumnDescriptor(TEST_FAMILY); - HColumnDescriptor datafam2 = new HColumnDescriptor(TEST_FAMILY2); - datafam.setMaxVersions(Integer.MAX_VALUE); - datafam2.setMaxVersions(Integer.MAX_VALUE); - test_table_desc.addFamily(datafam); - test_table_desc.addFamily(datafam2); + HTableDescriptor test_table_desc = new HTableDescriptor(TableName.valueOf(tableName)); + for (String family: families) { + HColumnDescriptor datafam = new HColumnDescriptor(family); + datafam.setMaxVersions(Integer.MAX_VALUE); + test_table_desc.addFamily(datafam); + } admin.createTable(test_table_desc); } @@ -172,6 +184,37 @@ protected TransactionManager newTransactionManager(ITestContext context) throws return newTransactionManager(context, getClient(context)); } + protected TransactionManager newTransactionManager(ITestContext context, HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Exception { + return newTransactionManager(context, getClient(context), cfLevel); + } + + protected TransactionManager newTransactionManager(ITestContext context, TSOClient tsoClient, + HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Exception { + HBaseOmidClientConfiguration clientConf = new HBaseOmidClientConfiguration(); + clientConf.setConnectionString("localhost:1234"); + clientConf.setHBaseConfiguration(hbaseConf); + return HBaseTransactionManager.builder(clientConf) + .commitTableClient(getCommitTable(context).getClient()) + .commitTableWriter(getCommitTable(context).getWriter()) + .conflictDetectionLevel(cfLevel) + .tsoClient(tsoClient).build(); + } + + protected TransactionManager newTransactionManager(ITestContext context, + PostCommitActions postCommitActions, + HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Exception { + HBaseOmidClientConfiguration clientConf = new HBaseOmidClientConfiguration(); + clientConf.setConnectionString("localhost:1234"); + clientConf.setHBaseConfiguration(hbaseConf); + return HBaseTransactionManager.builder(clientConf) + .postCommitter(postCommitActions) + .commitTableClient(getCommitTable(context).getClient()) + .commitTableWriter(getCommitTable(context).getWriter()) + .conflictDetectionLevel(cfLevel) + .tsoClient(getClient(context)).build(); + } + + protected TransactionManager newTransactionManager(ITestContext context, PostCommitActions postCommitActions) throws Exception { HBaseOmidClientConfiguration clientConf = new HBaseOmidClientConfiguration(); clientConf.setConnectionString("localhost:1234"); @@ -221,6 +264,8 @@ public void afterMethod() { try { LOG.info("tearing Down"); Admin admin = hBaseUtils.getHBaseAdmin(); + HTableDescriptor[] tables = admin.listTables(); + deleteTable(admin, TableName.valueOf(TEST_TABLE)); createTestTable(); if (hBaseCommitTableConfig != null) { diff --git a/hbase-client/src/test/java/org/apache/omid/transaction/TestDeletion.java b/hbase-client/src/test/java/org/apache/omid/transaction/TestDeletion.java index 3c4387d23..f4f2b1e46 100644 --- a/hbase-client/src/test/java/org/apache/omid/transaction/TestDeletion.java +++ b/hbase-client/src/test/java/org/apache/omid/transaction/TestDeletion.java @@ -30,8 +30,9 @@ 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.Table; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.omid.tso.client.OmidClientConfiguration.ConflictDetectionLevel; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.ITestContext; @@ -61,13 +62,13 @@ private static class FamCol { } - @Test(timeOut = 10_000) + + @Test(timeOut = 60_000) public void runTestDeleteFamilyRow(ITestContext context) throws Exception { - TransactionManager tm = newTransactionManager(context); + TransactionManager tm = newTransactionManager(context, HBaseTransactionManager.ConflictDetectionLevel.ROW); TTable tt = new TTable(connection, TEST_TABLE); - ((HBaseTransactionManager) tm).setConflictDetectionLevel(ConflictDetectionLevel.ROW); Transaction t1 = tm.begin(); LOG.info("Transaction created " + t1); @@ -82,6 +83,7 @@ public void runTestDeleteFamilyRow(ITestContext context) throws Exception { d.addFamily(famA); tt.delete(t2, d); + Transaction tscan = tm.begin(); ResultScanner rs = tt.getScanner(tscan, new Scan()); @@ -109,9 +111,6 @@ public void runTestDeleteFamilyRow(ITestContext context) throws Exception { count = countColsInRows(rs, famColA); countFamColA = count.get(famColA); assertEquals(countFamColA, null); - - ((HBaseTransactionManager) tm).setConflictDetectionLevel(ConflictDetectionLevel.CELL); - } @Test(timeOut = 10_000) @@ -205,11 +204,9 @@ public void runTestDeleteFamily(ITestContext context) throws Exception { @Test(timeOut = 10_000) public void runTestDeleteFamilyRowLevelCA(ITestContext context) throws Exception { - TransactionManager tm = newTransactionManager(context); + TransactionManager tm = newTransactionManager(context, HBaseTransactionManager.ConflictDetectionLevel.ROW); TTable tt = new TTable(connection, TEST_TABLE); - ((HBaseTransactionManager) tm).setConflictDetectionLevel(ConflictDetectionLevel.ROW); - Transaction t1 = tm.begin(); LOG.info("Transaction created " + t1); @@ -242,17 +239,14 @@ public void runTestDeleteFamilyRowLevelCA(ITestContext context) throws Exception assertEquals((int) count.get(famColA), (rowsWritten - 1), "ColA count should be equal to rowsWritten - 1"); assertEquals((int) count.get(famColB), rowsWritten, "ColB count should be equal to rowsWritten"); - ((HBaseTransactionManager) tm).setConflictDetectionLevel(ConflictDetectionLevel.CELL); } @Test(timeOut = 10_000) public void runTestDeleteFamilyAborts(ITestContext context) throws Exception { - TransactionManager tm = newTransactionManager(context); + TransactionManager tm = newTransactionManager(context, HBaseTransactionManager.ConflictDetectionLevel.ROW); TTable tt = new TTable(connection, TEST_TABLE); - ((HBaseTransactionManager) tm).setConflictDetectionLevel(ConflictDetectionLevel.ROW); - Transaction t1 = tm.begin(); LOG.info("Transaction created " + t1); @@ -283,7 +277,6 @@ public void runTestDeleteFamilyAborts(ITestContext context) throws Exception { assertEquals((int) count.get(famColA), rowsWritten, "ColA count should be equal to rowsWritten"); assertEquals((int) count.get(famColB), rowsWritten, "ColB count should be equal to rowsWritten"); - ((HBaseTransactionManager) tm).setConflictDetectionLevel(ConflictDetectionLevel.CELL); } @Test(timeOut = 10_000) diff --git a/hbase-client/src/test/java/org/apache/omid/transaction/TestHBaseCells.java b/hbase-client/src/test/java/org/apache/omid/transaction/TestHBaseCells.java new file mode 100644 index 000000000..a6332373e --- /dev/null +++ b/hbase-client/src/test/java/org/apache/omid/transaction/TestHBaseCells.java @@ -0,0 +1,120 @@ +package org.apache.omid.transaction; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.util.Bytes; +import org.testng.ITestContext; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.util.Set; + +import static org.testng.AssertJUnit.assertEquals; + +@Test(groups = "sharedHBase") +public class TestHBaseCells extends OmidTestBase { + + + + + + + @Test + public void testCellModeFilter(ITestContext context) throws Exception { + + byte[] row1 = Bytes.toBytes("row123"); + byte[] qualifier = Bytes.toBytes("column123"); + byte[] qualifier2 = Bytes.toBytes("column123-2"); + byte[] data1 = Bytes.toBytes("data1"); + + Table table1 = connection.getTable(TableName.valueOf(TEST_TABLE)); + TransactionManager tm = newTransactionManager(context); + TTable ttable1 = new TTable(table1); + + AbstractTransaction tx1 = (AbstractTransaction) tm.begin(); + Put put = new Put(row1); + put.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier, data1); + ttable1.put(tx1, put); + ttable1.put(tx1, put); + assertEquals(1, tx1.getWriteSet().size()); + + //now add two from same row. + AbstractTransaction tx2 = (AbstractTransaction) tm.begin(); + Put put2 = new Put(row1); + put2.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier, data1); + put2.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier2, data1); + ttable1.put(tx2, put2); + assertEquals(2, tx2.getWriteSet().size()); + + //add from different table + deleteTable(hBaseUtils.getHBaseAdmin(), TableName.valueOf("table2")); + createTable("table2"); + Table table2 = connection.getTable(TableName.valueOf("table2")); + TTable ttable2 = new TTable(table2); + + AbstractTransaction tx3 = (AbstractTransaction) tm.begin(); + Put put3 = new Put(row1); + put3.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier, data1); + + ttable1.put(tx3, put3); + ttable2.put(tx3, put3); + assertEquals(2, tx3.getWriteSet().size()); + + } + + + @Test + public void testRowModeFilter(ITestContext context) throws Exception { + + byte[] row1 = Bytes.toBytes("row123"); + byte[] qualifier = Bytes.toBytes("column123"); + byte[] qualifier2 = Bytes.toBytes("column123-2"); + byte[] data1 = Bytes.toBytes("data1"); + + Table table1 = connection.getTable(TableName.valueOf(TEST_TABLE)); + TransactionManager tm = newTransactionManager(context, HBaseTransactionManager.ConflictDetectionLevel.ROW); + TTable ttable1 = new TTable(table1); + + AbstractTransaction tx1 = (AbstractTransaction) tm.begin(); + Put put = new Put(row1); + put.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier, data1); + ttable1.put(tx1, put); + ttable1.put(tx1, put); + assertEquals(1, tx1.getWriteSet().size()); + + //now add two from same row. + AbstractTransaction tx2 = (AbstractTransaction) tm.begin(); + Put put2 = new Put(row1); + put2.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier, data1); + put2.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier2, data1); + ttable1.put(tx2, put2); + assertEquals(1, tx2.getWriteSet().size()); + + //add from different table + deleteTable(hBaseUtils.getHBaseAdmin(), TableName.valueOf("table2")); + createTable("table2"); + Table table2 = connection.getTable(TableName.valueOf("table2")); + TTable ttable2 = new TTable(table2); + + AbstractTransaction tx3 = (AbstractTransaction) tm.begin(); + Put put3 = new Put(row1); + put3.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier, data1); + + ttable1.put(tx3, put3); + ttable2.put(tx3, put3); + assertEquals(2, tx3.getWriteSet().size()); + + } + + + @Test + public void testCellCompare() { + + } + +} diff --git a/hbase-client/src/test/java/org/apache/omid/transaction/TestHBaseTransactionClient.java b/hbase-client/src/test/java/org/apache/omid/transaction/TestHBaseTransactionClient.java index fb5efdf7f..ef34d0daa 100644 --- a/hbase-client/src/test/java/org/apache/omid/transaction/TestHBaseTransactionClient.java +++ b/hbase-client/src/test/java/org/apache/omid/transaction/TestHBaseTransactionClient.java @@ -82,9 +82,9 @@ public void testIsCommitted(ITestContext context) throws Exception { table.put(t3, put); tm.commit(t3); - HBaseCellId hBaseCellId1 = new HBaseCellId(table, row1, family, qualifier, t1.getStartTimestamp()); - HBaseCellId hBaseCellId2 = new HBaseCellId(table, row2, family, qualifier, t2.getStartTimestamp()); - HBaseCellId hBaseCellId3 = new HBaseCellId(table, row2, family, qualifier, t3.getStartTimestamp()); + HBaseCellId hBaseCellId1 = HBaseCellId.valueOf(t1, table, row1, family, qualifier, t1.getStartTimestamp()); + HBaseCellId hBaseCellId2 = HBaseCellId.valueOf(t2, table, row2, family, qualifier, t2.getStartTimestamp()); + HBaseCellId hBaseCellId3 = HBaseCellId.valueOf(t3, table, row2, family, qualifier, t3.getStartTimestamp()); assertTrue(snapshotFilter.isCommitted(hBaseCellId1, 0, false), "row1 should be committed"); assertFalse(snapshotFilter.isCommitted(hBaseCellId2, 0, false), "row2 should not be committed for kv2"); @@ -121,7 +121,7 @@ public void testCrashAfterCommit(ITestContext context) throws Exception { assertFalse(CellUtils.hasShadowCell(row1, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Shadow cell should not be there"); - HBaseCellId hBaseCellId = new HBaseCellId(table, row1, family, qualifier, t1.getStartTimestamp()); + HBaseCellId hBaseCellId = HBaseCellId.valueOf(t1, table, row1, family, qualifier, t1.getStartTimestamp()); HBaseTransactionClient hbaseTm = (HBaseTransactionClient) newTransactionManager(context); assertTrue(snapshotFilter.isCommitted(hBaseCellId, 0, false), "row1 should be committed"); @@ -195,7 +195,7 @@ public void testReadCommitTimestampFromShadowCell(ITestContext context) throws E try (TTable table = spy(new TTable(htable, snapshotFilter, false))) { // Test first we can not found a non-existent cell ts - HBaseCellId hBaseCellId = new HBaseCellId(table, row1, family, qualifier, NON_EXISTING_CELL_TS); + HBaseCellId hBaseCellId = HBaseCellId.valueOf(tm.getConflictDetectionLevel(), table, row1, family, qualifier, NON_EXISTING_CELL_TS); // Set an empty cache to allow to bypass the checking CommitTimestampLocator ctLocator = new CommitTimestampLocatorImpl(hBaseCellId, Maps.newHashMap()); @@ -236,7 +236,7 @@ public void testCellCommitTimestampIsLocatedInCache(ITestContext context) throws tm.getCommitTableClient()); TTable table = new TTable(htable, snapshotFilter, false); - HBaseCellId hBaseCellId = new HBaseCellId(table, row1, family, qualifier, CELL_ST); + HBaseCellId hBaseCellId = HBaseCellId.valueOf(tm.getConflictDetectionLevel(), table, row1, family, qualifier, CELL_ST); Map fakeCache = Maps.newHashMap(); fakeCache.put(CELL_ST, CELL_CT); @@ -280,7 +280,7 @@ public void testCellCommitTimestampIsLocatedInCommitTable(ITestContext context) } // Test the locator finds the appropriate data in the commit table - HBaseCellId hBaseCellId = new HBaseCellId(table, row1, family, qualifier, + HBaseCellId hBaseCellId = HBaseCellId.valueOf(tx1, table, row1, family, qualifier, tx1.getStartTimestamp()); CommitTimestampLocator ctLocator = new CommitTimestampLocatorImpl(hBaseCellId, Maps.newHashMap()); @@ -314,7 +314,7 @@ public void testCellCommitTimestampIsLocatedInShadowCells(ITestContext context) // Upon commit, the commit data should be in the shadow cells // Test the locator finds the appropriate data in the shadow cells - HBaseCellId hBaseCellId = new HBaseCellId(table, row1, family, qualifier, + HBaseCellId hBaseCellId = HBaseCellId.valueOf(tx1, table, row1, family, qualifier, tx1.getStartTimestamp()); CommitTimestampLocator ctLocator = new CommitTimestampLocatorImpl(hBaseCellId, Maps.newHashMap()); @@ -355,7 +355,7 @@ public void testCellFromTransactionInPreviousEpochGetsInvalidComitTimestamp(ITes // Upon commit, the commit data should be in the shadow cells // Test a transaction in the previous epoch gets an InvalidCommitTimestamp class - HBaseCellId hBaseCellId = new HBaseCellId(table, row1, family, qualifier, + HBaseCellId hBaseCellId = HBaseCellId.valueOf(tx1, table, row1, family, qualifier, tx1.getStartTimestamp()); CommitTimestampLocator ctLocator = new CommitTimestampLocatorImpl(hBaseCellId, Maps.newHashMap()); @@ -404,7 +404,7 @@ public void testCellCommitTimestampIsLocatedInCommitTableAfterNotBeingInvalidate } // Test the locator finds the appropriate data in the commit table - HBaseCellId hBaseCellId = new HBaseCellId(table, row1, family, qualifier, + HBaseCellId hBaseCellId = HBaseCellId.valueOf(tx1, table, row1, family, qualifier, tx1.getStartTimestamp()); CommitTimestampLocator ctLocator = new CommitTimestampLocatorImpl(hBaseCellId, Maps.newHashMap()); @@ -444,7 +444,7 @@ public void testCellCommitTimestampIsLocatedInShadowCellsAfterNotBeingInvalidate // Upon commit, the commit data should be in the shadow cells // Test the locator finds the appropriate data in the shadow cells - HBaseCellId hBaseCellId = new HBaseCellId(table, row1, family, qualifier, + HBaseCellId hBaseCellId = HBaseCellId.valueOf(tx1, table, row1, family, qualifier, tx1.getStartTimestamp()); CommitTimestampLocator ctLocator = new CommitTimestampLocatorImpl(hBaseCellId, Maps.newHashMap()); @@ -475,7 +475,7 @@ public void testCTLocatorReturnsAValidCTWhenNotPresent(ITestContext context) thr tm.getCommitTableClient()); try (TTable table = spy(new TTable(htable, snapshotFilter, false))) { - HBaseCellId hBaseCellId = new HBaseCellId(table, row1, family, qualifier, CELL_TS); + HBaseCellId hBaseCellId = HBaseCellId.valueOf(((HBaseTransactionManager)tm).getConflictDetectionLevel(), table, row1, family, qualifier, CELL_TS); CommitTimestampLocator ctLocator = new CommitTimestampLocatorImpl(hBaseCellId, Maps.newHashMap()); CommitTimestamp ct = snapshotFilter.locateCellCommitTimestamp(CELL_TS, tm.tsoClient.getEpoch(), @@ -484,6 +484,9 @@ public void testCTLocatorReturnsAValidCTWhenNotPresent(ITestContext context) thr assertEquals(ct.getValue(), -1L); assertTrue(ct.getLocation().compareTo(NOT_PRESENT) == 0); } - } + + + + } diff --git a/hbase-client/src/test/java/org/apache/omid/transaction/TestOmidLLRaces.java b/hbase-client/src/test/java/org/apache/omid/transaction/TestOmidLLRaces.java index 213615dce..6709221eb 100644 --- a/hbase-client/src/test/java/org/apache/omid/transaction/TestOmidLLRaces.java +++ b/hbase-client/src/test/java/org/apache/omid/transaction/TestOmidLLRaces.java @@ -194,9 +194,9 @@ public void testIsCommitted() throws Exception { table.put(t3, put); tm.commit(t3); - HBaseCellId hBaseCellId1 = new HBaseCellId(table, row1, family, qualifier, t1.getStartTimestamp()); - HBaseCellId hBaseCellId2 = new HBaseCellId(table, row2, family, qualifier, t2.getStartTimestamp()); - HBaseCellId hBaseCellId3 = new HBaseCellId(table, row2, family, qualifier, t3.getStartTimestamp()); + HBaseCellId hBaseCellId1 = HBaseCellId.valueOf(t1, table, row1, family, qualifier, t1.getStartTimestamp()); + HBaseCellId hBaseCellId2 = HBaseCellId.valueOf(t2, table, row2, family, qualifier, t2.getStartTimestamp()); + HBaseCellId hBaseCellId3 = HBaseCellId.valueOf(t3, table, row2, family, qualifier, t3.getStartTimestamp()); assertTrue(snapshotFilter.isCommitted(hBaseCellId1, 0, false), "row1 should be committed"); assertFalse(snapshotFilter.isCommitted(hBaseCellId2, 0, false), "row2 should not be committed for kv2"); diff --git a/hbase-client/src/test/java/org/apache/omid/transaction/TestRowLevelTSOConflict.java b/hbase-client/src/test/java/org/apache/omid/transaction/TestRowLevelTSOConflict.java new file mode 100644 index 000000000..f4c6260e4 --- /dev/null +++ b/hbase-client/src/test/java/org/apache/omid/transaction/TestRowLevelTSOConflict.java @@ -0,0 +1,116 @@ +package org.apache.omid.transaction; + + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.util.Bytes; +import org.testng.ITestContext; +import org.testng.annotations.Test; + +import static org.testng.AssertJUnit.fail; + +@Test(groups = "sharedHBase") +public class TestRowLevelTSOConflict extends OmidTestBase { + + + @Test + public void testRowLevelConflict(ITestContext context) throws Exception { + byte[] row1 = Bytes.toBytes("row123"); + byte[] qualifier = Bytes.toBytes("column123"); + byte[] qualifier2 = Bytes.toBytes("column123-2"); + byte[] data1 = Bytes.toBytes("data1"); + + Table table1 = connection.getTable(TableName.valueOf(TEST_TABLE)); + TransactionManager tm = newTransactionManager(context, HBaseTransactionManager.ConflictDetectionLevel.ROW); + TTable ttable1 = new TTable(table1); + + AbstractTransaction tx1 = (AbstractTransaction) tm.begin(); + Put put = new Put(row1); + put.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier, data1); + ttable1.put(tx1, put); + + AbstractTransaction tx2 = (AbstractTransaction) tm.begin(); + Put put2 = new Put(row1); + put2.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier2, data1); + ttable1.put(tx2, put2); + + tm.commit(tx1); + try { + tm.commit(tx2); + } catch (RollbackException e) { + return; + } + fail(); + } + + + @Test + public void testRowLevelNoConflict(ITestContext context) throws Exception { + byte[] row1 = Bytes.toBytes("row123"); + byte[] row2 = Bytes.toBytes("row2-123"); + byte[] qualifier = Bytes.toBytes("column123"); + byte[] data1 = Bytes.toBytes("data1"); + + Table table1 = connection.getTable(TableName.valueOf(TEST_TABLE)); + TransactionManager tm = newTransactionManager(context, HBaseTransactionManager.ConflictDetectionLevel.ROW); + TTable ttable1 = new TTable(table1); + + AbstractTransaction tx1 = (AbstractTransaction) tm.begin(); + Put put = new Put(row1); + put.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier, data1); + ttable1.put(tx1, put); + + AbstractTransaction tx2 = (AbstractTransaction) tm.begin(); + Put put2 = new Put(row2); + put2.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier, data1); + ttable1.put(tx2, put2); + + tm.commit(tx1); + try { + tm.commit(tx2); + } catch (RollbackException e) { + fail(); + } + + } + + + + @Test + public void testRowLevelConflict2Tables(ITestContext context) throws Exception { + byte[] row1 = Bytes.toBytes("row123"); + byte[] qualifier = Bytes.toBytes("column123"); + byte[] data1 = Bytes.toBytes("data1"); + + Table table1 = connection.getTable(TableName.valueOf(TEST_TABLE)); + TTable ttable1 = new TTable(table1); + deleteTable(hBaseUtils.getHBaseAdmin(), TableName.valueOf("table2")); + createTable("table2"); + Table table2 = connection.getTable(TableName.valueOf("table2")); + TTable ttable2 = new TTable(table2); + TransactionManager tm = newTransactionManager(context, HBaseTransactionManager.ConflictDetectionLevel.ROW); + + //Same put to be put in different tables + Put put = new Put(row1); + put.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier, data1); + Put put2 = new Put(row1); + put2.addColumn(Bytes.toBytes(TEST_FAMILY), qualifier, data1); + + AbstractTransaction tx1 = (AbstractTransaction) tm.begin(); + ttable1.put(tx1, put); + + AbstractTransaction tx2 = (AbstractTransaction) tm.begin(); + ttable2.put(tx2, put2); + + tm.commit(tx1); + try { + tm.commit(tx2); + } catch (RollbackException e) { + fail(); + } + } + + + +} diff --git a/hbase-client/src/test/java/org/apache/omid/transaction/TestShadowCells.java b/hbase-client/src/test/java/org/apache/omid/transaction/TestShadowCells.java index 23ea8098c..998d4cb22 100644 --- a/hbase-client/src/test/java/org/apache/omid/transaction/TestShadowCells.java +++ b/hbase-client/src/test/java/org/apache/omid/transaction/TestShadowCells.java @@ -17,8 +17,11 @@ */ package org.apache.omid.transaction; +import static org.apache.omid.transaction.CellUtils.SHARED_FAMILY_QUALIFIER; import static org.apache.omid.transaction.CellUtils.hasCell; import static org.apache.omid.transaction.CellUtils.hasShadowCell; +import static org.apache.omid.transaction.HBaseTransactionManager.ConflictDetectionLevel.CELL; +import static org.apache.omid.transaction.HBaseTransactionManager.ConflictDetectionLevel.ROW; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyLong; @@ -28,10 +31,9 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNull; -import static org.testng.Assert.assertTrue; +import static org.testng.Assert.*; +import java.io.IOException; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -59,6 +61,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.ITestContext; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import com.google.common.base.Charsets; @@ -85,10 +88,72 @@ public class TestShadowCells extends OmidTestBase { private static final byte[] data1 = Bytes.toBytes("testWrite-1"); + @DataProvider(name = "cflevel") + public static Object[][] cfLevels() { + return new Object[][] { +// {CELL}, + {ROW}}; + } + + + private byte[] shadowCellQualifier(HBaseTransactionManager.ConflictDetectionLevel cdLevel) { + if (cdLevel == ROW) { + return SHARED_FAMILY_QUALIFIER; + } else { + return qualifier; + } + } + + @Test(timeOut = 60_000) - public void testShadowCellsBasics(ITestContext context) throws Exception { + public void testOneShadowCellPerFamily(ITestContext context) throws Exception { + TransactionManager tm = newTransactionManager(context, HBaseTransactionManager.ConflictDetectionLevel.ROW); + TTable table = new TTable(connection, TEST_TABLE); + + HBaseTransaction t1 = (HBaseTransaction) tm.begin(); + Put put = new Put(row); + + for (int i = 0; i < 3; ++i) { + put.addColumn(Bytes.toBytes(TEST_FAMILY), Bytes.toBytes(i), data1); + } + for (int i = 0; i < 3; ++i) { + put.addColumn(Bytes.toBytes(TEST_FAMILY2), Bytes.toBytes(i), data1); + } - TransactionManager tm = newTransactionManager(context); + table.put(t1, put); + + assertFalse(hasShadowCell(row, Bytes.toBytes(TEST_FAMILY), SHARED_FAMILY_QUALIFIER, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + "Shadow cell shouldn't be there"); + assertFalse(hasShadowCell(row, Bytes.toBytes(TEST_FAMILY2), SHARED_FAMILY_QUALIFIER, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + "Shadow cell shouldn't be there"); + + tm.commit(t1); + + assertTrue(hasShadowCell(row, Bytes.toBytes(TEST_FAMILY), SHARED_FAMILY_QUALIFIER, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + "Shadow cell shouldn't be there"); + assertTrue(hasShadowCell(row, Bytes.toBytes(TEST_FAMILY2), SHARED_FAMILY_QUALIFIER, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + "Shadow cell shouldn't be there"); + + int counter = 0; + ResultScanner scanner = table.getHTable().getScanner(new Scan()); + Result res = scanner.next(); + while (res != null) { + for (Cell cell: res.listCells()) { + if (CellUtils.isShadowCell(cell)) { + counter ++; + } + } + res = scanner.next(); + } + + assertEquals(2, counter); + } + + + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testShadowCellsBasics(ITestContext context, HBaseTransactionManager.ConflictDetectionLevel cdLevel) throws Exception { + + TransactionManager tm = newTransactionManager(context,cdLevel); TTable table = new TTable(connection, TEST_TABLE); @@ -102,7 +167,7 @@ public void testShadowCellsBasics(ITestContext context) throws Exception { // Before commit test that only the cell is there assertTrue(hasCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Cell should be there"); - assertFalse(hasShadowCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + assertFalse(hasShadowCell(row, family, shadowCellQualifier(cdLevel), t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Shadow cell shouldn't be there"); tm.commit(t1); @@ -110,7 +175,7 @@ public void testShadowCellsBasics(ITestContext context) throws Exception { // After commit test that both cell and shadow cell are there assertTrue(hasCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Cell should be there"); - assertTrue(hasShadowCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + assertTrue(hasShadowCell(row, family, shadowCellQualifier(cdLevel), t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Shadow cell should be there"); // Test that we can make a valid read after adding a shadow cell without hitting the commit table @@ -121,6 +186,7 @@ public void testShadowCellsBasics(ITestContext context) throws Exception { hbaseOmidClientConf.setHBaseConfiguration(hbaseConf); TransactionManager tm2 = HBaseTransactionManager.builder(hbaseOmidClientConf) .commitTableClient(commitTableClient) + .conflictDetectionLevel(cdLevel) .build(); Transaction t2 = tm2.begin(); @@ -132,8 +198,8 @@ public void testShadowCellsBasics(ITestContext context) throws Exception { verify(commitTableClient, never()).getCommitTimestamp(anyLong()); } - @Test(timeOut = 60_000) - public void testCrashingAfterCommitDoesNotWriteShadowCells(ITestContext context) throws Exception { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testCrashingAfterCommitDoesNotWriteShadowCells(ITestContext context, HBaseTransactionManager.ConflictDetectionLevel cdLevel) throws Exception { CommitTable.Client commitTableClient = spy(getCommitTable(context).getClient()); @@ -146,6 +212,7 @@ public void testCrashingAfterCommitDoesNotWriteShadowCells(ITestContext context) .postCommitter(syncPostCommitter) .commitTableClient(commitTableClient) .commitTableWriter(getCommitTable(context).getWriter()) + .conflictDetectionLevel(cdLevel) .build()); // The following line emulates a crash after commit that is observed in (*) below @@ -168,7 +235,7 @@ public void testCrashingAfterCommitDoesNotWriteShadowCells(ITestContext context) // After commit with the emulated crash, test that only the cell is there assertTrue(hasCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Cell should be there"); - assertFalse(hasShadowCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + assertFalse(hasShadowCell(row, family, shadowCellQualifier(cdLevel), t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Shadow cell should not be there"); Transaction t2 = tm.begin(); @@ -180,8 +247,8 @@ public void testCrashingAfterCommitDoesNotWriteShadowCells(ITestContext context) verify(commitTableClient, times(1)).getCommitTimestamp(anyLong()); } - @Test(timeOut = 60_000) - public void testShadowCellIsHealedAfterCommitCrash(ITestContext context) throws Exception { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testShadowCellIsHealedAfterCommitCrash(ITestContext context, HBaseTransactionManager.ConflictDetectionLevel cdLevel) throws Exception { CommitTable.Client commitTableClient = spy(getCommitTable(context).getClient()); @@ -194,6 +261,7 @@ public void testShadowCellIsHealedAfterCommitCrash(ITestContext context) throws .postCommitter(syncPostCommitter) .commitTableWriter(getCommitTable(context).getWriter()) .commitTableClient(commitTableClient) + .conflictDetectionLevel(cdLevel) .build()); // The following line emulates a crash after commit that is observed in (*) below @@ -215,7 +283,7 @@ public void testShadowCellIsHealedAfterCommitCrash(ITestContext context) throws assertTrue(hasCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Cell should be there"); - assertFalse(hasShadowCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + assertFalse(hasShadowCell(row, family, shadowCellQualifier(cdLevel), t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Shadow cell should not be there"); Transaction t2 = tm.begin(); @@ -229,7 +297,7 @@ public void testShadowCellIsHealedAfterCommitCrash(ITestContext context) throws assertTrue(hasCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Cell should be there"); - assertTrue(hasShadowCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + assertTrue(hasShadowCell(row, family, shadowCellQualifier(cdLevel), t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Shadow cell should be there after being healed"); // As the shadow cell is healed, this get shouldn't have to hit the storage, @@ -240,8 +308,8 @@ public void testShadowCellIsHealedAfterCommitCrash(ITestContext context) throws verify(commitTableClient, times(1)).getCommitTimestamp(anyLong()); } - @Test(timeOut = 60_000) - public void testTransactionNeverCompletesWhenAnExceptionIsThrownUpdatingShadowCells(ITestContext context) + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testTransactionNeverCompletesWhenAnExceptionIsThrownUpdatingShadowCells(ITestContext context, HBaseTransactionManager.ConflictDetectionLevel cdLevel) throws Exception { CommitTable.Client commitTableClient = spy(getCommitTable(context).getClient()); @@ -254,6 +322,7 @@ public void testTransactionNeverCompletesWhenAnExceptionIsThrownUpdatingShadowCe AbstractTransactionManager tm = spy((AbstractTransactionManager) HBaseTransactionManager.builder(hbaseOmidClientConf) .postCommitter(syncPostCommitter) .commitTableClient(commitTableClient) + .conflictDetectionLevel(cdLevel) .commitTableWriter(getCommitTable(context).getWriter()) .build()); @@ -291,7 +360,7 @@ public ListenableFuture answer(InvocationOnMock invocation) throws Throwab // 1) check that shadow cell is not created... assertTrue(hasCell(row, family, qualifier, tx.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Cell should be there"); - assertFalse(hasShadowCell(row, family, qualifier, tx.getStartTimestamp(), new TTableCellGetterAdapter(table)), + assertFalse(hasShadowCell(row, family, shadowCellQualifier(cdLevel), tx.getStartTimestamp(), new TTableCellGetterAdapter(table)), "Shadow cell should not be there"); // 2) and thus, deleteCommitEntry() was never called on the commit table... verify(commitTableClient, times(0)).deleteCommitEntry(anyLong()); @@ -300,8 +369,8 @@ public ListenableFuture answer(InvocationOnMock invocation) throws Throwab } - @Test(timeOut = 60_000) - public void testRaceConditionBetweenReaderAndWriterThreads(final ITestContext context) throws Exception { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testRaceConditionBetweenReaderAndWriterThreads(final ITestContext context, final HBaseTransactionManager.ConflictDetectionLevel cdLevel) throws Exception { final CountDownLatch readAfterCommit = new CountDownLatch(1); final CountDownLatch postCommitBegin = new CountDownLatch(1); final CountDownLatch postCommitEnd = new CountDownLatch(1); @@ -309,7 +378,7 @@ public void testRaceConditionBetweenReaderAndWriterThreads(final ITestContext co final AtomicBoolean readFailed = new AtomicBoolean(false); PostCommitActions syncPostCommitter = spy(new HBaseSyncPostCommitter(new NullMetricsProvider(), getCommitTable(context).getClient())); - AbstractTransactionManager tm = (AbstractTransactionManager) newTransactionManager(context, syncPostCommitter); + AbstractTransactionManager tm = (AbstractTransactionManager) newTransactionManager(context, syncPostCommitter,cdLevel); doAnswer(new Answer>() { @Override @@ -355,10 +424,10 @@ public List answer(InvocationOnMock invocation) throws Throwable { }).when(snapshotFilter).filterCellsForSnapshot(Matchers.>any(), any(HBaseTransaction.class), anyInt(), Matchers.>any(), Matchers.>any()); - TransactionManager tm = newTransactionManager(context); + TransactionManager tm = newTransactionManager(context,cdLevel); if (hasShadowCell(row, family, - qualifier, + shadowCellQualifier(cdLevel), t1.getStartTimestamp(), new TTableCellGetterAdapter(table))) { readFailed.set(true); @@ -373,7 +442,7 @@ public List answer(InvocationOnMock invocation) throws Throwable { if (!Arrays.equals(data1, CellUtil.cloneValue(cell)) || !hasShadowCell(row, family, - qualifier, + shadowCellQualifier(cdLevel), cell.getTimestamp(), new TTableCellGetterAdapter(table))) { readFailed.set(true); @@ -406,10 +475,10 @@ public List answer(InvocationOnMock invocation) throws Throwable { /** * Test that the new client can read shadow cells written by the old client. */ - @Test(timeOut = 60_000) - public void testGetOldShadowCells(ITestContext context) throws Exception { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testGetOldShadowCells(ITestContext context, HBaseTransactionManager.ConflictDetectionLevel cdLevel) throws Exception { - TransactionManager tm = newTransactionManager(context); + TransactionManager tm = newTransactionManager(context,cdLevel); TTable table = new TTable(connection, TEST_TABLE); Table htable = table.getHTable(); @@ -445,7 +514,7 @@ public void testGetOldShadowCells(ITestContext context) throws Exception { // delete new shadow cell Delete del = new Delete(row2); - del.addColumn(family, CellUtils.addShadowCellSuffixPrefix(qualifier)); + del.addColumn(family, CellUtils.addShadowCellSuffixPrefix(shadowCellQualifier(cdLevel))); htable.delete(del); table.flushCommits(); @@ -474,7 +543,7 @@ public void testGetOldShadowCells(ITestContext context) throws Exception { // now add in the previous legacy shadow cell for that row put = new Put(row2); put.addColumn(family, - addLegacyShadowCellSuffix(qualifier), + addLegacyShadowCellSuffix(shadowCellQualifier(cdLevel)), t2.getStartTimestamp(), Bytes.toBytes(t2.getCommitTimestamp())); htable.put(put); @@ -503,6 +572,147 @@ public void testGetOldShadowCells(ITestContext context) throws Exception { assertTrue(result2.containsColumn(family, qualifier), "Should have column family"); } + + @Test(timeOut = 30_000, dataProvider = "cflevel") + public void testResusedShadowCellsAfterHealing(ITestContext context, + HBaseTransactionManager.ConflictDetectionLevel cdLevel) throws IOException, InterruptedException { + CommitTable.Client commitTableClient = spy(getCommitTable(context).getClient()); + + HBaseOmidClientConfiguration hbaseOmidClientConf = new HBaseOmidClientConfiguration(); + hbaseOmidClientConf.setConnectionString(TSO_SERVER_HOST + ":" + TSO_SERVER_PORT); + hbaseOmidClientConf.setHBaseConfiguration(hbaseConf); + PostCommitActions syncPostCommitter = spy( + new HBaseSyncPostCommitter(new NullMetricsProvider(), commitTableClient)); + AbstractTransactionManager tm = spy((AbstractTransactionManager) HBaseTransactionManager.builder(hbaseOmidClientConf) + .postCommitter(syncPostCommitter) + .commitTableWriter(getCommitTable(context).getWriter()) + .commitTableClient(commitTableClient) + .conflictDetectionLevel(cdLevel) + .build()); + + // The following line emulates a crash after commit that is observed in (*) below + doThrow(new RuntimeException()).when(syncPostCommitter).updateShadowCells(any(HBaseTransaction.class)); + + TTable table = new TTable(connection, TEST_TABLE); + + HBaseTransaction t1 = (HBaseTransaction) tm.begin(); + + // Test shadow cell are created properly + byte[] qualifier2 = Bytes.toBytes("qual2"); + byte[] family2 = Bytes.toBytes(TEST_FAMILY2); + + + Put put = new Put(row); + put.addColumn(family, qualifier, data1); + put.addColumn(family2, qualifier2 , data1); + + table.put(t1, put); + try { + tm.commit(t1); + } catch (Exception e) { // (*) Crash + // Do nothing + } + + assertTrue(hasCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + "Cell should be there"); + assertTrue(hasCell(row, family2, qualifier2, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + "Cell should be there"); + + assertFalse(hasShadowCell(row, family, shadowCellQualifier(cdLevel), t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + "Shadow cell should not be there"); + assertFalse(hasShadowCell(row, family2, shadowCellQualifier(cdLevel), t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + "Shadow cell should not be there"); + + Transaction t2 = tm.begin(); + Get get = new Get(row); + get.addColumn(family, qualifier); + get.addColumn(family2, qualifier2); + + // This get should heal the shadow cell, and use it in commitCache for other cell reads instead of jumping to commit table + Result getResult = table.get(t2, get); + assertTrue(Arrays.equals(data1, getResult.getValue(family, qualifier)), "Values should be the same"); + verify(commitTableClient, times(1)).getCommitTimestamp(anyLong()); + + assertTrue(hasCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + "Cell should be there"); + //only 1 shadow cell will be healed, the others cells got read from commitCache + int count=0; + + count += hasShadowCell(row, family, shadowCellQualifier(cdLevel), t1.getStartTimestamp(), + new TTableCellGetterAdapter(table))?1:0; + count += hasShadowCell(row, family2, shadowCellQualifier(cdLevel), t1.getStartTimestamp(), + new TTableCellGetterAdapter(table))?1:0; + + assertEquals(1,count); + + } + + + @Test(timeOut = 30_0000000, dataProvider = "cflevel") + public void testStackedFamilyShadowCellsRetryAndNotCommitTable(ITestContext context, + HBaseTransactionManager.ConflictDetectionLevel cdLevel) throws Exception { + + TransactionManager tm = newTransactionManager(context,cdLevel); + + TTable table = new TTable(connection, TEST_TABLE); + + //create stack of shadow cells + HBaseTransaction t1 = (HBaseTransaction) tm.begin(); + Put put = new Put(row); + put.addColumn(family, qualifier, data1); + table.put(t1, put); + tm.commit(t1); + + HBaseTransaction t2 = (HBaseTransaction) tm.begin(); + put = new Put(row); + put.addColumn(family, qualifier, data1); + table.put(t2, put); + tm.commit(t2); + + HBaseTransaction t3 = (HBaseTransaction) tm.begin(); + put = new Put(row); + put.addColumn(family, qualifier, data1); + table.put(t3, put); + tm.commit(t3); + + + HBaseTransaction t4 = (HBaseTransaction) tm.begin(); + Put put2 = new Put(row); + byte[] qualifier2 = Bytes.toBytes("qual2"); + put2.addColumn(family, qualifier2, data1); + table.put(t4, put2); + tm.commit(t4); + + + // After commit test that both cell and shadow cell are there + assertTrue(hasCell(row, family, qualifier, t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + "Cell should be there"); + assertTrue(hasShadowCell(row, family, shadowCellQualifier(cdLevel), t1.getStartTimestamp(), new TTableCellGetterAdapter(table)), + "Shadow cell should be there"); + + // Test that we can make a valid read after adding a shadow cell without hitting the commit table + CommitTable.Client commitTableClient = spy(getCommitTable(context).getClient()); + + HBaseOmidClientConfiguration hbaseOmidClientConf = new HBaseOmidClientConfiguration(); + hbaseOmidClientConf.setConnectionString(TSO_SERVER_HOST + ":" + TSO_SERVER_PORT); + hbaseOmidClientConf.setHBaseConfiguration(hbaseConf); + TransactionManager tm2 = HBaseTransactionManager.builder(hbaseOmidClientConf) + .commitTableClient(commitTableClient) + .conflictDetectionLevel(cdLevel) + .build(); + + Transaction t5 = tm2.begin(); + Get get = new Get(row); + get.addColumn(family, qualifier); + get.addColumn(family, qualifier2); + + Result getResult = table.get(t5, get); + assertTrue(Arrays.equals(data1, getResult.getValue(family, qualifier)), "Values should be the same"); + verify(commitTableClient, never()).getCommitTimestamp(anyLong()); + } + + + // ---------------------------------------------------------------------------------------------------------------- // Helper methods // ---------------------------------------------------------------------------------------------------------------- diff --git a/hbase-client/src/test/java/org/apache/omid/transaction/TestTransactionCleanup.java b/hbase-client/src/test/java/org/apache/omid/transaction/TestTransactionCleanup.java index acc74afd7..6ec6707da 100644 --- a/hbase-client/src/test/java/org/apache/omid/transaction/TestTransactionCleanup.java +++ b/hbase-client/src/test/java/org/apache/omid/transaction/TestTransactionCleanup.java @@ -35,6 +35,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.ITestContext; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import com.google.common.util.concurrent.SettableFuture; @@ -51,14 +52,24 @@ public class TestTransactionCleanup extends OmidTestBase { private byte[] qual = Bytes.toBytes("qual"); private byte[] data = Bytes.toBytes("data"); + + @DataProvider(name = "cflevel") + public static Object[][] cfLevels() { + return new Object[][] { + {HBaseTransactionManager.ConflictDetectionLevel.CELL}, + {HBaseTransactionManager.ConflictDetectionLevel.ROW}}; + } + // NOTE: This test is maybe redundant with runTestCleanupAfterConflict() // and testCleanupWithDeleteRow() tests in TestTransactionCleanup class. // Code in TestTransactionCleanup is a little more difficult to follow, // lacks some assertions and includes some magic numbers, so we should // try to review and improve the tests in these two classes in a further // commit. - @Test(timeOut = 10_000) - public void testTransactionIsCleanedUpAfterBeingAborted(ITestContext context) throws Exception { + @Test(timeOut = 10_000, dataProvider = "cflevel") + public void testTransactionIsCleanedUpAfterBeingAborted(ITestContext context, + HBaseTransactionManager.ConflictDetectionLevel cdLevel) + throws Exception { final int ROWS_MODIFIED = 1; @@ -80,7 +91,7 @@ public void testTransactionIsCleanedUpAfterBeingAborted(ITestContext context) th doReturn(abortingFF) .when(mockedTSOClient).commit(eq(START_TS), anySetOf(HBaseCellId.class), anySetOf(HBaseCellId.class)); - try (TransactionManager tm = newTransactionManager(context, mockedTSOClient); + try (TransactionManager tm = newTransactionManager(context, mockedTSOClient,cdLevel); TTable txTable = new TTable(connection, TEST_TABLE)) { // Start a transaction and put some data in a column diff --git a/hbase-common/src/main/java/org/apache/omid/transaction/CellUtils.java b/hbase-common/src/main/java/org/apache/omid/transaction/CellUtils.java index 019ab74d7..5b34701bd 100644 --- a/hbase-common/src/main/java/org/apache/omid/transaction/CellUtils.java +++ b/hbase-common/src/main/java/org/apache/omid/transaction/CellUtils.java @@ -55,9 +55,13 @@ public final class CellUtils { static byte[] LEGACY_DELETE_TOMBSTONE = Bytes.toBytes("__OMID_TOMBSTONE__"); public static final byte[] FAMILY_DELETE_QUALIFIER = HConstants.EMPTY_BYTE_ARRAY; public static final String TRANSACTION_ATTRIBUTE = "__OMID_TRANSACTION__"; + + // Used as the shared qualifier the shadow cell will be attached to. in case of + public static final byte[] SHARED_FAMILY_QUALIFIER = "\u0090".getBytes(Charsets.UTF_8); /**/ public static final String CLIENT_GET_ATTRIBUTE = "__OMID_CLIENT_GET__"; public static final String LL_ATTRIBUTE = "__OMID_LL__"; + public static final String ROW_LEVEL_CONFLICTS_ATTRIBUTE = "__OMID_ROWCF__"; /** * Utility interface to get rid of the dependency on HBase server package diff --git a/hbase-coprocessor/src/main/java/org/apache/omid/transaction/CompactorScanner.java b/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/CompactorScanner.java similarity index 94% rename from hbase-coprocessor/src/main/java/org/apache/omid/transaction/CompactorScanner.java rename to hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/CompactorScanner.java index cf9316376..513c83f40 100644 --- a/hbase-coprocessor/src/main/java/org/apache/omid/transaction/CompactorScanner.java +++ b/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/CompactorScanner.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.omid.transaction.HBaseTransactionManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,6 +50,7 @@ import java.util.concurrent.ExecutionException; import static org.apache.omid.committable.CommitTable.CommitTimestamp.Location.SHADOW_CELL; +import static org.apache.omid.transaction.HBaseTransactionManager.ConflictDetectionLevel.ROW; public class CompactorScanner implements InternalScanner { private static final Logger LOG = LoggerFactory.getLogger(CompactorScanner.class); @@ -60,6 +62,7 @@ public class CompactorScanner implements InternalScanner { private final long lowWatermark; private final Region hRegion; + private final HBaseTransactionManager.ConflictDetectionLevel cdLevel; private boolean hasMoreRows = false; private List currentRowWorthValues = new ArrayList(); @@ -69,7 +72,8 @@ public CompactorScanner(ObserverContext e, Client commitTableClient, Queue commitTableClientQueue, boolean isMajorCompaction, - boolean preserveNonTransactionallyDeletedCells) throws IOException { + boolean preserveNonTransactionallyDeletedCells, + HBaseTransactionManager.ConflictDetectionLevel cdLevel) throws IOException { this.internalScanner = internalScanner; this.commitTableClient = commitTableClient; this.commitTableClientQueue = commitTableClientQueue; @@ -78,6 +82,7 @@ public CompactorScanner(ObserverContext e, this.lowWatermark = getLowWatermarkFromCommitTable(); // Obtain the table in which the scanner is going to operate this.hRegion = HBaseShims.getRegionCoprocessorRegion(e.getEnvironment()); + this.cdLevel = cdLevel; LOG.info("Scanner cleaning up uncommitted txs older than LW [{}] in region [{}]", lowWatermark, hRegion.getRegionInfo()); } @@ -235,9 +240,15 @@ private Optional queryCommitTimestamp(Cell cell) throws IOExcep } else { Get g = new Get(CellUtil.cloneRow(cell)); byte[] family = CellUtil.cloneFamily(cell); - byte[] qualifier = CellUtils.addShadowCellSuffixPrefix(cell.getQualifierArray(), - cell.getQualifierOffset(), - cell.getQualifierLength()); + byte[] qualifier; + if (cdLevel == ROW) { + qualifier = CellUtils.addShadowCellSuffixPrefix(CellUtils.SHARED_FAMILY_QUALIFIER); + } else { + qualifier = CellUtils.addShadowCellSuffixPrefix(cell.getQualifierArray(), + cell.getQualifierOffset(), + cell.getQualifierLength()); + } + g.addColumn(family, qualifier); g.setTimeStamp(cell.getTimestamp()); Result r = hRegion.get(g); diff --git a/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidCompactor.java b/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidCompactor.java index f8ed6b7bf..b0af62142 100644 --- a/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidCompactor.java +++ b/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidCompactor.java @@ -44,6 +44,8 @@ import java.util.concurrent.ConcurrentLinkedQueue; import static org.apache.omid.committable.hbase.HBaseCommitTableConfig.COMMIT_TABLE_NAME_KEY; +import static org.apache.omid.transaction.HBaseTransactionManager.ConflictDetectionLevel.CELL; +import static org.apache.omid.transaction.HBaseTransactionManager.ConflictDetectionLevel.ROW; /** * Garbage collector for stale data: triggered upon HBase @@ -59,6 +61,7 @@ public class OmidCompactor extends BaseRegionObserver { private static final boolean HBASE_RETAIN_NON_TRANSACTIONALLY_DELETED_CELLS_DEFAULT = true; final static String OMID_COMPACTABLE_CF_FLAG = "OMID_ENABLED"; + private HBaseTransactionManager.ConflictDetectionLevel cdLevel; private boolean enableCompactorForAllFamilies = false; @@ -76,12 +79,15 @@ public class OmidCompactor extends BaseRegionObserver { private CommitTable commitTable; public OmidCompactor() { - this(false); + this(false, + CELL); } - public OmidCompactor(boolean enableCompactorForAllFamilies) { + public OmidCompactor(boolean enableCompactorForAllFamilies, + HBaseTransactionManager.ConflictDetectionLevel cdLevel) { LOG.info("Compactor coprocessor initialized"); this.enableCompactorForAllFamilies = enableCompactorForAllFamilies; + this.cdLevel = cdLevel; } @Override @@ -89,6 +95,15 @@ public void start(CoprocessorEnvironment env) throws IOException { LOG.info("Starting compactor coprocessor"); commitTableConf = new HBaseCommitTableConfig(); String commitTableName = env.getConfiguration().get(COMMIT_TABLE_NAME_KEY); + String conflictDetectionLevel = env.getConfiguration().get("omid.conflictdetection.level"); + if (conflictDetectionLevel != null) { + if (conflictDetectionLevel.equals(ROW.toString())) { + cdLevel = ROW; + } else { + cdLevel = CELL; + } + } + if (commitTableName != null) { commitTableConf.setTableName(commitTableName); } @@ -145,7 +160,8 @@ public InternalScanner preCompact(ObserverContext commitTableClient, commitTableClientQueue, isMajorCompaction, - retainNonTransactionallyDeletedCells); + retainNonTransactionallyDeletedCells, + cdLevel); } } catch (IOException e) { throw e; diff --git a/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidSnapshotFilter.java b/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidSnapshotFilter.java index eb5d50f6d..06ddff693 100644 --- a/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidSnapshotFilter.java +++ b/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidSnapshotFilter.java @@ -113,8 +113,11 @@ public void preGetOp(ObserverContext e, Get get, L if (get.getAttribute(CellUtils.CLIENT_GET_ATTRIBUTE) == null) return; boolean isLowLatency = Bytes.toBoolean(get.getAttribute(CellUtils.LL_ATTRIBUTE)); + boolean isRowCD = Bytes.toBoolean(get.getAttribute(CellUtils.ROW_LEVEL_CONFLICTS_ATTRIBUTE)); + HBaseTransactionManager.ConflictDetectionLevel cdLevel = isRowCD? HBaseTransactionManager.ConflictDetectionLevel.ROW: + HBaseTransactionManager.ConflictDetectionLevel.CELL; HBaseTransaction hbaseTransaction = getHBaseTransaction(get.getAttribute(CellUtils.TRANSACTION_ATTRIBUTE), - isLowLatency); + isLowLatency, cdLevel); SnapshotFilterImpl snapshotFilter = getSnapshotFilter(e); snapshotFilterMap.put(get, snapshotFilter); @@ -154,7 +157,10 @@ public void preScannerOpen(ObserverContext e, return; } boolean isLowLatency = Bytes.toBoolean(scan.getAttribute(CellUtils.LL_ATTRIBUTE)); - HBaseTransaction hbaseTransaction = getHBaseTransaction(byteTransaction, isLowLatency); + boolean isRowCD = Bytes.toBoolean(scan.getAttribute(CellUtils.ROW_LEVEL_CONFLICTS_ATTRIBUTE)); + HBaseTransactionManager.ConflictDetectionLevel cdLevel = isRowCD? HBaseTransactionManager.ConflictDetectionLevel.ROW: + HBaseTransactionManager.ConflictDetectionLevel.CELL; + HBaseTransaction hbaseTransaction = getHBaseTransaction(byteTransaction, isLowLatency,cdLevel); SnapshotFilterImpl snapshotFilter = getSnapshotFilter(e); scan.setMaxVersions(); @@ -164,7 +170,7 @@ public void preScannerOpen(ObserverContext e, return; } - private HBaseTransaction getHBaseTransaction(byte[] byteTransaction, boolean isLowLatency) + private HBaseTransaction getHBaseTransaction(byte[] byteTransaction, boolean isLowLatency, HBaseTransactionManager.ConflictDetectionLevel cdLevel) throws InvalidProtocolBufferException { TSOProto.Transaction transaction = TSOProto.Transaction.parseFrom(byteTransaction); long id = transaction.getTimestamp(); @@ -172,7 +178,7 @@ private HBaseTransaction getHBaseTransaction(byte[] byteTransaction, boolean isL long epoch = transaction.getEpoch(); VisibilityLevel visibilityLevel = VisibilityLevel.fromInteger(transaction.getVisibilityLevel()); - return new HBaseTransaction(id, readTs, visibilityLevel, epoch, new HashSet(), new HashSet(), null, + return new HBaseTransaction(id, readTs, visibilityLevel, epoch, new HashSet(), new HashSet(), cdLevel, isLowLatency); } diff --git a/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompaction.java b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompaction.java index a4bf65e5a..2661811d3 100644 --- a/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompaction.java +++ b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompaction.java @@ -672,7 +672,7 @@ public Void answer(InvocationOnMock invocation) } }).when(failableHTable).flushCommits(); - newWriteSet.add(new HBaseCellId(failableHTable, + newWriteSet.add(HBaseCellId.valueOf(tx3,failableHTable, id.getRow(), id.getFamily(), id.getQualifier(), id.getTimestamp())); } diff --git a/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactionRowCD.java b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactionRowCD.java new file mode 100644 index 000000000..59947d956 --- /dev/null +++ b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactionRowCD.java @@ -0,0 +1,1204 @@ +/* + * 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.omid.transaction; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +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.RetriesExhaustedWithDetailsException; +import org.apache.hadoop.hbase.client.Row; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.omid.HBaseShims; +import org.apache.omid.TestUtils; +import org.apache.omid.committable.CommitTable; +import org.apache.omid.committable.hbase.HBaseCommitTableConfig; +import org.apache.omid.metrics.NullMetricsProvider; +import org.apache.omid.timestamp.storage.HBaseTimestampStorageConfig; +import org.apache.omid.tso.TSOServer; +import org.apache.omid.tso.TSOServerConfig; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import com.google.common.util.concurrent.SettableFuture; +import com.google.inject.Guice; +import com.google.inject.Injector; + +public class TestCompactionRowCD { + + private static final Logger LOG = LoggerFactory.getLogger(TestCompaction.class); + + private static final String TEST_FAMILY = "test-fam"; + private static final String TEST_QUALIFIER = "test-qual"; + + private final byte[] fam = Bytes.toBytes(TEST_FAMILY); + private final byte[] qual = Bytes.toBytes(TEST_QUALIFIER); + private final byte[] data = Bytes.toBytes("testWrite-1"); + + private static final int MAX_VERSIONS = 3; + + private Random randomGenerator; + private AbstractTransactionManager tm; + + private Injector injector; + + private Admin admin; + private Configuration hbaseConf; + private HBaseTestingUtility hbaseTestUtil; + private MiniHBaseCluster hbaseCluster; + + private TSOServer tso; + + + private CommitTable commitTable; + private PostCommitActions syncPostCommitter; + private static Connection connection; + + @BeforeClass + public void setupTestCompation() throws Exception { + TSOServerConfig tsoConfig = new TSOServerConfig(); + tsoConfig.setPort(1234); + tsoConfig.setConflictMapSize(1); + tsoConfig.setWaitStrategy("LOW_CPU"); + injector = Guice.createInjector(new TSOForHBaseCompactorTestModule(tsoConfig)); + hbaseConf = injector.getInstance(Configuration.class); + HBaseCommitTableConfig hBaseCommitTableConfig = injector.getInstance(HBaseCommitTableConfig.class); + HBaseTimestampStorageConfig hBaseTimestampStorageConfig = injector.getInstance(HBaseTimestampStorageConfig.class); + + // settings required for #testDuplicateDeletes() + hbaseConf.setInt("hbase.hstore.compaction.min", 2); + hbaseConf.setInt("hbase.hstore.compaction.max", 2); + hbaseConf.setStrings("omid.conflictdetection.level", "ROW"); + setupHBase(); + connection = ConnectionFactory.createConnection(hbaseConf); + admin = connection.getAdmin(); + createRequiredHBaseTables(hBaseTimestampStorageConfig, hBaseCommitTableConfig); + setupTSO(); + + commitTable = injector.getInstance(CommitTable.class); + } + + private void setupHBase() throws Exception { + LOG.info("--------------------------------------------------------------------------------------------------"); + LOG.info("Setting up HBase"); + LOG.info("--------------------------------------------------------------------------------------------------"); + hbaseTestUtil = new HBaseTestingUtility(hbaseConf); + LOG.info("--------------------------------------------------------------------------------------------------"); + LOG.info("Creating HBase MiniCluster"); + LOG.info("--------------------------------------------------------------------------------------------------"); + hbaseCluster = hbaseTestUtil.startMiniCluster(1); + } + + private void createRequiredHBaseTables(HBaseTimestampStorageConfig timestampStorageConfig, + HBaseCommitTableConfig hBaseCommitTableConfig) throws IOException { + createTableIfNotExists(timestampStorageConfig.getTableName(), timestampStorageConfig.getFamilyName().getBytes()); + + createTableIfNotExists(hBaseCommitTableConfig.getTableName(), hBaseCommitTableConfig.getCommitTableFamily(), hBaseCommitTableConfig.getLowWatermarkFamily()); + } + + private void createTableIfNotExists(String tableName, byte[]... families) throws IOException { + if (!admin.tableExists(TableName.valueOf(tableName))) { + LOG.info("Creating {} table...", tableName); + HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); + + for (byte[] family : families) { + HColumnDescriptor datafam = new HColumnDescriptor(family); + datafam.setMaxVersions(MAX_VERSIONS); + desc.addFamily(datafam); + } + + desc.addCoprocessor("org.apache.hadoop.hbase.coprocessor.AggregateImplementation",null,Coprocessor.PRIORITY_HIGHEST,null); + admin.createTable(desc); + for (byte[] family : families) { + CompactorUtil.enableOmidCompaction(connection, TableName.valueOf(tableName), family); + } + } + + } + + private void setupTSO() throws IOException, InterruptedException { + tso = injector.getInstance(TSOServer.class); + tso.startAndWait(); + TestUtils.waitForSocketListening("localhost", 1234, 100); + Thread.currentThread().setName("UnitTest(s) thread"); + } + + @AfterClass + public void cleanupTestCompation() throws Exception { + teardownTSO(); + hbaseCluster.shutdown(); + } + + private void teardownTSO() throws IOException, InterruptedException { + tso.stopAndWait(); + TestUtils.waitForSocketNotListening("localhost", 1234, 1000); + } + + @BeforeMethod + public void setupTestCompactionIndividualTest() throws Exception { + randomGenerator = new Random(0xfeedcafeL); + tm = spy((AbstractTransactionManager) newTransactionManager()); + } + + private TransactionManager newTransactionManager() throws Exception { + HBaseOmidClientConfiguration hbaseOmidClientConf = new HBaseOmidClientConfiguration(); + hbaseOmidClientConf.setConnectionString("localhost:1234"); + hbaseOmidClientConf.setHBaseConfiguration(hbaseConf); + CommitTable.Client commitTableClient = commitTable.getClient(); + syncPostCommitter = + spy(new HBaseSyncPostCommitter(new NullMetricsProvider(),commitTableClient)); + return HBaseTransactionManager.builder(hbaseOmidClientConf) + .postCommitter(syncPostCommitter) + .commitTableClient(commitTableClient) + .conflictDetectionLevel(HBaseTransactionManager.ConflictDetectionLevel.ROW) + .build(); + } + + @Test(timeOut = 60_000) + public void testStandardTXsWithShadowCellsAndWithSTBelowAndAboveLWMArePresevedAfterCompaction() throws Throwable { + String TEST_TABLE = "testStandardTXsWithShadowCellsAndWithSTBelowAndAboveLWMArePresevedAfterCompaction"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + final int ROWS_TO_ADD = 5; + + long fakeAssignedLowWatermark = 0L; + for (int i = 0; i < ROWS_TO_ADD; ++i) { + long rowId = randomGenerator.nextLong(); + Transaction tx = tm.begin(); + if (i == (ROWS_TO_ADD / 2)) { + fakeAssignedLowWatermark = tx.getTransactionId(); + LOG.info("AssignedLowWatermark " + fakeAssignedLowWatermark); + } + Put put = new Put(Bytes.toBytes(rowId)); + put.addColumn(fam, qual, data); + txTable.put(tx, put); + tm.commit(tx); + } + + LOG.info("Flushing table {}", TEST_TABLE); + admin.flush(TableName.valueOf(TEST_TABLE)); + + // Return a LWM that triggers compaction & stays between 1 and the max start timestamp assigned to previous TXs + LOG.info("Regions in table {}: {}", TEST_TABLE, hbaseCluster.getRegions(Bytes.toBytes(TEST_TABLE)).size()); + OmidCompactor omidCompactor = (OmidCompactor) hbaseCluster.getRegions(Bytes.toBytes(TEST_TABLE)).get(0) + .getCoprocessorHost().findCoprocessor(OmidCompactor.class.getName()); + CommitTable commitTable = injector.getInstance(CommitTable.class); + CommitTable.Client commitTableClient = spy(commitTable.getClient()); + SettableFuture f = SettableFuture.create(); + f.set(fakeAssignedLowWatermark); + doReturn(f).when(commitTableClient).readLowWatermark(); + omidCompactor.commitTableClientQueue.add(commitTableClient); + LOG.info("Compacting table {}", TEST_TABLE); + admin.majorCompact(TableName.valueOf(TEST_TABLE)); + + LOG.info("Sleeping for 3 secs"); + Thread.sleep(3000); + LOG.info("Waking up after 3 secs"); + + // No rows should have been discarded after compacting + assertEquals(rowCount(TEST_TABLE, fam), ROWS_TO_ADD, "Rows in table after compacting should be " + ROWS_TO_ADD); + } + + @Test(timeOut = 60_000) + public void testTXWithoutShadowCellsAndWithSTBelowLWMGetsShadowCellHealedAfterCompaction() throws Exception { + String TEST_TABLE = "testTXWithoutShadowCellsAndWithSTBelowLWMGetsShadowCellHealedAfterCompaction"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + // The following line emulates a crash after commit that is observed in (*) below + doThrow(new RuntimeException()).when(syncPostCommitter).updateShadowCells(any(HBaseTransaction.class)); + + HBaseTransaction problematicTx = (HBaseTransaction) tm.begin(); + + long row = randomGenerator.nextLong(); + + // Test shadow cell are created properly + Put put = new Put(Bytes.toBytes(row)); + put.addColumn(fam, qual, data); + txTable.put(problematicTx, put); + try { + tm.commit(problematicTx); + } catch (Exception e) { // (*) Crash + // Do nothing + } + + assertTrue(CellUtils.hasCell(Bytes.toBytes(row), fam, qual, problematicTx.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Cell should be there"); + assertFalse(CellUtils.hasShadowCell(Bytes.toBytes(row), fam, qual, problematicTx.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Shadow cell should not be there"); + + // Return a LWM that triggers compaction and has all the possible start timestamps below it + LOG.info("Regions in table {}: {}", TEST_TABLE, hbaseCluster.getRegions(Bytes.toBytes(TEST_TABLE)).size()); + OmidCompactor omidCompactor = (OmidCompactor) hbaseCluster.getRegions(Bytes.toBytes(TEST_TABLE)).get(0) + .getCoprocessorHost().findCoprocessor(OmidCompactor.class.getName()); + CommitTable commitTable = injector.getInstance(CommitTable.class); + CommitTable.Client commitTableClient = spy(commitTable.getClient()); + SettableFuture f = SettableFuture.create(); + f.set(Long.MAX_VALUE); + doReturn(f).when(commitTableClient).readLowWatermark(); + omidCompactor.commitTableClientQueue.add(commitTableClient); + + LOG.info("Flushing table {}", TEST_TABLE); + admin.flush(TableName.valueOf(TEST_TABLE)); + + LOG.info("Compacting table {}", TEST_TABLE); + admin.majorCompact(TableName.valueOf(TEST_TABLE)); + + LOG.info("Sleeping for 3 secs"); + Thread.sleep(3000); + LOG.info("Waking up after 3 secs"); + + assertTrue(CellUtils.hasCell(Bytes.toBytes(row), fam, qual, problematicTx.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Cell should be there"); + assertTrue(CellUtils.hasShadowCell(Bytes.toBytes(row), fam, qual, problematicTx.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Shadow cell should not be there"); + } + + @Test(timeOut = 60_000) + public void testNeverendingTXsWithSTBelowAndAboveLWMAreDiscardedAndPreservedRespectivelyAfterCompaction() + throws Throwable { + String + TEST_TABLE = + "testNeverendingTXsWithSTBelowAndAboveLWMAreDiscardedAndPreservedRespectivelyAfterCompaction"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + // The KV in this transaction should be discarded + HBaseTransaction neverendingTxBelowLowWatermark = (HBaseTransaction) tm.begin(); + long rowId = randomGenerator.nextLong(); + Put put = new Put(Bytes.toBytes(rowId)); + put.addColumn(fam, qual, data); + txTable.put(neverendingTxBelowLowWatermark, put); + assertTrue(CellUtils.hasCell(Bytes.toBytes(rowId), fam, qual, neverendingTxBelowLowWatermark.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Cell should be there"); + assertFalse(CellUtils.hasShadowCell(Bytes.toBytes(rowId), fam, qual, neverendingTxBelowLowWatermark.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Shadow cell should not be there"); + + // The KV in this transaction should be added without the shadow cells + HBaseTransaction neverendingTxAboveLowWatermark = (HBaseTransaction) tm.begin(); + long anotherRowId = randomGenerator.nextLong(); + put = new Put(Bytes.toBytes(anotherRowId)); + put.addColumn(fam, qual, data); + txTable.put(neverendingTxAboveLowWatermark, put); + assertTrue(CellUtils.hasCell(Bytes.toBytes(anotherRowId), fam, qual, neverendingTxAboveLowWatermark.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Cell should be there"); + assertFalse(CellUtils.hasShadowCell(Bytes.toBytes(anotherRowId), fam, qual, neverendingTxAboveLowWatermark.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Shadow cell should not be there"); + + assertEquals(rowCount(TEST_TABLE, fam), 2, "Rows in table before flushing should be 2"); + LOG.info("Flushing table {}", TEST_TABLE); + admin.flush(TableName.valueOf(TEST_TABLE)); + assertEquals(rowCount(TEST_TABLE, fam), 2, "Rows in table after flushing should be 2"); + + // Return a LWM that triggers compaction and stays between both ST of TXs, so assign 1st TX's start timestamp + LOG.info("Regions in table {}: {}", TEST_TABLE, hbaseCluster.getRegions(Bytes.toBytes(TEST_TABLE)).size()); + OmidCompactor omidCompactor = (OmidCompactor) hbaseCluster.getRegions(Bytes.toBytes(TEST_TABLE)).get(0) + .getCoprocessorHost().findCoprocessor(OmidCompactor.class.getName()); + CommitTable commitTable = injector.getInstance(CommitTable.class); + CommitTable.Client commitTableClient = spy(commitTable.getClient()); + SettableFuture f = SettableFuture.create(); + f.set(neverendingTxBelowLowWatermark.getStartTimestamp()); + doReturn(f).when(commitTableClient).readLowWatermark(); + omidCompactor.commitTableClientQueue.add(commitTableClient); + LOG.info("Compacting table {}", TEST_TABLE); + admin.majorCompact(TableName.valueOf(TEST_TABLE)); + + LOG.info("Sleeping for 3 secs"); + Thread.sleep(3000); + LOG.info("Waking up after 3 secs"); + + // One row should have been discarded after compacting + assertEquals(rowCount(TEST_TABLE, fam), 1, "There should be only one row in table after compacting"); + // The row from the TX below the LWM should not be there (nor its Shadow Cell) + assertFalse(CellUtils.hasCell(Bytes.toBytes(rowId), fam, qual, neverendingTxBelowLowWatermark.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Cell should not be there"); + assertFalse(CellUtils.hasShadowCell(Bytes.toBytes(rowId), fam, qual, neverendingTxBelowLowWatermark.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Shadow cell should not be there"); + // The row from the TX above the LWM should be there without the Shadow Cell + assertTrue(CellUtils.hasCell(Bytes.toBytes(anotherRowId), fam, qual, neverendingTxAboveLowWatermark.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Cell should be there"); + assertFalse(CellUtils.hasShadowCell(Bytes.toBytes(anotherRowId), fam, qual, neverendingTxAboveLowWatermark.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Shadow cell should not be there"); + + } + + @Test(timeOut = 60_000) + public void testRowsUnalteredWhenCommitTableCannotBeReached() throws Throwable { + String TEST_TABLE = "testRowsUnalteredWhenCommitTableCannotBeReached"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + // The KV in this transaction should be discarded but in the end should remain there because + // the commit table won't be accessed (simulating an error on access) + HBaseTransaction neverendingTx = (HBaseTransaction) tm.begin(); + long rowId = randomGenerator.nextLong(); + Put put = new Put(Bytes.toBytes(rowId)); + put.addColumn(fam, qual, data); + txTable.put(neverendingTx, put); + assertTrue(CellUtils.hasCell(Bytes.toBytes(rowId), fam, qual, neverendingTx.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Cell should be there"); + assertFalse(CellUtils.hasShadowCell(Bytes.toBytes(rowId), fam, qual, neverendingTx.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Shadow cell should not be there"); + + assertEquals(rowCount(TEST_TABLE, fam), 1, "There should be only one rows in table before flushing"); + LOG.info("Flushing table {}", TEST_TABLE); + admin.flush(TableName.valueOf(TEST_TABLE)); + assertEquals(rowCount(TEST_TABLE, fam), 1, "There should be only one rows in table after flushing"); + + // Break access to CommitTable functionality in Compactor + LOG.info("Regions in table {}: {}", TEST_TABLE, hbaseCluster.getRegions(Bytes.toBytes(TEST_TABLE)).size()); + OmidCompactor omidCompactor = (OmidCompactor) hbaseCluster.getRegions(Bytes.toBytes(TEST_TABLE)).get(0) + .getCoprocessorHost().findCoprocessor(OmidCompactor.class.getName()); + CommitTable commitTable = injector.getInstance(CommitTable.class); + CommitTable.Client commitTableClient = spy(commitTable.getClient()); + SettableFuture f = SettableFuture.create(); + f.setException(new IOException("Unable to read")); + doReturn(f).when(commitTableClient).readLowWatermark(); + omidCompactor.commitTableClientQueue.add(commitTableClient); + + LOG.info("Compacting table {}", TEST_TABLE); + admin.majorCompact(TableName.valueOf(TEST_TABLE)); // Should trigger the error when accessing CommitTable funct. + + LOG.info("Sleeping for 3 secs"); + Thread.sleep(3000); + LOG.info("Waking up after 3 secs"); + + // All rows should be there after the failed compaction + assertEquals(rowCount(TEST_TABLE, fam), 1, "There should be only one row in table after compacting"); + assertTrue(CellUtils.hasCell(Bytes.toBytes(rowId), fam, qual, neverendingTx.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Cell should be there"); + assertFalse(CellUtils.hasShadowCell(Bytes.toBytes(rowId), fam, qual, neverendingTx.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Shadow cell should not be there"); + } + + @Test(timeOut = 60_000) + public void testOriginalTableParametersAreAvoidedAlsoWhenCompacting() throws Throwable { + String TEST_TABLE = "testOriginalTableParametersAreAvoidedAlsoWhenCompacting"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + long rowId = randomGenerator.nextLong(); + for (int versionCount = 0; versionCount <= (2 * MAX_VERSIONS); versionCount++) { + Transaction tx = tm.begin(); + Put put = new Put(Bytes.toBytes(rowId)); + put.addColumn(fam, qual, Bytes.toBytes("testWrite-" + versionCount)); + txTable.put(tx, put); + tm.commit(tx); + } + + Transaction tx = tm.begin(); + Get get = new Get(Bytes.toBytes(rowId)); + get.setMaxVersions(2 * MAX_VERSIONS); + assertEquals(get.getMaxVersions(), (2 * MAX_VERSIONS), "Max versions should be set to " + (2 * MAX_VERSIONS)); + get.addColumn(fam, qual); + Result result = txTable.get(tx, get); + tm.commit(tx); + List column = result.getColumnCells(fam, qual); + assertEquals(column.size(), 1, "There should be only one version in the result"); + + assertEquals(rowCount(TEST_TABLE, fam), 1, "There should be only one row in table before flushing"); + LOG.info("Flushing table {}", TEST_TABLE); + admin.flush(TableName.valueOf(TEST_TABLE)); + assertEquals(rowCount(TEST_TABLE, fam), 1, "There should be only one row in table after flushing"); + + // Return a LWM that triggers compaction + compactEverything(TEST_TABLE); + + // One row should have been discarded after compacting + assertEquals(rowCount(TEST_TABLE, fam), 1, "There should be only one row in table after compacting"); + + tx = tm.begin(); + get = new Get(Bytes.toBytes(rowId)); + get.setMaxVersions(2 * MAX_VERSIONS); + assertEquals(get.getMaxVersions(), (2 * MAX_VERSIONS), "Max versions should be set to " + (2 * MAX_VERSIONS)); + get.addColumn(fam, qual); + result = txTable.get(tx, get); + tm.commit(tx); + column = result.getColumnCells(fam, qual); + assertEquals(column.size(), 1, "There should be only one version in the result"); + assertEquals(Bytes.toString(CellUtil.cloneValue(column.get(0))), "testWrite-" + (2 * MAX_VERSIONS), + "Values don't match"); + } + + // manually flush the regions on the region server. + // flushing like this prevents compaction running + // directly after the flush, which we want to avoid. + private void manualFlush(String tableName) throws Throwable { + LOG.info("Manually flushing all regions and waiting 2 secs"); + HBaseShims.flushAllOnlineRegions(hbaseTestUtil.getHBaseCluster().getRegionServer(0), + TableName.valueOf(tableName)); + TimeUnit.SECONDS.sleep(2); + } + + @Test(timeOut = 60_000) + public void testOldCellsAreDiscardedAfterCompaction() throws Exception { + String TEST_TABLE = "testOldCellsAreDiscardedAfterCompaction"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + byte[] rowId = Bytes.toBytes("row"); + + // Create 3 transactions modifying the same cell in a particular row + HBaseTransaction tx1 = (HBaseTransaction) tm.begin(); + Put put1 = new Put(rowId); + put1.addColumn(fam, qual, Bytes.toBytes("testValue 1")); + txTable.put(tx1, put1); + tm.commit(tx1); + + HBaseTransaction tx2 = (HBaseTransaction) tm.begin(); + Put put2 = new Put(rowId); + put2.addColumn(fam, qual, Bytes.toBytes("testValue 2")); + txTable.put(tx2, put2); + tm.commit(tx2); + + HBaseTransaction tx3 = (HBaseTransaction) tm.begin(); + Put put3 = new Put(rowId); + put3.addColumn(fam, qual, Bytes.toBytes("testValue 3")); + txTable.put(tx3, put3); + tm.commit(tx3); + + // Before compaction, the three timestamped values for the cell should be there + TTableCellGetterAdapter getter = new TTableCellGetterAdapter(txTable); + assertTrue(CellUtils.hasCell(rowId, fam, qual, tx1.getStartTimestamp(), getter), + "Put cell of Tx1 should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx1.getStartTimestamp(), getter), + "Put shadow cell of Tx1 should be there"); + assertTrue(CellUtils.hasCell(rowId, fam, qual, tx2.getStartTimestamp(), getter), + "Put cell of Tx2 cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx2.getStartTimestamp(), getter), + "Put shadow cell of Tx2 should be there"); + assertTrue(CellUtils.hasCell(rowId, fam, qual, tx3.getStartTimestamp(), getter), + "Put cell of Tx3 cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx3.getStartTimestamp(), getter), + "Put shadow cell of Tx3 should be there"); + + // Compact + HBaseTransaction lwmTx = (HBaseTransaction) tm.begin(); + compactWithLWM(lwmTx.getStartTimestamp(), TEST_TABLE); + + // After compaction, only the last value for the cell should have survived + assertFalse(CellUtils.hasCell(rowId, fam, qual, tx1.getStartTimestamp(), getter), + "Put cell of Tx1 should not be there"); + assertFalse(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx1.getStartTimestamp(), getter), + "Put shadow cell of Tx1 should not be there"); + assertFalse(CellUtils.hasCell(rowId, fam, qual, tx2.getStartTimestamp(), getter), + "Put cell of Tx2 should not be there"); + assertFalse(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx2.getStartTimestamp(), getter), + "Put shadow cell of Tx2 should not be there"); + assertTrue(CellUtils.hasCell(rowId, fam, qual, tx3.getStartTimestamp(), getter), + "Put cell of Tx3 cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx3.getStartTimestamp(), getter), + "Put shadow cell of Tx3 should be there"); + + // A new transaction after compaction should read the last value written + HBaseTransaction newTx1 = (HBaseTransaction) tm.begin(); + Get newGet1 = new Get(rowId); + newGet1.addColumn(fam, qual); + Result result = txTable.get(newTx1, newGet1); + assertEquals(Bytes.toBytes("testValue 3"), result.getValue(fam, qual)); + // Write a new value + Put newPut1 = new Put(rowId); + newPut1.addColumn(fam, qual, Bytes.toBytes("new testValue 1")); + txTable.put(newTx1, newPut1); + + // Start a second new transaction + HBaseTransaction newTx2 = (HBaseTransaction) tm.begin(); + // Commit first of the new tx + tm.commit(newTx1); + + // The second transaction should still read the previous value + Get newGet2 = new Get(rowId); + newGet2.addColumn(fam, qual); + result = txTable.get(newTx2, newGet2); + assertEquals(Bytes.toBytes("testValue 3"), result.getValue(fam, qual)); + tm.commit(newTx2); + + // Only two values -the new written by newTx1 and the last value + // for the cell after compaction- should have survived + assertFalse(CellUtils.hasCell(rowId, fam, qual, tx1.getStartTimestamp(), getter), + "Put cell of Tx1 should not be there"); + assertFalse(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx1.getStartTimestamp(), getter), + "Put shadow cell of Tx1 should not be there"); + assertFalse(CellUtils.hasCell(rowId, fam, qual, tx2.getStartTimestamp(), getter), + "Put cell of Tx2 should not be there"); + assertFalse(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx2.getStartTimestamp(), getter), + "Put shadow cell of Tx2 should not be there"); + assertTrue(CellUtils.hasCell(rowId, fam, qual, tx3.getStartTimestamp(), getter), + "Put cell of Tx3 cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx3.getStartTimestamp(), getter), + "Put shadow cell of Tx3 should be there"); + assertTrue(CellUtils.hasCell(rowId, fam, qual, newTx1.getStartTimestamp(), getter), + "Put cell of NewTx1 cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, newTx1.getStartTimestamp(), getter), + "Put shadow cell of NewTx1 should be there"); + } + + /** + * Tests a case where a temporary failure to flush causes the compactor to crash + */ + @Test(timeOut = 60_000) + public void testDuplicateDeletes() throws Throwable { + String TEST_TABLE = "testDuplicateDeletes"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + // jump through hoops to trigger a minor compaction. + // a minor compaction will only run if there are enough + // files to be compacted, but that is less than the number + // of total files, in which case it will run a major + // compaction. The issue this is testing only shows up + // with minor compaction, as only Deletes can be duplicate + // and major compactions filter them out. + byte[] firstRow = "FirstRow".getBytes(); + HBaseTransaction tx0 = (HBaseTransaction) tm.begin(); + Put put0 = new Put(firstRow); + put0.addColumn(fam, qual, Bytes.toBytes("testWrite-1")); + txTable.put(tx0, put0); + tm.commit(tx0); + + // create the first hfile + manualFlush(TEST_TABLE); + + // write a row, it won't be committed + byte[] rowToBeCompactedAway = "compactMe".getBytes(); + HBaseTransaction tx1 = (HBaseTransaction) tm.begin(); + Put put1 = new Put(rowToBeCompactedAway); + put1.addColumn(fam, qual, Bytes.toBytes("testWrite-1")); + txTable.put(tx1, put1); + txTable.flushCommits(); + + // write a row to trigger the double delete problem + byte[] row = "iCauseErrors".getBytes(); + HBaseTransaction tx2 = (HBaseTransaction) tm.begin(); + Put put2 = new Put(row); + put2.addColumn(fam, qual, Bytes.toBytes("testWrite-1")); + txTable.put(tx2, put2); + tm.commit(tx2); + + HBaseTransaction tx3 = (HBaseTransaction) tm.begin(); + Put put3 = new Put(row); + put3.addColumn(fam, qual, Bytes.toBytes("testWrite-1")); + txTable.put(tx3, put3); + txTable.flushCommits(); + + // cause a failure on HBaseTM#preCommit(); + Set writeSet = tx3.getWriteSet(); + assertEquals(1, writeSet.size()); + List newWriteSet = new ArrayList<>(); + final AtomicBoolean flushFailing = new AtomicBoolean(true); + for (HBaseCellId id : writeSet) { + TTable failableHTable = spy(id.getTable()); + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) + throws Throwable { + if (flushFailing.get()) { + throw new RetriesExhaustedWithDetailsException(new ArrayList(), + new ArrayList(), new ArrayList()); + } else { + invocation.callRealMethod(); + } + return null; + } + }).when(failableHTable).flushCommits(); + + newWriteSet.add(HBaseCellId.valueOf(tx3,failableHTable, + id.getRow(), id.getFamily(), + id.getQualifier(), id.getTimestamp())); + } + writeSet.clear(); + writeSet.addAll(newWriteSet); + + try { + tm.commit(tx3); + fail("Shouldn't succeed"); + } catch (TransactionException tme) { + flushFailing.set(false); + tm.rollback(tx3); + } + + // create second hfile, + // it should contain multiple deletes + manualFlush(TEST_TABLE); + + // create loads of files + byte[] anotherRow = "someotherrow".getBytes(); + HBaseTransaction tx4 = (HBaseTransaction) tm.begin(); + Put put4 = new Put(anotherRow); + put4.addColumn(fam, qual, Bytes.toBytes("testWrite-1")); + txTable.put(tx4, put4); + tm.commit(tx4); + + // create third hfile + manualFlush(TEST_TABLE); + + // trigger minor compaction and give it time to run + setCompactorLWM(tx4.getStartTimestamp(), TEST_TABLE); + admin.compact(TableName.valueOf(TEST_TABLE)); + Thread.sleep(3000); + + // check if the cell that should be compacted, is compacted + assertFalse(CellUtils.hasCell(rowToBeCompactedAway, fam, qual, tx1.getStartTimestamp(), + new TTableCellGetterAdapter(txTable)), + "Cell should not be be there"); + } + + @Test(timeOut = 60_000) + public void testNonOmidCFIsUntouched() throws Throwable { + String TEST_TABLE = "testNonOmidCFIsUntouched"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + admin.disableTable(TableName.valueOf(TEST_TABLE)); + byte[] nonOmidCF = Bytes.toBytes("nonOmidCF"); + byte[] nonOmidQual = Bytes.toBytes("nonOmidCol"); + HColumnDescriptor nonomidfam = new HColumnDescriptor(nonOmidCF); + nonomidfam.setMaxVersions(MAX_VERSIONS); + admin.addColumn(TableName.valueOf(TEST_TABLE), nonomidfam); + admin.enableTable(TableName.valueOf(TEST_TABLE)); + + byte[] rowId = Bytes.toBytes("testRow"); + Transaction tx = tm.begin(); + Put put = new Put(rowId); + put.addColumn(fam, qual, Bytes.toBytes("testValue")); + txTable.put(tx, put); + + Put nonTxPut = new Put(rowId); + nonTxPut.addColumn(nonOmidCF, nonOmidQual, Bytes.toBytes("nonTxVal")); + txTable.getHTable().put(nonTxPut); + txTable.flushCommits(); // to make sure it left the client + + Get g = new Get(rowId); + Result result = txTable.getHTable().get(g); + assertEquals(result.getColumnCells(nonOmidCF, nonOmidQual).size(), 1, "Should be there, precompact"); + assertEquals(result.getColumnCells(fam, qual).size(), 1, "Should be there, precompact"); + + compactEverything(TEST_TABLE); + + result = txTable.getHTable().get(g); + assertEquals(result.getColumnCells(nonOmidCF, nonOmidQual).size(), 1, "Should be there, postcompact"); + assertEquals(result.getColumnCells(fam, qual).size(), 0, "Should not be there, postcompact"); + } + + // ---------------------------------------------------------------------------------------------------------------- + // Tests on tombstones and non-transactional Deletes + // ---------------------------------------------------------------------------------------------------------------- + + /** + * Test that when a major compaction runs, cells that were deleted non-transactionally dissapear + */ + @Test(timeOut = 60_000) + public void testACellDeletedNonTransactionallyDoesNotAppearWhenAMajorCompactionOccurs() throws Throwable { + String TEST_TABLE = "testACellDeletedNonTransactionallyDoesNotAppearWhenAMajorCompactionOccurs"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + Table table = txTable.getHTable(); + + // Write first a value transactionally + HBaseTransaction tx0 = (HBaseTransaction) tm.begin(); + byte[] rowId = Bytes.toBytes("row1"); + Put p0 = new Put(rowId); + p0.addColumn(fam, qual, Bytes.toBytes("testValue-0")); + txTable.put(tx0, p0); + tm.commit(tx0); + + // Then perform a non-transactional Delete + Delete d = new Delete(rowId); + d.addColumn(fam, qual); + table.delete(d); + + // Trigger a major compaction + HBaseTransaction lwmTx = (HBaseTransaction) tm.begin(); + compactWithLWM(lwmTx.getStartTimestamp(), TEST_TABLE); + + // Then perform a non-tx (raw) scan... + Scan scan = new Scan(); + scan.setRaw(true); + ResultScanner scannerResults = table.getScanner(scan); + + // ...and test the deleted cell is not there anymore + assertNull(scannerResults.next(), "There should be no results in scan results"); + + table.close(); + + } + + /** + * Test that when a minor compaction runs, cells that were deleted non-transactionally are preserved. This is to + * allow users still access the cells when doing "improper" operations on a transactional table + */ + @Test(timeOut = 60_000) + public void testACellDeletedNonTransactionallyIsPreservedWhenMinorCompactionOccurs() throws Throwable { + String TEST_TABLE = "testACellDeletedNonTransactionallyIsPreservedWhenMinorCompactionOccurs"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + Table table = txTable.getHTable(); + + // Configure the environment to create a minor compaction + + // Write first a value transactionally + HBaseTransaction tx0 = (HBaseTransaction) tm.begin(); + byte[] rowId = Bytes.toBytes("row1"); + Put p0 = new Put(rowId); + p0.addColumn(fam, qual, Bytes.toBytes("testValue-0")); + txTable.put(tx0, p0); + tm.commit(tx0); + + // create the first hfile + manualFlush(TEST_TABLE); + + // Write another value transactionally + HBaseTransaction tx1 = (HBaseTransaction) tm.begin(); + Put p1 = new Put(rowId); + p1.addColumn(fam, qual, Bytes.toBytes("testValue-1")); + txTable.put(tx1, p1); + tm.commit(tx1); + + // create the second hfile + manualFlush(TEST_TABLE); + + // Write yet another value transactionally + HBaseTransaction tx2 = (HBaseTransaction) tm.begin(); + Put p2 = new Put(rowId); + p2.addColumn(fam, qual, Bytes.toBytes("testValue-2")); + txTable.put(tx2, p2); + tm.commit(tx2); + + // create a third hfile + manualFlush(TEST_TABLE); + + // Then perform a non-transactional Delete + Delete d = new Delete(rowId); + d.addColumn(fam, qual); + table.delete(d); + + // create the fourth hfile + manualFlush(TEST_TABLE); + + // Trigger the minor compaction + HBaseTransaction lwmTx = (HBaseTransaction) tm.begin(); + setCompactorLWM(lwmTx.getStartTimestamp(), TEST_TABLE); + admin.compact(TableName.valueOf(TEST_TABLE)); + Thread.sleep(5000); + + // Then perform a non-tx (raw) scan... + Scan scan = new Scan(); + scan.setRaw(true); + ResultScanner scannerResults = table.getScanner(scan); + + // ...and test the deleted cell is still there + int count = 0; + Result scanResult; + List listOfCellsScanned = new ArrayList<>(); + while ((scanResult = scannerResults.next()) != null) { + listOfCellsScanned = scanResult.listCells(); // equivalent to rawCells() + count++; + } + assertEquals(count, 1, "There should be only one result in scan results"); + assertEquals(listOfCellsScanned.size(), 3, "There should be 3 cell entries in scan results (2 puts, 1 del)"); + boolean wasDeletedCellFound = false; + int numberOfDeletedCellsFound = 0; + for (Cell cell : listOfCellsScanned) { + if (CellUtil.isDelete(cell)) { + wasDeletedCellFound = true; + numberOfDeletedCellsFound++; + } + } + assertTrue(wasDeletedCellFound, "We should have found a non-transactionally deleted cell"); + assertEquals(numberOfDeletedCellsFound, 1, "There should be only only one deleted cell"); + + table.close(); + } + + /** + * Test that when a minor compaction runs, tombstones are not cleaned up + */ + @Test(timeOut = 60_000) + public void testTombstonesAreNotCleanedUpWhenMinorCompactionOccurs() throws Throwable { + String TEST_TABLE = "testTombstonesAreNotCleanedUpWhenMinorCompactionOccurs"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + // Configure the environment to create a minor compaction + + HBaseTransaction tx0 = (HBaseTransaction) tm.begin(); + byte[] rowId = Bytes.toBytes("case1"); + Put p = new Put(rowId); + p.addColumn(fam, qual, Bytes.toBytes("testValue-0")); + txTable.put(tx0, p); + tm.commit(tx0); + + // create the first hfile + manualFlush(TEST_TABLE); + + // Create the tombstone + HBaseTransaction deleteTx = (HBaseTransaction) tm.begin(); + Delete d = new Delete(rowId); + d.addColumn(fam, qual); + txTable.delete(deleteTx, d); + tm.commit(deleteTx); + + // create the second hfile + manualFlush(TEST_TABLE); + + HBaseTransaction tx1 = (HBaseTransaction) tm.begin(); + Put p1 = new Put(rowId); + p1.addColumn(fam, qual, Bytes.toBytes("testValue-11")); + txTable.put(tx1, p1); + tm.commit(tx1); + + // create the third hfile + manualFlush(TEST_TABLE); + + HBaseTransaction lastTx = (HBaseTransaction) tm.begin(); + Put p2 = new Put(rowId); + p2.addColumn(fam, qual, Bytes.toBytes("testValue-222")); + txTable.put(lastTx, p2); + tm.commit(lastTx); + + // Trigger the minor compaction + HBaseTransaction lwmTx = (HBaseTransaction) tm.begin(); + setCompactorLWM(lwmTx.getStartTimestamp(), TEST_TABLE); + admin.compact(TableName.valueOf(TEST_TABLE)); + Thread.sleep(5000); + + // Checks on results after compaction + TTableCellGetterAdapter getter = new TTableCellGetterAdapter(txTable); + assertFalse(CellUtils.hasCell(rowId, fam, qual, tx0.getStartTimestamp(), getter), "Put cell should be there"); + assertFalse(CellUtils.hasShadowCell(rowId, fam, qual, tx0.getStartTimestamp(), getter), + "Put shadow cell should be there"); + assertTrue(CellUtils.hasCell(rowId, fam, qual, tx1.getStartTimestamp(), getter), "Put cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx1.getStartTimestamp(), getter), + "Put shadow cell should be there"); + assertTrue(CellUtils.hasCell(rowId, fam, qual, deleteTx.getStartTimestamp(), getter), + "Delete cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, deleteTx.getStartTimestamp(), getter), + "Delete shadow cell should be there"); + assertTrue(CellUtils.hasCell(rowId, fam, qual, lastTx.getStartTimestamp(), getter), + "Put cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, lastTx.getStartTimestamp(), getter), + "Put shadow cell should be there"); + } + + + /** + * Test that when compaction runs, tombstones are cleaned up case1: 1 put (ts < lwm) then tombstone (ts > lwm) + */ + @Test(timeOut = 60_000) + public void testTombstonesAreCleanedUpCase1() throws Exception { + String TEST_TABLE = "testTombstonesAreCleanedUpCase1"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + HBaseTransaction tx1 = (HBaseTransaction) tm.begin(); + byte[] rowId = Bytes.toBytes("case1"); + Put p = new Put(rowId); + p.addColumn(fam, qual, Bytes.toBytes("testValue")); + txTable.put(tx1, p); + tm.commit(tx1); + + HBaseTransaction lwmTx = (HBaseTransaction) tm.begin(); + setCompactorLWM(lwmTx.getStartTimestamp(), TEST_TABLE); + + HBaseTransaction tx2 = (HBaseTransaction) tm.begin(); + Delete d = new Delete(rowId); + d.addColumn(fam, qual); + txTable.delete(tx2, d); + tm.commit(tx2); + + TTableCellGetterAdapter getter = new TTableCellGetterAdapter(txTable); + assertTrue(CellUtils.hasCell(rowId, fam, qual, tx1.getStartTimestamp(), getter), + "Put cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx1.getStartTimestamp(), getter), + "Put shadow cell should be there"); + assertTrue(CellUtils.hasCell(rowId, fam, qual, tx2.getStartTimestamp(), getter), + "Delete cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx2.getStartTimestamp(), getter), + "Delete shadow cell should be there"); + } + + /** + * Test that when compaction runs, tombstones are cleaned up case2: 1 put (ts < lwm) then tombstone (ts < lwm) + */ + @Test(timeOut = 60_000) + public void testTombstonesAreCleanedUpCase2() throws Exception { + String TEST_TABLE = "testTombstonesAreCleanedUpCase2"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + HBaseTransaction tx1 = (HBaseTransaction) tm.begin(); + byte[] rowId = Bytes.toBytes("case2"); + Put p = new Put(rowId); + p.addColumn(fam, qual, Bytes.toBytes("testValue")); + txTable.put(tx1, p); + tm.commit(tx1); + + HBaseTransaction tx2 = (HBaseTransaction) tm.begin(); + Delete d = new Delete(rowId); + d.addColumn(fam, qual); + txTable.delete(tx2, d); + tm.commit(tx2); + + HBaseTransaction lwmTx = (HBaseTransaction) tm.begin(); + compactWithLWM(lwmTx.getStartTimestamp(), TEST_TABLE); + + TTableCellGetterAdapter getter = new TTableCellGetterAdapter(txTable); + assertFalse(CellUtils.hasCell(rowId, fam, qual, tx1.getStartTimestamp(), getter), + "Put cell shouldn't be there"); + assertFalse(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx1.getStartTimestamp(), getter), + "Put shadow cell shouldn't be there"); + assertFalse(CellUtils.hasCell(rowId, fam, qual, tx2.getStartTimestamp(), getter), + "Delete cell shouldn't be there"); + assertFalse(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx2.getStartTimestamp(), getter), + "Delete shadow cell shouldn't be there"); + } + + /** + * Test that when compaction runs, tombstones are cleaned up case3: 1 put (ts < lwm) then tombstone (ts < lwm) not + * committed + */ + @Test(timeOut = 60_000) + public void testTombstonesAreCleanedUpCase3() throws Exception { + String TEST_TABLE = "testTombstonesAreCleanedUpCase3"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + HBaseTransaction tx1 = (HBaseTransaction) tm.begin(); + byte[] rowId = Bytes.toBytes("case3"); + Put p = new Put(rowId); + p.addColumn(fam, qual, Bytes.toBytes("testValue")); + txTable.put(tx1, p); + tm.commit(tx1); + + HBaseTransaction tx2 = (HBaseTransaction) tm.begin(); + Delete d = new Delete(rowId); + d.addColumn(fam, qual); + txTable.delete(tx2, d); + + HBaseTransaction lwmTx = (HBaseTransaction) tm.begin(); + compactWithLWM(lwmTx.getStartTimestamp(), TEST_TABLE); + + TTableCellGetterAdapter getter = new TTableCellGetterAdapter(txTable); + assertTrue(CellUtils.hasCell(rowId, fam, qual, tx1.getStartTimestamp(), getter), + "Put cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx1.getStartTimestamp(), getter), + "Put shadow cell shouldn't be there"); + assertFalse(CellUtils.hasCell(rowId, fam, qual, tx2.getStartTimestamp(), getter), + "Delete cell shouldn't be there"); + assertFalse(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx2.getStartTimestamp(), getter), + "Delete shadow cell shouldn't be there"); + } + + /** + * Test that when compaction runs, tombstones are cleaned up case4: 1 put (ts < lwm) then tombstone (ts > lwm) not + * committed + */ + @Test(timeOut = 60_000) + public void testTombstonesAreCleanedUpCase4() throws Exception { + String TEST_TABLE = "testTombstonesAreCleanedUpCase4"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + HBaseTransaction tx1 = (HBaseTransaction) tm.begin(); + byte[] rowId = Bytes.toBytes("case4"); + Put p = new Put(rowId); + p.addColumn(fam, qual, Bytes.toBytes("testValue")); + txTable.put(tx1, p); + tm.commit(tx1); + + HBaseTransaction lwmTx = (HBaseTransaction) tm.begin(); + + HBaseTransaction tx2 = (HBaseTransaction) tm.begin(); + Delete d = new Delete(rowId); + d.addColumn(fam, qual); + txTable.delete(tx2, d); + compactWithLWM(lwmTx.getStartTimestamp(), TEST_TABLE); + + TTableCellGetterAdapter getter = new TTableCellGetterAdapter(txTable); + assertTrue(CellUtils.hasCell(rowId, fam, qual, tx1.getStartTimestamp(), getter), + "Put cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx1.getStartTimestamp(), getter), + "Put shadow cell shouldn't be there"); + assertTrue(CellUtils.hasCell(rowId, fam, qual,tx2.getStartTimestamp(), getter), + "Delete cell should be there"); + assertFalse(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx2.getStartTimestamp(), getter), + "Delete shadow cell shouldn't be there"); + } + + /** + * Test that when compaction runs, tombstones are cleaned up case5: tombstone (ts < lwm) + */ + @Test(timeOut = 60_000) + public void testTombstonesAreCleanedUpCase5() throws Exception { + String TEST_TABLE = "testTombstonesAreCleanedUpCase5"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + + HBaseTransaction tx1 = (HBaseTransaction) tm.begin(); + byte[] rowId = Bytes.toBytes("case5"); + Delete d = new Delete(rowId); + d.addColumn(fam, qual); + txTable.delete(tx1, d); + tm.commit(tx1); + + HBaseTransaction lwmTx = (HBaseTransaction) tm.begin(); + compactWithLWM(lwmTx.getStartTimestamp(), TEST_TABLE); + + TTableCellGetterAdapter getter = new TTableCellGetterAdapter(txTable); + assertFalse(CellUtils.hasCell(rowId, fam, qual, tx1.getStartTimestamp(), getter), + "Delete cell shouldn't be there"); + assertFalse(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx1.getStartTimestamp(), getter), + "Delete shadow cell shouldn't be there"); + } + + /** + * Test that when compaction runs, tombstones are cleaned up case6: tombstone (ts < lwm), then put (ts < lwm) + */ + @Test(timeOut = 60_000) + public void testTombstonesAreCleanedUpCase6() throws Exception { + String TEST_TABLE = "testTombstonesAreCleanedUpCase6"; + createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); + TTable txTable = new TTable(connection, TEST_TABLE); + byte[] rowId = Bytes.toBytes("case6"); + + HBaseTransaction tx1 = (HBaseTransaction) tm.begin(); + Delete d = new Delete(rowId); + d.addColumn(fam, qual); + txTable.delete(tx1, d); + tm.commit(tx1); + + HBaseTransaction tx2 = (HBaseTransaction) tm.begin(); + Put p = new Put(rowId); + p.addColumn(fam, qual, Bytes.toBytes("testValue")); + txTable.put(tx2, p); + tm.commit(tx2); + + HBaseTransaction lwmTx = (HBaseTransaction) tm.begin(); + compactWithLWM(lwmTx.getStartTimestamp(), TEST_TABLE); + + TTableCellGetterAdapter getter = new TTableCellGetterAdapter(txTable); + assertFalse(CellUtils.hasCell(rowId, fam, qual, tx1.getStartTimestamp(), getter), + "Delete cell shouldn't be there"); + assertFalse(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx1.getStartTimestamp(), getter), + "Delete shadow cell shouldn't be there"); + assertTrue(CellUtils.hasCell(rowId, fam, qual, tx2.getStartTimestamp(), getter), + "Put cell should be there"); + assertTrue(CellUtils.hasShadowCell(rowId, fam, CellUtils.SHARED_FAMILY_QUALIFIER, tx2.getStartTimestamp(), getter), + "Put shadow cell shouldn't be there"); + } + + private void setCompactorLWM(long lwm, String tableName) throws Exception { + OmidCompactor omidCompactor = (OmidCompactor) hbaseCluster.getRegions(Bytes.toBytes(tableName)).get(0) + .getCoprocessorHost().findCoprocessor(OmidCompactor.class.getName()); + CommitTable commitTable = injector.getInstance(CommitTable.class); + CommitTable.Client commitTableClient = spy(commitTable.getClient()); + SettableFuture f = SettableFuture.create(); + f.set(lwm); + doReturn(f).when(commitTableClient).readLowWatermark(); + omidCompactor.commitTableClientQueue.add(commitTableClient); + } + + private void compactEverything(String tableName) throws Exception { + compactWithLWM(Long.MAX_VALUE, tableName); + } + + private void compactWithLWM(long lwm, String tableName) throws Exception { + admin.flush(TableName.valueOf(tableName)); + + LOG.info("Regions in table {}: {}", tableName, hbaseCluster.getRegions(Bytes.toBytes(tableName)).size()); + setCompactorLWM(lwm, tableName); + LOG.info("Compacting table {}", tableName); + admin.majorCompact(TableName.valueOf(tableName)); + + LOG.info("Sleeping for 3 secs"); + Thread.sleep(3000); + LOG.info("Waking up after 3 secs"); + } + + private static long rowCount(String tableName, byte[] family) throws Throwable { + Scan scan = new Scan(); + scan.addFamily(family); + Table table = connection.getTable(TableName.valueOf(tableName)); + try (ResultScanner scanner = table.getScanner(scan)) { + int count = 0; + while (scanner.next() != null) { + count++; + } + return count; + } + } +} diff --git a/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactorScanner.java b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactorScanner.java index 8a217b31a..1ede20d08 100644 --- a/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactorScanner.java +++ b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactorScanner.java @@ -36,6 +36,8 @@ import java.util.Queue; +import static org.apache.omid.transaction.HBaseTransactionManager.ConflictDetectionLevel.CELL; +import static org.apache.omid.transaction.HBaseTransactionManager.ConflictDetectionLevel.ROW; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -84,7 +86,7 @@ public void testShouldRetainNonTransactionallyDeletedCellMethod(int optionIdx, b ctClient, queue, false, - retainOption))) { + retainOption,CELL))) { // Different cell types to test KeyValue regularKV = new KeyValue(Bytes.toBytes("test-row"), TEST_TS, Type.Put); diff --git a/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestSnapshotFilter.java b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestSnapshotFilter.java index 46b1c4a13..2b422e52d 100644 --- a/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestSnapshotFilter.java +++ b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestSnapshotFilter.java @@ -26,6 +26,8 @@ import static org.testng.Assert.assertTrue; import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -63,14 +65,17 @@ import org.apache.omid.timestamp.storage.HBaseTimestampStorageConfig; import org.apache.omid.tso.TSOServer; import org.apache.omid.tso.TSOServerConfig; + import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import static org.testng.Assert.fail; @@ -78,6 +83,7 @@ import com.google.inject.Guice; import com.google.inject.Injector; + public class TestSnapshotFilter { private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotFilter.class); @@ -100,6 +106,8 @@ public class TestSnapshotFilter { private CommitTable commitTable; private PostCommitActions syncPostCommitter; private Connection connection; + private HBaseTimestampStorageConfig hBaseTimestampStorageConfig; + private HBaseCommitTableConfig hBaseCommitTableConfig; @BeforeClass public void setupTestSnapshotFilter() throws Exception { @@ -122,8 +130,8 @@ public void setupTestSnapshotFilter() throws Exception { hbaseConf.setInt("hbase.regionserver.info.port", 0); - HBaseCommitTableConfig hBaseCommitTableConfig = injector.getInstance(HBaseCommitTableConfig.class); - HBaseTimestampStorageConfig hBaseTimestampStorageConfig = injector.getInstance(HBaseTimestampStorageConfig.class); + hBaseCommitTableConfig = injector.getInstance(HBaseCommitTableConfig.class); + hBaseTimestampStorageConfig = injector.getInstance(HBaseTimestampStorageConfig.class); setupHBase(); connection = ConnectionFactory.createConnection(hbaseConf); @@ -196,12 +204,19 @@ private void teardownTSO() throws IOException, InterruptedException { TestUtils.waitForSocketNotListening("localhost", 5679, 1000); } + @BeforeMethod - public void setupTestSnapshotFilterIndividualTest() throws Exception { - tm = spy((AbstractTransactionManager) newTransactionManager()); + public void setupTestSnapshotFilterIndividualTest(Object[] testArgs) throws Exception { + HBaseTransactionManager.ConflictDetectionLevel cfLevel; + if (testArgs.length == 0) { + cfLevel = HBaseTransactionManager.ConflictDetectionLevel.CELL; + } else { + cfLevel = (HBaseTransactionManager.ConflictDetectionLevel) testArgs[0]; + } + tm = spy((AbstractTransactionManager) newTransactionManager(cfLevel)); } - private TransactionManager newTransactionManager() throws Exception { + private TransactionManager newTransactionManager(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Exception { HBaseOmidClientConfiguration hbaseOmidClientConf = new HBaseOmidClientConfiguration(); hbaseOmidClientConf.setConnectionString("localhost:5679"); hbaseOmidClientConf.setHBaseConfiguration(hbaseConf); @@ -211,18 +226,27 @@ private TransactionManager newTransactionManager() throws Exception { return HBaseTransactionManager.builder(hbaseOmidClientConf) .postCommitter(syncPostCommitter) .commitTableClient(commitTableClient) + .conflictDetectionLevel(cfLevel) .build(); } - @Test(timeOut = 60_000) - public void testGetFirstResult() throws Throwable { + + @DataProvider(name = "cflevel") + public static Object[][] cfLevels() { + return new Object[][] { + {HBaseTransactionManager.ConflictDetectionLevel.CELL}, + {HBaseTransactionManager.ConflictDetectionLevel.ROW}}; + } + + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testGetFirstResult(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] famName1 = Bytes.toBytes(TEST_FAMILY); byte[] colName1 = Bytes.toBytes("col1"); byte[] dataValue1 = Bytes.toBytes("testWrite-1"); - String TEST_TABLE = "testGetFirstResult"; + String TEST_TABLE = cfLevel+"testGetFirstResult"; createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); TTable tt = new TTable(connection, TEST_TABLE); @@ -271,15 +295,15 @@ public void testGetFirstResult() throws Throwable { // This test will fail if filtering is done before snapshot filtering - @Test(timeOut = 60_000) - public void testServerSideSnapshotFiltering() throws Throwable { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testServerSideSnapshotFiltering(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] famName1 = Bytes.toBytes(TEST_FAMILY); byte[] colName1 = Bytes.toBytes("col1"); byte[] dataValue1 = Bytes.toBytes("testWrite-1"); byte[] dataValue2 = Bytes.toBytes("testWrite-2"); - String TEST_TABLE = "testServerSideSnapshotFiltering"; + String TEST_TABLE = cfLevel+"testServerSideSnapshotFiltering"; createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); TTable tt = new TTable(connection, TEST_TABLE); @@ -313,15 +337,15 @@ public void testServerSideSnapshotFiltering() throws Throwable { // This test will fail if filtering is done before snapshot filtering - @Test(timeOut = 60_000) - public void testServerSideSnapshotScannerFiltering() throws Throwable { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testServerSideSnapshotScannerFiltering(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] famName1 = Bytes.toBytes(TEST_FAMILY); byte[] colName1 = Bytes.toBytes("col1"); byte[] dataValue1 = Bytes.toBytes("testWrite-1"); byte[] dataValue2 = Bytes.toBytes("testWrite-2"); - String TEST_TABLE = "testServerSideSnapshotFiltering"; + String TEST_TABLE = cfLevel+"testServerSideSnapshotFiltering"; createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); TTable tt = new TTable(connection, TEST_TABLE); @@ -358,8 +382,8 @@ public void testServerSideSnapshotScannerFiltering() throws Throwable { } - @Test(timeOut = 60_000) - public void testGetWithFamilyDelete() throws Throwable { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testGetWithFamilyDelete(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] famName1 = Bytes.toBytes(TEST_FAMILY); byte[] famName2 = Bytes.toBytes("test-fam2"); @@ -367,7 +391,7 @@ public void testGetWithFamilyDelete() throws Throwable { byte[] colName2 = Bytes.toBytes("col2"); byte[] dataValue1 = Bytes.toBytes("testWrite-1"); - String TEST_TABLE = "testGetWithFamilyDelete"; + String TEST_TABLE = cfLevel+"testGetWithFamilyDelete"; createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY), famName2); TTable tt = new TTable(connection, TEST_TABLE); @@ -419,20 +443,20 @@ public void testGetWithFamilyDelete() throws Throwable { tt.close(); } - @Test(timeOut = 60_000) - public void testReadFromCommitTable() throws Exception { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testReadFromCommitTable(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Exception { final byte[] rowName1 = Bytes.toBytes("row1"); byte[] famName1 = Bytes.toBytes(TEST_FAMILY); byte[] colName1 = Bytes.toBytes("col1"); byte[] dataValue1 = Bytes.toBytes("testWrite-1"); - final String TEST_TABLE = "testReadFromCommitTable"; + final String TEST_TABLE = cfLevel+"testReadFromCommitTable"; final byte[] famName2 = Bytes.toBytes("test-fam2"); final CountDownLatch readAfterCommit = new CountDownLatch(1); final CountDownLatch postCommitBegin = new CountDownLatch(1); final AtomicBoolean readFailed = new AtomicBoolean(false); - final AbstractTransactionManager tm = (AbstractTransactionManager) newTransactionManager(); + final AbstractTransactionManager tm = (AbstractTransactionManager) newTransactionManager(cfLevel); createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY), famName2); doAnswer(new Answer>() { @@ -497,8 +521,8 @@ public void run() { - @Test(timeOut = 60_000) - public void testGetWithFilter() throws Throwable { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testGetWithFilter(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] famName1 = Bytes.toBytes(TEST_FAMILY); byte[] famName2 = Bytes.toBytes("test-fam2"); @@ -506,7 +530,7 @@ public void testGetWithFilter() throws Throwable { byte[] colName2 = Bytes.toBytes("col2"); byte[] dataValue1 = Bytes.toBytes("testWrite-1"); - String TEST_TABLE = "testGetWithFilter"; + String TEST_TABLE = cfLevel+"testGetWithFilter"; createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY), famName2); TTable tt = new TTable(connection, TEST_TABLE); @@ -550,14 +574,14 @@ public void testGetWithFilter() throws Throwable { } - @Test(timeOut = 60_000) - public void testGetSecondResult() throws Throwable { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testGetSecondResult(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] famName1 = Bytes.toBytes(TEST_FAMILY); byte[] colName1 = Bytes.toBytes("col1"); byte[] dataValue1 = Bytes.toBytes("testWrite-1"); - String TEST_TABLE = "testGetSecondResult"; + String TEST_TABLE = cfLevel+"testGetSecondResult"; createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); TTable tt = new TTable(connection, TEST_TABLE); @@ -589,15 +613,15 @@ public void testGetSecondResult() throws Throwable { tt.close(); } - @Test(timeOut = 60_000) - public void testScanFirstResult() throws Throwable { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testScanFirstResult(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] famName1 = Bytes.toBytes(TEST_FAMILY); byte[] colName1 = Bytes.toBytes("col1"); byte[] dataValue1 = Bytes.toBytes("testWrite-1"); - String TEST_TABLE = "testScanFirstResult"; + String TEST_TABLE = cfLevel+"testScanFirstResult"; createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); TTable tt = new TTable(connection, TEST_TABLE); @@ -642,8 +666,8 @@ public void testScanFirstResult() throws Throwable { } - @Test(timeOut = 60_000) - public void testScanWithFilter() throws Throwable { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testScanWithFilter(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] famName1 = Bytes.toBytes(TEST_FAMILY); @@ -652,7 +676,7 @@ public void testScanWithFilter() throws Throwable { byte[] colName2 = Bytes.toBytes("col2"); byte[] dataValue1 = Bytes.toBytes("testWrite-1"); - String TEST_TABLE = "testScanWithFilter"; + String TEST_TABLE = cfLevel+"testScanWithFilter"; createTableIfNotExists(TEST_TABLE, famName1, famName2); TTable tt = new TTable(connection, TEST_TABLE); @@ -694,15 +718,15 @@ public void testScanWithFilter() throws Throwable { } - @Test(timeOut = 60_000) - public void testScanSecondResult() throws Throwable { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testScanSecondResult(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] famName1 = Bytes.toBytes(TEST_FAMILY); byte[] colName1 = Bytes.toBytes("col1"); byte[] dataValue1 = Bytes.toBytes("testWrite-1"); - String TEST_TABLE = "testScanSecondResult"; + String TEST_TABLE = cfLevel+"testScanSecondResult"; createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); TTable tt = new TTable(connection, TEST_TABLE); @@ -734,8 +758,8 @@ public void testScanSecondResult() throws Throwable { tt.close(); } - @Test (timeOut = 60_000) - public void testScanFewResults() throws Throwable { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testScanFewResults(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] rowName2 = Bytes.toBytes("row2"); @@ -746,7 +770,7 @@ public void testScanFewResults() throws Throwable { byte[] dataValue1 = Bytes.toBytes("testWrite-1"); byte[] dataValue2 = Bytes.toBytes("testWrite-2"); - String TEST_TABLE = "testScanFewResults"; + String TEST_TABLE = cfLevel+"testScanFewResults"; createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); TTable tt = new TTable(connection, TEST_TABLE); @@ -784,8 +808,8 @@ public void testScanFewResults() throws Throwable { tt.close(); } - @Test (timeOut = 60_000) - public void testScanFewResultsDifferentTransaction() throws Throwable { + @Test(timeOut = 60_000, dataProvider = "cflevel") + public void testScanFewResultsDifferentTransaction(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] rowName2 = Bytes.toBytes("row2"); @@ -796,7 +820,7 @@ public void testScanFewResultsDifferentTransaction() throws Throwable { byte[] dataValue1 = Bytes.toBytes("testWrite-1"); byte[] dataValue2 = Bytes.toBytes("testWrite-2"); - String TEST_TABLE = "testScanFewResultsDifferentTransaction"; + String TEST_TABLE = cfLevel+"testScanFewResultsDifferentTransaction"; createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); TTable tt = new TTable(connection, TEST_TABLE); @@ -837,8 +861,8 @@ public void testScanFewResultsDifferentTransaction() throws Throwable { tt.close(); } - @Test (timeOut = 60_000) - public void testScanFewResultsSameTransaction() throws Throwable { + @Test (timeOut = 60_000, dataProvider = "cflevel") + public void testScanFewResultsSameTransaction(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { byte[] rowName1 = Bytes.toBytes("row1"); byte[] rowName2 = Bytes.toBytes("row2"); @@ -849,7 +873,7 @@ public void testScanFewResultsSameTransaction() throws Throwable { byte[] dataValue1 = Bytes.toBytes("testWrite-1"); byte[] dataValue2 = Bytes.toBytes("testWrite-2"); - String TEST_TABLE = "testScanFewResultsSameTransaction"; + String TEST_TABLE = cfLevel+"testScanFewResultsSameTransaction"; createTableIfNotExists(TEST_TABLE, Bytes.toBytes(TEST_FAMILY)); TTable tt = new TTable(connection, TEST_TABLE); @@ -889,9 +913,9 @@ public void testScanFewResultsSameTransaction() throws Throwable { } - @Test (timeOut = 60_000) - public void testFilterCommitCacheInSnapshot() throws Throwable { - String TEST_TABLE = "testScanWithFilter"; + @Test (timeOut = 60_000, dataProvider = "cflevel") + public void testFilterCommitCacheInSnapshot(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { + String TEST_TABLE = cfLevel+"testFilterCommitCacheInSnapshot"; byte[] rowName = Bytes.toBytes("row1"); byte[] famName = Bytes.toBytes(TEST_FAMILY); @@ -932,9 +956,9 @@ public void testFilterCommitCacheInSnapshot() throws Throwable { tt.close(); } - @Test (timeOut = 60_000) - public void testFilterCommitCacheNotInSnapshot() throws Throwable { - String TEST_TABLE = "testScanWithFilter"; + @Test (timeOut = 60_000, dataProvider = "cflevel") + public void testFilterCommitCacheNotInSnapshot(HBaseTransactionManager.ConflictDetectionLevel cfLevel) throws Throwable { + String TEST_TABLE = cfLevel+"testFilterCommitCacheNotInSnapshot"; byte[] rowName = Bytes.toBytes("row1"); byte[] famName = Bytes.toBytes(TEST_FAMILY); diff --git a/transaction-client/src/main/java/org/apache/omid/tso/client/MockTSOClient.java b/transaction-client/src/main/java/org/apache/omid/tso/client/MockTSOClient.java index 75351439c..81955db59 100644 --- a/transaction-client/src/main/java/org/apache/omid/tso/client/MockTSOClient.java +++ b/transaction-client/src/main/java/org/apache/omid/tso/client/MockTSOClient.java @@ -163,16 +163,6 @@ public boolean isLowLatency() { return false; } - @Override - public void setConflictDetectionLevel(OmidClientConfiguration.ConflictDetectionLevel conflictDetectionLevel) { - - } - - @Override - public OmidClientConfiguration.ConflictDetectionLevel getConflictDetectionLevel() { - return null; - } - @Override public long getEpoch() { return 0; diff --git a/transaction-client/src/main/java/org/apache/omid/tso/client/OmidClientConfiguration.java b/transaction-client/src/main/java/org/apache/omid/tso/client/OmidClientConfiguration.java index 6bc648120..735a5e494 100644 --- a/transaction-client/src/main/java/org/apache/omid/tso/client/OmidClientConfiguration.java +++ b/transaction-client/src/main/java/org/apache/omid/tso/client/OmidClientConfiguration.java @@ -32,8 +32,6 @@ public enum ConnType {DIRECT, HA} public enum PostCommitMode {SYNC, ASYNC} - public enum ConflictDetectionLevel {CELL, ROW} - // Basic connection related params private ConnType connectionType = ConnType.DIRECT; @@ -53,7 +51,7 @@ public enum ConflictDetectionLevel {CELL, ROW} // Transaction Manager related params private PostCommitMode postCommitMode = PostCommitMode.SYNC; - private ConflictDetectionLevel conflictAnalysisLevel = ConflictDetectionLevel.CELL; + // ---------------------------------------------------------------------------------------------------------------- // Instantiation @@ -176,14 +174,4 @@ public PostCommitMode getPostCommitMode() { public void setPostCommitMode(PostCommitMode postCommitMode) { this.postCommitMode = postCommitMode; } - - public ConflictDetectionLevel getConflictAnalysisLevel() { - return conflictAnalysisLevel; - } - - @Inject(optional = true) - @Named("omid.tm.conflictAnalysisLevel") - public void setConflictAnalysisLevel(ConflictDetectionLevel conflictAnalysisLevel) { - this.conflictAnalysisLevel = conflictAnalysisLevel; - } } diff --git a/transaction-client/src/main/java/org/apache/omid/tso/client/TSOClient.java b/transaction-client/src/main/java/org/apache/omid/tso/client/TSOClient.java index 28e8686de..e1d2a4559 100644 --- a/transaction-client/src/main/java/org/apache/omid/tso/client/TSOClient.java +++ b/transaction-client/src/main/java/org/apache/omid/tso/client/TSOClient.java @@ -23,7 +23,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.omid.proto.TSOProto; -import org.apache.omid.tso.client.OmidClientConfiguration.ConflictDetectionLevel; import org.apache.omid.zk.ZKUtils; import org.apache.statemachine.StateMachine; import org.apache.curator.framework.CuratorFramework; @@ -98,12 +97,6 @@ public class TSOClient implements TSOProtocol, NodeCacheListener { private boolean lowLatency; - // Use to extract unique table identifiers from the modified cells list. - private final Set tableIDs; - // Conflict detection level of the entire system. Can either be Row or Cell level. - private ConflictDetectionLevel conflictDetectionLevel; - private Set rowLevelWriteSet; - // ---------------------------------------------------------------------------------------------------------------- // Construction // ---------------------------------------------------------------------------------------------------------------- @@ -173,10 +166,6 @@ private TSOClient(OmidClientConfiguration omidConf) throws IOException { bootstrap.setOption("connectTimeoutMillis", 100); lowLatency = false; - this.tableIDs = new HashSet(); - - conflictDetectionLevel = omidConf.getConflictAnalysisLevel(); - rowLevelWriteSet = new HashSet(); } // ---------------------------------------------------------------------------------------------------------------- @@ -213,28 +202,9 @@ public TSOFuture commit(long transactionId, Set cells, S TSOProto.CommitRequest.Builder commitbuilder = TSOProto.CommitRequest.newBuilder(); commitbuilder.setStartTimestamp(transactionId); - rowLevelWriteSet.clear(); + HashSet tableIDs = new HashSet(); for (CellId cell : cells) { - long id; - - switch (conflictDetectionLevel) { - case ROW: - id = cell.getRowId(); - if (rowLevelWriteSet.contains(id)) { - continue; - } else { - rowLevelWriteSet.add(id); - } - break; - case CELL: - id = cell.getCellId(); - break; - default: - id = 0; - assert (false); - } - - commitbuilder.addCellId(id); + commitbuilder.addCellId(cell.getCellId()); tableIDs.add(cell.getTableId()); } @@ -312,20 +282,6 @@ public long getEpoch() { return epoch; } - /** - * Used for family deletion - * @return the conflict detection level. - */ - public ConflictDetectionLevel getConflictDetectionLevel() { - return conflictDetectionLevel; - } - - /** - * Used for family deletion testing - */ - public void setConflictDetectionLevel(ConflictDetectionLevel conflictDetectionLevel) { - this.conflictDetectionLevel = conflictDetectionLevel; - } // ---------------------------------------------------------------------------------------------------------------- // NodeCacheListener interface diff --git a/transaction-client/src/main/java/org/apache/omid/tso/client/TSOProtocol.java b/transaction-client/src/main/java/org/apache/omid/tso/client/TSOProtocol.java index 921b3d177..7f93ce78b 100644 --- a/transaction-client/src/main/java/org/apache/omid/tso/client/TSOProtocol.java +++ b/transaction-client/src/main/java/org/apache/omid/tso/client/TSOProtocol.java @@ -93,8 +93,4 @@ public interface TSOProtocol { * @return */ boolean isLowLatency(); - - void setConflictDetectionLevel(OmidClientConfiguration.ConflictDetectionLevel conflictDetectionLevel); - - OmidClientConfiguration.ConflictDetectionLevel getConflictDetectionLevel(); } diff --git a/transaction-client/src/main/resources/omid-client-config.yml b/transaction-client/src/main/resources/omid-client-config.yml index 478bd488c..1ad68f8e3 100644 --- a/transaction-client/src/main/resources/omid-client-config.yml +++ b/transaction-client/src/main/resources/omid-client-config.yml @@ -38,6 +38,3 @@ executorThreads: 3 # before returning to the control to the client (SYNC) or in parallel (ASYNC) postCommitMode: !!org.apache.omid.tso.client.OmidClientConfiguration$PostCommitMode SYNC -# Conflict analysis level -# Can either be cell level or row level. Default is cell level -conflictDetectionLevel: !!org.apache.omid.tso.client.OmidClientConfiguration$ConflictDetectionLevel CELL diff --git a/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientRowAndCellLevelConflict.java b/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientRowAndCellLevelConflict.java index da655a3b3..4ff8ea2d7 100644 --- a/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientRowAndCellLevelConflict.java +++ b/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientRowAndCellLevelConflict.java @@ -26,7 +26,7 @@ import org.apache.omid.tso.TSOMockModule; import org.apache.omid.tso.TSOServer; import org.apache.omid.tso.TSOServerConfig; -import org.apache.omid.tso.client.OmidClientConfiguration.ConflictDetectionLevel; + import org.apache.omid.tso.util.DummyCellIdImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,116 +88,116 @@ public void afterMethod() throws Exception { TestUtils.waitForSocketNotListening(TSO_SERVER_HOST, TSO_SERVER_PORT, 1000); } - @Test(timeOut = 30_000) - public void testRowLevelConflictAnalysisConflict() throws Exception { - - tsoClientConf.setConflictAnalysisLevel(ConflictDetectionLevel.ROW); - - TSOClient client = TSOClient.newInstance(tsoClientConf); - - CellId c1 = new DummyCellIdImpl(0xdeadbeefL, 0xdeadbeeeL); - CellId c2 = new DummyCellIdImpl(0xfeedcafeL, 0xdeadbeeeL); - - Set testWriteSet1 = Sets.newHashSet(c1); - Set testWriteSet2 = Sets.newHashSet(c2); - - long ts1 = client.getNewStartTimestamp().get(); - long ts2 = client.getNewStartTimestamp().get(); - - client.commit(ts1, testWriteSet1).get(); - - try { - client.commit(ts2, testWriteSet2).get(); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof AbortException, "Transaction should be aborted"); - return; - } - - assertTrue(false, "Transaction should be aborted"); - } - - @Test(timeOut = 30_000) - public void testRowLevelConflictAnalysisCommit() throws Exception { - - tsoClientConf.setConflictAnalysisLevel(ConflictDetectionLevel.ROW); - - TSOClient client = TSOClient.newInstance(tsoClientConf); - - CellId c1 = new DummyCellIdImpl(0xdeadbeefL, 0xdeadbeeeL); - CellId c2 = new DummyCellIdImpl(0xfeedcafeL, 0xdeadbeefL); - - Set testWriteSet1 = Sets.newHashSet(c1); - Set testWriteSet2 = Sets.newHashSet(c2); - - long ts1 = client.getNewStartTimestamp().get(); - long ts2 = client.getNewStartTimestamp().get(); - - client.commit(ts1, testWriteSet1).get(); - - try { - client.commit(ts2, testWriteSet2).get(); - } catch (ExecutionException e) { - assertFalse(e.getCause() instanceof AbortException, "Transaction should be committed"); - return; - } - - assertTrue(true, "Transaction should be committed"); - } - - @Test(timeOut = 30_000) - public void testCellLevelConflictAnalysisConflict() throws Exception { - - tsoClientConf.setConflictAnalysisLevel(ConflictDetectionLevel.CELL); - - TSOClient client = TSOClient.newInstance(tsoClientConf); - - CellId c1 = new DummyCellIdImpl(0xdeadbeefL, 0xdeadbeeeL); - CellId c2 = new DummyCellIdImpl(0xdeadbeefL, 0xdeadbeeeL); - - Set testWriteSet1 = Sets.newHashSet(c1); - Set testWriteSet2 = Sets.newHashSet(c2); - - long ts1 = client.getNewStartTimestamp().get(); - long ts2 = client.getNewStartTimestamp().get(); - - client.commit(ts1, testWriteSet1).get(); - - try { - client.commit(ts2, testWriteSet2).get(); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof AbortException, "Transaction should be aborted"); - return; - } - - assertTrue(false, "Transaction should be aborted"); - } - - @Test(timeOut = 30_000) - public void testCellLevelConflictAnalysisCommit() throws Exception { - - tsoClientConf.setConflictAnalysisLevel(ConflictDetectionLevel.CELL); - - TSOClient client = TSOClient.newInstance(tsoClientConf); - - CellId c1 = new DummyCellIdImpl(0xdeadbeefL, 0xdeadbeeeL); - CellId c2 = new DummyCellIdImpl(0xfeedcafeL, 0xdeadbeefL); - - Set testWriteSet1 = Sets.newHashSet(c1); - Set testWriteSet2 = Sets.newHashSet(c2); - - long ts1 = client.getNewStartTimestamp().get(); - long ts2 = client.getNewStartTimestamp().get(); - - client.commit(ts1, testWriteSet1).get(); - - try { - client.commit(ts2, testWriteSet2).get(); - } catch (ExecutionException e) { - assertFalse(e.getCause() instanceof AbortException, "Transaction should be committed"); - return; - } - - assertTrue(true, "Transaction should be committed"); - } +// @Test(timeOut = 30_000) +// public void testRowLevelConflictAnalysisConflict() throws Exception { +// +// tsoClientConf.setConflictAnalysisLevel(ConflictDetectionLevel.ROW); +// +// TSOClient client = TSOClient.newInstance(tsoClientConf); +// +// CellId c1 = new DummyCellIdImpl(0xdeadbeefL, 0xdeadbeeeL); +// CellId c2 = new DummyCellIdImpl(0xfeedcafeL, 0xdeadbeeeL); +// +// Set testWriteSet1 = Sets.newHashSet(c1); +// Set testWriteSet2 = Sets.newHashSet(c2); +// +// long ts1 = client.getNewStartTimestamp().get(); +// long ts2 = client.getNewStartTimestamp().get(); +// +// client.commit(ts1, testWriteSet1).get(); +// +// try { +// client.commit(ts2, testWriteSet2).get(); +// } catch (ExecutionException e) { +// assertTrue(e.getCause() instanceof AbortException, "Transaction should be aborted"); +// return; +// } +// +// assertTrue(false, "Transaction should be aborted"); +// } +// +// @Test(timeOut = 30_000) +// public void testRowLevelConflictAnalysisCommit() throws Exception { +// +// tsoClientConf.setConflictAnalysisLevel(ConflictDetectionLevel.ROW); +// +// TSOClient client = TSOClient.newInstance(tsoClientConf); +// +// CellId c1 = new DummyCellIdImpl(0xdeadbeefL, 0xdeadbeeeL); +// CellId c2 = new DummyCellIdImpl(0xfeedcafeL, 0xdeadbeefL); +// +// Set testWriteSet1 = Sets.newHashSet(c1); +// Set testWriteSet2 = Sets.newHashSet(c2); +// +// long ts1 = client.getNewStartTimestamp().get(); +// long ts2 = client.getNewStartTimestamp().get(); +// +// client.commit(ts1, testWriteSet1).get(); +// +// try { +// client.commit(ts2, testWriteSet2).get(); +// } catch (ExecutionException e) { +// assertFalse(e.getCause() instanceof AbortException, "Transaction should be committed"); +// return; +// } +// +// assertTrue(true, "Transaction should be committed"); +// } +// +// @Test(timeOut = 30_000) +// public void testCellLevelConflictAnalysisConflict() throws Exception { +// +// tsoClientConf.setConflictAnalysisLevel(ConflictDetectionLevel.CELL); +// +// TSOClient client = TSOClient.newInstance(tsoClientConf); +// +// CellId c1 = new DummyCellIdImpl(0xdeadbeefL, 0xdeadbeeeL); +// CellId c2 = new DummyCellIdImpl(0xdeadbeefL, 0xdeadbeeeL); +// +// Set testWriteSet1 = Sets.newHashSet(c1); +// Set testWriteSet2 = Sets.newHashSet(c2); +// +// long ts1 = client.getNewStartTimestamp().get(); +// long ts2 = client.getNewStartTimestamp().get(); +// +// client.commit(ts1, testWriteSet1).get(); +// +// try { +// client.commit(ts2, testWriteSet2).get(); +// } catch (ExecutionException e) { +// assertTrue(e.getCause() instanceof AbortException, "Transaction should be aborted"); +// return; +// } +// +// assertTrue(false, "Transaction should be aborted"); +// } +// +// @Test(timeOut = 30_000) +// public void testCellLevelConflictAnalysisCommit() throws Exception { +// +// tsoClientConf.setConflictAnalysisLevel(ConflictDetectionLevel.CELL); +// +// TSOClient client = TSOClient.newInstance(tsoClientConf); +// +// CellId c1 = new DummyCellIdImpl(0xdeadbeefL, 0xdeadbeeeL); +// CellId c2 = new DummyCellIdImpl(0xfeedcafeL, 0xdeadbeefL); +// +// Set testWriteSet1 = Sets.newHashSet(c1); +// Set testWriteSet2 = Sets.newHashSet(c2); +// +// long ts1 = client.getNewStartTimestamp().get(); +// long ts2 = client.getNewStartTimestamp().get(); +// +// client.commit(ts1, testWriteSet1).get(); +// +// try { +// client.commit(ts2, testWriteSet2).get(); +// } catch (ExecutionException e) { +// assertFalse(e.getCause() instanceof AbortException, "Transaction should be committed"); +// return; +// } +// +// assertTrue(true, "Transaction should be committed"); +// } }