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

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


The following commit(s) were added to refs/heads/2.1 by this push:
     new de0bb39ebc fixes verification of walog location in VolumeChooserIT 
(#5923)
de0bb39ebc is described below

commit de0bb39ebc27ba862af7496e0b86f1f7fee1a680
Author: Keith Turner <[email protected]>
AuthorDate: Tue Sep 23 22:02:02 2025 -0400

    fixes verification of walog location in VolumeChooserIT (#5923)
    
    Retrieve the WAL volumes using the WalStateManager to fix VolumeChooserIT
    
    fixes #5917
---
 .../org/apache/accumulo/test/VolumeChooserIT.java  | 47 +++++-----------------
 1 file changed, 9 insertions(+), 38 deletions(-)

diff --git a/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java 
b/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
index a0cba2784c..b56c330fe0 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
@@ -19,8 +19,8 @@
 package org.apache.accumulo.test;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.File;
 import java.time.Duration;
@@ -48,13 +48,13 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
-import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.spi.fs.DelegatingChooser;
 import org.apache.accumulo.core.spi.fs.PreferredVolumeChooser;
 import org.apache.accumulo.core.spi.fs.RandomVolumeChooser;
 import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.log.WalStateManager;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -205,17 +205,6 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     assertEquals(26, fileCount, "Wrong number of files");
   }
 
-  public static void verifyNoVolumes(AccumuloClient accumuloClient, Range 
tableRange)
-      throws Exception {
-    try (Scanner scanner = accumuloClient.createScanner(MetadataTable.NAME, 
Authorizations.EMPTY)) {
-      scanner.setRange(tableRange);
-      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
-      for (Entry<Key,Value> entry : scanner) {
-        fail("Data incorrectly written to " + 
entry.getKey().getColumnQualifier());
-      }
-    }
-  }
-
   private void configureNamespace(AccumuloClient accumuloClient, String 
volumeChooserClassName,
       String configuredVolumes, String namespace) throws Exception {
     accumuloClient.namespaceOperations().create(namespace);
@@ -241,30 +230,6 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     verifyVolumes(accumuloClient, TabletsSection.getRange(tableID), 
expectedVolumes);
   }
 
-  public static void verifyWaLogVolumes(AccumuloClient accumuloClient, Range 
tableRange, String vol)
-      throws TableNotFoundException {
-    // Verify the new files are written to the Volumes specified
-    ArrayList<String> volumes = new ArrayList<>();
-    Collections.addAll(volumes, vol.split(","));
-
-    TreeSet<String> volumesSeen = new TreeSet<>();
-    try (Scanner scanner = accumuloClient.createScanner(MetadataTable.NAME, 
Authorizations.EMPTY)) {
-      scanner.setRange(tableRange);
-      scanner.fetchColumnFamily(LogColumnFamily.NAME);
-      for (Entry<Key,Value> entry : scanner) {
-        boolean inVolume = false;
-        for (String volume : volumes) {
-          if (entry.getKey().getColumnQualifier().toString().contains(volume)) 
{
-            volumesSeen.add(volume);
-          }
-          inVolume = true;
-        }
-        assertTrue(inVolume,
-            "Data not written to the correct volumes.  " + 
entry.getKey().getColumnQualifier());
-      }
-    }
-  }
-
   // Test that uses two tables with 10 split points each. They each use the 
PreferredVolumeChooser
   // to choose volumes.
   @Test
@@ -350,7 +315,13 @@ public class VolumeChooserIT extends ConfigurableMacBase {
       VolumeChooserIT.addSplits(client, tableName);
       VolumeChooserIT.writeDataToTable(client, tableName, alpha_rows);
       // should only go to v2 as per configuration in configure()
-      VolumeChooserIT.verifyWaLogVolumes(client, new Range(), v2.toString());
+      var walMgr = new WalStateManager(getServerContext());
+      Map<Path,WalStateManager.WalState> allLogs = walMgr.getAllState();
+      assertFalse(allLogs.isEmpty());
+      String volume = v2.toString();
+      for (var path : allLogs.keySet()) {
+        assertTrue(path.toString().contains(volume), () -> path + " did not 
contain " + volume);
+      }
     }
   }
 }

Reply via email to