You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by do...@apache.org on 2022/04/28 13:05:57 UTC

[accumulo] branch main updated: Replace iterator usage (#2646)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new d564715e82 Replace iterator usage (#2646)
d564715e82 is described below

commit d564715e827940af1e45b504ab7c100fb08140e5
Author: Dom G <do...@gmail.com>
AuthorDate: Thu Apr 28 09:05:52 2022 -0400

    Replace iterator usage (#2646)
    
    * Use streams to simplify iterator usage
    
    * Add more short-circuit empty/non-empty checks
---
 .../accumulo/core/client/IsolatedScanner.java      |  1 -
 .../core/client/rfile/RFileClientTest.java         |  7 ++--
 .../hadoop/its/mapred/AccumuloOutputFormatIT.java  | 25 ++++++-------
 .../accumulo/hadoop/its/mapred/TokenFileIT.java    | 25 ++++++-------
 .../its/mapreduce/AccumuloOutputFormatIT.java      | 26 ++++++--------
 .../accumulo/hadoop/its/mapreduce/TokenFileIT.java | 25 ++++++-------
 .../accumulo/iteratortest/IteratorTestBase.java    |  2 --
 .../accumulo/test/IsolationAndDeepCopyIT.java      | 13 ++-----
 .../org/apache/accumulo/test/SplitRecoveryIT.java  |  4 +--
 .../accumulo/test/functional/BigRootTabletIT.java  |  4 +--
 .../accumulo/test/functional/ClassLoaderIT.java    | 13 ++-----
 .../accumulo/test/functional/CombinerIT.java       | 13 ++-----
 .../accumulo/test/functional/CreateAndUseIT.java   |  8 ++---
 .../test/functional/DeleteEverythingIT.java        | 14 +++-----
 .../accumulo/test/functional/DeleteRowsIT.java     |  4 +--
 .../test/functional/GarbageCollectorIT.java        |  7 +---
 .../accumulo/test/functional/KerberosIT.java       |  6 +---
 .../accumulo/test/functional/MetadataIT.java       | 20 +++--------
 .../accumulo/test/functional/RowDeleteIT.java      | 41 +++++++++-------------
 .../apache/accumulo/test/functional/TableIT.java   |  6 ++--
 .../test/functional/ZookeeperRestartIT.java        | 12 ++-----
 .../test/mapred/AccumuloOutputFormatIT.java        | 11 +++---
 .../apache/accumulo/test/mapred/TokenFileIT.java   | 11 +++---
 .../test/mapreduce/AccumuloOutputFormatIT.java     | 12 +++----
 .../accumulo/test/mapreduce/TokenFileIT.java       | 23 ++++++------
 25 files changed, 111 insertions(+), 222 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
index 75e505a7c6..a1471642ae 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
@@ -131,7 +131,6 @@ public class IsolatedScanner extends ScannerOptions implements Scanner {
         setOptions((ScannerOptions) scanner, opts);
 
         return scanner.iterator();
-        // return new FaultyIterator(scanner.iterator());
       }
     }
 
diff --git a/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileClientTest.java b/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileClientTest.java
index 54dc2856e0..20fb18e16c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileClientTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileClientTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.core.client.rfile;
 
+import static com.google.common.collect.MoreCollectors.onlyElement;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -521,10 +522,8 @@ public class RFileClientTest {
 
     random.ints(100, 0, 10_000).forEach(r -> {
       scanner.setRange(new Range(rowStr(r)));
-      Iterator<Entry<Key,Value>> iter = scanner.iterator();
-      assertTrue(iter.hasNext());
-      assertEquals(rowStr(r), iter.next().getKey().getRow().toString());
-      assertFalse(iter.hasNext());
+      String actual = scanner.stream().collect(onlyElement()).getKey().getRow().toString();
+      assertEquals(rowStr(r), actual);
     });
 
     scanner.close();
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
index 6c168754d3..868961102d 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
@@ -18,16 +18,15 @@
  */
 package org.apache.accumulo.hadoop.its.mapred;
 
+import static com.google.common.collect.MoreCollectors.onlyElement;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
@@ -204,24 +203,22 @@ public class AccumuloOutputFormatIT extends ConfigurableMacBase {
       String table2 = instanceName + "_t2";
       c.tableOperations().create(table1);
       c.tableOperations().create(table2);
-      BatchWriter bw = c.createBatchWriter(table1);
-      for (int i = 0; i < 100; i++) {
-        Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-        m.put("", "", String.format("%09x", i));
-        bw.addMutation(m);
+      try (BatchWriter bw = c.createBatchWriter(table1)) {
+        for (int i = 0; i < 100; i++) {
+          Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+          m.put("", "", String.format("%09x", i));
+          bw.addMutation(m);
+        }
       }
-      bw.close();
 
       MRTester.main(new String[] {"root", ROOT_PASSWORD, table1, table2, instanceName,
           getCluster().getZooKeepers()});
       assertNull(e1);
 
       try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        Iterator<Entry<Key,Value>> iter = scanner.iterator();
-        assertTrue(iter.hasNext());
-        Entry<Key,Value> entry = iter.next();
-        assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-        assertFalse(iter.hasNext());
+        int i = scanner.stream().map(Map.Entry::getValue).map(Value::get).map(String::new)
+            .map(Integer::parseInt).collect(onlyElement());
+        assertEquals(100, i);
       }
     }
   }
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java
index 0599d6d615..70b0dda388 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.hadoop.its.mapred;
 
+import static com.google.common.collect.MoreCollectors.onlyElement;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -27,8 +27,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.file.Paths;
-import java.util.Iterator;
-import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -152,13 +151,13 @@ public class TokenFileIT extends AccumuloClusterHarness {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       c.tableOperations().create(table1);
       c.tableOperations().create(table2);
-      BatchWriter bw = c.createBatchWriter(table1);
-      for (int i = 0; i < 100; i++) {
-        Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-        m.put("", "", String.format("%09x", i));
-        bw.addMutation(m);
+      try (BatchWriter bw = c.createBatchWriter(table1)) {
+        for (int i = 0; i < 100; i++) {
+          Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+          m.put("", "", String.format("%09x", i));
+          bw.addMutation(m);
+        }
       }
-      bw.close();
 
       File tf = new File(tempDir, "client.properties");
       assertTrue(tf.createNewFile(), "Failed to create file: " + tf);
@@ -170,11 +169,9 @@ public class TokenFileIT extends AccumuloClusterHarness {
       assertNull(e1);
 
       try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        Iterator<Entry<Key,Value>> iter = scanner.iterator();
-        assertTrue(iter.hasNext());
-        Entry<Key,Value> entry = iter.next();
-        assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-        assertFalse(iter.hasNext());
+        int i = scanner.stream().map(Map.Entry::getValue).map(Value::get).map(String::new)
+            .map(Integer::parseInt).collect(onlyElement());
+        assertEquals(100, i);
       }
     }
   }
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java
index 5a70558d43..fc884a9a0a 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java
@@ -18,15 +18,13 @@
  */
 package org.apache.accumulo.hadoop.its.mapreduce;
 
+import static com.google.common.collect.MoreCollectors.onlyElement;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map.Entry;
+import java.util.Map;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -132,23 +130,21 @@ public class AccumuloOutputFormatIT extends AccumuloClusterHarness {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       c.tableOperations().create(table1);
       c.tableOperations().create(table2);
-      BatchWriter bw = c.createBatchWriter(table1);
-      for (int i = 0; i < 100; i++) {
-        Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-        m.put("", "", String.format("%09x", i));
-        bw.addMutation(m);
+      try (BatchWriter bw = c.createBatchWriter(table1)) {
+        for (int i = 0; i < 100; i++) {
+          Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+          m.put("", "", String.format("%09x", i));
+          bw.addMutation(m);
+        }
       }
-      bw.close();
 
       MRTester.main(new String[] {table1, table2});
       assertNull(e1);
 
       try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        Iterator<Entry<Key,Value>> iter = scanner.iterator();
-        assertTrue(iter.hasNext());
-        Entry<Key,Value> entry = iter.next();
-        assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-        assertFalse(iter.hasNext());
+        int i = scanner.stream().map(Map.Entry::getValue).map(Value::get).map(String::new)
+            .map(Integer::parseInt).collect(onlyElement());
+        assertEquals(100, i);
       }
     }
   }
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java
index cf2cf6b9d9..670b71ff4d 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.hadoop.its.mapreduce;
 
