You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@ode.apache.org by Sathwik B P <sa...@gmail.com> on 2013/03/01 07:51:17 UTC

Re: JobNoLongerInDbException and Incomplete instances.

Hi,
This is really strange.

This is the ideal behaviour:
If a job fails for any reason it gets retried defined by the parameter
(immediateRetryCount default 3 times with a time interval
_immediateTransactionRetryInterval default 1 sec) and then the scheduler
will put it on a exponential backoff defined by pow(5,retryCount) where
retryCount is <= 10.

If the rollback doesn't happen incase of any exception then none of the
jobs will ever complete since it will never go into the exponential backoff
path.

In my opinion the transaction manager will maintain the jdbc connection
object throughout it's execution, no matter how many times the connection
is borrowed during the transaction.

Which database are you using and what configuration changes have you done
in ode-axis.properties.

regards,
sathwik

On Fri, Mar 1, 2013 at 1:31 AM, Nandika Jayawardana <ja...@gmail.com>wrote:

> Hi All,
> I am running ode trunk build with apache tomcat as described in [1] . I
> have an asynchronous bpel process which has a receive, invoke and a
> receive. When I run this process for a while, I see that there are few
> incomplete instances, although all the expected messages reached ode. From
> the debug logs, I figured that it is happening as follows.
>
>   If a thread executing a job tries to acquire the process instance lock,
> while another thread is executing on the same instance and times out, it
> will throw a timeout exception at InstanceLockManager which will be wrapped
> to a  JobProcessorException.
>
> In SimpleScheduler, RunJob.call method, when the execution of a job starts,
> it will try to delete the job from the db. For the initial try, it
> would succeed since the job is in db. However, when
> the JobProcessorException exception happens due to timeout on instance
> lock, the transaction gets rolled back. Ideally, the job should be restored
> back when the rollback happens. However, the job does not get restored to
> db as the transaction manager and db resources are not associated. Hence
> when the scheduler  tries to retry 3 times by default, it will fail with
> job no longer in db error.  This results in few of the process instances
> never completing since the job was abandoned even though the messages
> reached ode.
>
> Following log extracts from the ode log explains the scenario.
>
> grep instanceid
>
> 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> Thread[ODEServer-78,5,main]: lock(iid=36423, time=1MICROSECONDS)
> 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> Thread[ODEServer-78,5,main]: lock(iid=36423,
> time=1MICROSECONDS)-->WAITING(held by Thread[ODEServer-9,5,main])
> 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> Thread[ODEServer-78,5,main]: lock(iid=36423, time=1MICROSECONDS)-->TIMEOUT
> (held by Thread[ODEServer-9,5,main])
> 16:36:12,115 ODEServer-78 DEBUG [BpelEngineImpl] Instance 36423 is busy,
> rescheduling job.
> 16:36:12,239 ODEServer-9 DEBUG [InstanceLockManager]
> Thread[ODEServer-9,5,main]: unlock(iid=36423)
> 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in db
> forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28 16:36:11 IST
> transacted: true persisted: true details: JobDetails( instanceId: 36423
> mexId: null processId: null type: MATCHER channel: null correlatorId:
> DebugCallbackPL.debugOpCallback correlationKeySet:
> @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount: null
> inMem: false detailsExt: {})
>
> grep jobid
>
> 16:36:11,960 ODEServer-9 DEBUG [JdbcDelegate] insertJob
> hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94 loaded=true
> 16:36:12,007 ODEServer-1 DEBUG [SimpleScheduler] todo.enqueue job from db:
> hqejbhcnphr8357nokgnxp for 1362049571960(16:36:11,960)
> 16:36:12,007 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> 16:36:12,032 ODEServer-9 DEBUG [SimpleScheduler] scheduled immediate job:
> hqejbhcnphr8357nokgnxp
> 16:36:12,239 ODEServer-9 DEBUG [SimpleScheduler] Job hqejbhcnphr8357nokgnxp
> is being processed (outstanding job)
> 16:36:13,116 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no longer in
> db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
> 16:36:14,118 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no longer in
> db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
> 16:36:15,119 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in db
> forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28 16:36:11 IST
> transacted: true persisted: true details: JobDetails( instanceId: 36423
> mexId: null processId: null type: MATCHER channel: null correlatorId:
> DebugCallbackPL.debugOpCallback correlationKeySet:
> @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount: null
> inMem: false detailsExt: {})
>
>
> Is this the expected behavior or is there any additional settings i should
> configure to make transaction manager restore job to db at rollback ?
> Will reinserting the job back to db when the  JobProcessorException happens
>  fix this problem ?
>
> Regards
> Nandika
>
> [1] http://ode.apache.org/war-deployment.html
>

Re: JobNoLongerInDbException and Incomplete instances.

Posted by Nandika Jayawardana <ja...@gmail.com>.
Hi Sathwik

Thanks for looking into the issue. I will check with the Bitronix
transaction manager.

Regards
Nandika


On Wed, Jun 26, 2013 at 5:26 PM, Sathwik B P <sa...@gmail.com> wrote:

