@Test
  public void testSettingOfSdkPipelineOptions() throws IOException {
    DataflowPipelineOptions options = buildPipelineOptions();
    options.setRunner(DataflowPipelineRunner.class);

    Pipeline p = buildPipeline(options);
    p.traverseTopologically(new RecordingPipelineVisitor());
    Job job =
        DataflowPipelineTranslator.fromOptions(options)
            .translate(p, Collections.<DataflowPackage>emptyList())
            .getJob();

    // Note that the contents of this materialized map may be changed by the act of reading an
    // option, which will cause the default to get materialized whereas it would otherwise be
    // left absent. It is permissible to simply alter this test to reflect current behavior.
    assertEquals(
        ImmutableMap.of(
            "options",
            ImmutableMap.builder()
                .put("appName", "DataflowPipelineTranslatorTest")
                .put("project", "some-project")
                .put(
                    "pathValidatorClass",
                    "com.google.cloud.dataflow.sdk.util.DataflowPathValidator")
                .put("runner", "com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner")
                .put("jobName", "some-job-name")
                .put("tempLocation", "gs://somebucket/some/path")
                .put("stagingLocation", "gs://somebucket/some/path/staging")
                .put("stableUniqueNames", "WARNING")
                .put("streaming", false)
                .put("numberOfWorkerHarnessThreads", 0)
                .build()),
        job.getEnvironment().getSdkPipelineOptions());
  }
  @Test
  public void testScalingAlgorithmMissing() throws IOException {
    DataflowPipelineOptions options = buildPipelineOptions();

    Pipeline p = buildPipeline(options);
    p.traverseTopologically(new RecordingPipelineVisitor());
    Job job =
        DataflowPipelineTranslator.fromOptions(options)
            .translate(p, Collections.<DataflowPackage>emptyList())
            .getJob();

    assertEquals(1, job.getEnvironment().getWorkerPools().size());
    assertNull(job.getEnvironment().getWorkerPools().get(0).getAutoscalingSettings());
  }
  private static Dataflow buildMockDataflow(ArgumentMatcher<Job> jobMatcher) throws IOException {
    Dataflow mockDataflowClient = mock(Dataflow.class);
    Dataflow.Projects mockProjects = mock(Dataflow.Projects.class);
    Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class);
    Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);

    when(mockDataflowClient.projects()).thenReturn(mockProjects);
    when(mockProjects.jobs()).thenReturn(mockJobs);
    when(mockJobs.create(eq("someProject"), argThat(jobMatcher))).thenReturn(mockRequest);

    Job resultJob = new Job();
    resultJob.setId("newid");
    when(mockRequest.execute()).thenReturn(resultJob);
    return mockDataflowClient;
  }
  /** 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 testDiskSizeGbConfig() throws IOException {
    final Integer diskSizeGb = 1234;

    DataflowPipelineOptions options = buildPipelineOptions();
    options.setDiskSizeGb(diskSizeGb);

    Pipeline p = buildPipeline(options);
    p.traverseTopologically(new RecordingPipelineVisitor());
    Job job =
        DataflowPipelineTranslator.fromOptions(options)
            .translate(p, Collections.<DataflowPackage>emptyList())
            .getJob();

    assertEquals(1, job.getEnvironment().getWorkerPools().size());
    assertEquals(diskSizeGb, job.getEnvironment().getWorkerPools().get(0).getDiskSizeGb());
  }
  @Test
  public void testZoneConfig() throws IOException {
    final String testZone = "test-zone-1";

    DataflowPipelineOptions options = buildPipelineOptions();
    options.setZone(testZone);

    Pipeline p = buildPipeline(options);
    p.traverseTopologically(new RecordingPipelineVisitor());
    Job job =
        DataflowPipelineTranslator.fromOptions(options)
            .translate(p, Collections.<DataflowPackage>emptyList())
            .getJob();

    assertEquals(1, job.getEnvironment().getWorkerPools().size());
    assertEquals(testZone, job.getEnvironment().getWorkerPools().get(0).getZone());
  }
  @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 testScalingAlgorithmNone() throws IOException {
    final DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType noScaling =
        DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE;

    DataflowPipelineOptions options = buildPipelineOptions();
    options.setAutoscalingAlgorithm(noScaling);

    Pipeline p = buildPipeline(options);
    p.traverseTopologically(new RecordingPipelineVisitor());
    Job job =
        DataflowPipelineTranslator.fromOptions(options)
            .translate(p, Collections.<DataflowPackage>emptyList())
            .getJob();

    assertEquals(1, job.getEnvironment().getWorkerPools().size());
    assertEquals(
        "AUTOSCALING_ALGORITHM_NONE",
        job.getEnvironment().getWorkerPools().get(0).getAutoscalingSettings().getAlgorithm());
  }
  @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());
  }
 @Override
 public boolean matches(Object o) {
   Job job = (Job) o;
   return job.getId() == null
       && job.getProjectId() == null
       && job.getName() != null
       && job.getType() != null
       && job.getEnvironment() != null
       && job.getSteps() != null
       && job.getCurrentState() == null
       && job.getCurrentStateTime() == null
       && job.getExecutionInfo() == null
       && job.getCreateTime() == null;
 }