You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@geode.apache.org by "Barry Oglesby (JIRA)" <ji...@apache.org> on 2015/12/11 01:59:10 UTC

[jira] [Commented] (GEODE-657) Two backups occurring simultaneously can cause the BackupLock to be obtained but not released

    [ https://issues.apache.org/jira/browse/GEODE-657?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15051942#comment-15051942 ] 

Barry Oglesby commented on GEODE-657:
-------------------------------------

Here are some more symptoms of this issue.

The logs contain messages like these:
{noformat}
[warning 2015/11/18 13:03:49.419 EST server3 <ServerConnection on port 40438 Thread 11> tid=0x340] 15 secs have elapsed waiting for a primary for bucket [BucketAdvisor /__PR/_B__dataRegion_77:718: state=NO_PRIMARY_HOSTING]. Current bucket owners [11.3.17.200(server3:438)<v360>:30436]

[warning 2015/11/19 13:13:14.961 EST server3 <ServerConnection on port 40438 Thread 3> tid=0x1cd] 15 secs have elapsed waiting for a primary for bucket [BucketAdvisor /__PR/_B__dataRegion_16:657: state=NO_PRIMARY_NOT_HOSTING]. Current bucket owners [11.3.17.200(server3:438)<v360>:30436, 11.3.17.205(server1:3889)<v357>:30435]
{noformat}
These messages mean that a bucket has been created, but no primary bucket has been determined.

When a test with 4 servers is in this state, then put operations do not complete. When a put is done that attempts to create a bucket, it times out, and the threads are stuck here:

The {{ServerConnection}} thread (processing the put operation) for 3 members is stuck here:
{noformat}
"ServerConnection on port 63861 Thread 3" prio=5 tid=0x00007fe3a5123800 nid=0x5727 in Object.wait() [0x0000000156537000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
	at java.lang.Object.wait(Native Method)
	- waiting on <0x0000000120cf7700> (a com.gemstone.gemfire.internal.cache.BucketAdvisor)
	at com.gemstone.gemfire.internal.cache.BucketAdvisor.waitForPrimaryMember(BucketAdvisor.java:1530)
	- locked <0x0000000120cf7700> (a com.gemstone.gemfire.internal.cache.BucketAdvisor)
	at com.gemstone.gemfire.internal.cache.BucketAdvisor.waitForNewPrimary(BucketAdvisor.java:901)
	at com.gemstone.gemfire.internal.cache.BucketAdvisor.getPrimary(BucketAdvisor.java:872)
	at com.gemstone.gemfire.internal.cache.partitioned.RegionAdvisor.getPrimaryMemberForBucket(RegionAdvisor.java:1232)
	at com.gemstone.gemfire.internal.cache.PartitionedRegion.getBucketPrimary(PartitionedRegion.java:9566)
	at com.gemstone.gemfire.internal.cache.PartitionedRegion.waitForNoStorageOrPrimary(PartitionedRegion.java:3194)
	at com.gemstone.gemfire.internal.cache.PartitionedRegion.getNodeForBucketWrite(PartitionedRegion.java:3177)
	at com.gemstone.gemfire.internal.cache.PartitionedRegion.virtualPut(PartitionedRegion.java:2051)
	at com.gemstone.gemfire.internal.cache.LocalRegionDataView.putEntry(LocalRegionDataView.java:118)
	at com.gemstone.gemfire.internal.cache.LocalRegion.basicUpdate(LocalRegion.java:5780)
	at com.gemstone.gemfire.internal.cache.LocalRegion.basicBridgePut(LocalRegion.java:5412)
	at com.gemstone.gemfire.internal.cache.tier.sockets.command.Put65.cmdExecute(Put65.java:399)
	at com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand.execute(BaseCommand.java:182)
	at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.doNormalMsg(ServerConnection.java:789)
	at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.doOneMessage(ServerConnection.java:920)
	at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.run(ServerConnection.java:1165)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl$1$1.run(AcceptorImpl.java:577)
	at java.lang.Thread.run(Thread.java:745)
{noformat}
The {{ServerConnection}} thread for the 4th member is trying to create the and attempting to lock the {{BackupLock}}:
{noformat}
"ServerConnection on port 63880 Thread 3" prio=5 tid=0x00007fc3fc39c800 nid=0xe61f waiting on condition [0x0000000158300000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00000001221f6630> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitUninterruptibly(AbstractQueuedSynchronizer.java:1979)
	at com.gemstone.gemfire.internal.cache.BackupLock.lock(BackupLock.java:79)
	at com.gemstone.gemfire.internal.cache.BackupLock.lock(BackupLock.java:59)
	at com.gemstone.gemfire.internal.cache.DiskInitFile.addMyInitializingPMID(DiskInitFile.java:2248)
	at com.gemstone.gemfire.internal.cache.DiskStoreImpl.setInitializing(DiskStoreImpl.java:3147)
	at com.gemstone.gemfire.internal.cache.AbstractDiskRegion.setInitializing(AbstractDiskRegion.java:493)
	at com.gemstone.gemfire.internal.cache.persistence.PersistenceAdvisorImpl.setInitializing(PersistenceAdvisorImpl.java:418)
	at com.gemstone.gemfire.internal.cache.BucketPersistenceAdvisor.setInitializing(BucketPersistenceAdvisor.java:391)
	at com.gemstone.gemfire.internal.cache.persistence.PersistenceAdvisorImpl.setOnline(PersistenceAdvisorImpl.java:290)
	at com.gemstone.gemfire.internal.cache.BucketPersistenceAdvisor.endBucketCreation(BucketPersistenceAdvisor.java:435)
	at com.gemstone.gemfire.internal.cache.PRHARedundancyProvider.endBucketCreationLocally(PRHARedundancyProvider.java:883)
	- locked <0x00000001223bb060> (a com.gemstone.gemfire.internal.cache.ProxyBucketRegion)
	at com.gemstone.gemfire.internal.cache.PRHARedundancyProvider.endBucketCreation(PRHARedundancyProvider.java:816)
	at com.gemstone.gemfire.internal.cache.PRHARedundancyProvider.createBucketAtomically(PRHARedundancyProvider.java:676)
	- locked <0x00000001221b94f0> (a com.gemstone.gemfire.internal.cache.PRHARedundancyProvider)
	at com.gemstone.gemfire.internal.cache.PartitionedRegion.createBucket(PartitionedRegion.java:3484)
	at com.gemstone.gemfire.internal.cache.PartitionedRegion.virtualPut(PartitionedRegion.java:2069)
	at com.gemstone.gemfire.internal.cache.LocalRegionDataView.putEntry(LocalRegionDataView.java:118)
	at com.gemstone.gemfire.internal.cache.LocalRegion.basicUpdate(LocalRegion.java:5780)
	at com.gemstone.gemfire.internal.cache.LocalRegion.basicBridgePut(LocalRegion.java:5412)
	at com.gemstone.gemfire.internal.cache.tier.sockets.command.Put65.cmdExecute(Put65.java:399)
	at com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand.execute(BaseCommand.java:182)
	at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.doNormalMsg(ServerConnection.java:789)
	at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.doOneMessage(ServerConnection.java:920)
	at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.run(ServerConnection.java:1165)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl$1$1.run(AcceptorImpl.java:577)
	at java.lang.Thread.run(Thread.java:745)
{noformat}
In addition, one of the other members is also attempting to create the bucket and attempting to lock the {{BackupLock}}:
{noformat}
"Pooled Waiting Message Processor 0" daemon prio=5 tid=0x00007ff77e073000 nid=0x9d03 waiting on condition [0x0000000155d57000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00000001202cdca8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitUninterruptibly(AbstractQueuedSynchronizer.java:1979)
	at com.gemstone.gemfire.internal.cache.BackupLock.lock(BackupLock.java:79)
	at com.gemstone.gemfire.internal.cache.BackupLock.lock(BackupLock.java:59)
	at com.gemstone.gemfire.internal.cache.DiskInitFile.addMyInitializingPMID(DiskInitFile.java:2248)
	at com.gemstone.gemfire.internal.cache.DiskStoreImpl.setInitializing(DiskStoreImpl.java:3147)
	at com.gemstone.gemfire.internal.cache.AbstractDiskRegion.setInitializing(AbstractDiskRegion.java:493)
	at com.gemstone.gemfire.internal.cache.persistence.PersistenceAdvisorImpl.setInitializing(PersistenceAdvisorImpl.java:418)
	at com.gemstone.gemfire.internal.cache.BucketPersistenceAdvisor.setInitializing(BucketPersistenceAdvisor.java:391)
	at com.gemstone.gemfire.internal.cache.persistence.PersistenceAdvisorImpl.setOnline(PersistenceAdvisorImpl.java:290)
	at com.gemstone.gemfire.internal.cache.BucketPersistenceAdvisor.endBucketCreation(BucketPersistenceAdvisor.java:435)
	at com.gemstone.gemfire.internal.cache.PRHARedundancyProvider.endBucketCreationLocally(PRHARedundancyProvider.java:883)
	- locked <0x0000000120e192d0> (a com.gemstone.gemfire.internal.cache.ProxyBucketRegion)
	at com.gemstone.gemfire.internal.cache.partitioned.EndBucketCreationMessage.operateOnPartitionedRegion(EndBucketCreationMessage.java:104)
	at com.gemstone.gemfire.internal.cache.partitioned.PartitionMessage.process(PartitionMessage.java:300)
	at com.gemstone.gemfire.distributed.internal.DistributionMessage.scheduleAction(DistributionMessage.java:386)
	at com.gemstone.gemfire.distributed.internal.DistributionMessage$1.run(DistributionMessage.java:457)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at com.gemstone.gemfire.distributed.internal.DistributionManager.runUntilShutdown(DistributionManager.java:692)
	at com.gemstone.gemfire.distributed.internal.DistributionManager$6$1.run(DistributionManager.java:1029)
	at java.lang.Thread.run(Thread.java:745)
{noformat}

> Two backups occurring simultaneously can cause the BackupLock to be obtained but not released
> ---------------------------------------------------------------------------------------------
>
>                 Key: GEODE-657
>                 URL: https://issues.apache.org/jira/browse/GEODE-657
>             Project: Geode
>          Issue Type: Bug
>          Components: persistence
>            Reporter: Barry Oglesby
>
> Backup is done in three phases, namely {{FlushToDiskRequest}}, {{PrepareBackupRequest}} and {{FinishBackupRequest}}.
> Here is some debugging in the locator showing the backup phases:
> {noformat}
> [warning 2015/12/01 14:37:38.783 PST locator <RMI TCP Connection(3)-192.168.2.11> tid=0x99] AdminDistributedSystemImpl backupAllMembers STARTED
> [warning 2015/12/01 14:37:38.794 PST locator <RMI TCP Connection(3)-192.168.2.11> tid=0x99] AdminDistributedSystemImpl backupAllMembers about to send FlushToDiskRequest
> [warning 2015/12/01 14:37:38.977 PST locator <RMI TCP Connection(3)-192.168.2.11> tid=0x99] AdminDistributedSystemImpl backupAllMembers received all responses to FlushToDiskRequest
> [warning 2015/12/01 14:37:38.977 PST locator <RMI TCP Connection(3)-192.168.2.11> tid=0x99] AdminDistributedSystemImpl backupAllMembers about to send PrepareBackupRequest
> [warning 2015/12/01 14:37:38.984 PST locator <RMI TCP Connection(3)-192.168.2.11> tid=0x99] AdminDistributedSystemImpl backupAllMembers received all responses to PrepareBackupRequest
> [warning 2015/12/01 14:37:38.985 PST locator <RMI TCP Connection(3)-192.168.2.11> tid=0x99] AdminDistributedSystemImpl backupAllMembers about to send FinishBackupRequest
> [warning 2015/12/01 14:37:51.184 PST locator <RMI TCP Connection(3)-192.168.2.11> tid=0x99] AdminDistributedSystemImpl backupAllMembers received all responses to FinishBackupRequest
> [warning 2015/12/01 14:37:51.184 PST locator <RMI TCP Connection(3)-192.168.2.11> tid=0x99] AdminDistributedSystemImpl backupAllMembers COMPLETED
> {noformat}
> Here is some debugging in one of the members. The others are similar.
> {noformat}
> [warning 2015/12/01 14:37:38.795 PST  <Pooled Message Processor 2> tid=0x66] FlushToDiskRequest createResponse received FlushToDiskRequest
> [warning 2015/12/01 14:37:38.974 PST  <Pooled Message Processor 2> tid=0x66] FlushToDiskRequest createResponse sending FlushToDiskResponse
> [warning 2015/12/01 14:37:38.978 PST  <Pooled Message Processor 2> tid=0x66] PrepareBackupRequest createResponse received PrepareBackupRequest
> [warning 2015/12/01 14:37:38.978 PST  <Pooled Message Processor 2> tid=0x66] PrepareBackupResponse createResponse: About to start backup
> [warning 2015/12/01 14:37:38.980 PST  <Pooled Message Processor 2> tid=0x66] PrepareBackupResponse createResponse: About to prepare backup
> [warning 2015/12/01 14:37:38.980 PST  <Pooled Message Processor 2> tid=0x66] BackupLock lockForBackup invoked
> [warning 2015/12/01 14:37:38.980 PST  <Pooled Message Processor 2> tid=0x66] PrepareBackupRequest createResponse sending PrepareBackupResponse
> [warning 2015/12/01 14:37:38.986 PST  <Pooled Message Processor 2> tid=0x66] FinishBackupRequest createResponse received FinishBackupRequest
> [warning 2015/12/01 14:37:39.970 PST  <Pooled Message Processor 2> tid=0x66] BackupLock unlockForBackup invoked
> [warning 2015/12/01 14:37:49.885 PST  <Pooled Message Processor 2> tid=0x66] FinishBackupRequest createResponse sending FinishBackupResponse
> {noformat}
> Processing the {{PrepareBackupRequest}} is where the {{BackupLock}} is taken (in {{BackupLock lockForBackup}}). Processing the {{FinishBackupRequest}} is where it is released (in {{BackupLock unlockForBackup}}).
> All members operate independently. If one completes its backup before the others, the issue can happen. That member is available to start a new backup, but the others aren't.
> In this case, the member successfully processes the {{PrepareBackupRequest}} and takes the {{BackupLock}}.
> The other members do not successfully process the {{PrepareBackupRequest}} and do not take the {{BackupLock}}:
> {noformat}
> [warning 2015/12/01 14:46:02.447 PST  <Pooled Message Processor 3> tid=0xd7] PrepareBackupRequest createResponse received PrepareBackupRequest
> [warning 2015/12/01 14:46:02.448 PST  <Pooled Message Processor 3> tid=0xd7] PrepareBackupResponse createResponse: About to start backup
> [error 2015/12/01 14:46:02.448 PST  <Pooled Message Processor 3> tid=0xd7] Error processing request class com.gemstone.gemfire.admin.internal.PrepareBackupRequest.
> java.io.IOException: Backup already in progress
> 	at com.gemstone.gemfire.internal.cache.GemFireCacheImpl.startBackup(GemFireCacheImpl.java:4315)
> 	at com.gemstone.gemfire.admin.internal.PrepareBackupRequest.createResponse(PrepareBackupRequest.java:87)
> 	at com.gemstone.gemfire.internal.admin.remote.CliLegacyMessage.process(CliLegacyMessage.java:28)
> 	at com.gemstone.gemfire.distributed.internal.DistributionMessage.scheduleAction(DistributionMessage.java:386)
> 	at com.gemstone.gemfire.distributed.internal.DistributionMessage$1.run(DistributionMessage.java:457)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at com.gemstone.gemfire.distributed.internal.DistributionManager.runUntilShutdown(DistributionManager.java:692)
> 	at com.gemstone.gemfire.distributed.internal.DistributionManager$4$1.run(DistributionManager.java:963)
> 	at java.lang.Thread.run(Thread.java:745)
> [warning 2015/12/01 14:46:02.448 PST  <Pooled Message Processor 3> tid=0xd7] PrepareBackupResponse createResponse: Returning exception java.io.IOException: Backup already in progress
> {noformat}
> Once this happens, the members are out of sync. One has taken the {{BackupLock}}; the others have not. The member that has taken the {{BackupLock}} will never release it.
> One solution would be to prevent any backup from starting if one is in progress (in the locator). Another would be to force the member to release the {{BackupLock}} if other members fail to get it. I think this would require a new message, though.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)