> Hi,
>
> I have been able to solve the problem by using the config setting specified
> in the BTM jira http://jira.codehaus.org/browse/BTM-35
>
> The original problem is still a question unanswered. Why is the connection
> retrieved during deleteJob(..) is not enlisted in the transaction when
> using Geronimo TM?
>
> regards,
> sathwik
>
> On Wed, Jun 26, 2013 at 3:37 PM, Sathwik B P <sa...@gmail.com> wrote:
>
> > Hi,
> >
> > I am able to see this problem on the trunk. PFA the logs.
> >
> > When the job is being executed by the scheduler RunJob.call()
> >
> > 1) if (!_db.deleteJob(job.jobId, _nodeId))
> >           throw new JobNoLongerInDbException(job.jobId, _nodeId);
> > 2) processor.onScheduledJob(jobInfo);
> >
> > The job is deleted first and then the processor.onScheduledJob(jobInfo)
> > throws an exception in this case due to non availability of instance
> lock.
> > This exception gets caught in the execTransaction() and goes in for a
> > transaction rollback, followed by a retry. But the rollback doesn't
> reverse
> > the deleted job. And during the second iteration it throws a
> > JobNoLongerInDbException.
> >
> > line 286: 2013-06-24 12:00:27,803 6058570 DEBUG
> > [org.apache.ode.scheduler.simple.SimpleScheduler] (ODEServer-18:) Job :
> > hqejbhcnphr8dw7m78o3qq has been deleted
> > line 291: 2013-06-24 12:00:27,804 6058571 DEBUG
> > [org.apache.ode.bpel.engine.BpelEngineImpl] (ODEServer-18:) Instance 2608
> > is busy, rescheduling job.
> > line 294: org.apache.ode.bpel.iapi.Scheduler$JobProcessorException
> > line 313: 2013-06-24 12:00:28,804 6059571 DEBUG
> > [org.apache.ode.scheduler.simple.SimpleScheduler] (ODEServer-18:)
> Beginning
> > a new transaction
> > line 314: 2013-06-24 12:00:28,805 6059572 DEBUG
> > [org.apache.ode.scheduler.simple.JdbcDelegate] (ODEServer-18:) deleteJob
> > hqejbhcnphr8dw7m78o3qq on node hqejbhcnphr8dw7m78o3p4
> > line 318: org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job
> no
> > longer in db: hqejbhcnphr8dw7m78o3qq nodeId=hqejbhcnphr8dw7m78o3p4
> >
> >
> > I then integrated Bitronix transaction manager in my tomcat. Bitronix is
> > also complaining that a second new connection is being enlisted.
> >
> > 2013-06-26 13:54:38,604 89798 DEBUG
> > [org.apache.ode.scheduler.simple.JdbcDelegate] (ODEServer-2:) insertJob
> > hqejbhcnphr8e518sj4h72 on node hqejbhcnphr8e518sj4h70 loaded=true
> > 2013-06-26 13:54:38,607 89801 ERROR
> > [org.apache.ode.scheduler.simple.SimpleScheduler] (ODEServer-2:) Database
> > error.
> > org.apache.ode.scheduler.simple.DatabaseException: java.sql.SQLException:
> > error enlisting a JdbcConnectionHandle of a JdbcPooledConnection from
> > datasource jdbc/ode in state ACCESSIBLE with usage count 1 wrapping a
> JDBC
> > LrcXAConnection on com.mysql.jdbc.JDBC4Connection@49d456d0 on a JDBC
> > LrcConnectionHandle on a JDBC LrcXAResource in state NO_TX
> >
> >
> > regards,
> > sathwik
> >
> >
> > On Sun, Mar 24, 2013 at 6:13 PM, Sathwik B P <sa...@gmail.com>
> wrote:
> >
> >> Hi,
> >> The transaction manager is initialised even before the database config
> is
> >> loaded. Without initialization of the transaction manager the ode server
> >> will not start. Irrespective of the DB mode the transaction manager
> will be
> >> set on the Database object that has being configured, be it internal or
> >> external.
> >> Please check the init(..) method in org.apache.ode.axis2.ODEServer
> >>
> >> init(...){
> >>     initTxMgr()
> >>     initDataSource();
> >>     initDAO();
> >> }
> >>
> >> Please share your configuration files and logs files.
> >>
> >> regards,
> >> sathwik
> >>
> >>
> >> On Sat, Mar 23, 2013 at 9:32 PM, Nandika Jayawardana <
> jayawark@gmail.com>wrote:
> >>
> >>> I did some further checking and with the database configured as db.mode
> >>> internal , everything works fine. For the internal db mode, db
> >>> connections
> >>> are associated with default geronimo transaction manager (
> Database.java
> >>> ).
> >>> I think for the default external db configuration given , this does not
> >>> happen and hence the issue.
> >>>
> >>> Regards
> >>> Nandika
> >>>
> >>> On Fri, Mar 1, 2013 at 2:17 PM, Nandika Jayawardana <
> jayawark@gmail.com
> >>> >wrote:
> >>>
> >>> > Hi Sathwik,
> >>> >
> >>> > I am running ode with tomcat 7.0.29 and mysql 5.5.29 version. I used
> >>> the
> >>> > configuration settings given under "Configuring ODE in Tomcat with
> >>> MySql
> >>> > database". from ode war deployment guide. {
> >>> > http://ode.apache.org/war-deployment.html }.
> >>> > As you have explained, when the JobProcessorException is thrown due
> to
> >>> >  instance lock timeout , the transaction will be rollback and default
> >>> retry
> >>> > setting of 3 times will happen. However, the restoration of the
> >>> deleted job
> >>> > back to job table does not happen. Therefore subsequent retries will
> >>> also
> >>> > result in JobNoLongerInDbException. At execTransaction method, when
> the
> >>> > retry loop is over, the exception thrown will also be
> >>> >  JobNoLongerInDbException. Since this exception is caught at  "catch
> >>> > (JobNoLongerInDbException jde) " block, it will never go into the
> >>> > exponential back off setting.
> >>> >
> >>> > Is there any additional configuration settings I need to do ?
> >>> >
> >>> > Regards
> >>> > Nandika
> >>> >
> >>> >
> >>> > On Fri, Mar 1, 2013 at 12:21 PM, Sathwik B P <sa...@gmail.com>
> >>> wrote:
> >>> >
> >>> >> Hi,
> >>> >> This is really strange.
> >>> >>
> >>> >> This is the ideal behaviour:
> >>> >> If a job fails for any reason it gets retried defined by the
> parameter
> >>> >> (immediateRetryCount default 3 times with a time interval
> >>> >> _immediateTransactionRetryInterval default 1 sec) and then the
> >>> scheduler
> >>> >> will put it on a exponential backoff defined by pow(5,retryCount)
> >>> where
> >>> >> retryCount is <= 10.
> >>> >>
> >>> >> If the rollback doesn't happen incase of any exception then none of
> >>> the
> >>> >> jobs will ever complete since it will never go into the exponential
> >>> >> backoff
> >>> >> path.
> >>> >>
> >>> >> In my opinion the transaction manager will maintain the jdbc
> >>> connection
> >>> >> object throughout it's execution, no matter how many times the
> >>> connection
> >>> >> is borrowed during the transaction.
> >>> >>
> >>> >> Which database are you using and what configuration changes have you
> >>> done
> >>> >> in ode-axis.properties.
> >>> >>
> >>> >> regards,
> >>> >> sathwik
> >>> >>
> >>> >> On Fri, Mar 1, 2013 at 1:31 AM, Nandika Jayawardana <
> >>> jayawark@gmail.com
> >>> >> >wrote:
> >>> >>
> >>> >> > Hi All,
> >>> >> > I am running ode trunk build with apache tomcat as described in
> [1]
> >>> . I
> >>> >> > have an asynchronous bpel process which has a receive, invoke and
> a
> >>> >> > receive. When I run this process for a while, I see that there are
> >>> few
> >>> >> > incomplete instances, although all the expected messages reached
> >>> ode.
> >>> >> From
> >>> >> > the debug logs, I figured that it is happening as follows.
> >>> >> >
> >>> >> >   If a thread executing a job tries to acquire the process
> instance
> >>> >> lock,
> >>> >> > while another thread is executing on the same instance and times
> >>> out, it
> >>> >> > will throw a timeout exception at InstanceLockManager which will
> be
> >>> >> wrapped
> >>> >> > to a  JobProcessorException.
> >>> >> >
> >>> >> > In SimpleScheduler, RunJob.call method, when the execution of a
> job
> >>> >> starts,
> >>> >> > it will try to delete the job from the db. For the initial try, it
> >>> >> > would succeed since the job is in db. However, when
> >>> >> > the JobProcessorException exception happens due to timeout on
> >>> instance
> >>> >> > lock, the transaction gets rolled back. Ideally, the job should be
> >>> >> restored
> >>> >> > back when the rollback happens. However, the job does not get
> >>> restored
> >>> >> to
> >>> >> > db as the transaction manager and db resources are not associated.
> >>> Hence
> >>> >> > when the scheduler  tries to retry 3 times by default, it will
> fail
> >>> with
> >>> >> > job no longer in db error.  This results in few of the process
> >>> instances
> >>> >> > never completing since the job was abandoned even though the
> >>> messages
> >>> >> > reached ode.
> >>> >> >
> >>> >> > Following log extracts from the ode log explains the scenario.
> >>> >> >
> >>> >> > grep instanceid
> >>> >> >
> >>> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> >>> >> > Thread[ODEServer-78,5,main]: lock(iid=36423, time=1MICROSECONDS)
> >>> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> >>> >> > Thread[ODEServer-78,5,main]: lock(iid=36423,
> >>> >> > time=1MICROSECONDS)-->WAITING(held by Thread[ODEServer-9,5,main])
> >>> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> >>> >> > Thread[ODEServer-78,5,main]: lock(iid=36423,
> >>> >> time=1MICROSECONDS)-->TIMEOUT
> >>> >> > (held by Thread[ODEServer-9,5,main])
> >>> >> > 16:36:12,115 ODEServer-78 DEBUG [BpelEngineImpl] Instance 36423 is
> >>> busy,
> >>> >> > rescheduling job.
> >>> >> > 16:36:12,239 ODEServer-9 DEBUG [InstanceLockManager]
> >>> >> > Thread[ODEServer-9,5,main]: unlock(iid=36423)
> >>> >> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in
> >>> db
> >>> >> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28
> >>> 16:36:11
> >>> >> IST
> >>> >> > transacted: true persisted: true details: JobDetails( instanceId:
> >>> 36423
> >>> >> > mexId: null processId: null type: MATCHER channel: null
> >>> correlatorId:
> >>> >> > DebugCallbackPL.debugOpCallback correlationKeySet:
> >>> >> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377]
> retryCount:
> >>> null
> >>> >> > inMem: false detailsExt: {})
> >>> >> >
> >>> >> > grep jobid
> >>> >> >
> >>> >> > 16:36:11,960 ODEServer-9 DEBUG [JdbcDelegate] insertJob
> >>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94 loaded=true
> >>> >> > 16:36:12,007 ODEServer-1 DEBUG [SimpleScheduler] todo.enqueue job
> >>> from
> >>> >> db:
> >>> >> > hqejbhcnphr8357nokgnxp for 1362049571960(16:36:11,960)
> >>> >> > 16:36:12,007 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> >>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> >>> >> > 16:36:12,032 ODEServer-9 DEBUG [SimpleScheduler] scheduled
> immediate
> >>> >> job:
> >>> >> > hqejbhcnphr8357nokgnxp
> >>> >> > 16:36:12,239 ODEServer-9 DEBUG [SimpleScheduler] Job
> >>> >> hqejbhcnphr8357nokgnxp
> >>> >> > is being processed (outstanding job)
> >>> >> > 16:36:13,116 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> >>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> >>> >> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no
> >>> longer
> >>> >> in
> >>> >> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
> >>> >> > 16:36:14,118 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> >>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> >>> >> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no
> >>> longer
> >>> >> in
> >>> >> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
> >>> >> > 16:36:15,119 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> >>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> >>> >> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in
> >>> db
> >>> >> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28
> >>> 16:36:11
> >>> >> IST
> >>> >> > transacted: true persisted: true details: JobDetails( instanceId:
> >>> 36423
> >>> >> > mexId: null processId: null type: MATCHER channel: null
> >>> correlatorId:
> >>> >> > DebugCallbackPL.debugOpCallback correlationKeySet:
> >>> >> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377]
> retryCount:
> >>> null
> >>> >> > inMem: false detailsExt: {})
> >>> >> >
> >>> >> >
> >>> >> > Is this the expected behavior or is there any additional settings
> i
> >>> >> should
> >>> >> > configure to make transaction manager restore job to db at
> rollback
> >>> ?
> >>> >> > Will reinserting the job back to db when the
>  JobProcessorException
> >>> >> happens
> >>> >> >  fix this problem ?
> >>> >> >
> >>> >> > Regards
> >>> >> > Nandika
> >>> >> >
> >>> >> > [1] http://ode.apache.org/war-deployment.html
> >>> >> >
> >>> >>
> >>> >
> >>> >
> >>>
> >>
> >>
> >
>

Re: JobNoLongerInDbException and Incomplete instances.

Posted by Sathwik B P <sa...@gmail.com>.
Hi,

I have been able to solve the problem by using the config setting specified
in the BTM jira http://jira.codehaus.org/browse/BTM-35

The original problem is still a question unanswered. Why is the connection
retrieved during deleteJob(..) is not enlisted in the transaction when
using Geronimo TM?

regards,
sathwik

On Wed, Jun 26, 2013 at 3:37 PM, Sathwik B P <sa...@gmail.com> wrote:

> Hi,
>
> I am able to see this problem on the trunk. PFA the logs.
>
> When the job is being executed by the scheduler RunJob.call()
>
> 1) if (!_db.deleteJob(job.jobId, _nodeId))
>           throw new JobNoLongerInDbException(job.jobId, _nodeId);
> 2) processor.onScheduledJob(jobInfo);
>
> The job is deleted first and then the processor.onScheduledJob(jobInfo)
> throws an exception in this case due to non availability of instance lock.
> This exception gets caught in the execTransaction() and goes in for a
> transaction rollback, followed by a retry. But the rollback doesn't reverse
> the deleted job. And during the second iteration it throws a
> JobNoLongerInDbException.
>
> line 286: 2013-06-24 12:00:27,803 6058570 DEBUG
> [org.apache.ode.scheduler.simple.SimpleScheduler] (ODEServer-18:) Job :
> hqejbhcnphr8dw7m78o3qq has been deleted
> line 291: 2013-06-24 12:00:27,804 6058571 DEBUG
> [org.apache.ode.bpel.engine.BpelEngineImpl] (ODEServer-18:) Instance 2608
> is busy, rescheduling job.
> line 294: org.apache.ode.bpel.iapi.Scheduler$JobProcessorException
> line 313: 2013-06-24 12:00:28,804 6059571 DEBUG
> [org.apache.ode.scheduler.simple.SimpleScheduler] (ODEServer-18:) Beginning
> a new transaction
> line 314: 2013-06-24 12:00:28,805 6059572 DEBUG
> [org.apache.ode.scheduler.simple.JdbcDelegate] (ODEServer-18:) deleteJob
> hqejbhcnphr8dw7m78o3qq on node hqejbhcnphr8dw7m78o3p4
> line 318: org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no
> longer in db: hqejbhcnphr8dw7m78o3qq nodeId=hqejbhcnphr8dw7m78o3p4
>
>
> I then integrated Bitronix transaction manager in my tomcat. Bitronix is
> also complaining that a second new connection is being enlisted.
>
> 2013-06-26 13:54:38,604 89798 DEBUG
> [org.apache.ode.scheduler.simple.JdbcDelegate] (ODEServer-2:) insertJob
> hqejbhcnphr8e518sj4h72 on node hqejbhcnphr8e518sj4h70 loaded=true
> 2013-06-26 13:54:38,607 89801 ERROR
> [org.apache.ode.scheduler.simple.SimpleScheduler] (ODEServer-2:) Database
> error.
> org.apache.ode.scheduler.simple.DatabaseException: java.sql.SQLException:
> error enlisting a JdbcConnectionHandle of a JdbcPooledConnection from
> datasource jdbc/ode in state ACCESSIBLE with usage count 1 wrapping a JDBC
> LrcXAConnection on com.mysql.jdbc.JDBC4Connection@49d456d0 on a JDBC
> LrcConnectionHandle on a JDBC LrcXAResource in state NO_TX
>
>
> regards,
> sathwik
>
>
> On Sun, Mar 24, 2013 at 6:13 PM, Sathwik B P <sa...@gmail.com> wrote:
>
>> Hi,
>> The transaction manager is initialised even before the database config is
>> loaded. Without initialization of the transaction manager the ode server
>> will not start. Irrespective of the DB mode the transaction manager will be
>> set on the Database object that has being configured, be it internal or
>> external.
>> Please check the init(..) method in org.apache.ode.axis2.ODEServer
>>
>> init(...){
>>     initTxMgr()
>>     initDataSource();
>>     initDAO();
>> }
>>
>> Please share your configuration files and logs files.
>>
>> regards,
>> sathwik
>>
>>
>> On Sat, Mar 23, 2013 at 9:32 PM, Nandika Jayawardana <ja...@gmail.com>wrote:
>>
>>> I did some further checking and with the database configured as db.mode
>>> internal , everything works fine. For the internal db mode, db
>>> connections
>>> are associated with default geronimo transaction manager ( Database.java
>>> ).
>>> I think for the default external db configuration given , this does not
>>> happen and hence the issue.
>>>
>>> Regards
>>> Nandika
>>>
>>> On Fri, Mar 1, 2013 at 2:17 PM, Nandika Jayawardana <jayawark@gmail.com
>>> >wrote:
>>>
>>> > Hi Sathwik,
>>> >
>>> > I am running ode with tomcat 7.0.29 and mysql 5.5.29 version. I used
>>> the
>>> > configuration settings given under "Configuring ODE in Tomcat with
>>> MySql
>>> > database". from ode war deployment guide. {
>>> > http://ode.apache.org/war-deployment.html }.
>>> > As you have explained, when the JobProcessorException is thrown due to
>>> >  instance lock timeout , the transaction will be rollback and default
>>> retry
>>> > setting of 3 times will happen. However, the restoration of the
>>> deleted job
>>> > back to job table does not happen. Therefore subsequent retries will
>>> also
>>> > result in JobNoLongerInDbException. At execTransaction method, when the
>>> > retry loop is over, the exception thrown will also be
>>> >  JobNoLongerInDbException. Since this exception is caught at  "catch
>>> > (JobNoLongerInDbException jde) " block, it will never go into the
>>> > exponential back off setting.
>>> >
>>> > Is there any additional configuration settings I need to do ?
>>> >
>>> > Regards
>>> > Nandika
>>> >
>>> >
>>> > On Fri, Mar 1, 2013 at 12:21 PM, Sathwik B P <sa...@gmail.com>
>>> wrote:
>>> >
>>> >> Hi,
>>> >> This is really strange.
>>> >>
>>> >> This is the ideal behaviour:
>>> >> If a job fails for any reason it gets retried defined by the parameter
>>> >> (immediateRetryCount default 3 times with a time interval
>>> >> _immediateTransactionRetryInterval default 1 sec) and then the
>>> scheduler
>>> >> will put it on a exponential backoff defined by pow(5,retryCount)
>>> where
>>> >> retryCount is <= 10.
>>> >>
>>> >> If the rollback doesn't happen incase of any exception then none of
>>> the
>>> >> jobs will ever complete since it will never go into the exponential
>>> >> backoff
>>> >> path.
>>> >>
>>> >> In my opinion the transaction manager will maintain the jdbc
>>> connection
>>> >> object throughout it's execution, no matter how many times the
>>> connection
>>> >> is borrowed during the transaction.
>>> >>
>>> >> Which database are you using and what configuration changes have you
>>> done
>>> >> in ode-axis.properties.
>>> >>
>>> >> regards,
>>> >> sathwik
>>> >>
>>> >> On Fri, Mar 1, 2013 at 1:31 AM, Nandika Jayawardana <
>>> jayawark@gmail.com
>>> >> >wrote:
>>> >>
>>> >> > Hi All,
>>> >> > I am running ode trunk build with apache tomcat as described in [1]
>>> . I
>>> >> > have an asynchronous bpel process which has a receive, invoke and a
>>> >> > receive. When I run this process for a while, I see that there are
>>> few
>>> >> > incomplete instances, although all the expected messages reached
>>> ode.
>>> >> From
>>> >> > the debug logs, I figured that it is happening as follows.
>>> >> >
>>> >> >   If a thread executing a job tries to acquire the process instance
>>> >> lock,
>>> >> > while another thread is executing on the same instance and times
>>> out, it
>>> >> > will throw a timeout exception at InstanceLockManager which will be
>>> >> wrapped
>>> >> > to a  JobProcessorException.
>>> >> >
>>> >> > In SimpleScheduler, RunJob.call method, when the execution of a job
>>> >> starts,
>>> >> > it will try to delete the job from the db. For the initial try, it
>>> >> > would succeed since the job is in db. However, when
>>> >> > the JobProcessorException exception happens due to timeout on
>>> instance
>>> >> > lock, the transaction gets rolled back. Ideally, the job should be
>>> >> restored
>>> >> > back when the rollback happens. However, the job does not get
>>> restored
>>> >> to
>>> >> > db as the transaction manager and db resources are not associated.
>>> Hence
>>> >> > when the scheduler  tries to retry 3 times by default, it will fail
>>> with
>>> >> > job no longer in db error.  This results in few of the process
>>> instances
>>> >> > never completing since the job was abandoned even though the
>>> messages
>>> >> > reached ode.
>>> >> >
>>> >> > Following log extracts from the ode log explains the scenario.
>>> >> >
>>> >> > grep instanceid
>>> >> >
>>> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
>>> >> > Thread[ODEServer-78,5,main]: lock(iid=36423, time=1MICROSECONDS)
>>> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
>>> >> > Thread[ODEServer-78,5,main]: lock(iid=36423,
>>> >> > time=1MICROSECONDS)-->WAITING(held by Thread[ODEServer-9,5,main])
>>> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
>>> >> > Thread[ODEServer-78,5,main]: lock(iid=36423,
>>> >> time=1MICROSECONDS)-->TIMEOUT
>>> >> > (held by Thread[ODEServer-9,5,main])
>>> >> > 16:36:12,115 ODEServer-78 DEBUG [BpelEngineImpl] Instance 36423 is
>>> busy,
>>> >> > rescheduling job.
>>> >> > 16:36:12,239 ODEServer-9 DEBUG [InstanceLockManager]
>>> >> > Thread[ODEServer-9,5,main]: unlock(iid=36423)
>>> >> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in
>>> db
>>> >> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28
>>> 16:36:11
>>> >> IST
>>> >> > transacted: true persisted: true details: JobDetails( instanceId:
>>> 36423
>>> >> > mexId: null processId: null type: MATCHER channel: null
>>> correlatorId:
>>> >> > DebugCallbackPL.debugOpCallback correlationKeySet:
>>> >> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount:
>>> null
>>> >> > inMem: false detailsExt: {})
>>> >> >
>>> >> > grep jobid
>>> >> >
>>> >> > 16:36:11,960 ODEServer-9 DEBUG [JdbcDelegate] insertJob
>>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94 loaded=true
>>> >> > 16:36:12,007 ODEServer-1 DEBUG [SimpleScheduler] todo.enqueue job
>>> from
>>> >> db:
>>> >> > hqejbhcnphr8357nokgnxp for 1362049571960(16:36:11,960)
>>> >> > 16:36:12,007 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>>> >> > 16:36:12,032 ODEServer-9 DEBUG [SimpleScheduler] scheduled immediate
>>> >> job:
>>> >> > hqejbhcnphr8357nokgnxp
>>> >> > 16:36:12,239 ODEServer-9 DEBUG [SimpleScheduler] Job
>>> >> hqejbhcnphr8357nokgnxp
>>> >> > is being processed (outstanding job)
>>> >> > 16:36:13,116 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>>> >> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no
>>> longer
>>> >> in
>>> >> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
>>> >> > 16:36:14,118 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>>> >> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no
>>> longer
>>> >> in
>>> >> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
>>> >> > 16:36:15,119 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>>> >> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in
>>> db
>>> >> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28
>>> 16:36:11
>>> >> IST
>>> >> > transacted: true persisted: true details: JobDetails( instanceId:
>>> 36423
>>> >> > mexId: null processId: null type: MATCHER channel: null
>>> correlatorId:
>>> >> > DebugCallbackPL.debugOpCallback correlationKeySet:
>>> >> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount:
>>> null
>>> >> > inMem: false detailsExt: {})
>>> >> >
>>> >> >
>>> >> > Is this the expected behavior or is there any additional settings i
>>> >> should
>>> >> > configure to make transaction manager restore job to db at rollback
>>> ?
>>> >> > Will reinserting the job back to db when the  JobProcessorException
>>> >> happens
>>> >> >  fix this problem ?
>>> >> >
>>> >> > Regards
>>> >> > Nandika
>>> >> >
>>> >> > [1] http://ode.apache.org/war-deployment.html
>>> >> >
>>> >>
>>> >
>>> >
>>>
>>
>>
>

