You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by dl...@apache.org on 2023/05/31 11:31:59 UTC

[accumulo] branch elasticity updated: Avoid passing tablet end row through string in TabletManagement (#3441)

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

dlmarion 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 63c94b686a Avoid passing tablet end row through string in TabletManagement (#3441)
63c94b686a is described below

commit 63c94b686a28726215fb409b6c5870aa7be29b8e
Author: Keith Turner <kt...@apache.org>
AuthorDate: Wed May 31 07:31:52 2023 -0400

    Avoid passing tablet end row through string in TabletManagement (#3441)
    
    Was seeing binary data being corrupted when going through a string.
---
 .../core/manager/state/TabletManagement.java       | 12 +++++---
 .../server/manager/state/TabletManagementTest.java | 36 ++++++++++++++++++++--
 2 files changed, 40 insertions(+), 8 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/manager/state/TabletManagement.java b/core/src/main/java/org/apache/accumulo/core/manager/state/TabletManagement.java
index 73d7971898..d6aece245f 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/state/TabletManagement.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/state/TabletManagement.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.hadoop.io.Text;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Splitter;
@@ -46,7 +47,9 @@ public class TabletManagement {
           ColumnType.CHOPPED, ColumnType.HOSTING_GOAL, ColumnType.HOSTING_REQUESTED,
           ColumnType.FILES, ColumnType.LAST, ColumnType.OPID);
 
-  private static final String REASONS_COLUMN_NAME = "REASONS";
+  private static final Text REASONS_COLUMN_NAME = new Text("REASONS");
+
+  private static final Text EMPTY = new Text("");
 
   public static enum ManagementAction {
     BAD_STATE, NEEDS_COMPACTING, NEEDS_LOCATION_UPDATE, IS_MERGING, NEEDS_SPLITTING;
@@ -54,8 +57,7 @@ public class TabletManagement {
 
   public static void addActions(final SortedMap<Key,Value> decodedRow,
       final Set<ManagementAction> actions) {
-    final Key reasonsKey =
-        new Key(decodedRow.firstKey().getRow().toString(), REASONS_COLUMN_NAME, "");
+    final Key reasonsKey = new Key(decodedRow.firstKey().getRow(), REASONS_COLUMN_NAME, EMPTY);
     final Value reasonsValue = new Value(Joiner.on(',').join(actions));
     decodedRow.put(reasonsKey, reasonsValue);
   }
@@ -74,8 +76,8 @@ public class TabletManagement {
 
   public TabletManagement(Key wholeRowKey, Value wholeRowValue, boolean saveKV) throws IOException {
     final SortedMap<Key,Value> decodedRow = WholeRowIterator.decodeRow(wholeRowKey, wholeRowValue);
-    String row = decodedRow.firstKey().getRow().toString();
-    Value val = decodedRow.remove(new Key(row, REASONS_COLUMN_NAME, ""));
+    Text row = decodedRow.firstKey().getRow();
+    Value val = decodedRow.remove(new Key(row, REASONS_COLUMN_NAME, EMPTY));
     Set<ManagementAction> actions = new HashSet<>();
     Splitter.on(',').split(val.toString()).forEach(a -> actions.add(ManagementAction.valueOf(a)));
     TabletMetadata tm = TabletMetadata.convertRow(decodedRow.entrySet().iterator(),
diff --git a/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementTest.java
index e8c59e33df..e09cca2b34 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementTest.java
@@ -23,6 +23,7 @@ import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSec
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.FLUSH_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN;
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.ArrayList;
 import java.util.Set;
@@ -64,8 +65,7 @@ public class TabletManagementTest {
     return rowMap;
   }
 
-  private SortedMap<Key,Value> createMetadataEntryKV() {
-    KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da"));
+  private SortedMap<Key,Value> createMetadataEntryKV(KeyExtent extent) {
 
     Mutation mutation = TabletColumnFamily.createPrevRowMutation(extent);
 
@@ -112,9 +112,12 @@ public class TabletManagementTest {
 
   @Test
   public void testEncodeDecodeWithReasons() throws Exception {
+    KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da"));
+
     final Set<ManagementAction> actions =
         Set.of(ManagementAction.NEEDS_LOCATION_UPDATE, ManagementAction.NEEDS_SPLITTING);
-    final SortedMap<Key,Value> entries = createMetadataEntryKV();
+
+    final SortedMap<Key,Value> entries = createMetadataEntryKV(extent);
 
     TabletManagement.addActions(entries, actions);
     Key key = entries.firstKey();
@@ -130,4 +133,31 @@ public class TabletManagementTest {
     assertEquals(actions, tmi.getActions());
   }
 
+  @Test
+  public void testBinary() throws Exception {
+    // test end row with non ascii data
+    Text endRow = new Text(new byte[] {'m', (byte) 0xff});
+    KeyExtent extent = new KeyExtent(TableId.of("5"), endRow, new Text("da"));
+
+    final Set<ManagementAction> actions =
+        Set.of(ManagementAction.NEEDS_LOCATION_UPDATE, ManagementAction.NEEDS_SPLITTING);
+
+    final SortedMap<Key,Value> entries = createMetadataEntryKV(extent);
+
+    TabletManagement.addActions(entries, actions);
+    Key key = entries.firstKey();
+    Value val = WholeRowIterator.encodeRow(new ArrayList<>(entries.keySet()),
+        new ArrayList<>(entries.values()));
+
+    assertTrue(entries.keySet().stream().allMatch(k -> k.getRow().equals(extent.toMetaRow())));
+
+    // Remove the REASONS column from the entries map for the comparison check
+    // below
+    entries.remove(new Key(key.getRow(), new Text("REASONS"), new Text("")));
+
+    TabletManagement tmi = new TabletManagement(key, val, true);
+    assertEquals(entries, tmi.getTabletMetadata().getKeyValues());
+    assertEquals(actions, tmi.getActions());
+
+  }
 }