accumulo-notifications mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Josh Elser (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (ACCUMULO-3783) Unexpected Filesystem Closed exceptions
Date Thu, 14 May 2015 15:40:00 GMT

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

Josh Elser commented on ACCUMULO-3783:
--------------------------------------

Oh interesting. Master got stuck unable to do anything last night because it's filesystem
was closed

{noformat}
2015-05-14 03:37:53,470 [recovery.HadoopLogCloser] INFO : Waiting for file to be closed hdfs://cn020:8020/accumulo/wal/cn020.l42scl.hortonworks.com+9997/654276ac-7ad0-45e3-a0d2-cbdca3adf29a
...
2015-05-14 03:37:58,472 [recovery.HadoopLogCloser] WARN : Error recovering lease on hdfs://cn020:8020/accumulo/wal/cn020.l42scl.hortonworks.com+9997/654276ac-7ad0-45e3-a0d2-cbdca3adf29a
java.io.IOException: Filesystem closed
        at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:795)
        at org.apache.hadoop.hdfs.DFSClient.recoverLease(DFSClient.java:1235)
        at org.apache.hadoop.hdfs.DistributedFileSystem$2.doCall(DistributedFileSystem.java:278)
        at org.apache.hadoop.hdfs.DistributedFileSystem$2.doCall(DistributedFileSystem.java:274)
        at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
        at org.apache.hadoop.hdfs.DistributedFileSystem.recoverLease(DistributedFileSystem.java:274)
        at org.apache.accumulo.server.master.recovery.HadoopLogCloser.close(HadoopLogCloser.java:55)
        at org.apache.accumulo.master.recovery.RecoveryManager$LogSortTask.run(RecoveryManager.java:96)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:178)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:292)
        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)
2015-05-14 03:37:58,474 [recovery.RecoveryManager] WARN : Failed to initiate log sort hdfs://cn020:8020/accumulo/wal/cn020.l42scl.hortonworks.com+9997/654276ac-7ad0-45e3-a0d2-cbdca3adf29a
java.io.IOException: Filesystem closed
        at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:795)
        at org.apache.hadoop.hdfs.DFSClient.append(DFSClient.java:1802)
        at org.apache.hadoop.hdfs.DFSClient.append(DFSClient.java:1796)
        at org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:323)
        at org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:319)
        at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
        at org.apache.hadoop.hdfs.DistributedFileSystem.append(DistributedFileSystem.java:319)
        at org.apache.hadoop.fs.FileSystem.append(FileSystem.java:1163)
        at org.apache.accumulo.server.master.recovery.HadoopLogCloser.close(HadoopLogCloser.java:64)
        at org.apache.accumulo.master.recovery.RecoveryManager$LogSortTask.run(RecoveryManager.java:96)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:178)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:292)
        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}

Sadly, there's nothing else inbetween those two messages which say _what_ closed the filesystem.
IIRC, there were no other places in our code that we actually called close on the FileSystem.
This then proceeded to hang the master for the next 5 hours. A+

> 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
>             Fix For: 1.8.0, 1.7.1
>
>
> 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)

Mime
View raw message