You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@omid.apache.org by oh...@apache.org on 2018/07/31 14:19:29 UTC

incubator-omid git commit: [OMID-105] When a tentative family deletion marker is found. We need to continue looking until we either find a committed one in the past or no committed family deletion marker for this column is found. Otherwise, we might

Repository: incubator-omid
Updated Branches:
  refs/heads/phoenix-integration 0f14b27fe -> 7d0986ecb


[OMID-105] When a tentative family deletion marker is
 found. We need to continue looking until we either find a  committed one in
 the past or no committed family deletion marker for this column is found.
 Otherwise, we might miss committed family deletion markers that exists in a
 transaction snapshot.


Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/7d0986ec
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/7d0986ec
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/7d0986ec

Branch: refs/heads/phoenix-integration
Commit: 7d0986ecbf449262c6d1a029fee932d202bf0e09
Parents: 0f14b27
Author: Ohad Shacham <oh...@yahoo-inc.com>
Authored: Tue Jul 31 17:18:48 2018 +0300
Committer: Ohad Shacham <oh...@yahoo-inc.com>
Committed: Tue Jul 31 17:18:48 2018 +0300

----------------------------------------------------------------------
 .../transaction/AttributeSetSnapshotFilter.java |   2 +-
 .../apache/omid/transaction/SnapshotFilter.java |   2 +-
 .../omid/transaction/SnapshotFilterImpl.java    | 101 ++++++++++++-------
 .../org/apache/omid/transaction/TTable.java     |   6 +-
 .../apache/omid/transaction/TestDeletion.java   |  98 +++++++++++++++++-
 .../omid/transaction/TestShadowCells.java       |   2 +-
 .../org/apache/omid/transaction/CellUtils.java  |  11 ++
 .../hbase/regionserver/OmidRegionScanner.java   |   4 +-
 .../omid/transaction/OmidSnapshotFilter.java    |   2 +-
 9 files changed, 182 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/7d0986ec/hbase-client/src/main/java/org/apache/omid/transaction/AttributeSetSnapshotFilter.java
