Uploaded image for project: 'Pig'
  1. Pig
  2. PIG-4266 Umbrella jira for unit tests for Spark
  3. PIG-4807

Fix test cases of "TestEvalPipelineLocal" test suite.

VotersWatch issueWatchersLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Closed
    • Major
    • Resolution: Fixed
    • spark-branch
    • spark-branch
    • spark
    • None
    • Hide
      Following test cases have been fixed -
      1. org.apache.pig.test.TestEvalPipelineLocal.testSetLocationCalledInFE
      2. org.apache.pig.test.TestEvalPipelineLocal.testExplainInDotGraph
      3. org.apache.pig.test.TestEvalPipelineLocal.testSortWithUDF

      1 was failing because of not saving UDF_CONTEXT configuration in jobConf. This leads UDFContext.getUDFProperties() to return NULL.

      public Properties getUDFProperties(Class c) {
          UDFContextKey k = generateKey(c, null);
          Properties p = udfConfs.get(k);
          if (p == null) {
              p = new Properties();
              udfConfs.put(k, p);
          }
          return p;
      }

      Here, udfConfs remains empty even when it was set while processing the pig query.
      udf configuration in jobConf is getting lost while running the job.
      In the code udf configuration is meant to be saved by serializing them in jobConf.

      Currently, serialization is done before loading configuration in jobConf.
      It is done in 'newJobConf(PigContext pigContext)'
      It needs to be done after loading configuration.

      JobConf jobConf = SparkUtil.newJobConf(pigContext);
      configureLoader(physicalPlan, op, jobConf);
      UDFContext.getUDFContext().serialize(jobConf);

      2 was failing because of pig-spark not supporting 'explain' in dot format. I have added the DotSparkPrinter to fix the same.

      3 was failing because instead of UDFSortComparator, SortConveter class was using SortComparator.

      JavaPairRDD<Tuple, Object> sorted = r.sortByKey(
                      sortOperator.new SortComparator(), true);

      It should be using mComparator stored in POSort class. I have changed it to following

      JavaPairRDD<Tuple, Object> sorted = r.sortByKey(
                      sortOperator.getMComparator(), true);
      Show
      Following test cases have been fixed - 1. org.apache.pig.test.TestEvalPipelineLocal.testSetLocationCalledInFE 2. org.apache.pig.test.TestEvalPipelineLocal.testExplainInDotGraph 3. org.apache.pig.test.TestEvalPipelineLocal.testSortWithUDF 1 was failing because of not saving UDF_CONTEXT configuration in jobConf. This leads UDFContext.getUDFProperties() to return NULL. public Properties getUDFProperties(Class c) {     UDFContextKey k = generateKey(c, null);     Properties p = udfConfs.get(k);     if (p == null) {         p = new Properties();         udfConfs.put(k, p);     }     return p; } Here, udfConfs remains empty even when it was set while processing the pig query. udf configuration in jobConf is getting lost while running the job. In the code udf configuration is meant to be saved by serializing them in jobConf. Currently, serialization is done before loading configuration in jobConf. It is done in 'newJobConf(PigContext pigContext)' It needs to be done after loading configuration. JobConf jobConf = SparkUtil.newJobConf(pigContext); configureLoader(physicalPlan, op, jobConf); UDFContext.getUDFContext().serialize(jobConf); 2 was failing because of pig-spark not supporting 'explain' in dot format. I have added the DotSparkPrinter to fix the same. 3 was failing because instead of UDFSortComparator, SortConveter class was using SortComparator. JavaPairRDD<Tuple, Object> sorted = r.sortByKey(                 sortOperator.new SortComparator(), true); It should be using mComparator stored in POSort class. I have changed it to following JavaPairRDD<Tuple, Object> sorted = r.sortByKey(                 sortOperator.getMComparator(), true);

    Description

      This jira is created to address the failure of test cases
      org.apache.pig.test.TestEvalPipelineLocal.testSetLocationCalledInFE
      org.apache.pig.test.TestEvalPipelineLocal.testExplainInDotGraph
      org.apache.pig.test.TestEvalPipelineLocal.testSortWithUDF

      Attachments

        1. diff_1
          12 kB
          Prateek Vaishnav
        2. diff_2
          12 kB
          Prateek Vaishnav

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            Pratyy Prateek Vaishnav
            Pratyy Prateek Vaishnav
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment