@Override public Watermark getWatermark() { return (Watermark) getStreamElement(); }
/** * True if the stream element queue entry has been completed; otherwise false. * * @return True if the stream element queue entry has been completed; otherwise false. */ public boolean isDone() { return getFuture().isDone(); }
@Override public <T> void put(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException { lock.lockInterruptibly(); try { while (numberEntries >= capacity) { notFull.await(); } addEntry(streamElementQueueEntry); } finally { lock.unlock(); } }
private void output(AsyncResult asyncResult) throws InterruptedException { if (asyncResult.isWatermark()) { synchronized (checkpointLock) { AsyncWatermarkResult asyncWatermarkResult = asyncResult.asWatermark(); output.emitWatermark(asyncWatermarkResult.getWatermark()); streamElementQueue.poll(); AsyncCollectionResult<OUT> streamRecordResult = asyncResult.asResultCollection(); if (streamRecordResult.hasTimestamp()) { timestampedCollector.setAbsoluteTimestamp(streamRecordResult.getTimestamp()); } else { timestampedCollector.eraseTimestamp(); Collection<OUT> resultCollection = streamRecordResult.get(); streamElementQueue.poll();
@Test public void testPoll() throws InterruptedException { OperatorActions operatorActions = mock(OperatorActions.class); StreamElementQueue queue = createStreamElementQueue(2, operatorActions); WatermarkQueueEntry watermarkQueueEntry = new WatermarkQueueEntry(new Watermark(0L)); StreamRecordQueueEntry<Integer> streamRecordQueueEntry = new StreamRecordQueueEntry<>(new StreamRecord<>(42, 1L)); queue.put(watermarkQueueEntry); queue.put(streamRecordQueueEntry); Assert.assertEquals(watermarkQueueEntry, queue.peekBlockingly()); Assert.assertEquals(2, queue.size()); Assert.assertEquals(watermarkQueueEntry, queue.poll()); Assert.assertEquals(1, queue.size()); streamRecordQueueEntry.complete(Collections.<Integer>emptyList()); Assert.assertEquals(streamRecordQueueEntry, queue.poll()); Assert.assertEquals(0, queue.size()); Assert.assertTrue(queue.isEmpty()); verify(operatorActions, never()).failOperator(any(Exception.class)); }
@Test public void testPut() throws InterruptedException { OperatorActions operatorActions = mock(OperatorActions.class); StreamElementQueue queue = createStreamElementQueue(2, operatorActions); final Watermark watermark = new Watermark(0L); final StreamRecord<Integer> streamRecord = new StreamRecord<>(42, 1L); final Watermark nextWatermark = new Watermark(2L); final WatermarkQueueEntry watermarkQueueEntry = new WatermarkQueueEntry(watermark); final StreamRecordQueueEntry<Integer> streamRecordQueueEntry = new StreamRecordQueueEntry<>(streamRecord); queue.put(watermarkQueueEntry); queue.put(streamRecordQueueEntry); Assert.assertEquals(2, queue.size()); Assert.assertFalse(queue.tryPut(new WatermarkQueueEntry(nextWatermark))); Collection<StreamElementQueueEntry<?>> actualValues = queue.values(); List<StreamElementQueueEntry<?>> expectedValues = Arrays.asList(watermarkQueueEntry, streamRecordQueueEntry); Assert.assertEquals(expectedValues, actualValues); verify(operatorActions, never()).failOperator(any(Exception.class)); }
public void testBlockingPut() throws Exception { OperatorActions operatorActions = mock(OperatorActions.class); final StreamElementQueue queue = createStreamElementQueue(1, operatorActions); StreamRecordQueueEntry<Integer> streamRecordQueueEntry = new StreamRecordQueueEntry<>(new StreamRecord<>(42, 0L)); final StreamRecordQueueEntry<Integer> streamRecordQueueEntry2 = new StreamRecordQueueEntry<>(new StreamRecord<>(43, 1L)); queue.put(streamRecordQueueEntry); Assert.assertEquals(1, queue.size()); queue.put(streamRecordQueueEntry2); } catch (InterruptedException e) { throw new CompletionException(e); streamRecordQueueEntry.complete(Collections.<Integer>emptyList()); Assert.assertEquals(streamRecordQueueEntry, queue.poll());
public StreamElementQueue createStreamElementQueue(int capacity, OperatorActions operatorActions) { switch (streamElementQueueType) { case OrderedStreamElementQueueType: return new OrderedStreamElementQueue(capacity, executor, operatorActions); case UnorderedStreamElementQueueType: return new UnorderedStreamElementQueue(capacity, executor, operatorActions); default: throw new IllegalStateException("Unknown stream element queue type: " + streamElementQueueType); } }
@Override public void snapshotState(StateSnapshotContext context) throws Exception { super.snapshotState(context); ListState<StreamElement> partitionableState = getOperatorStateBackend().getListState(new ListStateDescriptor<>(STATE_NAME, inStreamElementSerializer)); partitionableState.clear(); Collection<StreamElementQueueEntry<?>> values = queue.values(); try { for (StreamElementQueueEntry<?> value : values) { partitionableState.add(value.getStreamElement()); } // add the pending stream element queue entry if the stream element queue is currently full if (pendingStreamElementQueueEntry != null) { partitionableState.add(pendingStreamElementQueueEntry.getStreamElement()); } } catch (Exception e) { partitionableState.clear(); throw new Exception("Could not add stream element queue entries to operator state " + "backend of operator " + getOperatorName() + '.', e); } }
/** * Add the given {@link StreamElementQueueEntry} to the queue. Additionally, this method * registers a onComplete callback which is triggered once the given queue entry is completed. * * @param streamElementQueueEntry to be inserted * @param <T> Type of the stream element queue entry's result */ private <T> void addEntry(StreamElementQueueEntry<T> streamElementQueueEntry) { assert(lock.isHeldByCurrentThread()); queue.addLast(streamElementQueueEntry); streamElementQueueEntry.onComplete( (StreamElementQueueEntry<T> value) -> { try { onCompleteHandler(value); } catch (InterruptedException e) { // we got interrupted. This indicates a shutdown of the executor LOG.debug("AsyncBufferEntry could not be properly completed because the " + "executor thread has been interrupted.", e); } catch (Throwable t) { operatorActions.failOperator(new Exception("Could not complete the " + "stream element queue entry: " + value + '.', t)); } }, executor); }
@Override public AsyncResult poll() throws InterruptedException { lock.lockInterruptibly(); try { while (queue.isEmpty() || !queue.peek().isDone()) { headIsCompleted.await(); } notFull.signalAll(); LOG.debug("Polled head element from ordered stream element queue. New filling degree " + "({}/{}).", queue.size() - 1, capacity); return queue.poll(); } finally { lock.unlock(); } }
@Override public void processWatermark(Watermark mark) throws Exception { WatermarkQueueEntry watermarkBufferEntry = new WatermarkQueueEntry(mark); addAsyncBufferEntry(watermarkBufferEntry); }
/** * Add the given stream element queue entry to the operator's stream element queue. This * operation blocks until the element has been added. * * <p>For that it tries to put the element into the queue and if not successful then it waits on * the checkpointing lock. The checkpointing lock is also used by the {@link Emitter} to output * elements. The emitter is also responsible for notifying this method if the queue has capacity * left again, by calling notifyAll on the checkpointing lock. * * @param streamElementQueueEntry to add to the operator's queue * @param <T> Type of the stream element queue entry's result * @throws InterruptedException if the current thread has been interrupted */ private <T> void addAsyncBufferEntry(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException { assert(Thread.holdsLock(checkpointingLock)); pendingStreamElementQueueEntry = streamElementQueueEntry; while (!queue.tryPut(streamElementQueueEntry)) { // we wait for the emitter to notify us if the queue has space left again checkpointingLock.wait(); } pendingStreamElementQueueEntry = null; }
@Override public void run() { try { while (running) { LOG.debug("Wait for next completed async stream element result."); AsyncResult streamElementEntry = streamElementQueue.peekBlockingly(); output(streamElementEntry); } } catch (InterruptedException e) { if (running) { operatorActions.failOperator(e); } else { // Thread got interrupted which means that it should shut down LOG.debug("Emitter thread got interrupted, shutting down."); } } catch (Throwable t) { operatorActions.failOperator(new Exception("AsyncWaitOperator's emitter caught an " + "unexpected throwable.", t)); } }
@Override public <T> void put(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException { lock.lockInterruptibly(); try { while (queue.size() >= capacity) { notFull.await(); } addEntry(streamElementQueueEntry); } finally { lock.unlock(); } }
/** * Check if the completed {@link StreamElementQueueEntry} is the current head. If this is the * case, then notify the consumer thread about a new consumable entry. * * @param streamElementQueueEntry which has been completed * @throws InterruptedException if the current thread is interrupted */ private void onCompleteHandler(StreamElementQueueEntry<?> streamElementQueueEntry) throws InterruptedException { lock.lockInterruptibly(); try { if (!queue.isEmpty() && queue.peek().isDone()) { LOG.debug("Signal ordered stream element queue has completed head element."); headIsCompleted.signalAll(); } } finally { lock.unlock(); } } }
/** * Register the given complete function to be called once this queue entry has been completed. * * @param completeFunction to call when the queue entry has been completed * @param executor to run the complete function */ public void onComplete( final Consumer<StreamElementQueueEntry<T>> completeFunction, Executor executor) { final StreamElementQueueEntry<T> thisReference = this; getFuture().whenCompleteAsync( // call the complete function for normal completion as well as exceptional completion // see FLINK-6435 (value, throwable) -> completeFunction.accept(thisReference), executor); }
@Override public <T> boolean tryPut(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException { lock.lockInterruptibly(); try { if (numberEntries < capacity) { addEntry(streamElementQueueEntry); LOG.debug("Put element into unordered stream element queue. New filling degree " + "({}/{}).", numberEntries, capacity); return true; } else { LOG.debug("Failed to put element into unordered stream element queue because it " + "was full ({}/{}).", numberEntries, capacity); return false; } } finally { lock.unlock(); } }
@Override public <T> boolean tryPut(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException { lock.lockInterruptibly(); try { if (queue.size() < capacity) { addEntry(streamElementQueueEntry); LOG.debug("Put element into ordered stream element queue. New filling degree " + "({}/{}).", queue.size(), capacity); return true; } else { LOG.debug("Failed to put element into ordered stream element queue because it " + "was full ({}/{}).", queue.size(), capacity); return false; } } finally { lock.unlock(); } }
@Override public AsyncResult peekBlockingly() throws InterruptedException { lock.lockInterruptibly(); try { while (queue.isEmpty() || !queue.peek().isDone()) { headIsCompleted.await(); } LOG.debug("Peeked head element from ordered stream element queue with filling degree " + "({}/{}).", queue.size(), capacity); return queue.peek(); } finally { lock.unlock(); } }