You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-dev@hadoop.apache.org by "Hairong Kuang (JIRA)" <ji...@apache.org> on 2007/03/15 21:48:09 UTC
[jira] Created: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
------------------------------------------------------------------------------------
Key: HADOOP-1123
URL: https://issues.apache.org/jira/browse/HADOOP-1123
Project: Hadoop
Issue Type: Bug
Components: fs
Affects Versions: 0.12.0
Reporter: Hairong Kuang
Assigned To: Hairong Kuang
Fix For: 0.13.0
NullPointerException occurs when run a large sort
java.lang.NullPointerException
at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
at java.io.DataInputStream.readInt(DataInputStream.java:370)
at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12481312 ]
Hairong Kuang commented on HADOOP-1123:
---------------------------------------
When a checksumed file system tries to recover from a ChecksumError, it first reports the checksum error and then tries to read from a different replica.
For the local file system, it closes the input stream when reporting checksum error. In read retry, it does a seek on the closed input stream and thus causes a NPE.
One solution to this problem is not to retry reading for the local file system. We could let reportCheckError returns a flag indicating if we want to retry or not.
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.13.0
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Tom White (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12484966 ]
Tom White commented on HADOOP-1123:
-----------------------------------
I just ran the unit tests with this patch and I get the following consistent failure in testLocalFileCorruption. Any ideas?
A ChecksumException is expected to be logged.
2007-03-28 20:02:08,528 INFO fs.FSInputChecker (ChecksumFileSystem.java:readBuffer(217)) - Found checksum error: org.apache.hadoop.fs.ChecksumException: Check
sum error: /Users/tom/Documents/workspace/hadoop-committer/build/test/data/corruptFile at 0
at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.verifySum(ChecksumFileSystem.java:258)
at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:213)
at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
at java.io.BufferedInputStream.read(BufferedInputStream.java:235)
at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
at java.io.DataInputStream.readByte(DataInputStream.java:241)
at org.apache.hadoop.dfs.TestFileCorruption.testLocalFileCorruption(TestFileCorruption.java:90)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
at java.lang.reflect.Method.invoke(Method.java:585)
at junit.framework.TestCase.runTest(TestCase.java:154)
at junit.framework.TestCase.runBare(TestCase.java:127)
at junit.framework.TestResult$1.protect(TestResult.java:106)
at junit.framework.TestResult.runProtected(TestResult.java:124)
at junit.framework.TestResult.run(TestResult.java:109)
at junit.framework.TestCase.run(TestCase.java:118)
at junit.framework.TestSuite.runTest(TestSuite.java:208)
at junit.framework.TestSuite.run(TestSuite.java:203)
at org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner.run(JUnitTestRunner.java:297)
at org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner.launch(JUnitTestRunner.java:672)
at org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner.main(JUnitTestRunner.java:567)
------------- ---------------- ---------------
------------- Standard Error -----------------
Waiting for the NameNode to initialize...
Waiting for the Mini HDFS Cluster to start...
------------- ---------------- ---------------
Testcase: testFileCorruption took 4.929 sec
Testcase: testLocalFileCorruption took 0.04 sec
Caused an ERROR
null
java.lang.NullPointerException
at org.apache.hadoop.fs.LocalFileSystem.reportChecksumFailure(LocalFileSystem.java:76)
at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:219)
at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
at java.io.BufferedInputStream.read(BufferedInputStream.java:235)
at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
at java.io.DataInputStream.readByte(DataInputStream.java:241)
at org.apache.hadoop.dfs.TestFileCorruption.testLocalFileCorruption(TestFileCorruption.java:90)
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: 1123_with_test.patch, NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12485039 ]
Hadoop QA commented on HADOOP-1123:
-----------------------------------
+1, because http://issues.apache.org/jira/secure/attachment/12354474/LocalFileCorruption.patch applied and successfully tested against trunk revision http://svn.apache.org/repos/asf/lucene/hadoop/trunk/523072. Results are at http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Patch
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: 1123_with_test.patch, LocalFileCorruption.patch, NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Updated: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Hairong Kuang updated HADOOP-1123:
----------------------------------
Status: Patch Available (was: Open)
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: 1123_with_test.patch, LocalFileCorruption.patch, NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Updated: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Nigel Daley (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Nigel Daley updated HADOOP-1123:
--------------------------------
Attachment: 1123_with_test.patch
New patch with a test case.
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: 1123_with_test.patch, NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Updated: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Nigel Daley (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Nigel Daley updated HADOOP-1123:
--------------------------------
Status: Patch Available (was: Open)
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: 1123_with_test.patch, NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Updated: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Hairong Kuang updated HADOOP-1123:
----------------------------------
Attachment: (was: NPEReportChecksum.patch)
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.13.0
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Updated: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Hairong Kuang updated HADOOP-1123:
----------------------------------
Attachment: LocalFileCorruption.patch
This patch includes the fix to NPE that Tom got.
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: 1123_with_test.patch, LocalFileCorruption.patch, NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Updated: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Hairong Kuang updated HADOOP-1123:
----------------------------------
Attachment: NPEReportChecksum.patch
a patch for review.
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.13.0
>
> Attachments: NPEReportChecksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12484633 ]
Hadoop QA commented on HADOOP-1123:
-----------------------------------
+1, because http://issues.apache.org/jira/secure/attachment/12354361/1123_with_test.patch applied and successfully tested against trunk revision http://svn.apache.org/repos/asf/lucene/hadoop/trunk/523072. Results are at http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Patch
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: 1123_with_test.patch, NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Owen O'Malley (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12484622 ]
Owen O'Malley commented on HADOOP-1123:
---------------------------------------
+1
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: 1123_with_test.patch, NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Updated: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Tom White (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Tom White updated HADOOP-1123:
------------------------------
Resolution: Fixed
Status: Resolved (was: Patch Available)
The new patch fixed the NPE I was seeing, so I've committed this. Thanks Hairong & Nigel!
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: 1123_with_test.patch, LocalFileCorruption.patch, NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Updated: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Hairong Kuang updated HADOOP-1123:
----------------------------------
Attachment: NPE_LocalFS_checksum.patch
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.13.0
>
> Attachments: NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Updated: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Hairong Kuang updated HADOOP-1123:
----------------------------------
Status: Open (was: Patch Available)
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: 1123_with_test.patch, LocalFileCorruption.patch, NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Owen O'Malley (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12484320 ]
Owen O'Malley commented on HADOOP-1123:
---------------------------------------
This patch should have a unit test case, but otherwise looks good.
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.13.0
>
> Attachments: NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Updated: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Owen O'Malley (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Owen O'Malley updated HADOOP-1123:
----------------------------------
Fix Version/s: (was: 0.13.0)
0.12.3
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1123) LocalFileSystem gets a
NullPointerException when tries to recover from ChecksumError
Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/HADOOP-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12485478 ]
Hadoop QA commented on HADOOP-1123:
-----------------------------------
Integrated in Hadoop-Nightly #41 (See http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Nightly/41/)
> LocalFileSystem gets a NullPointerException when tries to recover from ChecksumError
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-1123
> URL: https://issues.apache.org/jira/browse/HADOOP-1123
> Project: Hadoop
> Issue Type: Bug
> Components: fs
> Affects Versions: 0.12.0
> Reporter: Hairong Kuang
> Assigned To: Hairong Kuang
> Fix For: 0.12.3
>
> Attachments: 1123_with_test.patch, LocalFileCorruption.patch, NPE_LocalFS_checksum.patch
>
>
> NullPointerException occurs when run a large sort
> java.lang.NullPointerException
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:74)
> at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:121)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:221)
> at org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:167)
> at org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:41)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> at org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:93)
> at java.io.DataInputStream.readInt(DataInputStream.java:370)
> at org.apache.hadoop.io.SequenceFile$Reader.nextRawKey(SequenceFile.java:1616)
> at org.apache.hadoop.io.SequenceFile$Sorter$SegmentDescriptor.nextRawKey(SequenceFile.java:2567)
> at org.apache.hadoop.io.SequenceFile$Sorter$MergeQueue.next(SequenceFile.java:2353)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:180)
> at org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:149)
> at org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:313)
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.