You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@tephra.apache.org by poornachandra <gi...@git.apache.org> on 2017/02/10 08:30:51 UTC

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

GitHub user poornachandra opened a pull request:

    https://github.com/apache/incubator-tephra/pull/34

    TEPHRA-216 Handle empty transactional regions during inactive list pruning

    JIRA - https://issues.apache.org/jira/browse/TEPHRA-216
    
    Approach:
    1. Transaction Processor records a region as empty (at current time) in the post flush handler if the memstore size for the region is zero and the region does not have any store files.
    2. The regions that are recorded as empty after inactiveTransactionBoundTime will not have invalid data for transactions started before or on inactiveTransactionBoundTime. Hence we can consider the prune upper bound for these empty regions as the inactiveTransactionBound.
    
    TODO: Clean-up empty regions in pruneComplete()
    
    Note: The first two commits are re-factoring, and would be simple to review them separately.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/poornachandra/incubator-tephra feature/prune-empty-tables

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-tephra/pull/34.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #34
    
----
commit 2b796fa59c31c9003ede095d3c8c41b067a8142b
Author: poorna <po...@cask.co>
Date:   2017-02-09T22:44:14Z

    Cleanup test and add warning in prune writer

commit 04e8a2207eead0d49e55539ed2b215958fadf6d3
Author: poorna <po...@cask.co>
Date:   2017-02-10T01:07:30Z

    Refactor pruning plugin to handle empty regions