----------------------------------------------------------------------
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 bf2adf5..482dc4c 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
@@ -65,7 +65,7 @@ public class AttributeSetSnapshotFilter implements SnapshotFilter {
 
     @Override
     public List<Cell> filterCellsForSnapshot(List<Cell> rawCells, HBaseTransaction transaction,
-                                      int versionsToRequest, Map<String, List<Cell>> familyDeletionCache, Map<String,byte[]> attributeMap) throws IOException {
+                                      int versionsToRequest, Map<String, Long> familyDeletionCache, Map<String,byte[]> attributeMap) throws IOException {
         throw new UnsupportedOperationException();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/7d0986ec/hbase-client/src/main/java/org/apache/omid/transaction/SnapshotFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/omid/transaction/SnapshotFilter.java b/hbase-client/src/main/java/org/apache/omid/transaction/SnapshotFilter.java
index 3027d89..4668c9e 100644
--- a/hbase-client/src/main/java/org/apache/omid/transaction/SnapshotFilter.java
+++ b/hbase-client/src/main/java/org/apache/omid/transaction/SnapshotFilter.java
@@ -37,7 +37,7 @@ public interface SnapshotFilter {
     public ResultScanner getScanner(TTable ttable, Scan scan, HBaseTransaction transaction) throws IOException;
 
     public List<Cell> filterCellsForSnapshot(List<Cell> rawCells, HBaseTransaction transaction,
-            int versionsToRequest, Map<String, List<Cell>> familyDeletionCache, Map<String,byte[]> attributeMap) throws IOException;
+            int versionsToRequest, Map<String, Long> familyDeletionCache, Map<String,byte[]> attributeMap) throws IOException;
 
     public boolean isCommitted(HBaseCellId hBaseCellId, long epoch) throws TransactionException;
 

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/7d0986ec/hbase-client/src/main/java/org/apache/omid/transaction/SnapshotFilterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/omid/transaction/SnapshotFilterImpl.java b/hbase-client/src/main/java/org/apache/omid/transaction/SnapshotFilterImpl.java
index c317783..d0472d2 100644
--- a/hbase-client/src/main/java/org/apache/omid/transaction/SnapshotFilterImpl.java
+++ b/hbase-client/src/main/java/org/apache/omid/transaction/SnapshotFilterImpl.java
@@ -89,6 +89,10 @@ public class SnapshotFilterImpl implements SnapshotFilter {
         this.commitTableClient = commitTableClient;
     }
 
+    private String getRowFamilyString(Cell cell) {
+        return Bytes.toString((CellUtil.cloneRow(cell))) + ":" + Bytes.toString(CellUtil.cloneFamily(cell));
+    }
+
     /**
      * Check whether a cell was deleted using family deletion marker
      *
@@ -98,20 +102,13 @@ public class SnapshotFilterImpl implements SnapshotFilter {
      * @param commitCache         Holds shadow cells information
      * @return Whether the cell was deleted
      */
-    private boolean checkFamilyDeletionCache(Cell cell, HBaseTransaction transaction, Map<String, List<Cell>> familyDeletionCache, Map<Long, Long> commitCache) throws IOException {
-        List<Cell> familyDeletionCells = familyDeletionCache.get(Bytes.toString((cell.getRow())));
-        if (familyDeletionCells != null) {
-            for(Cell familyDeletionCell : familyDeletionCells) {
-                String family = Bytes.toString(cell.getFamily());
-                String familyDeletion = Bytes.toString(familyDeletionCell.getFamily());
-                if (family.equals(familyDeletion)) {
-                    Optional<Long> familyDeletionCommitTimestamp = getCommitTimestamp(familyDeletionCell, transaction, commitCache);
-                    if (familyDeletionCommitTimestamp.isPresent() && familyDeletionCommitTimestamp.get() >= cell.getTimestamp()) {
-                        return true;
-                    }
-                }
-            }
+    private boolean checkFamilyDeletionCache(Cell cell, HBaseTransaction transaction, Map<String, Long> familyDeletionCache, Map<Long, Long> commitCache) throws IOException {
+        String key = getRowFamilyString(cell);
+        Long familyDeletionCommitTimestamp = familyDeletionCache.get(key);
+        if (familyDeletionCommitTimestamp != null && familyDeletionCommitTimestamp >= cell.getTimestamp()) {
+            return true;
         }
+
         return false;
     }
 
@@ -298,26 +295,55 @@ public class SnapshotFilterImpl implements SnapshotFilter {
         return commitCache;
     }
 
-    private void buildFamilyDeletionCache(List<Cell> rawCells, Map<String, List<Cell>> familyDeletionCache) {
-
+    private void buildFamilyDeletionCache(HBaseTransaction transaction, List<Cell> rawCells, Map<String, Long> familyDeletionCache, Map<Long, Long> commitCache, Map<String,byte[]> attributeMap) throws IOException {
         for (Cell cell : rawCells) {
-            if (CellUtil.matchingQualifier(cell, CellUtils.FAMILY_DELETE_QUALIFIER) &&
-                    CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY)) {
-
-                String row = Bytes.toString(cell.getRow());
-                List<Cell> cells = familyDeletionCache.get(row);
-                if (cells == null) {
-                    cells = new ArrayList<>();
-                    familyDeletionCache.put(row, cells);
+            if (CellUtils.isFamilyDeleteCell(cell)) {
+                String key = getRowFamilyString(cell);
+
+                if (familyDeletionCache.containsKey(key))
+                    return;
+
+                Optional<Long> commitTimeStamp = getTSIfInTransaction(cell, transaction, commitCache);
+
+                if (!commitTimeStamp.isPresent()) {
+                    commitTimeStamp = getTSIfInSnapshot(cell, transaction, commitCache);
                 }
 
-                cells.add(cell);
+                if (commitTimeStamp.isPresent()) {
+                    familyDeletionCache.put(key, commitTimeStamp.get());
+                } else {
+                    Cell lastCell = cell;
+                    Map<Long, Long> cmtCache;
+                    boolean foundCommittedFamilyDeletion = false;
+                    while (!foundCommittedFamilyDeletion) {
+
+                        Get g = createPendingGet(lastCell, 3);
+
+                        Result result = tableAccessWrapper.get(g);
+                        List<Cell> resultCells = result.listCells();
+                        if (resultCells == null) {
+                            break;
+                        }
+
+                        cmtCache = buildCommitCache(resultCells);
+                        for (Cell c : resultCells) {
+                            if (CellUtils.isFamilyDeleteCell(c)) {
+                                    commitTimeStamp = getTSIfInSnapshot(c, transaction, cmtCache);
+                                    if (commitTimeStamp.isPresent()) {
+                                        familyDeletionCache.put(key, commitTimeStamp.get());
+                                        foundCommittedFamilyDeletion = true;
+                                        break;
+                                    }
+                                    lastCell = c;
+                            }
+                        }
+                    }
+                }
             }
         }
-
     }
 
-    private boolean isCellInTransaction(Cell kv, HBaseTransaction transaction, Map<Long, Long> commitCache) {
+    private Optional<Long> getTSIfInTransaction(Cell kv, HBaseTransaction transaction, Map<Long, Long> commitCache) {
 
         long startTimestamp = transaction.getStartTimestamp();
         long readTimestamp = transaction.getReadTimestamp();
@@ -326,18 +352,21 @@ public class SnapshotFilterImpl implements SnapshotFilter {
         // There also might be a case where the cell was written by the transaction and its timestamp equals to its writeTimestamp, however,
         // this case occurs after checkpoint and in this case we do not want to read this data.
         if (kv.getTimestamp() >= startTimestamp && kv.getTimestamp() <= readTimestamp) {
-            return true;
+            return Optional.of(kv.getTimestamp());
         }
 
-        return false;
+        return Optional.absent();
     }
 
-    private boolean isCellInSnapshot(Cell kv, HBaseTransaction transaction, Map<Long, Long> commitCache)
+    private Optional<Long> getTSIfInSnapshot(Cell kv, HBaseTransaction transaction, Map<Long, Long> commitCache)
         throws IOException {
 
         Optional<Long> commitTimestamp = getCommitTimestamp(kv, transaction, commitCache);
 
-        return commitTimestamp.isPresent() && commitTimestamp.get() < transaction.getStartTimestamp();
+        if (commitTimestamp.isPresent() && commitTimestamp.get() < transaction.getStartTimestamp())
+            return commitTimestamp;
+
+        return Optional.absent();
     }
 
     private Get createPendingGet(Cell cell, int versionCount) throws IOException {
@@ -366,7 +395,7 @@ public class SnapshotFilterImpl implements SnapshotFilter {
      */
     @Override
     public List<Cell> filterCellsForSnapshot(List<Cell> rawCells, HBaseTransaction transaction,
-                                      int versionsToRequest, Map<String, List<Cell>> familyDeletionCache, Map<String,byte[]> attributeMap) throws IOException {
+                                      int versionsToRequest, Map<String, Long> familyDeletionCache, Map<String,byte[]> attributeMap) throws IOException {
 
         assert (rawCells != null && transaction != null && versionsToRequest >= 1);
 
@@ -379,7 +408,7 @@ public class SnapshotFilterImpl implements SnapshotFilter {
         }
 
         Map<Long, Long> commitCache = buildCommitCache(rawCells);
-        buildFamilyDeletionCache(rawCells, familyDeletionCache);
+        buildFamilyDeletionCache(transaction, rawCells, familyDeletionCache, commitCache, attributeMap);
 
         for (Collection<Cell> columnCells : groupCellsByColumnFilteringShadowCellsAndFamilyDeletion(rawCells)) {
             boolean snapshotValueFound = false;
@@ -395,8 +424,8 @@ public class SnapshotFilterImpl implements SnapshotFilter {
                     }
                 }
 
-                if (isCellInTransaction(cell, transaction, commitCache) ||
-                    isCellInSnapshot(cell, transaction, commitCache)) {
+                if (getTSIfInTransaction(cell, transaction, commitCache).isPresent() ||
+                    getTSIfInSnapshot(cell, transaction, commitCache).isPresent()) {
                     if (!CellUtil.matchingValue(cell, CellUtils.DELETE_TOMBSTONE)) {
                         keyValuesInSnapshot.add(cell);
                     }
@@ -405,7 +434,7 @@ public class SnapshotFilterImpl implements SnapshotFilter {
                     // 1. if we found a result and we are not in SNAPSHOT_ALL mode.
                     // 2. if we found a result that was not written by the current transaction.
                     if (transaction.getVisibilityLevel() != VisibilityLevel.SNAPSHOT_ALL ||
-                        !isCellInTransaction(cell, transaction, commitCache)) {
+                        !getTSIfInTransaction(cell, transaction, commitCache).isPresent()) {
                         snapshotValueFound = true;
                         break;
                     }
@@ -443,7 +472,7 @@ public class SnapshotFilterImpl implements SnapshotFilter {
 
         List<Cell> filteredKeyValues = Collections.emptyList();
         if (!result.isEmpty()) {
-            filteredKeyValues = ttable.filterCellsForSnapshot(result.listCells(), transaction, get.getMaxVersions(), new HashMap<String, List<Cell>>(), get.getAttributesMap());
+            filteredKeyValues = ttable.filterCellsForSnapshot(result.listCells(), transaction, get.getMaxVersions(), new HashMap<String, Long>(), get.getAttributesMap());
         }
 
         return Result.create(filteredKeyValues);

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/7d0986ec/hbase-client/src/main/java/org/apache/omid/transaction/TTable.java
----------------------------------------------------------------------
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 bc41de5..aab2e2f 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
@@ -447,7 +447,7 @@ public class TTable implements Closeable {
 
 
     List<Cell> filterCellsForSnapshot(List<Cell> rawCells, HBaseTransaction transaction,
-                                      int versionsToRequest, Map<String, List<Cell>> familyDeletionCache, Map<String,byte[]> attributeMap) throws IOException {
+                                      int versionsToRequest, Map<String, Long> familyDeletionCache, Map<String,byte[]> attributeMap) throws IOException {
         return snapshotFilter.filterCellsForSnapshot(rawCells, transaction, versionsToRequest, familyDeletionCache, attributeMap);
     }
 
@@ -457,7 +457,7 @@ public class TTable implements Closeable {
         private HBaseTransaction state;
         private ResultScanner innerScanner;
         private int maxVersions;
-        Map<String, List<Cell>> familyDeletionCache;
+        Map<String, Long> familyDeletionCache;
         private Map<String,byte[]> attributeMap;
 
         TransactionalClientScanner(HBaseTransaction state, Scan scan, int maxVersions)
@@ -465,7 +465,7 @@ public class TTable implements Closeable {
             this.state = state;
             this.innerScanner = table.getScanner(scan);
             this.maxVersions = maxVersions;
-            this.familyDeletionCache = new HashMap<String, List<Cell>>();
+            this.familyDeletionCache = new HashMap<String, Long>();
             this.attributeMap = scan.getAttributesMap();
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/7d0986ec/hbase-client/src/test/java/org/apache/omid/transaction/TestDeletion.java
----------------------------------------------------------------------
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 5f5fd45..c426c1b 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
@@ -48,7 +48,7 @@ public class TestDeletion extends OmidTestBase {
     private byte[] colA = Bytes.toBytes("testdataA");
     private byte[] colB = Bytes.toBytes("testdataB");
     private byte[] data1 = Bytes.toBytes("testWrite-1");
-    private byte[] modrow = Bytes.toBytes("test-del" + 3);
+    private byte[] modrow = Bytes.toBytes("test-del" + 0);
 
     private static class FamCol {
 
@@ -63,6 +63,102 @@ public class TestDeletion extends OmidTestBase {
     }
 
     @Test(timeOut = 10_000)
+    public void runTestDeleteFamilyRow(ITestContext context) throws Exception {
+
+        TransactionManager tm = newTransactionManager(context);
+        TTable tt = new TTable(hbaseConf, TEST_TABLE);
+
+        ((HBaseTransactionManager) tm).setConflictDetectionLevel(ConflictDetectionLevel.ROW);
+
+        Transaction t1 = tm.begin();
+        LOG.info("Transaction created " + t1);
+
+        int rowsWritten = 1;
+        FamCol famColA = new FamCol(famA, colA);
+        writeRows(tt, t1, rowsWritten, famColA);
+        tm.commit(t1);
+
+        Transaction t2 = tm.begin();
+        Delete d = new Delete(modrow);
+        d.deleteFamily(famA);
+        tt.delete(t2, d);
+
+        Transaction tscan = tm.begin();
+        ResultScanner rs = tt.getScanner(tscan, new Scan());
+
+        Map<FamCol, Integer> count = countColsInRows(rs, famColA);
+        assertEquals((int) count.get(famColA), rowsWritten, "ColA count should be equal to rowsWritten");
+        tm.commit(t2);
+
+        tscan = tm.begin();
+        rs = tt.getScanner(tscan, new Scan());
+
+        count = countColsInRows(rs, famColA);
+        Integer countFamColA = count.get(famColA);
+        assertEquals(countFamColA, null);
+
+        Transaction t3 = tm.begin();
+        d.deleteFamily(famA);
+        tt.delete(t3, d);
+
+        tscan = tm.begin();
+        rs = tt.getScanner(tscan, new Scan());
+
+        count = countColsInRows(rs, famColA);
+        countFamColA = count.get(famColA);
+        assertEquals(countFamColA, null);
+
+        ((HBaseTransactionManager) tm).setConflictDetectionLevel(ConflictDetectionLevel.CELL);
+
+    }
+
+    @Test(timeOut = 10_000)
+    public void runTestDeleteFamilyCell(ITestContext context) throws Exception {
+
+        TransactionManager tm = newTransactionManager(context);
+        TTable tt = new TTable(hbaseConf, TEST_TABLE);
+
+        Transaction t1 = tm.begin();
+        LOG.info("Transaction created " + t1);
+
+        int rowsWritten = 1;
+        FamCol famColA = new FamCol(famA, colA);
+        writeRows(tt, t1, rowsWritten, famColA);
+        tm.commit(t1);
+
+        Transaction t2 = tm.begin();
+        Delete d = new Delete(modrow);
+        d.deleteFamily(famA);
+        tt.delete(t2, d);
+
+        Transaction tscan = tm.begin();
+        ResultScanner rs = tt.getScanner(tscan, new Scan());
+
+        Map<FamCol, Integer> count = countColsInRows(rs, famColA);
+        assertEquals((int) count.get(famColA), rowsWritten, "ColA count should be equal to rowsWritten");
+        tm.commit(t2);
+
+        tscan = tm.begin();
+        rs = tt.getScanner(tscan, new Scan());
+
+        count = countColsInRows(rs, famColA);
+        Integer countFamColA = count.get(famColA);
+        assertEquals(countFamColA, null);
+
+        Transaction t3 = tm.begin();
+        d.deleteFamily(famA);
+        tt.delete(t3, d);
+
+        tscan = tm.begin();
+        rs = tt.getScanner(tscan, new Scan());
+
+        count = countColsInRows(rs, famColA);
+        countFamColA = count.get(famColA);
+        assertEquals(countFamColA, null);
+
+    }
+
+    @Test(timeOut = 10_000)
     public void runTestDeleteFamily(ITestContext context) throws Exception {
 
         TransactionManager tm = newTransactionManager(context);

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/7d0986ec/hbase-client/src/test/java/org/apache/omid/transaction/TestShadowCells.java
----------------------------------------------------------------------
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 75b8ee9..6b430f7 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
@@ -346,7 +346,7 @@ public class TestShadowCells extends OmidTestBase {
                             return (List<KeyValue>) invocation.callRealMethod();
                         }
                     }).when(table).filterCellsForSnapshot(Matchers.<List<Cell>>any(),
-                            any(HBaseTransaction.class), anyInt(), Matchers.<Map<String, List<Cell>>>any(), Matchers.<Map<String,byte[]>>any());
+                            any(HBaseTransaction.class), anyInt(), Matchers.<Map<String, Long>>any(), Matchers.<Map<String,byte[]>>any());
 
                     TransactionManager tm = newTransactionManager(context);
                     if (hasShadowCell(row,

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/7d0986ec/hbase-common/src/main/java/org/apache/omid/transaction/CellUtils.java
----------------------------------------------------------------------
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 013c41c..c515e97 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
@@ -24,6 +24,7 @@ import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.hash.Hasher;
 import com.google.common.hash.Hashing;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -236,6 +237,16 @@ public final class CellUtils {
     }
 
     /**
+     * Returns if a cell is a family deletion marker.
+     * @param cell the cell to check
+     * @return whether the cell is marked as a family deletion marker
+     */
+    public static boolean isFamilyDeleteCell(Cell cell) {
+        return CellUtil.matchingQualifier(cell, CellUtils.FAMILY_DELETE_QUALIFIER) &&
+                CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY);
+    }
+
+    /**
      * Returns a new shadow cell created from a particular cell.
      * @param cell
      *            the cell to reconstruct the shadow cell from.

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/7d0986ec/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/OmidRegionScanner.java
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/OmidRegionScanner.java b/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/OmidRegionScanner.java
index a3bcafe..7ccca00 100644
--- a/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/OmidRegionScanner.java
+++ b/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/OmidRegionScanner.java
@@ -37,7 +37,7 @@ public class OmidRegionScanner implements RegionScanner {
     private SnapshotFilterImpl snapshotFilter;
     private HBaseTransaction transaction;
     private int maxVersions;
-    private Map<String, List<Cell>> familyDeletionCache;
+    private Map<String, Long> familyDeletionCache;
     private Map<String,byte[]> attributeMap;
 
     public OmidRegionScanner(SnapshotFilterImpl snapshotFilter,
@@ -49,7 +49,7 @@ public class OmidRegionScanner implements RegionScanner {
         this.scanner = s;
         this.transaction = transaction;
         this.maxVersions = maxVersions;
-        this.familyDeletionCache = new HashMap<String, List<Cell>>();
+        this.familyDeletionCache = new HashMap<String, Long>();
         this.attributeMap = attributeMap;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/7d0986ec/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidSnapshotFilter.java
----------------------------------------------------------------------
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 2cd7efb..2c9caff 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
@@ -126,7 +126,7 @@ public class OmidSnapshotFilter extends BaseRegionObserver {
 
 
                 HBaseTransaction hbaseTransaction = new HBaseTransaction(id, readTs, visibilityLevel, epoch, new HashSet<HBaseCellId>(), new HashSet<HBaseCellId>(), null);
-                filteredKeyValues = snapshotFilter.filterCellsForSnapshot(res.listCells(), hbaseTransaction, get.getMaxVersions(), new HashMap<String, List<Cell>>(), get.getAttributesMap());
+                filteredKeyValues = snapshotFilter.filterCellsForSnapshot(res.listCells(), hbaseTransaction, get.getMaxVersions(), new HashMap<String, Long>(), get.getAttributesMap());
             }
 
             for (Cell cell : filteredKeyValues) {