Handle failures in parallelAllSSTableOperation (cleanup/upgradesstables/etc)
Patch by marcuse; reviewed by Benedict Elliott Smith for CASSANDRA-14657 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/9be43706 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/9be43706 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/9be43706 Branch: refs/heads/trunk Commit: 9be437064f5348fe7f8fc6665b747ad751699f49 Parents: 21ec39a Author: Marcus Eriksson <marc...@apache.org> Authored: Mon Aug 20 11:06:03 2018 +0200 Committer: Marcus Eriksson <marc...@apache.org> Committed: Fri Sep 14 08:42:28 2018 +0200 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../cassandra/db/compaction/CompactionManager.java | 14 +++++++++++--- 2 files changed, 12 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/9be43706/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index d55ddb6..12c16b7 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.18 + * Handle failures in parallelAllSSTableOperation (cleanup/upgradesstables/etc) (CASSANDRA-14657) * Improve TokenMetaData cache populating performance avoid long locking (CASSANDRA-14660) * Fix static column order for SELECT * wildcard queries (CASSANDRA-14638) * sstableloader should use discovered broadcast address to connect intra-cluster (CASSANDRA-14522) http://git-wip-us.apache.org/repos/asf/cassandra/blob/9be43706/src/java/org/apache/cassandra/db/compaction/CompactionManager.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java index f033bf2..5813f32 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java @@ -292,6 +292,7 @@ public class CompactionManager implements CompactionManagerMBean private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs, OperationType operationType) throws ExecutionException, InterruptedException { List<LifecycleTransaction> transactions = new ArrayList<>(); + List<Future<?>> futures = new ArrayList<>(); try (LifecycleTransaction compacting = cfs.markAllCompacting(operationType)) { if (compacting == null) @@ -304,8 +305,6 @@ public class CompactionManager implements CompactionManagerMBean return AllSSTableOpStatus.SUCCESSFUL; } - List<Future<?>> futures = new ArrayList<>(); - for (final SSTableReader sstable : sstables) { final LifecycleTransaction txn = compacting.split(singleton(sstable)); @@ -337,9 +336,18 @@ public class CompactionManager implements CompactionManagerMBean } finally { + // wait on any unfinished futures to make sure we don't close an ongoing transaction + try + { + FBUtilities.waitOnFutures(futures); + } + catch (Throwable t) + { + // these are handled/logged in CompactionExecutor#afterExecute + } Throwable fail = Throwables.close(null, transactions); if (fail != null) - logger.error("Failed to cleanup lifecycle transactions {}", fail); + logger.error("Failed to cleanup lifecycle transactions", fail); } } --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org