[ 
https://issues.apache.org/jira/browse/BEAM-3060?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16281008#comment-16281008
 ] 

ASF GitHub Bot commented on BEAM-3060:
--------------------------------------

jkff closed pull request #4189: [BEAM-3060] add TFRecordIOIT
URL: https://github.com/apache/beam/pull/4189
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java
 
b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java
index 5a29d4f8126..e7b475d4caa 100644
--- 
a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java
+++ 
b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java
@@ -19,6 +19,7 @@
 
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.Validation;
 import org.apache.beam.sdk.testing.TestPipelineOptions;
 
 /**
@@ -96,7 +97,7 @@
   void setNumberOfRecords(Long count);
 
   @Description("Destination prefix for files generated by the test")
-  @Default.String("TEXTIOIT")
+  @Validation.Required
   String getFilenamePrefix();
 
   void setFilenamePrefix(String prefix);
diff --git 
a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/common/FileBasedIOITHelper.java
 
b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/common/FileBasedIOITHelper.java
new file mode 100644
index 00000000000..cf20d8e5954
--- /dev/null
+++ 
b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/common/FileBasedIOITHelper.java
@@ -0,0 +1,103 @@
+/*
+ * 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
+ *
+ *     http://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.beam.sdk.io.common;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.io.fs.ResourceId;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.DoFn;
+
+/**
+ * Contains helper methods for file based IO Integration tests.
+ */
+public class FileBasedIOITHelper {
+
+  private FileBasedIOITHelper() {
+  }
+
+  public static IOTestPipelineOptions readTestPipelineOptions() {
+    PipelineOptionsFactory.register(IOTestPipelineOptions.class);
+    IOTestPipelineOptions options = TestPipeline
+        .testingPipelineOptions()
+        .as(IOTestPipelineOptions.class);
+
+    return PipelineOptionsValidator.validate(IOTestPipelineOptions.class, 
options);
+  }
+
+  public static String appendTimestampToPrefix(String filenamePrefix) {
+    return String.format("%s_%s", filenamePrefix, new Date().getTime());
+  }
+
+  public static String getExpectedHashForLineCount(Long lineCount) {
+    Map<Long, String> expectedHashes = ImmutableMap.of(
+        100_000L, "4c8bb3b99dcc59459b20fefba400d446",
+        1_000_000L, "9796db06e7a7960f974d5a91164afff1",
+        100_000_000L, "6ce05f456e2fdc846ded2abd0ec1de95"
+    );
+
+    String hash = expectedHashes.get(lineCount);
+    if (hash == null) {
+      throw new UnsupportedOperationException(
+          String.format("No hash for that line count: %s", lineCount)
+      );
+    }
+    return hash;
+  }
+
+  /**
+   * Constructs text lines in files used for testing.
+   */
+  public static class DeterministicallyConstructTestTextLineFn extends 
DoFn<Long, String> {
+
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.output(String.format("IO IT Test line of text. Line seed: %s", 
c.element()));
+    }
+  }
+
+  /**
+   * Deletes matching files using the FileSystems API.
+   */
+  public static class DeleteFileFn extends DoFn<String, Void> {
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws IOException {
+      MatchResult match = Iterables
+          
.getOnlyElement(FileSystems.match(Collections.singletonList(c.element())));
+
+      Set<ResourceId> resourceIds = new HashSet<>();
+      for (MatchResult.Metadata metadataElem : match.metadata()) {
+        resourceIds.add(metadataElem.resourceId());
+      }
+
+      FileSystems.delete(resourceIds);
+    }
+  }
+}
diff --git 
a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java
 
b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java
index e9aac8001b1..f9fad80696a 100644
--- 
a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java
+++ 
b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java
@@ -19,32 +19,20 @@
 package org.apache.beam.sdk.io.text;
 
 import static org.apache.beam.sdk.io.Compression.AUTO;
