Hi Airavata Devs, I'm going to update staging branch with the suggestions provided by the Helix team and changes will be immediately reflected on Testing environment. If we are confident with the stability of the framework, we will apply them to Staging environment.
Here are the changes I'm going to make 1. Stopping cleanup agent that externally monitors and clears completed and failed workflows. 2. Setting and expiry time of 30 minutes in workflow config as a flag for Helix to delete the workflows after they are completed. Thanks Dimuthu On Tue, Nov 13, 2018 at 10:31 AM DImuthu Upeksha <[email protected]> wrote: > Hi Junkai, > > Thanks a lot. I'll try with expiry time then. Is this[1] the place where > Helix has implemented this logic? If that so, default expiry time should be > 24 hours. Am I right? > > [1] > https://github.com/apache/helix/blob/master/helix-core/src/main/java/org/apache/helix/task/TaskUtil.java#L711 > > Thanks > Dimuthu > > On Mon, Nov 12, 2018 at 10:17 PM Xue Junkai <[email protected]> wrote: > >> 1 and 2 are correct. 3 is wrong. The expiry time start counting only when >> the workflow is completed. If it is not scheduled ( dont have enought >> resource) or still running, Helix never deletes it. >> >> >> >> On Sun, Nov 11, 2018 at 8:01 PM DImuthu Upeksha < >> [email protected]> wrote: >> >>> Hi Junkai, >>> >>> Thanks for the clarification. That helped a lot. >>> >>> I our case, each of the task of the workflow are depending on the >>> previous >>> task. So there is no parallel execution. And we are not using Job Queues. >>> >>> Regarding the expiry time, what are the rules that you are imposing on >>> that? For example let's say I setup an expiry time to 2 hours, I assume >>> following situations are covered in Helix, >>> >>> 1. Even though the workflow is completed before 2 hours, resources >>> related >>> to that workflow will not be cleared until 2 hours are elapsed and >>> exactly >>> after 2 hours, all the resources will be cleared by the framework. >>> 2. If the workflow failed, resources will not be cleared even after 2 >>> hours >>> 3. If the workflow wasn't scheduled within 2 hours in a participant, it >>> will be deleted >>> >>> Is my understanding correct? >>> >>> Thanks >>> Dimuthu >>> >>> >>> On Sat, Nov 10, 2018 at 4:26 PM Xue Junkai <[email protected]> wrote: >>> >>> > Hi Dimuthu, >>> > >>> > Couple things here: >>> > 1. Only JobQueue in Helix is single branch DAG and 1 job running at a >>> time >>> > with defining parallel job number to be 1. Otherwise, you may see many >>> jobs >>> > running at same time as you set parallel job number to be a different >>> > number. For generic workflow, all jobs without dependencies could be >>> > dispatched together. >>> > 2. Helix only cleans up the completed generic workflows by deleting all >>> > the related znode, not for JobQueue. For JobQueue you have to set up >>> > periodical purge time. As Helix defined, JobQueue never finishes and >>> only >>> > can be terminated by manual kill and it can keep accepting dynamic >>> jobs. >>> > Thus you have to understand your workflow is generic workflow or >>> JobQueue. >>> > For failed generic workflow, even if you setup the expiry time, Helix >>> will >>> > not clean it up as Helix would like to keep it for user further >>> > investigation. >>> > 3. For Helix controller, if Helix failed to clean up workflows, the >>> only >>> > thing you can see is the having workflows with context but no resource >>> > config and idealstate there. This is because of ZK write fail to clean >>> last >>> > piece, context node. And there is no ideal state can trigger clean up >>> again >>> > for this workflow. >>> > >>> > Please take a look for this task framework tutorial for detailed >>> > configurations: >>> > https://helix.apache.org/0.8.2-docs/tutorial_task_framework.html >>> > >>> > Best, >>> > >>> > Junkai >>> > >>> > On Sat, Nov 10, 2018 at 8:29 AM DImuthu Upeksha < >>> > [email protected]> wrote: >>> > >>> >> Hi Junkai, >>> >> >>> >> Thanks for the clarification. There are few special properties in our >>> >> workflows. All the workflows are single branch DAGs so there will be >>> only >>> >> one job running at a time. By looking at the log, I could see that >>> only >>> >> the >>> >> task with this error has been failed. Cleanup agent deleted this >>> workflow >>> >> after this task is failed so it is clear that no other task is >>> triggering >>> >> this issue (I checked the timestamp). >>> >> >>> >> However for the instance, I disabled the cleanup agent for a while. >>> Reason >>> >> for adding this agent is because Helix became slow to schedule pending >>> >> jobs >>> >> when the load is high and participant was waiting without running >>> anything >>> >> for few minutes. We discussed this on thread "Sporadic delays in task >>> >> execution". Before implementing this agent, I noticed that, there were >>> >> lots >>> >> of uncleared znodes related to Completed and Failed workflows and I >>> though >>> >> that was the reason to slow down controller / participant. After >>> >> implementing this agent, things went smoothly until this point. >>> >> >>> >> Now I understand that you have your own workflow cleanup logic >>> implemented >>> >> in Helix but we might need to tune it to our case. Can you point me >>> into >>> >> code / documentation where I can have an idea about that? >>> >> >>> >> And this for my understanding, let's say that for some reason Helix >>> failed >>> >> to clean up completed workflows and related resources in zk. Will that >>> >> affect to the performance of controller / participant? My >>> understanding >>> >> was >>> >> that Helix was registering zk watchers for all the paths irrespective >>> of >>> >> the status of the workflow/ job/ task. Please correct me if I'm wrong. >>> >> >>> >> Thanks >>> >> Dimuthu >>> >> >>> >> On Sat, Nov 10, 2018 at 1:49 AM Xue Junkai <[email protected]> >>> wrote: >>> >> >>> >> > It is possible. For example, if other jobs caused the workflow >>> failed, >>> >> it >>> >> > will trigger the monitoring to clean up the workflow. Then if this >>> job >>> >> is >>> >> > still running, you may see the problem. That's what I am trying to >>> ask >>> >> for, >>> >> > extra thread deleting/cleaning workflows. >>> >> > >>> >> > I can understand it clean up the failed workflow. But I am >>> wondering why >>> >> > not just set expiry and let Helix controller does the clean up for >>> >> > completed workflows. >>> >> > >>> >> > On Sat, Nov 10, 2018 at 1:30 PM DImuthu Upeksha < >>> >> > [email protected]> wrote: >>> >> > >>> >> >> Hi Junkai, >>> >> >> >>> >> >> There is a cleanup agent [1] who is monitoring currently available >>> >> >> workflows and deleting completed and failed workflows to clear up >>> >> >> zookeeper >>> >> >> storage. Do you think that this will be causing this issue? >>> >> >> >>> >> >> [1] >>> >> >> >>> >> >> >>> >> >>> https://github.com/apache/airavata/blob/staging/modules/airavata-helix/helix-spectator/src/main/java/org/apache/airavata/helix/impl/controller/WorkflowCleanupAgent.java >>> >> >> >>> >> >> Thanks >>> >> >> Dimuthu >>> >> >> >>> >> >> On Fri, Nov 9, 2018 at 11:14 PM DImuthu Upeksha < >>> >> >> [email protected]> >>> >> >> wrote: >>> >> >> >>> >> >> > Hi Junkai, >>> >> >> > >>> >> >> > There is no manual workflow killing logic implemented but as you >>> have >>> >> >> > suggested, I need to verify that. Unfortunately all the helix log >>> >> >> levels in >>> >> >> > our servers were set to WARN as helix is printing a whole lot of >>> >> logs in >>> >> >> > INFO level so there is no much valuable information in logs. Can >>> you >>> >> >> > specify which class is printing logs associated for workflow >>> >> termination >>> >> >> > and I'll enable DEBUG level for that class and observe further. >>> >> >> > >>> >> >> > Thanks >>> >> >> > Dimuthu >>> >> >> > >>> >> >> > On Fri, Nov 9, 2018 at 9:18 PM Xue Junkai <[email protected]> >>> >> wrote: >>> >> >> > >>> >> >> >> Hmm, that's very strange. The user content store znode only has >>> been >>> >> >> >> deleted when the workflow is gone. From the log, it shows the >>> znode >>> >> is >>> >> >> >> gone. Could you please try to dig the log to find whether the >>> >> workflow >>> >> >> has >>> >> >> >> been manually killed? If that's the case, then it is possible >>> you >>> >> have >>> >> >> the >>> >> >> >> problem. >>> >> >> >> >>> >> >> >> On Fri, Nov 9, 2018 at 12:13 PM DImuthu Upeksha < >>> >> >> >> [email protected]> >>> >> >> >> wrote: >>> >> >> >> >>> >> >> >> > Hi Junkai, >>> >> >> >> > >>> >> >> >> > Thanks for your suggestion. You have captured most of the >>> parts >>> >> >> >> correctly. >>> >> >> >> > There are two jobs as job1 and job2. And there is a dependency >>> >> that >>> >> >> job2 >>> >> >> >> > depends on job1. Until job1 is completed job2 should not be >>> >> >> scheduled. >>> >> >> >> And >>> >> >> >> > task 1 in job 1 is calling that method and it is not updating >>> >> >> anyone's >>> >> >> >> > content. It's just putting and value in workflow level. What >>> do >>> >> you >>> >> >> >> mean my >>> >> >> >> > keeping a key-value store in workflow level? I already use >>> that >>> >> key >>> >> >> >> value >>> >> >> >> > store given by helix by calling putUserContent method. >>> >> >> >> > >>> >> >> >> > public void sendNextJob(String jobId) { >>> >> >> >> > putUserContent(WORKFLOW_STARTED, "TRUE", Scope.WORKFLOW); >>> >> >> >> > if (jobId != null) { >>> >> >> >> > putUserContent(NEXT_JOB, jobId, Scope.WORKFLOW); >>> >> >> >> > } >>> >> >> >> > } >>> >> >> >> > >>> >> >> >> > Dimuthu >>> >> >> >> > >>> >> >> >> > >>> >> >> >> > On Fri, Nov 9, 2018 at 2:48 PM Xue Junkai < >>> [email protected]> >>> >> >> wrote: >>> >> >> >> > >>> >> >> >> > > In my understanding, it could be you have job1 and job2. The >>> >> task >>> >> >> >> running >>> >> >> >> > > in job1 tries to update content for job2. Then, there could >>> be a >>> >> >> race >>> >> >> >> > > condition happening here that job2 is not scheduled. >>> >> >> >> > > >>> >> >> >> > > If that's the case, I suggest you can put key-value store at >>> >> >> workflow >>> >> >> >> > level >>> >> >> >> > > since this is cross-job operation. >>> >> >> >> > > >>> >> >> >> > > Best, >>> >> >> >> > > >>> >> >> >> > > Junkai >>> >> >> >> > > >>> >> >> >> > > On Fri, Nov 9, 2018 at 11:45 AM DImuthu Upeksha < >>> >> >> >> > > [email protected]> >>> >> >> >> > > wrote: >>> >> >> >> > > >>> >> >> >> > > > Hi Junkai, >>> >> >> >> > > > >>> >> >> >> > > > This method is being called inside a running task. And it >>> is >>> >> >> working >>> >> >> >> > for >>> >> >> >> > > > most of the time. I only saw this in 2 occasions for last >>> few >>> >> >> months >>> >> >> >> > and >>> >> >> >> > > > both of them happened today and yesterday. >>> >> >> >> > > > >>> >> >> >> > > > Thanks >>> >> >> >> > > > Dimuthu >>> >> >> >> > > > >>> >> >> >> > > > On Fri, Nov 9, 2018 at 2:40 PM Xue Junkai < >>> >> [email protected]> >>> >> >> >> > wrote: >>> >> >> >> > > > >>> >> >> >> > > > > User content store node will be created one the job has >>> been >>> >> >> >> > scheduled. >>> >> >> >> > > > In >>> >> >> >> > > > > your case, I think the job is not scheduled. This method >>> >> >> usually >>> >> >> >> has >>> >> >> >> > > been >>> >> >> >> > > > > utilized in running task. >>> >> >> >> > > > > >>> >> >> >> > > > > Best, >>> >> >> >> > > > > >>> >> >> >> > > > > Junkai >>> >> >> >> > > > > >>> >> >> >> > > > > On Fri, Nov 9, 2018 at 8:19 AM DImuthu Upeksha < >>> >> >> >> > > > [email protected] >>> >> >> >> > > > > > >>> >> >> >> > > > > wrote: >>> >> >> >> > > > > >>> >> >> >> > > > > > Hi Helix Folks, >>> >> >> >> > > > > > >>> >> >> >> > > > > > I'm having this sporadic issue in some tasks of our >>> >> workflows >>> >> >> >> when >>> >> >> >> > we >>> >> >> >> > > > try >>> >> >> >> > > > > > to store a value in the workflow context and I have >>> added >>> >> >> both >>> >> >> >> code >>> >> >> >> > > > > section >>> >> >> >> > > > > > and error message below. Do you have an idea what's >>> >> causing >>> >> >> >> this? >>> >> >> >> > > > Please >>> >> >> >> > > > > > let me know if you need further information. We are >>> using >>> >> >> Helix >>> >> >> >> > 0.8.2 >>> >> >> >> > > > > > >>> >> >> >> > > > > > public void sendNextJob(String jobId) { >>> >> >> >> > > > > > putUserContent(WORKFLOW_STARTED, "TRUE", >>> >> Scope.WORKFLOW); >>> >> >> >> > > > > > if (jobId != null) { >>> >> >> >> > > > > > putUserContent(NEXT_JOB, jobId, >>> Scope.WORKFLOW); >>> >> >> >> > > > > > } >>> >> >> >> > > > > > } >>> >> >> >> > > > > > >>> >> >> >> > > > > > Failed to setup environment of task >>> >> >> >> > > > > > TASK_55096de4-2cb6-4b09-84fd-7fdddba93435 >>> >> >> >> > > > > > java.lang.NullPointerException: null >>> >> >> >> > > > > > at >>> >> >> >> > org.apache.helix.task.TaskUtil$1.update(TaskUtil.java:358) >>> >> >> >> > > > > > at >>> >> >> >> > org.apache.helix.task.TaskUtil$1.update(TaskUtil.java:356) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> org.apache.helix.manager.zk.HelixGroupCommit.commit(HelixGroupCommit.java:126) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> org.apache.helix.manager.zk.ZkCacheBaseDataAccessor.update(ZkCacheBaseDataAccessor.java:306) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> org.apache.helix.store.zk.AutoFallbackPropertyStore.update(AutoFallbackPropertyStore.java:61) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> org.apache.helix.task.TaskUtil.addWorkflowJobUserContent(TaskUtil.java:356) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> org.apache.helix.task.UserContentStore.putUserContent(UserContentStore.java:78) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> org.apache.airavata.helix.core.AbstractTask.sendNextJob(AbstractTask.java:136) >>> >> >> >> > > > > > at >>> >> >> >> > > > >>> org.apache.airavata.helix.core.OutPort.invoke(OutPort.java:42) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> org.apache.airavata.helix.core.AbstractTask.onSuccess(AbstractTask.java:123) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> org.apache.airavata.helix.impl.task.AiravataTask.onSuccess(AiravataTask.java:97) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> org.apache.airavata.helix.impl.task.env.EnvSetupTask.onRun(EnvSetupTask.java:52) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> org.apache.airavata.helix.impl.task.AiravataTask.onRun(AiravataTask.java:349) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > >>> >> org.apache.airavata.helix.core.AbstractTask.run(AbstractTask.java:92) >>> >> >> >> > > > > > at >>> >> >> >> org.apache.helix.task.TaskRunner.run(TaskRunner.java:71) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > >>> >> >> >> >>> >> >>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) >>> >> >> >> > > > > > at >>> >> >> >> java.util.concurrent.FutureTask.run(FutureTask.java:266) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) >>> >> >> >> > > > > > at >>> >> >> >> > > > > > >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >>> >> >>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) >>> >> >> >> > > > > > at java.lang.Thread.run(Thread.java:748) >>> >> >> >> > > > > > >>> >> >> >> > > > > > Thanks >>> >> >> >> > > > > > Dimuthu >>> >> >> >> > > > > > >>> >> >> >> > > > > >>> >> >> >> > > > > >>> >> >> >> > > > > -- >>> >> >> >> > > > > Junkai Xue >>> >> >> >> > > > > >>> >> >> >> > > > >>> >> >> >> > > >>> >> >> >> > > >>> >> >> >> > > -- >>> >> >> >> > > Junkai Xue >>> >> >> >> > > >>> >> >> >> > >>> >> >> >> >>> >> >> >> >>> >> >> >> -- >>> >> >> >> Junkai Xue >>> >> >> >> >>> >> >> > >>> >> >> >>> >> > >>> >> > >>> >> > -- >>> >> > Junkai Xue >>> >> > >>> >> >>> > >>> > >>> > -- >>> > Junkai Xue >>> > >>> >> >> >> -- >> Junkai Xue >> >
