Author: cutting Date: Thu Sep 6 14:19:16 2007 New Revision: 573383 URL: http://svn.apache.org/viewvc?rev=573383&view=rev Log: HADOOP-1817. Fix MultiFileSplit to read and write the split length, so that it is not always zero in map tasks. Contributed by Thomas Friol.
Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestMultiFileSplit.java Modified: lucene/hadoop/trunk/CHANGES.txt lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MultiFileSplit.java Modified: lucene/hadoop/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=573383&r1=573382&r2=573383&view=diff ============================================================================== --- lucene/hadoop/trunk/CHANGES.txt (original) +++ lucene/hadoop/trunk/CHANGES.txt Thu Sep 6 14:19:16 2007 @@ -92,6 +92,10 @@ HADOOP-1758. Fix escape processing in librecordio to not be quadratic. (Vivek Ratan via cutting) + HADOOP-1817. Fix MultiFileSplit to read and write the split + length, so that it is not always zero in map tasks. + (Thomas Friol via cutting) + IMPROVEMENTS HADOOP-1779. Replace INodeDirectory.getINode() by a getExistingPathINodes() Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MultiFileSplit.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MultiFileSplit.java?rev=573383&r1=573382&r2=573383&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MultiFileSplit.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MultiFileSplit.java Thu Sep 6 14:19:16 2007 @@ -90,7 +90,7 @@ HashSet<String> hostSet = new HashSet<String>(); for (Path file : paths) { String[][] hints = FileSystem.get(job) - .getFileCacheHints(file, 0, FileSystem.get(job).getLength(file)); + .getFileCacheHints(file, 0, FileSystem.get(job).getFileStatus(file).getLen()); if (hints != null && hints.length > 0) { addToSet(hostSet, hints[0]); } @@ -104,6 +104,7 @@ } public void readFields(DataInput in) throws IOException { + totLength = in.readLong(); int arrLength = in.readInt(); lengths = new long[arrLength]; for(int i=0; i<arrLength;i++) { @@ -117,6 +118,7 @@ } public void write(DataOutput out) throws IOException { + out.writeLong(totLength); out.writeInt(lengths.length); for(long length : lengths) out.writeLong(length); @@ -124,6 +126,19 @@ for(Path p : paths) { Text.writeString(out, p.toString()); } + } + + @Override + public String toString() { + StringBuffer sb = new StringBuffer(); + for(int i=0; i < paths.length; i++) { + sb.append(paths[i].toUri().getPath() + ":0+" + lengths[i]); + if (i < paths.length -1) { + sb.append("\n"); + } + } + + return sb.toString(); } } Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestMultiFileSplit.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestMultiFileSplit.java?rev=573383&view=auto ============================================================================== --- lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestMultiFileSplit.java (added) +++ lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestMultiFileSplit.java Thu Sep 6 14:19:16 2007 @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.mapred; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.util.Arrays; + +import junit.framework.TestCase; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IOUtils; + +public class TestMultiFileSplit extends TestCase{ + + public void testReadWrite() throws Exception { + MultiFileSplit split = new MultiFileSplit(new JobConf(), new Path[] {new Path("/test/path/1"), new Path("/test/path/2")}, new long[] {100,200}); + + ByteArrayOutputStream bos = null; + byte[] result = null; + try { + bos = new ByteArrayOutputStream(); + split.write(new DataOutputStream(bos)); + result = bos.toByteArray(); + } finally { + IOUtils.closeStream(bos); + } + + MultiFileSplit readSplit = new MultiFileSplit(); + ByteArrayInputStream bis = null; + try { + bis = new ByteArrayInputStream(result); + readSplit.readFields(new DataInputStream(bis)); + } finally { + IOUtils.closeStream(bis); + } + + assertTrue(split.getLength() != 0); + assertEquals(split.getLength(), readSplit.getLength()); + assertTrue(Arrays.equals(split.getPaths(), readSplit.getPaths())); + assertTrue(Arrays.equals(split.getLengths(), readSplit.getLengths())); + System.out.println(split.toString()); + } +}