+import static com.google.common.collect.MoreCollectors.onlyElement;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -27,8 +27,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.file.Paths;
-import java.util.Iterator;
-import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -146,13 +145,13 @@ public class TokenFileIT extends AccumuloClusterHarness {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       c.tableOperations().create(table1);
       c.tableOperations().create(table2);
-      BatchWriter bw = c.createBatchWriter(table1);
-      for (int i = 0; i < 100; i++) {
-        Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-        m.put("", "", String.format("%09x", i));
-        bw.addMutation(m);
+      try (BatchWriter bw = c.createBatchWriter(table1)) {
+        for (int i = 0; i < 100; i++) {
+          Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+          m.put("", "", String.format("%09x", i));
+          bw.addMutation(m);
+        }
       }
-      bw.close();
 
       File tf = new File(tempDir, "client.properties");
       assertTrue(tf.createNewFile(), "Failed to create file: " + tf);
@@ -164,11 +163,9 @@ public class TokenFileIT extends AccumuloClusterHarness {
       assertNull(e1);
 
       try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        Iterator<Entry<Key,Value>> iter = scanner.iterator();
-        assertTrue(iter.hasNext());
-        Entry<Key,Value> entry = iter.next();
-        assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-        assertFalse(iter.hasNext());
+        int i = scanner.stream().map(Map.Entry::getValue).map(Value::get).map(String::new)
+            .map(Integer::parseInt).collect(onlyElement());
+        assertEquals(100, i);
       }
     }
   }
