public static void put_xor(RotatingMap<Tuple, Long> pending, Tuple key, Long id) {
   // synchronized (pending) {
   Long curr = pending.get(key);
   if (curr == null) {
     curr = Long.valueOf(0);
   }
   pending.put(key, JStormUtils.bit_xor(curr, id));
   // }
 }
  @Override
  public void onEvent(Object event, long sequence, boolean endOfBatch) throws Exception {

    if (event == null) {
      return;
    }

    boltExeTimer.start();

    try {

      if (event instanceof RotatingMapTrigger.Tick) {
        // don't check the timetick name to improve performance

        Map<Tuple, Long> timeoutMap = tuple_start_times.rotate();

        if (ackerNum > 0) {
          // only when acker is enable
          for (Entry<Tuple, Long> entry : timeoutMap.entrySet()) {
            Tuple input = entry.getKey();
            task_stats.bolt_failed_tuple(input.getSourceComponent(), input.getSourceStreamId());
          }
        }

        return;
      }

      Tuple tuple = (Tuple) event;

      task_stats.recv_tuple(tuple.getSourceComponent(), tuple.getSourceStreamId());

      tuple_start_times.put(tuple, System.currentTimeMillis());

      try {
        bolt.execute(tuple);
      } catch (Throwable e) {
        error = e;
        LOG.error("bolt execute error ", e);
        report_error.report(e);
      }

      if (ackerNum == 0) {
        // only when acker is disable
        // get tuple process latency
        Long start_time = (Long) tuple_start_times.remove(tuple);
        if (start_time != null) {
          Long delta = TimeUtils.time_delta_ms(start_time);
          task_stats.bolt_acked_tuple(tuple.getSourceComponent(), tuple.getSourceStreamId(), delta);
        }
      }
    } finally {
      boltExeTimer.stop();
    }
  }
 // Utility functions, just used here
 public static Long tuple_time_delta(RotatingMap<Tuple, Long> start_times, Tuple tuple) {
   Long start_time = (Long) start_times.remove(tuple);
   if (start_time != null) {
     return TimeUtils.time_delta_ms(start_time);
   }
   return null;
 }
  @Override
  public void ack(Tuple input) {

    if (ackerNum > 0) {

      Long ack_val = Long.valueOf(0);
      Object pend_val = pending_acks.remove(input);
      if (pend_val != null) {
        ack_val = (Long) (pend_val);
      }

      for (Entry<Long, Long> e : input.getMessageId().getAnchorsToIds().entrySet()) {

        UnanchoredSend.send(
            topologyContext,
            sendTargets,
            taskTransfer,
            Acker.ACKER_ACK_STREAM_ID,
            JStormUtils.mk_list((Object) e.getKey(), JStormUtils.bit_xor(e.getValue(), ack_val)));
      }
    }

    Long delta = tuple_time_delta(tuple_start_times, input);
    if (delta != null) {
      task_stats.bolt_acked_tuple(input.getSourceComponent(), input.getSourceStreamId(), delta);
    }
  }
  @Override
  public void fail(Tuple input) {
    // if ackerNum == 0, we can just return
    if (ackerNum > 0) {
      pending_acks.remove(input);
      for (Entry<Long, Long> e : input.getMessageId().getAnchorsToIds().entrySet()) {
        UnanchoredSend.send(
            topologyContext,
            sendTargets,
            taskTransfer,
            Acker.ACKER_FAIL_STREAM_ID,
            JStormUtils.mk_list((Object) e.getKey()));
      }
    }

    task_stats.bolt_failed_tuple(input.getSourceComponent(), input.getSourceStreamId());
  }
  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>();
  }