Aitozi commented on a change in pull request #8431: [FLINK-12480] Introduce mailbox to StreamTask main-loop URL: https://github.com/apache/flink/pull/8431#discussion_r285896856
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxImpl.java ########## @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.tasks.mailbox; + +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nonnull; +import javax.annotation.concurrent.GuardedBy; + +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Implementation of {@link Mailbox} inspired by {@link java.util.concurrent.ArrayBlockingQueue} and tailored towards + * our use case with multiple writers, single reader and volatile reads instead of lock & read on {@link #count}. + */ +public class MailboxImpl implements Mailbox { + + /** + * The enqueued letters. + */ + @GuardedBy("lock") + private final Runnable[] ringBuffer; + + /** + * Lock for all concurrent ops. + */ + private final ReentrantLock lock; + + /** + * Condition that is triggered when the buffer is no longer empty. + */ + @GuardedBy("lock") + private final Condition notEmpty; + + /** + * Condition that is triggered when the buffer is no longer full. + */ + @GuardedBy("lock") + private final Condition notFull; + + /** + * Index of the ring buffer head. + */ + @GuardedBy("lock") + private int headIndex; + + /** + * Index of the ring buffer tail. + */ + @GuardedBy("lock") + private int tailIndex; + + /** + * Number of letters in the mailbox. + */ + @GuardedBy("lock") + private volatile int count; + + /** + * A mask to wrap around the indexes of the ring buffer. We use this to avoid ifs or modulo ops. + */ + private final int moduloMask; + + public MailboxImpl() { + this(6); // 2^6 = 64 + } + + public MailboxImpl(int capacityPow2) { + final int capacity = 1 << capacityPow2; + Preconditions.checkState(capacity > 0); + this.moduloMask = capacity - 1; + this.ringBuffer = new Runnable[capacity]; + this.lock = new ReentrantLock(); + this.notEmpty = lock.newCondition(); + this.notFull = lock.newCondition(); + } + + @Override + public boolean hasMail() { + return count > 0; + } + + @Override + public Runnable tryTakeMail() { + final ReentrantLock lock = this.lock; + lock.lock(); + try { + return isEmpty() ? null : takeInternal(); + } finally { + lock.unlock(); + } + } + + @Nonnull + @Override + public Runnable takeMail() throws InterruptedException { + final ReentrantLock lock = this.lock; + lock.lockInterruptibly(); + try { + while (isEmpty()) { + notEmpty.await(); + } + return takeInternal(); + } finally { + lock.unlock(); + } + } + + @Override + public void waitUntilHasMail() throws InterruptedException { + final ReentrantLock lock = this.lock; + lock.lockInterruptibly(); + try { + while (isEmpty()) { + notEmpty.await(); + } + } finally { + lock.unlock(); + } + } + + //------------------------------------------------------------------------------------------------------------------ + + @Override + public boolean tryPutMail(@Nonnull Runnable letter) { + final ReentrantLock lock = this.lock; + lock.lock(); + try { + if (isFull()) { Review comment: > @Aitozi thanks for thanking this through, here is how I concluded that this is not problematic: > > * If you check the code, checkpoints and timers are even currently not invoked from the "main thread", but via executors. > * Right now both can also block on their attempt to acquire the checkpointing lock, which is very similar in effect to blocking on the mailbox. > * We have non-blocking ways of emitting letters to the mailbox, if the need arises. > > Does this resolve your concerns? sorry i do not refresh to see your reply just now, I will check the code again. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services