public static CombineValuesFn create(
      PipelineOptions options,
      CloudObject cloudUserFn,
      String stepName,
      @Nullable List<SideInputInfo> sideInputInfos,
      @Nullable List<MultiOutputInfo> multiOutputInfos,
      Integer numOutputs,
      ExecutionContext executionContext,
      CounterSet.AddCounterMutator addCounterMutator,
      StateSampler stateSampler /* unused */)
      throws Exception {
    Object deserializedFn =
        SerializableUtils.deserializeFromByteArray(
            getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), "serialized user fn");
    Preconditions.checkArgument(deserializedFn instanceof Combine.KeyedCombineFn);
    final Combine.KeyedCombineFn combineFn = (Combine.KeyedCombineFn) deserializedFn;

    // Get the combine phase, default to ALL. (The implementation
    // doesn't have to split the combiner).
    final String phase = getString(cloudUserFn, PropertyNames.PHASE, CombinePhase.ALL);

    Preconditions.checkArgument(
        sideInputInfos == null || sideInputInfos.size() == 0,
        "unexpected side inputs for CombineValuesFn");
    Preconditions.checkArgument(numOutputs == 1, "expected exactly one output for CombineValuesFn");

    DoFnInfoFactory fnFactory =
        new DoFnInfoFactory() {
          @Override
          public DoFnInfo createDoFnInfo() {
            DoFn doFn = null;
            switch (phase) {
              case CombinePhase.ALL:
                doFn = new CombineValuesDoFn(combineFn);
                break;
              case CombinePhase.ADD:
                doFn = new AddInputsDoFn(combineFn);
                break;
              case CombinePhase.MERGE:
                doFn = new MergeAccumulatorsDoFn(combineFn);
                break;
              case CombinePhase.EXTRACT:
                doFn = new ExtractOutputDoFn(combineFn);
                break;
              default:
                throw new IllegalArgumentException(
                    "phase must be one of 'all', 'add', 'merge', 'extract'");
            }
            return new DoFnInfo(doFn, null);
          }
        };
    return new CombineValuesFn(options, fnFactory, stepName, executionContext, addCounterMutator);
  }
 @Override
 public void setConf(Configuration conf) {
   this.conf = conf;
   if (conf != null) {
     String headerString = conf.get(SAM_HEADER_PROPERTY_NAME);
     if (headerString == null) {
       throw new IllegalStateException("SAM file header has not been set");
     }
     byte[] headerBytes = Base64.getDecoder().decode(headerString);
     setSAMHeader(
         (SAMFileHeader)
             SerializableUtils.deserializeFromByteArray(headerBytes, "SAMFileHeader"));
   }
 }
  @SuppressWarnings({"rawtypes", "unchecked"})
  public static GroupAlsoByWindowsParDoFn create(
      PipelineOptions options,
      CloudObject cloudUserFn,
      String stepName,
      @Nullable List<SideInputInfo> sideInputInfos,
      @Nullable List<MultiOutputInfo> multiOutputInfos,
      Integer numOutputs,
      ExecutionContext executionContext,
      CounterSet.AddCounterMutator addCounterMutator,
      StateSampler sampler /* unused */)
      throws Exception {
    Object windowingStrategyObj;
    byte[] encodedWindowingStrategy = getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN);
    if (encodedWindowingStrategy.length == 0) {
      windowingStrategyObj = WindowingStrategy.globalDefault();
    } else {
      windowingStrategyObj =
          SerializableUtils.deserializeFromByteArray(
              encodedWindowingStrategy, "serialized windowing strategy");
      if (!(windowingStrategyObj instanceof WindowingStrategy)) {
        throw new Exception(
            "unexpected kind of WindowingStrategy: " + windowingStrategyObj.getClass().getName());
      }
    }
    WindowingStrategy windowingStrategy = (WindowingStrategy) windowingStrategyObj;

    byte[] serializedCombineFn = getBytes(cloudUserFn, PropertyNames.COMBINE_FN, null);
    KeyedCombineFn combineFn;
    if (serializedCombineFn != null) {
      Object combineFnObj =
          SerializableUtils.deserializeFromByteArray(serializedCombineFn, "serialized combine fn");
      if (!(combineFnObj instanceof KeyedCombineFn)) {
        throw new Exception(
            "unexpected kind of KeyedCombineFn: " + combineFnObj.getClass().getName());
      }
      combineFn = (KeyedCombineFn) combineFnObj;
    } else {
      combineFn = null;
    }

    Map<String, Object> inputCoderObject = getObject(cloudUserFn, PropertyNames.INPUT_CODER);

    Coder inputCoder = Serializer.deserialize(inputCoderObject, Coder.class);
    if (!(inputCoder instanceof WindowedValueCoder)) {
      throw new Exception(
          "Expected WindowedValueCoder for inputCoder, got: " + inputCoder.getClass().getName());
    }
    Coder elemCoder = ((WindowedValueCoder) inputCoder).getValueCoder();
    if (!(elemCoder instanceof KvCoder)) {
      throw new Exception(
          "Expected KvCoder for inputCoder, got: " + elemCoder.getClass().getName());
    }
    KvCoder kvCoder = (KvCoder) elemCoder;

    boolean isStreamingPipeline = false;
    if (options instanceof StreamingOptions) {
      isStreamingPipeline = ((StreamingOptions) options).isStreaming();
    }

    KeyedCombineFn maybeMergingCombineFn = null;
    if (combineFn != null) {
      String phase = getString(cloudUserFn, PropertyNames.PHASE, CombinePhase.ALL);
      Preconditions.checkArgument(
          phase.equals(CombinePhase.ALL) || phase.equals(CombinePhase.MERGE),
          "Unexpected phase: " + phase);
      if (phase.equals(CombinePhase.MERGE)) {
        maybeMergingCombineFn = new MergingKeyedCombineFn(combineFn);
      } else {
        maybeMergingCombineFn = combineFn;
      }
    }

    DoFnInfoFactory fnFactory;
    final DoFn groupAlsoByWindowsDoFn =
        getGroupAlsoByWindowsDoFn(
            isStreamingPipeline, windowingStrategy, kvCoder, maybeMergingCombineFn);

    fnFactory =
        new DoFnInfoFactory() {
          @Override
          public DoFnInfo createDoFnInfo() {
            return new DoFnInfo(groupAlsoByWindowsDoFn, null);
          }
        };
    return new GroupAlsoByWindowsParDoFn(
        options, fnFactory, stepName, executionContext, addCounterMutator);
  }