Uploaded image for project: 'Pig'
  1. Pig
  2. PIG-3446 Umbrella jira for Pig on Tez
  3. PIG-3959

Skewed join followed by replicated join fails in Tez

    XMLWordPrintableJSON

Details

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

    Description

      To reproduce the issue, run the following query-

      x = LOAD 'foo' AS (x:int, y:chararray);
      y = LOAD 'bar' AS (x:int, y:chararray);
      a = JOIN x BY x, y BY x USING 'skewed';
      z = LOAD 'zoo' AS (x:int, y:chararray);
      b = JOIN a BY x::x, z BY x USING 'replicated';
      DUMP b;
      

      This fails at runtime with the following error-

      : Container released by application, AttemptID:attempt_1399657418038_0357_1_04_000000_3 Info:Error: org.apache.pig.backend.executionengine.ExecException: ERROR 2135: Received error from POLocalRearrage function.wrong key class: class org.apache.pig.impl.io.NullableIntWritable is not class org.apache.pig.impl.io.NullablePartitionWritable
      : at org.apache.pig.backend.hadoop.executionengine.tez.POLocalRearrangeTez.getNextTuple(POLocalRearrangeTez.java:175)
      : at org.apache.pig.backend.hadoop.executionengine.tez.PigProcessor.runPipeline(PigProcessor.java:276)
      : at org.apache.pig.backend.hadoop.executionengine.tez.PigProcessor.run(PigProcessor.java:175)
      : at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:307)
      : at org.apache.hadoop.mapred.YarnTezDagChild$5.run(YarnTezDagChild.java:581)
      : at java.security.AccessController.doPrivileged(Native Method)
      : at javax.security.auth.Subject.doAs(Subject.java:415)
      : at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1548)
      : at org.apache.hadoop.mapred.YarnTezDagChild.main(YarnTezDagChild.java:570)
      : Caused by: java.io.IOException: wrong key class: class org.apache.pig.impl.io.NullableIntWritable is not class org.apache.pig.impl.io.NullablePartitionWritable
      : at org.apache.tez.runtime.library.common.sort.impl.IFile$Writer.append(IFile.java:212)
      : at org.apache.tez.runtime.library.broadcast.output.FileBasedKVWriter.write(FileBasedKVWriter.java:149)
      : at org.apache.pig.backend.hadoop.executionengine.tez.POLocalRearrangeTez.getNextTuple(POLocalRearrangeTez.java:160)
      : ... 8 more
      

      Attachments

        1. PIG-3959-1.patch
          4 kB
          Cheolsoo Park

        Activity

          People

            cheolsoo Cheolsoo Park
            cheolsoo Cheolsoo Park
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: