[BEAM-9120] Make FileChecksumMatcher extend TypeSafeMatcher<ShardedFile> (#10588)

diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
index 836ea08d..11d3d78 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
@@ -17,14 +17,17 @@
  */
 package org.apache.beam.examples;
 
+import static org.apache.beam.sdk.testing.FileChecksumMatcher.fileContentsHaveChecksum;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 import java.util.Date;
 import org.apache.beam.examples.WordCount.WordCountOptions;
 import org.apache.beam.sdk.io.FileSystems;
 import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.FileChecksumMatcher;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.util.NumberedShardedFile;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -63,9 +66,9 @@
             .resolve("output", StandardResolveOptions.RESOLVE_DIRECTORY)
             .resolve("results", StandardResolveOptions.RESOLVE_FILE)
             .toString());
-    options.setOnSuccessMatcher(
-        new FileChecksumMatcher(DEFAULT_OUTPUT_CHECKSUM, options.getOutput() + "*-of-*"));
-
     WordCount.runWordCount(options);
+    assertThat(
+        new NumberedShardedFile(options.getOutput() + "*-of-*"),
+        fileContentsHaveChecksum(DEFAULT_OUTPUT_CHECKSUM));
   }
 }
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfIT.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfIT.java
index e6701c8..64fdd79 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfIT.java
@@ -17,15 +17,18 @@
  */
 package org.apache.beam.examples.complete;
 
+import static org.apache.beam.sdk.testing.FileChecksumMatcher.fileContentsHaveChecksum;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 import java.util.Date;
 import java.util.regex.Pattern;
 import org.apache.beam.examples.complete.TfIdf.Options;
 import org.apache.beam.sdk.io.FileSystems;
 import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.FileChecksumMatcher;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.util.NumberedShardedFile;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -64,10 +67,10 @@
             .resolve("output", StandardResolveOptions.RESOLVE_DIRECTORY)
             .resolve("results", StandardResolveOptions.RESOLVE_FILE)
             .toString());
-    options.setOnSuccessMatcher(
-        new FileChecksumMatcher(
-            EXPECTED_OUTPUT_CHECKSUM, options.getOutput() + "*-of-*.csv", DEFAULT_SHARD_TEMPLATE));
-
     TfIdf.runTfIdf(options);
+
+    assertThat(
+        new NumberedShardedFile(options.getOutput() + "*-of-*.csv", DEFAULT_SHARD_TEMPLATE),
+        fileContentsHaveChecksum(EXPECTED_OUTPUT_CHECKSUM));
   }
 }
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsIT.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsIT.java
index 9eb9a86..765fed5 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsIT.java
@@ -17,13 +17,16 @@
  */
 package org.apache.beam.examples.complete;
 
+import static org.apache.beam.sdk.testing.FileChecksumMatcher.fileContentsHaveChecksum;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 import java.util.Date;
 import org.apache.beam.sdk.io.FileSystems;
 import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.FileChecksumMatcher;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.util.NumberedShardedFile;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -60,9 +63,11 @@
             .resolve("output", StandardResolveOptions.RESOLVE_DIRECTORY)
             .resolve("results", StandardResolveOptions.RESOLVE_FILE)
             .toString());
-    options.setOnSuccessMatcher(
-        new FileChecksumMatcher(DEFAULT_OUTPUT_CHECKSUM, options.getOutput() + "*-of-*"));
 
     TopWikipediaSessions.run(options);
+
+    assertThat(
+        new NumberedShardedFile(options.getOutput() + "*-of-*"),
+        fileContentsHaveChecksum(DEFAULT_OUTPUT_CHECKSUM));
   }
 }
diff --git a/examples/kotlin/src/test/java/org/apache/beam/examples/WindowedWordCountITKotlin.kt b/examples/kotlin/src/test/java/org/apache/beam/examples/WindowedWordCountITKotlin.kt
index ee2ec23..9298bfe 100644
--- a/examples/kotlin/src/test/java/org/apache/beam/examples/WindowedWordCountITKotlin.kt
+++ b/examples/kotlin/src/test/java/org/apache/beam/examples/WindowedWordCountITKotlin.kt
@@ -35,6 +35,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists
+import org.hamcrest.MatcherAssert.assertThat
 import org.hamcrest.Description
 import org.hamcrest.Matchers.equalTo
 import org.hamcrest.TypeSafeMatcher
