This is an automated email from the ASF dual-hosted git repository.

kturner pushed a commit to branch elasticity
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/elasticity by this push:
     new 6b7f6aca3d fixes volume replace when log recovery is needed (#4018)
6b7f6aca3d is described below

commit 6b7f6aca3d690f51ad764a921aa560fdf0013558
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Mon Dec 4 19:14:49 2023 -0500

    fixes volume replace when log recovery is needed (#4018)
    
    When log recovery and volume replace were both needed for a tablet it
    was not working.  Modified TGW to fix this.
    
    Noticed that tablets needing volume replacement were compacting while
    looking into this.  Those compactions were starting and failing.
    Modified TGW to avoid split and compact for tablets needed vol
    replacement.
---
 .../accumulo/manager/TabletGroupWatcher.java       | 45 ++++++++++++----------
 .../java/org/apache/accumulo/test/VolumeIT.java    |  3 --
 2 files changed, 25 insertions(+), 23 deletions(-)

diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
index c507f74369..af4c0b5ff0 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
@@ -433,28 +433,31 @@ abstract class TabletGroupWatcher extends 
AccumuloDaemonThread {
       final TabletGoalState goal =
           TabletGoalState.compute(tm, state, manager.tabletBalancer, 
tableMgmtParams);
 
-      if (actions.contains(ManagementAction.NEEDS_VOLUME_REPLACEMENT)
-          && state == TabletState.UNASSIGNED) {
+      if (actions.contains(ManagementAction.NEEDS_VOLUME_REPLACEMENT)) {
         tableMgmtStats.totalVolumeReplacements++;
-        var volRep =
-            
VolumeUtil.computeVolumeReplacements(tableMgmtParams.getVolumeReplacements(), 
tm);
-
-        if (volRep.logsToRemove.size() + volRep.filesToRemove.size() > 0) {
-          if (tm.getLocation() != null) {
-            // since the totalVolumeReplacements counter was incremented, 
should try this again
-            // later after its unassigned
-            LOG.debug("Volume replacement needed for {} but it has a location 
{}.", tm.getExtent(),
-                tm.getLocation());
-          } else if (tm.getOperationId() != null) {
-            LOG.debug("Volume replacement needed for {} but it has an active 
operation {}.",
-                tm.getExtent(), tm.getOperationId());
+        if (state == TabletState.UNASSIGNED || state == TabletState.SUSPENDED) 
{
+          var volRep =
+              
VolumeUtil.computeVolumeReplacements(tableMgmtParams.getVolumeReplacements(), 
tm);
+          if (volRep.logsToRemove.size() + volRep.filesToRemove.size() > 0) {
+            if (tm.getLocation() != null) {
+              // since the totalVolumeReplacements counter was incremented, 
should try this again
+              // later after its unassigned
+              LOG.debug("Volume replacement needed for {} but it has a 
location {}.",
+                  tm.getExtent(), tm.getLocation());
+            } else if (tm.getOperationId() != null) {
+              LOG.debug("Volume replacement needed for {} but it has an active 
operation {}.",
+                  tm.getExtent(), tm.getOperationId());
+            } else {
+              LOG.debug("Volume replacement needed for {}.", tm.getExtent());
+              // buffer replacements so that multiple mutations can be done at 
once
+              tLists.volumeReplacements.add(volRep);
+            }
           } else {
-            LOG.debug("Volume replacement needed for {}.", tm.getExtent());
-            // buffer replacements so that multiple mutations can be done at 
once
-            tLists.volumeReplacements.add(volRep);
+            LOG.debug("Volume replacement evaluation for {} returned no 
changes.", tm.getExtent());
           }
         } else {
-          LOG.debug("Volume replacement evaluation for {} returned no 
changes.", tm.getExtent());
+          LOG.debug("Volume replacement needed for {} but its tablet state is 
{}.", tm.getExtent(),
+              state);
         }
       }
 
@@ -482,7 +485,8 @@ abstract class TabletGroupWatcher extends 
AccumuloDaemonThread {
             state, goal, actions);
       }
 
-      if (actions.contains(ManagementAction.NEEDS_SPLITTING)) {
+      if (actions.contains(ManagementAction.NEEDS_SPLITTING)
+          && !actions.contains(ManagementAction.NEEDS_VOLUME_REPLACEMENT)) {
         LOG.debug("{} may need splitting.", tm.getExtent());
         if (manager.getSplitter().isSplittable(tm)) {
           if (manager.getSplitter().addSplitStarting(tm.getExtent())) {
@@ -497,7 +501,8 @@ abstract class TabletGroupWatcher extends 
AccumuloDaemonThread {
         // sendSplitRequest(mergeStats.getMergeInfo(), state, tm);
       }
 
-      if (actions.contains(ManagementAction.NEEDS_COMPACTING)) {
+      if (actions.contains(ManagementAction.NEEDS_COMPACTING)
+          && !actions.contains(ManagementAction.NEEDS_VOLUME_REPLACEMENT)) {
         var jobs = compactionGenerator.generateJobs(tm,
             TabletManagementIterator.determineCompactionKinds(actions));
         LOG.debug("{} may need compacting adding {} jobs", tm.getExtent(), 
jobs.size());
diff --git a/test/src/main/java/org/apache/accumulo/test/VolumeIT.java 
b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
index 5360cecb92..2e7dd8a5b0 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
@@ -87,7 +87,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.io.Text;
 import org.apache.zookeeper.KeeperException.NoNodeException;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
@@ -494,7 +493,6 @@ public class VolumeIT extends ConfigurableMacBase {
   }
 
   @Test
-  @Disabled // ELASTICITY_TODO: Log Recovery is not working yet
   public void testDirtyReplaceVolumes() throws Exception {
     try (AccumuloClient client = 
Accumulo.newClient().from(getClientProperties()).build()) {
       testReplaceVolume(client, false, false);
@@ -509,7 +507,6 @@ public class VolumeIT extends ConfigurableMacBase {
   }
 
   @Test
-  @Disabled // ELASTICITY_TODO: Log Recovery is not working yet
   public void testDirtyReplaceVolumesWithRangedFiles() throws Exception {
     try (AccumuloClient client = 
Accumulo.newClient().from(getClientProperties()).build()) {
       testReplaceVolume(client, false, true);

Reply via email to