public boolean hasPendingWork() {
   for (PTOperator oper : container.getOperators()) {
     if (oper.getState() == PTOperator.State.PENDING_DEPLOY) {
       return true;
     }
   }
   return false;
 }
 public ContainerInfo getContainerInfo() {
   ContainerInfo ci = new ContainerInfo();
   ci.id = container.getExternalId();
   ci.host = container.host;
   ci.state = container.getState().name();
   ci.jvmName = this.jvmName;
   ci.numOperators = container.getOperators().size();
   ci.memoryMBAllocated = container.getAllocatedMemoryMB();
   ci.lastHeartbeat = lastHeartbeatMillis;
   ci.memoryMBFree = this.memoryMBFree;
   ci.gcCollectionCount = this.gcCollectionCount;
   ci.gcCollectionTime = this.gcCollectionTime;
   ci.startedTime = container.getStartedTime();
   ci.finishedTime = container.getFinishedTime();
   if (this.container.nodeHttpAddress != null) {
     YarnConfiguration conf = new YarnConfiguration();
     ci.containerLogsUrl =
         ConfigUtils.getSchemePrefix(conf)
             + this.container.nodeHttpAddress
             + "/node/containerlogs/"
             + ci.id
             + "/"
             + System.getenv(ApplicationConstants.Environment.USER.toString());
     ci.rawContainerLogsUrl =
         ConfigUtils.getRawContainerLogsUrl(
             conf,
             container.nodeHttpAddress,
             container.getPlan().getLogicalPlan().getAttributes().get(LogicalPlan.APPLICATION_ID),
             ci.id);
   }
   return ci;
 }
 private int countState(PTContainer c, PTOperator.State state) {
   int count = 0;
   for (PTOperator o : c.getOperators()) {
     if (o.getState() == state) {
       count++;
     }
   }
   return count;
 }
  private void testDownStreamPartition(Locality locality) throws Exception {
    TestGeneratorInputOperator o1 = dag.addOperator("o1", TestGeneratorInputOperator.class);
    GenericTestOperator o2 = dag.addOperator("o2", GenericTestOperator.class);
    dag.setAttribute(
        o2, OperatorContext.PARTITIONER, new StatelessPartitioner<GenericTestOperator>(2));
    dag.addStream("o1Output1", o1.outport, o2.inport1).setLocality(locality);

    int maxContainers = 5;
    dag.setAttribute(LogicalPlan.CONTAINERS_MAX_COUNT, maxContainers);
    dag.setAttribute(OperatorContext.STORAGE_AGENT, new StramTestSupport.MemoryStorageAgent());
    dag.validate();
    PhysicalPlan plan = new PhysicalPlan(dag, new TestPlanContext());
    Assert.assertEquals("number of containers", 1, plan.getContainers().size());

    PTContainer container1 = plan.getContainers().get(0);
    Assert.assertEquals("number operators " + container1, 3, container1.getOperators().size());
    StramLocalCluster slc = new StramLocalCluster(dag);
    slc.run(5000);
  }
  @Test
  public void testRecoveryOrder() throws Exception {
    GenericTestOperator node1 = dag.addOperator("node1", GenericTestOperator.class);
    GenericTestOperator node2 = dag.addOperator("node2", GenericTestOperator.class);
    GenericTestOperator node3 = dag.addOperator("node3", GenericTestOperator.class);

    dag.addStream("n1n2", node1.outport1, node2.inport1);
    dag.addStream("n2n3", node2.outport1, node3.inport1);

    dag.getAttributes().put(LogicalPlan.CONTAINERS_MAX_COUNT, 2);
    dag.setAttribute(OperatorContext.STORAGE_AGENT, new MemoryStorageAgent());

    StreamingContainerManager scm = new StreamingContainerManager(dag);
    Assert.assertEquals("" + scm.containerStartRequests, 2, scm.containerStartRequests.size());
    scm.containerStartRequests.clear();

    PhysicalPlan plan = scm.getPhysicalPlan();

    List<PTContainer> containers = plan.getContainers();
    Assert.assertEquals("" + containers, 2, plan.getContainers().size());

    PTContainer c1 = containers.get(0);
    Assert.assertEquals("c1.operators " + c1.getOperators(), 2, c1.getOperators().size());

    PTContainer c2 = containers.get(1);
    Assert.assertEquals("c2.operators " + c2.getOperators(), 1, c2.getOperators().size());

    assignContainer(scm, "container1");
    assignContainer(scm, "container2");

    StreamingContainerAgent sca1 = scm.getContainerAgent(c1.getExternalId());
    StreamingContainerAgent sca2 = scm.getContainerAgent(c2.getExternalId());
    Assert.assertEquals("", 0, countState(sca1.container, PTOperator.State.PENDING_UNDEPLOY));
    Assert.assertEquals("", 2, countState(sca1.container, PTOperator.State.PENDING_DEPLOY));

    scm.scheduleContainerRestart(c1.getExternalId());
    Assert.assertEquals("", 0, countState(sca1.container, PTOperator.State.PENDING_UNDEPLOY));
    Assert.assertEquals("", 2, countState(sca1.container, PTOperator.State.PENDING_DEPLOY));
    Assert.assertEquals("" + scm.containerStartRequests, 1, scm.containerStartRequests.size());
    ContainerStartRequest dr = scm.containerStartRequests.peek();
    Assert.assertNotNull(dr);

    Assert.assertEquals(
        "" + sca2.container, 1, countState(sca2.container, PTOperator.State.PENDING_UNDEPLOY));
    Assert.assertEquals(
        "" + sca2.container, 0, countState(sca2.container, PTOperator.State.PENDING_DEPLOY));
  }
  @Test
  public void testRecoveryUpstreamInline() throws Exception {
    GenericTestOperator o1 = dag.addOperator("o1", GenericTestOperator.class);
    GenericTestOperator o2 = dag.addOperator("o2", GenericTestOperator.class);
    GenericTestOperator o3 = dag.addOperator("o3", GenericTestOperator.class);

    dag.addStream("o1o3", o1.outport1, o3.inport1);
    dag.addStream("o2o3", o2.outport1, o3.inport2);

    dag.getAttributes().put(LogicalPlan.CONTAINERS_MAX_COUNT, 2);
    dag.setAttribute(OperatorContext.STORAGE_AGENT, new MemoryStorageAgent());

    StreamingContainerManager scm = new StreamingContainerManager(dag);
    PhysicalPlan plan = scm.getPhysicalPlan();
    Assert.assertEquals(2, plan.getContainers().size());

    plan.getOperators(dag.getMeta(o1)).get(0);

    Assert.assertEquals(2, plan.getContainers().size());
    PTContainer c1 = plan.getContainers().get(0);
    Assert.assertEquals(
        Sets.newHashSet(
            plan.getOperators(dag.getMeta(o1)).get(0), plan.getOperators(dag.getMeta(o3)).get(0)),
        Sets.newHashSet(c1.getOperators()));
    PTContainer c2 = plan.getContainers().get(1);

    assignContainer(scm, "c1");
    assignContainer(scm, "c2");

    for (PTOperator oper : c1.getOperators()) {
      Assert.assertEquals("state " + oper, PTOperator.State.PENDING_DEPLOY, oper.getState());
    }
    scm.scheduleContainerRestart(c2.getExternalId());
    for (PTOperator oper : c1.getOperators()) {
      Assert.assertEquals("state " + oper, PTOperator.State.PENDING_UNDEPLOY, oper.getState());
    }
  }
예제 #7
0
  @Test
  public void testWriteAheadLog() throws Exception {
    final MutableInt flushCount = new MutableInt();
    final MutableBoolean isClosed = new MutableBoolean(false);
    LogicalPlan dag = new LogicalPlan();
    dag.setAttribute(LogicalPlan.APPLICATION_PATH, testMeta.dir);
    dag.setAttribute(OperatorContext.STORAGE_AGENT, new FSStorageAgent(testMeta.dir, null));

    TestGeneratorInputOperator o1 = dag.addOperator("o1", TestGeneratorInputOperator.class);
    StreamingContainerManager scm = new StreamingContainerManager(dag);
    PhysicalPlan plan = scm.getPhysicalPlan();
    Journal j = scm.getJournal();
    ByteArrayOutputStream bos =
        new ByteArrayOutputStream() {
          @Override
          public void flush() throws IOException {
            super.flush();
            flushCount.increment();
          }

          @Override
          public void close() throws IOException {
            super.close();
            isClosed.setValue(true);
          }
        };
    j.setOutputStream(new DataOutputStream(bos));

    PTOperator o1p1 = plan.getOperators(dag.getMeta(o1)).get(0);
    assertEquals(PTOperator.State.PENDING_DEPLOY, o1p1.getState());
    String externalId = new MockContainer(scm, o1p1.getContainer()).container.getExternalId();
    assertEquals("flush count", 1, flushCount.intValue());

    o1p1.setState(PTOperator.State.ACTIVE);
    assertEquals(PTOperator.State.ACTIVE, o1p1.getState());
    assertEquals("flush count", 2, flushCount.intValue());
    assertEquals("is closed", false, isClosed.booleanValue());

    // this will close the stream. There are 2 calls to flush() during the close() - one in Kryo
    // Output and one
    // in FilterOutputStream
    j.setOutputStream(null);
    assertEquals("flush count", 4, flushCount.intValue());
    assertEquals("is closed", true, isClosed.booleanValue());

    // output stream is closed, so state will be changed without recording it in the journal
    o1p1.setState(PTOperator.State.INACTIVE);
    assertEquals(PTOperator.State.INACTIVE, o1p1.getState());
    assertEquals("flush count", 4, flushCount.intValue());

    ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
    j.replay(new DataInputStream(bis));
    assertEquals(PTOperator.State.ACTIVE, o1p1.getState());

    InetSocketAddress addr1 = InetSocketAddress.createUnresolved("host1", 1);
    PTContainer c1 = plan.getContainers().get(0);
    c1.setState(PTContainer.State.ALLOCATED);
    c1.host = "host1";
    c1.bufferServerAddress = addr1;
    c1.setAllocatedMemoryMB(2);
    c1.setRequiredMemoryMB(1);
    c1.setAllocatedVCores(3);
    c1.setRequiredVCores(4);

    j.setOutputStream(new DataOutputStream(bos));
    j.write(c1.getSetContainerState());

    c1.setExternalId(null);
    c1.setState(PTContainer.State.NEW);
    c1.setExternalId(null);
    c1.host = null;
    c1.bufferServerAddress = null;

    bis = new ByteArrayInputStream(bos.toByteArray());
    j.replay(new DataInputStream(bis));

    assertEquals(externalId, c1.getExternalId());
    assertEquals(PTContainer.State.ALLOCATED, c1.getState());
    assertEquals("host1", c1.host);
    assertEquals(addr1, c1.bufferServerAddress);
    assertEquals(1, c1.getRequiredMemoryMB());
    assertEquals(2, c1.getAllocatedMemoryMB());
    assertEquals(3, c1.getAllocatedVCores());
    assertEquals(4, c1.getRequiredVCores());

    j.write(scm.getSetOperatorProperty("o1", "maxTuples", "100"));
    o1.setMaxTuples(10);
    j.setOutputStream(null);
    bis = new ByteArrayInputStream(bos.toByteArray());
    j.replay(new DataInputStream(bis));
    assertEquals(100, o1.getMaxTuples());

    j.setOutputStream(new DataOutputStream(bos));
    scm.setOperatorProperty("o1", "maxTuples", "10");
    assertEquals(10, o1.getMaxTuples());
    o1.setMaxTuples(100);
    assertEquals(100, o1.getMaxTuples());
    j.setOutputStream(null);

    bis = new ByteArrayInputStream(bos.toByteArray());
    j.replay(new DataInputStream(bis));
    assertEquals(10, o1.getMaxTuples());

    j.setOutputStream(new DataOutputStream(bos));
    scm.setPhysicalOperatorProperty(o1p1.getId(), "maxTuples", "50");
  }
