You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@crail.apache.org by pe...@apache.org on 2018/04/10 11:19:41 UTC
[02/16] incubator-crail git commit: NvmfFuture: fix concurrency bug
NvmfFuture: fix concurrency bug
Fix bug when completed counter was incremented in parallel
and never reached 2.
Signed-off-by: Jonas Pfefferle <pe...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/incubator-crail/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-crail/commit/feb765c3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-crail/tree/feb765c3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-crail/diff/feb765c3
Branch: refs/heads/master
Commit: feb765c3143d35f3526909b1b69aa33790b21a9d
Parents: ac9e299
Author: Jonas Pfefferle <pe...@apache.org>
Authored: Mon Apr 9 15:18:40 2018 +0200
Committer: Jonas Pfefferle <pe...@apache.org>
Committed: Tue Apr 10 13:19:02 2018 +0200
----------------------------------------------------------------------
.../org/apache/crail/storage/nvmf/client/NvmfFuture.java | 11 ++++++-----
1 file changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-crail/blob/feb765c3/storage-nvmf/src/main/java/org/apache/crail/storage/nvmf/client/NvmfFuture.java
----------------------------------------------------------------------
diff --git a/storage-nvmf/src/main/java/org/apache/crail/storage/nvmf/client/NvmfFuture.java b/storage-nvmf/src/main/java/org/apache/crail/storage/nvmf/client/NvmfFuture.java
index 639f568..885121d 100644
--- a/storage-nvmf/src/main/java/org/apache/crail/storage/nvmf/client/NvmfFuture.java
+++ b/storage-nvmf/src/main/java/org/apache/crail/storage/nvmf/client/NvmfFuture.java
@@ -28,16 +28,17 @@ import java.util.Queue;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
public class NvmfFuture<Command extends NvmIoCommand<? extends NvmIoCommandCapsule>> implements StorageFuture, OperationCallback {
private final NvmfStorageEndpoint endpoint;
private final Command command;
private final Queue<Command> operations;
- private boolean done;
+ private volatile boolean done;
private RdmaException exception;
private final StorageResult storageResult;
private final Response<NvmResponseCapsule> response;
- private int completed;
+ private final AtomicInteger completed;
NvmfFuture(NvmfStorageEndpoint endpoint, Command command, Response<NvmResponseCapsule> response,
Queue<Command> operations, int length) {
@@ -47,7 +48,7 @@ public class NvmfFuture<Command extends NvmIoCommand<? extends NvmIoCommandCapsu
this.done = false;
this.storageResult = () -> length;
this.response = response;
- this.completed = 0;
+ this.completed = new AtomicInteger(0);
}
@Override
@@ -129,8 +130,8 @@ public class NvmfFuture<Command extends NvmIoCommand<? extends NvmIoCommandCapsu
@Override
public void onComplete() {
assert !done;
- assert completed < 2;
- if (++completed == 2) {
+ assert completed.get() < 2;
+ if (completed.incrementAndGet() == 2) {
/* we need to complete command and response */
operations.add(command);
this.done = true;