You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@oozie.apache.org by Christian Gapp <cs...@uibk.ac.at> on 2014/10/03 16:47:41 UTC
Custom async action
Hello everybody,
I have a problem implementing a custom asynchronnous action. My class
extends from ActionExecutor, and overwrites the methods initActionType,
start, end, check, kill and isCompleted.
In the start method, i want to to start a YARN job, that is implemented
through my BiohadoopClient class. To make the call asynchronous, i
wrapped the client.run() method in a Callable:
public void start(final Context context, final WorkflowAction action) {
...
Callable<String> biohadoop = new Callable<String>() {
BiohadoopClient client = new BiohadoopClient();
client.run();
}
// submit callable to executor
executor.submit(biohadoop);
// set the start data, according to
https://oozie.apache.org/docs/4.0.1/DG_CustomActionExecutor.html
context.setStartData(externalId, callBackUrl, callBackUrl);
...
}
This works fine, and for example when I use my custom action in a
fork/join manner, the execution of the actions runs in parallel.
Now, the problem is, that Oozie remains in a RUNNING state for this
actions. It seems impossible to change that to a completed state. The
check() method is never called by Oozie, the same is true for the end()
method. It doesn't help to set the context.setExternalStatus(),
context.setExecutionData() and context.setEndData() manually in the
Callable (after the client.run() has finished). I tried also to queue
manually an ActionEndXCommand, but without luck.
When I wait in the start() method for the Callable to complete, the
state gets updated correctly, but the execution in fork/join isn't
parallel anymore (which seem logic, as the execution waits for the
Callable to complete).
So far I haven't found any example howto write an asynchronous custom
action, can anybody please help me?
Thank you
Christian
Re: Custom async action
Posted by Christian Gapp <cs...@uibk.ac.at>.
Hi Robert,
first, thank you for your input, I'm searching really hard to find a
solution and am thankful for any kind of help. But still, no luck...
Maybe I should explain, what I'm trying to achieve: I have a Hadoop YARN
program (Biohadoop), that is able to communicate with other running
instances of Biohadoop to exchange data. Therefor, I want to start
several Biohadoop instances in parallel. I want to achieve this with
Oozie, by running a custom action that starts Biohadoop, and use several
of this custom actions in a fork/join action. An example workflow.xml
looks like this:
<workflow-app xmlns="uri:oozie:workflow:0.2" name="biohadoop-wf">
<start to="biohadoop-node"/>
<fork name="biohadoop-node">
<path start="biohadoop-node1"/>
<path start="biohadoop-node2"/>
</fork>
<action name="biohadoop-node1">
<biohadoop xmlns="uri:custom:biohadoop-action:0.1">
<config-file>/biohadoop/conf/biohadoop-echo.json</config-file>
</biohadoop>
<ok to="biohadoop-join"/>
<error to="fail"/>
</action>
<action name="biohadoop-node2">
<biohadoop xmlns="uri:custom:biohadoop-action:0.1">
<config-file>/biohadoop/conf/biohadoop-echo.json</config-file>
</biohadoop>
<ok to="biohadoop-join"/>
<error to="fail"/>
</action>
<join name="biohadoop-join" to="end"/>
<kill name="fail">
<message>Biohadoop failed, error
message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<end name="end"/>
</workflow-app>
What I expect from this workflow is, that two instances of Biohadoop are
started in parallel (by fork), and then, after they have finished, Oozie
advances to the join action. This is the reason, why I'm starting
Biohadoop in a Callable. If Biohadoop is not started in a Callable, the
two nodes (biohadoop-node1 and biohadoop-node2) run sequentially: at the
beginning, the first node is executed and waits for Biohadoop to return,
after which the second node is executed. When executing the workflow in
this way (no Callable), Oozie sets the finish state correctly - but the
two Biohadoop instances don't run in parallel. The problem is, that if I
start Biohadoop in a Callable, the check() and end() methods are never
called by Oozie, the two nodes remain in a RUNNING state indefinitely.
About your solutions:
1) On several occasions, I waited for more than 15 minutes for the
Biohadoop actions to change to the finished state, but had no luck
2) I issued a GET request to the callback URL using a webbrowser. The
URL is taken from the method context.getCallbackUrl("finished"), that is
invoked in the start() method. An example URL is
http://master:11000/oozie/callback?id=0000001-141011154758957-oozie-root-W@biohadoop-node1&status=finished&
- does this seem correct? I ask this, because I can see in the log files
that something strange happens when I invoke this URL: it seems that a
new instance of BiohadoopAction class is created (BiohadoopAction is the
implementing class for the custom Oozie action), but neither check() nor
end() are called.
I also tried to use the callback URL mentioned above, and wait for 20
minutes for the action to finish, but it didn't change anything.
Unfortunately, calling check() and end() directly didn't help either. I
think I have two solutions: try to extend the Java action, or make a
custom action, that runs several Biohadoop instances in parallel. What
is your opinion?
One more question: how are the asynchronous custom actions supposed to
work? This would be very interesting, as there seems to be no
information about this topic (at least as far as I can say)
If you want to take a look at my implementation, you can find it at
https://github.com/gappc/biooozie
Again, thank you for your help
Christian
Am 2014-10-10 um 01:40 schrieb Robert Kanter:
> Hi Christian,
>
> There are two main things that trigger Oozie to call the check() method:
> 1) There's a time interval at which it calls check(). This gets called by
> ActionCheckXCommand, which gets queued by the ActionCheckerService. By
> default, the service queues up ActionCheckXCommands every minute but only
> per action every 10 minutes. In other words, it can take up to 10-11
> minutes for check() to be called. The reason it's 10 minutes is that this
> is considered a fallback procedure in case #2 doesn't work (see below).
> 2) Oozie has a CallbackServlet and CallbackService, which are responsible
> for listening for an HTTP callback from completed actions. This triggers a
> call to check() to verify that it has indeed finished (and isn't lying).
> You'll want to make sure that something does a GET on the correct callback
> URL (CallbackService can generate it for you).
>
> By an asynchronous action, do you mean that your action calls
> something in BiohadoopClient
> and you want Oozie to continue with the next action in the workflow,
> without waiting for BiohadoopClient to complete whatever it's doing?
> Oozie's current actions aren't really designed to work that way, and it
> might be simpler to just use the Java action if that's all you want to do.
> That said, if you want Oozie to continue while BiohadoopClient is still
> running, you can probably just call check() or end() directly from start().
>
>
> - Robert
>
> On Thu, Oct 9, 2014 at 7:46 AM, Christian Gapp <cs...@uibk.ac.at> wrote:
>
>> Hello,
>>
>> has somebody so far a solution for this problem? Or at least hints?
>>
>> Christian
>>
>> Am 2014-10-03 um 16:47 schrieb Christian Gapp:
>>
>> Hello everybody,
>>> I have a problem implementing a custom asynchronnous action. My class
>>> extends from ActionExecutor, and overwrites the methods initActionType,
>>> start, end, check, kill and isCompleted.
>>>
>>> In the start method, i want to to start a YARN job, that is implemented
>>> through my BiohadoopClient class. To make the call asynchronous, i wrapped
>>> the client.run() method in a Callable:
>>>
>>> public void start(final Context context, final WorkflowAction action) {
>>> ...
>>> Callable<String> biohadoop = new Callable<String>() {
>>> BiohadoopClient client = new BiohadoopClient();
>>> client.run();
>>> }
>>>
>>> // submit callable to executor
>>> executor.submit(biohadoop);
>>>
>>> // set the start data, according to https://oozie.apache.org/docs/
>>> 4.0.1/DG_CustomActionExecutor.html
>>> context.setStartData(externalId, callBackUrl, callBackUrl);
>>> ...
>>> }
>>>
>>> This works fine, and for example when I use my custom action in a
>>> fork/join manner, the execution of the actions runs in parallel.
>>>
>>> Now, the problem is, that Oozie remains in a RUNNING state for this
>>> actions. It seems impossible to change that to a completed state. The
>>> check() method is never called by Oozie, the same is true for the end()
>>> method. It doesn't help to set the context.setExternalStatus(),
>>> context.setExecutionData() and context.setEndData() manually in the
>>> Callable (after the client.run() has finished). I tried also to queue
>>> manually an ActionEndXCommand, but without luck.
>>>
>>> When I wait in the start() method for the Callable to complete, the state
>>> gets updated correctly, but the execution in fork/join isn't parallel
>>> anymore (which seem logic, as the execution waits for the Callable to
>>> complete).
>>>
>>> So far I haven't found any example howto write an asynchronous custom
>>> action, can anybody please help me?
>>>
>>> Thank you
>>> Christian
>>>
>>>
Re: Custom async action
Posted by Robert Kanter <rk...@cloudera.com>.
Hi Christian,
There are two main things that trigger Oozie to call the check() method:
1) There's a time interval at which it calls check(). This gets called by
ActionCheckXCommand, which gets queued by the ActionCheckerService. By
default, the service queues up ActionCheckXCommands every minute but only
per action every 10 minutes. In other words, it can take up to 10-11
minutes for check() to be called. The reason it's 10 minutes is that this
is considered a fallback procedure in case #2 doesn't work (see below).
2) Oozie has a CallbackServlet and CallbackService, which are responsible
for listening for an HTTP callback from completed actions. This triggers a
call to check() to verify that it has indeed finished (and isn't lying).
You'll want to make sure that something does a GET on the correct callback
URL (CallbackService can generate it for you).
By an asynchronous action, do you mean that your action calls
something in BiohadoopClient
and you want Oozie to continue with the next action in the workflow,
without waiting for BiohadoopClient to complete whatever it's doing?
Oozie's current actions aren't really designed to work that way, and it
might be simpler to just use the Java action if that's all you want to do.
That said, if you want Oozie to continue while BiohadoopClient is still
running, you can probably just call check() or end() directly from start().
- Robert
On Thu, Oct 9, 2014 at 7:46 AM, Christian Gapp <cs...@uibk.ac.at> wrote:
> Hello,
>
> has somebody so far a solution for this problem? Or at least hints?
>
> Christian
>
> Am 2014-10-03 um 16:47 schrieb Christian Gapp:
>
> Hello everybody,
>>
>> I have a problem implementing a custom asynchronnous action. My class
>> extends from ActionExecutor, and overwrites the methods initActionType,
>> start, end, check, kill and isCompleted.
>>
>> In the start method, i want to to start a YARN job, that is implemented
>> through my BiohadoopClient class. To make the call asynchronous, i wrapped
>> the client.run() method in a Callable:
>>
>> public void start(final Context context, final WorkflowAction action) {
>> ...
>> Callable<String> biohadoop = new Callable<String>() {
>> BiohadoopClient client = new BiohadoopClient();
>> client.run();
>> }
>>
>> // submit callable to executor
>> executor.submit(biohadoop);
>>
>> // set the start data, according to https://oozie.apache.org/docs/
>> 4.0.1/DG_CustomActionExecutor.html
>> context.setStartData(externalId, callBackUrl, callBackUrl);
>> ...
>> }
>>
>> This works fine, and for example when I use my custom action in a
>> fork/join manner, the execution of the actions runs in parallel.
>>
>> Now, the problem is, that Oozie remains in a RUNNING state for this
>> actions. It seems impossible to change that to a completed state. The
>> check() method is never called by Oozie, the same is true for the end()
>> method. It doesn't help to set the context.setExternalStatus(),
>> context.setExecutionData() and context.setEndData() manually in the
>> Callable (after the client.run() has finished). I tried also to queue
>> manually an ActionEndXCommand, but without luck.
>>
>> When I wait in the start() method for the Callable to complete, the state
>> gets updated correctly, but the execution in fork/join isn't parallel
>> anymore (which seem logic, as the execution waits for the Callable to
>> complete).
>>
>> So far I haven't found any example howto write an asynchronous custom
>> action, can anybody please help me?
>>
>> Thank you
>> Christian
>>
>>
>
Re: Custom async action
Posted by Christian Gapp <cs...@uibk.ac.at>.
Hello,
has somebody so far a solution for this problem? Or at least hints?
Christian
Am 2014-10-03 um 16:47 schrieb Christian Gapp:
> Hello everybody,
>
> I have a problem implementing a custom asynchronnous action. My class
> extends from ActionExecutor, and overwrites the methods
> initActionType, start, end, check, kill and isCompleted.
>
> In the start method, i want to to start a YARN job, that is
> implemented through my BiohadoopClient class. To make the call
> asynchronous, i wrapped the client.run() method in a Callable:
>
> public void start(final Context context, final WorkflowAction action) {
> ...
> Callable<String> biohadoop = new Callable<String>() {
> BiohadoopClient client = new BiohadoopClient();
> client.run();
> }
>
> // submit callable to executor
> executor.submit(biohadoop);
>
> // set the start data, according to
> https://oozie.apache.org/docs/4.0.1/DG_CustomActionExecutor.html
> context.setStartData(externalId, callBackUrl, callBackUrl);
> ...
> }
>
> This works fine, and for example when I use my custom action in a
> fork/join manner, the execution of the actions runs in parallel.
>
> Now, the problem is, that Oozie remains in a RUNNING state for this
> actions. It seems impossible to change that to a completed state. The
> check() method is never called by Oozie, the same is true for the
> end() method. It doesn't help to set the context.setExternalStatus(),
> context.setExecutionData() and context.setEndData() manually in the
> Callable (after the client.run() has finished). I tried also to queue
> manually an ActionEndXCommand, but without luck.
>
> When I wait in the start() method for the Callable to complete, the
> state gets updated correctly, but the execution in fork/join isn't
> parallel anymore (which seem logic, as the execution waits for the
> Callable to complete).
>
> So far I haven't found any example howto write an asynchronous custom
> action, can anybody please help me?
>
> Thank you
> Christian
>