-
Notifications
You must be signed in to change notification settings - Fork 588
HDDS-8575. Intermittent failure in TestCloseContainerEventHandler.testCloseContainerWithDelayByLeaseManager #4688
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 3 commits
224db88
f8cbc68
c89abc0
3d89d98
c7812ef
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,12 +19,16 @@ | |
|
|
||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.concurrent.BlockingQueue; | ||
| import java.util.concurrent.Callable; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Executors; | ||
|
|
||
| import static org.apache.hadoop.ozone.lease.Lease.messageForResource; | ||
|
|
||
| import java.util.concurrent.LinkedBlockingQueue; | ||
| import java.util.concurrent.TimeUnit; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -45,8 +49,8 @@ public class LeaseManager<T> { | |
|
|
||
| private final String name; | ||
| private final long defaultTimeout; | ||
| private final Object monitor = new Object(); | ||
| private Map<T, Lease<T>> activeLeases; | ||
| private BlockingQueue<T> leaseKeyBlockingQueue; | ||
| private LeaseMonitor leaseMonitor; | ||
| private Thread leaseMonitorThread; | ||
| private boolean isRunning; | ||
|
|
@@ -62,6 +66,7 @@ public class LeaseManager<T> { | |
| public LeaseManager(String name, long defaultTimeout) { | ||
| this.name = name; | ||
| this.defaultTimeout = defaultTimeout; | ||
| leaseKeyBlockingQueue = new LinkedBlockingQueue<>(); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -120,9 +125,7 @@ public synchronized Lease<T> acquire(T resource, long timeout) | |
| } | ||
| Lease<T> lease = new Lease<>(resource, timeout); | ||
| activeLeases.put(resource, lease); | ||
| synchronized (monitor) { | ||
| monitor.notifyAll(); | ||
| } | ||
| leaseKeyBlockingQueue.add(resource); | ||
| return lease; | ||
| } | ||
|
|
||
|
|
@@ -151,9 +154,7 @@ public synchronized Lease<T> acquire( | |
| Lease<T> lease = new Lease<>(resource, timeout); | ||
| lease.registerCallBack(callback); | ||
| activeLeases.put(resource, lease); | ||
| synchronized (monitor) { | ||
| monitor.notifyAll(); | ||
| } | ||
| leaseKeyBlockingQueue.add(resource); | ||
| return lease; | ||
| } | ||
|
|
||
|
|
@@ -204,9 +205,8 @@ public void shutdown() { | |
| checkStatus(); | ||
| LOG.debug("Shutting down LeaseManager service"); | ||
| leaseMonitor.disable(); | ||
| synchronized (monitor) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Updated...
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @szetszwo currently I have reverted this as shutdown is threadsafe. Adding synchronization needs protect all member variables as findbug, which is not required. |
||
| monitor.notifyAll(); | ||
| } | ||
| leaseMonitorThread.interrupt(); | ||
| leaseKeyBlockingQueue.clear(); | ||
| for (T resource : activeLeases.keySet()) { | ||
| try { | ||
| release(resource); | ||
|
|
@@ -265,11 +265,12 @@ public void run() { | |
| } | ||
|
|
||
| try { | ||
| synchronized (monitor) { | ||
| monitor.wait(sleepTime); | ||
| } | ||
| // block for event and clear all events as will be | ||
| // handled by activeLeases before going for next wait | ||
| // ignore return value | ||
| T task = leaseKeyBlockingQueue.poll(sleepTime, TimeUnit.MILLISECONDS); | ||
| leaseKeyBlockingQueue.clear(); | ||
| } catch (InterruptedException e) { | ||
| // This means a new lease is added to activeLeases. | ||
| LOG.warn("Lease manager is interrupted. Shutting down...", e); | ||
| Thread.currentThread().interrupt(); | ||
| } | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should use Semaphore in this case; see https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Semaphore.html
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@szetszwo Thanks for review, I have done changes as per patch, but added extra release in shutdown() as possibility for concurrency issue. i.e.
For step 2, if MonitorThread waiting, then interrupt will work else ignored. So later moves to waiting for semaphore, then there is no way to exist thread.
So added extra "semaphore.release()" in shutdown to ensure exit from tryAcquire() wait in above case.