Skip to content

Commit 8654420

Browse files
committed
HBASE-28546 Make WAL rolling exception clear (#5848)
Signed-off-by: Duo Zhang <zhangduo@apache.org> (cherry picked from commit debb192)
1 parent fa53fac commit 8654420

1 file changed

Lines changed: 4 additions & 1 deletion

File tree

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

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import org.apache.hadoop.fs.Path;
3333
import org.apache.hadoop.hbase.Cell;
3434
import org.apache.hadoop.hbase.HBaseConfiguration;
35+
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
3536
import org.apache.hadoop.hbase.io.ByteBufferWriter;
3637
import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutput;
3738
import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutputHelper;
@@ -205,9 +206,11 @@ private long writeWALMetadata(Consumer<CompletableFuture<Long>> action) throws I
205206
InterruptedIOException ioe = new InterruptedIOException();
206207
ioe.initCause(e);
207208
throw ioe;
208-
} catch (ExecutionException | TimeoutException e) {
209+
} catch (ExecutionException e) {
209210
Throwables.propagateIfPossible(e.getCause(), IOException.class);
210211
throw new RuntimeException(e.getCause());
212+
} catch (TimeoutException e) {
213+
throw new TimeoutIOException(e);
211214
}
212215
}
213216

0 commit comments

Comments
 (0)