MAPREDUCE-6287. Deprecated methods in org.apache.hadoop.examples.Sort. Contributed by Chao Zhang.

(cherry picked from commit b375d1fc93)
This commit is contained in:
Harsh J 2015-03-23 03:48:36 +05:30
parent 6e6e0e4a37
commit 2cd2431289
2 changed files with 7 additions and 3 deletions

View File

@ -8,6 +8,9 @@ Release 2.8.0 - UNRELEASED
IMPROVEMENTS IMPROVEMENTS
MAPREDUCE-6287. Deprecated methods in org.apache.hadoop.examples.Sort
(Chao Zhang via harsh)
MAPREDUCE-5190. Unnecessary condition test in RandomSampler. MAPREDUCE-5190. Unnecessary condition test in RandomSampler.
(Jingguo Yao via harsh) (Jingguo Yao via harsh)

View File

@ -24,7 +24,7 @@ import java.util.*;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured; import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.mapreduce.filecache.DistributedCache; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.Writable;
@ -160,13 +160,14 @@ public class Sort<K,V> extends Configured implements Tool {
System.out.println("Sampling input to effect total-order sort..."); System.out.println("Sampling input to effect total-order sort...");
job.setPartitionerClass(TotalOrderPartitioner.class); job.setPartitionerClass(TotalOrderPartitioner.class);
Path inputDir = FileInputFormat.getInputPaths(job)[0]; Path inputDir = FileInputFormat.getInputPaths(job)[0];
inputDir = inputDir.makeQualified(inputDir.getFileSystem(conf)); FileSystem fs = inputDir.getFileSystem(conf);
inputDir = inputDir.makeQualified(fs.getUri(), fs.getWorkingDirectory());
Path partitionFile = new Path(inputDir, "_sortPartitioning"); Path partitionFile = new Path(inputDir, "_sortPartitioning");
TotalOrderPartitioner.setPartitionFile(conf, partitionFile); TotalOrderPartitioner.setPartitionFile(conf, partitionFile);
InputSampler.<K,V>writePartitionFile(job, sampler); InputSampler.<K,V>writePartitionFile(job, sampler);
URI partitionUri = new URI(partitionFile.toString() + URI partitionUri = new URI(partitionFile.toString() +
"#" + "_sortPartitioning"); "#" + "_sortPartitioning");
DistributedCache.addCacheFile(partitionUri, conf); job.addCacheFile(partitionUri);
} }
System.out.println("Running on " + System.out.println("Running on " +