Skip to content

Commit

Permalink
Parametrized test
Browse files Browse the repository at this point in the history
  • Loading branch information
szewi committed Nov 3, 2017
1 parent 6d04d78 commit e9b7418
Show file tree
Hide file tree
Showing 2 changed files with 33 additions and 2 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
package org.apache.beam.sdk.io.text;

import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.testing.TestPipelineOptions;

/**
* Pipeline options specific for filebased IO tests.
*/

public interface IOTextTestPipelineOptions extends TestPipelineOptions{
@Description("Test row count")
@Default.Long(10000)
Long getTestSize();
void setTestSize(Long value);
}
Original file line number Diff line number Diff line change
Expand Up @@ -2,9 +2,12 @@

import com.google.common.hash.Hashing;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;

import org.apache.beam.sdk.io.GenerateSequence;
import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.io.common.HashingFn;
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;
Expand All @@ -13,6 +16,7 @@
import org.apache.beam.sdk.transforms.Values;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TypeDescriptors;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
Expand All @@ -23,16 +27,27 @@
public class TextIOIT {

private static final String FILE_BASENAME = "textioit";
private static final long LINES_OF_TEXT_COUNT = 10000L;
private static long linesOfTextCount = 10000L;
private static final String EXPECTED_HASHCODE = "ccae48ff685c1822e9f4d510363bf018";

@Rule public TestPipeline pipeline = TestPipeline.create();

@BeforeClass
public static void setup() throws ParseException {
PipelineOptionsFactory.register(IOTextTestPipelineOptions.class);
IOTextTestPipelineOptions options = TestPipeline.testingPipelineOptions()
.as(IOTextTestPipelineOptions.class);

if (options.getTestSize() != null) {
linesOfTextCount = options.getTestSize();
}
}

@Test
public void testWriteThenRead() {

PCollection<String> consolidatedContentHashcode = pipeline
.apply("Generate sequence", GenerateSequence.from(0).to(LINES_OF_TEXT_COUNT))
.apply("Generate sequence", GenerateSequence.from(0).to(linesOfTextCount))
.apply("Produce text", MapElements.into(TypeDescriptors.strings()).via(produceTextLine()))
.apply("Write content to files", TextIO.write().to(FILE_BASENAME).withOutputFilenames())
.getPerDestinationOutputFilenames()
Expand Down

0 comments on commit e9b7418

Please sign in to comment.