private JCQueue createQueue(String name, int batchSize, int queueSize) { return new JCQueue(name, queueSize, 0, batchSize, waitStrategy, "test", "test", 1000, 1000); }
public void haltTransferThd() { transferQueue.close(); }
private void dropMessage(AddressedTuple tuple, JCQueue queue) { ++dropCount; queue.recordMsgDrop(); LOG.warn( "Dropping message as overflow threshold has reached for Q = {}. OverflowCount = {}. Total Drop Count= {}, Dropped Message : {}", queue.getName(), queue.getOverflowCount(), dropCount, tuple); }
@Test public void testBasicBackPressure() throws Exception { final int MESSAGES = 100; final int CAPACITY = 64; final JCQueue queue = createQueue("testBackPressure", CAPACITY); // pump more msgs than Q size & verify msg count is as expexted for (int i = 0; i < MESSAGES; i++) { if (i >= CAPACITY) { Assert.assertFalse(queue.tryPublish(i)); } else { Assert.assertTrue(queue.tryPublish(i)); } } Assert.assertEquals(CAPACITY, queue.size()); Assert.assertEquals(0, queue.getOverflowCount()); // drain 1 element and ensure BP is relieved (i.e tryPublish() succeeds) final MutableLong consumeCount = new MutableLong(0); queue.consume(new TestConsumer(), () -> consumeCount.increment() <= 1); Assert.assertEquals(CAPACITY - 1, queue.size()); Assert.assertTrue(queue.tryPublish(0)); queue.close(); }
@Test public void testNoReOrderingUnderBackPressure() throws Exception { final int MESSAGES = 100; final int CAPACITY = 64; final JCQueue queue = createQueue("testBackPressure", CAPACITY); for (int i = 0; i < MESSAGES; i++) { if (!queue.tryPublish(i)) { Assert.assertTrue(queue.tryPublishToOverflow(i)); } } TestConsumer consumer = new TestConsumer(); queue.consume(consumer); Assert.assertEquals(MESSAGES, consumer.lastMsg); queue.close(); }
private void transferLocalBatch(ArrayList<AddressedTuple> tupleBatch) { int lastOverflowCount = 0; // overflowQ size at the time the last BPStatus was sent for (int i = 0; i < tupleBatch.size(); i++) { AddressedTuple tuple = tupleBatch.get(i); JCQueue queue = taskToExecutorQueue.get(tuple.dest); // 1- try adding to main queue if its overflow is not empty if (queue.isEmptyOverflow()) { if (queue.tryPublish(tuple)) { continue; } } // 2- BP detected (i.e MainQ is full). So try adding to overflow int currOverflowCount = queue.getOverflowCount(); if (bpTracker.recordBackPressure(tuple.dest)) { receiver.sendBackPressureStatus(bpTracker.getCurrStatus()); lastOverflowCount = currOverflowCount; } else { if (currOverflowCount - lastOverflowCount > 10000) { // resend BP status, in case prev notification was missed or reordered BackPressureStatus bpStatus = bpTracker.getCurrStatus(); receiver.sendBackPressureStatus(bpStatus); lastOverflowCount = currOverflowCount; LOG.debug("Re-sent BackPressure Status. OverflowCount = {}, BP Status ID = {}. ", currOverflowCount, bpStatus.id); } } if (!queue.tryPublishToOverflow(tuple)) { dropMessage(tuple, queue); } } }
protected void setupMetrics() { for (final Integer interval : intervalToTaskToMetricToRegistry.keySet()) { StormTimer timerTask = workerData.getUserTimer(); timerTask.scheduleRecurring(interval, interval, () -> { TupleImpl tuple = new TupleImpl(workerTopologyContext, new Values(interval), Constants.SYSTEM_COMPONENT_ID, (int) Constants.SYSTEM_TASK_ID, Constants.METRICS_TICK_STREAM_ID); AddressedTuple metricsTickTuple = new AddressedTuple(AddressedTuple.BROADCAST_DEST, tuple); try { receiveQueue.publish(metricsTickTuple); receiveQueue.flush(); // avoid buffering } catch (InterruptedException e) { LOG.warn("Thread interrupted when publishing metrics. Setting interrupt flag."); Thread.currentThread().interrupt(); return; } } ); } }
@Override public void accept(Object event) { try { spoutInQ.publish(event); } catch (InterruptedException e) { throw new RuntimeException(e); } }
/** * Non blocking. Returns immediately if Q is empty. Returns number of elements consumed from Q */ public int consume(JCQueue.Consumer consumer) { return consume(consumer, continueRunning); }
@Override public void flush() throws InterruptedException { spoutInQ.flush(); } }
public boolean refreshBpTaskList() { boolean changed = false; LOG.debug("Running Back Pressure status change check"); for (Entry<Integer, BackpressureState> entry : tasks.entrySet()) { BackpressureState state = entry.getValue(); if (state.backpressure.get() && state.queue.isEmptyOverflow()) { recordNoBackPressure(entry.getKey()); changed = true; } } return changed; }
/** * Adds tuple to localQueue (if overflow is empty). If localQueue is full adds to pendingEmits instead. pendingEmits can be null. * Returns false if unable to add to localQueue. */ public boolean tryTransferLocal(AddressedTuple tuple, JCQueue localQueue, Queue<AddressedTuple> pendingEmits) { workerData.checkSerialize(serializer, tuple); if (pendingEmits != null) { if (pendingEmits.isEmpty() && localQueue.tryPublish(tuple)) { queuesToFlush.set(tuple.dest - indexingBase, localQueue); return true; } else { pendingEmits.add(tuple); return false; } } else { return localQueue.tryPublish(tuple); } }
@Override public void credentialsChanged(Credentials credentials) { TupleImpl tuple = new TupleImpl(executor.getWorkerTopologyContext(), new Values(credentials), Constants.SYSTEM_COMPONENT_ID, (int) Constants.SYSTEM_TASK_ID, Constants.CREDENTIALS_CHANGED_STREAM_ID); AddressedTuple addressedTuple = new AddressedTuple(AddressedTuple.BROADCAST_DEST, tuple); try { executor.getReceiveQueue().publish(addressedTuple); executor.getReceiveQueue().flush(); } catch (InterruptedException e) { throw new RuntimeException(e); } }
@Override public void run() { try { long start = System.currentTimeMillis(); while (!Thread.interrupted()) { q.publish(++count); } runTime = System.currentTimeMillis() - start; } catch (InterruptedException e) { return; } }
@Override public void run() { Handler handler = new Handler(); long start = System.currentTimeMillis(); while (!Thread.interrupted()) { int x = inq.consume(handler); if (x == 0) { LockSupport.parkNanos(1); } } runTime = System.currentTimeMillis() - start; }
public void flushRemotes() throws InterruptedException { transferQueue.flush(); }
@Test public void testRefreshBackPressureWithNonEmptyOverflow() { int taskId = 1; JCQueue queue = mock(JCQueue.class); when(queue.isEmptyOverflow()).thenReturn(false); BackPressureTracker tracker = new BackPressureTracker(WORKER_ID, ImmutableMap.of( taskId, queue)); tracker.recordBackPressure(taskId); boolean backpressureChanged = tracker.refreshBpTaskList(); BackPressureStatus status = tracker.getCurrStatus(); assertThat(backpressureChanged, is(false)); assertThat(status.workerId, is(WORKER_ID)); assertThat(status.bpTasks, contains(taskId)); }
public boolean tryTransferRemote(AddressedTuple addressedTuple, Queue<AddressedTuple> pendingEmits, ITupleSerializer serializer) { if (pendingEmits != null && !pendingEmits.isEmpty()) { pendingEmits.add(addressedTuple); return false; } if (!remoteBackPressureStatus[addressedTuple.dest].get()) { TaskMessage tm = new TaskMessage(addressedTuple.getDest(), serializer.serialize(addressedTuple.getTuple())); if (transferQueue.tryPublish(tm)) { return true; } } else { LOG.debug("Noticed Back Pressure in remote task {}", addressedTuple.dest); } if (pendingEmits != null) { pendingEmits.add(addressedTuple); } return false; }