Re: JobNoLongerInDbException and Incomplete instances.

Posted by Sathwik B P <sa...@gmail.com>.
Hi,

I am able to see this problem on the trunk. PFA the logs.

When the job is being executed by the scheduler RunJob.call()

1) if (!_db.deleteJob(job.jobId, _nodeId))
          throw new JobNoLongerInDbException(job.jobId, _nodeId);
2) processor.onScheduledJob(jobInfo);

The job is deleted first and then the processor.onScheduledJob(jobInfo)
throws an exception in this case due to non availability of instance lock.
This exception gets caught in the execTransaction() and goes in for a
transaction rollback, followed by a retry. But the rollback doesn't reverse
the deleted job. And during the second iteration it throws a
JobNoLongerInDbException.

line 286: 2013-06-24 12:00:27,803 6058570 DEBUG
[org.apache.ode.scheduler.simple.SimpleScheduler] (ODEServer-18:) Job :
hqejbhcnphr8dw7m78o3qq has been deleted
line 291: 2013-06-24 12:00:27,804 6058571 DEBUG
[org.apache.ode.bpel.engine.BpelEngineImpl] (ODEServer-18:) Instance 2608
is busy, rescheduling job.
line 294: org.apache.ode.bpel.iapi.Scheduler$JobProcessorException
line 313: 2013-06-24 12:00:28,804 6059571 DEBUG
[org.apache.ode.scheduler.simple.SimpleScheduler] (ODEServer-18:) Beginning
a new transaction
line 314: 2013-06-24 12:00:28,805 6059572 DEBUG
[org.apache.ode.scheduler.simple.JdbcDelegate] (ODEServer-18:) deleteJob
hqejbhcnphr8dw7m78o3qq on node hqejbhcnphr8dw7m78o3p4
line 318: org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no
longer in db: hqejbhcnphr8dw7m78o3qq nodeId=hqejbhcnphr8dw7m78o3p4


