private DataflowPipeline buildPipeline(DataflowPipelineOptions options) { DataflowPipeline p = DataflowPipeline.create(options); p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object")) .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object")); return p; }
/** Returns a Step for a DoFn by creating and translating a pipeline. */ private static Step createPredefinedStep() throws Exception { DataflowPipelineOptions options = buildPipelineOptions(); DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options); DataflowPipeline pipeline = DataflowPipeline.create(options); String stepName = "DoFn1"; pipeline .apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in")) .apply(ParDo.of(new NoOpFn()).named(stepName)) .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/out")); Job job = translator.translate(pipeline, Collections.<DataflowPackage>emptyList()).getJob(); assertEquals(3, job.getSteps().size()); Step step = job.getSteps().get(1); assertEquals(stepName, getString(step.getProperties(), PropertyNames.USER_NAME)); return step; }
@Test public void testPredefinedAddStep() throws Exception { DataflowPipelineOptions options = buildPipelineOptions(); DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options); DataflowPipelineTranslator.registerTransformTranslator( EmbeddedTransform.class, new EmbeddedTranslator()); // Create a predefined step using another pipeline Step predefinedStep = createPredefinedStep(); // Create a pipeline that the predefined step will be embedded into DataflowPipeline pipeline = DataflowPipeline.create(options); pipeline .apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in")) .apply(ParDo.of(new NoOpFn())) .apply(new EmbeddedTransform(predefinedStep.clone())) .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/out")); Job job = translator.translate(pipeline, Collections.<DataflowPackage>emptyList()).getJob(); List<Step> steps = job.getSteps(); assertEquals(4, steps.size()); // The input to the embedded step should match the output of the step before Map<String, Object> step1Out = getOutputPortReference(steps.get(1)); Map<String, Object> step2In = getDictionary(steps.get(2).getProperties(), PropertyNames.PARALLEL_INPUT); assertEquals(step1Out, step2In); // The output from the embedded step should match the input of the step after Map<String, Object> step2Out = getOutputPortReference(steps.get(2)); Map<String, Object> step3In = getDictionary(steps.get(3).getProperties(), PropertyNames.PARALLEL_INPUT); assertEquals(step2Out, step3In); // The step should not have been modified other than remapping the input Step predefinedStepClone = predefinedStep.clone(); Step embeddedStepClone = steps.get(2).clone(); predefinedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT); embeddedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT); assertEquals(predefinedStepClone, embeddedStepClone); }
@Test public void testPartiallyBoundFailure() throws IOException { Pipeline p = DataflowPipeline.create(buildPipelineOptions()); PCollection<Integer> input = p.begin().apply(Create.of(1, 2, 3)); thrown.expect(IllegalStateException.class); input.apply(new PartiallyBoundOutputCreator()); Assert.fail("Failure expected from use of partially bound output"); }
@Test public void testToIterableTranslation() throws Exception { // A "change detector" test that makes sure the translation // of getting a PCollectionView<Iterable<T>> does not change // in bad ways during refactor DataflowPipelineOptions options = buildPipelineOptions(); DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options); DataflowPipeline pipeline = DataflowPipeline.create(options); pipeline.apply(Create.of(1, 2, 3)).apply(View.<Integer>asIterable()); Job job = translator.translate(pipeline, Collections.<DataflowPackage>emptyList()).getJob(); List<Step> steps = job.getSteps(); assertEquals(2, steps.size()); Step createStep = steps.get(0); assertEquals("CreateCollection", createStep.getKind()); Step collectionToSingletonStep = steps.get(1); assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind()); }
/** Recursive wildcards are not supported. This tests "**". */ @Test public void testBadWildcardRecursive() throws Exception { DataflowPipelineOptions options = buildPipelineOptions(); Pipeline pipeline = DataflowPipeline.create(options); DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options); pipeline.apply(TextIO.Read.from("gs://bucket/foo**/baz")); // Check that translation does fail. thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Unsupported wildcard usage"); t.translate(pipeline, Collections.<DataflowPackage>emptyList()); }
@Test public void testMultiGraphPipelineSerialization() throws IOException { Pipeline p = DataflowPipeline.create(buildPipelineOptions()); PCollection<Integer> input = p.begin().apply(Create.of(1, 2, 3)); input.apply(new UnrelatedOutputCreator()); input.apply(new UnboundOutputCreator()); DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions( PipelineOptionsFactory.as(DataflowPipelineOptions.class)); // Check that translation doesn't fail. t.translate(p, Collections.<DataflowPackage>emptyList()); }
/** This tests a few corner cases that should not crash. */ @Test public void testGoodWildcards() throws Exception { DataflowPipelineOptions options = buildPipelineOptions(); Pipeline pipeline = DataflowPipeline.create(options); DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options); applyRead(pipeline, "gs://bucket/foo"); applyRead(pipeline, "gs://bucket/foo/"); applyRead(pipeline, "gs://bucket/foo/*"); applyRead(pipeline, "gs://bucket/foo/?"); applyRead(pipeline, "gs://bucket/foo/[0-9]"); applyRead(pipeline, "gs://bucket/foo/*baz*"); applyRead(pipeline, "gs://bucket/foo/*baz?"); applyRead(pipeline, "gs://bucket/foo/[0-9]baz?"); applyRead(pipeline, "gs://bucket/foo/baz/*"); applyRead(pipeline, "gs://bucket/foo/baz/*wonka*"); applyRead(pipeline, "gs://bucket/foo/*baz/wonka*"); applyRead(pipeline, "gs://bucket/foo*/baz"); applyRead(pipeline, "gs://bucket/foo?/baz"); applyRead(pipeline, "gs://bucket/foo[0-9]/baz"); // Check that translation doesn't fail. t.translate(pipeline, Collections.<DataflowPackage>emptyList()); }