예제 #8
0
  /**
   * Test serialization of the container manager with mock execution layer.
   *
   * @throws Exception
   */
  private void testContainerManager(StorageAgent agent) throws Exception {
    FileUtils.deleteDirectory(new File(testMeta.dir)); // clean any state from previous run

    LogicalPlan dag = new LogicalPlan();
    dag.setAttribute(LogicalPlan.APPLICATION_PATH, testMeta.dir);
    dag.setAttribute(OperatorContext.STORAGE_AGENT, agent);

    StatsListeningOperator o1 = dag.addOperator("o1", StatsListeningOperator.class);

    FSRecoveryHandler recoveryHandler =
        new FSRecoveryHandler(dag.assertAppPath(), new Configuration(false));
    StreamingContainerManager scm =
        StreamingContainerManager.getInstance(recoveryHandler, dag, false);
    File expFile = new File(recoveryHandler.getDir(), FSRecoveryHandler.FILE_SNAPSHOT);
    Assert.assertTrue("snapshot file " + expFile, expFile.exists());

    PhysicalPlan plan = scm.getPhysicalPlan();
    assertEquals("number required containers", 1, plan.getContainers().size());

    PTOperator o1p1 = plan.getOperators(dag.getMeta(o1)).get(0);

    @SuppressWarnings(
        "UnusedAssignment") /* sneaky: the constructor does some changes to the container */
    MockContainer mc = new MockContainer(scm, o1p1.getContainer());
    PTContainer originalContainer = o1p1.getContainer();

    Assert.assertNotNull(o1p1.getContainer().bufferServerAddress);
    assertEquals(PTContainer.State.ACTIVE, o1p1.getContainer().getState());
    assertEquals("state " + o1p1, PTOperator.State.PENDING_DEPLOY, o1p1.getState());

    // test restore initial snapshot + log
    dag = new LogicalPlan();
    dag.setAttribute(LogicalPlan.APPLICATION_PATH, testMeta.dir);
    scm =
        StreamingContainerManager.getInstance(
            new FSRecoveryHandler(dag.assertAppPath(), new Configuration(false)), dag, false);
    dag = scm.getLogicalPlan();
    plan = scm.getPhysicalPlan();

    o1p1 = plan.getOperators(dag.getOperatorMeta("o1")).get(0);
    assertEquals("post restore state " + o1p1, PTOperator.State.PENDING_DEPLOY, o1p1.getState());
    o1 = (StatsListeningOperator) o1p1.getOperatorMeta().getOperator();
    assertEquals(
        "containerId", originalContainer.getExternalId(), o1p1.getContainer().getExternalId());
    assertEquals("stats listener", 1, o1p1.statsListeners.size());
    assertEquals("number stats calls", 0, o1.processStatsCnt); // stats are not logged
    assertEquals("post restore 1", PTContainer.State.ALLOCATED, o1p1.getContainer().getState());
    assertEquals(
        "post restore 1",
        originalContainer.bufferServerAddress,
        o1p1.getContainer().bufferServerAddress);

    StreamingContainerAgent sca = scm.getContainerAgent(originalContainer.getExternalId());
    Assert.assertNotNull("allocated container restored " + originalContainer, sca);
    assertEquals(
        "memory usage allocated container",
        (int) OperatorContext.MEMORY_MB.defaultValue,
        sca.container.getAllocatedMemoryMB());

    // YARN-1490 - simulate container terminated on AM recovery
    scm.scheduleContainerRestart(originalContainer.getExternalId());
    assertEquals("memory usage of failed container", 0, sca.container.getAllocatedMemoryMB());

    Checkpoint firstCheckpoint = new Checkpoint(3, 0, 0);
    mc = new MockContainer(scm, o1p1.getContainer());
    checkpoint(scm, o1p1, firstCheckpoint);
    mc.stats(o1p1.getId())
        .deployState(OperatorHeartbeat.DeployState.ACTIVE)
        .currentWindowId(3)
        .checkpointWindowId(3);
    mc.sendHeartbeat();
    assertEquals("state " + o1p1, PTOperator.State.ACTIVE, o1p1.getState());

    // logical plan modification triggers snapshot
    CreateOperatorRequest cor = new CreateOperatorRequest();
    cor.setOperatorFQCN(GenericTestOperator.class.getName());
    cor.setOperatorName("o2");
    CreateStreamRequest csr = new CreateStreamRequest();
    csr.setSourceOperatorName("o1");
    csr.setSourceOperatorPortName("outport");
    csr.setSinkOperatorName("o2");
    csr.setSinkOperatorPortName("inport1");
    FutureTask<?> lpmf = scm.logicalPlanModification(Lists.newArrayList(cor, csr));
    while (!lpmf.isDone()) {
      scm.monitorHeartbeat();
    }
    Assert.assertNull(lpmf.get()); // unmask exception, if any

    Assert.assertSame("dag references", dag, scm.getLogicalPlan());
    assertEquals("number operators after plan modification", 2, dag.getAllOperators().size());

    // set operator state triggers journal write
    o1p1.setState(PTOperator.State.INACTIVE);

    Checkpoint offlineCheckpoint = new Checkpoint(10, 0, 0);
    // write checkpoint while AM is out,
    // it needs to be picked up as part of restore
    checkpoint(scm, o1p1, offlineCheckpoint);

    // test restore
    dag = new LogicalPlan();
    dag.setAttribute(LogicalPlan.APPLICATION_PATH, testMeta.dir);
    scm =
        StreamingContainerManager.getInstance(
            new FSRecoveryHandler(dag.assertAppPath(), new Configuration(false)), dag, false);

    Assert.assertNotSame("dag references", dag, scm.getLogicalPlan());
    assertEquals(
        "number operators after restore", 2, scm.getLogicalPlan().getAllOperators().size());

    dag = scm.getLogicalPlan();
    plan = scm.getPhysicalPlan();

    o1p1 = plan.getOperators(dag.getOperatorMeta("o1")).get(0);
    assertEquals("post restore state " + o1p1, PTOperator.State.INACTIVE, o1p1.getState());
    o1 = (StatsListeningOperator) o1p1.getOperatorMeta().getOperator();
    assertEquals("stats listener", 1, o1p1.statsListeners.size());
    assertEquals("number stats calls post restore", 1, o1.processStatsCnt);
    assertEquals("post restore 1", PTContainer.State.ACTIVE, o1p1.getContainer().getState());
    assertEquals(
        "post restore 1",
        originalContainer.bufferServerAddress,
        o1p1.getContainer().bufferServerAddress);

    // offline checkpoint detection
    assertEquals(
        "checkpoints after recovery",
        Lists.newArrayList(firstCheckpoint, offlineCheckpoint),
        o1p1.checkpoints);
  }
  /**
   * 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());
  }
예제 #10
0
 public void addOperatorRequest(StramToNodeRequest r) {
   LOG.info("Adding operator request {} {}", container.getExternalId(), r);
   this.operatorRequests.add(r);
 }
  @Test
  public void testStaticPartitioning() {
    //
    //            ,---> node2----,
    //            |              |
    //    node1---+---> node2----+--->unifier--->node3
    //            |              |
    //            '---> node2----'
    //
    GenericTestOperator node1 = dag.addOperator("node1", GenericTestOperator.class);
    PhysicalPlanTest.PartitioningTestOperator node2 =
        dag.addOperator("node2", PhysicalPlanTest.PartitioningTestOperator.class);
    node2.setPartitionCount(3);
    dag.setAttribute(
        node2, OperatorContext.SPIN_MILLIS, 10); /* this should not affect anything materially */
    dag.setOutputPortAttribute(node2.outport1, PortContext.QUEUE_CAPACITY, 1111);
    GenericTestOperator node3 = dag.addOperator("node3", GenericTestOperator.class);
    dag.setInputPortAttribute(node3.inport1, PortContext.QUEUE_CAPACITY, 2222);

    LogicalPlan.StreamMeta n1n2 = dag.addStream("n1n2", node1.outport1, node2.inport1);
    LogicalPlan.StreamMeta n2n3 = dag.addStream("n2n3", node2.outport1, node3.inport1);

    dag.setAttribute(LogicalPlan.CONTAINERS_MAX_COUNT, Integer.MAX_VALUE);
    MemoryStorageAgent msa = new MemoryStorageAgent();
    dag.setAttribute(OperatorContext.STORAGE_AGENT, msa);

    StreamingContainerManager dnm = new StreamingContainerManager(dag);
    PhysicalPlan plan = dnm.getPhysicalPlan();

    Assert.assertEquals("number containers", 6, plan.getContainers().size());
    List<StreamingContainerAgent> containerAgents = Lists.newArrayList();
    for (int i = 0; i < plan.getContainers().size(); i++) {
      containerAgents.add(assignContainer(dnm, "container" + (i + 1)));
    }

    PTContainer c = plan.getOperators(dag.getMeta(node1)).get(0).getContainer();
    StreamingContainerAgent sca1 = dnm.getContainerAgent(c.getExternalId());
    List<OperatorDeployInfo> c1 = getDeployInfo(sca1);
    Assert.assertEquals("number operators assigned to container", 1, c1.size());
    Assert.assertTrue(
        dag.getMeta(node2) + " assigned to " + sca1.container.getExternalId(),
        containsNodeContext(c1, dag.getMeta(node1)));

    List<PTOperator> o2Partitions = plan.getOperators(dag.getMeta(node2));
    Assert.assertEquals(
        "number partitions", TestStaticPartitioningSerDe.partitions.length, o2Partitions.size());

    for (int i = 0; i < o2Partitions.size(); i++) {
      String containerId = o2Partitions.get(i).getContainer().getExternalId();
      List<OperatorDeployInfo> cc = getDeployInfo(dnm.getContainerAgent(containerId));
      Assert.assertEquals("number operators assigned to container", 1, cc.size());
      Assert.assertTrue(
          dag.getMeta(node2) + " assigned to " + containerId,
          containsNodeContext(cc, dag.getMeta(node2)));

      // n1n2 in, mergeStream out
      OperatorDeployInfo ndi = cc.get(0);
      Assert.assertEquals("type " + ndi, OperatorDeployInfo.OperatorType.GENERIC, ndi.type);
      Assert.assertEquals("inputs " + ndi, 1, ndi.inputs.size());
      Assert.assertEquals("outputs " + ndi, 1, ndi.outputs.size());

      InputDeployInfo nidi = ndi.inputs.get(0);
      Assert.assertEquals("stream " + nidi, n1n2.getName(), nidi.declaredStreamId);
      Assert.assertEquals(
          "partition for " + containerId,
          Sets.newHashSet(node2.partitionKeys[i]),
          nidi.partitionKeys);
      Assert.assertEquals("number stream codecs for " + nidi, 1, nidi.streamCodecs.size());
    }

    // unifier
    List<PTOperator> o2Unifiers = plan.getMergeOperators(dag.getMeta(node2));
    Assert.assertEquals("number unifiers", 1, o2Unifiers.size());
    List<OperatorDeployInfo> cUnifier =
        getDeployInfo(dnm.getContainerAgent(o2Unifiers.get(0).getContainer().getExternalId()));
    Assert.assertEquals("number operators " + cUnifier, 1, cUnifier.size());

    OperatorDeployInfo mergeNodeDI =
        getNodeDeployInfo(cUnifier, dag.getMeta(node2).getMeta(node2.outport1).getUnifierMeta());
    Assert.assertNotNull("unifier for " + node2, mergeNodeDI);
    Assert.assertEquals(
        "type " + mergeNodeDI, OperatorDeployInfo.OperatorType.UNIFIER, mergeNodeDI.type);
    Assert.assertEquals("inputs " + mergeNodeDI, 3, mergeNodeDI.inputs.size());
    List<Integer> sourceNodeIds = Lists.newArrayList();
    for (InputDeployInfo nidi : mergeNodeDI.inputs) {
      Assert.assertEquals("streamName " + nidi, n2n3.getName(), nidi.declaredStreamId);
      String mergePortName =
          "<merge#" + dag.getMeta(node2).getMeta(node2.outport1).getPortName() + ">";
      Assert.assertEquals("portName " + nidi, mergePortName, nidi.portName);
      Assert.assertNotNull("sourceNodeId " + nidi, nidi.sourceNodeId);
      Assert.assertNotNull("contextAttributes " + nidi, nidi.contextAttributes);
      Assert.assertEquals(
          "contextAttributes ", new Integer(1111), nidi.getValue(PortContext.QUEUE_CAPACITY));
      sourceNodeIds.add(nidi.sourceNodeId);
    }
    for (PTOperator node : dnm.getPhysicalPlan().getOperators(dag.getMeta(node2))) {
      Assert.assertTrue(
          sourceNodeIds + " contains " + node.getId(), sourceNodeIds.contains(node.getId()));
    }

    Assert.assertEquals("outputs " + mergeNodeDI, 1, mergeNodeDI.outputs.size());
    for (OutputDeployInfo odi : mergeNodeDI.outputs) {
      Assert.assertNotNull("contextAttributes " + odi, odi.contextAttributes);
      Assert.assertEquals(
          "contextAttributes ", new Integer(2222), odi.getValue(PortContext.QUEUE_CAPACITY));
    }

    try {
      Object operator = msa.load(mergeNodeDI.id, Stateless.WINDOW_ID);
      Assert.assertTrue("" + operator, operator instanceof DefaultUnifier);
    } catch (IOException ex) {
      throw new RuntimeException(ex);
    }

    // node3 container
    c = plan.getOperators(dag.getMeta(node3)).get(0).getContainer();
    List<OperatorDeployInfo> cmerge = getDeployInfo(dnm.getContainerAgent(c.getExternalId()));
    Assert.assertEquals("number operators " + cmerge, 1, cmerge.size());

    OperatorDeployInfo node3DI = getNodeDeployInfo(cmerge, dag.getMeta(node3));
    Assert.assertNotNull(dag.getMeta(node3) + " assigned", node3DI);
    Assert.assertEquals("inputs " + node3DI, 1, node3DI.inputs.size());
    InputDeployInfo node3In = node3DI.inputs.get(0);
    Assert.assertEquals("streamName " + node3In, n2n3.getName(), node3In.declaredStreamId);
    Assert.assertEquals(
        "portName " + node3In,
        dag.getMeta(node3).getMeta(node3.inport1).getPortName(),
        node3In.portName);
    Assert.assertNotNull("sourceNodeId " + node3DI, node3In.sourceNodeId);
    Assert.assertEquals(
        "sourcePortName " + node3DI, mergeNodeDI.outputs.get(0).portName, node3In.sourcePortName);
  }