commit 73e1b83c4d937a409d4caf40715ab466b12253f1
Author: poorna <po...@cask.co>
Date:   2017-02-10T08:18:13Z

    TEPHRA-216 Handle empty transactional regions during inactive list pruning

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/34#discussion_r100647433
  
    --- Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java ---
    @@ -288,23 +288,27 @@ private long computePruneUpperBound(TimeRegions timeRegions) throws IOException
           SortedSet<byte[]> transactionalRegions = timeRegions.getRegions();
           long time = timeRegions.getTime();
     
    +      long inactiveTransactionBound = dataJanitorState.getInactiveTransactionBoundForTime(time);
    +      LOG.debug("Got inactive transaction bound {}", inactiveTransactionBound);
    +      // If inactiveTransactionBound is not recorded then that means the data is not complete for these regions
    +      if (inactiveTransactionBound == -1) {
    +        if (LOG.isDebugEnabled()) {
    --- End diff --
    
    Ah - now I remember. I added it avoid the string concatenation due to line break.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/34#discussion_r100648684
  
    --- Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/DataJanitorState.java ---
    @@ -366,6 +371,56 @@ public void deleteInactiveTransactionBoundsOnOrBeforeTime(long time) throws IOEx
         }
       }
     
    +  // --------------------------------------------------------
    +  // ------- Methods for empty regions at a given time -------
    +  // --------------------------------------------------------
    +  // Key: 0x4<time><region-id>
    +  // Col 'e': <empty byte array>
    +  // --------------------------------------------------------
    +
    +  /**
    +   * Save the given region as empty as of the given time.
    +   *
    +   * @param time time in milliseconds
    +   * @param regionId region id
    +   */
    +  public void saveEmptyRegionForTime(long time, byte[] regionId) throws IOException {
    +    byte[] timeBytes = Bytes.toBytes(time);
    +    try (Table stateTable = stateTableSupplier.get()) {
    +      Put put = new Put(makeEmptyRegionTimeKey(timeBytes, regionId));
    +      put.addColumn(FAMILY, EMPTY_REGION_TIME_COL, EMPTY_BYTE_ARRAY);
    --- End diff --
    
    Sure - will replace empty byte with '1'


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/34#discussion_r100648648
  
    --- Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java ---
    @@ -322,6 +333,28 @@ private long computePruneUpperBound(TimeRegions timeRegions) throws IOException
         return -1;
       }
     
    +  private Map<byte[], Long> handleEmptyRegions(long inactiveTransactionBound,
    +                                               SortedSet<byte[]> transactionalRegions,
    +                                               Map<byte[], Long> pruneUpperBoundRegions) throws IOException {
    +    long inactiveTransactionBoundTime = TxUtils.getTimestamp(inactiveTransactionBound);
    +    SortedSet<byte[]> emptyRegions =
    +      dataJanitorState.getEmptyRegionsAfterTime(inactiveTransactionBoundTime, transactionalRegions);
    +    LOG.debug("Got empty transactional regions for inactive transaction bound time {}: {}",
    +              inactiveTransactionBoundTime, Iterables.transform(emptyRegions, TimeRegions.BYTE_ARR_TO_STRING_FN));
    +
    +    // The regions that are recorded as empty after inactiveTransactionBoundTime will not have invalid data
    +    // for transactions started before or on inactiveTransactionBoundTime. Hence we can consider the prune upper bound
    +    // for these empty regions as inactiveTransactionBound
    +    Map<byte[], Long> pubWithEmptyRegions = new TreeMap<>(Bytes.BYTES_COMPARATOR);
    +    pubWithEmptyRegions.putAll(pruneUpperBoundRegions);
    +    for (byte[] emptyRegion : emptyRegions) {
    +      if (!pruneUpperBoundRegions.containsKey(emptyRegion)) {
    +        pubWithEmptyRegions.put(emptyRegion, inactiveTransactionBound);
    --- End diff --
    
    If an empty region does not contain a prune upper bound value then we put `inactiveTransactionBound` as its prune upper bound. How do you suggest this be changed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra issue #34: TEPHRA-216 Handle empty transactional regions du...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on the issue:

    https://github.com/apache/incubator-tephra/pull/34
  
    @anew @gokulavasan In addition to addressing comments, I have added two more commits, please take a look


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra issue #34: TEPHRA-216 Handle empty transactional regions du...

Posted by gokulavasan <gi...@git.apache.org>.
Github user gokulavasan commented on the issue:

    https://github.com/apache/incubator-tephra/pull/34
  
    Just one minor comment. LGTM \U0001f44d 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra issue #34: TEPHRA-216 Handle empty transactional regions du...

Posted by anew <gi...@git.apache.org>.
Github user anew commented on the issue:

    https://github.com/apache/incubator-tephra/pull/34
  
    LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/incubator-tephra/pull/34


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

Posted by gokulavasan <gi...@git.apache.org>.
Github user gokulavasan commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/34#discussion_r100632110
  
    --- Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java ---
    @@ -322,6 +333,28 @@ private long computePruneUpperBound(TimeRegions timeRegions) throws IOException
         return -1;
       }
     
    +  private Map<byte[], Long> handleEmptyRegions(long inactiveTransactionBound,
    +                                               SortedSet<byte[]> transactionalRegions,
    +                                               Map<byte[], Long> pruneUpperBoundRegions) throws IOException {
    +    long inactiveTransactionBoundTime = TxUtils.getTimestamp(inactiveTransactionBound);
    +    SortedSet<byte[]> emptyRegions =
    +      dataJanitorState.getEmptyRegionsAfterTime(inactiveTransactionBoundTime, transactionalRegions);
    +    LOG.debug("Got empty transactional regions for inactive transaction bound time {}: {}",
    +              inactiveTransactionBoundTime, Iterables.transform(emptyRegions, TimeRegions.BYTE_ARR_TO_STRING_FN));
    +
    +    // The regions that are recorded as empty after inactiveTransactionBoundTime will not have invalid data
    +    // for transactions started before or on inactiveTransactionBoundTime. Hence we can consider the prune upper bound
    +    // for these empty regions as inactiveTransactionBound
    +    Map<byte[], Long> pubWithEmptyRegions = new TreeMap<>(Bytes.BYTES_COMPARATOR);
    +    pubWithEmptyRegions.putAll(pruneUpperBoundRegions);
    +    for (byte[] emptyRegion : emptyRegions) {
    +      if (!pruneUpperBoundRegions.containsKey(emptyRegion)) {
    --- End diff --
    
    Shouldn't this condition be reversed? If the region is empty, set the upper bound for it to be ``inactiveTransactionBound`` .


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

Posted by anew <gi...@git.apache.org>.
Github user anew commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/34#discussion_r100608042
  
    --- Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/DataJanitorState.java ---
    @@ -366,6 +371,56 @@ public void deleteInactiveTransactionBoundsOnOrBeforeTime(long time) throws IOEx
         }
       }
     
    +  // --------------------------------------------------------
    +  // ------- Methods for empty regions at a given time -------
    +  // --------------------------------------------------------
    +  // Key: 0x4<time><region-id>
    +  // Col 'e': <empty byte array>
    +  // --------------------------------------------------------
    +
    +  /**
    +   * Save the given region as empty as of the given time.
    +   *
    +   * @param time time in milliseconds
    +   * @param regionId region id
    +   */
    +  public void saveEmptyRegionForTime(long time, byte[] regionId) throws IOException {
    +    byte[] timeBytes = Bytes.toBytes(time);
    +    try (Table stateTable = stateTableSupplier.get()) {
    +      Put put = new Put(makeEmptyRegionTimeKey(timeBytes, regionId));
    +      put.addColumn(FAMILY, EMPTY_REGION_TIME_COL, EMPTY_BYTE_ARRAY);
    --- End diff --
    
    maybe use a value that is visible to the human eye? Just thinking if you debug and scan the table, a value of "X" is easier to spot than an empty value.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

Posted by gokulavasan <gi...@git.apache.org>.
Github user gokulavasan commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/34#discussion_r100632563
  
    --- Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java ---
    @@ -322,6 +333,28 @@ private long computePruneUpperBound(TimeRegions timeRegions) throws IOException
         return -1;
       }
     
    +  private Map<byte[], Long> handleEmptyRegions(long inactiveTransactionBound,
    +                                               SortedSet<byte[]> transactionalRegions,
    +                                               Map<byte[], Long> pruneUpperBoundRegions) throws IOException {
    +    long inactiveTransactionBoundTime = TxUtils.getTimestamp(inactiveTransactionBound);
    +    SortedSet<byte[]> emptyRegions =
    +      dataJanitorState.getEmptyRegionsAfterTime(inactiveTransactionBoundTime, transactionalRegions);
    +    LOG.debug("Got empty transactional regions for inactive transaction bound time {}: {}",
    +              inactiveTransactionBoundTime, Iterables.transform(emptyRegions, TimeRegions.BYTE_ARR_TO_STRING_FN));
    +
    +    // The regions that are recorded as empty after inactiveTransactionBoundTime will not have invalid data
    +    // for transactions started before or on inactiveTransactionBoundTime. Hence we can consider the prune upper bound
    +    // for these empty regions as inactiveTransactionBound
    +    Map<byte[], Long> pubWithEmptyRegions = new TreeMap<>(Bytes.BYTES_COMPARATOR);
    +    pubWithEmptyRegions.putAll(pruneUpperBoundRegions);
    +    for (byte[] emptyRegion : emptyRegions) {
    +      if (!pruneUpperBoundRegions.containsKey(emptyRegion)) {
    +        pubWithEmptyRegions.put(emptyRegion, inactiveTransactionBound);
    --- End diff --
    
    Shouldn't this condition be reversed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

Posted by anew <gi...@git.apache.org>.
Github user anew commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/34#discussion_r100603640
  
    --- Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java ---
    @@ -288,23 +288,27 @@ private long computePruneUpperBound(TimeRegions timeRegions) throws IOException
           SortedSet<byte[]> transactionalRegions = timeRegions.getRegions();
           long time = timeRegions.getTime();
     
    +      long inactiveTransactionBound = dataJanitorState.getInactiveTransactionBoundForTime(time);
    +      LOG.debug("Got inactive transaction bound {}", inactiveTransactionBound);
    +      // If inactiveTransactionBound is not recorded then that means the data is not complete for these regions
    +      if (inactiveTransactionBound == -1) {
    +        if (LOG.isDebugEnabled()) {
    --- End diff --
    
    is this if necessary?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

Posted by gokulavasan <gi...@git.apache.org>.
Github user gokulavasan commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/34#discussion_r100868455
  
    --- Diff: tephra-hbase-compat-0.96/src/test/java/org/apache/tephra/hbase/txprune/InvalidListPruneTest.java ---
    @@ -341,6 +310,87 @@ public HTableInterface get() throws IOException {
         }
       }
     
    +  @Test
    +  public void testPruneEmptyTable() throws Exception {
    +    // Make sure that empty tables do not block the progress of pruning
    +
    +    // Create an empty table
    +    TableName txEmptyTable = TableName.valueOf("emptyPruneTestTable");
    +    HTable emptyHTable = createTable(txEmptyTable.getName(), new byte[][]{family}, false,
    +                                     Collections.singletonList(TestTransactionProcessor.class.getName()));
    +
    +    TransactionPruningPlugin transactionPruningPlugin = new TestTransactionPruningPlugin();
    +    transactionPruningPlugin.initialize(conf);
    +
    +    try {
    +      long now1 = System.currentTimeMillis();
    +      long inactiveTxTimeNow1 = (now1 - 150) * TxConstants.MAX_TX_PER_MS;
    +      long noPruneUpperBound = -1;
    +      long expectedPruneUpperBound1 = (now1 - 200) * TxConstants.MAX_TX_PER_MS;
    +      InMemoryTransactionStateCache.setTransactionSnapshot(
    +        new TransactionSnapshot(expectedPruneUpperBound1, expectedPruneUpperBound1, expectedPruneUpperBound1,
    +                                ImmutableSet.of(expectedPruneUpperBound1),
    +                                ImmutableSortedMap.<Long, TransactionManager.InProgressTx>of()));
    +      testUtil.compact(txEmptyTable, true);
    +      testUtil.compact(txDataTable1, true);
    +      // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
    +      TimeUnit.SECONDS.sleep(2);
    +
    +      // fetch prune upper bound, there should be no prune upper bound since txEmptyTable cannot be compacted
    +      long pruneUpperBound1 = transactionPruningPlugin.fetchPruneUpperBound(now1, inactiveTxTimeNow1);
    +      Assert.assertEquals(noPruneUpperBound, pruneUpperBound1);
    +      transactionPruningPlugin.pruneComplete(now1, noPruneUpperBound);
    +
    +      // Now flush the empty table, this will record the table region as empty, and then pruning will continue
    +      testUtil.flush(txEmptyTable);
    +      // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
    +      TimeUnit.SECONDS.sleep(2);
    +
    +      // fetch prune upper bound, again, this time it should work
    +      pruneUpperBound1 = transactionPruningPlugin.fetchPruneUpperBound(now1, inactiveTxTimeNow1);
    +      Assert.assertEquals(expectedPruneUpperBound1, pruneUpperBound1);
    +      transactionPruningPlugin.pruneComplete(now1, expectedPruneUpperBound1);
    +
    +      // Now add some data to the empty table
    +      // (adding data non-transactionally is okay too, we just need some data for the compaction to run)
    +      emptyHTable.put(new Put(Bytes.toBytes(1)).add(family, qualifier, Bytes.toBytes(1)));
    +      emptyHTable.close();
    +
    +      // Now run another compaction on txDataTable1 with an updated tx snapshot
    +      long now2 = System.currentTimeMillis();
    +      long inactiveTxTimeNow2 = (now2 - 150) * TxConstants.MAX_TX_PER_MS;
    +      long expectedPruneUpperBound2 = (now2 - 200) * TxConstants.MAX_TX_PER_MS;
    +      InMemoryTransactionStateCache.setTransactionSnapshot(
    +        new TransactionSnapshot(expectedPruneUpperBound2, expectedPruneUpperBound2, expectedPruneUpperBound2,
    +                                ImmutableSet.of(expectedPruneUpperBound2),
    +                                ImmutableSortedMap.<Long, TransactionManager.InProgressTx>of()));
    +      testUtil.flush(txEmptyTable);
    +      testUtil.compact(txDataTable1, true);
    +      // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
    +      TimeUnit.SECONDS.sleep(2);
    +
    +      // Running a prune now should still return min(inactiveTxTimeNow1, expectedPruneUpperBound1) since
    +      // txEmptyTable is no longer empty. This information is returned since the txEmptyTable was recorded as being
    +      // empty in the previous run with inactiveTxTimeNow1
    +      long pruneUpperBound2 = transactionPruningPlugin.fetchPruneUpperBound(now2, inactiveTxTimeNow2);
    +      Assert.assertEquals(inactiveTxTimeNow1, pruneUpperBound2);
    +      transactionPruningPlugin.pruneComplete(now2, expectedPruneUpperBound1);
    +
    +      // However, after compacting txEmptyTable we should get the latest upper bound
    +      testUtil.flush(txEmptyTable);
    +      testUtil.compact(txEmptyTable, true);
    +      // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
    +      TimeUnit.SECONDS.sleep(2);
    +      pruneUpperBound2 = transactionPruningPlugin.fetchPruneUpperBound(now2, inactiveTxTimeNow2);
    +      Assert.assertEquals(expectedPruneUpperBound2, pruneUpperBound2);
    +      transactionPruningPlugin.pruneComplete(now2, expectedPruneUpperBound2);
    +    } finally {
    +      transactionPruningPlugin.destroy();
    +      hBaseAdmin.disableTable(txEmptyTable);
    +      hBaseAdmin.deleteTable(txEmptyTable);
    +    }
    +    }
    --- End diff --
    
    Fix alignment


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra issue #34: TEPHRA-216 Handle empty transactional regions du...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on the issue:

    https://github.com/apache/incubator-tephra/pull/34
  
    Thanks @anew. I will squash the commits and then port changes to other compat modules.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

Posted by gokulavasan <gi...@git.apache.org>.
Github user gokulavasan commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/34#discussion_r100655264
  
    --- Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java ---
    @@ -322,6 +333,28 @@ private long computePruneUpperBound(TimeRegions timeRegions) throws IOException
         return -1;
       }
     
    +  private Map<byte[], Long> handleEmptyRegions(long inactiveTransactionBound,
    +                                               SortedSet<byte[]> transactionalRegions,
    +                                               Map<byte[], Long> pruneUpperBoundRegions) throws IOException {
    +    long inactiveTransactionBoundTime = TxUtils.getTimestamp(inactiveTransactionBound);
    +    SortedSet<byte[]> emptyRegions =
    +      dataJanitorState.getEmptyRegionsAfterTime(inactiveTransactionBoundTime, transactionalRegions);
    +    LOG.debug("Got empty transactional regions for inactive transaction bound time {}: {}",
    +              inactiveTransactionBoundTime, Iterables.transform(emptyRegions, TimeRegions.BYTE_ARR_TO_STRING_FN));
    +
    +    // The regions that are recorded as empty after inactiveTransactionBoundTime will not have invalid data
    +    // for transactions started before or on inactiveTransactionBoundTime. Hence we can consider the prune upper bound
    +    // for these empty regions as inactiveTransactionBound
    +    Map<byte[], Long> pubWithEmptyRegions = new TreeMap<>(Bytes.BYTES_COMPARATOR);
    +    pubWithEmptyRegions.putAll(pruneUpperBoundRegions);
    +    for (byte[] emptyRegion : emptyRegions) {
    +      if (!pruneUpperBoundRegions.containsKey(emptyRegion)) {
    +        pubWithEmptyRegions.put(emptyRegion, inactiveTransactionBound);
    --- End diff --
    
    Ah I see sorry misunderstood the comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #34: TEPHRA-216 Handle empty transactional reg...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/34#discussion_r100638050
  
    --- Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java ---
    @@ -288,23 +288,27 @@ private long computePruneUpperBound(TimeRegions timeRegions) throws IOException
           SortedSet<byte[]> transactionalRegions = timeRegions.getRegions();
           long time = timeRegions.getTime();
     
    +      long inactiveTransactionBound = dataJanitorState.getInactiveTransactionBoundForTime(time);
    +      LOG.debug("Got inactive transaction bound {}", inactiveTransactionBound);
    +      // If inactiveTransactionBound is not recorded then that means the data is not complete for these regions
    +      if (inactiveTransactionBound == -1) {
    +        if (LOG.isDebugEnabled()) {
    --- End diff --
    
    You are right, it is not necessary. I'll remove it


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra issue #34: TEPHRA-216 Handle empty transactional regions du...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on the issue:

    https://github.com/apache/incubator-tephra/pull/34
  
    @gokulavasan I have ported the changes to other compat modules, please take a look.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---