/**
   * Create deploy info for operator.
   *
   * <p>
   *
   * @return {@link com.datatorrent.stram.api.OperatorDeployInfo}
   */
  private OperatorDeployInfo createOperatorDeployInfo(PTOperator oper) {
    OperatorDeployInfo ndi;

    if (oper.isUnifier()) {
      UnifierDeployInfo udi = new UnifierDeployInfo(); /* the constructor auto sets the type */
      try {
        udi.operatorAttributes = oper.getUnifiedOperatorMeta().getAttributes().clone();
      } catch (CloneNotSupportedException ex) {
        throw new RuntimeException("Cannot clone unifier attributes", ex);
      }
      ndi = udi;
    } else {
      ndi = new OperatorDeployInfo();
      Operator operator = oper.getOperatorMeta().getOperator();
      if (operator instanceof InputOperator) {
        ndi.type = OperatorType.INPUT;

        if (!oper.getInputs().isEmpty()) {
          // If there are no input ports then it has to be an input operator. But if there are input
          // ports then
          // we check if any input port is connected which would make it a Generic operator.
          for (PTOperator.PTInput ptInput : oper.getInputs()) {
            if (ptInput.logicalStream != null && ptInput.logicalStream.getSource() != null) {
              ndi.type = OperatorType.GENERIC;
              break;
            }
          }
        }
      } else {
        ndi.type = OperatorType.GENERIC;
      }
    }

    Checkpoint checkpoint = oper.getRecoveryCheckpoint();
    ProcessingMode pm = oper.getOperatorMeta().getValue(OperatorContext.PROCESSING_MODE);

    if (pm == ProcessingMode.AT_MOST_ONCE || pm == ProcessingMode.EXACTLY_ONCE) {
      // TODO: following should be handled in the container at deploy time
      // for exactly once container should also purge previous checkpoint
      // whenever new checkpoint is written.
      StorageAgent agent =
          oper.getOperatorMeta().getAttributes().get(OperatorContext.STORAGE_AGENT);
      if (agent == null) {
        agent = initCtx.getValue(OperatorContext.STORAGE_AGENT);
      }
      // pick checkpoint most recently written
      try {
        long[] windowIds = agent.getWindowIds(oper.getId());
        long checkpointId = Stateless.WINDOW_ID;
        for (long windowId : windowIds) {
          if (windowId > checkpointId) {
            checkpointId = windowId;
          }
        }
        if (checkpoint == null || checkpoint.windowId != checkpointId) {
          checkpoint = new Checkpoint(checkpointId, 0, 0);
        }
      } catch (Exception e) {
        throw new RuntimeException("Failed to determine checkpoint window id " + oper, e);
      }
    }

    LOG.debug("{} recovery checkpoint {}", oper, checkpoint);
    ndi.checkpoint = checkpoint;
    ndi.name = oper.getOperatorMeta().getName();
    ndi.id = oper.getId();
    try {
      // clone map before modifying it
      ndi.contextAttributes = oper.getOperatorMeta().getAttributes().clone();
    } catch (CloneNotSupportedException ex) {
      throw new RuntimeException("Cannot clone operator attributes", ex);
    }
    if (oper.isOperatorStateLess()) {
      ndi.contextAttributes.put(OperatorContext.STATELESS, true);
    }
    return ndi;
  }
  /**
   * Create deploy info for StramChild.
   *
   * @param operators
   * @return StreamingContainerContext
   */
  public List<OperatorDeployInfo> getDeployInfoList(Collection<PTOperator> operators) {

    if (container.bufferServerAddress == null) {
      throw new AssertionError("No buffer server address assigned");
    }

    Map<OperatorDeployInfo, PTOperator> nodes = new LinkedHashMap<>();
    HashSet<PTOperator.PTOutput> publishers = new HashSet<>();

    PhysicalPlan physicalPlan = dnmgr.getPhysicalPlan();

    for (PTOperator oper : operators) {
      if (oper.getState() != State.PENDING_DEPLOY) {
        LOG.debug("Skipping deploy for operator {} state {}", oper, oper.getState());
        continue;
      }
      OperatorDeployInfo ndi = createOperatorDeployInfo(oper);

      nodes.put(ndi, oper);
      ndi.inputs = new ArrayList<>(oper.getInputs().size());
      ndi.outputs = new ArrayList<>(oper.getOutputs().size());

      for (PTOperator.PTOutput out : oper.getOutputs()) {
        final StreamMeta streamMeta = out.logicalStream;
        // buffer server or inline publisher
        OutputDeployInfo portInfo = new OutputDeployInfo();
        portInfo.declaredStreamId = streamMeta.getName();
        portInfo.portName = out.portName;

        try {
          portInfo.contextAttributes = streamMeta.getSource().getAttributes().clone();
        } catch (CloneNotSupportedException ex) {
          throw new RuntimeException("Cannot clone attributes", ex);
        }

        boolean outputUnified = false;
        for (PTOperator.PTInput input : out.sinks) {
          if (input.target.isUnifier()) {
            outputUnified = true;
            break;
          }
        }
        portInfo.contextAttributes.put(PortContext.IS_OUTPUT_UNIFIED, outputUnified);

        if (ndi.type == OperatorDeployInfo.OperatorType.UNIFIER) {
          // input attributes of the downstream operator
          for (InputPortMeta sink : streamMeta.getSinks()) {
            portInfo.contextAttributes = sink.getAttributes();
            break;
          }
        }

        if (!out.isDownStreamInline()) {
          portInfo.bufferServerHost = oper.getContainer().bufferServerAddress.getHostName();
          portInfo.bufferServerPort = oper.getContainer().bufferServerAddress.getPort();
          portInfo.bufferServerToken = oper.getContainer().getBufferServerToken();
          // Build the stream codec configuration of all sinks connected to this port
          for (PTOperator.PTInput input : out.sinks) {
            // Create mappings for all non-inline operators
            if (input.target.getContainer() != out.source.getContainer()) {
              InputPortMeta inputPortMeta = getIdentifyingInputPortMeta(input);
              StreamCodec<?> streamCodecInfo = getStreamCodec(inputPortMeta);
              Integer id = physicalPlan.getStreamCodecIdentifier(streamCodecInfo);
              if (!portInfo.streamCodecs.containsKey(id)) {
                portInfo.streamCodecs.put(id, streamCodecInfo);
              }
            }
          }
        }

        ndi.outputs.add(portInfo);
        publishers.add(out);
      }
    }

    // after we know all publishers within container, determine subscribers

    for (Map.Entry<OperatorDeployInfo, PTOperator> operEntry : nodes.entrySet()) {
      OperatorDeployInfo ndi = operEntry.getKey();
      PTOperator oper = operEntry.getValue();
      for (PTOperator.PTInput in : oper.getInputs()) {
        final StreamMeta streamMeta = in.logicalStream;
        if (streamMeta.getSource() == null) {
          throw new AssertionError("source is null: " + in);
        }
        PTOperator.PTOutput sourceOutput = in.source;

        InputDeployInfo inputInfo = new InputDeployInfo();
        inputInfo.declaredStreamId = streamMeta.getName();
        inputInfo.portName = in.portName;
        InputPortMeta inputPortMeta = getInputPortMeta(oper.getOperatorMeta(), streamMeta);

        if (inputPortMeta != null) {
          inputInfo.contextAttributes = inputPortMeta.getAttributes();
        }

        if (inputInfo.contextAttributes == null
            && ndi.type == OperatorDeployInfo.OperatorType.UNIFIER) {
          inputInfo.contextAttributes = in.source.logicalStream.getSource().getAttributes();
        }

        inputInfo.sourceNodeId = sourceOutput.source.getId();
        inputInfo.sourcePortName = sourceOutput.portName;
        if (in.partitions != null && in.partitions.mask != 0) {
          inputInfo.partitionMask = in.partitions.mask;
          inputInfo.partitionKeys = in.partitions.partitions;
        }

        if (sourceOutput.source.getContainer() == oper.getContainer()) {
          // both operators in same container
          if (!publishers.contains(sourceOutput)) {
            throw new AssertionError(
                "Source not deployed for container local stream " + sourceOutput + " " + in);
          }
          if (streamMeta.getLocality() == Locality.THREAD_LOCAL) {
            inputInfo.locality = Locality.THREAD_LOCAL;
            ndi.type = OperatorType.OIO;
          } else {
            inputInfo.locality = Locality.CONTAINER_LOCAL;
          }

        } else {
          // buffer server input
          PTContainer container = sourceOutput.source.getContainer();
          InetSocketAddress addr = container.bufferServerAddress;
          if (addr == null) {
            throw new AssertionError("upstream address not assigned: " + sourceOutput);
          }
          inputInfo.bufferServerHost = addr.getHostName();
          inputInfo.bufferServerPort = addr.getPort();
          inputInfo.bufferServerToken = container.getBufferServerToken();
        }

        // On the input side there is a unlikely scenario of partitions even for inline stream that
        // is being
        // handled. Always specifying a stream codec configuration in case that scenario happens.
        InputPortMeta idInputPortMeta = getIdentifyingInputPortMeta(in);
        StreamCodec<?> streamCodecInfo = getStreamCodec(idInputPortMeta);
        Integer id = physicalPlan.getStreamCodecIdentifier(streamCodecInfo);
        inputInfo.streamCodecs.put(id, streamCodecInfo);
        ndi.inputs.add(inputInfo);
      }
    }

    return new ArrayList<>(nodes.keySet());
  }