I then integrated Bitronix transaction manager in my tomcat. Bitronix is
also complaining that a second new connection is being enlisted.

2013-06-26 13:54:38,604 89798 DEBUG
[org.apache.ode.scheduler.simple.JdbcDelegate] (ODEServer-2:) insertJob
hqejbhcnphr8e518sj4h72 on node hqejbhcnphr8e518sj4h70 loaded=true
2013-06-26 13:54:38,607 89801 ERROR
[org.apache.ode.scheduler.simple.SimpleScheduler] (ODEServer-2:) Database
error.
org.apache.ode.scheduler.simple.DatabaseException: java.sql.SQLException:
error enlisting a JdbcConnectionHandle of a JdbcPooledConnection from
datasource jdbc/ode in state ACCESSIBLE with usage count 1 wrapping a JDBC
LrcXAConnection on com.mysql.jdbc.JDBC4Connection@49d456d0 on a JDBC
LrcConnectionHandle on a JDBC LrcXAResource in state NO_TX


regards,
sathwik

On Sun, Mar 24, 2013 at 6:13 PM, Sathwik B P <sa...@gmail.com> wrote:

> Hi,
> The transaction manager is initialised even before the database config is
> loaded. Without initialization of the transaction manager the ode server
> will not start. Irrespective of the DB mode the transaction manager will be
> set on the Database object that has being configured, be it internal or
> external.
> Please check the init(..) method in org.apache.ode.axis2.ODEServer
>
> init(...){
>     initTxMgr()
>     initDataSource();
>     initDAO();
> }
>
> Please share your configuration files and logs files.
>
> regards,
> sathwik
>
>
> On Sat, Mar 23, 2013 at 9:32 PM, Nandika Jayawardana <ja...@gmail.com>wrote:
>
>> I did some further checking and with the database configured as db.mode
>> internal , everything works fine. For the internal db mode, db connections
>> are associated with default geronimo transaction manager ( Database.java
>> ).
>> I think for the default external db configuration given , this does not
>> happen and hence the issue.
>>
>> Regards
>> Nandika
>>
>> On Fri, Mar 1, 2013 at 2:17 PM, Nandika Jayawardana <jayawark@gmail.com
>> >wrote:
>>
>> > Hi Sathwik,
>> >
>> > I am running ode with tomcat 7.0.29 and mysql 5.5.29 version. I used the
>> > configuration settings given under "Configuring ODE in Tomcat with MySql
>> > database". from ode war deployment guide. {
>> > http://ode.apache.org/war-deployment.html }.
>> > As you have explained, when the JobProcessorException is thrown due to
>> >  instance lock timeout , the transaction will be rollback and default
>> retry
>> > setting of 3 times will happen. However, the restoration of the deleted
>> job
>> > back to job table does not happen. Therefore subsequent retries will
>> also
>> > result in JobNoLongerInDbException. At execTransaction method, when the
>> > retry loop is over, the exception thrown will also be
>> >  JobNoLongerInDbException. Since this exception is caught at  "catch
>> > (JobNoLongerInDbException jde) " block, it will never go into the
>> > exponential back off setting.
>> >
>> > Is there any additional configuration settings I need to do ?
>> >
>> > Regards
>> > Nandika
>> >
>> >
>> > On Fri, Mar 1, 2013 at 12:21 PM, Sathwik B P <sa...@gmail.com>
>> wrote:
>> >
>> >> Hi,
>> >> This is really strange.
>> >>
>> >> This is the ideal behaviour:
>> >> If a job fails for any reason it gets retried defined by the parameter
>> >> (immediateRetryCount default 3 times with a time interval
>> >> _immediateTransactionRetryInterval default 1 sec) and then the
>> scheduler
>> >> will put it on a exponential backoff defined by pow(5,retryCount) where
>> >> retryCount is <= 10.
>> >>
>> >> If the rollback doesn't happen incase of any exception then none of the
>> >> jobs will ever complete since it will never go into the exponential
>> >> backoff
>> >> path.
>> >>
>> >> In my opinion the transaction manager will maintain the jdbc connection
>> >> object throughout it's execution, no matter how many times the
>> connection
>> >> is borrowed during the transaction.
>> >>
>> >> Which database are you using and what configuration changes have you
>> done
>> >> in ode-axis.properties.
>> >>
>> >> regards,
>> >> sathwik
>> >>
>> >> On Fri, Mar 1, 2013 at 1:31 AM, Nandika Jayawardana <
>> jayawark@gmail.com
>> >> >wrote:
>> >>
>> >> > Hi All,
>> >> > I am running ode trunk build with apache tomcat as described in [1]
>> . I
>> >> > have an asynchronous bpel process which has a receive, invoke and a
>> >> > receive. When I run this process for a while, I see that there are
>> few
>> >> > incomplete instances, although all the expected messages reached ode.
>> >> From
>> >> > the debug logs, I figured that it is happening as follows.
>> >> >
>> >> >   If a thread executing a job tries to acquire the process instance
>> >> lock,
>> >> > while another thread is executing on the same instance and times
>> out, it
>> >> > will throw a timeout exception at InstanceLockManager which will be
>> >> wrapped
>> >> > to a  JobProcessorException.
>> >> >
>> >> > In SimpleScheduler, RunJob.call method, when the execution of a job
>> >> starts,
>> >> > it will try to delete the job from the db. For the initial try, it
>> >> > would succeed since the job is in db. However, when
>> >> > the JobProcessorException exception happens due to timeout on
>> instance
>> >> > lock, the transaction gets rolled back. Ideally, the job should be
>> >> restored
>> >> > back when the rollback happens. However, the job does not get
>> restored
>> >> to
>> >> > db as the transaction manager and db resources are not associated.
>> Hence
>> >> > when the scheduler  tries to retry 3 times by default, it will fail
>> with
>> >> > job no longer in db error.  This results in few of the process
>> instances
>> >> > never completing since the job was abandoned even though the messages
>> >> > reached ode.
>> >> >
>> >> > Following log extracts from the ode log explains the scenario.
>> >> >
>> >> > grep instanceid
>> >> >
>> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
>> >> > Thread[ODEServer-78,5,main]: lock(iid=36423, time=1MICROSECONDS)
>> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
>> >> > Thread[ODEServer-78,5,main]: lock(iid=36423,
>> >> > time=1MICROSECONDS)-->WAITING(held by Thread[ODEServer-9,5,main])
>> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
>> >> > Thread[ODEServer-78,5,main]: lock(iid=36423,
>> >> time=1MICROSECONDS)-->TIMEOUT
>> >> > (held by Thread[ODEServer-9,5,main])
>> >> > 16:36:12,115 ODEServer-78 DEBUG [BpelEngineImpl] Instance 36423 is
>> busy,
>> >> > rescheduling job.
>> >> > 16:36:12,239 ODEServer-9 DEBUG [InstanceLockManager]
>> >> > Thread[ODEServer-9,5,main]: unlock(iid=36423)
>> >> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in db
>> >> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28 16:36:11
>> >> IST
>> >> > transacted: true persisted: true details: JobDetails( instanceId:
>> 36423
>> >> > mexId: null processId: null type: MATCHER channel: null correlatorId:
>> >> > DebugCallbackPL.debugOpCallback correlationKeySet:
>> >> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount:
>> null
>> >> > inMem: false detailsExt: {})
>> >> >
>> >> > grep jobid
>> >> >
>> >> > 16:36:11,960 ODEServer-9 DEBUG [JdbcDelegate] insertJob
>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94 loaded=true
>> >> > 16:36:12,007 ODEServer-1 DEBUG [SimpleScheduler] todo.enqueue job
>> from
>> >> db:
>> >> > hqejbhcnphr8357nokgnxp for 1362049571960(16:36:11,960)
>> >> > 16:36:12,007 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>> >> > 16:36:12,032 ODEServer-9 DEBUG [SimpleScheduler] scheduled immediate
>> >> job:
>> >> > hqejbhcnphr8357nokgnxp
>> >> > 16:36:12,239 ODEServer-9 DEBUG [SimpleScheduler] Job
>> >> hqejbhcnphr8357nokgnxp
>> >> > is being processed (outstanding job)
>> >> > 16:36:13,116 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>> >> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no
>> longer
>> >> in
>> >> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
>> >> > 16:36:14,118 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>> >> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no
>> longer
>> >> in
>> >> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
>> >> > 16:36:15,119 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>> >> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in db
>> >> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28 16:36:11
>> >> IST
>> >> > transacted: true persisted: true details: JobDetails( instanceId:
>> 36423
>> >> > mexId: null processId: null type: MATCHER channel: null correlatorId:
>> >> > DebugCallbackPL.debugOpCallback correlationKeySet:
>> >> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount:
>> null
>> >> > inMem: false detailsExt: {})
>> >> >
>> >> >
>> >> > Is this the expected behavior or is there any additional settings i
>> >> should
>> >> > configure to make transaction manager restore job to db at rollback ?
>> >> > Will reinserting the job back to db when the  JobProcessorException
>> >> happens
>> >> >  fix this problem ?
>> >> >
>> >> > Regards
>> >> > Nandika
>> >> >
>> >> > [1] http://ode.apache.org/war-deployment.html
>> >> >
>> >>
>> >
>> >
>>
>
>