diff --git a/examples/kotlin/src/test/java/org/apache/beam/examples/WordCountITKotlin.java b/examples/kotlin/src/test/java/org/apache/beam/examples/WordCountITKotlin.java
index ef016b9..dba629c 100644
--- a/examples/kotlin/src/test/java/org/apache/beam/examples/WordCountITKotlin.java
+++ b/examples/kotlin/src/test/java/org/apache/beam/examples/WordCountITKotlin.java
@@ -17,15 +17,18 @@
  */
 package org.apache.beam.examples;
 
+import static org.apache.beam.sdk.testing.FileChecksumMatcher.fileContentsHaveChecksum;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 import java.util.Date;
 import org.apache.beam.examples.kotlin.WordCount;
 import org.apache.beam.examples.kotlin.WordCount.WordCountOptions;
 import org.apache.beam.sdk.io.FileSystems;
 import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.FileChecksumMatcher;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.util.NumberedShardedFile;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -64,9 +67,11 @@
             .resolve("output", StandardResolveOptions.RESOLVE_DIRECTORY)
             .resolve("results", StandardResolveOptions.RESOLVE_FILE)
             .toString());
-    options.setOnSuccessMatcher(
-        new FileChecksumMatcher(DEFAULT_OUTPUT_CHECKSUM, options.getOutput() + "*-of-*"));
 
     WordCount.runWordCount(options);
+
+    assertThat(
+        new NumberedShardedFile(options.getOutput() + "*-of-*"),
+        fileContentsHaveChecksum(DEFAULT_OUTPUT_CHECKSUM));
   }
 }
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java
index d5be223..6908563 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.runners.flink;
 
+import static org.apache.beam.sdk.testing.FileChecksumMatcher.fileContentsHaveChecksum;
 import static org.hamcrest.MatcherAssert.assertThat;
 
-import java.util.Collections;
 import java.util.Date;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
@@ -31,8 +31,6 @@
 import org.apache.beam.sdk.io.fs.ResourceId;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.FileChecksumMatcher;
