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 6e0f2a14fa fixes continue scan in tablet mgmt iterator (#4457)
6e0f2a14fa is described below

commit 6e0f2a14fa051adb2c7873877bc59e1bcb0ad0b8
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Mon Apr 15 11:25:48 2024 -0400

    fixes continue scan in tablet mgmt iterator (#4457)
    
    Many places in the accumulo code will read a batch of key/values and
    then use the last key in the batch to construct a range to get the next
    batch.  The last key in the batch will be a non inclusive start key for
    the range.  The tablet mgmt iterator was not handling this case and
    returning the key that should have been excluded.
---
 .../manager/state/TabletManagementIterator.java    |  6 ++-
 .../functional/TabletManagementIteratorIT.java     | 51 +++++++++++++++++-----
 2 files changed, 44 insertions(+), 13 deletions(-)

diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementIterator.java
 
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementIterator.java
index b3ebe61c1d..3f5397a7a4 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementIterator.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementIterator.java
@@ -217,7 +217,11 @@ public class TabletManagementIterator extends 
SkippingIterator {
           // can pull this K,V pair from the results by looking at the colf.
           TabletManagement.addActions(decodedRow, actions);
         }
-        topKey = decodedRow.firstKey();
+
+        // This key is being created exactly the same way as the whole row 
iterator creates keys.
+        // This is important for ensuring that seek works as expected in the 
continue case. See
+        // WholeRowIterator seek function for details, it looks for keys w/o 
columns.
+        topKey = new Key(decodedRow.firstKey().getRow());
         topValue = WholeRowIterator.encodeRow(new 
ArrayList<>(decodedRow.keySet()),
             new ArrayList<>(decodedRow.values()));
         LOG.trace("Returning extent {} with reasons: {}", tm.getExtent(), 
actions);
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java
 
b/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java
index 6d74d0c16e..7af3e0ba00 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java
@@ -18,8 +18,13 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static 
org.apache.accumulo.core.manager.state.TabletManagement.ManagementAction.BAD_STATE;
 import static 
org.apache.accumulo.core.manager.state.TabletManagement.ManagementAction.NEEDS_LOCATION_UPDATE;
+import static 
org.apache.accumulo.core.manager.state.TabletManagement.ManagementAction.NEEDS_RECOVERY;
+import static 
org.apache.accumulo.core.manager.state.TabletManagement.ManagementAction.NEEDS_SPLITTING;
+import static 
org.apache.accumulo.core.manager.state.TabletManagement.ManagementAction.NEEDS_VOLUME_REPLACEMENT;
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 import java.time.Duration;
@@ -187,6 +192,10 @@ public class TabletManagementIteratorIT extends 
AccumuloClusterHarness {
       assertEquals(expected, findTabletsNeedingAttention(client, metaCopy1, 
tabletMgmtParams),
           "Should have four tablets with hosting availability changes");
 
+      // test continue scan functionality, this test needs a table and tablet 
mgmt params that will
+      // return more than one tablet
+      testContinueScan(client, metaCopy1, tabletMgmtParams);
+
       // test the assigned case (no location)
       removeLocation(client, metaCopy1, t3);
       expected =
@@ -210,18 +219,15 @@ public class TabletManagementIteratorIT extends 
AccumuloClusterHarness {
       // Test the recovery cases
       createLogEntry(client, metaCopy5, t1);
       setTabletAvailability(client, metaCopy5, t1, 
TabletAvailability.UNHOSTED.name());
-      expected = Map.of(endR1,
-          Set.of(NEEDS_LOCATION_UPDATE, 
TabletManagement.ManagementAction.NEEDS_RECOVERY));
+      expected = Map.of(endR1, Set.of(NEEDS_LOCATION_UPDATE, NEEDS_RECOVERY));
       assertEquals(expected, findTabletsNeedingAttention(client, metaCopy5, 
tabletMgmtParams),
           "Only 1 of 2 tablets in table t1 should be returned");
       setTabletAvailability(client, metaCopy5, t1, 
TabletAvailability.ONDEMAND.name());
-      expected = Map.of(endR1,
-          Set.of(NEEDS_LOCATION_UPDATE, 
TabletManagement.ManagementAction.NEEDS_RECOVERY));
+      expected = Map.of(endR1, Set.of(NEEDS_LOCATION_UPDATE, NEEDS_RECOVERY));
       assertEquals(expected, findTabletsNeedingAttention(client, metaCopy5, 
tabletMgmtParams),
           "Only 1 of 2 tablets in table t1 should be returned");
       setTabletAvailability(client, metaCopy5, t1, 
TabletAvailability.HOSTED.name());
-      expected = Map.of(endR1,
-          Set.of(NEEDS_LOCATION_UPDATE, 
TabletManagement.ManagementAction.NEEDS_RECOVERY), prevR1,
+      expected = Map.of(endR1, Set.of(NEEDS_LOCATION_UPDATE, NEEDS_RECOVERY), 
prevR1,
           Set.of(NEEDS_LOCATION_UPDATE));
       assertEquals(expected, findTabletsNeedingAttention(client, metaCopy5, 
tabletMgmtParams),
           "2 tablets in table t1 should be returned");
@@ -248,8 +254,7 @@ public class TabletManagementIteratorIT extends 
AccumuloClusterHarness {
       // for both MERGING and SPLITTING
       setOperationId(client, metaCopy4, t4, null, TabletOperationType.MERGING);
       createLogEntry(client, metaCopy4, t4);
-      expected = Map.of(endR4,
-          Set.of(NEEDS_LOCATION_UPDATE, 
TabletManagement.ManagementAction.NEEDS_RECOVERY));
+      expected = Map.of(endR4, Set.of(NEEDS_LOCATION_UPDATE, NEEDS_RECOVERY));
       assertEquals(expected, findTabletsNeedingAttention(client, metaCopy4, 
tabletMgmtParams),
           "Should have a tablet needing attention because of wals");
       // Switch op to SPLITTING which should also need attention like MERGING
@@ -266,8 +271,7 @@ public class TabletManagementIteratorIT extends 
AccumuloClusterHarness {
       // test the bad tablet location state case (inconsistent metadata)
       tabletMgmtParams = createParameters(client);
       addDuplicateLocation(client, metaCopy3, t3);
-      expected = Map.of(prevR3,
-          Set.of(NEEDS_LOCATION_UPDATE, 
TabletManagement.ManagementAction.BAD_STATE));
+      expected = Map.of(prevR3, Set.of(NEEDS_LOCATION_UPDATE, BAD_STATE));
       assertEquals(expected, findTabletsNeedingAttention(client, metaCopy3, 
tabletMgmtParams),
           "Should have 1 tablet that needs a metadata repair");
 
@@ -278,7 +282,7 @@ public class TabletManagementIteratorIT extends 
AccumuloClusterHarness {
       Map<Path,Path> replacements =
           Map.of(new Path("file:/vol1/accumulo/inst_id"), new 
Path("file:/vol2/accumulo/inst_id"));
       tabletMgmtParams = createParameters(client, replacements);
-      expected = Map.of(prevR4, 
Set.of(TabletManagement.ManagementAction.NEEDS_VOLUME_REPLACEMENT));
+      expected = Map.of(prevR4, Set.of(NEEDS_VOLUME_REPLACEMENT));
       assertEquals(expected, findTabletsNeedingAttention(client, metaCopy4, 
tabletMgmtParams),
           "Should have one tablet that needs a volume replacement");
 
@@ -291,7 +295,7 @@ public class TabletManagementIteratorIT extends 
AccumuloClusterHarness {
       // Lower the split threshold for the table, should cause the files added 
to need attention.
       client.tableOperations().setProperty(tables[3], 
Property.TABLE_SPLIT_THRESHOLD.getKey(),
           "1K");
-      expected = Map.of(prevR4, 
Set.of(TabletManagement.ManagementAction.NEEDS_SPLITTING));
+      expected = Map.of(prevR4, Set.of(NEEDS_SPLITTING));
       assertEquals(expected, findTabletsNeedingAttention(client, metaCopy6, 
tabletMgmtParams),
           "Should have one tablet that needs splitting");
 
@@ -444,6 +448,29 @@ public class TabletManagementIteratorIT extends 
AccumuloClusterHarness {
     return results;
   }
 
+  // Multiple places in the accumulo code will read a batch of keys and then 
take the last key and
+  // make it non inclusive to get the next batch. This test code simulates 
that and ensures the
+  // tablet mgmt iterator works with that.
+  private void testContinueScan(AccumuloClient client, String table,
+      TabletManagementParameters tabletMgmtParams) throws 
TableNotFoundException {
+    try (Scanner scanner = client.createScanner(table, Authorizations.EMPTY)) {
+      TabletManagementIterator.configureScanner(scanner, tabletMgmtParams);
+      List<Entry<Key,Value>> entries1 = new ArrayList<>();
+      scanner.forEach(e -> entries1.add(e));
+      assertTrue(entries1.size() > 1);
+
+      // Create a range that does not include the first key from the last scan.
+      var range = new Range(entries1.get(0).getKey(), false, null, true);
+      scanner.setRange(range);
+
+      // Ensure the first key excluded from the scan
+      List<Entry<Key,Value>> entries2 = new ArrayList<>();
+      scanner.forEach(e -> entries2.add(e));
+      assertEquals(entries1.size() - 1, entries2.size());
+      assertEquals(entries1.get(1).getKey(), entries2.get(0).getKey());
+    }
+  }
+
   private void createTable(AccumuloClient client, String t, boolean online)
       throws AccumuloSecurityException, AccumuloException, 
TableNotFoundException,
       TableExistsException {

Reply via email to