Re: JobNoLongerInDbException and Incomplete instances.

Posted by Sathwik B P <sa...@gmail.com>.
Hi,
The transaction manager is initialised even before the database config is
loaded. Without initialization of the transaction manager the ode server
will not start. Irrespective of the DB mode the transaction manager will be
set on the Database object that has being configured, be it internal or
external.
Please check the init(..) method in org.apache.ode.axis2.ODEServer

init(...){
    initTxMgr()
    initDataSource();
    initDAO();
}

Please share your configuration files and logs files.

regards,
sathwik

On Sat, Mar 23, 2013 at 9:32 PM, Nandika Jayawardana <ja...@gmail.com>wrote:

> I did some further checking and with the database configured as db.mode
> internal , everything works fine. For the internal db mode, db connections
> are associated with default geronimo transaction manager ( Database.java ).
> I think for the default external db configuration given , this does not
> happen and hence the issue.
>
> Regards
> Nandika
>
> On Fri, Mar 1, 2013 at 2:17 PM, Nandika Jayawardana <jayawark@gmail.com
> >wrote:
>
> > Hi Sathwik,
> >
> > I am running ode with tomcat 7.0.29 and mysql 5.5.29 version. I used the
> > configuration settings given under "Configuring ODE in Tomcat with MySql
> > database". from ode war deployment guide. {
> > http://ode.apache.org/war-deployment.html }.
> > As you have explained, when the JobProcessorException is thrown due to
> >  instance lock timeout , the transaction will be rollback and default
> retry
> > setting of 3 times will happen. However, the restoration of the deleted
> job
> > back to job table does not happen. Therefore subsequent retries will also
> > result in JobNoLongerInDbException. At execTransaction method, when the
> > retry loop is over, the exception thrown will also be
> >  JobNoLongerInDbException. Since this exception is caught at  "catch
> > (JobNoLongerInDbException jde) " block, it will never go into the
> > exponential back off setting.
> >
> > Is there any additional configuration settings I need to do ?
> >
> > Regards
> > Nandika
> >
> >
> > On Fri, Mar 1, 2013 at 12:21 PM, Sathwik B P <sa...@gmail.com>
> wrote:
> >
> >> Hi,
> >> This is really strange.
> >>
> >> This is the ideal behaviour:
> >> If a job fails for any reason it gets retried defined by the parameter
> >> (immediateRetryCount default 3 times with a time interval
> >> _immediateTransactionRetryInterval default 1 sec) and then the scheduler
> >> will put it on a exponential backoff defined by pow(5,retryCount) where
> >> retryCount is <= 10.
> >>
> >> If the rollback doesn't happen incase of any exception then none of the
> >> jobs will ever complete since it will never go into the exponential
> >> backoff
> >> path.
> >>
> >> In my opinion the transaction manager will maintain the jdbc connection
> >> object throughout it's execution, no matter how many times the
> connection
> >> is borrowed during the transaction.
> >>
> >> Which database are you using and what configuration changes have you
> done
> >> in ode-axis.properties.
> >>
> >> regards,
> >> sathwik
> >>
> >> On Fri, Mar 1, 2013 at 1:31 AM, Nandika Jayawardana <jayawark@gmail.com
> >> >wrote:
> >>
> >> > Hi All,
> >> > I am running ode trunk build with apache tomcat as described in [1] .
> I
> >> > have an asynchronous bpel process which has a receive, invoke and a
> >> > receive. When I run this process for a while, I see that there are few
> >> > incomplete instances, although all the expected messages reached ode.
> >> From
> >> > the debug logs, I figured that it is happening as follows.
> >> >
> >> >   If a thread executing a job tries to acquire the process instance
> >> lock,
> >> > while another thread is executing on the same instance and times out,
> it
> >> > will throw a timeout exception at InstanceLockManager which will be
> >> wrapped
> >> > to a  JobProcessorException.
> >> >
> >> > In SimpleScheduler, RunJob.call method, when the execution of a job
> >> starts,
> >> > it will try to delete the job from the db. For the initial try, it
> >> > would succeed since the job is in db. However, when
> >> > the JobProcessorException exception happens due to timeout on instance
> >> > lock, the transaction gets rolled back. Ideally, the job should be
> >> restored
> >> > back when the rollback happens. However, the job does not get restored
> >> to
> >> > db as the transaction manager and db resources are not associated.
> Hence
> >> > when the scheduler  tries to retry 3 times by default, it will fail
> with
> >> > job no longer in db error.  This results in few of the process
> instances
> >> > never completing since the job was abandoned even though the messages
> >> > reached ode.
> >> >
> >> > Following log extracts from the ode log explains the scenario.
> >> >
> >> > grep instanceid
> >> >
> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> >> > Thread[ODEServer-78,5,main]: lock(iid=36423, time=1MICROSECONDS)
> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> >> > Thread[ODEServer-78,5,main]: lock(iid=36423,
> >> > time=1MICROSECONDS)-->WAITING(held by Thread[ODEServer-9,5,main])
> >> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> >> > Thread[ODEServer-78,5,main]: lock(iid=36423,
> >> time=1MICROSECONDS)-->TIMEOUT
> >> > (held by Thread[ODEServer-9,5,main])
> >> > 16:36:12,115 ODEServer-78 DEBUG [BpelEngineImpl] Instance 36423 is
> busy,
> >> > rescheduling job.
> >> > 16:36:12,239 ODEServer-9 DEBUG [InstanceLockManager]
> >> > Thread[ODEServer-9,5,main]: unlock(iid=36423)
> >> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in db
> >> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28 16:36:11
> >> IST
> >> > transacted: true persisted: true details: JobDetails( instanceId:
> 36423
> >> > mexId: null processId: null type: MATCHER channel: null correlatorId:
> >> > DebugCallbackPL.debugOpCallback correlationKeySet:
> >> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount:
> null
> >> > inMem: false detailsExt: {})
> >> >
> >> > grep jobid
> >> >
> >> > 16:36:11,960 ODEServer-9 DEBUG [JdbcDelegate] insertJob
> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94 loaded=true
> >> > 16:36:12,007 ODEServer-1 DEBUG [SimpleScheduler] todo.enqueue job from
> >> db:
> >> > hqejbhcnphr8357nokgnxp for 1362049571960(16:36:11,960)
> >> > 16:36:12,007 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> >> > 16:36:12,032 ODEServer-9 DEBUG [SimpleScheduler] scheduled immediate
> >> job:
> >> > hqejbhcnphr8357nokgnxp
> >> > 16:36:12,239 ODEServer-9 DEBUG [SimpleScheduler] Job
> >> hqejbhcnphr8357nokgnxp
> >> > is being processed (outstanding job)
> >> > 16:36:13,116 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> >> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no
> longer
> >> in
> >> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
> >> > 16:36:14,118 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> >> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no
> longer
> >> in
> >> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
> >> > 16:36:15,119 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> >> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> >> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in db
> >> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28 16:36:11
> >> IST
> >> > transacted: true persisted: true details: JobDetails( instanceId:
> 36423
> >> > mexId: null processId: null type: MATCHER channel: null correlatorId:
> >> > DebugCallbackPL.debugOpCallback correlationKeySet:
> >> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount:
> null
> >> > inMem: false detailsExt: {})
> >> >
> >> >
> >> > Is this the expected behavior or is there any additional settings i
> >> should
> >> > configure to make transaction manager restore job to db at rollback ?
> >> > Will reinserting the job back to db when the  JobProcessorException
> >> happens
> >> >  fix this problem ?
> >> >
> >> > Regards
> >> > Nandika
> >> >
> >> > [1] http://ode.apache.org/war-deployment.html
> >> >
> >>
> >
> >
>

