[
https://issues.apache.org/jira/browse/HBASE-25893?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Rushabh Shah updated HBASE-25893:
---------------------------------
Description:
Recently we encountered RS aborts due to NPE while replaying edits from split
logs during region open.
{noformat}
2021-05-13 19:34:28,871 ERROR [<hostname>:60020-17] handler.OpenRegionHandler -
Failed open of
region=<table-name>,1619036437822.0556ab96be88000b6f5f3fad47938ccd., starting
to roll back the global memstore size.
java.lang.NullPointerException
at org.apache.hadoop.hbase.CellUtil.matchingFamily(CellUtil.java:411)
at
org.apache.hadoop.hbase.regionserver.HRegion.replayRecoveredEdits(HRegion.java:4682)
at
org.apache.hadoop.hbase.regionserver.HRegion.replayRecoveredEditsForPaths(HRegion.java:4557)
at
org.apache.hadoop.hbase.regionserver.HRegion.replayRecoveredEditsIfAny(HRegion.java:4470)
at
org.apache.hadoop.hbase.regionserver.HRegion.initializeRegionInternals(HRegion.java:949)
at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:908)
at
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7253)
at
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7214)
at
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7185)
at
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7141)
at
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7092)
at
org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.openRegion(OpenRegionHandler.java:364)
at
org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process(OpenRegionHandler.java:131)
at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:129)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
{noformat}
Tracing back how the corrupt wal was generated.
{noformat}
2021-05-12 05:21:23,333 FATAL [<hostname>:60020-0-Writer-1] wal.WALSplitter -
556ab96be88000b6f5f3fad47938ccd/5039807= to log
java.nio.channels.ClosedChannelException
at
org.apache.hadoop.hdfs.DataStreamer$LastExceptionInStreamer.throwException4Close(DataStreamer.java:331)
at
org.apache.hadoop.hdfs.DFSOutputStream.checkClosed(DFSOutputStream.java:151)
at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:105)
at
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:58)
at java.io.DataOutputStream.write(DataOutputStream.java:107)
at org.apache.hadoop.hbase.KeyValue.write(KeyValue.java:2543)
at
org.apache.phoenix.hbase.index.wal.KeyValueCodec.write(KeyValueCodec.java:104)
at
org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec$IndexKeyValueEncoder.write(IndexedWALEditCodec.java:218)
at
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter.append(ProtobufLogWriter.java:128)
at
org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.appendBuffer(WALSplitter.java:1742)
at
org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.append(WALSplitter.java:1714)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.writeBuffer(WALSplitter.java:1179)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.doRun(WALSplitter.java:1171)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.run(WALSplitter.java:1141)
2021-05-12 05:21:23,333 ERROR [<hostname>:60020-0-Writer-1] wal.WALSplitter -
Exiting thread
java.nio.channels.ClosedChannelException
at
org.apache.hadoop.hdfs.DataStreamer$LastExceptionInStreamer.throwException4Close(DataStreamer.java:331)
at
org.apache.hadoop.hdfs.DFSOutputStream.checkClosed(DFSOutputStream.java:151)
at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:105)
at
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:58)
at java.io.DataOutputStream.write(DataOutputStream.java:107)
at org.apache.hadoop.hbase.KeyValue.write(KeyValue.java:2543)
at
org.apache.phoenix.hbase.index.wal.KeyValueCodec.write(KeyValueCodec.java:104)
at
org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec$IndexKeyValueEncoder.write(IndexedWALEditCodec.java:218)
at
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter.append(ProtobufLogWriter.java:128)
at
org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.appendBuffer(WALSplitter.java:1742)
at
org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.append(WALSplitter.java:1714)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.writeBuffer(WALSplitter.java:1179)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.doRun(WALSplitter.java:1171)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.run(WALSplitter.java:1141)
{noformat}
3 milliseconds ago, it looks like we closed the log file
{noformat}
2021-05-12 05:21:23,330 DEBUG [it-log-closeStream-1] wal.WALSplitter - Closed
wap
hdfs://<cluster-name>/hbase/data/default/<table-name>/0556ab96be88000b6f5f3fad47938ccd/recovered.edits/0000000000005039408-<hostname>%2C60020%2C1620792238779.1620794103718.temp
(wrote 85 edits, skipped 0 edits in 204ms
2021-05-12 05:21:23,343 INFO [it-log-closeStream-1] wal.WALSplitter - Rename
hdfs://<cluster-name>/hbase/data/default/<table-name>/0556ab96be88000b6f5f3fad47938ccd/recovered.edits/0000000000005039408-<hostname>%2C60020%2C1620792238779.1620794103718.temp
to
hdfs://<cluster-name>/hbase/data/default/<table-name>/0556ab96be88000b6f5f3fad47938ccd/recovered.edits/0000000000005039806
{noformat}
It looks like we closed the log file while the writer thread is still writing
to it.
Here is the code from branch-1
{code:java}
@Override
public List<Path> finishWritingAndClose() throws IOException {
boolean isSuccessful = false;
List<Path> result = null;
try {
isSuccessful = finishWriting(false); ----> Received an exception in
this method. Writer is still writing.
} finally {
result = close(); ------> We close the log file here
List<IOException> thrown = closeLogWriters(null); -------> We close the
writer here.
if (thrown != null && !thrown.isEmpty()) {
throw MultipleIOException.createIOException(thrown);
}
}
{code}
We *should close the writer first* and then close the log file.
SplitLogWorker thread exited with the following exception couple of seconds ago.
{noformat}
2021-05-12 05:21:19,060 INFO [<hostname>:60020] regionserver.SplitLogWorker -
Sending interrupt to stop the worker thread
2021-05-12 05:21:19,064 INFO [/<hostname>:60020] regionserver.HRegionServer -
Stopping infoServer
2021-05-12 05:21:19,064 WARN [<hostname>:60020]
coordination.ZkSplitLogWorkerCoordination - Interrupted while yielding for
other region servers
java.lang.InterruptedException: sleep interrupted
at java.lang.Thread.sleep(Native Method)
at
org.apache.hadoop.hbase.coordination.ZkSplitLogWorkerCoordination.grabTask(ZkSplitLogWorkerCoordination.java:274)
at
org.apache.hadoop.hbase.coordination.ZkSplitLogWorkerCoordination.taskLoop(ZkSplitLogWorkerCoordination.java:426)
at
org.apache.hadoop.hbase.regionserver.SplitLogWorker.run(SplitLogWorker.java:148)
at java.lang.Thread.run(Thread.java:748)
2021-05-12 05:21:19,065 INFO [<hostname>:60020] regionserver.SplitLogWorker -
SplitLogWorker <hostname>,60020,1620796868880 exiting
{noformat}
was:
Recently we encountered RS aborts due to NPE while replaying edits from split
logs during region open.
{noformat}
2021-05-13 19:34:28,871 ERROR [<hostname>:60020-17] handler.OpenRegionHandler -
Failed open of
region=<table-name>,1619036437822.0556ab96be88000b6f5f3fad47938ccd., starting
to roll back the global memstore size.
java.lang.NullPointerException
at org.apache.hadoop.hbase.CellUtil.matchingFamily(CellUtil.java:411)
at
org.apache.hadoop.hbase.regionserver.HRegion.replayRecoveredEdits(HRegion.java:4682)
at
org.apache.hadoop.hbase.regionserver.HRegion.replayRecoveredEditsForPaths(HRegion.java:4557)
at
org.apache.hadoop.hbase.regionserver.HRegion.replayRecoveredEditsIfAny(HRegion.java:4470)
at
org.apache.hadoop.hbase.regionserver.HRegion.initializeRegionInternals(HRegion.java:949)
at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:908)
at
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7253)
at
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7214)
at
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7185)
at
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7141)
at
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7092)
at
org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.openRegion(OpenRegionHandler.java:364)
at
org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process(OpenRegionHandler.java:131)
at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:129)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
{noformat}
Tracing back how the corrupt wal was generated.
{noformat}
2021-05-12 05:21:23,333 FATAL [<hostname>:60020-0-Writer-1] wal.WALSplitter -
556ab96be88000b6f5f3fad47938ccd/5039807= to log
java.nio.channels.ClosedChannelException
at
org.apache.hadoop.hdfs.DataStreamer$LastExceptionInStreamer.throwException4Close(DataStreamer.java:331)
at
org.apache.hadoop.hdfs.DFSOutputStream.checkClosed(DFSOutputStream.java:151)
at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:105)
at
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:58)
at java.io.DataOutputStream.write(DataOutputStream.java:107)
at org.apache.hadoop.hbase.KeyValue.write(KeyValue.java:2543)
at
org.apache.phoenix.hbase.index.wal.KeyValueCodec.write(KeyValueCodec.java:104)
at
org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec$IndexKeyValueEncoder.write(IndexedWALEditCodec.java:218)
at
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter.append(ProtobufLogWriter.java:128)
at
org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.appendBuffer(WALSplitter.java:1742)
at
org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.append(WALSplitter.java:1714)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.writeBuffer(WALSplitter.java:1179)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.doRun(WALSplitter.java:1171)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.run(WALSplitter.java:1141)
2021-05-12 05:21:23,333 ERROR [<hostname>:60020-0-Writer-1] wal.WALSplitter -
Exiting thread
java.nio.channels.ClosedChannelException
at
org.apache.hadoop.hdfs.DataStreamer$LastExceptionInStreamer.throwException4Close(DataStreamer.java:331)
at
org.apache.hadoop.hdfs.DFSOutputStream.checkClosed(DFSOutputStream.java:151)
at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:105)
at
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:58)
at java.io.DataOutputStream.write(DataOutputStream.java:107)
at org.apache.hadoop.hbase.KeyValue.write(KeyValue.java:2543)
at
org.apache.phoenix.hbase.index.wal.KeyValueCodec.write(KeyValueCodec.java:104)
at
org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec$IndexKeyValueEncoder.write(IndexedWALEditCodec.java:218)
at
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter.append(ProtobufLogWriter.java:128)
at
org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.appendBuffer(WALSplitter.java:1742)
at
org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.append(WALSplitter.java:1714)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.writeBuffer(WALSplitter.java:1179)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.doRun(WALSplitter.java:1171)
at
org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.run(WALSplitter.java:1141)
{noformat}
3 milliseconds ago, it looks like we closed the log file
{noformat}
2021-05-12 05:21:23,330 DEBUG [it-log-closeStream-1] wal.WALSplitter - Closed
wap
hdfs://<cluster-name>/hbase/data/default/<table-name>/0556ab96be88000b6f5f3fad47938ccd/recovered.edits/0000000000005039408-<hostname>%2C60020%2C1620792238779.1620794103718.temp
(wrote 85 edits, skipped 0 edits in 204ms
2021-05-12 05:21:23,343 INFO [it-log-closeStream-1] wal.WALSplitter - Rename
hdfs://<cluster-name>/hbase/data/default/<table-name>/0556ab96be88000b6f5f3fad47938ccd/recovered.edits/0000000000005039408-<hostname>%2C60020%2C1620792238779.1620794103718.temp
to
hdfs://<cluster-name>/hbase/data/default/<table-name>/0556ab96be88000b6f5f3fad47938ccd/recovered.edits/0000000000005039806
{noformat}
It looks like we closed the log file while the writer thread is still writing
to it.
Here is the code from branch-1
{code:java}
@Override
public List<Path> finishWritingAndClose() throws IOException {
boolean isSuccessful = false;
List<Path> result = null;
try {
isSuccessful = finishWriting(false); ----> Received an exception in
this method. Writer is still writing.
} finally {
result = close(); ------> We close the log file here
List<IOException> thrown = closeLogWriters(null); -------> We close the
writer here.
if (thrown != null && !thrown.isEmpty()) {
throw MultipleIOException.createIOException(thrown);
}
}
{code}
We should close the writer first and then close the log file.
SplitLogWorker thread exited with the following exception couple of seconds ago.
{noformat}
2021-05-12 05:21:19,060 INFO [<hostname>:60020] regionserver.SplitLogWorker -
Sending interrupt to stop the worker thread
2021-05-12 05:21:19,064 INFO [/<hostname>:60020] regionserver.HRegionServer -
Stopping infoServer
2021-05-12 05:21:19,064 WARN [<hostname>:60020]
coordination.ZkSplitLogWorkerCoordination - Interrupted while yielding for
other region servers
java.lang.InterruptedException: sleep interrupted
at java.lang.Thread.sleep(Native Method)
at
org.apache.hadoop.hbase.coordination.ZkSplitLogWorkerCoordination.grabTask(ZkSplitLogWorkerCoordination.java:274)
at
org.apache.hadoop.hbase.coordination.ZkSplitLogWorkerCoordination.taskLoop(ZkSplitLogWorkerCoordination.java:426)
at
org.apache.hadoop.hbase.regionserver.SplitLogWorker.run(SplitLogWorker.java:148)
at java.lang.Thread.run(Thread.java:748)
2021-05-12 05:21:19,065 INFO [<hostname>:60020] regionserver.SplitLogWorker -
SplitLogWorker <hostname>,60020,1620796868880 exiting
{noformat}
> Corruption in recovered WAL in WALSplitter
> ------------------------------------------
>
> Key: HBASE-25893
> URL: https://issues.apache.org/jira/browse/HBASE-25893
> Project: HBase
> Issue Type: Improvement
> Components: regionserver, wal
> Affects Versions: 1.6.0
> Reporter: Rushabh Shah
> Assignee: Rushabh Shah
> Priority: Critical
>
> Recently we encountered RS aborts due to NPE while replaying edits from split
> logs during region open.
> {noformat}
> 2021-05-13 19:34:28,871 ERROR [<hostname>:60020-17] handler.OpenRegionHandler
> - Failed open of
> region=<table-name>,1619036437822.0556ab96be88000b6f5f3fad47938ccd., starting
> to roll back the global memstore size.
> java.lang.NullPointerException
> at org.apache.hadoop.hbase.CellUtil.matchingFamily(CellUtil.java:411)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.replayRecoveredEdits(HRegion.java:4682)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.replayRecoveredEditsForPaths(HRegion.java:4557)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.replayRecoveredEditsIfAny(HRegion.java:4470)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.initializeRegionInternals(HRegion.java:949)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:908)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7253)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7214)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7185)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7141)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7092)
> at
> org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.openRegion(OpenRegionHandler.java:364)
> at
> org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process(OpenRegionHandler.java:131)
> at
> org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:129)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> {noformat}
> Tracing back how the corrupt wal was generated.
> {noformat}
> 2021-05-12 05:21:23,333 FATAL [<hostname>:60020-0-Writer-1] wal.WALSplitter
> - 556ab96be88000b6f5f3fad47938ccd/5039807= to log
> java.nio.channels.ClosedChannelException
> at
> org.apache.hadoop.hdfs.DataStreamer$LastExceptionInStreamer.throwException4Close(DataStreamer.java:331)
> at
> org.apache.hadoop.hdfs.DFSOutputStream.checkClosed(DFSOutputStream.java:151)
> at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:105)
> at
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:58)
> at java.io.DataOutputStream.write(DataOutputStream.java:107)
> at org.apache.hadoop.hbase.KeyValue.write(KeyValue.java:2543)
> at
> org.apache.phoenix.hbase.index.wal.KeyValueCodec.write(KeyValueCodec.java:104)
> at
> org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec$IndexKeyValueEncoder.write(IndexedWALEditCodec.java:218)
> at
> org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter.append(ProtobufLogWriter.java:128)
> at
> org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.appendBuffer(WALSplitter.java:1742)
> at
> org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.append(WALSplitter.java:1714)
> at
> org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.writeBuffer(WALSplitter.java:1179)
> at
> org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.doRun(WALSplitter.java:1171)
> at
> org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.run(WALSplitter.java:1141)
> 2021-05-12 05:21:23,333 ERROR [<hostname>:60020-0-Writer-1] wal.WALSplitter -
> Exiting thread
> java.nio.channels.ClosedChannelException
> at
> org.apache.hadoop.hdfs.DataStreamer$LastExceptionInStreamer.throwException4Close(DataStreamer.java:331)
> at
> org.apache.hadoop.hdfs.DFSOutputStream.checkClosed(DFSOutputStream.java:151)
> at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:105)
> at
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:58)
> at java.io.DataOutputStream.write(DataOutputStream.java:107)
> at org.apache.hadoop.hbase.KeyValue.write(KeyValue.java:2543)
> at
> org.apache.phoenix.hbase.index.wal.KeyValueCodec.write(KeyValueCodec.java:104)
> at
> org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec$IndexKeyValueEncoder.write(IndexedWALEditCodec.java:218)
> at
> org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter.append(ProtobufLogWriter.java:128)
> at
> org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.appendBuffer(WALSplitter.java:1742)
> at
> org.apache.hadoop.hbase.wal.WALSplitter$LogRecoveredEditsOutputSink.append(WALSplitter.java:1714)
> at
> org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.writeBuffer(WALSplitter.java:1179)
> at
> org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.doRun(WALSplitter.java:1171)
> at
> org.apache.hadoop.hbase.wal.WALSplitter$WriterThread.run(WALSplitter.java:1141)
> {noformat}
> 3 milliseconds ago, it looks like we closed the log file
> {noformat}
> 2021-05-12 05:21:23,330 DEBUG [it-log-closeStream-1] wal.WALSplitter -
> Closed wap
> hdfs://<cluster-name>/hbase/data/default/<table-name>/0556ab96be88000b6f5f3fad47938ccd/recovered.edits/0000000000005039408-<hostname>%2C60020%2C1620792238779.1620794103718.temp
> (wrote 85 edits, skipped 0 edits in 204ms
> 2021-05-12 05:21:23,343 INFO [it-log-closeStream-1] wal.WALSplitter - Rename
> hdfs://<cluster-name>/hbase/data/default/<table-name>/0556ab96be88000b6f5f3fad47938ccd/recovered.edits/0000000000005039408-<hostname>%2C60020%2C1620792238779.1620794103718.temp
> to
> hdfs://<cluster-name>/hbase/data/default/<table-name>/0556ab96be88000b6f5f3fad47938ccd/recovered.edits/0000000000005039806
> {noformat}
> It looks like we closed the log file while the writer thread is still writing
> to it.
> Here is the code from branch-1
> {code:java}
> @Override
> public List<Path> finishWritingAndClose() throws IOException {
> boolean isSuccessful = false;
> List<Path> result = null;
> try {
> isSuccessful = finishWriting(false); ----> Received an exception in
> this method. Writer is still writing.
> } finally {
> result = close(); ------> We close the log file here
> List<IOException> thrown = closeLogWriters(null); -------> We close
> the writer here.
> if (thrown != null && !thrown.isEmpty()) {
> throw MultipleIOException.createIOException(thrown);
> }
> }
> {code}
> We *should close the writer first* and then close the log file.
> SplitLogWorker thread exited with the following exception couple of seconds
> ago.
> {noformat}
> 2021-05-12 05:21:19,060 INFO [<hostname>:60020] regionserver.SplitLogWorker
> - Sending interrupt to stop the worker thread
> 2021-05-12 05:21:19,064 INFO [/<hostname>:60020] regionserver.HRegionServer
> - Stopping infoServer
> 2021-05-12 05:21:19,064 WARN [<hostname>:60020]
> coordination.ZkSplitLogWorkerCoordination - Interrupted while yielding for
> other region servers
> java.lang.InterruptedException: sleep interrupted
> at java.lang.Thread.sleep(Native Method)
> at
> org.apache.hadoop.hbase.coordination.ZkSplitLogWorkerCoordination.grabTask(ZkSplitLogWorkerCoordination.java:274)
> at
> org.apache.hadoop.hbase.coordination.ZkSplitLogWorkerCoordination.taskLoop(ZkSplitLogWorkerCoordination.java:426)
> at
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.run(SplitLogWorker.java:148)
> at java.lang.Thread.run(Thread.java:748)
> 2021-05-12 05:21:19,065 INFO [<hostname>:60020] regionserver.SplitLogWorker
> - SplitLogWorker <hostname>,60020,1620796868880 exiting
> {noformat}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)