Uploaded image for project: 'Pig'
  1. Pig
  2. PIG-4059 Pig on Spark
  3. PIG-4565

Support custom MR partitioners for Spark engine

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Closed
    • Major
    • Resolution: Fixed
    • spark-branch
    • spark-branch
    • spark
    • None

    Description

      Shuffle operations like DISTINCT, GROUP, JOIN, CROSS allow custom MR partitioners to be specified.

      Example:

      B = GROUP A BY $0 PARTITION BY org.apache.pig.test.utils.SimpleCustomPartitioner PARALLEL 2;
      
      public class SimpleCustomPartitioner extends Partitioner <PigNullableWritable, Writable> { 
           //@Override 
          public int getPartition(PigNullableWritable key, Writable value, int numPartitions) { 
              if(key.getValueAsPigType() instanceof Integer) { 
                  int ret = (((Integer)key.getValueAsPigType()).intValue() % numPartitions); 
                  return ret; 
             } 
             else { 
                  return (key.hashCode()) % numPartitions; 
              } 
          } 
      }
      

      Since Spark's shuffle APIs takes a different parititioner class (org.apache.spark.Partitioner) compared to MapReduce (org.apache.hadoop.mapreduce.Partitioner), we need to wrap custom partitioners written for MapReduce inside a Spark Partitioner.

      Attachments

        1. PIG-4565.patch
          29 kB
          Mohit Sabharwal
        2. PIG-4565.1.patch
          30 kB
          Mohit Sabharwal
        3. PIG-4565.2.patch
          31 kB
          Mohit Sabharwal
        4. PIG-4565.3.patch
          32 kB
          Mohit Sabharwal

        Issue Links

          Activity

            People

              mohitsabharwal Mohit Sabharwal
              mohitsabharwal Mohit Sabharwal
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: