[
https://issues.apache.org/jira/browse/IGNITE-498?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14390475#comment-14390475
]
Ivan Veselovsky edited comment on IGNITE-498 at 4/1/15 12:56 PM:
-----------------------------------------------------------------
fixed by removing the problematic code that closes
IgfsFileAffinityRange#markDone():
{code}
- // Check if we have enough free space to do colocated writes.
- if (dataCachePrj.igfsDataSpaceUsed() > dataCachePrj.igfsDataSpaceMax()
*
- igfsCtx.configuration().getFragmentizerLocalWritesRatio()) {
- // Forbid further co-location.
- locRange.markDone();
-
- return new IgfsBlockKey(fileInfo.id(), null,
fileInfo.evictExclude(), block);
- }
-
{code}
The configuration().getFragmentizerLocalWritesRatio() is not used any more.
was (Author: iveselovskiy):
fixed by removing the problematic code that closes
IgfsFileAffinityRange#markDone():
- // Check if we have enough free space to do colocated writes.
- if (dataCachePrj.igfsDataSpaceUsed() > dataCachePrj.igfsDataSpaceMax()
*
- igfsCtx.configuration().getFragmentizerLocalWritesRatio()) {
- // Forbid further co-location.
- locRange.markDone();
-
- return new IgfsBlockKey(fileInfo.id(), null,
fileInfo.evictExclude(), block);
- }
-
The configuration().getFragmentizerLocalWritesRatio() is not used any more.
> "Failed to retrieve file's data block" error on igfs:// when amount of used
> igfs:// space is close to maximum
> -------------------------------------------------------------------------------------------------------------
>
> Key: IGNITE-498
> URL: https://issues.apache.org/jira/browse/IGNITE-498
> Project: Ignite
> Issue Type: Bug
> Components: hadoop
> Affects Versions: sprint-2
> Reporter: Ivan Veselovsky
> Assignee: Vladimir Ozerov
> Fix For: sprint-3
>
> Attachments: IGNITE-498--a.patch
>
>
> Steps to reproduce:
> 1) configure Hadoop to use pure igfs:// filesystem,
> + Ignite mapreduce framework.
> 2) run taragen, terasort, teravalidate Hadoop sample with 2 million items;
> (Ignite node should be running with default memory settings).
> => The job fails with message "Failed to retrieve file's data block
> (corrupted file?) [path=/tmp/terasort/part-r-00000, blockIdx=1481]":
> (Observed on BigTop 0.8.0 (Hadoop 2.4.1) + Ignite 1.0.0 RC3-snapshot)
> [16:37:19,914][INFO
> ][Hadoop-task-33040673-9353-4aaf-8201-47eecb43345f_8-REDUCE-0-0-#204%null%][FileOutputCommitter]
> Saved output of task
> 'attempt_33040673-9353-4aaf-8201-47eecb43345f_0008_r_000000_0' to
> igfs://igfs@localhost/tmp/terasort/_temporary/0/task_33040673-9353-4aaf-8201-47eecb43345f_0008_r_000000
> [16:37:32,348][ERROR][Hadoop-task-33040673-9353-4aaf-8201-47eecb43345f_9-MAP-0-0-#213%null%][HadoopRunnableTask]
> Task execution failed.
> class org.apache.ignite.IgniteCheckedException: class
> org.apache.ignite.IgniteCheckedException: Failed to retrieve file's data
> block (corrupted file?) [path=/tmp/terasort/part-r-00000, blockIdx=1481]
> at
> org.apache.ignite.internal.processors.hadoop.v2.HadoopV2MapTask.run0(HadoopV2MapTask.java:102)
> at
> org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Task.run(HadoopV2Task.java:50)
> at
> org.apache.ignite.internal.processors.hadoop.v2.HadoopV2TaskContext.run(HadoopV2TaskContext.java:193)
> at
> org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopRunnableTask.runTask(HadoopRunnableTask.java:176)
> at
> org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopRunnableTask.call(HadoopRunnableTask.java:120)
> at
> org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopRunnableTask.call(HadoopRunnableTask.java:36)
> at
> org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopExecutorService$2.body(HadoopExecutorService.java:183)
> at
> org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:120)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: class org.apache.ignite.igfs.IgfsCorruptedFileException: Failed to
> retrieve file's data block (corrupted file?)
> [path=/tmp/terasort/part-r-00000, blockIdx=1481]
> at
> org.apache.ignite.internal.processors.igfs.IgfsInputStreamImpl.block(IgfsInputStreamImpl.java:463)
> at
> org.apache.ignite.internal.processors.igfs.IgfsInputStreamImpl.blockFragmentizerSafe(IgfsInputStreamImpl.java:412)
> at
> org.apache.ignite.internal.processors.igfs.IgfsInputStreamImpl.readFromStore(IgfsInputStreamImpl.java:361)
> at
> org.apache.ignite.internal.processors.igfs.IgfsInputStreamImpl.readFully(IgfsInputStreamImpl.java:210)
> at
> org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsInProc.readData(HadoopIgfsInProc.java:324)
> at
> org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsInputStream$DoubleFetchBuffer.fetch(HadoopIgfsInputStream.java:622)
> at
> org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsInputStream$DoubleFetchBuffer.refreshAhead(HadoopIgfsInputStream.java:558)
> at
> org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsInputStream.read(HadoopIgfsInputStream.java:189)
> at java.io.DataInputStream.read(DataInputStream.java:149)
> at
> org.apache.hadoop.examples.terasort.TeraInputFormat$TeraRecordReader.nextKeyValue(TeraInputFormat.java:256)
> at
> org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Context.nextKeyValue(HadoopV2Context.java:108)
> at
> org.apache.hadoop.mapreduce.lib.map.WrappedMapper$Context.nextKeyValue(WrappedMapper.java:91)
> at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
> at
> org.apache.ignite.internal.processors.hadoop.v2.HadoopV2MapTask.run0(HadoopV2MapTask.java:84)
> ... 8 more
> at
> org.apache.ignite.internal.processors.hadoop.HadoopUtils.transformException(HadoopUtils.java:273)
> at
> org.apache.ignite.internal.processors.hadoop.v2.HadoopV2TaskContext.run(HadoopV2TaskContext.java:196)
> at
> org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopRunnableTask.runTask(HadoopRunnableTask.java:176)
> at
> org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopRunnableTask.call(HadoopRunnableTask.java:120)
> at
> org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopRunnableTask.call(HadoopRunnableTask.java:36)
> at
> org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopExecutorService$2.body(HadoopExecutorService.java:183)
> at
> org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:120)
> at java.lang.Thread.run(Thread.java:745)
> [16:38:10,877][INFO ][ignite-metrics-logger][IgniteKernal]
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)