/** * 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); } }
/** * 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))); } }