[ https://issues.apache.org/jira/browse/IGNITE-4862?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15949291#comment-15949291 ]
Ivan Veselovsky edited comment on IGNITE-4862 at 3/31/17 2:45 PM: ------------------------------------------------------------------ The property {{prefetchBlocks}} must be set to non-zero value to reproduce the problem ({{perNodeParallelBatchCount}} should be > 0, but it is 16 by default): {code} <bean class="org.apache.ignite.configuration.FileSystemConfiguration"> ..... <property name="prefetchBlocks" value="32"/> {code} After that the problem was reproducible just after node start with an attempt to read a 44Mb file via IGFS (with {{./hadoop-ig fs -copyToLocal igfs://localhost:10500/tmp/myfile /tmp/ }} ) that exists on HDFS (secondary Fs), but is yet missing in the IGFS caches. The following exception happens on the Ignite node at the same time: {code} Exception in thread "igfs-#50%null%" java.lang.NullPointerException at org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsSecondaryFileSystemPositionedReadable.read(HadoopIgfsSecondaryFileSystemPositionedReadable.java:104) at org.apache.ignite.internal.processors.igfs.IgfsLazySecondaryFileSystemPositionedReadable.read(IgfsLazySecondaryFileSystemPositionedReadable.java:64) at org.apache.ignite.internal.processors.igfs.IgfsDataManager.secondaryDataBlock(IgfsDataManager.java:419) at org.apache.ignite.internal.processors.igfs.IgfsDataManager$4.applyx(IgfsDataManager.java:357) at org.apache.ignite.internal.processors.igfs.IgfsDataManager$4.applyx(IgfsDataManager.java:346) at org.apache.ignite.internal.util.lang.IgniteClosureX.apply(IgniteClosureX.java:38) at org.apache.ignite.internal.util.future.GridFutureChainListener.applyCallback(GridFutureChainListener.java:78) at org.apache.ignite.internal.util.future.GridFutureChainListener.access$000(GridFutureChainListener.java:30) at org.apache.ignite.internal.util.future.GridFutureChainListener$1.run(GridFutureChainListener.java:65) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {code} The problem is that class `org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsSecondaryFileSystemPositionedReadable` was designed fro single-threaded access, but in fact was accessed by multiple threads. That lead to unexpected situation, when org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsSecondaryFileSystemPositionedReadable#in returned null , that caused a bunch of further errors. Fixed by making HadoopIgfsSecondaryFileSystemPositionedReadable thread safe (underlying secondary file system input stream has its own mechanisms to preserve consistency upon multithreaded access.) was (Author: iveselovskiy): The property {{prefetchBlocks}} must be set to non-zero value to reproduce the problem ({{perNodeParallelBatchCount}} should be > 0, but it is 16 by default): {code} <bean class="org.apache.ignite.configuration.FileSystemConfiguration"> ..... <property name="prefetchBlocks" value="32"/> {code} After that the problem was reproducible just after node start with an attempt to read a 44Mb file via IGFS (with {{./hadoop-ig fs -copyToLocal igfs://localhost:10500/tmp/myfile /tmp/ }}) that exists on HDFS (secondary Fs), but is yet missing in the IGFS caches. The following exception happens on the Ignite node at the same time: {code} Exception in thread "igfs-#50%null%" java.lang.NullPointerException at org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsSecondaryFileSystemPositionedReadable.read(HadoopIgfsSecondaryFileSystemPositionedReadable.java:104) at org.apache.ignite.internal.processors.igfs.IgfsLazySecondaryFileSystemPositionedReadable.read(IgfsLazySecondaryFileSystemPositionedReadable.java:64) at org.apache.ignite.internal.processors.igfs.IgfsDataManager.secondaryDataBlock(IgfsDataManager.java:419) at org.apache.ignite.internal.processors.igfs.IgfsDataManager$4.applyx(IgfsDataManager.java:357) at org.apache.ignite.internal.processors.igfs.IgfsDataManager$4.applyx(IgfsDataManager.java:346) at org.apache.ignite.internal.util.lang.IgniteClosureX.apply(IgniteClosureX.java:38) at org.apache.ignite.internal.util.future.GridFutureChainListener.applyCallback(GridFutureChainListener.java:78) at org.apache.ignite.internal.util.future.GridFutureChainListener.access$000(GridFutureChainListener.java:30) at org.apache.ignite.internal.util.future.GridFutureChainListener$1.run(GridFutureChainListener.java:65) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {code} The problem is that class `org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsSecondaryFileSystemPositionedReadable` was designed fro single-threaded access, but in fact was accessed by multiple threads. That lead to unexpected situation, when org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsSecondaryFileSystemPositionedReadable#in returned null , that caused a bunch of further errors. Fixed by making HadoopIgfsSecondaryFileSystemPositionedReadable thread safe (underlying secondary file system input stream has its own mechanisms to preserve consistency upon multithreaded access.) > NPE in reading data from IGFS > ----------------------------- > > Key: IGNITE-4862 > URL: https://issues.apache.org/jira/browse/IGNITE-4862 > Project: Ignite > Issue Type: Bug > Components: hadoop > Affects Versions: 1.9 > Reporter: Dmitry Karachentsev > Assignee: Ivan Veselovsky > Priority: Minor > Fix For: 2.0 > > > {noformat} > D:\app\CodexRT.CodexRT_20170314-1>hadoop\bin\hadoop fs -copyToLocal > igfs:///cacheLink/test3.orc D:\test3.orc > -copyToLocal: Fatal internal error > java.lang.NullPointerException > at > org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInputStream$FetchBufferPart.flatten(HadoopIgfsInputStream.java:458) > at > org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInputStream$DoubleFetchBuffer.flatten(HadoopIgfsInputStream.java:511) > at > org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInputStream.read(HadoopIgfsInputStream.java:177) > at java.io.DataInputStream.read(DataInputStream.java:100) > at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:91) > at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:59) > at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:119) > at > org.apache.hadoop.fs.shell.CommandWithDestination$TargetFileSystem.writeStreamToFile(CommandWithDestination.java:466) > at > org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(CommandWithDestination.java:391) > at > org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(CommandWithDestination.java:328) > at > org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:263) > at > org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:248) > at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:317) > at > org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:289) > at > org.apache.hadoop.fs.shell.CommandWithDestination.processPathArgument(CommandWithDestination.java:243) > at > org.apache.hadoop.fs.shell.Command.processArgument(Command.java:271) > at > org.apache.hadoop.fs.shell.Command.processArguments(Command.java:255) > at > org.apache.hadoop.fs.shell.CommandWithDestination.processArguments(CommandWithDestination.java:220) > at > org.apache.hadoop.fs.shell.Command.processRawArguments(Command.java:201) > at org.apache.hadoop.fs.shell.Command.run(Command.java:165) > at org.apache.hadoop.fs.FsShell.run(FsShell.java:287) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:84) > at org.apache.hadoop.fs.FsShell.main(FsShell.java:340) > {noformat} > Details in discussion: > [http://apache-ignite-users.70518.x6.nabble.com/NullPointerException-when-using-IGFS-td11328.html] -- This message was sent by Atlassian JIRA (v6.3.15#6346)