From debb1921edc79ee7d712ae19381b777d4aafdff0 Mon Sep 17 00:00:00 2001 From: Wei-Chiu Chuang Date: Fri, 31 May 2024 11:15:11 -0700 Subject: [PATCH] HBASE-28546 Make WAL rolling exception clear (#5848) Signed-off-by: Duo Zhang --- .../hbase/regionserver/wal/AsyncProtobufLogWriter.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java index f10f39222722..bb874a001d2f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.exceptions.TimeoutIOException; import org.apache.hadoop.hbase.io.ByteBufferWriter; import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutput; import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutputHelper; @@ -205,9 +206,11 @@ private long writeWALMetadata(Consumer> action) throws I InterruptedIOException ioe = new InterruptedIOException(); ioe.initCause(e); throw ioe; - } catch (ExecutionException | TimeoutException e) { + } catch (ExecutionException e) { Throwables.propagateIfPossible(e.getCause(), IOException.class); throw new RuntimeException(e.getCause()); + } catch (TimeoutException e) { + throw new TimeoutIOException(e); } }