[DLOG] Avoid double read in readahead

There was a bug in the ReadAheadReader whereby, if it was in a paused
state and the last segment it had read had been closed, then when a
writer started writing new segments, the ReadAheadReader would issue a
double read. This would end up triggering an error when it got to the
end of the segment as the issued one of the issued reads would then be
cancelled when the segment was closed.

This fix adds an explicit state machine to the reader and moves all
state transitions to run on the ordered executor, so only one read can
ever be issued at a time.

Reviewers: Enrico Olivelli <eolivelli@gmail.com>, Sijie Guo <sijie@apache.org>

This closes #1973 from ivankelly/dlog-reader-crash-master
(cherry picked from commit db5d7e7cf30d58c9c463ce29c7d7900115717206)

Signed-off-by: Ivan Kelly <ivank@apache.org>
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
index c9bca44..ccd42eb 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
@@ -67,6 +67,15 @@
 
     private static final Logger logger = LoggerFactory.getLogger(ReadAheadEntryReader.class);
 
+    private enum State {
+        IDLE,
+        READING,
+        PAUSED,
+        CLOSED,
+        ERROR
+    }
+    private State state = State.IDLE;
+
     //
     // Static Functions
     //
@@ -460,6 +469,11 @@
     }
 
     private void unsafeAsyncClose(CompletableFuture<Void> closePromise) {
+        if (logger.isDebugEnabled()) {
+            logger.debug("[{}][state:{}] Closing read ahead", streamName, state);
+        }
+        state = State.CLOSED;
+
         List<CompletableFuture<Void>> closeFutures = Lists.newArrayListWithExpectedSize(
                 segmentReaders.size() + segmentReadersToClose.size() + 1);
         if (null != currentSegmentReader) {
@@ -512,6 +526,16 @@
         }
         // the exception is set and notify the state change
         notifyStateChangeOnFailure(cause);
+
+        orderedSubmit(new CloseableRunnable() {
+                @Override
+                public void safeRun() {
+                    if (logger.isDebugEnabled()) {
+                        logger.debug("[{}][state:{}] Read ahead errored", streamName, state);
+                    }
+                    state = State.ERROR;
+                }
+            });
     }
 
     void checkLastException() throws IOException {
@@ -571,15 +595,15 @@
         }
         // notify on data available
         notifyStateChangeOnSuccess();
-        if (entryQueue.size() >= maxCachedEntries) {
-            pauseReadAheadOnCacheFull();
-        } else {
-            scheduleReadNext();
-        }
+
+        completeRead();
+        scheduleRead();
     }
 
     @Override
     public void onFailure(Throwable cause) {
+        completeRead();
+
         if (cause instanceof EndOfLogSegmentException) {
             // we reach end of the log segment
             moveToNextLogSegment();
@@ -592,24 +616,6 @@
         }
     }
 
-    private synchronized void invokeReadAhead() {
-        if (readAheadPaused) {
-            scheduleReadNext();
-            readAheadPaused = false;
-        }
-    }
-
-    private synchronized void pauseReadAheadOnCacheFull() {
-        this.readAheadPaused = true;
-        if (!isCacheFull()) {
-            invokeReadAhead();
-        }
-    }
-
-    private synchronized void pauseReadAheadOnNoMoreLogSegments() {
-        this.readAheadPaused = true;
-    }
-
     //
     // Cache Related Methods
     //
@@ -630,7 +636,7 @@
         } finally {
             // resume readahead if the cache becomes empty
             if (null != entry && !isCacheFull()) {
-                invokeReadAhead();
+                scheduleRead();
             }
         }
     }
@@ -784,7 +790,7 @@
             unsafeMoveToNextLogSegment();
         }
         // resume readahead if necessary
