/** * Simulated time wait for a cluster. This is intended for internal testing */ public static void advanceClusterTime(ILocalCluster cluster, Integer secs, Integer step) throws InterruptedException { cluster.advanceClusterTime(secs, step); }
/** * Read a metric from the tracked cluster (NOT JUST THIS TOPOLOGY) * @param key one of "spout-emitted", "processed", or "transferred" * @return the amount of that metric */ public int globalAmt(String key) { return globalAmt(cluster.getTrackedId(), key); } }
CompleteTopologyParam param) throws TException, InterruptedException { Map<String, List<FixedTuple>> ret = null; IStormClusterState state = cluster.getClusterState(); CapturedTopology<StormTopology> capTopo = captureTopology(topology); topology = capTopo.topology; cluster.submitTopology(topoName, param.getStormConf(), topology); cluster.advanceClusterTime(11); cluster.killTopologyWithOpts(topoName, killOpts);
cluster.submitTopology(topoName, new HashMap<>(), stormTopology); cluster.rebalance("topo1", opts); Utils.sleep(SLEEP_TIME_BETWEEN_RETRY); String confRaw = cluster.getTopologyConf(topoNameToId(topoName, cluster)); StormTopology readStormTopology = cluster.getTopology(topoNameToId(topoName, cluster)); String componentConfRaw = readStormTopology.get_spouts().get("spout-1").get_common().get_json_conf();
topoConf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, TICK_INTERVAL_SECS); try (ILocalTopology topo = cluster.submitTopology("test", topoConf, builder.createTopology())) { cluster.advanceClusterTime(TICK_INTERVAL_SECS); waitForTicks(1); cluster.advanceClusterTime(TICK_INTERVAL_SECS); waitForTicks(2); cluster.advanceClusterTime(TICK_INTERVAL_SECS); waitForTicks(3); } catch (ConditionTimeoutException e) {
@Test public void testSubmitTopologyToLocalNimbus() throws Exception { int port = Utils.getAvailablePort(); try (ILocalCluster localCluster = new LocalCluster.Builder() .withNimbusDaemon(true) .withDaemonConf(Config.NIMBUS_THRIFT_PORT, port) .build()) { Config topoConf = new Config(); topoConf.putAll(Utils.readDefaultConfig()); topoConf.setDebug(true); topoConf.put("storm.cluster.mode", "local"); // default is aways "distributed" but here local cluster is being used. topoConf.put(Config.STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN, InmemoryTopologySubmitterHook.class.getName()); topoConf.put(Config.NIMBUS_THRIFT_PORT, port); List<TopologyDetails> topologyNames = new ArrayList<>(); for (int i = 0; i < 4; i++) { final String topologyName = "word-count-" + UUID.randomUUID().toString(); final StormTopology stormTopology = createTestTopology(); topologyNames.add(new TopologyDetails(topologyName, stormTopology)); localCluster.submitTopology(topologyName, topoConf, stormTopology); } Assert.assertEquals(InmemoryTopologySubmitterHook.submittedTopologies, topologyNames); } }
LOG.info("Submitting topology " + name + " in local mode"); if (opts!=null) { localNimbus.submitTopologyWithOpts(name, stormConf, topology, opts); } else { localNimbus.submitTopology(name, stormConf, topology);
public boolean checkTopologyUp(String topoName, ILocalCluster cluster) throws TException { ClusterSummary sum = cluster.getClusterInfo(); for (TopologySummary topoSum : sum.get_topologies()) { if (topoSum.get_name().equals(topoName)) { return true; } } return false; } }
@Override public void run(ILocalCluster cluster) { try { for (LGAdapter adapter : startAdapters) { cluster.submitTopology(adapter.getAdapterName(), adapter.getConfig(), adapter.getTopology()); } while(true) { //Don't time-out this thread. It kills the launched topologies Thread.sleep(300000); // ~18 seconds to boot up on my machine } } catch (Exception e) { e.printStackTrace(); } } });
public static String topoNameToId(String topoName, ILocalCluster cluster) throws TException { for (TopologySummary topoSum : cluster.getClusterInfo().get_topologies()) { if (topoSum.get_name().equals(topoName)) { return topoSum.get_id(); } } return null; }
/** * If using simulated time simulate waiting for 10 seconds. This is intended for internal testing only. */ public static void simulateWait(ILocalCluster cluster) throws InterruptedException { if (Time.isSimulating()) { cluster.advanceClusterTime(10); Thread.sleep(100); } }
public boolean checkTopologyScheduled(String topoName, ILocalCluster cluster) throws TException { if (checkTopologyUp(topoName, cluster)) { ClusterSummary sum = cluster.getClusterInfo(); for (TopologySummary topoSum : sum.get_topologies()) { if (topoSum.get_name().equals(topoName)) { String status = topoSum.get_status(); String sched_status = topoSum.get_sched_status(); if (status.equals("ACTIVE") && (sched_status != null && !sched_status.equals(""))) { return true; } } } } return false; }
/** * 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); }
/** * Create a new topology to be tracked. * @param origTopo the original topology. * @param cluster a cluster that should have been launched with tracking enabled. */ public TrackedTopology(StormTopology origTopo, ILocalCluster cluster) { LOG.warn("CLUSTER {} - {}", cluster, cluster.getTrackedId()); this.cluster = cluster; lastSpoutCommit = new AtomicInteger(0); String id = cluster.getTrackedId(); topology = origTopo.deepCopy(); for (Bolt bolt : topology.get_bolts().values()) { IRichBolt obj = (IRichBolt) Thrift.deserializeComponentObject(bolt.get_bolt_object()); bolt.set_bolt_object(Thrift.serializeComponentObject(new BoltTracker(obj, id))); } for (SpoutSpec spout : topology.get_spouts().values()) { IRichSpout obj = (IRichSpout) Thrift.deserializeComponentObject(spout.get_spout_object()); spout.set_spout_object(Thrift.serializeComponentObject(new SpoutTracker(obj, id))); } }