Re: JobNoLongerInDbException and Incomplete instances.

Posted by Nandika Jayawardana <ja...@gmail.com>.
I did some further checking and with the database configured as db.mode
internal , everything works fine. For the internal db mode, db connections
are associated with default geronimo transaction manager ( Database.java ).
I think for the default external db configuration given , this does not
happen and hence the issue.

Regards
Nandika

On Fri, Mar 1, 2013 at 2:17 PM, Nandika Jayawardana <ja...@gmail.com>wrote:

> Hi Sathwik,
>
> I am running ode with tomcat 7.0.29 and mysql 5.5.29 version. I used the
> configuration settings given under "Configuring ODE in Tomcat with MySql
> database". from ode war deployment guide. {
> http://ode.apache.org/war-deployment.html }.
> As you have explained, when the JobProcessorException is thrown due to
>  instance lock timeout , the transaction will be rollback and default retry
> setting of 3 times will happen. However, the restoration of the deleted job
> back to job table does not happen. Therefore subsequent retries will also
> result in JobNoLongerInDbException. At execTransaction method, when the
> retry loop is over, the exception thrown will also be
>  JobNoLongerInDbException. Since this exception is caught at  "catch
> (JobNoLongerInDbException jde) " block, it will never go into the
> exponential back off setting.
>
> Is there any additional configuration settings I need to do ?
>
> Regards
> Nandika
>
>
> On Fri, Mar 1, 2013 at 12:21 PM, Sathwik B P <sa...@gmail.com> wrote:
>
>> Hi,
>> This is really strange.
>>
>> This is the ideal behaviour:
>> If a job fails for any reason it gets retried defined by the parameter
>> (immediateRetryCount default 3 times with a time interval
>> _immediateTransactionRetryInterval default 1 sec) and then the scheduler
>> will put it on a exponential backoff defined by pow(5,retryCount) where
>> retryCount is <= 10.
>>
>> If the rollback doesn't happen incase of any exception then none of the
>> jobs will ever complete since it will never go into the exponential
>> backoff
>> path.
>>
>> In my opinion the transaction manager will maintain the jdbc connection
>> object throughout it's execution, no matter how many times the connection
>> is borrowed during the transaction.
>>
>> Which database are you using and what configuration changes have you done
>> in ode-axis.properties.
>>
>> regards,
>> sathwik
>>
>> On Fri, Mar 1, 2013 at 1:31 AM, Nandika Jayawardana <jayawark@gmail.com
>> >wrote:
>>
>> > Hi All,
>> > I am running ode trunk build with apache tomcat as described in [1] . I
>> > have an asynchronous bpel process which has a receive, invoke and a
>> > receive. When I run this process for a while, I see that there are few
>> > incomplete instances, although all the expected messages reached ode.
>> From
>> > the debug logs, I figured that it is happening as follows.
>> >
>> >   If a thread executing a job tries to acquire the process instance
>> lock,
>> > while another thread is executing on the same instance and times out, it
>> > will throw a timeout exception at InstanceLockManager which will be
>> wrapped
>> > to a  JobProcessorException.
>> >
>> > In SimpleScheduler, RunJob.call method, when the execution of a job
>> starts,
>> > it will try to delete the job from the db. For the initial try, it
>> > would succeed since the job is in db. However, when
>> > the JobProcessorException exception happens due to timeout on instance
>> > lock, the transaction gets rolled back. Ideally, the job should be
>> restored
>> > back when the rollback happens. However, the job does not get restored
>> to
>> > db as the transaction manager and db resources are not associated. Hence
>> > when the scheduler  tries to retry 3 times by default, it will fail with
>> > job no longer in db error.  This results in few of the process instances
>> > never completing since the job was abandoned even though the messages
>> > reached ode.
>> >
>> > Following log extracts from the ode log explains the scenario.
>> >
>> > grep instanceid
>> >
>> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
>> > Thread[ODEServer-78,5,main]: lock(iid=36423, time=1MICROSECONDS)
>> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
>> > Thread[ODEServer-78,5,main]: lock(iid=36423,
>> > time=1MICROSECONDS)-->WAITING(held by Thread[ODEServer-9,5,main])
>> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
>> > Thread[ODEServer-78,5,main]: lock(iid=36423,
>> time=1MICROSECONDS)-->TIMEOUT
>> > (held by Thread[ODEServer-9,5,main])
>> > 16:36:12,115 ODEServer-78 DEBUG [BpelEngineImpl] Instance 36423 is busy,
>> > rescheduling job.
>> > 16:36:12,239 ODEServer-9 DEBUG [InstanceLockManager]
>> > Thread[ODEServer-9,5,main]: unlock(iid=36423)
>> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in db
>> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28 16:36:11
>> IST
>> > transacted: true persisted: true details: JobDetails( instanceId: 36423
>> > mexId: null processId: null type: MATCHER channel: null correlatorId:
>> > DebugCallbackPL.debugOpCallback correlationKeySet:
>> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount: null
>> > inMem: false detailsExt: {})
>> >
>> > grep jobid
>> >
>> > 16:36:11,960 ODEServer-9 DEBUG [JdbcDelegate] insertJob
>> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94 loaded=true
>> > 16:36:12,007 ODEServer-1 DEBUG [SimpleScheduler] todo.enqueue job from
>> db:
>> > hqejbhcnphr8357nokgnxp for 1362049571960(16:36:11,960)
>> > 16:36:12,007 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>> > 16:36:12,032 ODEServer-9 DEBUG [SimpleScheduler] scheduled immediate
>> job:
>> > hqejbhcnphr8357nokgnxp
>> > 16:36:12,239 ODEServer-9 DEBUG [SimpleScheduler] Job
>> hqejbhcnphr8357nokgnxp
>> > is being processed (outstanding job)
>> > 16:36:13,116 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no longer
>> in
>> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
>> > 16:36:14,118 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no longer
>> in
>> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
>> > 16:36:15,119 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
>> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
>> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in db
>> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28 16:36:11
>> IST
>> > transacted: true persisted: true details: JobDetails( instanceId: 36423
>> > mexId: null processId: null type: MATCHER channel: null correlatorId:
>> > DebugCallbackPL.debugOpCallback correlationKeySet:
>> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount: null
>> > inMem: false detailsExt: {})
>> >
>> >
>> > Is this the expected behavior or is there any additional settings i
>> should
>> > configure to make transaction manager restore job to db at rollback ?
>> > Will reinserting the job back to db when the  JobProcessorException
>> happens
>> >  fix this problem ?
>> >
>> > Regards
>> > Nandika
>> >
>> > [1] http://ode.apache.org/war-deployment.html
>> >
>>
>
>