-import org.apache.beam.sdk.testing.SerializableMatchers;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.MapElements;
@@ -167,12 +165,11 @@
     waitUntilJobIsDone();
 
     assertThat(
-        new FlinkRunnerResult(Collections.emptyMap(), 1L),
-        SerializableMatchers.allOf(
-            new FileChecksumMatcher(
-                VALUE_CHECKSUM, new FilePatternMatchingShardedFile(singleOutputPrefix + "*")),
-            new FileChecksumMatcher(
-                VALUE_CHECKSUM, new FilePatternMatchingShardedFile(multiOutputPrefix + "*"))));
+        new FilePatternMatchingShardedFile(singleOutputPrefix + "*"),
+        fileContentsHaveChecksum(VALUE_CHECKSUM));
+    assertThat(
+        new FilePatternMatchingShardedFile(multiOutputPrefix + "*"),
+        fileContentsHaveChecksum(VALUE_CHECKSUM));
   }
 
   private JobGraph getJobGraph(Pipeline pipeline) {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java
index 28349ab..497d69b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java
@@ -18,17 +18,12 @@
 package org.apache.beam.sdk.testing;
 
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
 
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.regex.Pattern;
 import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.util.FluentBackoff;
-import org.apache.beam.sdk.util.NumberedShardedFile;
 import org.apache.beam.sdk.util.ShardedFile;
 import org.apache.beam.sdk.util.Sleeper;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
@@ -41,25 +36,25 @@
 import org.slf4j.LoggerFactory;
 
 /**
- * Matcher to verify file checksum in E2E test.
+ * Matcher to verify checksum of the contents of an {@link ShardedFile} in E2E test.
  *
  * <p>For example:
  *
  * <pre>{@code
- * assertThat(job, new FileChecksumMatcher(checksumString, filePath));
+ * assertThat(new NumberedShardedFile(filePath), fileContentsHaveChecksum(checksumString));
  * }</pre>
  *
  * or
  *
  * <pre>{@code
- * assertThat(job, new FileChecksumMatcher(checksumString, filePath, shardTemplate));
+ * assertThat(new NumberedShardedFile(filePath, shardTemplate), fileContentsHaveChecksum(checksumString));
  * }</pre>
  *
  * <p>Checksum of outputs is generated based on SHA-1 algorithm. If output file is empty, SHA-1 hash
  * of empty string (da39a3ee5e6b4b0d3255bfef95601890afd80709) is used as expected.
  */
-public class FileChecksumMatcher extends TypeSafeMatcher<PipelineResult>
-    implements SerializableMatcher<PipelineResult> {
+public class FileChecksumMatcher extends TypeSafeMatcher<ShardedFile>
+    implements SerializableMatcher<ShardedFile> {
 
   private static final Logger LOG = LoggerFactory.getLogger(FileChecksumMatcher.class);
 
@@ -70,83 +65,40 @@
           .withInitialBackoff(DEFAULT_SLEEP_DURATION)
           .withMaxRetries(MAX_READ_RETRIES);
 
-  private static final Pattern DEFAULT_SHARD_TEMPLATE =
-      Pattern.compile("(?x) \\S* (?<shardnum> \\d+) -of- (?<numshards> \\d+)");
-
   private final String expectedChecksum;
-  private final ShardedFile shardedFile;
+  private String actualChecksum;
 
-  /** Access via {@link #getActualChecksum()}. */
-  @Nullable private String actualChecksum;
-
-  /**
-   * Constructor that uses default shard template.
-   *
-   * @param checksum expected checksum string used to verify file content.
-   * @param filePath path of files that's to be verified.
-   */
-  public FileChecksumMatcher(String checksum, String filePath) {
-    this(checksum, filePath, DEFAULT_SHARD_TEMPLATE);
-  }
-
-  /**
-   * Constructor using a custom shard template.
-   *
-   * @param checksum expected checksum string used to verify file content.
-   * @param filePath path of files that's to be verified.
-   * @param shardTemplate template of shard name to parse out the total number of shards which is
-   *     used in I/O retry to avoid inconsistency of filesystem. Customized template should assign
-   *     name "numshards" to capturing group - total shard number.
-   */
-  public FileChecksumMatcher(String checksum, String filePath, Pattern shardTemplate) {
+  private FileChecksumMatcher(String checksum) {
     checkArgument(
         !Strings.isNullOrEmpty(checksum), "Expected valid checksum, but received %s", checksum);
-    checkArgument(
-        !Strings.isNullOrEmpty(filePath), "Expected valid file path, but received %s", filePath);
-    checkNotNull(
-        shardTemplate,
-        "Expected non-null shard pattern. "
-            + "Please call the other constructor to use default pattern: %s",
-        DEFAULT_SHARD_TEMPLATE);
-
     this.expectedChecksum = checksum;
-    this.shardedFile = new NumberedShardedFile(filePath, shardTemplate);
   }
 
-  /**
-   * Constructor using an entirely custom {@link ShardedFile} implementation.
-   *
-   * <p>For internal use only.
-   */
-  public FileChecksumMatcher(String expectedChecksum, ShardedFile shardedFile) {
-    this.expectedChecksum = expectedChecksum;
-    this.shardedFile = shardedFile;
+  public static FileChecksumMatcher fileContentsHaveChecksum(String checksum) {
+    return new FileChecksumMatcher(checksum);
   }
 
   @Override
-  public boolean matchesSafely(PipelineResult pipelineResult) {
-    return getActualChecksum().equals(expectedChecksum);
+  public boolean matchesSafely(ShardedFile shardedFile) {
+    return getActualChecksum(shardedFile).equals(expectedChecksum);
   }
 
   /**
-   * Computes a checksum of the sharded file specified in the constructor. Not safe to call until
-   * the writing is complete.
+   * Computes a checksum of the given sharded file. Not safe to call until the writing is complete.
    */
-  private String getActualChecksum() {
-    if (actualChecksum == null) {
-      // Load output data
-      List<String> outputs;
-      try {
-        outputs = shardedFile.readFilesWithRetries(Sleeper.DEFAULT, BACK_OFF_FACTORY.backoff());
-      } catch (Exception e) {
-        throw new RuntimeException(String.format("Failed to read from: %s", shardedFile), e);
-      }
-
-      // Verify outputs. Checksum is computed using SHA-1 algorithm
-      actualChecksum = computeHash(outputs);
-      LOG.debug("Generated checksum: {}", actualChecksum);
+  private String getActualChecksum(ShardedFile shardedFile) {
+    // Load output data
+    List<String> outputs;
+    try {
+      outputs = shardedFile.readFilesWithRetries(Sleeper.DEFAULT, BACK_OFF_FACTORY.backoff());
+    } catch (Exception e) {
+      throw new RuntimeException(String.format("Failed to read from: %s", shardedFile), e);
     }
 
+    // Verify outputs. Checksum is computed using SHA-1 algorithm
+    actualChecksum = computeHash(outputs);
+    LOG.debug("Generated checksum: {}", actualChecksum);
+
     return actualChecksum;
   }
 
@@ -168,7 +120,7 @@
   }
 
   @Override
-  public void describeMismatchSafely(PipelineResult pResult, Description description) {
-    description.appendText("was (").appendText(getActualChecksum()).appendText(")");
+  public void describeMismatchSafely(ShardedFile shardedFile, Description description) {
+    description.appendText("was (").appendText(actualChecksum).appendText(")");
   }
 }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/RequiresStableInputIT.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/RequiresStableInputIT.java
index 1403753..6d06d33 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/RequiresStableInputIT.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/RequiresStableInputIT.java
@@ -17,6 +17,9 @@
  */
 package org.apache.beam.sdk;
 
+import static org.apache.beam.sdk.testing.FileChecksumMatcher.fileContentsHaveChecksum;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.WritableByteChannel;
@@ -28,8 +31,6 @@
 import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
 import org.apache.beam.sdk.io.fs.ResourceId;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.FileChecksumMatcher;
-import org.apache.beam.sdk.testing.SerializableMatchers;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.testing.TestPipelineOptions;
 import org.apache.beam.sdk.transforms.Create;
@@ -137,13 +138,6 @@
             .resolve("key-", StandardResolveOptions.RESOLVE_FILE)
             .toString();
 
-    options.setOnSuccessMatcher(
-        SerializableMatchers.allOf(
-            new FileChecksumMatcher(
-                VALUE_CHECKSUM, new FilePatternMatchingShardedFile(singleOutputPrefix + "*")),
-            new FileChecksumMatcher(
-                VALUE_CHECKSUM, new FilePatternMatchingShardedFile(multiOutputPrefix + "*"))));
-
     Pipeline p = Pipeline.create(options);
 
     SerializableFunction<Void, Void> firstTime =
@@ -168,5 +162,12 @@
                 .withOutputTags(new TupleTag<>(), TupleTagList.empty()));
 
     p.run().waitUntilFinish();
+
+    assertThat(
+        new FilePatternMatchingShardedFile(singleOutputPrefix + "*"),
+        fileContentsHaveChecksum(VALUE_CHECKSUM));
+    assertThat(
+        new FilePatternMatchingShardedFile(multiOutputPrefix + "*"),
+        fileContentsHaveChecksum(VALUE_CHECKSUM));
   }
 }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java
index fd3927f..b2c1eb1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.testing;
 
+import static org.apache.beam.sdk.testing.FileChecksumMatcher.fileContentsHaveChecksum;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.containsString;
 
@@ -24,7 +25,7 @@
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.regex.Pattern;
-import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.util.NumberedShardedFile;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.Files;
 import org.junit.Rule;
 import org.junit.Test;
@@ -32,8 +33,6 @@
 import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.Mockito;
 
 /** Tests for {@link FileChecksumMatcher}. */
 @RunWith(JUnit4.class)
@@ -41,53 +40,28 @@
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
   @Rule public ExpectedException thrown = ExpectedException.none();
 
-  @Mock private PipelineResult pResult = Mockito.mock(PipelineResult.class);
-
   @Test
   public void testPreconditionChecksumIsNull() throws IOException {
-    String tmpPath = tmpFolder.newFile().getPath();
-
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage(containsString("Expected valid checksum, but received"));
-    new FileChecksumMatcher(null, tmpPath);
+    fileContentsHaveChecksum(null);
   }
 
   @Test
   public void testPreconditionChecksumIsEmpty() throws IOException {
-    String tmpPath = tmpFolder.newFile().getPath();
-
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage(containsString("Expected valid checksum, but received"));
-    new FileChecksumMatcher("", tmpPath);
-  }
-
-  @Test
-  public void testPreconditionFilePathIsEmpty() {
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(containsString("Expected valid file path, but received"));
-    new FileChecksumMatcher("checksumString", "");
-  }
-
-  @Test
-  public void testPreconditionShardTemplateIsNull() throws IOException {
-    String tmpPath = tmpFolder.newFile().getPath();
-
-    thrown.expect(NullPointerException.class);
-    thrown.expectMessage(
-        containsString(
-            "Expected non-null shard pattern. "
-                + "Please call the other constructor to use default pattern:"));
-    new FileChecksumMatcher("checksumString", tmpPath, null);
+    fileContentsHaveChecksum("");
   }
 
   @Test
   public void testMatcherThatVerifiesSingleFile() throws IOException {
     File tmpFile = tmpFolder.newFile("result-000-of-001");
     Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8);
-    FileChecksumMatcher matcher =
-        new FileChecksumMatcher("a8772322f5d7b851777f820fc79d050f9d302915", tmpFile.getPath());
 
-    assertThat(pResult, matcher);
+    assertThat(
+        new NumberedShardedFile(tmpFile.getPath()),
+        fileContentsHaveChecksum("a8772322f5d7b851777f820fc79d050f9d302915"));
   }
 
   @Test
@@ -99,24 +73,19 @@
     Files.write("it is not a question.", tmpFile2, StandardCharsets.UTF_8);
     Files.write("tmp", tmpFile3, StandardCharsets.UTF_8);
 
-    FileChecksumMatcher matcher =
-        new FileChecksumMatcher(
-            "90552392c28396935fe4f123bd0b5c2d0f6260c8",
-            tmpFolder.getRoot().toPath().resolve("result-*").toString());
-
-    assertThat(pResult, matcher);
+    assertThat(
+        new NumberedShardedFile(tmpFolder.getRoot().toPath().resolve("result-*").toString()),
+        fileContentsHaveChecksum("90552392c28396935fe4f123bd0b5c2d0f6260c8"));
   }
 
   @Test
   public void testMatcherThatVerifiesFileWithEmptyContent() throws IOException {
     File emptyFile = tmpFolder.newFile("result-000-of-001");
     Files.write("", emptyFile, StandardCharsets.UTF_8);
-    FileChecksumMatcher matcher =
-        new FileChecksumMatcher(
-            "da39a3ee5e6b4b0d3255bfef95601890afd80709",
-            tmpFolder.getRoot().toPath().resolve("*").toString());
 
-    assertThat(pResult, matcher);
+    assertThat(
+        new NumberedShardedFile(tmpFolder.getRoot().toPath().resolve("*").toString()),
+        fileContentsHaveChecksum("da39a3ee5e6b4b0d3255bfef95601890afd80709"));
   }
 
   @Test
@@ -129,12 +98,10 @@
 
     Pattern customizedTemplate =
         Pattern.compile("(?x) result (?<shardnum>\\d+) - total (?<numshards>\\d+)");
-    FileChecksumMatcher matcher =
-        new FileChecksumMatcher(
-            "90552392c28396935fe4f123bd0b5c2d0f6260c8",
-            tmpFolder.getRoot().toPath().resolve("*").toString(),
-            customizedTemplate);
 
-    assertThat(pResult, matcher);
+    assertThat(
+        new NumberedShardedFile(
+            tmpFolder.getRoot().toPath().resolve("*").toString(), customizedTemplate),
+        fileContentsHaveChecksum("90552392c28396935fe4f123bd0b5c2d0f6260c8"));
   }
 }
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java
index 9f092e7..66fe8bf 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.io.gcp.storage;
 
+import static org.apache.beam.sdk.testing.FileChecksumMatcher.fileContentsHaveChecksum;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertNotNull;
@@ -36,10 +37,10 @@
 import org.apache.beam.sdk.io.fs.MatchResult.Metadata;
 import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
 import org.apache.beam.sdk.io.fs.ResourceId;
-import org.apache.beam.sdk.testing.FileChecksumMatcher;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.testing.TestPipelineOptions;
 import org.apache.beam.sdk.testing.UsesKms;
+import org.apache.beam.sdk.util.NumberedShardedFile;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -87,7 +88,7 @@
     assertThat(state, equalTo(State.DONE));
 
     String filePattern = filenamePrefix + "*-of-*";
-    assertThat(result, new FileChecksumMatcher(EXPECTED_CHECKSUM, filePattern));
+    assertThat(new NumberedShardedFile(filePattern), fileContentsHaveChecksum(EXPECTED_CHECKSUM));
 
     // Verify objects have KMS key set.
     try {