+import static 
org.apache.beam.sdk.io.common.FileBasedIOITHelper.appendTimestampToPrefix;
+import static 
org.apache.beam.sdk.io.common.FileBasedIOITHelper.getExpectedHashForLineCount;
+import static 
org.apache.beam.sdk.io.common.FileBasedIOITHelper.readTestPipelineOptions;
 
-import com.google.common.base.Function;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-
-import java.io.IOException;
 import java.text.ParseException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Map;
-
 import org.apache.beam.sdk.io.Compression;
-import org.apache.beam.sdk.io.FileSystems;
 import org.apache.beam.sdk.io.GenerateSequence;
 import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.common.FileBasedIOITHelper;
 import org.apache.beam.sdk.io.common.HashingFn;
 import org.apache.beam.sdk.io.common.IOTestPipelineOptions;
-import org.apache.beam.sdk.io.fs.MatchResult;
-import org.apache.beam.sdk.io.fs.ResourceId;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.Values;
 import org.apache.beam.sdk.transforms.View;
@@ -64,14 +52,14 @@
  *  -Dit.test=org.apache.beam.sdk.io.text.TextIOIT
  *  -DintegrationTestPipelineOptions='[
  *  "--numberOfRecords=100000",
- *  "--filenamePrefix=TEXTIOIT"
+ *  "--filenamePrefix=output_file_path",
  *  "--compressionType=GZIP"
  *  ]'
  * </pre>
  * </p>
  * <p>Please see 'sdks/java/io/file-based-io-tests/pom.xml' for instructions 
regarding
  * running this test using Beam performance testing framework.</p>
