Uploaded image for project: 'DataFu'
  1. DataFu
  2. DATAFU-6

MonitoredUDF annotation does not work with AliasableEvalFunc

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • None
    • 1.3.2
    • None

    Description

      This was reported by seregasheypak on GitHub (https://github.com/linkedin/datafu/issues/89). We were able to reproduce this by adding the annotation to BagLeftOuterJoin and running its tests. Simply adding the annotation causes problems. In ContextualEvalFunc.getContextProperties, the properties retrieved for the class are empty.

      seregasheypak:

      Hi, If I use

      @MonitoredUDF(timeUnit = TimeUnit.MINUTES, duration = 10, errorCallback = NplRecMatcherErrorCallback.class)
      class NplRecFirstLevelMatcher extends AliasableEvalFunc<Tuple> implements DebuggableUDF{
      //some cool stuff goes here!
      }
      

      I do get exception:

      14/01/15 23:52:52 ERROR udf.NplRecFirstLevelMatcher: Class: class NplRecFirstLevelMatcher
      14/01/15 23:52:52 ERROR udf.NplRecFirstLevelMatcher: Instance name: 30
      14/01/15 23:52:52 ERROR udf.NplRecFirstLevelMatcher: Properties: {30={}}
      *** ***A debug output from my handler method***  ***
      NplRecMatcherErrorCallback.handleError
      
      null
      ERROR: java.lang.RuntimeException: Could not retrieve aliases from properties using aliasMap
      java.util.concurrent.ExecutionException: java.lang.RuntimeException: Could not retrieve aliases from properties using aliasMap
      	at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:232)
      	at java.util.concurrent.FutureTask.get(FutureTask.java:91)
      	at com.google.common.util.concurrent.ForwardingFuture.get(ForwardingFuture.java:69)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.MonitoredUDFExecutor.monitorExec(MonitoredUDFExecutor.java:183)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc.getNext(POUserFunc.java:335)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc.getNext(POUserFunc.java:376)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.getNext(PhysicalOperator.java:354)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.processPlan(POForEach.java:372)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.getNext(POForEach.java:297)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:308)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.getNext(POForEach.java:241)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:308)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFilter.getNext(POFilter.java:95)
      	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Reduce.runPipeline(PigGenericMapReduce.java:465)
      	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Reduce.processOnePackageOutput(PigGenericMapReduce.java:433)
      	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Reduce.reduce(PigGenericMapReduce.java:413)
      	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Reduce.reduce(PigGenericMapReduce.java:257)
      	at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:164)
      	at org.apache.hadoop.mapred.ReduceTask.runNewReducer(ReduceTask.java:610)
      	at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:444)
      	at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:449)
      Caused by: java.lang.RuntimeException: Could not retrieve aliases from properties using aliasMap
      	at datafu.pig.util.AliasableEvalFunc.getFieldAliases(AliasableEvalFunc.java:164)
      	at datafu.pig.util.AliasableEvalFunc.getPosition(AliasableEvalFunc.java:171)
      	at datafu.pig.util.AliasableEvalFunc.getBag(AliasableEvalFunc.java:253)
      	at datafu.pig.util.AliasableEvalFunc$getBag.callCurrent(Unknown Source)
      	at org.codehaus.groovy.runtime.callsite.CallSiteArray.defaultCallCurrent(CallSiteArray.java:49)
      	at org.codehaus.groovy.runtime.callsite.AbstractCallSite.callCurrent(AbstractCallSite.java:133)
      	at org.codehaus.groovy.runtime.callsite.AbstractCallSite.callCurrent(AbstractCallSite.java:145)
      	at NplRecFirstLevelMatcher.exec(NplRecFirstLevelMatcher.groovy:53)
      	at NplRecFirstLevelMatcher.exec(NplRecFirstLevelMatcher.groovy)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.MonitoredUDFExecutor$1.apply(MonitoredUDFExecutor.java:95)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.MonitoredUDFExecutor$1.apply(MonitoredUDFExecutor.java:91)
      	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.MonitoredUDFExecutor$2.call(MonitoredUDFExecutor.java:164)
      	at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:138)
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:439)
      	at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:138)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:98)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:206)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
      	at java.lang.Thread.run(Thread.java:662)
      

      Exception happens on line:

      itemsBag = getBag(input, ORDERED)
      

      If I put away annotation @MonitoredUDF, it works fine, tests are passed.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              mhayes Matthew Hayes
              Votes:
              1 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: