You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by "Abhishek Singh Chouhan (JIRA)" <ji...@apache.org> on 2017/09/07 08:43:00 UTC
[jira] [Created] (HBASE-18771) Incorrect StoreFileRefresh leading
to split and compaction failures
Abhishek Singh Chouhan created HBASE-18771:
----------------------------------------------
Summary: Incorrect StoreFileRefresh leading to split and compaction failures
Key: HBASE-18771
URL: https://issues.apache.org/jira/browse/HBASE-18771
Project: HBase
Issue Type: Bug
Affects Versions: 1.3.1
Reporter: Abhishek Singh Chouhan
Assignee: Abhishek Singh Chouhan
Priority: Blocker
Fix For: 3.0.0, 1.4.0, 1.3.2, 1.5.0, 2.0.0-alpha-3
We ran into issues of compaction and split failures with 1.3 similar to HBASE-18186 and HBASE-17406. Here's what i believe is happening -
Lets say we have 4 store files that are compacted to form a new one. At this point we now have 5 store files, however only 1(the newly formed) is open now for the store and rest are waiting to get archived by HFileArchiver
Now before the files are archived we get a FNFE in a scanner. This results in HRegion.RegionScannerImpl.handleFileNotFound(FileNotFoundException fnfe) being called which results in region.refreshStoreFiles(true) -> HStore.refreshStoreFiles()
HStore.refreshStoreFiles now checks the hdfs dir and adds the previously compacted files back to the store, however these files are also present in StoreFileManager's compactedFiles list. Now at this point HFileArchiver runs, checks compactedFiles list and moves these files into the archive directory.
Now when compaction runs it gets:
2017-09-04 12:30:13,899 ERROR [ctions-1504505399609] regionserver.CompactSplitThread - Compaction selection failed regionName = xxxx, storeName = 0, priority = 26, time = 1504528213899
java.io.FileNotFoundException: File does not exist: hdfs://xxxx
at org.apache.hadoop.hdfs.DistributedFileSystem$23.doCall(DistributedFileSystem.java:1337)
at org.apache.hadoop.hdfs.DistributedFileSystem$23.doCall(DistributedFileSystem.java:1329)
at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1329)
at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:422)
at org.apache.hadoop.hbase.regionserver.StoreFileInfo.getReferencedFileStatus(StoreFileInfo.java:342)
at org.apache.hadoop.hbase.regionserver.StoreFileInfo.getFileStatus(StoreFileInfo.java:355)
at org.apache.hadoop.hbase.regionserver.StoreFileInfo.getModificationTime(StoreFileInfo.java:360)
at org.apache.hadoop.hbase.regionserver.StoreFile.getModificationTimeStamp(StoreFile.java:325)
at org.apache.hadoop.hbase.regionserver.StoreUtils.getLowestTimestamp(StoreUtils.java:63)
at org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy.shouldPerformMajorCompaction(RatioBasedCompactionPolicy.java:65)
at org.apache.hadoop.hbase.regionserver.compactions.SortedCompactionPolicy.selectCompaction(SortedCompactionPolicy.java:82)
at org.apache.hadoop.hbase.regionserver.DefaultStoreEngine$DefaultCompactionContext.select(DefaultStoreEngine.java:107)
at org.apache.hadoop.hbase.regionserver.HStore.requestCompaction(HStore.java:1679)
Similarly if a split happens after archival we fail after PONR while opening daughter regions due to FNFE. This results in parent offline and daughters also in a limbo since they're unable to open. Since we get the error after PONR we also end up aborting the RS.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)