Skip to content

Commit 82bd00d

Browse files
committed
HBASE-28569: fix race condition during WAL splitting leading to corrupt recovered.edits
If an exception happens in the call to finishWriterThreads in the org.apache.hadoop.hbase.wal.RecoveredEditsOutputSink.close method, the call to closeWriters should not execute, as it may lead to a race condition that leads to file corruption if the regionserver aborts. The execution of closeWriters in this case would write the trailer in parallel with writer threads, causing corruption, and then the corrupt file would get renamed and finalized when it should not be. This corruption causes problems when the region is then to be assigned. To fix this, when finishWriterThreads throws an exception or is not successful, the corrupt files should not be renamed and finalized.
1 parent 2990a0f commit 82bd00d

File tree

4 files changed

+180
-29
lines changed

4 files changed

+180
-29
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractRecoveredEditsOutputSink.java

Lines changed: 47 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -73,33 +73,27 @@ protected RecoveredEditsWriter createRecoveredEditsWriter(TableName tableName, b
7373
return new RecoveredEditsWriter(region, regionEditsPath, w, seqId);
7474
}
7575

76-
protected Path closeRecoveredEditsWriter(RecoveredEditsWriter editsWriter,
76+
/**
77+
* abortRecoveredEditsWriter closes the editsWriter, but does not rename and finalize the
78+
* recovered edits WAL files. Please see HBASE-28569.
79+
*/
80+
protected void abortRecoveredEditsWriter(RecoveredEditsWriter editsWriter,
7781
List<IOException> thrown) throws IOException {
78-
try {
79-
editsWriter.writer.close();
80-
} catch (IOException ioe) {
81-
final String errorMsg = "Could not close recovered edits at " + editsWriter.path;
82-
LOG.error(errorMsg, ioe);
83-
updateStatusWithMsg(errorMsg);
84-
thrown.add(ioe);
82+
closeRecoveredEditsWriter(editsWriter, thrown);
83+
if (editsWriter.editsWritten == 0) {
84+
// just remove the empty recovered.edits file
85+
removeRecoveredEditsFile(editsWriter);
86+
}
87+
}
88+
89+
protected Path closeRecoveredEditsWriterAndFinalizeEdits(RecoveredEditsWriter editsWriter,
90+
List<IOException> thrown) throws IOException {
91+
if (!closeRecoveredEditsWriter(editsWriter, thrown)) {
8592
return null;
8693
}
87-
final String msg = "Closed recovered edits writer path=" + editsWriter.path + " (wrote "
88-
+ editsWriter.editsWritten + " edits, skipped " + editsWriter.editsSkipped + " edits in "
89-
+ (editsWriter.nanosSpent / 1000 / 1000) + " ms)";
90-
LOG.info(msg);
91-
updateStatusWithMsg(msg);
9294
if (editsWriter.editsWritten == 0) {
9395
// just remove the empty recovered.edits file
94-
if (
95-
walSplitter.walFS.exists(editsWriter.path)
96-
&& !walSplitter.walFS.delete(editsWriter.path, false)
97-
) {
98-
final String errorMsg = "Failed deleting empty " + editsWriter.path;
99-
LOG.warn(errorMsg);
100-
updateStatusWithMsg(errorMsg);
101-
throw new IOException("Failed deleting empty " + editsWriter.path);
102-
}
96+
removeRecoveredEditsFile(editsWriter);
10397
return null;
10498
}
10599

@@ -133,6 +127,37 @@ protected Path closeRecoveredEditsWriter(RecoveredEditsWriter editsWriter,
133127
return dst;
134128
}
135129

130+
private boolean closeRecoveredEditsWriter(RecoveredEditsWriter editsWriter,
131+
List<IOException> thrown) {
132+
try {
133+
editsWriter.writer.close();
134+
} catch (IOException ioe) {
135+
final String errorMsg = "Could not close recovered edits at " + editsWriter.path;
136+
LOG.error(errorMsg, ioe);
137+
updateStatusWithMsg(errorMsg);
138+
thrown.add(ioe);
139+
return false;
140+
}
141+
final String msg = "Closed recovered edits writer path=" + editsWriter.path + " (wrote "
142+
+ editsWriter.editsWritten + " edits, skipped " + editsWriter.editsSkipped + " edits in "
143+
+ (editsWriter.nanosSpent / 1000 / 1000) + " ms)";
144+
LOG.info(msg);
145+
updateStatusWithMsg(msg);
146+
return true;
147+
}
148+
149+
private void removeRecoveredEditsFile(RecoveredEditsWriter editsWriter) throws IOException {
150+
if (
151+
walSplitter.walFS.exists(editsWriter.path)
152+
&& !walSplitter.walFS.delete(editsWriter.path, false)
153+
) {
154+
final String errorMsg = "Failed deleting empty " + editsWriter.path;
155+
LOG.warn(errorMsg);
156+
updateStatusWithMsg(errorMsg);
157+
throw new IOException("Failed deleting empty " + editsWriter.path);
158+
}
159+
}
160+
136161
@Override
137162
public boolean keepRegionEvent(WAL.Entry entry) {
138163
ArrayList<Cell> cells = entry.getEdit().getCells();

hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredEditsOutputSink.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -70,7 +70,7 @@ public void append(EntryBuffers.RegionEntryBuffer buffer) throws IOException {
7070
regionEditsWrittenMap.compute(Bytes.toString(buffer.encodedRegionName),
7171
(k, v) -> v == null ? writer.editsWritten : v + writer.editsWritten);
7272
List<IOException> thrown = new ArrayList<>();
73-
Path dst = closeRecoveredEditsWriter(writer, thrown);
73+
Path dst = closeRecoveredEditsWriterAndFinalizeEdits(writer, thrown);
7474
splits.add(dst);
7575
openingWritersNum.decrementAndGet();
7676
if (!thrown.isEmpty()) {

hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RecoveredEditsOutputSink.java

Lines changed: 21 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -89,24 +89,39 @@ private RecoveredEditsWriter getRecoveredEditsWriter(TableName tableName, byte[]
8989

9090
@Override
9191
public List<Path> close() throws IOException {
92-
boolean isSuccessful = true;
92+
boolean isSuccessful;
9393
try {
9494
isSuccessful = finishWriterThreads();
95-
} finally {
96-
isSuccessful &= closeWriters();
95+
} catch (IOException e) {
96+
closeWriters(false);
97+
throw e;
98+
}
99+
if (!isSuccessful) {
100+
// Even if an exception is not thrown, finishWriterThreads() not being successful is an
101+
// error case where the WAL files should not be finalized.
102+
closeWriters(false);
103+
return null;
97104
}
105+
isSuccessful = closeWriters(true);
98106
return isSuccessful ? splits : null;
99107
}
100108

101109
/**
102-
* Close all of the output streams.
110+
* Close all the output streams.
111+
* @param finalizeEdits true in the successful close case, false when we don't want to rename and
112+
* finalize the temporary, possibly corrupted WAL files, such as when there
113+
* was a previous failure or exception. Please see HBASE-28569.
103114
* @return true when there is no error.
104115
*/
105-
private boolean closeWriters() throws IOException {
116+
boolean closeWriters(boolean finalizeEdits) throws IOException {
106117
List<IOException> thrown = Lists.newArrayList();
107118
for (RecoveredEditsWriter writer : writers.values()) {
108119
closeCompletionService.submit(() -> {
109-
Path dst = closeRecoveredEditsWriter(writer, thrown);
120+
if (!finalizeEdits) {
121+
abortRecoveredEditsWriter(writer, thrown);
122+
return null;
123+
}
124+
Path dst = closeRecoveredEditsWriterAndFinalizeEdits(writer, thrown);
110125
LOG.trace("Closed {}", dst);
111126
splits.add(dst);
112127
return null;
Lines changed: 111 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,111 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.hbase.wal;
19+
20+
import static org.junit.Assert.assertThrows;
21+
22+
import java.io.IOException;
23+
import java.io.InterruptedIOException;
24+
import org.apache.hadoop.conf.Configuration;
25+
import org.apache.hadoop.fs.FileSystem;
26+
import org.apache.hadoop.fs.Path;
27+
import org.apache.hadoop.hbase.HBaseClassTestRule;
28+
import org.apache.hadoop.hbase.HBaseTestingUtil;
29+
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
30+
import org.apache.hadoop.hbase.testclassification.SmallTests;
31+
import org.apache.hadoop.hbase.util.CommonFSUtils;
32+
import org.junit.AfterClass;
33+
import org.junit.BeforeClass;
34+
import org.junit.ClassRule;
35+
import org.junit.Test;
36+
import org.junit.experimental.categories.Category;
37+
import org.mockito.Mockito;
38+
39+
@Category({ RegionServerTests.class, SmallTests.class })
40+
public class TestRecoveredEditsOutputSink {
41+
42+
@ClassRule
43+
public static final HBaseClassTestRule CLASS_RULE =
44+
HBaseClassTestRule.forClass(TestRecoveredEditsOutputSink.class);
45+
46+
private static WALFactory wals;
47+
private static FileSystem fs;
48+
private static Path rootDir;
49+
private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
50+
51+
private static RecoveredEditsOutputSink outputSink;
52+
53+
@BeforeClass
54+
public static void setUpBeforeClass() throws Exception {
55+
Configuration conf = TEST_UTIL.getConfiguration();
56+
conf.set(WALFactory.WAL_PROVIDER, "filesystem");
57+
rootDir = TEST_UTIL.createRootDir();
58+
fs = CommonFSUtils.getRootDirFileSystem(conf);
59+
wals = new WALFactory(conf, "testRecoveredEditsOutputSinkWALFactory");
60+
WALSplitter splitter = new WALSplitter(wals, conf, rootDir, fs, rootDir, fs);
61+
WALSplitter.PipelineController pipelineController = new WALSplitter.PipelineController();
62+
EntryBuffers sink = new EntryBuffers(pipelineController, 1024 * 1024);
63+
outputSink = new RecoveredEditsOutputSink(splitter, pipelineController, sink, 3);
64+
}
65+
66+
@AfterClass
67+
public static void tearDownAfterClass() throws Exception {
68+
wals.close();
69+
fs.delete(rootDir, true);
70+
}
71+
72+
@Test
73+
public void testCloseSuccess() throws IOException {
74+
RecoveredEditsOutputSink spyOutputSink = Mockito.spy(outputSink);
75+
spyOutputSink.close();
76+
Mockito.verify(spyOutputSink, Mockito.times(1)).finishWriterThreads();
77+
Mockito.verify(spyOutputSink, Mockito.times(1)).closeWriters(true);
78+
}
79+
80+
/**
81+
* When a WAL split is interrupted (ex. by a RegionServer abort), the thread join in
82+
* finishWriterThreads() will get interrupted, rethrowing the exception without stopping the
83+
* writer threads. Test to ensure that when this happens, RecoveredEditsOutputSink.close() does
84+
* not rename the recoveredEdits WAL files as this can cause corruption. Please see HBASE-28569.
85+
* However, the writers must still be closed.
86+
*/
87+
@Test
88+
public void testCloseWALSplitInterrupted() throws IOException {
89+
RecoveredEditsOutputSink spyOutputSink = Mockito.spy(outputSink);
90+
// The race condition will lead to an InterruptedException to be caught by finishWriterThreads()
91+
// which is then rethrown as an InterruptedIOException.
92+
Mockito.doThrow(new InterruptedIOException()).when(spyOutputSink).finishWriterThreads();
93+
assertThrows(InterruptedIOException.class, spyOutputSink::close);
94+
Mockito.verify(spyOutputSink, Mockito.times(1)).finishWriterThreads();
95+
Mockito.verify(spyOutputSink, Mockito.times(1)).closeWriters(false);
96+
}
97+
98+
/**
99+
* When finishWriterThreads fails but does not throw an exception, ensure the writers are handled
100+
* like in the exception case - the writers are closed but the recoveredEdits WAL files are not
101+
* renamed.
102+
*/
103+
@Test
104+
public void testCloseWALFinishWriterThreadsFailed() throws IOException {
105+
RecoveredEditsOutputSink spyOutputSink = Mockito.spy(outputSink);
106+
Mockito.doReturn(false).when(spyOutputSink).finishWriterThreads();
107+
spyOutputSink.close();
108+
Mockito.verify(spyOutputSink, Mockito.times(1)).finishWriterThreads();
109+
Mockito.verify(spyOutputSink, Mockito.times(1)).closeWriters(false);
110+
}
111+
}

0 commit comments

Comments
 (0)