private void testPhysicalPlanSerialization(StorageAgent agent) throws Exception { LogicalPlan dag = new LogicalPlan(); GenericTestOperator o1 = dag.addOperator("o1", GenericTestOperator.class); PartitioningTestOperator o2 = dag.addOperator("o2", PartitioningTestOperator.class); o2.setPartitionCount(3); GenericTestOperator o3 = dag.addOperator("o3", GenericTestOperator.class); dag.addStream("o1.outport1", o1.outport1, o2.inport1, o2.inportWithCodec); dag.addStream("mergeStream", o2.outport1, o3.inport1); dag.getAttributes().put(LogicalPlan.CONTAINERS_MAX_COUNT, 2); TestPlanContext ctx = new TestPlanContext(); dag.setAttribute(OperatorContext.STORAGE_AGENT, agent); PhysicalPlan plan = new PhysicalPlan(dag, ctx); ByteArrayOutputStream bos = new ByteArrayOutputStream(); LogicalPlan.write(dag, bos); LOG.debug("logicalPlan size: " + bos.toByteArray().length); bos = new ByteArrayOutputStream(); ObjectOutputStream oos = new ObjectOutputStream(bos); oos.writeObject(plan); LOG.debug("physicalPlan size: " + bos.toByteArray().length); ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray()); plan = (PhysicalPlan) new ObjectInputStream(bis).readObject(); dag = plan.getLogicalPlan(); Field f = PhysicalPlan.class.getDeclaredField("ctx"); f.setAccessible(true); f.set(plan, ctx); f.setAccessible(false); OperatorMeta o2Meta = dag.getOperatorMeta("o2"); List<PTOperator> o2Partitions = plan.getOperators(o2Meta); assertEquals(3, o2Partitions.size()); for (PTOperator o : o2Partitions) { Assert.assertNotNull("partition null " + o, o.getPartitionKeys()); assertEquals( "partition keys " + o + " " + o.getPartitionKeys(), 2, o.getPartitionKeys().size()); PartitioningTestOperator partitionedInstance = (PartitioningTestOperator) plan.loadOperator(o); assertEquals( "instance per partition", o.getPartitionKeys().values().toString(), partitionedInstance.pks); Assert.assertNotNull("partition stats null " + o, o.stats); } }
private void testRestartApp(StorageAgent agent, String appPath1) throws Exception { FileUtils.deleteDirectory(new File(testMeta.dir)); // clean any state from previous run String appId1 = "app1"; String appId2 = "app2"; String appPath2 = testMeta.dir + "/" + appId2; LogicalPlan dag = new LogicalPlan(); dag.setAttribute(LogicalPlan.APPLICATION_ID, appId1); dag.setAttribute(LogicalPlan.APPLICATION_PATH, appPath1); dag.setAttribute(LogicalPlan.APPLICATION_ATTEMPT_ID, 1); dag.setAttribute(OperatorContext.STORAGE_AGENT, agent); dag.addOperator("o1", StatsListeningOperator.class); FSRecoveryHandler recoveryHandler = new FSRecoveryHandler(dag.assertAppPath(), new Configuration(false)); StreamingContainerManager.getInstance(recoveryHandler, dag, false); // test restore initial snapshot + log dag = new LogicalPlan(); dag.setAttribute(LogicalPlan.APPLICATION_PATH, appPath1); StreamingContainerManager scm = StreamingContainerManager.getInstance( new FSRecoveryHandler(dag.assertAppPath(), new Configuration(false)), dag, false); PhysicalPlan plan = scm.getPhysicalPlan(); dag = plan.getLogicalPlan(); // original plan Assert.assertNotNull("operator", dag.getOperatorMeta("o1")); PTOperator o1p1 = plan.getOperators(dag.getOperatorMeta("o1")).get(0); long[] ids = new FSStorageAgent(appPath1 + "/" + LogicalPlan.SUBDIR_CHECKPOINTS, new Configuration()) .getWindowIds(o1p1.getId()); Assert.assertArrayEquals(new long[] {o1p1.getRecoveryCheckpoint().getWindowId()}, ids); Assert.assertNull(o1p1.getContainer().getExternalId()); // trigger journal write o1p1.getContainer().setExternalId("cid1"); scm.writeJournal(o1p1.getContainer().getSetContainerState()); dag = new LogicalPlan(); dag.setAttribute(LogicalPlan.APPLICATION_PATH, appPath2); dag.setAttribute(LogicalPlan.APPLICATION_ID, appId2); StramClient sc = new StramClient(new Configuration(), dag); try { sc.start(); sc.copyInitialState(new Path(appPath1)); } finally { sc.stop(); } scm = StreamingContainerManager.getInstance( new FSRecoveryHandler(dag.assertAppPath(), new Configuration(false)), dag, false); plan = scm.getPhysicalPlan(); dag = plan.getLogicalPlan(); assertEquals("modified appId", appId2, dag.getValue(LogicalPlan.APPLICATION_ID)); assertEquals("modified appPath", appPath2, dag.getValue(LogicalPlan.APPLICATION_PATH)); Assert.assertNotNull("operator", dag.getOperatorMeta("o1")); o1p1 = plan.getOperators(dag.getOperatorMeta("o1")).get(0); assertEquals("journal copied", "cid1", o1p1.getContainer().getExternalId()); ids = new FSStorageAgent(appPath2 + "/" + LogicalPlan.SUBDIR_CHECKPOINTS, new Configuration()) .getWindowIds(o1p1.getId()); Assert.assertArrayEquals( "checkpoints copied", new long[] {o1p1.getRecoveryCheckpoint().getWindowId()}, ids); }
/** * 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); }