Uploaded image for project: 'Tephra'
  1. Tephra
  2. TEPHRA-244

Invalid tx pruning does not handle deletion of tables well

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: 0.12.0-incubating
    • Fix Version/s: 0.13.0-incubating
    • Component/s: core, manager
    • Labels:
      None

      Description

      Suppose an application regularly creates and deletes tables (for example, temporary tables). In such a scenario, there will always be such a temporary table when pruning runs, and its regions will be recorded for that time. However, the region will be deleted before it ever compacts or flushes, and it will never record prune information. Because all prune times have such regions, there will never be a set of transactional regions that all have prune info, and pruning will never happen.

      The fix is to exclude deleted tables from the list of regions at each time. This is all regions of deleted tables (tables that do not exist any more), not deleted regions: a region may disappear due to a split and its data will be in new regions.

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/incubator-tephra/pull/55
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user poornachandra commented on a diff in the pull request:

          https://github.com/apache/incubator-tephra/pull/55#discussion_r137981943

          — Diff: tephra-hbase-compat-1.1-base/src/test/resources/logback-test.xml —
          @@ -0,0 +1,39 @@
          +<?xml version="1.0" encoding="UTF-8"?>
          +<!--
          + ~ 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.
          + -->
          +
          +<configuration>
          + <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
          + <encoder>
          + <pattern>%d

          {ISO8601}

          - %-5p [%t:%C

          {1}

          @%L] - %m%n</pattern>
          + </encoder>
          + </appender>
          +
          + <logger name="org.apache.hadoop" level="WARN" />
          + <!-- BlockStateChange is used by org.apache.hadoop.hdfs.server.blockmanagement.BlockManager -->
          — End diff –

          I'll copy over this file to the other compat modules

          Show
          githubbot ASF GitHub Bot added a comment - Github user poornachandra commented on a diff in the pull request: https://github.com/apache/incubator-tephra/pull/55#discussion_r137981943 — Diff: tephra-hbase-compat-1.1-base/src/test/resources/logback-test.xml — @@ -0,0 +1,39 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ 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. + --> + +<configuration> + <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern>%d {ISO8601} - %-5p [%t:%C {1} @%L] - %m%n</pattern> + </encoder> + </appender> + + <logger name="org.apache.hadoop" level="WARN" /> + <!-- BlockStateChange is used by org.apache.hadoop.hdfs.server.blockmanagement.BlockManager --> — End diff – I'll copy over this file to the other compat modules
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user poornachandra commented on a diff in the pull request:

          https://github.com/apache/incubator-tephra/pull/55#discussion_r137981906

          — Diff: tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/txprune/InvalidListPruneTest.java —
          @@ -384,7 +384,67 @@ public void testPruneEmptyTable() throws Exception

          { hBaseAdmin.disableTable(txEmptyTable); hBaseAdmin.deleteTable(txEmptyTable); }

          + }
          +
          + @Test
          + public void testPruneTransientTable() throws Exception {
          + // Make sure that transient tables do not block the progress of pruning
          +
          + // Create a temp table
          + TableName txTempTable = TableName.valueOf("tempTable");
          + createTable(txTempTable.getName(), new byte[][]

          {family}, false,
          + Collections.singletonList(TestTransactionProcessor.class.getName()));
          +
          + TableName txDataTable2 = null;
          +
          + 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()));
          +
          + // fetch prune upper bound, there should be no prune upper bound since nothing has been compacted yet.
          + // This run is only to store the initial set of regions
          + long pruneUpperBound1 = transactionPruningPlugin.fetchPruneUpperBound(now1, inactiveTxTimeNow1);
          + Assert.assertEquals(noPruneUpperBound, pruneUpperBound1);
          + transactionPruningPlugin.pruneComplete(now1, noPruneUpperBound);
          +
          + // Now delete the transient table
          + hBaseAdmin.disableTable(txTempTable);
          + hBaseAdmin.deleteTable(txTempTable);
          +
          + // Compact the data table now
          + 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);
          +
          + // Create a new table that will not be compacted
          + txDataTable2 = TableName.valueOf("invalidListPruneTestTable2");
          + createTable(txDataTable2.getName(), new byte[][]{family}

          , false,
          + Collections.singletonList(TestTransactionProcessor.class.getName()));
          +
          + // fetch prune upper bound, there should be a prune upper bound even though txTempTable does not exist anymore,
          + // and txDataTable2 has not been compacted/flushed yet
          + long now2 = System.currentTimeMillis();
          + long inactiveTxTimeNow2 = (now1 - 150) * TxConstants.MAX_TX_PER_MS;
          + long pruneUpperBound2 = transactionPruningPlugin.fetchPruneUpperBound(now2, inactiveTxTimeNow2);
          + Assert.assertEquals(expectedPruneUpperBound1, pruneUpperBound2);
          + transactionPruningPlugin.pruneComplete(now2, expectedPruneUpperBound1);
          + } finally {
          + transactionPruningPlugin.destroy();
          + if (txDataTable2 != null) {
          — End diff –

          👍

          Show
          githubbot ASF GitHub Bot added a comment - Github user poornachandra commented on a diff in the pull request: https://github.com/apache/incubator-tephra/pull/55#discussion_r137981906 — Diff: tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/txprune/InvalidListPruneTest.java — @@ -384,7 +384,67 @@ public void testPruneEmptyTable() throws Exception { hBaseAdmin.disableTable(txEmptyTable); hBaseAdmin.deleteTable(txEmptyTable); } + } + + @Test + public void testPruneTransientTable() throws Exception { + // Make sure that transient tables do not block the progress of pruning + + // Create a temp table + TableName txTempTable = TableName.valueOf("tempTable"); + createTable(txTempTable.getName(), new byte[][] {family}, false, + Collections.singletonList(TestTransactionProcessor.class.getName())); + + TableName txDataTable2 = null; + + 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())); + + // fetch prune upper bound, there should be no prune upper bound since nothing has been compacted yet. + // This run is only to store the initial set of regions + long pruneUpperBound1 = transactionPruningPlugin.fetchPruneUpperBound(now1, inactiveTxTimeNow1); + Assert.assertEquals(noPruneUpperBound, pruneUpperBound1); + transactionPruningPlugin.pruneComplete(now1, noPruneUpperBound); + + // Now delete the transient table + hBaseAdmin.disableTable(txTempTable); + hBaseAdmin.deleteTable(txTempTable); + + // Compact the data table now + 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); + + // Create a new table that will not be compacted + txDataTable2 = TableName.valueOf("invalidListPruneTestTable2"); + createTable(txDataTable2.getName(), new byte[][]{family} , false, + Collections.singletonList(TestTransactionProcessor.class.getName())); + + // fetch prune upper bound, there should be a prune upper bound even though txTempTable does not exist anymore, + // and txDataTable2 has not been compacted/flushed yet + long now2 = System.currentTimeMillis(); + long inactiveTxTimeNow2 = (now1 - 150) * TxConstants.MAX_TX_PER_MS; + long pruneUpperBound2 = transactionPruningPlugin.fetchPruneUpperBound(now2, inactiveTxTimeNow2); + Assert.assertEquals(expectedPruneUpperBound1, pruneUpperBound2); + transactionPruningPlugin.pruneComplete(now2, expectedPruneUpperBound1); + } finally { + transactionPruningPlugin.destroy(); + if (txDataTable2 != null) { — End diff – 👍
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user anew commented on a diff in the pull request:

          https://github.com/apache/incubator-tephra/pull/55#discussion_r137981190

          — Diff: tephra-hbase-compat-1.1-base/src/test/resources/logback-test.xml —
          @@ -0,0 +1,39 @@
          +<?xml version="1.0" encoding="UTF-8"?>
          +<!--
          + ~ 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.
          + -->
          +
          +<configuration>
          + <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
          + <encoder>
          + <pattern>%d

          {ISO8601}

          - %-5p [%t:%C

          {1}

          @%L] - %m%n</pattern>
          + </encoder>
          + </appender>
          +
          + <logger name="org.apache.hadoop" level="WARN" />
          + <!-- BlockStateChange is used by org.apache.hadoop.hdfs.server.blockmanagement.BlockManager -->
          — End diff –

          seems like this would be useful for all test cases... it removes a lot of distracting log messages.

          Show
          githubbot ASF GitHub Bot added a comment - Github user anew commented on a diff in the pull request: https://github.com/apache/incubator-tephra/pull/55#discussion_r137981190 — Diff: tephra-hbase-compat-1.1-base/src/test/resources/logback-test.xml — @@ -0,0 +1,39 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ 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. + --> + +<configuration> + <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern>%d {ISO8601} - %-5p [%t:%C {1} @%L] - %m%n</pattern> + </encoder> + </appender> + + <logger name="org.apache.hadoop" level="WARN" /> + <!-- BlockStateChange is used by org.apache.hadoop.hdfs.server.blockmanagement.BlockManager --> — End diff – seems like this would be useful for all test cases... it removes a lot of distracting log messages.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user anew commented on a diff in the pull request:

          https://github.com/apache/incubator-tephra/pull/55#discussion_r137981077

          — Diff: tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/txprune/InvalidListPruneTest.java —
          @@ -384,7 +384,67 @@ public void testPruneEmptyTable() throws Exception

          { hBaseAdmin.disableTable(txEmptyTable); hBaseAdmin.deleteTable(txEmptyTable); }

          + }
          +
          + @Test
          + public void testPruneTransientTable() throws Exception {
          + // Make sure that transient tables do not block the progress of pruning
          +
          + // Create a temp table
          + TableName txTempTable = TableName.valueOf("tempTable");
          + createTable(txTempTable.getName(), new byte[][]

          {family}, false,
          + Collections.singletonList(TestTransactionProcessor.class.getName()));
          +
          + TableName txDataTable2 = null;
          +
          + 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()));
          +
          + // fetch prune upper bound, there should be no prune upper bound since nothing has been compacted yet.
          + // This run is only to store the initial set of regions
          + long pruneUpperBound1 = transactionPruningPlugin.fetchPruneUpperBound(now1, inactiveTxTimeNow1);
          + Assert.assertEquals(noPruneUpperBound, pruneUpperBound1);
          + transactionPruningPlugin.pruneComplete(now1, noPruneUpperBound);
          +
          + // Now delete the transient table
          + hBaseAdmin.disableTable(txTempTable);
          + hBaseAdmin.deleteTable(txTempTable);
          +
          + // Compact the data table now
          + 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);
          +
          + // Create a new table that will not be compacted
          + txDataTable2 = TableName.valueOf("invalidListPruneTestTable2");
          + createTable(txDataTable2.getName(), new byte[][]{family}

          , false,
          + Collections.singletonList(TestTransactionProcessor.class.getName()));
          +
          + // fetch prune upper bound, there should be a prune upper bound even though txTempTable does not exist anymore,
          + // and txDataTable2 has not been compacted/flushed yet
          + long now2 = System.currentTimeMillis();
          + long inactiveTxTimeNow2 = (now1 - 150) * TxConstants.MAX_TX_PER_MS;
          + long pruneUpperBound2 = transactionPruningPlugin.fetchPruneUpperBound(now2, inactiveTxTimeNow2);
          + Assert.assertEquals(expectedPruneUpperBound1, pruneUpperBound2);
          + transactionPruningPlugin.pruneComplete(now2, expectedPruneUpperBound1);
          + } finally {
          + transactionPruningPlugin.destroy();
          + if (txDataTable2 != null) {
          — End diff –

          you should also do this for txTempTable. The test case deletes it, but if it fails before that, the table never gets deleted.

          Show
          githubbot ASF GitHub Bot added a comment - Github user anew commented on a diff in the pull request: https://github.com/apache/incubator-tephra/pull/55#discussion_r137981077 — Diff: tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/txprune/InvalidListPruneTest.java — @@ -384,7 +384,67 @@ public void testPruneEmptyTable() throws Exception { hBaseAdmin.disableTable(txEmptyTable); hBaseAdmin.deleteTable(txEmptyTable); } + } + + @Test + public void testPruneTransientTable() throws Exception { + // Make sure that transient tables do not block the progress of pruning + + // Create a temp table + TableName txTempTable = TableName.valueOf("tempTable"); + createTable(txTempTable.getName(), new byte[][] {family}, false, + Collections.singletonList(TestTransactionProcessor.class.getName())); + + TableName txDataTable2 = null; + + 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())); + + // fetch prune upper bound, there should be no prune upper bound since nothing has been compacted yet. + // This run is only to store the initial set of regions + long pruneUpperBound1 = transactionPruningPlugin.fetchPruneUpperBound(now1, inactiveTxTimeNow1); + Assert.assertEquals(noPruneUpperBound, pruneUpperBound1); + transactionPruningPlugin.pruneComplete(now1, noPruneUpperBound); + + // Now delete the transient table + hBaseAdmin.disableTable(txTempTable); + hBaseAdmin.deleteTable(txTempTable); + + // Compact the data table now + 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); + + // Create a new table that will not be compacted + txDataTable2 = TableName.valueOf("invalidListPruneTestTable2"); + createTable(txDataTable2.getName(), new byte[][]{family} , false, + Collections.singletonList(TestTransactionProcessor.class.getName())); + + // fetch prune upper bound, there should be a prune upper bound even though txTempTable does not exist anymore, + // and txDataTable2 has not been compacted/flushed yet + long now2 = System.currentTimeMillis(); + long inactiveTxTimeNow2 = (now1 - 150) * TxConstants.MAX_TX_PER_MS; + long pruneUpperBound2 = transactionPruningPlugin.fetchPruneUpperBound(now2, inactiveTxTimeNow2); + Assert.assertEquals(expectedPruneUpperBound1, pruneUpperBound2); + transactionPruningPlugin.pruneComplete(now2, expectedPruneUpperBound1); + } finally { + transactionPruningPlugin.destroy(); + if (txDataTable2 != null) { — End diff – you should also do this for txTempTable. The test case deletes it, but if it fails before that, the table never gets deleted.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user anew commented on a diff in the pull request:

          https://github.com/apache/incubator-tephra/pull/55#discussion_r137980868

          — Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java —
          @@ -335,6 +351,25 @@ private long computePruneUpperBound(TimeRegions timeRegions) throws IOException
          return -1;
          }

          + private SortedSet<byte[]> filterDeletedRegions(final Set<TableName> existingTables,
          — End diff –

          Better name this filterDeletedTables(), or filterDeletedTableRegions()

          • it does not filter out deleted regions
          • it does filter out all regions of deleted tables
          Show
          githubbot ASF GitHub Bot added a comment - Github user anew commented on a diff in the pull request: https://github.com/apache/incubator-tephra/pull/55#discussion_r137980868 — Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java — @@ -335,6 +351,25 @@ private long computePruneUpperBound(TimeRegions timeRegions) throws IOException return -1; } + private SortedSet<byte[]> filterDeletedRegions(final Set<TableName> existingTables, — End diff – Better name this filterDeletedTables(), or filterDeletedTableRegions() it does not filter out deleted regions it does filter out all regions of deleted tables
          Hide
          poornachandra Poorna Chandra added a comment -

          https://github.com/apache/incubator-tephra/pull/55 - This PR just removes regions of deleted tables when computing prune upper bound. This ensures that transient tables do not block pruning. However, if a transient table is created/deleted again with the same name and never gets a chance to be compacted, then it has a potential to block pruning. Filed TEPHRA-254 to handle such cases.

          Show
          poornachandra Poorna Chandra added a comment - https://github.com/apache/incubator-tephra/pull/55 - This PR just removes regions of deleted tables when computing prune upper bound. This ensures that transient tables do not block pruning. However, if a transient table is created/deleted again with the same name and never gets a chance to be compacted, then it has a potential to block pruning. Filed TEPHRA-254 to handle such cases.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user poornachandra opened a pull request:

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

          TEPHRA-244 Remove regions of deleted tables when computing prune upper bound

          JIRA - https://issues.apache.org/jira/browse/TEPHRA-244

          This PR just removes regions of deleted tables when computing prune upper bound. This ensures that transient tables do not block pruning. However, if a transient table is created/deleted again with the same name and never gets a chance to be compacted, then it has a potential to block pruning.

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

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

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

          https://github.com/apache/incubator-tephra/pull/55.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 #55


          commit ddfed1bf9f764cf2a29d900ed55f283b45861d02
          Author: poorna <poorna@cask.co>
          Date: 2017-09-11T04:38:08Z

          TEPHRA-244 Remove regions of deleted tables when computing prune upper bound


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user poornachandra opened a pull request: https://github.com/apache/incubator-tephra/pull/55 TEPHRA-244 Remove regions of deleted tables when computing prune upper bound JIRA - https://issues.apache.org/jira/browse/TEPHRA-244 This PR just removes regions of deleted tables when computing prune upper bound. This ensures that transient tables do not block pruning. However, if a transient table is created/deleted again with the same name and never gets a chance to be compacted, then it has a potential to block pruning. You can merge this pull request into a Git repository by running: $ git pull https://github.com/poornachandra/incubator-tephra feature/prune-transient-tables Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-tephra/pull/55.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 #55 commit ddfed1bf9f764cf2a29d900ed55f283b45861d02 Author: poorna <poorna@cask.co> Date: 2017-09-11T04:38:08Z TEPHRA-244 Remove regions of deleted tables when computing prune upper bound

            People

            • Assignee:
              poornachandra Poorna Chandra
              Reporter:
              anew Andreas Neumann
            • Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development