Re: JobNoLongerInDbException and Incomplete instances.

Posted by Nandika Jayawardana <ja...@gmail.com>.
Hi Sathwik,

I am running ode with tomcat 7.0.29 and mysql 5.5.29 version. I used the
configuration settings given under "Configuring ODE in Tomcat with MySql
database". from ode war deployment guide. {
http://ode.apache.org/war-deployment.html }.
As you have explained, when the JobProcessorException is thrown due to
 instance lock timeout , the transaction will be rollback and default retry
setting of 3 times will happen. However, the restoration of the deleted job
back to job table does not happen. Therefore subsequent retries will also
result in JobNoLongerInDbException. At execTransaction method, when the
retry loop is over, the exception thrown will also be
 JobNoLongerInDbException. Since this exception is caught at  "catch
(JobNoLongerInDbException jde) " block, it will never go into the
exponential back off setting.

Is there any additional configuration settings I need to do ?

Regards
Nandika


On Fri, Mar 1, 2013 at 12:21 PM, Sathwik B P <sa...@gmail.com> wrote:

> Hi,
> This is really strange.
>
> This is the ideal behaviour:
> If a job fails for any reason it gets retried defined by the parameter
> (immediateRetryCount default 3 times with a time interval
> _immediateTransactionRetryInterval default 1 sec) and then the scheduler
> will put it on a exponential backoff defined by pow(5,retryCount) where
> retryCount is <= 10.
>
> If the rollback doesn't happen incase of any exception then none of the
> jobs will ever complete since it will never go into the exponential backoff
> path.
>
> In my opinion the transaction manager will maintain the jdbc connection
> object throughout it's execution, no matter how many times the connection
> is borrowed during the transaction.
>
> Which database are you using and what configuration changes have you done
> in ode-axis.properties.
>
> regards,
> sathwik
>
> On Fri, Mar 1, 2013 at 1:31 AM, Nandika Jayawardana <jayawark@gmail.com
> >wrote:
>
> > Hi All,
> > I am running ode trunk build with apache tomcat as described in [1] . I
> > have an asynchronous bpel process which has a receive, invoke and a
> > receive. When I run this process for a while, I see that there are few
> > incomplete instances, although all the expected messages reached ode.
> From
> > the debug logs, I figured that it is happening as follows.
> >
> >   If a thread executing a job tries to acquire the process instance lock,
> > while another thread is executing on the same instance and times out, it
> > will throw a timeout exception at InstanceLockManager which will be
> wrapped
> > to a  JobProcessorException.
> >
> > In SimpleScheduler, RunJob.call method, when the execution of a job
> starts,
> > it will try to delete the job from the db. For the initial try, it
> > would succeed since the job is in db. However, when
> > the JobProcessorException exception happens due to timeout on instance
> > lock, the transaction gets rolled back. Ideally, the job should be
> restored
> > back when the rollback happens. However, the job does not get restored to
> > db as the transaction manager and db resources are not associated. Hence
> > when the scheduler  tries to retry 3 times by default, it will fail with
> > job no longer in db error.  This results in few of the process instances
> > never completing since the job was abandoned even though the messages
> > reached ode.
> >
> > Following log extracts from the ode log explains the scenario.
> >
> > grep instanceid
> >
> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> > Thread[ODEServer-78,5,main]: lock(iid=36423, time=1MICROSECONDS)
> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> > Thread[ODEServer-78,5,main]: lock(iid=36423,
> > time=1MICROSECONDS)-->WAITING(held by Thread[ODEServer-9,5,main])
> > 16:36:12,115 ODEServer-78 DEBUG [InstanceLockManager]
> > Thread[ODEServer-78,5,main]: lock(iid=36423,
> time=1MICROSECONDS)-->TIMEOUT
> > (held by Thread[ODEServer-9,5,main])
> > 16:36:12,115 ODEServer-78 DEBUG [BpelEngineImpl] Instance 36423 is busy,
> > rescheduling job.
> > 16:36:12,239 ODEServer-9 DEBUG [InstanceLockManager]
> > Thread[ODEServer-9,5,main]: unlock(iid=36423)
> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in db
> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28 16:36:11 IST
> > transacted: true persisted: true details: JobDetails( instanceId: 36423
> > mexId: null processId: null type: MATCHER channel: null correlatorId:
> > DebugCallbackPL.debugOpCallback correlationKeySet:
> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount: null
> > inMem: false detailsExt: {})
> >
> > grep jobid
> >
> > 16:36:11,960 ODEServer-9 DEBUG [JdbcDelegate] insertJob
> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94 loaded=true
> > 16:36:12,007 ODEServer-1 DEBUG [SimpleScheduler] todo.enqueue job from
> db:
> > hqejbhcnphr8357nokgnxp for 1362049571960(16:36:11,960)
> > 16:36:12,007 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> > 16:36:12,032 ODEServer-9 DEBUG [SimpleScheduler] scheduled immediate job:
> > hqejbhcnphr8357nokgnxp
> > 16:36:12,239 ODEServer-9 DEBUG [SimpleScheduler] Job
> hqejbhcnphr8357nokgnxp
> > is being processed (outstanding job)
> > 16:36:13,116 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no longer
> in
> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
> > 16:36:14,118 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> > org.apache.ode.scheduler.simple.JobNoLongerInDbException: Job no longer
> in
> > db: hqejbhcnphr8357nokgnxp nodeId=hqejbhcnphr8357nokgj94
> > 16:36:15,119 ODEServer-78 DEBUG [JdbcDelegate] deleteJob
> > hqejbhcnphr8357nokgnxp on node hqejbhcnphr8357nokgj94
> > 16:36:15,120 ODEServer-78 DEBUG [SimpleScheduler] job no longer in db
> > forced rollback: Job hqejbhcnphr8357nokgnxp time: 2013-02-28 16:36:11 IST
> > transacted: true persisted: true details: JobDetails( instanceId: 36423
> > mexId: null processId: null type: MATCHER channel: null correlatorId:
> > DebugCallbackPL.debugOpCallback correlationKeySet:
> > @2[CorrelationSet~746ee3bf-4c4c-4da9-bdb0-233a760ce377] retryCount: null
> > inMem: false detailsExt: {})
> >
> >
> > Is this the expected behavior or is there any additional settings i
> should
> > configure to make transaction manager restore job to db at rollback ?
> > Will reinserting the job back to db when the  JobProcessorException
> happens
> >  fix this problem ?
> >
> > Regards
> > Nandika
> >
> > [1] http://ode.apache.org/war-deployment.html
> >
>