Skip to content

Commit

Permalink
All Translog inner closes should happen after tragedy exception is set (
Browse files Browse the repository at this point in the history
#32674)

We faced with the nasty race condition. See #32526
InternalEngine.failOnTragic method has thrown AssertionError.
If you carefully look at if branches in this method, you will spot that its only possible, if either Lucene IndexWriterhas closed from inside or Translog, has closed from inside, but tragedy exception is not set.
For now, let us concentrate on the Translog class.
We found out that there are two methods in Translog - namely rollGeneration and trimOperations that are closing Translog in case of Exception without tragedy exception being set.
This commit fixes these 2 methods. To fix it, we pull tragedyException from TranslogWriter up-to Translog class, because in these 2 methods IndexWriter could be innocent, but still Translog needs to be closed. Also, tragedyException is wrapped with TragicExceptionHolder to reuse CAS/addSuppresed functionality in Translog and TranslogWriter.
Also to protect us in the future and make sure close method is never called from inside Translog special assertion examining stack trace is added. Since we're still targeting Java 8 for runtime - no StackWalker API is used in the implementation.
In the stack-trace checking method, we're considering inner caller not only Translog methods but Translog child classes methods as well. It does mean that Translog is meant for extending it, but it's needed to be able to test this method.

Closes #32526

(cherry picked from commit 0749b18)
  • Loading branch information
Andrey Ershov committed Aug 21, 2018
1 parent 89ebe6e commit 3860ae9
Show file tree
Hide file tree
Showing 5 changed files with 136 additions and 35 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.translog;

import java.util.concurrent.atomic.AtomicReference;

public class TragicExceptionHolder {
private final AtomicReference<Exception> tragedy = new AtomicReference<>();

/**
* Sets the tragic exception or if the tragic exception is already set adds passed exception as suppressed exception
* @param ex tragic exception to set
*/
public void setTragicException(Exception ex) {
assert ex != null;
if (tragedy.compareAndSet(null, ex) == false) {
if (tragedy.get() != ex) { // to ensure there is no self-suppression
tragedy.get().addSuppressed(ex);
}
}
}

public Exception get() {
return tragedy.get();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@
import java.util.function.LongSupplier;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;

/**
Expand Down Expand Up @@ -119,6 +120,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
private final Path location;
private TranslogWriter current;

protected final TragicExceptionHolder tragedy = new TragicExceptionHolder();
private final AtomicBoolean closed = new AtomicBoolean();
private final TranslogConfig config;
private final LongSupplier globalCheckpointSupplier;
Expand Down Expand Up @@ -312,8 +314,28 @@ public boolean isOpen() {
return closed.get() == false;
}

private static boolean calledFromOutsideOrViaTragedyClose() {
List<StackTraceElement> frames = Stream.of(Thread.currentThread().getStackTrace()).
skip(3). //skip getStackTrace, current method and close method frames
limit(10). //limit depth of analysis to 10 frames, it should be enough to catch closing with, e.g. IOUtils
filter(f ->
{
try {
return Translog.class.isAssignableFrom(Class.forName(f.getClassName()));
} catch (Exception ignored) {
return false;
}
}
). //find all inner callers including Translog subclasses
collect(Collectors.toList());
//the list of inner callers should be either empty or should contain closeOnTragicEvent method
return frames.isEmpty() || frames.stream().anyMatch(f -> f.getMethodName().equals("closeOnTragicEvent"));
}

@Override
public void close() throws IOException {
assert calledFromOutsideOrViaTragedyClose() :
"Translog.close method is called from inside Translog, but not via closeOnTragicEvent method";
if (closed.compareAndSet(false, true)) {
try (ReleasableLock lock = writeLock.acquire()) {
try {
Expand Down Expand Up @@ -464,7 +486,7 @@ TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, lon
getChannelFactory(),
config.getBufferSize(),
initialMinTranslogGen, initialGlobalCheckpoint,
globalCheckpointSupplier, this::getMinFileGeneration, primaryTermSupplier.getAsLong());
globalCheckpointSupplier, this::getMinFileGeneration, primaryTermSupplier.getAsLong(), tragedy);
} catch (final IOException e) {
throw new TranslogException(shardId, "failed to create new translog file", e);
}
Expand Down Expand Up @@ -728,7 +750,8 @@ public void trimOperations(long belowTerm, long aboveSeqNo) throws IOException {
}
} catch (IOException e) {
IOUtils.closeWhileHandlingException(newReaders);
close();
tragedy.setTragicException(e);
closeOnTragicEvent(e);
throw e;
}

Expand Down Expand Up @@ -781,10 +804,10 @@ public boolean ensureSynced(Stream<Location> locations) throws IOException {
*
* @param ex if an exception occurs closing the translog, it will be suppressed into the provided exception
*/
private void closeOnTragicEvent(final Exception ex) {
protected void closeOnTragicEvent(final Exception ex) {
// we can not hold a read lock here because closing will attempt to obtain a write lock and that would result in self-deadlock
assert readLock.isHeldByCurrentThread() == false : Thread.currentThread().getName();
if (current.getTragicException() != null) {
if (tragedy.get() != null) {
try {
close();
} catch (final AlreadyClosedException inner) {
Expand Down Expand Up @@ -1610,7 +1633,8 @@ public void rollGeneration() throws IOException {
current = createWriter(current.getGeneration() + 1);
logger.trace("current translog set to [{}]", current.getGeneration());
} catch (final Exception e) {
IOUtils.closeWhileHandlingException(this); // tragic event
tragedy.setTragicException(e);
closeOnTragicEvent(e);
throw e;
}
}
Expand Down Expand Up @@ -1723,7 +1747,7 @@ long getFirstOperationPosition() { // for testing

private void ensureOpen() {
if (closed.get()) {
throw new AlreadyClosedException("translog is already closed", current.getTragicException());
throw new AlreadyClosedException("translog is already closed", tragedy.get());
}
}

Expand All @@ -1737,7 +1761,7 @@ ChannelFactory getChannelFactory() {
* Otherwise (no tragic exception has occurred) it returns null.
*/
public Exception getTragicException() {
return current.getTragicException();
return tragedy.get();
}

/** Reads and returns the current checkpoint */
Expand Down Expand Up @@ -1820,8 +1844,8 @@ static String createEmptyTranslog(Path location, long initialGlobalCheckpoint, S
final String translogUUID = UUIDs.randomBase64UUID();
TranslogWriter writer = TranslogWriter.create(shardId, translogUUID, 1, location.resolve(getFilename(1)), channelFactory,
new ByteSizeValue(10), 1, initialGlobalCheckpoint,
() -> { throw new UnsupportedOperationException(); }, () -> { throw new UnsupportedOperationException(); }, primaryTerm
);
() -> { throw new UnsupportedOperationException(); }, () -> { throw new UnsupportedOperationException(); }, primaryTerm,
new TragicExceptionHolder());
writer.close();
return translogUUID;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable {
/* the number of translog operations written to this file */
private volatile int operationCounter;
/* if we hit an exception that we can't recover from we assign it to this var and ship it with every AlreadyClosedException we throw */
private volatile Exception tragedy;
private final TragicExceptionHolder tragedy;
/* A buffered outputstream what writes to the writers channel */
private final OutputStream outputStream;
/* the total offset of this file including the bytes written to the file as well as into the buffer */
Expand All @@ -77,7 +77,10 @@ private TranslogWriter(
final FileChannel channel,
final Path path,
final ByteSizeValue bufferSize,
final LongSupplier globalCheckpointSupplier, LongSupplier minTranslogGenerationSupplier, TranslogHeader header) throws IOException {
final LongSupplier globalCheckpointSupplier, LongSupplier minTranslogGenerationSupplier, TranslogHeader header,
TragicExceptionHolder tragedy)
throws
IOException {
super(initialCheckpoint.generation, channel, path, header);
assert initialCheckpoint.offset == channel.position() :
"initial checkpoint offset [" + initialCheckpoint.offset + "] is different than current channel position ["
Expand All @@ -95,12 +98,13 @@ private TranslogWriter(
assert initialCheckpoint.trimmedAboveSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO : initialCheckpoint.trimmedAboveSeqNo;
this.globalCheckpointSupplier = globalCheckpointSupplier;
this.seenSequenceNumbers = Assertions.ENABLED ? new HashMap<>() : null;
this.tragedy = tragedy;
}

public static TranslogWriter create(ShardId shardId, String translogUUID, long fileGeneration, Path file, ChannelFactory channelFactory,
ByteSizeValue bufferSize, final long initialMinTranslogGen, long initialGlobalCheckpoint,
final LongSupplier globalCheckpointSupplier, final LongSupplier minTranslogGenerationSupplier,
final long primaryTerm)
final long primaryTerm, TragicExceptionHolder tragedy)
throws IOException {
final FileChannel channel = channelFactory.open(file);
try {
Expand All @@ -121,7 +125,7 @@ public static TranslogWriter create(ShardId shardId, String translogUUID, long f
writerGlobalCheckpointSupplier = globalCheckpointSupplier;
}
return new TranslogWriter(channelFactory, shardId, checkpoint, channel, file, bufferSize,
writerGlobalCheckpointSupplier, minTranslogGenerationSupplier, header);
writerGlobalCheckpointSupplier, minTranslogGenerationSupplier, header, tragedy);
} catch (Exception exception) {
// if we fail to bake the file-generation into the checkpoint we stick with the file and once we recover and that
// file exists we remove it. We only apply this logic to the checkpoint.generation+1 any other file with a higher generation is an error condition
Expand All @@ -130,24 +134,8 @@ public static TranslogWriter create(ShardId shardId, String translogUUID, long f
}
}

/**
* If this {@code TranslogWriter} was closed as a side-effect of a tragic exception,
* e.g. disk full while flushing a new segment, this returns the root cause exception.
* Otherwise (no tragic exception has occurred) it returns null.
*/
public Exception getTragicException() {
return tragedy;
}

private synchronized void closeWithTragicEvent(final Exception ex) {
assert ex != null;
if (tragedy == null) {
tragedy = ex;
} else if (tragedy != ex) {
// it should be safe to call closeWithTragicEvents on multiple layers without
// worrying about self suppression.
tragedy.addSuppressed(ex);
}
tragedy.setTragicException(ex);
try {
close();
} catch (final IOException | RuntimeException e) {
Expand Down Expand Up @@ -310,7 +298,8 @@ public TranslogReader closeIntoReader() throws IOException {
if (closed.compareAndSet(false, true)) {
return new TranslogReader(getLastSyncedCheckpoint(), channel, path, header);
} else {
throw new AlreadyClosedException("translog [" + getGeneration() + "] is already closed (path [" + path + "]", tragedy);
throw new AlreadyClosedException("translog [" + getGeneration() + "] is already closed (path [" + path + "]",
tragedy.get());
}
}
}
Expand Down Expand Up @@ -420,7 +409,7 @@ Checkpoint getLastSyncedCheckpoint() {

protected final void ensureOpen() {
if (isClosed()) {
throw new AlreadyClosedException("translog [" + getGeneration() + "] is already closed", tragedy);
throw new AlreadyClosedException("translog [" + getGeneration() + "] is already closed", tragedy.get());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -171,7 +171,7 @@ private Tuple<List<TranslogReader>, TranslogWriter> createReadersAndWriter(final
}
writer = TranslogWriter.create(new ShardId("index", "uuid", 0), translogUUID, gen,
tempDir.resolve(Translog.getFilename(gen)), FileChannel::open, TranslogConfig.DEFAULT_BUFFER_SIZE, 1L, 1L, () -> 1L,
() -> 1L, randomNonNegativeLong());
() -> 1L, randomNonNegativeLong(), new TragicExceptionHolder());
writer = Mockito.spy(writer);
Mockito.doReturn(now - (numberOfReaders - gen + 1) * 1000).when(writer).getLastModifiedTime();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.Version;
import org.elasticsearch.Assertions;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.Strings;
Expand Down Expand Up @@ -108,6 +109,7 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.LongSupplier;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.LongStream;
Expand Down Expand Up @@ -1673,7 +1675,7 @@ public void testRandomExceptionsOnTrimOperations( ) throws Exception {
}

assertThat(expectedException, is(not(nullValue())));

assertThat(failableTLog.getTragicException(), equalTo(expectedException));
assertThat(fileChannels, is(not(empty())));
assertThat("all file channels have to be closed",
fileChannels.stream().filter(f -> f.isOpen()).findFirst().isPresent(), is(false));
Expand Down Expand Up @@ -2524,11 +2526,13 @@ public void testWithRandomException() throws IOException {
syncedDocs.addAll(unsynced);
unsynced.clear();
} catch (TranslogException | MockDirectoryWrapper.FakeIOException ex) {
// fair enough
assertEquals(failableTLog.getTragicException(), ex);
} catch (IOException ex) {
assertEquals(ex.getMessage(), "__FAKE__ no space left on device");
assertEquals(failableTLog.getTragicException(), ex);
} catch (RuntimeException ex) {
assertEquals(ex.getMessage(), "simulated");
assertEquals(failableTLog.getTragicException(), ex);
} finally {
Checkpoint checkpoint = Translog.readCheckpoint(config.getTranslogPath());
if (checkpoint.numOps == unsynced.size() + syncedDocs.size()) {
Expand Down Expand Up @@ -2967,6 +2971,47 @@ public void testCloseSnapshotTwice() throws Exception {
}
}

// close method should never be called directly from Translog (the only exception is closeOnTragicEvent)
public void testTranslogCloseInvariant() throws IOException {
assumeTrue("test only works with assertions enabled", Assertions.ENABLED);
class MisbehavingTranslog extends Translog {
MisbehavingTranslog(TranslogConfig config, String translogUUID, TranslogDeletionPolicy deletionPolicy, LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier) throws IOException {
super(config, translogUUID, deletionPolicy, globalCheckpointSupplier, primaryTermSupplier);
}

void callCloseDirectly() throws IOException {
close();
}

void callCloseUsingIOUtilsWithExceptionHandling() {
IOUtils.closeWhileHandlingException(this);
}

void callCloseUsingIOUtils() throws IOException {
IOUtils.close(this);
}

void callCloseOnTragicEvent() {
Exception e = new Exception("test tragic exception");
tragedy.setTragicException(e);
closeOnTragicEvent(e);
}
}


globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
Path path = createTempDir();
final TranslogConfig translogConfig = getTranslogConfig(path);
final TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(translogConfig.getIndexSettings());
final String translogUUID = Translog.createEmptyTranslog(path, SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get());
MisbehavingTranslog misbehavingTranslog = new MisbehavingTranslog(translogConfig, translogUUID, deletionPolicy, () -> globalCheckpoint.get(), primaryTerm::get);

expectThrows(AssertionError.class, () -> misbehavingTranslog.callCloseDirectly());
expectThrows(AssertionError.class, () -> misbehavingTranslog.callCloseUsingIOUtils());
expectThrows(AssertionError.class, () -> misbehavingTranslog.callCloseUsingIOUtilsWithExceptionHandling());
misbehavingTranslog.callCloseOnTragicEvent();
}

static class SortedSnapshot implements Translog.Snapshot {
private final Translog.Snapshot snapshot;
private List<Translog.Operation> operations = null;
Expand Down

0 comments on commit 3860ae9

Please sign in to comment.