[ 
https://issues.apache.org/jira/browse/ACCUMULO-3783?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14602210#comment-14602210
 ] 

Dave Marion commented on ACCUMULO-3783:
---------------------------------------

I think the shutdown hook was necessary to close VFS connections to external 
resources as VFS has the ability to pull resources via http, ftp, hdfs, etc. It 
would be nice if we could just nuke the shutdown hook in 
AccumuloVFSClassLoader, but I'm not sure that's possible. Regarding the HDFS 
VFS objects, these same objects exist in the VFS 2.1 release. My intention was 
to remove the ones in Accumulo when 2.1 is ultimately released. If you make 
changes to them, please be sure to make the corresponding changes in Commons 
VFS. I would suggest building VFS 2.1 with these changes first, before 
committing to Accumulo, to ensure that all of the VFS tests pass.

> Unexpected Filesystem Closed exceptions
> ---------------------------------------
>
>                 Key: ACCUMULO-3783
>                 URL: https://issues.apache.org/jira/browse/ACCUMULO-3783
>             Project: Accumulo
>          Issue Type: Bug
>          Components: master, start, tserver
>    Affects Versions: 1.7.0
>            Reporter: Josh Elser
>            Assignee: Josh Elser
>             Fix For: 1.7.1, 1.8.0
>
>         Attachments: ACCUMULO-3783.patch
>
>
> Noticed this in testing 1.7.0 on my laptop tonight. Started two tservers, one 
> continuous ingest client and would kill/restart one of the tservers 
> occasionally. 
> {noformat}
> Failed to close map file
>       java.io.IOException: Filesystem closed
>               at 
> org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:795)
>               at 
> org.apache.hadoop.hdfs.DFSInputStream.close(DFSInputStream.java:629)
>               at java.io.FilterInputStream.close(FilterInputStream.java:181)
>               at 
> org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile$Reader.close(CachableBlockFile.java:409)
>               at 
> org.apache.accumulo.core.file.rfile.RFile$Reader.close(RFile.java:921)
>               at 
> org.apache.accumulo.tserver.tablet.Compactor.compactLocalityGroup(Compactor.java:391)
>               at 
> org.apache.accumulo.tserver.tablet.Compactor.call(Compactor.java:214)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet._majorCompact(Tablet.java:1981)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet.majorCompact(Tablet.java:2098)
>               at 
> org.apache.accumulo.tserver.tablet.CompactionRunner.run(CompactionRunner.java:44)
>               at 
> org.apache.htrace.wrappers.TraceRunnable.run(TraceRunnable.java:57)
>               at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>               at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>               at 
> org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
>               at java.lang.Thread.run(Thread.java:745)
> null
>       java.nio.channels.ClosedChannelException
>               at 
> org.apache.hadoop.hdfs.DFSOutputStream.checkClosed(DFSOutputStream.java:1622)
>               at 
> org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:104)
>               at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:58)
>               at java.io.DataOutputStream.write(DataOutputStream.java:107)
>               at 
> org.apache.accumulo.core.file.rfile.bcfile.SimpleBufferedOutputStream.flushBuffer(SimpleBufferedOutputStream.java:39)
>               at 
> org.apache.accumulo.core.file.rfile.bcfile.SimpleBufferedOutputStream.flush(SimpleBufferedOutputStream.java:68)
>               at 
> org.apache.hadoop.io.compress.CompressionOutputStream.flush(CompressionOutputStream.java:69)
>               at 
> org.apache.accumulo.core.file.rfile.bcfile.Compression$FinishOnFlushCompressionStream.flush(Compression.java:66)
>               at 
> java.io.BufferedOutputStream.flush(BufferedOutputStream.java:141)
>               at 
> org.apache.accumulo.core.file.rfile.bcfile.BCFile$Writer$WBlockState.finish(BCFile.java:233)
>               at 
> org.apache.accumulo.core.file.rfile.bcfile.BCFile$Writer$BlockAppender.close(BCFile.java:320)
>               at 
> org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile$BlockWrite.close(CachableBlockFile.java:121)
>               at 
> org.apache.accumulo.core.file.rfile.RFile$Writer.closeBlock(RFile.java:398)
>               at 
> org.apache.accumulo.core.file.rfile.RFile$Writer.append(RFile.java:382)
>               at 
> org.apache.accumulo.tserver.tablet.Compactor.compactLocalityGroup(Compactor.java:356)
>               at 
> org.apache.accumulo.tserver.tablet.Compactor.call(Compactor.java:214)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet._majorCompact(Tablet.java:1981)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet.majorCompact(Tablet.java:2098)
>               at 
> org.apache.accumulo.tserver.tablet.CompactionRunner.run(CompactionRunner.java:44)
>               at 
> org.apache.htrace.wrappers.TraceRunnable.run(TraceRunnable.java:57)
>               at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>               at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>               at 
> org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
>               at java.lang.Thread.run(Thread.java:745)
> Filesystem closed
>       java.io.IOException: Filesystem closed
>               at 
> org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:795)
>               at org.apache.hadoop.hdfs.DFSClient.delete(DFSClient.java:1927)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem$12.doCall(DistributedFileSystem.java:638)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem$12.doCall(DistributedFileSystem.java:634)
>               at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem.delete(DistributedFileSystem.java:634)
>               at 
> org.apache.accumulo.server.fs.VolumeManagerImpl.deleteRecursively(VolumeManagerImpl.java:193)
>               at 
> org.apache.accumulo.tserver.tablet.Compactor.call(Compactor.java:255)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet._majorCompact(Tablet.java:1981)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet.majorCompact(Tablet.java:2098)
>               at 
> org.apache.accumulo.tserver.tablet.CompactionRunner.run(CompactionRunner.java:44)
>               at 
> org.apache.htrace.wrappers.TraceRunnable.run(TraceRunnable.java:57)
>               at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>               at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>               at 
> org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
>               at java.lang.Thread.run(Thread.java:745)
> MinC failed (null) to create 
> hdfs://localhost:8020/accumulo17/tables/2/t-0000011/F00000yd.rf_tmp retrying 
> ...
> MajC Failed, extent = 2;5;45
>       java.io.IOException: Filesystem closed
>               at 
> org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:795)
>               at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1651)
>               at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1593)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:397)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:393)
>               at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:393)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:337)
>               at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:908)
>               at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:889)
>               at 
> org.apache.accumulo.core.file.rfile.RFileOperations.openWriter(RFileOperations.java:126)
>               at 
> org.apache.accumulo.core.file.rfile.RFileOperations.openWriter(RFileOperations.java:106)
>               at 
> org.apache.accumulo.core.file.DispatchingFileFactory.openWriter(DispatchingFileFactory.java:78)
>               at 
> org.apache.accumulo.tserver.tablet.Compactor.call(Compactor.java:192)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet._majorCompact(Tablet.java:1981)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet.majorCompact(Tablet.java:2098)
>               at 
> org.apache.accumulo.tserver.tablet.CompactionRunner.run(CompactionRunner.java:44)
>               at 
> org.apache.htrace.wrappers.TraceRunnable.run(TraceRunnable.java:57)
>               at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>               at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>               at 
> org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
>               at java.lang.Thread.run(Thread.java:745)
> MajC Failed, extent = 2;45;4
>       java.io.IOException: Filesystem closed
>               at 
> org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:795)
>               at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1651)
>               at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1593)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:397)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:393)
>               at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:393)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:337)
>               at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:908)
>               at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:889)
>               at 
> org.apache.accumulo.core.file.rfile.RFileOperations.openWriter(RFileOperations.java:126)
>               at 
> org.apache.accumulo.core.file.rfile.RFileOperations.openWriter(RFileOperations.java:106)
>               at 
> org.apache.accumulo.core.file.DispatchingFileFactory.openWriter(DispatchingFileFactory.java:78)
>               at 
> org.apache.accumulo.tserver.tablet.Compactor.call(Compactor.java:192)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet._majorCompact(Tablet.java:1981)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet.majorCompact(Tablet.java:2098)
>               at 
> org.apache.accumulo.tserver.tablet.CompactionRunner.run(CompactionRunner.java:44)
>               at 
> org.apache.htrace.wrappers.TraceRunnable.run(TraceRunnable.java:57)
>               at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>               at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>               at 
> org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
>               at java.lang.Thread.run(Thread.java:745)
> MajC Failed, extent = 2;35;3
>       java.io.IOException: Filesystem closed
>               at 
> org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:795)
>               at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1651)
>               at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1593)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:397)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:393)
>               at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:393)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:337)
>               at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:908)
>               at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:889)
>               at 
> org.apache.accumulo.core.file.rfile.RFileOperations.openWriter(RFileOperations.java:126)
>               at 
> org.apache.accumulo.core.file.rfile.RFileOperations.openWriter(RFileOperations.java:106)
>               at 
> org.apache.accumulo.core.file.DispatchingFileFactory.openWriter(DispatchingFileFactory.java:78)
>               at 
> org.apache.accumulo.tserver.tablet.Compactor.call(Compactor.java:192)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet._majorCompact(Tablet.java:1981)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet.majorCompact(Tablet.java:2098)
>               at 
> org.apache.accumulo.tserver.tablet.CompactionRunner.run(CompactionRunner.java:44)
>               at 
> org.apache.htrace.wrappers.TraceRunnable.run(TraceRunnable.java:57)
>               at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>               at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>               at 
> org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
>               at java.lang.Thread.run(Thread.java:745)
> Filesystem closed
>       java.io.IOException: Filesystem closed
>               at 
> org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:795)
>               at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1651)
>               at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1593)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:397)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:393)
>               at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:393)
>               at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:337)
>               at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:908)
>               at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:889)
>               at 
> org.apache.accumulo.core.file.rfile.RFileOperations.openWriter(RFileOperations.java:126)
>               at 
> org.apache.accumulo.core.file.rfile.RFileOperations.openWriter(RFileOperations.java:106)
>               at 
> org.apache.accumulo.core.file.DispatchingFileFactory.openWriter(DispatchingFileFactory.java:78)
>               at 
> org.apache.accumulo.tserver.tablet.Compactor.call(Compactor.java:192)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet._majorCompact(Tablet.java:1981)
>               at 
> org.apache.accumulo.tserver.tablet.Tablet.majorCompact(Tablet.java:2098)
>               at 
> org.apache.accumulo.tserver.tablet.CompactionRunner.run(CompactionRunner.java:44)
>               at 
> org.apache.htrace.wrappers.TraceRunnable.run(TraceRunnable.java:57)
>               at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>               at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>               at 
> org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
>               at java.lang.Thread.run(Thread.java:745)
> {noformat}
> This reminded me of something that [~bills] ran into and tried to get to the 
> bottom of.
> My hunch was that somethign registered a shutdown hook which closed the 
> filesystem object. I also remembered that FileSystem instances are cached.
> Bill pointed out that he thought this might still be a VFS related issue 
> (despite having nothign to do with VFS). After I noticed a shutdown hook in 
> our VFS usage, I think it's unsafe to be sharing FileSystem instances with 
> VFS code that might also be used by Accumulo for important things like 
> writing to files.
> We should get a unique FileSystem instance to provide to any VFS code. We can 
> do this by:
> # Setting {{fs.hdfs.impl.disable.cache}} in the Hadoop Configuration we 
> provide to {{FileSystem.get(Configuration)}}
> # Call {{FileSystem.newInstance(URI, Configuration)}}.
> Either seem to do what we want -- the latter possibly having more reliable 
> semantics than just a configuration value.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to