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 e68f9dd2ef Fixed performance issues that were impacting SplitMillionIT 
(#4563)
e68f9dd2ef is described below

commit e68f9dd2ef85677f52b922c4e34ef07bd1e13e15
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Sat May 18 13:19:16 2024 -0400

    Fixed performance issues that were impacting SplitMillionIT (#4563)
    
    The following changes were made as result of running Java Flight
    Recorder repeatedly on the manager and tablet server while
    SplitMillionIT was running.  After these changes the following methods
    would not show up as much in the JFR results.
    
     * Sped up validation of FateId.  When deleting 1 million tablets, 1
       million fate id are written and then read.  Was seeing the regex for
       validation show up when reading 1 million tablets w/ fate ids.
     * Sped up getting children from ZooCache.  The code related to service
       locks was calling this.
     * Sped up parsing of server locks by speeding up the UUID validation,
       that is where it was spending most of its time.
     * Sped up metadata constraint.  Seeing conditional mutation metadata
       updates spend a lot of time checking metadata constraints.
     * Sped up the conditional check for absent walogs by removing the
       creation of an empty TabletMetadata object
     * Sped up SetEncodingIterator.encode by having a special case for size
       1 and avoiding streams for size >1.
     * Increased memory of manager and tsevers in SplitMillionIT because GC
       pauses were being seen
     * Pre split the metadata table in SplitMillionIT.  This allowed the
       tablets to spread across the two tablet servers.  Pre splitting the
       metadata table uncovered a bug.  The add splits table operation would
       fail if metadata tables it needed were not hosted.  Fixed this bug.
     * Made some other misc changes for little things that were seen in
       profiling.
    
     SplitMillionIT is now running faster, however it still does not seem as
    fast as it used to be. Further investigation is needed. These changes
    are mostly good general performance fixes.  Can follow up wit more fixes
    as investigation continues.
    
    
    Co-authored-by: Dave Marion <dlmar...@apache.org>
---
 .../core/clientImpl/TableOperationsImpl.java       | 14 +++++--
 .../java/org/apache/accumulo/core/fate/Fate.java   |  9 +++++
 .../java/org/apache/accumulo/core/fate/FateId.java | 43 ++++++++++++++------
 .../accumulo/core/fate/zookeeper/ZooCache.java     | 14 ++++---
 .../accumulo/core/fate/zookeeper/ZooUtil.java      |  3 +-
 .../org/apache/accumulo/core/lock/ServiceLock.java |  9 ++---
 .../org/apache/accumulo/core/util/UuidUtil.java    | 46 ++++++++++++++++++++++
 .../org/apache/accumulo/core/fate/FateIdTest.java  | 24 +++++++++++
 .../server/constraints/MetadataConstraints.java    | 10 ++---
 .../metadata/ConditionalTabletMutatorImpl.java     |  7 +++-
 .../metadata/iterators/SetEncodingIterator.java    | 22 ++++++++---
 .../manager/tableOps/merge/DeleteTablets.java      |  8 ++--
 .../accumulo/test/functional/SplitMillionIT.java   | 25 ++++++++++--
 13 files changed, 188 insertions(+), 46 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
index 293177a557..c6f33c85b5 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
@@ -538,11 +538,19 @@ public class TableOperationsImpl extends 
TableOperationsHelper {
       var split = iterator.next();
 
       try {
+        Retry retry = 
Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+            
.incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofSeconds(2)).backOffFactor(1.5)
+            .logInterval(Duration.ofMinutes(3)).createRetry();
+
         var tablet = tabLocator.findTablet(context, split, false, 
LocationNeed.NOT_REQUIRED);
-        if (tablet == null) {
+        while (tablet == null) {
           context.requireTableExists(tableId, tableName);
-          throw new IllegalStateException("Unable to find a tablet for split " 
+ split
-              + " in table " + tableName + " " + tableId);
+          try {
+            retry.waitForNextAttempt(log, "Find tablet in " + tableId + " 
containing " + split);
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          }
+          tablet = tabLocator.findTablet(context, split, false, 
LocationNeed.NOT_REQUIRED);
         }
 
         if (split.equals(tablet.getExtent().endRow())) {
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java 
b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java
index eed785b39b..80f543c0ba 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java
@@ -54,6 +54,7 @@ import org.apache.accumulo.core.util.ShutdownUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
+import org.apache.accumulo.core.util.time.NanoTime;
 import org.apache.thrift.TApplicationException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -160,7 +161,10 @@ public class Fate<T> {
           } else if (status == SUBMITTED || status == IN_PROGRESS) {
             Repo<T> prevOp = null;
             try {
+              var startTime = NanoTime.now();
               deferTime = op.isReady(txStore.getID(), environment);
+              log.trace("Running {}.isReady() {} took {} ms and returned {}", 
op.getName(),
+                  txStore.getID(), startTime.elapsed().toMillis(), deferTime);
 
               // Here, deferTime is only used to determine success (zero) or 
failure (non-zero),
               // proceeding on success and returning to the while loop on 
failure.
@@ -170,7 +174,12 @@ public class Fate<T> {
                 if (status == SUBMITTED) {
                   txStore.setStatus(IN_PROGRESS);
                 }
+
+                startTime = NanoTime.now();
                 op = op.call(txStore.getID(), environment);
+                log.trace("Running {}.call() {} took {} ms and returned {}", 
prevOp.getName(),
+                    txStore.getID(), startTime.elapsed().toMillis(),
+                    op == null ? "null" : op.getName());
               } else {
                 continue;
               }
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateId.java 
b/core/src/main/java/org/apache/accumulo/core/fate/FateId.java
index e398ef4efe..85ec9f45ee 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/FateId.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/FateId.java
@@ -18,15 +18,17 @@
  */
 package org.apache.accumulo.core.fate;
 
+import static org.apache.accumulo.core.util.UuidUtil.isUUID;
+
+import java.util.Set;
 import java.util.UUID;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
 
 import org.apache.accumulo.core.data.AbstractId;
 import org.apache.accumulo.core.manager.thrift.TFateId;
 import org.apache.accumulo.core.manager.thrift.TFateInstanceType;
 
+import com.google.common.base.Preconditions;
+
 /**
  * A strongly typed FATE Transaction ID. This is used to uniquely identify a 
FATE transaction.
  * Consists of its {@link FateInstanceType} and its transaction {@link UUID}. 
The canonical string
@@ -36,11 +38,16 @@ public class FateId extends AbstractId<FateId> {
 
   private static final long serialVersionUID = 1L;
   private static final String PREFIX = "FATE:";
-  private static final String UUID_REGEX = 
"[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}";
-  private static final Pattern UUID_PATTERN = Pattern.compile("^" + UUID_REGEX 
+ "$");
-  private static final Pattern FATEID_PATTERN = Pattern.compile("^" + PREFIX + 
"("
-      + 
Stream.of(FateInstanceType.values()).map(Enum::name).collect(Collectors.joining("|"))
 + "):"
-      + UUID_REGEX + "$");
+
+  private static final String META_PREFIX = FateInstanceType.META.name() + ":";
+  private static final String USER_PREFIX = FateInstanceType.USER.name() + ":";
+
+  static {
+    // Validate the assumptions made by this class
+    Preconditions.checkState(Set.of(FateInstanceType.values())
+        .equals(Set.of(FateInstanceType.USER, FateInstanceType.META)));
+    Preconditions.checkState(META_PREFIX.length() == USER_PREFIX.length());
+  }
 
   private FateId(String canonical) {
     super(canonical);
@@ -86,7 +93,7 @@ public class FateId extends AbstractId<FateId> {
    * @return a new FateId object
    */
   public static FateId from(FateInstanceType type, String txUUIDStr) {
-    if (UUID_PATTERN.matcher(txUUIDStr).matches()) {
+    if (isUUID(txUUIDStr, 0)) {
       return new FateId(PREFIX + type + ":" + txUUIDStr);
     } else {
       throw new IllegalArgumentException("Invalid Transaction UUID: " + 
txUUIDStr);
@@ -98,7 +105,7 @@ public class FateId extends AbstractId<FateId> {
    * @return a new FateId object from the given string
    */
   public static FateId from(String fateIdStr) {
-    if (FATEID_PATTERN.matcher(fateIdStr).matches()) {
+    if (isFateId(fateIdStr)) {
       return new FateId(fateIdStr);
     } else {
       throw new IllegalArgumentException("Invalid Fate ID: " + fateIdStr);
@@ -110,7 +117,19 @@ public class FateId extends AbstractId<FateId> {
    * @return true if the string is a valid FateId, false otherwise
    */
   public static boolean isFateId(String fateIdStr) {
-    return FATEID_PATTERN.matcher(fateIdStr).matches();
+
+    if (!fateIdStr.startsWith(PREFIX)) {
+      return false;
+    }
+
+    if (!fateIdStr.startsWith(META_PREFIX, PREFIX.length())
+        && !fateIdStr.startsWith(USER_PREFIX, PREFIX.length())) {
+      return false;
+    }
+
+    // assuming that META_PREFIX and USER_PREFIX are the same length, so UUID 
always starts a same
+    // place
+    return isUUID(fateIdStr, PREFIX.length() + META_PREFIX.length());
   }
 
   /**
@@ -132,7 +151,7 @@ public class FateId extends AbstractId<FateId> {
         throw new IllegalArgumentException("Invalid TFateInstanceType: " + 
tFateId.getType());
     }
 
-    if (UUID_PATTERN.matcher(txUUIDStr).matches()) {
+    if (isUUID(txUUIDStr, 0)) {
       return new FateId(PREFIX + type + ":" + txUUIDStr);
     } else {
       throw new IllegalArgumentException("Invalid Transaction UUID: " + 
txUUIDStr);
diff --git 
a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java 
b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
index fc2b6f1c41..e2cf1fdb34 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
@@ -295,19 +295,23 @@ public class ZooCache {
 
         // only read volatile once for consistency
         ImmutableCacheCopies lic = immutableCache;
-        if (lic.childrenCache.containsKey(zPath)) {
-          return lic.childrenCache.get(zPath);
+        var cachedChildren = lic.childrenCache.get(zPath);
+        // null is cached as a value, that is the reason for the secondary 
containsKey check
+        if (cachedChildren != null || lic.childrenCache.containsKey(zPath)) {
+          return cachedChildren;
         }
 
         cacheWriteLock.lock();
         try {
-          if (childrenCache.containsKey(zPath)) {
-            return childrenCache.get(zPath);
+          List<String> children = childrenCache.get(zPath);
+          // null is cached as a value, that is the reason for the secondary 
containsKey check
+          if (children != null || lic.childrenCache.containsKey(zPath)) {
+            return children;
           }
 
           final ZooKeeper zooKeeper = getZooKeeper();
 
-          List<String> children = zooKeeper.getChildren(zPath, watcher);
+          children = zooKeeper.getChildren(zPath, watcher);
           if (children != null) {
             children = List.copyOf(children);
           }
diff --git 
a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java 
b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java
index 47d906fedb..51002ee574 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java
@@ -20,7 +20,6 @@ package org.apache.accumulo.core.fate.zookeeper;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
-import java.math.BigInteger;
 import java.security.NoSuchAlgorithmException;
 import java.time.Instant;
 import java.time.OffsetDateTime;
@@ -75,7 +74,7 @@ public class ZooUtil {
         path = root + "/" + sa[0].substring(0, lastSlash);
       }
       node = sa[0].substring(lastSlash + 1);
-      eid = new BigInteger(sa[1], 16).longValue();
+      eid = Long.parseUnsignedLong(sa[1], 16);
     }
 
     public LockID(String path, String node, long eid) {
diff --git a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java 
b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java
index ef80d4e52d..91a1232954 100644
--- a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java
+++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java
@@ -32,6 +32,7 @@ import 
org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.LockID;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.core.util.UuidUtil;
 import org.apache.accumulo.core.util.time.NanoTime;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -188,10 +189,10 @@ public class ServiceLock implements Watcher {
    */
   public static List<String> validateAndSort(ServiceLockPath path, 
List<String> children) {
     LOG.trace("validating and sorting children at path {}", path);
-    List<String> validChildren = new ArrayList<>();
     if (children == null || children.isEmpty()) {
-      return validChildren;
+      return List.of();
     }
+    List<String> validChildren = new ArrayList<>(children.size());
     children.forEach(c -> {
       LOG.trace("Validating {}", c);
       if (c.startsWith(ZLOCK_PREFIX)) {
@@ -202,9 +203,7 @@ public class ServiceLock implements Watcher {
           String sequenceNum = candidate.substring(idx + 1);
           try {
             LOG.trace("Testing uuid format of {}", uuid);
-            // string check guards uuids like "1-1-1-1-1" that parse to
-            // "00000001-0001-0001-0001-000000000001"
-            if (!uuid.equals(UUID.fromString(uuid).toString())) {
+            if (!UuidUtil.isUUID(uuid, 0)) {
               throw new IllegalArgumentException(uuid + " is an invalid UUID");
             }
             if (sequenceNum.length() == 10) {
diff --git a/core/src/main/java/org/apache/accumulo/core/util/UuidUtil.java 
b/core/src/main/java/org/apache/accumulo/core/util/UuidUtil.java
new file mode 100644
index 0000000000..83d41abe0f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/UuidUtil.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util;
+
+public class UuidUtil {
+  /**
+   * A fast method for verifying a suffix of a string looks like a uuid.
+   *
+   * @param offset location where the uuid starts. Its expected the uuid 
occupies the rest of the
+   *        string.
+   */
+  public static boolean isUUID(String uuid, int offset) {
+    if (uuid.length() - offset != 36) {
+      return false;
+    }
+    for (int i = 0; i < 36; i++) {
+      var c = uuid.charAt(i + offset);
+      if (i == 8 || i == 13 || i == 18 || i == 23) {
+        if (c != '-') {
+          // expect '-' char at above positions, did not see it
+          return false;
+        }
+      } else if (c < '0' || (c > '9' && c < 'A') || (c > 'F' && c < 'a') || c 
> 'f') {
+        // expected hex at all other positions, did not see hex chars
+        return false;
+      }
+    }
+    return true;
+  }
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/fate/FateIdTest.java 
b/core/src/test/java/org/apache/accumulo/core/fate/FateIdTest.java
index 3de10825e2..4faafa999c 100644
--- a/core/src/test/java/org/apache/accumulo/core/fate/FateIdTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/fate/FateIdTest.java
@@ -62,5 +62,29 @@ public class FateIdTest {
       assertThrows(IllegalArgumentException.class, () -> 
FateId.from(illegalId));
       assertFalse(FateId.isFateId(illegalId));
     }
+
+    // Try an illegal character in every position
+    for (int i = 0; i < legalId.toString().length(); i++) {
+      var chars = legalId.toString().toCharArray();
+      chars[i] = '#';
+      var illegalId = new String(chars);
+      assertThrows(IllegalArgumentException.class, () -> 
FateId.from(illegalId));
+      assertFalse(FateId.isFateId(illegalId));
+    }
+
+    // place number and dash chars in unexpected positions
+    for (int i = 0; i < legalId.toString().length(); i++) {
+      var chars = legalId.toString().toCharArray();
+      if (chars[i] == '-') {
+        chars[i] = '2';
+      } else {
+        chars[i] = '-';
+      }
+      var illegalId = new String(chars);
+      assertThrows(IllegalArgumentException.class, () -> 
FateId.from(illegalId));
+      assertFalse(FateId.isFateId(illegalId));
+
+    }
+
   }
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
 
b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
index 7f1f5c21bc..ad8cd05c38 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
@@ -121,13 +121,13 @@ public class MetadataConstraints implements Constraint {
       );
   // @formatter:on
 
-  private static boolean isValidColumn(ColumnUpdate cu) {
+  private static boolean isValidColumn(Text family, Text qualifier) {
 
-    if (validColumnFams.contains(new Text(cu.getColumnFamily()))) {
+    if (validColumnFams.contains(family)) {
       return true;
     }
 
-    return validColumnQuals.contains(new ColumnFQ(cu));
+    return validColumnQuals.contains(new ColumnFQ(family, qualifier));
   }
 
   private static ArrayList<Short> addViolation(ArrayList<Short> lst, int 
violation) {
@@ -226,7 +226,7 @@ public class MetadataConstraints implements Constraint {
       Text columnQualifier = new Text(columnUpdate.getColumnQualifier());
 
       if (columnUpdate.isDeleted()) {
-        if (!isValidColumn(columnUpdate)) {
+        if (!isValidColumn(columnFamily, columnQualifier)) {
           violations = addViolation(violations, 2);
         }
         continue;
@@ -365,7 +365,7 @@ public class MetadataConstraints implements Constraint {
           checkedBulk = true;
         }
       } else {
-        if (!isValidColumn(columnUpdate)) {
+        if (!isValidColumn(columnFamily, columnQualifier)) {
           violations = addViolation(violations, 2);
         } else {
           final var column = new ColumnFQ(columnUpdate);
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java
index 381b3e112e..85c5e2ebad 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java
@@ -31,6 +31,7 @@ import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSec
 
 import java.util.HashSet;
 import java.util.Objects;
+import java.util.Set;
 import java.util.function.BiConsumer;
 import java.util.function.Consumer;
 
@@ -55,6 +56,7 @@ import 
org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletMutatorBase;
 import org.apache.accumulo.core.metadata.schema.TabletOperationId;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.metadata.iterators.LocationExistsIterator;
@@ -278,8 +280,9 @@ public class ConditionalTabletMutatorImpl extends 
TabletMutatorBase<Ample.Condit
   @Override
   public Ample.ConditionalTabletMutator requireAbsentLogs() {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
-    // create a tablet metadata with an empty set of logs and require the same 
as that
-    requireSameSingle(TabletMetadata.builder(extent).build(ColumnType.LOGS), 
ColumnType.LOGS);
+    Condition c = SetEncodingIterator.createCondition(Set.<LogEntry>of(),
+        logEntry -> logEntry.getColumnQualifier().toString().getBytes(UTF_8), 
LogColumnFamily.NAME);
+    mutation.addCondition(c);
     return this;
   }
 
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/SetEncodingIterator.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/SetEncodingIterator.java
index af878263d3..facd747b47 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/SetEncodingIterator.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/SetEncodingIterator.java
@@ -24,8 +24,10 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.UncheckedIOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
@@ -184,15 +186,23 @@ public class SetEncodingIterator implements 
SortedKeyValueIterator<Key,Value> {
   private static <T> byte[] encode(Set<T> set, Function<T,byte[]> encoder) {
     try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dos = new DataOutputStream(baos)) {
-      set.stream().map(encoder).sorted(Arrays::compare).forEach(ba -> {
-        try {
+
+      if (set.size() == 1) {
+        // no need to sort or use a stream when the set is size 1.
+        var ba = encoder.apply(set.iterator().next());
+        dos.writeInt(ba.length);
+        dos.write(ba, 0, ba.length);
+      } else if (set.size() > 1) {
+        List<byte[]> copy = new ArrayList<>(set.size());
+        for (var entry : set) {
+          copy.add(encoder.apply(entry));
+        }
+        copy.sort(Arrays::compare);
+        for (var ba : copy) {
           dos.writeInt(ba.length);
           dos.write(ba, 0, ba.length);
-        } catch (IOException ioe) {
-          throw new UncheckedIOException(ioe);
         }
-
-      });
+      }
 
       dos.writeInt(set.size());
 
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/DeleteTablets.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/DeleteTablets.java
index 4ef90497e5..710b8779ad 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/DeleteTablets.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/DeleteTablets.java
@@ -100,9 +100,11 @@ public class DeleteTablets extends ManagerRepo {
           break;
         }
 
-        tabletMeta.getKeyValues().keySet().forEach(key -> {
-          log.trace("{} deleting {}", fateId, key);
-        });
+        if (log.isTraceEnabled()) {
+          tabletMeta.getKeyValues().keySet().forEach(key -> {
+            log.trace("{} deleting {}", fateId, key);
+          });
+        }
 
         tabletMutator.deleteAll(tabletMeta.getKeyValues().keySet());
         // if the tablet no longer exists, then it was successful
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/SplitMillionIT.java 
b/test/src/main/java/org/apache/accumulo/test/functional/SplitMillionIT.java
index 585428bb32..0953e7ccef 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitMillionIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitMillionIT.java
@@ -39,7 +39,11 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.Filter;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.minicluster.MemoryUnit;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
@@ -47,7 +51,7 @@ import org.slf4j.LoggerFactory;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
-public class SplitMillionIT extends AccumuloClusterHarness {
+public class SplitMillionIT extends ConfigurableMacBase {
 
   private static final Logger log = 
LoggerFactory.getLogger(SplitMillionIT.class);
 
@@ -59,15 +63,30 @@ public class SplitMillionIT extends AccumuloClusterHarness {
     }
   }
 
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration 
hadoopCoreSite) {
+    cfg.setMemory(ServerType.MANAGER, 1, MemoryUnit.GIGABYTE);
+    cfg.setMemory(ServerType.TABLET_SERVER, 1, MemoryUnit.GIGABYTE);
+  }
+
   @SuppressFBWarnings(value = {"PREDICTABLE_RANDOM", 
"DMI_RANDOM_USED_ONLY_ONCE"},
       justification = "predictable random is ok for testing")
   @Test
   public void testOneMillionTablets() throws Exception {
 
-    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
+    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProperties()).build()) {
       String tableName = getUniqueNames(1)[0];
       c.tableOperations().create(tableName);
 
+      // pre split the metadata table
+      var tableId = getServerContext().getTableId(tableName);
+      SortedSet<Text> metaSplits = new TreeSet<>();
+      for (int i = 1; i < 10; i++) {
+        String metaSplit = String.format("%s;%010d", tableId, 100_000_000 / 10 
* i);
+        metaSplits.add(new Text(metaSplit));
+      }
+      c.tableOperations().addSplits(AccumuloTable.METADATA.tableName(), 
metaSplits);
+
       SortedSet<Text> splits = new TreeSet<>();
 
       for (int i = 100; i < 100_000_000; i += 100) {

Reply via email to