-        invokeReadAhead();
+        scheduleRead();
     }
 
     /**
@@ -839,7 +845,9 @@
         currentSegmentReader.openReader();
         currentSegmentReader.startRead();
         currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
-        unsafeReadNext(currentSegmentReader);
+
+        scheduleRead();
+
         if (!segmentReaders.isEmpty()) {
             for (SegmentReader reader : segmentReaders) {
                 reader.openReader();
@@ -921,7 +929,6 @@
             currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
             nextSegmentReader = null;
             // start reading
-            unsafeReadNext(currentSegmentReader);
             unsafePrefetchNextSegment(true);
             hasSegmentToRead = true;
         } else {
@@ -931,7 +938,6 @@
                 logger.debug("move to read segment {}", currentSegmentReader.getSegment());
                 currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
                 nextSegmentReader = null;
-                unsafeReadNext(currentSegmentReader);
                 unsafePrefetchNextSegment(true);
                 hasSegmentToRead = true;
             }
@@ -942,25 +948,60 @@
                         readHandler.getFullyQualifiedName());
                 isCatchingUp = false;
             }
-            pauseReadAheadOnNoMoreLogSegments();
         }
+
+        scheduleRead();
     }
 
-    void scheduleReadNext() {
+    void completeRead() {
         orderedSubmit(new CloseableRunnable() {
-            @Override
-            public void safeRun() {
-                if (null == currentSegmentReader) {
-                    pauseReadAheadOnNoMoreLogSegments();
-                    return;
+                @Override
+                public void safeRun() {
+                    if (logger.isDebugEnabled()) {
+                        logger.debug("[{}][state:{}] Read completed", streamName, state);
+                    }
+                    if (state == State.READING) {
+                        state = State.IDLE;
+                    }
                 }
-                unsafeReadNext(currentSegmentReader);
-            }
-        });
+            });
     }
 
-    private void unsafeReadNext(SegmentReader reader) {
-        reader.readNext().whenComplete(this);
+    void scheduleRead() {
+        orderedSubmit(new CloseableRunnable() {
+                @Override
+                public void safeRun() {
+
+                    boolean cacheFull = isCacheFull();
+                    SegmentReader reader = currentSegmentReader;
+                    boolean hasMoreSegments = reader != null;
+                    if (logger.isDebugEnabled()) {
+                        logger.debug("[{}][state:{}] scheduling read, cacheFull {}, hasMoreSegments {}",
+                                     streamName, state, cacheFull, hasMoreSegments);
+                    }
+                    switch (state) {
+                    case IDLE:
+                        if (cacheFull || !hasMoreSegments) {
+                            state = State.PAUSED;
+                        } else {
+                            reader.readNext().whenComplete(ReadAheadEntryReader.this);
+                            state = State.READING;
+                        }
+                        break;
+                    case PAUSED:
+                        if (!cacheFull && hasMoreSegments) {
+                            reader.readNext().whenComplete(ReadAheadEntryReader.this);
+                            state = State.READING;
+                        }
+                        break;
+                    case READING:
+                    case ERROR:
+                    case CLOSED:
+                        // do nothing
+                        break;
+                    }
+                }
+            });
     }
 
     @Override
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestCancelledRead.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestCancelledRead.java
new file mode 100644
index 0000000..40cf632
--- /dev/null
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestCancelledRead.java
@@ -0,0 +1,76 @@
+/**
+ * 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.distributedlog;
+
+import static org.junit.Assert.assertNotNull;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.api.LogWriter;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Cases for RollLogSegments.
+ */
+public class TestCancelledRead extends TestDistributedLogBase {
+    private static final Logger logger = LoggerFactory.getLogger(TestRollLogSegments.class);
+
+    @Test(timeout = 600000)
+    public void testWritingAndTailing() throws Exception {
+        String name = "writing-and-tailing";
+        DistributedLogConfiguration conf = new DistributedLogConfiguration()
+            .setReadAheadWaitTime(5000)
+            .setOutputBufferSize(0)
+            .setCreateStreamIfNotExists(true)
+            .setImmediateFlushEnabled(true)
+            .setFailFastOnStreamNotReady(true)
+            .setPeriodicFlushFrequencyMilliSeconds(0)
+            .setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE)
+            .setEnableReadAhead(false)
+            .setLogSegmentRollingIntervalMinutes(0);
+
+        CompletableFuture<Void> f = new CompletableFuture<>();
+        long entryId = 0;
+
+        try (BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
+             LogWriter writer = dlm.startLogSegmentNonPartitioned()) {
+            entryId++;
+            writer.write(DLMTestUtil.getLogRecordInstance(entryId, 100000));
+        }
+
+        try (BKDistributedLogManager dlmReader = (BKDistributedLogManager) createNewDLM(conf, name)) {
+            BKAsyncLogReader reader = (BKAsyncLogReader) dlmReader.getAsyncLogReader(DLSN.InitialDLSN);
+
+            assertNotNull(reader.readNext().get());
+
+            conf.setMaxLogSegmentBytes(1000);
+            try (BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
+                 LogWriter writer = dlm.startLogSegmentNonPartitioned()) {
+                for (int i = 0; i < 100; i++) {
+                    entryId++;
+                    writer.write(DLMTestUtil.getLogRecordInstance(entryId, 100));
+
+                    assertNotNull(reader.readNext().get());
+                }
+            } finally {
+                reader.asyncClose().get();
+            }
+        }
+    }
+}