-
Notifications
You must be signed in to change notification settings - Fork 25.6k
All Translog inner closes should happen after tragedy exception is set #32674
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 10 commits
c330ec8
bc7187a
7d1d550
e01f31a
75ba167
941ccc9
b76b295
339aaeb
f42ad1d
b06f249
7fef412
87b5211
6259de5
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| 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<>(); | ||
|
|
||
| public boolean setTragicException(Exception ex){ | ||
| assert ex != null; | ||
| if (tragedy.compareAndSet(null, ex)) { | ||
| tragedy.set(ex); | ||
| return true; | ||
| } else { | ||
| if (tragedy.get() != ex) { //to ensure there is no self-suppression | ||
|
||
| tragedy.get().addSuppressed(ex); | ||
| } | ||
| return false; | ||
| } | ||
| } | ||
|
|
||
| public Exception get() { | ||
| return tragedy.get(); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -51,7 +51,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 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 */ | ||
|
|
@@ -76,7 +76,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 [" | ||
|
|
@@ -94,12 +97,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 { | ||
|
|
@@ -120,7 +124,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 | ||
|
|
@@ -129,28 +133,13 @@ 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) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this case got lost |
||
| // it should be safe to call closeWithTragicEvents on multiple layers without | ||
| // worrying about self suppression. | ||
| tragedy.addSuppressed(ex); | ||
| } | ||
| try { | ||
| close(); | ||
| } catch (final IOException | RuntimeException e) { | ||
| ex.addSuppressed(e); | ||
| if (tragedy.setTragicException(ex)) { | ||
|
||
| try { | ||
| close(); | ||
| } catch (final IOException | RuntimeException e) { | ||
| ex.addSuppressed(e); | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -296,7 +285,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()); | ||
| } | ||
| } | ||
| } | ||
|
|
@@ -406,7 +396,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()); | ||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -108,6 +108,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; | ||
|
|
@@ -1655,7 +1656,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)); | ||
|
|
@@ -2508,8 +2509,10 @@ public void testWithRandomException() throws IOException { | |
| // fair enough | ||
| } catch (IOException ex) { | ||
| assertEquals(ex.getMessage(), "__FAKE__ no space left on device"); | ||
| assertEquals(failableTLog.getTragicException(), ex); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what about the case where we catch |
||
| } 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()) { | ||
|
|
@@ -2931,6 +2934,46 @@ public void testCloseSnapshotTwice() throws Exception { | |
| } | ||
| } | ||
|
|
||
| public void testTranslogCloseInvariant() throws IOException { | ||
| //close method should never be called directly from Translog (the only exception is closeOnTragicEvent) | ||
|
||
| 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; | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why double-set here? Isn't the previous line already doing this?