MAPREDUCE-2779. JobSplitWriter.java can't handle large job.split file. Contributed by Ming Ma.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1177779 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
f48280ac56
commit
960d7643fb
|
@ -2072,6 +2072,9 @@ Release 0.22.0 - Unreleased
|
|||
MAPREDUCE-2991. queueinfo.jsp fails to show queue status for Capacity
|
||||
scheduler if queue names contain special symbols. (Priyo Mustafi via shv)
|
||||
|
||||
MAPREDUCE-2779. JobSplitWriter.java can't handle large job.split file.
|
||||
(Ming Ma via shv)
|
||||
|
||||
Release 0.21.1 - Unreleased
|
||||
|
||||
NEW FEATURES
|
||||
|
|
|
@ -114,15 +114,15 @@ public class JobSplitWriter {
|
|||
if (array.length != 0) {
|
||||
SerializationFactory factory = new SerializationFactory(conf);
|
||||
int i = 0;
|
||||
long offset = out.size();
|
||||
long offset = out.getPos();
|
||||
for(T split: array) {
|
||||
int prevCount = out.size();
|
||||
long prevCount = out.getPos();
|
||||
Text.writeString(out, split.getClass().getName());
|
||||
Serializer<T> serializer =
|
||||
factory.getSerializer((Class<T>) split.getClass());
|
||||
serializer.open(out);
|
||||
serializer.serialize(split);
|
||||
int currCount = out.size();
|
||||
long currCount = out.getPos();
|
||||
info[i++] =
|
||||
new JobSplit.SplitMetaInfo(
|
||||
split.getLocations(), offset,
|
||||
|
@ -139,12 +139,12 @@ public class JobSplitWriter {
|
|||
SplitMetaInfo[] info = new SplitMetaInfo[splits.length];
|
||||
if (splits.length != 0) {
|
||||
int i = 0;
|
||||
long offset = out.size();
|
||||
long offset = out.getPos();
|
||||
for(org.apache.hadoop.mapred.InputSplit split: splits) {
|
||||
int prevLen = out.size();
|
||||
long prevLen = out.getPos();
|
||||
Text.writeString(out, split.getClass().getName());
|
||||
split.write(out);
|
||||
int currLen = out.size();
|
||||
long currLen = out.getPos();
|
||||
info[i++] = new JobSplit.SplitMetaInfo(
|
||||
split.getLocations(), offset,
|
||||
split.getLength());
|
||||
|
|
Loading…
Reference in New Issue