- * */
+ */
 @RunWith(JUnit4.class)
 public class TextIOIT {
 
@@ -84,26 +72,11 @@
 
   @BeforeClass
   public static void setup() throws ParseException {
-    PipelineOptionsFactory.register(IOTestPipelineOptions.class);
-    IOTestPipelineOptions options = TestPipeline.testingPipelineOptions()
-        .as(IOTestPipelineOptions.class);
+    IOTestPipelineOptions options = readTestPipelineOptions();
 
     numberOfTextLines = options.getNumberOfRecords();
-    filenamePrefix = appendTimestamp(options.getFilenamePrefix());
-    compressionType = parseCompressionType(options.getCompressionType());
-  }
-
-  private static Compression parseCompressionType(String compressionType) {
-    try {
-      return Compression.valueOf(compressionType.toUpperCase());
-    } catch (IllegalArgumentException ex) {
-      throw new IllegalArgumentException(
-          String.format("Unsupported compression type: %s", compressionType));
-    }
-  }
-
-  private static String appendTimestamp(String filenamePrefix) {
-    return String.format("%s_%s", filenamePrefix, new Date().getTime());
+    filenamePrefix = appendTimestampToPrefix(options.getFilenamePrefix());
+    compressionType = Compression.valueOf(options.getCompressionType());
   }
 
   @Test
@@ -116,7 +89,8 @@ public void writeThenReadAll() {
 
     PCollection<String> testFilenames = pipeline
         .apply("Generate sequence", 
GenerateSequence.from(0).to(numberOfTextLines))
-        .apply("Produce text lines", ParDo.of(new 
DeterministicallyConstructTestTextLineFn()))
+        .apply("Produce text lines",
+            ParDo.of(new 
FileBasedIOITHelper.DeterministicallyConstructTestTextLineFn()))
         .apply("Write content to files", write)
         .getPerDestinationOutputFilenames().apply(Values.<String>create());
 
@@ -127,53 +101,9 @@ public void writeThenReadAll() {
     String expectedHash = getExpectedHashForLineCount(numberOfTextLines);
     PAssert.thatSingleton(consolidatedHashcode).isEqualTo(expectedHash);
 
-    testFilenames.apply("Delete test files", ParDo.of(new DeleteFileFn())
+    testFilenames.apply("Delete test files", ParDo.of(new 
FileBasedIOITHelper.DeleteFileFn())
         
.withSideInputs(consolidatedHashcode.apply(View.<String>asSingleton())));
 
     pipeline.run().waitUntilFinish();
   }
-
-  private static String getExpectedHashForLineCount(Long lineCount) {
-    Map<Long, String> expectedHashes = ImmutableMap.of(
-        100_000L, "4c8bb3b99dcc59459b20fefba400d446",
-        1_000_000L, "9796db06e7a7960f974d5a91164afff1",
-        100_000_000L, "6ce05f456e2fdc846ded2abd0ec1de95"
-    );
-
-    String hash = expectedHashes.get(lineCount);
-    if (hash == null) {
-      throw new UnsupportedOperationException(
-          String.format("No hash for that line count: %s", lineCount));
-    }
-    return hash;
-  }
-
-  private static class DeterministicallyConstructTestTextLineFn extends 
DoFn<Long, String> {
-
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      c.output(String.format("IO IT Test line of text. Line seed: %s", 
c.element()));
-    }
-  }
-
-  private static class DeleteFileFn extends DoFn<String, Void> {
-
-    @ProcessElement
-    public void processElement(ProcessContext c) throws IOException {
-      MatchResult match = Iterables
-          
.getOnlyElement(FileSystems.match(Collections.singletonList(c.element())));
-      FileSystems.delete(toResourceIds(match));
-    }
-
-    private Collection<ResourceId> toResourceIds(MatchResult match) throws 
IOException {
-      return FluentIterable.from(match.metadata())
-          .transform(new Function<MatchResult.Metadata, ResourceId>() {
-
-            @Override
-            public ResourceId apply(MatchResult.Metadata metadata) {
-              return metadata.resourceId();
-            }
-          }).toList();
-    }
-  }
 }
diff --git 
a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/tfrecord/TFRecordIOIT.java
 
b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/tfrecord/TFRecordIOIT.java
new file mode 100644
index 00000000000..b887316b187
--- /dev/null
+++ 
b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/tfrecord/TFRecordIOIT.java
@@ -0,0 +1,137 @@
+/*
+ * 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
+ *
+ *     http://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.beam.sdk.io.tfrecord;
+
+import static org.apache.beam.sdk.io.Compression.AUTO;
+import static 
org.apache.beam.sdk.io.common.FileBasedIOITHelper.appendTimestampToPrefix;
+import static 
org.apache.beam.sdk.io.common.FileBasedIOITHelper.getExpectedHashForLineCount;
+import static 
org.apache.beam.sdk.io.common.FileBasedIOITHelper.readTestPipelineOptions;
+
+import java.text.ParseException;
+import org.apache.beam.sdk.io.Compression;
+import org.apache.beam.sdk.io.GenerateSequence;
+import org.apache.beam.sdk.io.TFRecordIO;
+import org.apache.beam.sdk.io.common.FileBasedIOITHelper;
+import org.apache.beam.sdk.io.common.HashingFn;
+import org.apache.beam.sdk.io.common.IOTestPipelineOptions;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Integration tests for {@link org.apache.beam.sdk.io.TFRecordIO}.
+ *
+ * <p>Run this test using the command below. Pass in connection information 
via PipelineOptions:
+ * <pre>
+ *  mvn -e -Pio-it verify -pl sdks/java/io/file-based-io-tests
+ *  -Dit.test=org.apache.beam.sdk.io.tfrecord.TFRecordIOIT
+ *  -DintegrationTestPipelineOptions='[
+ *  "--numberOfRecords=100000",
+ *  "--filenamePrefix=output_file_path",
+ *  "--compressionType=GZIP"
+ *  ]'
+ * </pre>
+ * </p>
+ * <p>Please {@see 'sdks/java/io/file-based-io-tests/pom.xml'} for 
instructions regarding
+ * running this test using Beam performance testing framework.</p>
+ */
+@RunWith(JUnit4.class)
+public class TFRecordIOIT {
+
+  private static String filenamePrefix;
+  private static Long numberOfTextLines;
+  private static Compression compressionType;
+
+  @Rule
+  public TestPipeline writePipeline = TestPipeline.create();
+
+  @Rule
+  public TestPipeline readPipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void setup() throws ParseException {
+    IOTestPipelineOptions options = readTestPipelineOptions();
+
+    numberOfTextLines = options.getNumberOfRecords();
+    filenamePrefix = appendTimestampToPrefix(options.getFilenamePrefix());
+    compressionType = Compression.valueOf(options.getCompressionType());
+  }
+
+  private static String createFilenamePattern() {
+    return filenamePrefix + "*";
+  }
+
+  // TODO: There are two pipelines due to: 
https://issues.apache.org/jira/browse/BEAM-3267
+  @Test
+  public void writeThenReadAll() {
+    TFRecordIO.Write writeTransform = TFRecordIO
+        .write()
+        .to(filenamePrefix)
+        .withCompression(compressionType)
+        .withSuffix(".tfrecord");
+
+    writePipeline
+        .apply("Generate sequence", 
GenerateSequence.from(0).to(numberOfTextLines))
+        .apply("Produce text lines",
+            ParDo.of(new 
FileBasedIOITHelper.DeterministicallyConstructTestTextLineFn()))
+        .apply("Transform strings to bytes", MapElements.via(new 
StringToByteArray()))
+        .apply("Write content to files", writeTransform);
+
+    writePipeline.run().waitUntilFinish();
+
+    String filenamePattern = createFilenamePattern();
+    PCollection<String> consolidatedHashcode = readPipeline
+        .apply(TFRecordIO.read().from(filenamePattern).withCompression(AUTO))
+        .apply("Transform bytes to strings", MapElements.via(new 
ByteArrayToString()))
+        .apply("Calculate hashcode", Combine.globally(new HashingFn()));
+
+    String expectedHash = getExpectedHashForLineCount(numberOfTextLines);
+    PAssert.thatSingleton(consolidatedHashcode).isEqualTo(expectedHash);
+
+    readPipeline.apply(Create.of(filenamePattern))
+        .apply("Delete test files", ParDo.of(new 
FileBasedIOITHelper.DeleteFileFn())
+        
.withSideInputs(consolidatedHashcode.apply(View.<String>asSingleton())));
+    readPipeline.run().waitUntilFinish();
+  }
+
+  static class StringToByteArray extends SimpleFunction<String, byte[]> {
+    @Override
+    public byte[] apply(String input) {
+      return input.getBytes();
+    }
+  }
+
+  static class ByteArrayToString extends SimpleFunction<byte[], String> {
+    @Override
+    public String apply(byte[] input) {
+      return new String(input);
+    }
+  }
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Add performance tests for commonly used file-based I/O PTransforms
> ------------------------------------------------------------------
>
>                 Key: BEAM-3060
>                 URL: https://issues.apache.org/jira/browse/BEAM-3060
>             Project: Beam
>          Issue Type: Test
>          Components: sdk-java-core
>            Reporter: Chamikara Jayalath
>            Assignee: Szymon Nieradka
>
> We recently added a performance testing framework [1] that can be used to do 
> following.
> (1) Execute Beam tests using PerfkitBenchmarker
> (2) Manage Kubernetes-based deployments of data stores.
> (3) Easily publish benchmark results. 
> I think it will be useful to add performance tests for commonly used 
> file-based I/O PTransforms using this framework. I suggest looking into 
> following formats initially.
> (1) AvroIO
> (2) TextIO
> (3) Compressed text using TextIO
> (4) TFRecordIO
> It should be possibly to run these tests for various Beam runners (Direct, 
> Dataflow, Flink, Spark, etc.) and file-systems (GCS, local, HDFS, etc.) 
> easily.
> In the initial version, tests can be made manually triggerable for PRs 
> through Jenkins. Later, we could make some of these tests run periodically 
> and publish benchmark results (to BigQuery) through PerfkitBenchmarker.
> [1] https://beam.apache.org/documentation/io/testing/



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to