Hey Steven,

This is a genuine bug in MR. I've filed
https://issues.apache.org/jira/browse/MAPREDUCE-4574 and will work on
it by this week's end.

On Thu, Aug 16, 2012 at 3:03 AM, Steven Willis <[email protected]> wrote:
> Hi,
>
> I was wondering if it was possible to implement a total sort using the 
> InputSampler.RandomSampler and TotalOrderPartitioner with avro mapreduce? I 
> tried adding the following lines to my job:
>
> InputSampler.Sampler<AvroKey, AvroValue> sampler = new 
> InputSampler.RandomSampler<AvroKey, AvroValue>(0.1, 10000, 10);
> InputSampler.writePartitionFile(jobConf, sampler);
> jobConf.setPartitionerClass(TotalOrderPartitioner.class);
> DistributedCache.addCacheFile(new 
> URI(TotalOrderPartitioner.getPartitionFile(jobConf)), jobConf);
>
> But that just gives me:
>
> 12/08/15 17:23:05 INFO partition.InputSampler: Using 10000 samples
> Exception in thread "main" java.lang.ClassCastException: 
> org.apache.hadoop.io.LongWritable cannot be cast to 
> org.apache.avro.mapred.AvroWrapper
>         at 
> org.apache.avro.mapred.AvroKeyComparator.compare(AvroKeyComparator.java:30)
>         at java.util.Arrays.mergeSort(Arrays.java:1270)
>         at java.util.Arrays.mergeSort(Arrays.java:1281)
>         at java.util.Arrays.mergeSort(Arrays.java:1281)
>         at java.util.Arrays.mergeSort(Arrays.java:1281)
>         at java.util.Arrays.mergeSort(Arrays.java:1281)
>         at java.util.Arrays.mergeSort(Arrays.java:1281)
>         at java.util.Arrays.mergeSort(Arrays.java:1281)
>         at java.util.Arrays.mergeSort(Arrays.java:1281)
>         at java.util.Arrays.mergeSort(Arrays.java:1281)
>         at java.util.Arrays.mergeSort(Arrays.java:1281)
>         at java.util.Arrays.mergeSort(Arrays.java:1281)
>         at java.util.Arrays.mergeSort(Arrays.java:1281)
>         at java.util.Arrays.sort(Arrays.java:1210)
>         at 
> org.apache.hadoop.mapreduce.lib.partition.InputSampler.writePartitionFile(InputSampler.java:324)
>         at 
> org.apache.hadoop.mapred.lib.InputSampler.writePartitionFile(InputSampler.java:39)
>         at com.compete.avro.ParallelDataPull.run(ParallelDataPull.java:223)
>         at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70)
>         at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:84)
>         at com.compete.avro.ParallelDataPull.main(ParallelDataPull.java:55)
>         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:597)
>         at org.apache.hadoop.util.RunJar.main(RunJar.java:208)
>
> -Steven Willis



-- 
Harsh J

Reply via email to