conf.put(Config.TOPOLOGY_COMPONENT_RESOURCES_MAP, resourcesMap); try (ILocalCluster cluster = new LocalCluster.Builder().withDaemonConf(conf).build()) {
Boolean nimbusDaemon = (Boolean) clusterConf.getOrDefault("nimbus-daemon", false); try { return new LocalCluster.Builder() .withSupervisors(supervisors.intValue()) .withDaemonConf(conf) .withPortsPerSupervisor(ports.intValue()) .withINimbus(inimbus) .withSupervisorSlotPortMin(portMin) .withNimbusDaemon(nimbusDaemon) .build(); } catch (Exception e) { throw new RuntimeException(e);
Map<String, Object> daemonConf = new HashMap<>(); daemonConf.put(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS, true); try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .withDaemonConf(daemonConf) .build()) { TopologyBuilder topologyBuilder = new TopologyBuilder(); topologyBuilder.setSpout("1", new TestPlannerSpout(new Fields("conf")));
@ParameterizedTest @ValueSource(strings = {"true", "false"}) public void testBasicTopology(boolean useLocalMessaging) throws Exception { try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .withSupervisors(4) .withDaemonConf(Collections.singletonMap(Config.STORM_LOCAL_MODE_ZMQ, !useLocalMessaging)) .build()) { Map<String, SpoutDetails> spoutMap = Collections.singletonMap("1", Thrift.prepareSpoutDetails(new TestWordSpout(true), 3)); Map<String, BoltDetails> boltMap = new HashMap<>();
@Test public void testAcking() throws Exception { try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .withTracked() .build()) { AckTrackingFeeder feeder1 = new AckTrackingFeeder("num"); AckTrackingFeeder feeder2 = new AckTrackingFeeder("num");
@Test public void testResetTimeout() throws Exception { try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .withDaemonConf(Collections.singletonMap(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS, true)) .build()) { FeederSpout feeder = new FeederSpout(new Fields("field1")); AckFailMapTracker tracker = new AckFailMapTracker();
@Test public void testTickTupleWorksWithSystemBolt() throws Exception { try (ILocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .build()) {
@Test public void testSystemStream() throws Exception { try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .build()) { Map<String, SpoutDetails> spoutMap = Collections.singletonMap("1", Thrift.prepareSpoutDetails(new TestWordSpout(true), 3));
@Test public void testTimeout() throws Exception { try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .withSupervisors(4) .withDaemonConf(Collections.singletonMap(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS, true)) .build()) { FeederSpout feeder = new FeederSpout(new Fields("field1")); AckFailMapTracker tracker = new AckFailMapTracker();
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()) {
@Test public void testSubmitInactiveTopology() throws Exception { try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .withDaemonConf(Collections.singletonMap(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS, true)) .build()) { FeederSpout feeder = new FeederSpout(new Fields("field1")); AckFailMapTracker tracker = new AckFailMapTracker(); feeder.setAckFailDelegate(tracker); Map<String, SpoutDetails> spoutMap = new HashMap<>(); spoutMap.put("1", Thrift.prepareSpoutDetails(feeder)); spoutMap.put("2", Thrift.prepareSpoutDetails(new OpenTrackedSpout())); Map<String, BoltDetails> boltMap = new HashMap<>(); boltMap.put("3", Thrift.prepareBoltDetails(Collections.singletonMap(Utils.getGlobalStreamId("1", null), Thrift.prepareGlobalGrouping()), new PrepareTrackedBolt())); boltPrepared = false; spoutOpened = false; StormTopology topology = Thrift.buildTopology(spoutMap, boltMap); cluster.submitTopologyWithOpts("test", Collections.singletonMap(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 10), topology, new SubmitOptions(TopologyInitialStatus.INACTIVE)); cluster.advanceClusterTime(11); feeder.feed(new Values("a"), 1); cluster.advanceClusterTime(9); assertThat(boltPrepared, is(false)); assertThat(spoutOpened, is(false)); cluster.getNimbus().activate("test"); cluster.advanceClusterTime(12); assertAcked(tracker, 1); assertThat(boltPrepared, is(true)); assertThat(spoutOpened, is(true)); } }
Config daemonConf = new Config(); daemonConf.put(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS, false); try (LocalCluster cluster = new LocalCluster.Builder() .withDaemonConf(daemonConf) .withSimulatedTime() .build()) { FeederSpout feeder = new FeederSpout(new Fields("field1")); AckFailMapTracker tracker = new AckFailMapTracker();
@Test public void testKryoDecoratorsConfig() throws Exception { Map<String, Object> daemonConf = new HashMap<>(); daemonConf.put(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS, true); daemonConf.put(Config.TOPOLOGY_KRYO_DECORATORS, "this-is-overridden"); try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .withDaemonConf(daemonConf) .build()) { TopologyBuilder topologyBuilder = new TopologyBuilder(); topologyBuilder.setSpout("1", new TestPlannerSpout(new Fields("conf"))); topologyBuilder.setBolt("2", new TestConfBolt(Collections.singletonMap(Config.TOPOLOGY_KRYO_DECORATORS, Arrays.asList("one", "two")))) .shuffleGrouping("1"); List<FixedTuple> testTuples = Arrays.asList(new Values(Config.TOPOLOGY_KRYO_DECORATORS)).stream() .map(value -> new FixedTuple(value)) .collect(Collectors.toList()); MockedSources mockedSources = new MockedSources(Collections.singletonMap("1", testTuples)); CompleteTopologyParam completeTopologyParams = new CompleteTopologyParam(); completeTopologyParams.setMockedSources(mockedSources); completeTopologyParams.setStormConf(Collections.singletonMap(Config.TOPOLOGY_KRYO_DECORATORS, Arrays.asList("one", "three"))); Map<String, List<FixedTuple>> results = Testing.completeTopology(cluster, topologyBuilder.createTopology(), completeTopologyParams); List<Object> concatValues = Testing.readTuples(results, "2").stream() .flatMap(values -> values.stream()) .collect(Collectors.toList()); assertThat(concatValues.get(0), is(Config.TOPOLOGY_KRYO_DECORATORS)); assertThat(concatValues.get(1), is(Arrays.asList("one", "two", "three"))); } }
@Test public void testAckBranching() throws Exception { try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .withTracked() .build()) { AckTrackingFeeder feeder = new AckTrackingFeeder("num"); Map<String, SpoutDetails> spoutMap = new HashMap<>(); spoutMap.put("1", Thrift.prepareSpoutDetails(feeder.getSpout())); Map<String, BoltDetails> boltMap = new HashMap<>(); boltMap.put("2", Thrift.prepareBoltDetails(Collections.singletonMap(Utils.getGlobalStreamId("1", null), Thrift.prepareShuffleGrouping()), new IdentityBolt())); boltMap.put("3", Thrift.prepareBoltDetails(Collections.singletonMap(Utils.getGlobalStreamId("1", null), Thrift.prepareShuffleGrouping()), new IdentityBolt())); Map<GlobalStreamId, Grouping> aggregatorInputs = new HashMap<>(); aggregatorInputs.put(Utils.getGlobalStreamId("2", null), Thrift.prepareShuffleGrouping()); aggregatorInputs.put(Utils.getGlobalStreamId("3", null), Thrift.prepareShuffleGrouping()); boltMap.put("4", Thrift.prepareBoltDetails(aggregatorInputs, new AggBolt(4))); TrackedTopology tracked = new TrackedTopology(Thrift.buildTopology(spoutMap, boltMap), cluster);; cluster.submitTopology("test-acking2", Collections.emptyMap(), tracked); cluster.advanceClusterTime(11); feeder.feed(new Values(1)); Testing.trackedWait(tracked, 1); feeder.assertNumAcks(0); feeder.feed(new Values(1)); Testing.trackedWait(tracked, 1); feeder.assertNumAcks(2); } }
Config daemonConf = new Config(); daemonConf.put(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS, true); try (LocalCluster cluster = new LocalCluster.Builder() .withDaemonConf(daemonConf) .withSimulatedTime() .build()) { FeederSpout feeder = new FeederSpout(new Fields("field1")); AckFailMapTracker tracker = new AckFailMapTracker();
@Test public void testMultiTasksPerCluster() throws Exception { try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .withSupervisors(4) .build()) { Map<String, SpoutDetails> spoutMap = Collections.singletonMap("1", Thrift.prepareSpoutDetails(new TestWordSpout(true))); Map<String, BoltDetails> boltMap = new HashMap<>(); boltMap.put("2", Thrift.prepareBoltDetails( Collections.singletonMap( Utils.getGlobalStreamId("1", null), Thrift.prepareAllGrouping()), new EmitTaskIdBolt(), 3, Collections.singletonMap(Config.TOPOLOGY_TASKS, 6))); StormTopology topology = Thrift.buildTopology(spoutMap, boltMap); MockedSources mockedSources = new MockedSources(Collections.singletonMap("1", Collections.singletonList(new FixedTuple(new Values("a"))))); CompleteTopologyParam completeTopologyParams = new CompleteTopologyParam(); completeTopologyParams.setMockedSources(mockedSources); Map<String, List<FixedTuple>> results = Testing.completeTopology(cluster, topology, completeTopologyParams); assertThat(Testing.readTuples(results, "2"), containsInAnyOrder( new Values(0), new Values(1), new Values(2), new Values(3), new Values(4), new Values(5) )); } }
/** * Run c with a local mode cluster overriding the NimbusClient and DRPCClient calls. NOTE local mode override happens by default now * unless netty is turned on for the local cluster. * * @param c the callable to run in this mode * @param ttlSec the number of seconds to let the cluster run after c has completed * @param daemonConf configs to set for the daemon processes. * @return the result of calling C * * @throws Exception on any Exception. */ public static <T> T withLocalModeOverride(Callable<T> c, long ttlSec, Map<String, Object> daemonConf) throws Exception { LOG.info("\n\n\t\tSTARTING LOCAL MODE CLUSTER\n\n"); Builder builder = new Builder(); if (daemonConf != null) { builder.withDaemonConf(daemonConf); } try (LocalCluster local = builder.build(); LocalDRPC drpc = new LocalDRPC(local.metricRegistry); DRPCClient.LocalOverride drpcOverride = new DRPCClient.LocalOverride(drpc)) { T ret = c.call(); LOG.info("\n\n\t\tRUNNING LOCAL CLUSTER for {} seconds.\n\n", ttlSec); Thread.sleep(ttlSec * 1000); LOG.info("\n\n\t\tSTOPPING LOCAL MODE CLUSTER\n\n"); return ret; } }
@Test public void testWithTrackedCluster() throws Exception { try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .withTracked() .build()) { AckTrackingFeeder feeder = new AckTrackingFeeder("num"); Map<String, Thrift.SpoutDetails> spoutMap = new HashMap<>(); spoutMap.put("1", Thrift.prepareSpoutDetails(feeder.getSpout())); Map<String, Thrift.BoltDetails> boltMap = new HashMap<>(); boltMap.put("2", Thrift.prepareBoltDetails(Collections.singletonMap(Utils.getGlobalStreamId("1", null), Thrift.prepareShuffleGrouping()), new IdentityBolt())); boltMap.put("3", Thrift.prepareBoltDetails(Collections.singletonMap(Utils.getGlobalStreamId("1", null), Thrift.prepareShuffleGrouping()), new IdentityBolt())); Map<GlobalStreamId, Grouping> aggregatorInputs = new HashMap<>(); aggregatorInputs.put(Utils.getGlobalStreamId("2", null), Thrift.prepareShuffleGrouping()); aggregatorInputs.put(Utils.getGlobalStreamId("3", null), Thrift.prepareShuffleGrouping()); boltMap.put("4", Thrift.prepareBoltDetails(aggregatorInputs, new AggBolt(4))); TrackedTopology tracked = new TrackedTopology(Thrift.buildTopology(spoutMap, boltMap), cluster);; cluster.submitTopology("test-acking2", new Config(), tracked); cluster.advanceClusterTime(11); feeder.feed(new Values(1)); Testing.trackedWait(tracked, 1); feeder.assertNumAcks(0); feeder.feed(new Values(1)); Testing.trackedWait(tracked, 1); feeder.assertNumAcks(2); } }
@Test public void testHooks() throws Exception { try (LocalCluster cluster = new LocalCluster.Builder() .withSimulatedTime() .build()) { Map<String, SpoutDetails> spoutMap = Collections.singletonMap("1", Thrift.prepareSpoutDetails(new TestPlannerSpout(new Fields("conf")))); Map<String, BoltDetails> boltMap = Collections.singletonMap("2", Thrift.prepareBoltDetails( Collections.singletonMap(Utils.getGlobalStreamId("1", null), Thrift.prepareShuffleGrouping()), new HooksBolt())); StormTopology topology = Thrift.buildTopology(spoutMap, boltMap); List<FixedTuple> testTuples = Arrays.asList(1, 1, 1, 1).stream() .map(value -> new FixedTuple(new Values(value))) .collect(Collectors.toList()); MockedSources mockedSources = new MockedSources(Collections.singletonMap("1", testTuples)); CompleteTopologyParam completeTopologyParams = new CompleteTopologyParam(); completeTopologyParams.setMockedSources(mockedSources); Map<String, List<FixedTuple>> results = Testing.completeTopology(cluster, topology, completeTopologyParams); List<List<Object>> expectedTuples = Arrays.asList( Arrays.asList(0, 0, 0, 0), Arrays.asList(2, 1, 0, 1), Arrays.asList(4, 1, 1, 2), Arrays.asList(6, 2, 1, 3)); assertThat(Testing.readTuples(results, "2"), is(expectedTuples)); } }
@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()); } }