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 2019/07/04 11:26:33 UTC

[incubator-crail] branch master updated: [storage-rdma] Make StoragerServer ep active to prevent orphaned CQ

This is an automated email from the ASF dual-hosted git repository.

pepperjo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-crail.git


The following commit(s) were added to refs/heads/master by this push:
     new ec2179e  [storage-rdma] Make StoragerServer ep active to prevent orphaned CQ
ec2179e is described below

commit ec2179e8d85fd36ca0572a3178454b581e67d057
Author: BernardMetzler <bm...@zurich.ibm.com>
AuthorDate: Thu Jul 4 13:26:29 2019 +0200

    [storage-rdma] Make StoragerServer ep active to prevent orphaned CQ
    
    The passive endpoint model caused an orphaned CQ at DataNode
    after each client disconnect. Each new client connection caused
    creation of another CQ. Changing to an active endpoint model
    frees the per client CQ after client disconnect. Since the CQ is
    not actually used (the endpoint handles only RDMA Read and
    Write from client side), the active endpoint gets created with
    minimum resources (send/receive/completion queue size == 1),
    and the CQ event dispatcher is a nop.
    
    Fixes https://issues.apache.org/jira/projects/CRAIL/issues/CRAIL-103
    
    Signed-off-by: Bernard Metzler <bm...@zurich.ibm.com>
---
 .../org/apache/crail/storage/rdma/RdmaStorageEndpointFactory.java | 4 ++--
 .../java/org/apache/crail/storage/rdma/RdmaStorageServer.java     | 6 +++---
 .../org/apache/crail/storage/rdma/RdmaStorageServerEndpoint.java  | 8 ++++++--
 3 files changed, 11 insertions(+), 7 deletions(-)

diff --git a/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageEndpointFactory.java b/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageEndpointFactory.java
index e0eb1b6..9cd5d29 100644
--- a/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageEndpointFactory.java
+++ b/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageEndpointFactory.java
@@ -25,9 +25,9 @@ import com.ibm.disni.verbs.*;
 
 public class RdmaStorageEndpointFactory implements RdmaEndpointFactory<RdmaStorageServerEndpoint> {
 	private RdmaStorageServer closer;
-	private RdmaPassiveEndpointGroup<RdmaStorageServerEndpoint> group;
+	private RdmaActiveEndpointGroup<RdmaStorageServerEndpoint> group;
 	
-	public RdmaStorageEndpointFactory(RdmaPassiveEndpointGroup<RdmaStorageServerEndpoint> group, RdmaStorageServer closer){
+	public RdmaStorageEndpointFactory(RdmaActiveEndpointGroup<RdmaStorageServerEndpoint> group, RdmaStorageServer closer){
 		this.group = group;
 		this.closer = closer;
 	}
diff --git a/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageServer.java b/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageServer.java
index 5393275..e808864 100644
--- a/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageServer.java
+++ b/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageServer.java
@@ -39,7 +39,7 @@ public class RdmaStorageServer implements Runnable, StorageServer {
 	private static final Logger LOG = CrailUtils.getLogger();
 	
 	private InetSocketAddress serverAddr;
-	private RdmaPassiveEndpointGroup<RdmaStorageServerEndpoint> datanodeGroup;
+	private RdmaActiveEndpointGroup<RdmaStorageServerEndpoint> datanodeGroup;
 	private RdmaServerEndpoint<RdmaStorageServerEndpoint> datanodeServerEndpoint;
 	private ConcurrentHashMap<Integer, RdmaEndpoint> allEndpoints; 
 	private boolean isAlive;
@@ -64,8 +64,8 @@ public class RdmaStorageServer implements Runnable, StorageServer {
 			LOG.info("Configured network interface " + RdmaConstants.STORAGE_RDMA_INTERFACE + " cannot be found..exiting!!!");
 			return;
 		}
-		this.datanodeGroup = new RdmaPassiveEndpointGroup<RdmaStorageServerEndpoint>(-1, RdmaConstants.STORAGE_RDMA_QUEUESIZE, 4, RdmaConstants.STORAGE_RDMA_QUEUESIZE*100);
-		this.datanodeServerEndpoint = datanodeGroup.createServerEndpoint();		
+		this.datanodeGroup = new RdmaActiveEndpointGroup<RdmaStorageServerEndpoint>(-1, false, 1, 1, 1);
+		this.datanodeServerEndpoint = datanodeGroup.createServerEndpoint();
 		datanodeGroup.init(new RdmaStorageEndpointFactory(datanodeGroup, this));
 		datanodeServerEndpoint.bind(serverAddr, RdmaConstants.STORAGE_RDMA_BACKLOG);
 		
diff --git a/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageServerEndpoint.java b/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageServerEndpoint.java
index 7f24292..e144ad7 100644
--- a/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageServerEndpoint.java
+++ b/storage-rdma/src/main/java/org/apache/crail/storage/rdma/RdmaStorageServerEndpoint.java
@@ -23,13 +23,17 @@ import java.io.IOException;
 import com.ibm.disni.verbs.*;
 import com.ibm.disni.*;
 
-public class RdmaStorageServerEndpoint extends RdmaEndpoint {
+public class RdmaStorageServerEndpoint extends RdmaActiveEndpoint {
 	private RdmaStorageServer closer;
 
-	public RdmaStorageServerEndpoint(RdmaPassiveEndpointGroup<RdmaStorageServerEndpoint> endpointGroup, RdmaCmId idPriv, RdmaStorageServer closer, boolean serverSide) throws IOException {	
+	public RdmaStorageServerEndpoint(RdmaActiveEndpointGroup<RdmaStorageServerEndpoint> endpointGroup, RdmaCmId idPriv, RdmaStorageServer closer, boolean serverSide) throws IOException {
 		super(endpointGroup, idPriv, serverSide);
 		this.closer = closer;
 	}	
+
+	public void dispatchCqEvent(IbvWC wc) throws IOException {
+
+	}
 	
 	public synchronized void dispatchCmEvent(RdmaCmEvent cmEvent)
 			throws IOException {