Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-23679 Use new FileSystem objects during bulk loads #1019

Closed

Conversation

joshelser
Copy link
Member

Undoes lots of fanciness about FileSystem caching because of
an explicit bug that was present (FileSystem only closed on one
out of N regionservers, and not every time), but also because
the FS caching itself is dodgy and prone to error.

Each BulkLoad request will now get its own FileSystem instance
that it is responsible for closing.

The change to HRegion is because ${hbase.rootdir}/data is chmod 700 which means that a normal user cannot get the size of those files (you'll see lots of AccessDenied exceptions in the RS log). Using HRegionFilesystem instead keeps this from being an issue (reading the filesize as HBase instead of the user).

Undoes lots of fanciness about FileSystem caching because of
an explicit bug that was present (FileSystem only closed on one
out of N regionservers, and not every time), but also because
the FS caching itself is dodgy and prone to error.

Each BulkLoad request will now get its own FileSystem instance
that it is responsible for closing.
Copy link
Contributor

@busbey busbey left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 pending qabot

@joshelser
Copy link
Member Author

joshelser commented Jan 11, 2020

2020-01-11 00:15:00,797 WARN  [RpcServer.default.FPBQ.Fifo.handler=99,queue=9,port=16020] fs.FileSystem: Caching new filesystem: -279427062
java.lang.Exception
	at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3365)
	at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3320)
	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:479)
	at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
	at org.apache.hadoop.hbase.regionserver.HStore.assertBulkLoadHFileOk(HStore.java:761)
	at org.apache.hadoop.hbase.regionserver.HRegion.bulkLoadHFiles(HRegion.java:5958)
	at org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager$1.run(SecureBulkLoadManager.java:264)
	at org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager$1.run(SecureBulkLoadManager.java:233)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:360)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1710)
	at org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager.secureBulkLoadHFiles(SecureBulkLoadManager.java:233)
	at org.apache.hadoop.hbase.regionserver.RSRpcServices.bulkLoadHFile(RSRpcServices.java:2338)
	at org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:42004)
	at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:413)
	at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:131)
	at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:324)
	at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:304)

Looks like this isn't quite sufficient. Another leak (albeit, much slower) coming here. Need to do more to push down that DFS instance we made and use that until we move the files into the final location.

Added some debug to FileSystem.java to see the above. Testing is just done via IntegrationTestBulkLoad with high number of loops but small chain length.

@Apache-HBase
Copy link

💔 -1 overall

Vote Subsystem Runtime Comment
+0 🆗 reexec 0m 32s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+1 💚 hbaseanti 0m 0s Patch does not have any anti-patterns.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
-0 ⚠️ test4tests 0m 0s The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.
_ master Compile Tests _
+1 💚 mvninstall 5m 33s master passed
+1 💚 compile 0m 56s master passed
+1 💚 checkstyle 1m 24s master passed
+1 💚 shadedjars 4m 39s branch has no errors when building our shaded downstream artifacts.
+1 💚 javadoc 0m 39s master passed
+0 🆗 spotbugs 4m 25s Used deprecated FindBugs config; considering switching to SpotBugs.
+1 💚 findbugs 4m 22s master passed
_ Patch Compile Tests _
+1 💚 mvninstall 4m 56s the patch passed
+1 💚 compile 0m 59s the patch passed
+1 💚 javac 0m 59s the patch passed
-1 ❌ checkstyle 1m 24s hbase-server: The patch generated 2 new + 206 unchanged - 0 fixed = 208 total (was 206)
+1 💚 whitespace 0m 0s The patch has no whitespace issues.
+1 💚 shadedjars 4m 39s patch has no errors when building our shaded downstream artifacts.
+1 💚 hadoopcheck 15m 35s Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.
+1 💚 javadoc 0m 36s the patch passed
+1 💚 findbugs 4m 29s the patch passed
_ Other Tests _
+1 💚 unit 155m 44s hbase-server in the patch passed.
+1 💚 asflicense 0m 37s The patch does not generate ASF License warnings.
213m 29s
Subsystem Report/Notes
Docker Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1019/1/artifact/out/Dockerfile
GITHUB PR #1019
Optional Tests dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile
uname Linux 4121411bd04c 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 GNU/Linux
Build tool maven
Personality /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1019/out/precommit/personality/provided.sh
git revision master / 8ca6148
Default Java 1.8.0_181
checkstyle https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1019/1/artifact/out/diff-checkstyle-hbase-server.txt
Test Results https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1019/1/testReport/
Max. process+thread count 5823 (vs. ulimit of 10000)
modules C: hbase-server U: hbase-server
Console output https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1019/1/console
versions git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11
Powered by Apache Yetus 0.11.1 https://yetus.apache.org

This message was automatically generated.

@joshelser
Copy link
Member Author

joshelser commented Jan 11, 2020

Ugh, there's a few of these where, down in HStore, HRegion, and even the WAL code (ugh), which is all invoked via bulk load where we do a FileSystem.get(conf) or path.getFileSystem(conf). All of them will leak a FileSystem instance with the SBLM changes in 2.x.

2020-01-11 01:42:31,080 WARN  [RpcServer.default.FPBQ.Fifo.handler=97,queue=7,port=16020] fs.FileSystem: Caching new filesystem: -1042984133
java.lang.Exception
	at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3365)
	at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3320)
	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:479)
	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:227)
	at org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getServerNameFromWALDirectoryName(AbstractFSWALProvider.java:330)
	at org.apache.hadoop.hbase.fs.HFileSystem$ReorderWALBlocks.reorderBlocks(HFileSystem.java:426)
	at org.apache.hadoop.hbase.fs.HFileSystem$1.invoke(HFileSystem.java:378)
	at com.sun.proxy.$Proxy20.getBlockLocations(Unknown Source)
	at org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:862)
	at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:851)
	at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:840)
	at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:1004)
	at org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:326)
	at org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:322)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:334)
	at org.apache.hadoop.fs.FilterFileSystem.open(FilterFileSystem.java:164)
	at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:899)
	at org.apache.hadoop.hbase.io.FSDataInputStreamWrapper.<init>(FSDataInputStreamWrapper.java:125)
	at org.apache.hadoop.hbase.io.FSDataInputStreamWrapper.<init>(FSDataInputStreamWrapper.java:102)
	at org.apache.hadoop.hbase.regionserver.StoreFileInfo.open(StoreFileInfo.java:254)
	at org.apache.hadoop.hbase.regionserver.HStoreFile.open(HStoreFile.java:367)
	at org.apache.hadoop.hbase.regionserver.HStoreFile.initReader(HStoreFile.java:475)
	at org.apache.hadoop.hbase.regionserver.HStore.createStoreFileAndReader(HStore.java:690)
	at org.apache.hadoop.hbase.regionserver.HStore.createStoreFileAndReader(HStore.java:683)
	at org.apache.hadoop.hbase.regionserver.HStore.bulkLoadHFile(HStore.java:854)
	at org.apache.hadoop.hbase.regionserver.HRegion.bulkLoadHFiles(HRegion.java:6057)
	at org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager$1.run(SecureBulkLoadManager.java:264)
	at org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager$1.run(SecureBulkLoadManager.java:233)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:360)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1710)
	at org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager.secureBulkLoadHFiles(SecureBulkLoadManager.java:233)
	at org.apache.hadoop.hbase.regionserver.RSRpcServices.bulkLoadHFile(RSRpcServices.java:2338)
	at org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:42004)
	at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:413)
	at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:131)
	at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:324)
	at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:304)

The fix is to just push down the FileSystem or use one that is already created, but this gets tricky in some places. Will need to step back from this all and see if there's a better way to do this.

@joshelser
Copy link
Member Author

Thinking about this before I leave it for the night -- my initial thought was that as long as, in SecureBulkLoadManager, we managed the lifecycle of our FileSystem correctly, we'd be fine. But, the reality is that other parts of HBase are (inadvertently) relying on a FileSystem being cached.

The smaller fix may be to re-add the caching, keep the reference counting, but correct the clean-up logic.

@joshelser
Copy link
Member Author

Closing in favor of #1029

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
3 participants