@Override
    @SuppressWarnings("unchecked")
    public void translateNode(
        Flatten.FlattenPCollectionList<T> transform, FlinkBatchTranslationContext context) {

      List<TaggedPValue> allInputs = context.getInputs(transform);
      DataSet<WindowedValue<T>> result = null;

      if (allInputs.isEmpty()) {

        // create an empty dummy source to satisfy downstream operations
        // we cannot create an empty source in Flink, therefore we have to
        // add the flatMap that simply never forwards the single element
        DataSource<String> dummySource = context.getExecutionEnvironment().fromElements("dummy");
        result =
            dummySource
                .flatMap(
                    new FlatMapFunction<String, WindowedValue<T>>() {
                      @Override
                      public void flatMap(String s, Collector<WindowedValue<T>> collector)
                          throws Exception {
                        // never return anything
                      }
                    })
                .returns(
                    new CoderTypeInformation<>(
                        WindowedValue.getFullCoder(
                            (Coder<T>) VoidCoder.of(), GlobalWindow.Coder.INSTANCE)));
      } else {
        for (TaggedPValue taggedPc : allInputs) {
          checkArgument(
              taggedPc.getValue() instanceof PCollection,
              "Got non-PCollection input to flatten: %s of type %s",
              taggedPc.getValue(),
              taggedPc.getValue().getClass().getSimpleName());
          PCollection<T> collection = (PCollection<T>) taggedPc.getValue();
          DataSet<WindowedValue<T>> current = context.getInputDataSet(collection);
          if (result == null) {
            result = current;
          } else {
            result = result.union(current);
          }
        }
      }

      // insert a dummy filter, there seems to be a bug in Flink
      // that produces duplicate elements after the union in some cases
      // if we don't
      result =
          result
              .filter(
                  new FilterFunction<WindowedValue<T>>() {
                    @Override
                    public boolean filter(WindowedValue<T> tWindowedValue) throws Exception {
                      return true;
                    }
                  })
              .name("UnionFixFilter");
      context.setOutputDataSet(context.getOutput(transform), result);
    }
    @Override
    public void translateNode(Window.Bound<T> transform, FlinkBatchTranslationContext context) {
      PValue input = context.getInput(transform);

      TypeInformation<WindowedValue<T>> resultTypeInfo =
          context.getTypeInfo(context.getOutput(transform));

      DataSet<WindowedValue<T>> inputDataSet = context.getInputDataSet(input);

      @SuppressWarnings("unchecked")
      final WindowingStrategy<T, ? extends BoundedWindow> windowingStrategy =
          (WindowingStrategy<T, ? extends BoundedWindow>)
              context.getOutput(transform).getWindowingStrategy();

      WindowFn<T, ? extends BoundedWindow> windowFn = windowingStrategy.getWindowFn();

      FlinkAssignWindows<T, ? extends BoundedWindow> assignWindowsFunction =
          new FlinkAssignWindows<>(windowFn);

      DataSet<WindowedValue<T>> resultDataSet =
          inputDataSet
              .flatMap(assignWindowsFunction)
              .name(context.getOutput(transform).getName())
              .returns(resultTypeInfo);

      context.setOutputDataSet(context.getOutput(transform), resultDataSet);
    }
    @Override
    public void translateNode(
        Reshuffle<K, InputT> transform, FlinkBatchTranslationContext context) {

      DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
          context.getInputDataSet(context.getInput(transform));

      context.setOutputDataSet(context.getOutput(transform), inputDataSet.rebalance());
    }
    private <T> void pruneOutput(
        MapPartitionOperator<WindowedValue<InputT>, WindowedValue<RawUnionValue>> taggedDataSet,
        FlinkBatchTranslationContext context,
        int integerTag,
        PCollection<T> collection) {
      TypeInformation<WindowedValue<T>> outputType = context.getTypeInfo(collection);

      FlinkMultiOutputPruningFunction<T> pruningFunction =
          new FlinkMultiOutputPruningFunction<>(integerTag);

      FlatMapOperator<WindowedValue<RawUnionValue>, WindowedValue<T>> pruningOperator =
          new FlatMapOperator<>(taggedDataSet, outputType, pruningFunction, collection.getName());

      context.setOutputDataSet(collection, pruningOperator);
    }
    @Override
    public void translateNode(Read.Bounded<T> transform, FlinkBatchTranslationContext context) {
      String name = transform.getName();
      BoundedSource<T> source = transform.getSource();
      PCollection<T> output = context.getOutput(transform);

      TypeInformation<WindowedValue<T>> typeInformation = context.getTypeInfo(output);

      DataSource<WindowedValue<T>> dataSource =
          new DataSource<>(
              context.getExecutionEnvironment(),
              new SourceInputFormat<>(source, context.getPipelineOptions()),
              typeInformation,
              name);

      context.setOutputDataSet(output, dataSource);
    }
    @Override
    public void translateNode(
        ParDo.Bound<InputT, OutputT> transform, FlinkBatchTranslationContext context) {

      DoFn<InputT, OutputT> doFn = transform.getFn();
      rejectStateAndTimers(doFn);

      DataSet<WindowedValue<InputT>> inputDataSet =
          context.getInputDataSet(context.getInput(transform));

      TypeInformation<WindowedValue<OutputT>> typeInformation =
          context.getTypeInfo(context.getOutput(transform));

      List<PCollectionView<?>> sideInputs = transform.getSideInputs();

      // construct a map from side input to WindowingStrategy so that
      // the OldDoFn runner can map main-input windows to side input windows
      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputStrategies = new HashMap<>();
      for (PCollectionView<?> sideInput : sideInputs) {
        sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal());
      }

      FlinkDoFnFunction<InputT, OutputT> doFnWrapper =
          new FlinkDoFnFunction<>(
              doFn,
              context.getOutput(transform).getWindowingStrategy(),
              sideInputStrategies,
              context.getPipelineOptions());

      MapPartitionOperator<WindowedValue<InputT>, WindowedValue<OutputT>> outputDataSet =
          new MapPartitionOperator<>(
              inputDataSet, typeInformation, doFnWrapper, transform.getName());

      transformSideInputs(sideInputs, outputDataSet, context);

      context.setOutputDataSet(context.getOutput(transform), outputDataSet);
    }
    @Override
    @SuppressWarnings("unchecked")
    public void translateNode(
        Combine.PerKey<K, InputT, OutputT> transform, FlinkBatchTranslationContext context) {
      DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
          context.getInputDataSet(context.getInput(transform));

      CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, OutputT> combineFn =
          (CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, OutputT>) transform.getFn();

      KvCoder<K, InputT> inputCoder = (KvCoder<K, InputT>) context.getInput(transform).getCoder();

      Coder<AccumT> accumulatorCoder;

      try {
        accumulatorCoder =
            combineFn.getAccumulatorCoder(
                context.getInput(transform).getPipeline().getCoderRegistry(),
                inputCoder.getKeyCoder(),
                inputCoder.getValueCoder());
      } catch (CannotProvideCoderException e) {
        throw new RuntimeException(e);
      }

      WindowingStrategy<?, ?> windowingStrategy =
          context.getInput(transform).getWindowingStrategy();

      TypeInformation<WindowedValue<KV<K, AccumT>>> partialReduceTypeInfo =
          context.getTypeInfo(
              KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder), windowingStrategy);

      Grouping<WindowedValue<KV<K, InputT>>> inputGrouping =
          inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));

      // construct a map from side input to WindowingStrategy so that
      // the OldDoFn runner can map main-input windows to side input windows
      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputStrategies = new HashMap<>();
      for (PCollectionView<?> sideInput : transform.getSideInputs()) {
        sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal());
      }

      if (windowingStrategy.getWindowFn().isNonMerging()) {
        WindowingStrategy<?, BoundedWindow> boundedStrategy =
            (WindowingStrategy<?, BoundedWindow>) windowingStrategy;

        FlinkPartialReduceFunction<K, InputT, AccumT, ?> partialReduceFunction =
            new FlinkPartialReduceFunction<>(
                combineFn, boundedStrategy, sideInputStrategies, context.getPipelineOptions());

        FlinkReduceFunction<K, AccumT, OutputT, ?> reduceFunction =
            new FlinkReduceFunction<>(
                combineFn, boundedStrategy, sideInputStrategies, context.getPipelineOptions());

        // Partially GroupReduce the values into the intermediate format AccumT (combine)
        GroupCombineOperator<WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, AccumT>>>
            groupCombine =
                new GroupCombineOperator<>(
                    inputGrouping,
                    partialReduceTypeInfo,
                    partialReduceFunction,
                    "GroupCombine: " + transform.getName());

        transformSideInputs(transform.getSideInputs(), groupCombine, context);

        TypeInformation<WindowedValue<KV<K, OutputT>>> reduceTypeInfo =
            context.getTypeInfo(context.getOutput(transform));

        Grouping<WindowedValue<KV<K, AccumT>>> intermediateGrouping =
            groupCombine.groupBy(new KvKeySelector<AccumT, K>(inputCoder.getKeyCoder()));

        // Fully reduce the values and create output format OutputT
        GroupReduceOperator<WindowedValue<KV<K, AccumT>>, WindowedValue<KV<K, OutputT>>>
            outputDataSet =
                new GroupReduceOperator<>(
                    intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName());

        transformSideInputs(transform.getSideInputs(), outputDataSet, context);

        context.setOutputDataSet(context.getOutput(transform), outputDataSet);

      } else {
        if (!windowingStrategy.getWindowFn().windowCoder().equals(IntervalWindow.getCoder())) {
          throw new UnsupportedOperationException(
              "Merging WindowFn with windows other than IntervalWindow are not supported.");
        }

        // for merging windows we can't to a pre-shuffle combine step since
        // elements would not be in their correct windows for side-input access

        WindowingStrategy<?, IntervalWindow> intervalStrategy =
            (WindowingStrategy<?, IntervalWindow>) windowingStrategy;

        FlinkMergingNonShuffleReduceFunction<K, InputT, AccumT, OutputT, ?> reduceFunction =
            new FlinkMergingNonShuffleReduceFunction<>(
                combineFn, intervalStrategy, sideInputStrategies, context.getPipelineOptions());

        TypeInformation<WindowedValue<KV<K, OutputT>>> reduceTypeInfo =
            context.getTypeInfo(context.getOutput(transform));

        Grouping<WindowedValue<KV<K, InputT>>> grouping =
            inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));

        // Fully reduce the values and create output format OutputT
        GroupReduceOperator<WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, OutputT>>>
            outputDataSet =
                new GroupReduceOperator<>(
                    grouping, reduceTypeInfo, reduceFunction, transform.getName());

        transformSideInputs(transform.getSideInputs(), outputDataSet, context);

        context.setOutputDataSet(context.getOutput(transform), outputDataSet);
      }
    }
    @Override
    public void translateNode(
        GroupByKey<K, InputT> transform, FlinkBatchTranslationContext context) {

      // for now, this is copied from the Combine.PerKey translater. Once we have the new runner API
      // we can replace GroupByKey by a Combine.PerKey with the Concatenate CombineFn

      DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
          context.getInputDataSet(context.getInput(transform));

      Combine.KeyedCombineFn<K, InputT, List<InputT>, List<InputT>> combineFn =
          new Concatenate<InputT>().asKeyedFn();

      KvCoder<K, InputT> inputCoder = (KvCoder<K, InputT>) context.getInput(transform).getCoder();

      Coder<List<InputT>> accumulatorCoder;

      try {
        accumulatorCoder =
            combineFn.getAccumulatorCoder(
                context.getInput(transform).getPipeline().getCoderRegistry(),
                inputCoder.getKeyCoder(),
                inputCoder.getValueCoder());
      } catch (CannotProvideCoderException e) {
        throw new RuntimeException(e);
      }

      WindowingStrategy<?, ?> windowingStrategy =
          context.getInput(transform).getWindowingStrategy();

      TypeInformation<WindowedValue<KV<K, List<InputT>>>> partialReduceTypeInfo =
          new CoderTypeInformation<>(
              WindowedValue.getFullCoder(
                  KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder),
                  windowingStrategy.getWindowFn().windowCoder()));

      Grouping<WindowedValue<KV<K, InputT>>> inputGrouping =
          inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));

      FlinkPartialReduceFunction<K, InputT, List<InputT>, ?> partialReduceFunction;
      FlinkReduceFunction<K, List<InputT>, List<InputT>, ?> reduceFunction;

      if (windowingStrategy.getWindowFn().isNonMerging()) {
        @SuppressWarnings("unchecked")
        WindowingStrategy<?, BoundedWindow> boundedStrategy =
            (WindowingStrategy<?, BoundedWindow>) windowingStrategy;

        partialReduceFunction =
            new FlinkPartialReduceFunction<>(
                combineFn,
                boundedStrategy,
                Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
                context.getPipelineOptions());

        reduceFunction =
            new FlinkReduceFunction<>(
                combineFn,
                boundedStrategy,
                Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
                context.getPipelineOptions());

      } else {
        if (!windowingStrategy.getWindowFn().windowCoder().equals(IntervalWindow.getCoder())) {
          throw new UnsupportedOperationException(
              "Merging WindowFn with windows other than IntervalWindow are not supported.");
        }

        @SuppressWarnings("unchecked")
        WindowingStrategy<?, IntervalWindow> intervalStrategy =
            (WindowingStrategy<?, IntervalWindow>) windowingStrategy;

        partialReduceFunction =
            new FlinkMergingPartialReduceFunction<>(
                combineFn,
                intervalStrategy,
                Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
                context.getPipelineOptions());

        reduceFunction =
            new FlinkMergingReduceFunction<>(
                combineFn,
                intervalStrategy,
                Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
                context.getPipelineOptions());
      }

      // Partially GroupReduce the values into the intermediate format AccumT (combine)
      GroupCombineOperator<WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, List<InputT>>>>
          groupCombine =
              new GroupCombineOperator<>(
                  inputGrouping,
                  partialReduceTypeInfo,
                  partialReduceFunction,
                  "GroupCombine: " + transform.getName());

      Grouping<WindowedValue<KV<K, List<InputT>>>> intermediateGrouping =
          groupCombine.groupBy(new KvKeySelector<List<InputT>, K>(inputCoder.getKeyCoder()));

      // Fully reduce the values and create output format VO
      GroupReduceOperator<WindowedValue<KV<K, List<InputT>>>, WindowedValue<KV<K, List<InputT>>>>
          outputDataSet =
              new GroupReduceOperator<>(
                  intermediateGrouping, partialReduceTypeInfo, reduceFunction, transform.getName());

      context.setOutputDataSet(context.getOutput(transform), outputDataSet);
    }