private List<Integer> boltEmit( String out_stream_id, Collection<Tuple> anchors, List<Object> values, Integer out_task_id) { timer.start(); try { java.util.List<Integer> out_tasks = null; if (out_task_id != null) { out_tasks = sendTargets.get(out_task_id, out_stream_id, values); } else { out_tasks = sendTargets.get(out_stream_id, values); } for (Integer t : out_tasks) { Map<Long, Long> anchors_to_ids = new HashMap<Long, Long>(); if (anchors != null) { for (Tuple a : anchors) { // Long edge_id = MessageId.generateId(); Long edge_id = MessageId.generateId(random); long now = System.currentTimeMillis(); if (now - lastRotate > rotateTime) { pending_acks.rotate(); lastRotate = now; } put_xor(pending_acks, a, edge_id); for (Long root_id : a.getMessageId().getAnchorsToIds().keySet()) { put_xor(anchors_to_ids, root_id, edge_id); } } } MessageId msgid = MessageId.makeId(anchors_to_ids); TupleImplExt tupleExt = new TupleImplExt(topologyContext, values, task_id, out_stream_id, msgid); tupleExt.setTargetTaskId(t); taskTransfer.transfer(tupleExt); } return out_tasks; } catch (Exception e) { LOG.error("bolt emit", e); } finally { timer.stop(); } return new ArrayList<Integer>(); }
private List<Integer> sendSpoutMsg( String out_stream_id, List<Object> values, Object message_id, Integer out_task_id) { emitTotalTimer.start(); try { java.util.List<Integer> out_tasks = null; if (out_task_id != null) { out_tasks = sendTargets.get(out_task_id, out_stream_id, values); } else { out_tasks = sendTargets.get(out_stream_id, values); } if (out_tasks.size() == 0) { // don't need send tuple to other task return out_tasks; } List<Long> ackSeq = new ArrayList<Long>(); Boolean needAck = (message_id != null) && (ackerNum > 0); // This change storm logic // Storm can't make sure root_id is unique // storm's logic is root_id = MessageId.generateId(random); // when duplicate root_id, it will miss call ack/fail Long root_id = MessageId.generateId(random); if (needAck) { while (pending.containsKey(root_id) == true) { root_id = MessageId.generateId(random); } } for (Integer t : out_tasks) { MessageId msgid; if (needAck) { // Long as = MessageId.generateId(); Long as = MessageId.generateId(random); msgid = MessageId.makeRootId(root_id, as); ackSeq.add(as); } else { msgid = MessageId.makeUnanchored(); } TupleImplExt tp = new TupleImplExt(topology_context, values, task_id, out_stream_id, msgid); tp.setTargetTaskId(t); transfer_fn.transfer(tp); } if (needAck) { TupleInfo info = new TupleInfo(); info.setStream(out_stream_id); info.setValues(values); info.setMessageId(message_id); info.setTimestamp(System.currentTimeMillis()); pending.putHead(root_id, info); List<Object> ackerTuple = JStormUtils.mk_list((Object) root_id, JStormUtils.bit_xor_vals(ackSeq), task_id); UnanchoredSend.send( topology_context, sendTargets, transfer_fn, Acker.ACKER_INIT_STREAM_ID, ackerTuple); } else if (message_id != null) { TupleInfo info = new TupleInfo(); info.setStream(out_stream_id); info.setValues(values); info.setMessageId(message_id); info.setTimestamp(0); AckSpoutMsg ack = new AckSpoutMsg(spout, info, task_stats, isDebug); ack.run(); } return out_tasks; } finally { emitTotalTimer.stop(); } }