org.apache.hadoop.mapreduce.lib.partition
Class TotalOrderPartitioner<K extends WritableComparable<?>,V>

java.lang.Object
  extended by org.apache.hadoop.mapreduce.Partitioner<K,V>
      extended by org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner<K,V>
All Implemented Interfaces:
Configurable
Direct Known Subclasses:
TotalOrderPartitioner

@InterfaceAudience.Public
@InterfaceStability.Stable
public class TotalOrderPartitioner<K extends WritableComparable<?>,V>
extends Partitioner<K,V>
implements Configurable

Partitioner effecting a total order by reading split points from an externally generated source.


Field Summary
static String DEFAULT_PATH
           
static String MAX_TRIE_DEPTH
           
static String NATURAL_ORDER
           
static String PARTITIONER_PATH
           
 
Constructor Summary
TotalOrderPartitioner()
           
 
Method Summary
 Configuration getConf()
          Return the configuration used by this object.
 int getPartition(K key, V value, int numPartitions)
          Get the partition number for a given key (hence record) given the total number of partitions i.e.
static String getPartitionFile(Configuration conf)
          Get the path to the SequenceFile storing the sorted partition keyset.
 void setConf(Configuration conf)
          Read in the partition file and build indexing data structures.
static void setPartitionFile(Configuration conf, Path p)
          Set the path to the SequenceFile storing the sorted partition keyset.
 
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
 

Field Detail

DEFAULT_PATH

public static final String DEFAULT_PATH
See Also:
Constant Field Values

PARTITIONER_PATH

public static final String PARTITIONER_PATH
See Also:
Constant Field Values

MAX_TRIE_DEPTH

public static final String MAX_TRIE_DEPTH
See Also:
Constant Field Values

NATURAL_ORDER

public static final String NATURAL_ORDER
See Also:
Constant Field Values
Constructor Detail

TotalOrderPartitioner

public TotalOrderPartitioner()
Method Detail

setConf

public void setConf(Configuration conf)
Read in the partition file and build indexing data structures. If the keytype is BinaryComparable and total.order.partitioner.natural.order is not false, a trie of the first total.order.partitioner.max.trie.depth(2) + 1 bytes will be built. Otherwise, keys will be located using a binary search of the partition keyset using the RawComparator defined for this job. The input file must be sorted with the same comparator and contain JobContextImpl.getNumReduceTasks() - 1 keys.

Specified by:
setConf in interface Configurable

getConf

public Configuration getConf()
Description copied from interface: Configurable
Return the configuration used by this object.

Specified by:
getConf in interface Configurable

getPartition

public int getPartition(K key,
                        V value,
                        int numPartitions)
Description copied from class: Partitioner
Get the partition number for a given key (hence record) given the total number of partitions i.e. number of reduce-tasks for the job.

Typically a hash function on a all or a subset of the key.

Specified by:
getPartition in class Partitioner<K extends WritableComparable<?>,V>
Parameters:
key - the key to be partioned.
value - the entry value.
numPartitions - the total number of partitions.
Returns:
the partition number for the key.

setPartitionFile

public static void setPartitionFile(Configuration conf,
                                    Path p)
Set the path to the SequenceFile storing the sorted partition keyset. It must be the case that for R reduces, there are R-1 keys in the SequenceFile.


getPartitionFile

public static String getPartitionFile(Configuration conf)
Get the path to the SequenceFile storing the sorted partition keyset.

See Also:
setPartitionFile(Configuration, Path)


Copyright © 2014 Apache Software Foundation. All Rights Reserved.