Details
-
Bug
-
Status: Resolved
-
Critical
-
Resolution: Fixed
-
3.0.0-alpha-1, 1.7.0, 2.3.5, 2.4.4
-
None
-
Handles 0 length WAL files moved to oldWALs directory so that they do not block the replication queue.
Description
Recently we encountered a case where size of log queue was increasing to around 300 in few region servers in our production environment.
There were 295 wals in the oldWALs directory for that region server and the first file was a 0 length file.
Replication was throwing the following error.
2021-07-05 03:06:32,757 ERROR [20%2C1625185107182,1] regionserver.ReplicationSourceWALReaderThread - Failed to read stream of replication entries org.apache.hadoop.hbase.replication.regionserver.WALEntryStream$WALEntryStreamRuntimeException: java.io.EOFException: hdfs://<cluster-name>/hbase/oldWALs/<walName> not a SequenceFile at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:112) at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReaderThread.run(ReplicationSourceWALReaderThread.java:156) Caused by: java.io.EOFException: hdfs://<cluster-name>/hbase/oldWALs/<walName> not a SequenceFile at org.apache.hadoop.io.SequenceFile$Reader.init(SequenceFile.java:1934) at org.apache.hadoop.io.SequenceFile$Reader.initialize(SequenceFile.java:1893) at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1842) at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1856) at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader$WALReader.<init>(SequenceFileLogReader.java:70) at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader.reset(SequenceFileLogReader.java:168) at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader.initReader(SequenceFileLogReader.java:177) at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.init(ReaderBase.java:66) at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:313) at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:277) at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:265) at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:424) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:352) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.handleFileNotFound(WALEntryStream.java:341) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:359) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openNextLog(WALEntryStream.java:316) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.checkReader(WALEntryStream.java:306) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:207) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:110) ... 1 more
We fixed similar error via HBASE-25536 but the zero length file was in recovered sources.
There were more logs after the above stack trace.
2021-07-05 03:06:32,757 WARN [20%2C1625185107182,1] regionserver.ReplicationSourceWALReaderThread - Couldn't get file length information about log hdfs://<cluster-name>/hbase/WALs/<walName> 2021-07-05 03:06:32,754 INFO [20%2C1625185107182,1] regionserver.WALEntryStream - Log hdfs://<cluster-name>/hbase/WALs/<walName> was moved to hdfs://<cluster-name>/hbase/oldWALs/<walName>
There is a special logic in ReplicationSourceWALReader thread to handle 0 length files but we only look in WALs directory and not in oldWALs directory.
private boolean handleEofException(Exception e, WALEntryBatch batch) { PriorityBlockingQueue<Path> queue = logQueue.getQueue(walGroupId); // Dump the log even if logQueue size is 1 if the source is from recovered Source // since we don't add current log to recovered source queue so it is safe to remove. if ((e instanceof EOFException || e.getCause() instanceof EOFException) && (source.isRecovered() || queue.size() > 1) && this.eofAutoRecovery) { Path head = queue.peek(); try { if (fs.getFileStatus(head).getLen() == 0) { // head of the queue is an empty log file LOG.warn("Forcing removal of 0 length log in queue: {}", head); logQueue.remove(walGroupId); currentPosition = 0; if (batch != null) { // After we removed the WAL from the queue, we should try shipping the existing batch of // entries addBatchToShippingQueue(batch); } return true; } } catch (IOException ioe) { LOG.warn("Couldn't get file length information about log " + queue.peek(), ioe); } catch (InterruptedException ie) { LOG.trace("Interrupted while adding WAL batch to ship queue"); Thread.currentThread().interrupt(); } } return false; }
We need to add logic to look for such 0 length file in oldWALs directory also.
How this 0 length files are created ?
When the 0 length wal was created, I see the following log messages.
There were some datanode problems while creating the wal but assuming optimization, we just chugged along and again rolled the wal once FSHLog#append call failed.
2021-07-02 00:18:35,069 WARN [Thread-17] hdfs.DataStreamer - Could not get block locations. Source file "/hbase/WALs/<walName>" - Aborting... 2021-07-02 00:18:35,072 WARN [10.180.160.244:60020] hdfs.DFSClient - Error while syncing java.io.IOException: Could not get block locations. Source file "/hbase/WALs/<walName>" - Aborting... at org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1466) at org.apache.hadoop.hdfs.DataStreamer.processDatanodeError(DataStreamer.java:1251) at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:670) 2021-07-02 00:18:35,074 WARN [10.180.160.244:60020] wal.FSHLog - pre-sync failed but an optimization so keep going java.io.IOException: Could not get block locations. Source file "/hbase/WALs/<walName>" - Aborting... at org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1466) at org.apache.hadoop.hdfs.DataStreamer.processDatanodeError(DataStreamer.java:1251) at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:670) .... .... 2021-07-02 00:18:57,303 WARN [0020.append-pool8-t1] wal.FSHLog - Append sequenceId=311389, requesting roll of WAL java.io.IOException: Could not get block locations. Source file "/hbase/WALs/<walName>" - Aborting... at org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1466) at org.apache.hadoop.hdfs.DataStreamer.processDatanodeError(DataStreamer.java:1251) at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:670) 2021-07-02 00:18:57,473 WARN [.244:60020.logRoller] wal.ProtobufLogWriter - Failed to write trailer, non-fatal, continuing... java.io.IOException: Could not get block locations. Source file "/hbase/WALs/<walName>" - Aborting... at org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1466) at org.apache.hadoop.hdfs.DataStreamer.processDatanodeError(DataStreamer.java:1251) at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:670)
Attachments
Issue Links
- links to