/** * Run a topology to completion capturing all of the messages that are emitted. This only works when all of the spouts are * instances of {@link org.apache.storm.testing.CompletableSpout} * @param cluster the cluster to submit the topology to * @param topology the topology itself * @return a map of the component to the list of tuples it emitted. * @throws InterruptedException * @throws TException on any error from nimbus. */ public static Map<String, List<FixedTuple>> completeTopology(ILocalCluster cluster, StormTopology topology) throws InterruptedException, TException { return completeTopology(cluster, topology, new CompleteTopologyParam()); }
/** * Get all of the tuples from a given component on the default stream * @param results the results of running a completed topology * @param componentId the id of the component to look at * @return a list of the tuple values. */ public static List<List<Object>> readTuples(Map<String, List<FixedTuple>> results, String componentId) { return readTuples(results, componentId, Utils.DEFAULT_STREAM_ID); }
/** * Create a {@link org.apache.storm.tuple.Tuple} for use with testing * @param values the values to appear in the tuple */ public static Tuple testTuple(List<Object> values) { return testTuple(values, new MkTupleParam()); }
@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"))); } }
/** * Continue to execute body repeatedly until condition is true or TEST_TIMEOUT_MS has * passed * @param condition what we are waiting for * @param body what to run in the loop * @throws AssertionError if the loop timed out. */ public static void whileTimeout(Condition condition, Runnable body) { whileTimeout(TEST_TIMEOUT_MS, condition, body); }
Map<String, List<FixedTuple>> ret = null; IStormClusterState state = cluster.getClusterState(); CapturedTopology<StormTopology> capTopo = captureTopology(topology); topology = capTopo.topology; String topoName = param.getTopologyName(); simulateWait(cluster); Integer timeoutMs = param.getTimeoutMs(); if (timeoutMs == null) { timeoutMs = TEST_TIMEOUT_MS; whileTimeout(timeoutMs, () -> !isEvery(spoutObjects, (o) -> ((CompletableSpout) o).isExhausted()), () -> { try { simulateWait(cluster); } catch (Exception e) { throw new RuntimeException(); cluster.killTopologyWithOpts(topoName, killOpts); whileTimeout(timeoutMs, () -> state.assignmentInfo(topoId, null) != null, () -> { try { simulateWait(cluster); } catch (Exception e) { throw new RuntimeException();
/** * Run with a local cluster * @deprecated use ``` * try (LocalCluster cluster = new LocalCluster()) { * ... * } * ``` * @param code what to run */ @Deprecated public static void withLocalCluster(TestJob code) { withLocalCluster(new MkClusterParam(), code); }
/** * Run with a local cluster * @deprecated use ``` * try (LocalCluster cluster = new LocalCluster.Builder().withSimulatedTime().build()) { * ... * } * ``` * @param code what to run */ @Deprecated public static void withSimulatedTimeLocalCluster(TestJob code) { withSimulatedTimeLocalCluster(new MkClusterParam(), code); }
private static LocalCluster cluster(MkClusterParam param, boolean simulated) throws Exception { return cluster(param, null, simulated); }
/** * Simulated time wait for a cluster. This is intended for internal testing */ public static void advanceClusterTime(ILocalCluster cluster, Integer secs) throws InterruptedException { advanceClusterTime(cluster, secs, 1); }
/** * Track and capture a topology. * This is intended mostly for internal testing. */ public static CapturedTopology<TrackedTopology> trackAndCaptureTopology(ILocalCluster cluster, StormTopology topology) { CapturedTopology<StormTopology> captured = captureTopology(topology); return new CapturedTopology<>(new TrackedTopology(captured.topology, cluster), captured.capturer); }
completeTopologyParams.setStormConf(stormConf); Map<String, List<FixedTuple>> results = Testing.completeTopology(cluster, topologyBuilder.createTopology(), completeTopologyParams); expectedKryoRegister.put("fake.type2", "a.serializer"); expectedValues.put(Config.TOPOLOGY_KRYO_REGISTER, expectedKryoRegister); List<Object> concatValues = Testing.readTuples(results, "2").stream() .flatMap(values -> values.stream()) .collect(Collectors.toList());
/** * Wait for amt tuples to be fully processed timeoutMs happens. */ public void trackedWait(int amt, int timeoutMs) { final int target = amt + lastSpoutCommit.get(); final String id = cluster.getTrackedId(); Random rand = ThreadLocalRandom.current(); whileTimeout(timeoutMs, () -> { int se = globalAmt(id, "spout-emitted"); int transferred = globalAmt(id, "transferred"); int processed = globalAmt(id, "processed"); LOG.info("emitted {} target {} transferred {} processed {}", se, target, transferred, processed); return (target != se) || (transferred != processed); }, () -> { Time.advanceTimeSecs(1); try { Thread.sleep(rand.nextInt(200)); } catch (Exception e) { throw new RuntimeException(e); } }); lastSpoutCommit.set(target); }
@Test @IntegrationTest public void testCompleteTopologyLocal() throws Exception { MkClusterParam param = new MkClusterParam(); param.setSupervisors(4); Testing.withLocalCluster(param, COMPLETE_TOPOLOGY_TESTJOB); }
@Test @IntegrationTest public void testCompleteTopologyLocalSimulated() throws Exception { MkClusterParam param = new MkClusterParam(); param.setSupervisors(4); Testing.withSimulatedTimeLocalCluster(param, COMPLETE_TOPOLOGY_TESTJOB); }
private static LocalCluster cluster(MkClusterParam param) throws Exception { return cluster(param, null, false); }
@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)); } }
private void waitForNotNull(AtomicReference<TaskMessage> response) { Testing.whileTimeout(Testing.TEST_TIMEOUT_MS, () -> response.get() == null, sleep()); }
private Tuple createTestTuple(String... values) { MkTupleParam param = new MkTupleParam(); param.setFields("key", "message"); return Testing.testTuple(Arrays.asList(values), param); }
@Test @IntegrationTest public void testCompleteTopologyNetty() throws Exception { Config daemonConf = new Config(); daemonConf.put(Config.STORM_LOCAL_MODE_ZMQ, true); MkClusterParam param = new MkClusterParam(); param.setSupervisors(4); param.setDaemonConf(daemonConf); Testing.withLocalCluster(param, COMPLETE_TOPOLOGY_TESTJOB); }