diff --git a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/IteratorTestBase.java b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/IteratorTestBase.java
index 3d5bd8dcc0..426c94639a 100644
--- a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/IteratorTestBase.java
+++ b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/IteratorTestBase.java
@@ -69,8 +69,6 @@ public abstract class IteratorTestBase {
     Set<ClassInfo> classes = cp.getTopLevelClasses(searchPackage);
 
     final List<IteratorTestCase> testCases1 = new ArrayList<>();
-    // final Set<Class<? extends IteratorTestCase>> classes =
-    // reflections.getSubTypesOf(IteratorTestCase.class);
     for (ClassInfo classInfo : classes) {
       Class<?> clz;
       try {
diff --git a/test/src/main/java/org/apache/accumulo/test/IsolationAndDeepCopyIT.java b/test/src/main/java/org/apache/accumulo/test/IsolationAndDeepCopyIT.java
index 43e9cd92d3..9810ab3ef7 100644
--- a/test/src/main/java/org/apache/accumulo/test/IsolationAndDeepCopyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/IsolationAndDeepCopyIT.java
@@ -19,11 +19,6 @@
 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 java.util.Iterator;
-import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -31,9 +26,7 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
@@ -71,10 +64,8 @@ public class IsolationAndDeepCopyIT extends AccumuloClusterHarness {
         scanner.addScanIterator(iterCfg);
 
         for (int i = 0; i < 100; i++) {
-          Iterator<Entry<Key,Value>> iter = scanner.iterator();
-          assertTrue(iter.hasNext());
-          assertEquals("000A", iter.next().getKey().getColumnQualifierData().toString());
-          assertFalse(iter.hasNext());
+          String actual = getOnlyElement(scanner).getKey().getColumnQualifierData().toString();
+          assertEquals("000A", actual);
         }
       }
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java
index 38d14c9ce4..eaac8c890b 100644
--- a/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java
@@ -47,8 +47,6 @@ import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 
-import com.google.common.collect.Iterators;
-
 public class SplitRecoveryIT extends AccumuloClusterHarness {
 
   @Override
@@ -67,7 +65,7 @@ public class SplitRecoveryIT extends AccumuloClusterHarness {
     try (Scanner scanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
       scanner.setRange(new Range(new Text(tableId + ";"), new Text(tableId + "<")));
       scanner.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
-      return Iterators.size(scanner.iterator()) == 0;
+      return scanner.stream().findAny().isEmpty();
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
index fd85b71892..f59ad52354 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
@@ -34,8 +34,6 @@ import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.jupiter.api.Test;
 
-import com.google.common.collect.Iterators;
-
 public class BigRootTabletIT extends AccumuloClusterHarness {
   // ACCUMULO-542: A large root tablet will fail to load if it does't fit in the tserver scan
   // buffers
@@ -67,7 +65,7 @@ public class BigRootTabletIT extends AccumuloClusterHarness {
       cluster.stop();
       cluster.start();
       assertTrue(
-          Iterators.size(c.createScanner(RootTable.NAME, Authorizations.EMPTY).iterator()) > 0);
+          c.createScanner(RootTable.NAME, Authorizations.EMPTY).stream().findAny().isPresent());
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
index 7c1d36bd1d..ce15fceea6 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
@@ -21,8 +21,6 @@ package org.apache.accumulo.test.functional;
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
 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.Assumptions.assumeTrue;
 
 import java.io.IOException;
@@ -30,8 +28,6 @@ import java.io.InputStream;
 import java.time.Duration;
 import java.util.Collections;
 import java.util.EnumSet;
-import java.util.Iterator;
-import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -39,9 +35,7 @@ import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.Combiner;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
@@ -119,11 +113,8 @@ public class ClassLoaderIT extends AccumuloClusterHarness {
 
   private void scanCheck(AccumuloClient c, String tableName, String expected) throws Exception {
     try (Scanner bs = c.createScanner(tableName, Authorizations.EMPTY)) {
-      Iterator<Entry<Key,Value>> iterator = bs.iterator();
-      assertTrue(iterator.hasNext());
-      Entry<Key,Value> next = iterator.next();
-      assertFalse(iterator.hasNext());
-      assertEquals(expected, next.getValue().toString());
+      String actual = getOnlyElement(bs).getValue().toString();
+      assertEquals(expected, actual);
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CombinerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CombinerIT.java
index 231c8b84d3..f431a91f0d 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CombinerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CombinerIT.java
@@ -19,22 +19,16 @@
 package org.apache.accumulo.test.functional;
 
 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 java.time.Duration;
 import java.util.Collections;
-import java.util.Iterator;
-import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.LongCombiner.Type;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.security.Authorizations;
@@ -50,11 +44,8 @@ public class CombinerIT extends AccumuloClusterHarness {
 
   private void checkSum(String tableName, AccumuloClient c) throws Exception {
     try (Scanner s = c.createScanner(tableName, Authorizations.EMPTY)) {
-      Iterator<Entry<Key,Value>> i = s.iterator();
-      assertTrue(i.hasNext());
-      Entry<Key,Value> entry = i.next();
-      assertEquals("45", entry.getValue().toString());
-      assertFalse(i.hasNext());
+      String actual = getOnlyElement(s).getValue().toString();
+      assertEquals("45", actual);
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseIT.java
index 884ea399d4..a543b6bc1d 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseIT.java
@@ -19,10 +19,10 @@
 package org.apache.accumulo.test.functional;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -43,8 +43,6 @@ import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Test;
 
-import com.google.common.collect.Iterators;
-
 public class CreateAndUseIT extends AccumuloClusterHarness {
 
   private static NewTableConfiguration ntc;
@@ -133,9 +131,7 @@ public class CreateAndUseIT extends AccumuloClusterHarness {
 
       try (BatchScanner bs = client.createBatchScanner(table3)) {
         bs.setRanges(ranges);
-        Iterator<Entry<Key,Value>> iter = bs.iterator();
-        int count = Iterators.size(iter);
-        assertEquals(0, count, "Did not expect to find any entries");
+        assertTrue(bs.stream().findAny().isEmpty(), "Did not expect to find any entries");
       }
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
index 8cd2389bc8..eb06c14b22 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
@@ -19,7 +19,7 @@
 package org.apache.accumulo.test.functional;
 
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
-import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
 import java.util.Map;
@@ -42,9 +42,6 @@ import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-
 public class DeleteEverythingIT extends AccumuloClusterHarness {
 
   @Override
@@ -103,8 +100,8 @@ public class DeleteEverythingIT extends AccumuloClusterHarness {
 
       try (Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY)) {
         scanner.setRange(new Range());
-        int count = Iterators.size(scanner.iterator());
-        assertEquals(0, count, "count == " + count);
+
+        assertTrue(scanner.stream().findAny().isEmpty());
         c.tableOperations().flush(tableName, null, null, true);
 
         c.tableOperations().setProperty(tableName, Property.TABLE_MAJC_RATIO.getKey(), "1.0");
@@ -114,11 +111,8 @@ public class DeleteEverythingIT extends AccumuloClusterHarness {
 
         bw.close();
 
-        count = Iterables.size(scanner);
+        assertTrue(scanner.stream().findAny().isEmpty());
 
-        if (count != 0) {
-          throw new Exception("count == " + count);
-        }
       }
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
index cbcd79b0cb..32e06ed3f8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
@@ -43,8 +43,6 @@ import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Iterators;
-
 public class DeleteRowsIT extends AccumuloClusterHarness {
 
   private static final Logger log = LoggerFactory.getLogger(DeleteRowsIT.class);
@@ -75,7 +73,7 @@ public class DeleteRowsIT extends AccumuloClusterHarness {
         c.tableOperations().create(tableName);
         c.tableOperations().deleteRows(tableName, null, null);
         try (Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY)) {
-          assertEquals(0, Iterators.size(scanner.iterator()));
+          assertTrue(scanner.stream().findAny().isEmpty());
         }
       }
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
index a2d2644cdc..cbe43ccba4 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
@@ -20,7 +20,6 @@ package org.apache.accumulo.test.functional;
 
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotEquals;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -29,7 +28,6 @@ import static org.junit.jupiter.api.Assertions.fail;
 import java.io.UncheckedIOException;
 import java.time.Duration;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
@@ -242,14 +240,11 @@ public class GarbageCollectorIT extends ConfigurableMacBase {
       }
 
       try (Scanner scanner = c.createScanner(table, Authorizations.EMPTY)) {
-        Iterator<Entry<Key,Value>> iter = scanner.iterator();
-        assertTrue(iter.hasNext());
-        Entry<Key,Value> entry = iter.next();
+        Entry<Key,Value> entry = getOnlyElement(scanner);
         assertEquals("r1", entry.getKey().getRow().toString());
         assertEquals("cf1", entry.getKey().getColumnFamily().toString());
         assertEquals("cq1", entry.getKey().getColumnQualifier().toString());
         assertEquals("v1", entry.getValue().toString());
-        assertFalse(iter.hasNext());
       }
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
index 4d376d6b22..6013c18077 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
@@ -33,7 +33,6 @@ import java.time.Duration;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
@@ -379,12 +378,9 @@ public class KerberosIT extends AccumuloITBase {
 
       // Read (and proper authorizations)
       try (Scanner s = client.createScanner(table, new Authorizations(viz))) {
-        Iterator<Entry<Key,Value>> iter = s.iterator();
-        assertTrue(iter.hasNext(), "No results from iterator");
-        Entry<Key,Value> entry = iter.next();
+        Entry<Key,Value> entry = getOnlyElement(s);
         assertEquals(new Key("a", "b", "c", viz, ts), entry.getKey());
         assertEquals(new Value("d"), entry.getValue());
-        assertFalse(iter.hasNext(), "Had more results from iterator");
         return null;
       }
     });
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
index 6371697675..7f3ff88276 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
@@ -31,6 +31,7 @@ import java.time.Duration;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Map.Entry;
+import java.util.Objects;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -60,8 +61,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 
-import com.google.common.collect.Iterators;
-
 public class MetadataIT extends AccumuloClusterHarness {
 
   @Override
@@ -128,7 +127,7 @@ public class MetadataIT extends AccumuloClusterHarness {
       c.tableOperations().merge(MetadataTable.NAME, null, null);
       try (Scanner s = c.createScanner(RootTable.NAME, Authorizations.EMPTY)) {
         s.setRange(DeletesSection.getRange());
-        while (Iterators.size(s.iterator()) == 0) {
+        while (s.stream().findAny().isEmpty()) {
           sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
         }
         assertEquals(0, c.tableOperations().listSplits(MetadataTable.NAME).size());
@@ -143,26 +142,15 @@ public class MetadataIT extends AccumuloClusterHarness {
       c.tableOperations().create(tableName);
 
       // batch scan regular metadata table
-      int count = 0;
       try (BatchScanner s = c.createBatchScanner(MetadataTable.NAME)) {
         s.setRanges(Collections.singleton(new Range()));
-        for (Entry<Key,Value> e : s) {
-          if (e != null)
-            count++;
-        }
+        assertTrue(s.stream().anyMatch(Objects::nonNull));
       }
 
-      assertTrue(count > 0);
-
       // batch scan root metadata table
       try (BatchScanner s = c.createBatchScanner(RootTable.NAME)) {
         s.setRanges(Collections.singleton(new Range()));
-        count = 0;
-        for (Entry<Key,Value> e : s) {
-          if (e != null)
-            count++;
-        }
-        assertTrue(count > 0);
+        assertTrue(s.stream().anyMatch(Objects::nonNull));
       }
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RowDeleteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RowDeleteIT.java
index d586836904..5c3a1222e2 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RowDeleteIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RowDeleteIT.java
@@ -21,11 +21,10 @@ package org.apache.accumulo.test.functional;
 import static org.apache.accumulo.test.functional.FunctionalTestUtils.checkRFiles;
 import static org.apache.accumulo.test.functional.FunctionalTestUtils.nm;
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
-import java.util.Collections;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
@@ -34,6 +33,7 @@ import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.user.RowDeletingIterator;
@@ -63,27 +63,26 @@ public class RowDeleteIT extends AccumuloClusterHarness {
   @Test
   public void run() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = getUniqueNames(1)[0];
-      c.tableOperations().create(tableName);
-      Map<String,Set<Text>> groups = new HashMap<>();
-      groups.put("lg1", Collections.singleton(new Text("foo")));
-      c.tableOperations().setLocalityGroups(tableName, groups);
+      final String tableName = getUniqueNames(1)[0];
+      NewTableConfiguration ntc = new NewTableConfiguration();
       IteratorSetting setting = new IteratorSetting(30, RowDeletingIterator.class);
-      c.tableOperations().attachIterator(tableName, setting, EnumSet.of(IteratorScope.majc));
-      c.tableOperations().setProperty(tableName, Property.TABLE_MAJC_RATIO.getKey(), "100");
+      ntc.attachIterator(setting, EnumSet.of(IteratorScope.majc));
+      ntc.setLocalityGroups(Map.of("lg1", Set.of(new Text("foo"))));
+      ntc.setProperties(Map.of(Property.TABLE_MAJC_RATIO.getKey(), "100"));
+      c.tableOperations().create(tableName, ntc);
 
-      BatchWriter bw = c.createBatchWriter(tableName);
+      try (BatchWriter bw = c.createBatchWriter(tableName);
+          Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY)) {
 
-      bw.addMutation(nm("r1", "foo", "cf1", "v1"));
-      bw.addMutation(nm("r1", "bar", "cf1", "v2"));
+        bw.addMutation(nm("r1", "foo", "cf1", "v1"));
+        bw.addMutation(nm("r1", "bar", "cf1", "v2"));
 
-      bw.flush();
-      c.tableOperations().flush(tableName, null, null, true);
+        bw.flush();
+        c.tableOperations().flush(tableName, null, null, true);
 
-      checkRFiles(c, tableName, 1, 1, 1, 1);
+        checkRFiles(c, tableName, 1, 1, 1, 1);
 
-      int count;
-      try (Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY)) {
+        int count;
         count = Iterators.size(scanner.iterator());
         assertEquals(2, count, "count == " + count);
 
@@ -93,21 +92,15 @@ public class RowDeleteIT extends AccumuloClusterHarness {
         c.tableOperations().flush(tableName, null, null, true);
 
         checkRFiles(c, tableName, 1, 1, 2, 2);
-      }
 
-      try (Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY)) {
         count = Iterators.size(scanner.iterator());
         assertEquals(3, count, "count == " + count);
 
         c.tableOperations().compact(tableName, null, null, false, true);
 
         checkRFiles(c, tableName, 1, 1, 0, 0);
-      }
 
-      try (Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY)) {
-        count = Iterators.size(scanner.iterator());
-        assertEquals(0, count, "count == " + count);
-        bw.close();
+        assertTrue(scanner.stream().findAny().isEmpty());
       }
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
index f16ae619d8..85074bd5ea 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
@@ -47,8 +47,6 @@ import org.apache.hadoop.fs.Path;
 import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
 
-import com.google.common.collect.Iterators;
-
 @Tag(MINI_CLUSTER_ONLY)
 public class TableIT extends AccumuloClusterHarness {
 
@@ -78,12 +76,12 @@ public class TableIT extends AccumuloClusterHarness {
       try (Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
         s.setRange(new KeyExtent(id, null, null).toMetaRange());
         s.fetchColumnFamily(DataFileColumnFamily.NAME);
-        assertTrue(Iterators.size(s.iterator()) > 0);
+        assertTrue(s.stream().findAny().isPresent());
 
         FileSystem fs = getCluster().getFileSystem();
         assertTrue(fs.listStatus(new Path(rootPath + "/accumulo/tables/" + id)).length > 0);
         to.delete(tableName);
-        assertEquals(0, Iterators.size(s.iterator()));
+        assertTrue(s.stream().findAny().isEmpty());
 
         try {
           assertEquals(0, fs.listStatus(new Path(rootPath + "/accumulo/tables/" + id)).length);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
index ff16dc5eef..7c2092ab2f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
@@ -20,14 +20,10 @@ package org.apache.accumulo.test.functional;
 
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 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 java.time.Duration;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -36,9 +32,7 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -83,10 +77,8 @@ public class ZookeeperRestartIT extends ConfigurableMacBase {
 
       // use the tservers
       try (Scanner s = c.createScanner("test_ingest", Authorizations.EMPTY)) {
-        Iterator<Entry<Key,Value>> i = s.iterator();
-        assertTrue(i.hasNext());
-        assertEquals("row", i.next().getKey().getRow().toString());
-        assertFalse(i.hasNext());
+        String actual = getOnlyElement(s).getKey().getRow().toString();
+        assertEquals("row", actual);
         // use the manager
         c.tableOperations().delete("test_ingest");
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
index 2c96cf57a9..7b1930d00d 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
@@ -18,15 +18,14 @@
  */
 package org.apache.accumulo.test.mapred;
 
+import static com.google.common.collect.MoreCollectors.onlyElement;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
@@ -233,11 +232,9 @@ public class AccumuloOutputFormatIT extends ConfigurableMacBase {
       assertNull(e1);
 
       try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        Iterator<Entry<Key,Value>> iter = scanner.iterator();
-        assertTrue(iter.hasNext());
-        Entry<Key,Value> entry = iter.next();
-        assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-        assertFalse(iter.hasNext());
+        int actual = scanner.stream().map(Entry::getValue).map(Value::get).map(String::new)
+            .map(Integer::parseInt).collect(onlyElement());
+        assertEquals(100, actual);
       }
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
index 702f2c6684..d415aa76c0 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.test.mapred;
 
+import static com.google.common.collect.MoreCollectors.onlyElement;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -27,7 +27,6 @@ import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -199,11 +198,9 @@ public class TokenFileIT extends AccumuloClusterHarness {
       assertNull(e1);
 
       try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        Iterator<Entry<Key,Value>> iter = scanner.iterator();
-        assertTrue(iter.hasNext());
-        Entry<Key,Value> entry = iter.next();
-        assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-        assertFalse(iter.hasNext());
+        int actual = scanner.stream().map(Entry::getValue).map(Value::get).map(String::new)
+            .map(Integer::parseInt).collect(onlyElement());
+        assertEquals(100, actual);
       }
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java
index 2c5e5bcf99..dc0aaa9dd6 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java
@@ -18,14 +18,12 @@
  */
 package org.apache.accumulo.test.mapreduce;
 
+import static com.google.common.collect.MoreCollectors.onlyElement;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -164,11 +162,9 @@ public class AccumuloOutputFormatIT extends AccumuloClusterHarness {
       assertNull(e1);
 
       try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        Iterator<Entry<Key,Value>> iter = scanner.iterator();
-        assertTrue(iter.hasNext());
-        Entry<Key,Value> entry = iter.next();
-        assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-        assertFalse(iter.hasNext());
+        int actual = scanner.stream().map(Entry::getValue).map(Value::get).map(String::new)
+            .map(Integer::parseInt).collect(onlyElement());
+        assertEquals(100, actual);
       }
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
index 8a77b1a393..8079cf9d3e 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.test.mapreduce;
 
+import static com.google.common.collect.MoreCollectors.onlyElement;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -28,7 +28,6 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.net.URI;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -176,13 +175,13 @@ public class TokenFileIT extends AccumuloClusterHarness {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       c.tableOperations().create(table1);
       c.tableOperations().create(table2);
-      BatchWriter bw = c.createBatchWriter(table1);
-      for (int i = 0; i < 100; i++) {
-        Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-        m.put("", "", String.format("%09x", i));
-        bw.addMutation(m);
+      try (BatchWriter bw = c.createBatchWriter(table1)) {
+        for (int i = 0; i < 100; i++) {
+          Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+          m.put("", "", String.format("%09x", i));
+          bw.addMutation(m);
+        }
       }
-      bw.close();
 
       File tf = new File(tempDir, "root_test.pw");
       assertTrue(tf.createNewFile(), "Failed to create file: " + tf);
@@ -204,11 +203,9 @@ public class TokenFileIT extends AccumuloClusterHarness {
       assertNull(e1);
 
       try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        Iterator<Entry<Key,Value>> iter = scanner.iterator();
-        assertTrue(iter.hasNext());
-        Entry<Key,Value> entry = iter.next();
-        assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-        assertFalse(iter.hasNext());
+        int actual = scanner.stream().map(Entry::getValue).map(Value::get).map(String::new)
+            .map(Integer::parseInt).collect(onlyElement());
+        assertEquals(100, actual);
       }
     }
   }