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