.withSupervisors(supervisors.intValue()) .withDaemonConf(conf) .withPortsPerSupervisor(ports.intValue()) .withINimbus(inimbus) .withSupervisorSlotPortMin(portMin)
@Test public void testWithSimulatedTimeLocalCluster() throws Exception { assertThat(Time.isSimulating(), is(false)); try (LocalCluster cluster = new LocalCluster.Builder() .withSupervisors(2) .withPortsPerSupervisor(5) .withSimulatedTime() .build()) { assertThat(cluster, notNullValue()); assertThat(cluster.getNimbus(), notNullValue()); assertThat(Time.isSimulating(), is(true)); } }
@Test public void testWithLocalCluster() throws Exception { try (LocalCluster cluster = new LocalCluster.Builder() .withSupervisors(2) .withPortsPerSupervisor(5) .build()) { assertThat(cluster, notNullValue()); assertThat(cluster.getNimbus(), notNullValue()); } }
.withSupervisors(1).withPortsPerSupervisor(2) .withDaemonConf(topoConf).build()) {
@Test public void testLocalTransport() throws Exception { Config topoConf = new Config(); topoConf.put(Config.TOPOLOGY_WORKERS, 2); topoConf.put(Config.STORM_MESSAGING_TRANSPORT, "org.apache.storm.messaging.netty.Context"); try (ILocalCluster cluster = new LocalCluster.Builder().withSimulatedTime() .withSupervisors(1).withPortsPerSupervisor(2) .withDaemonConf(topoConf).build()) { TopologyBuilder builder = new TopologyBuilder(); builder.setSpout("1", new TestWordSpout(true), 2); builder.setBolt("2", new TestGlobalCount(), 6).shuffleGrouping("1"); StormTopology stormTopology = builder.createTopology(); List<FixedTuple> fixedTuples = new ArrayList<>(); for (int i = 0; i < 12; i++) { fixedTuples.add(new FixedTuple(Collections.singletonList("a"))); fixedTuples.add(new FixedTuple(Collections.singletonList("b"))); } Map<String, List<FixedTuple>> data = new HashMap<>(); data.put("1", fixedTuples); MockedSources mockedSources = new MockedSources(data); CompleteTopologyParam completeTopologyParam = new CompleteTopologyParam(); completeTopologyParam.setMockedSources(mockedSources); Map<String, List<FixedTuple>> results = Testing.completeTopology(cluster, stormTopology, completeTopologyParam); Assert.assertEquals(6 * 4, Testing.readTuples(results, "2").size()); } }
private static LocalCluster cluster(MkClusterParam param, String id, boolean simulated) throws Exception { Integer supervisors = param.getSupervisors(); if (supervisors == null) { supervisors = 2; } Integer ports = param.getPortsPerSupervisor(); if (ports == null) { ports = 3; } Map<String, Object> conf = param.getDaemonConf(); if (conf == null) { conf = new HashMap<>(); } return new LocalCluster.Builder() .withSupervisors(supervisors) .withPortsPerSupervisor(ports) .withDaemonConf(conf) .withNimbusDaemon(param.isNimbusDaemon()) .withTracked(id) .withSimulatedTime(simulated) .build(); }