You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by hi...@apache.org on 2016/06/03 16:50:44 UTC

incubator-geode git commit: GEODE-1372 Added support to keep multiple copies of cipher as those are not thread safe. Added unit test for it.

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-1372 667c4259f -> 105301940


GEODE-1372 Added support to keep multiple copies of cipher as those
are not thread safe. Added unit test for it.


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/10530194
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/10530194
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/10530194

Branch: refs/heads/feature/GEODE-1372
Commit: 1053019409f3ae6eb9477b56d48123f3a2f869d5
Parents: 667c425
Author: Hitesh Khamesra <hi...@yahoo.com>
Authored: Fri Jun 3 09:49:05 2016 -0700
Committer: Hitesh Khamesra <hi...@yahoo.com>
Committed: Fri Jun 3 09:49:05 2016 -0700

----------------------------------------------------------------------
 .../membership/gms/messenger/GMSEncrypt.java    |  89 ++++++++---
 .../gemfire/distributed/LocatorDUnitTest.java   |  30 ++--
 .../LocatorUDPSecurityDUnitTest.java            |   2 +-
 .../gms/messenger/GMSEncryptJUnitTest.java      | 148 +++++++++++++++----
 4 files changed, 206 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/10530194/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSEncrypt.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSEncrypt.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSEncrypt.java
index 047bb03..c9133b0 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSEncrypt.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSEncrypt.java
@@ -20,6 +20,7 @@ import java.math.BigInteger;
 import java.security.*;
 import java.security.spec.PKCS8EncodedKeySpec;
 import java.security.spec.X509EncodedKeySpec;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -35,8 +36,6 @@ import javax.crypto.spec.SecretKeySpec;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.NetView;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
-
-
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 
 public class GMSEncrypt implements Cloneable {
@@ -65,7 +64,11 @@ public class GMSEncrypt implements Cloneable {
 
   private NetView view;
 
-  private Map<InternalDistributedMember, PeerEncryptor> memberToPeerEncryptor = new ConcurrentHashMap<>();
+  private int numberOfPeerEncryptorCopies = 10;
+  private ConcurrentHashMap<InternalDistributedMember, PeerEncryptor>[] copyOfPeerEncryptors;
+  private ClusterEncryptor[] clusterEncryptors;
+  
+  private Map<InternalDistributedMember, byte[]> memberToPeerEncryptor = new ConcurrentHashMap<>();
 
   private ClusterEncryptor clusterEncryptor;
 
@@ -85,6 +88,7 @@ public class GMSEncrypt implements Cloneable {
   protected synchronized void initClusterSecretKey() throws Exception {
     if(this.clusterEncryptor == null) {
       this.clusterEncryptor = new ClusterEncryptor(this);
+      
     }
   }
 
@@ -94,19 +98,26 @@ public class GMSEncrypt implements Cloneable {
   }
 
   protected GMSEncrypt() {
-
+    initEncryptors();
   }
 
   public GMSEncrypt(Services services) throws Exception {
     this.services = services;
+    initEncryptors();
     initDHKeys(services.getConfig().getDistributionConfig());
   }
 
   public GMSEncrypt(Services services, InternalDistributedMember mbr) throws Exception {
     this.services = services;
     this.localMember = mbr;
+    initEncryptors();
     initDHKeys(services.getConfig().getDistributionConfig());
   }
+  
+  void initEncryptors() {
+    copyOfPeerEncryptors = new  ConcurrentHashMap[numberOfPeerEncryptorCopies];
+    clusterEncryptors = new ClusterEncryptor[numberOfPeerEncryptorCopies];
+  }
 
   public byte[] decryptData(byte[] data, InternalDistributedMember member) throws Exception {
     return getPeerEncryptor(member).decryptBytes(data);
@@ -117,7 +128,7 @@ public class GMSEncrypt implements Cloneable {
   }
 
   public byte[] decryptData(byte[] data) throws Exception {
-    return this.clusterEncryptor.decryptBytes(data);
+    return getClusterEncryptor().decryptBytes(data);
   }
   
   public byte[] decryptData(byte[] data, byte[] pkBytes) throws Exception {
@@ -126,7 +137,7 @@ public class GMSEncrypt implements Cloneable {
   }
 
   public byte[] encryptData(byte[] data) throws Exception {
-    return this.clusterEncryptor.encryptBytes(data);
+    return getClusterEncryptor().encryptBytes(data);
   }
 
   protected byte[] getPublicKeyBytes() {
@@ -147,7 +158,8 @@ public class GMSEncrypt implements Cloneable {
   
   protected void setPublicKey(byte[] publickey, InternalDistributedMember mbr) {
     try {
-      createPeerEncryptor(mbr, publickey);
+      //createPeerEncryptor(mbr, publickey);
+      memberToPeerEncryptor.put(mbr, publickey);
     }catch(Exception e) {
       throw new RuntimeException("Unable to create peer encryptor " +  mbr, e);
     }
@@ -200,21 +212,56 @@ public class GMSEncrypt implements Cloneable {
   }
 
   protected PeerEncryptor getPeerEncryptor(InternalDistributedMember member) throws Exception {
-    PeerEncryptor result = memberToPeerEncryptor.get(member);
+    Map<InternalDistributedMember, PeerEncryptor> m = getPeerEncryptorMap();
+
+    PeerEncryptor result = m.get(member);
     if (result == null) {
       synchronized (this) {
-        result = memberToPeerEncryptor.get(member);
+        result = m.get(member);
         if (result == null) {
-          result = createPeerEncryptor(member, (byte[]) view.getPublicKey(member));
+          byte[] pk = (byte[])memberToPeerEncryptor.get(member);
+          result = createPeerEncryptor(member, pk != null ? pk : (byte[]) view.getPublicKey(member));
+          m.put(member, result);
         }
       }
     }
     return result;
   }
 
+  private Map<InternalDistributedMember, PeerEncryptor> getPeerEncryptorMap() {
+    int h = Math.abs(Thread.currentThread().getName().hashCode() % numberOfPeerEncryptorCopies);
+    ConcurrentHashMap m = copyOfPeerEncryptors[h];
+    
+    if(m == null) {
+      synchronized (copyOfPeerEncryptors) {
+        m = copyOfPeerEncryptors[h];
+        if(m == null) {
+          m = new ConcurrentHashMap<InternalDistributedMember, PeerEncryptor>();
+          copyOfPeerEncryptors[h] = m;          
+        }
+      }
+    }
+    return m;
+  }
+  
+  private ClusterEncryptor getClusterEncryptor() {
+    int h = Math.abs(Thread.currentThread().getName().hashCode() % numberOfPeerEncryptorCopies);
+    ClusterEncryptor c = clusterEncryptors[h];
+    
+    if(c == null) {
+      synchronized (copyOfPeerEncryptors) {
+        c = clusterEncryptors[h];
+        if(c == null) {
+          c = new ClusterEncryptor(getClusterSecretKey());
+          clusterEncryptors[h] = c;          
+        }
+      }
+    }
+    return c;
+  }
+  
   private PeerEncryptor createPeerEncryptor(InternalDistributedMember member, byte[] peerKeyBytes) throws Exception {
-    PeerEncryptor result = new PeerEncryptor(peerKeyBytes);
-    memberToPeerEncryptor.put(member, result);
+    PeerEncryptor result = new PeerEncryptor(peerKeyBytes);    
     return result;
   }
 
@@ -265,18 +312,12 @@ public class GMSEncrypt implements Cloneable {
   }
 
   static public byte[] encryptBytes(byte[] data, Cipher encrypt) throws Exception {
-    synchronized (GMSEncrypt.class) {
-      encodingsPerformed++;
-    }
     return encrypt.doFinal(data);
   }
 
   static public byte[] decryptBytes(byte[] data, Cipher decrypt) throws Exception {
     try {
       byte[] decryptBytes = decrypt.doFinal(data);
-      synchronized (GMSEncrypt.class) {
-        decodingsPerformed++;
-      }
       return decryptBytes;
     } catch (Exception ex) {
       throw ex;
@@ -339,7 +380,8 @@ public class GMSEncrypt implements Cloneable {
 
   }
 
-  protected static Cipher getEncryptCipher(String dhSKAlgo, PrivateKey privateKey, PublicKey peerPublicKey) throws Exception {
+  //this needs to synchronize as it uses private key of that member
+  protected static synchronized Cipher getEncryptCipher(String dhSKAlgo, PrivateKey privateKey, PublicKey peerPublicKey) throws Exception {
     KeyAgreement ka = KeyAgreement.getInstance("DH");
     ka.init(privateKey);
     ka.doPhase(peerPublicKey, true);
@@ -392,7 +434,8 @@ public class GMSEncrypt implements Cloneable {
     return encrypt;
   }
 
-  protected static Cipher getDecryptCipher(String dhSKAlgo, PrivateKey privateKey, PublicKey publicKey) throws Exception {
+  //this needs to synchronize as it uses private key of that member
+  protected static synchronized Cipher getDecryptCipher(String dhSKAlgo, PrivateKey privateKey, PublicKey publicKey) throws Exception {
     KeyAgreement ka = KeyAgreement.getInstance("DH");
     ka.init(privateKey);
     ka.doPhase(publicKey, true);
@@ -499,8 +542,9 @@ public class GMSEncrypt implements Cloneable {
       try {
         if (encrypt == null) {
           synchronized (this) {
-            if (encrypt == null)
+            if (encrypt == null) {
               encrypt = GMSEncrypt.getEncryptCipher(dhSKAlgo, secretBytes);
+            }
           }
         }
       } catch (Exception ex) {
@@ -518,8 +562,9 @@ public class GMSEncrypt implements Cloneable {
     private Cipher getDecryptCipher(String dhSKAlgo) throws Exception {
       if (decrypt == null) {
         synchronized (this) {
-          if (decrypt == null)
+          if (decrypt == null) {
             decrypt = GMSEncrypt.getDecryptCipher(dhSKAlgo, secretBytes);
+          }
         }
       }
       return decrypt;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/10530194/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
index 701cc07..74872d2 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
@@ -125,7 +125,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    * it hung with the restarted locator trying to become elder again because
    * it put its address at the beginning of the new view it sent out.
    */
-  public void ntestCollocatedLocatorWithSecurity() throws Exception {
+  public void testCollocatedLocatorWithSecurity() throws Exception {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
     VM vm1 = host.getVM(1);
@@ -255,7 +255,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    *
    * @throws Exception
    */
-  public void ntestStartTwoLocators() throws Exception {
+  public void testStartTwoLocators() throws Exception {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
     VM loc1 = host.getVM(1);
@@ -368,7 +368,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
   /**
    * test lead member selection
    */
-  public void ntestLeadMemberSelection() throws Exception {
+  public void testLeadMemberSelection() throws Exception {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
     VM vm1 = host.getVM(1);
@@ -477,7 +477,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    * We then kill the lead member and demonstrate that the original locator
    * (which is now the sole remaining member) shuts itself down.
    */
-  public void ntestLeadAndCoordFailure() throws Exception {
+  public void testLeadAndCoordFailure() throws Exception {
     IgnoredException.addIgnoredException("Possible loss of quorum due");
     disconnectAllFromDS();
     Host host = Host.getHost(0);
@@ -610,7 +610,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    * We then shut down the group coordinator and observe the second locator
    * pick up the job and the remaining member continues to operate normally.
    */
-  public void ntestLeadFailureAndCoordShutdown() throws Exception {
+  public void testLeadFailureAndCoordShutdown() throws Exception {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
     VM vm1 = host.getVM(1);
@@ -756,7 +756,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    */
   // disabled on trunk - should be reenabled on cedar_dev_Oct12
   // this test leaves a CloserThread around forever that logs "pausing" messages every 500 ms
-  public void ntestForceDisconnectAndPeerShutdownCause() throws Exception {
+  public void testForceDisconnectAndPeerShutdownCause() throws Exception {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
     VM vm1 = host.getVM(1);
@@ -893,7 +893,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    * We kill the coordinator and shut down the lead member and observe the second locator
    * pick up the job and the remaining member continue to operate normally.
    */
-  public void ntestLeadShutdownAndCoordFailure() throws Exception {
+  public void testLeadShutdownAndCoordFailure() throws Exception {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
     VM vm1 = host.getVM(1);
@@ -1018,7 +1018,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    * Tests that attempting to connect to a distributed system in which
    * no locator is defined throws an exception.
    */
-  public void ntestNoLocator() {
+  public void testNoLocator() {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
     int port =
@@ -1078,7 +1078,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    * <p>The locator is then restarted and is shown to take over the
    * role of membership coordinator.
    */
-  public void ntestOneLocator() throws Exception {
+  public void testOneLocator() throws Exception {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
@@ -1171,7 +1171,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    * is correct.  It then restarts the locator to demonstrate that
    * it can connect to and function as the group coordinator
    */
-  public void ntestLocatorBecomesCoordinator() throws Exception {
+  public void testLocatorBecomesCoordinator() throws Exception {
     disconnectAllFromDS();
     final String expected = "java.net.ConnectException";
     final String addExpected =
@@ -1308,7 +1308,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
   /**
    * Tests starting multiple locators in multiple VMs.
    */
-  public void ntestMultipleLocators() throws Exception {
+  public void testMultipleLocators() throws Exception {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
@@ -1627,7 +1627,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
   /**
    * Tests starting multiple locators in multiple VMs.
    */
-  public void ntestMultipleMcastLocators() throws Exception {
+  public void testMultipleMcastLocators() throws Exception {
     disconnectAllFromDS();
     IgnoredException.addIgnoredException("Could not stop  Distribution Locator"); // shutdown timing issue in InternalLocator
     Host host = Host.getHost(0);
@@ -1744,7 +1744,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    * Tests that a VM can connect to a locator that is hosted in its
    * own VM.
    */
-  public void ntestConnectToOwnLocator() throws Exception {
+  public void testConnectToOwnLocator() throws Exception {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
 
@@ -1772,7 +1772,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
   /**
    * Tests that a single VM can NOT host multiple locators
    */
-  public void ntestHostingMultipleLocators() throws Exception {
+  public void testHostingMultipleLocators() throws Exception {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
     //VM vm = host.getVM(0);
@@ -1830,7 +1830,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    *
    * @since 4.1
    */
-  public void ntestRestartLocator() throws Exception {
+  public void testRestartLocator() throws Exception {
     disconnectAllFromDS();
     port1 =
         AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/10530194/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorUDPSecurityDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorUDPSecurityDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorUDPSecurityDUnitTest.java
index 37f14c3..f9614b6 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorUDPSecurityDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorUDPSecurityDUnitTest.java
@@ -14,7 +14,7 @@ public class LocatorUDPSecurityDUnitTest extends LocatorDUnitTest{
   
   @Test
   public void testLoop() throws Exception {
-    for(int i=0; i < 2; i++) {
+    for(int i=0; i < 1; i++) {
       testMultipleLocatorsRestartingAtSameTime();
       tearDown();
       setUp();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/10530194/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java
index e625f48..5d4086c 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java
@@ -18,12 +18,16 @@ import java.math.BigInteger;
 import java.security.Key;
 import java.security.KeyPair;
 import java.security.KeyPairGenerator;
+
 import org.junit.experimental.categories.Category;
 
 import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 
 import javax.crypto.KeyAgreement;
 import javax.crypto.Mac;
@@ -111,6 +115,71 @@ public class GMSEncryptJUnitTest {
   }
   
   @Test
+  public void testOneMemberCanDecryptAnothersMessageMultithreaded() throws Exception{
+    initMocks();
+    final int runs = 100000;
+    final GMSEncrypt gmsEncrypt1 = new GMSEncrypt(services, mockMembers[1]); // this will be the sender
+    final GMSEncrypt gmsEncrypt2 = new GMSEncrypt(services, mockMembers[2]); // this will be the receiver
+
+    // establish the public keys for the sender and receiver
+    netView.setPublicKey(mockMembers[1], gmsEncrypt1.getPublicKeyBytes());
+    netView.setPublicKey(mockMembers[2], gmsEncrypt2.getPublicKeyBytes());
+
+    gmsEncrypt1.installView(netView, mockMembers[1]);
+    gmsEncrypt2.installView(netView, mockMembers[2]);
+    int nthreads = 20;
+    ExecutorService executorService = Executors.newFixedThreadPool(nthreads);
+    final CountDownLatch countDownLatch = new CountDownLatch(nthreads);
+    
+    for(int j = 0; j < nthreads; j++)
+    executorService.execute(new Runnable() {
+      public void run() {
+        // sender encrypts a message, so use receiver's public key
+        try {
+          int count = 0;
+          for (int i = 0; i < runs; i++) {
+           // System.out.println("another run " + i + " threadid " + Thread.currentThread().getId());
+            String ch = "Hello world";
+            byte[] challenge = ch.getBytes();
+            byte[] encryptedChallenge = gmsEncrypt1.encryptData(challenge, mockMembers[2]);
+
+            // receiver decrypts the message using the sender's public key
+            byte[] decryptBytes = gmsEncrypt2.decryptData(encryptedChallenge, mockMembers[1]);
+
+            // now send a response
+            String response = "Hello yourself!";
+            byte[] responseBytes = response.getBytes();
+            byte[] encryptedResponse = gmsEncrypt2.encryptData(responseBytes, mockMembers[1]);
+
+            // receiver decodes the response
+            byte[] decryptedResponse = gmsEncrypt1.decryptData(encryptedResponse, mockMembers[2]);
+
+            Assert.assertFalse(Arrays.equals(challenge, encryptedChallenge));
+
+            Assert.assertTrue(Arrays.equals(challenge, decryptBytes));
+
+            Assert.assertFalse(Arrays.equals(responseBytes, encryptedResponse));
+
+            Assert.assertTrue(Arrays.equals(responseBytes, decryptedResponse));
+            count++;
+          }
+          Assert.assertEquals(runs, count);
+          countDownLatch.countDown();
+        } catch (Exception e) {
+          e.printStackTrace();
+          
+        }
+       
+      }
+    });
+
+    countDownLatch.await();
+    executorService.shutdown();
+    
+    
+  }
+  
+  @Test
   public void testPublicKeyPrivateKeyFromSameMember() throws Exception{
     initMocks();
 
@@ -178,12 +247,13 @@ public class GMSEncryptJUnitTest {
   }
   
   @Test
-  public void testForClusterSecretKeyFromOtherMember() throws Exception{
+  public void testForClusterSecretKeyFromOtherMemberMultipleThreads() throws Exception{
     initMocks();
 
-    GMSEncrypt gmsEncrypt1 = new GMSEncrypt(services, mockMembers[1]); // this will be the sender
+    final GMSEncrypt gmsEncrypt1 = new GMSEncrypt(services, mockMembers[1]); // this will be the sender
+    Thread.currentThread().sleep(100);
     gmsEncrypt1.initClusterSecretKey();
-    GMSEncrypt gmsEncrypt2 = new GMSEncrypt(services, mockMembers[2]); // this will be the sender
+    final GMSEncrypt gmsEncrypt2 = new GMSEncrypt(services, mockMembers[2]); // this will be the sender
     
     // establish the public keys for the sender and receiver
     netView.setPublicKey(mockMembers[1], gmsEncrypt1.getPublicKeyBytes());
@@ -196,29 +266,57 @@ public class GMSEncryptJUnitTest {
     
     gmsEncrypt2.installView(netView, mockMembers[1]);
     
-    // sender encrypts a message, so use receiver's public key
-    String ch = "Hello world";
-    byte[] challenge =  ch.getBytes();
-    byte[]  encryptedChallenge =  gmsEncrypt1.encryptData(challenge);
-
-    // receiver decrypts the message using the sender's public key
-    byte[] decryptBytes = gmsEncrypt2.decryptData(encryptedChallenge);
+    final int runs  = 100000;
+    int nthreads = 20;
+    ExecutorService executorService = Executors.newFixedThreadPool(nthreads);
+    final CountDownLatch countDownLatch = new CountDownLatch(nthreads);
     
-    // now send a response
-    String response = "Hello yourself!";
-    byte[] responseBytes = response.getBytes();
-    byte[] encryptedResponse = gmsEncrypt2.encryptData(responseBytes);
-
-    // receiver decodes the response
-    byte[] decryptedResponse = gmsEncrypt1.decryptData(encryptedResponse);
-
-    Assert.assertFalse(Arrays.equals(challenge, encryptedChallenge));
-
-    Assert.assertTrue(Arrays.equals(challenge, decryptBytes));
-
-    Assert.assertFalse(Arrays.equals(responseBytes, encryptedResponse));
-
-    Assert.assertTrue(Arrays.equals(responseBytes, decryptedResponse));  
+    for (int j = 0; j < nthreads; j++)
+      executorService.execute(new Runnable() {
+        public void run() {
+          // sender encrypts a message, so use receiver's public key
+          try {
+            int count = 0;
+            for (int i = 0; i < runs; i++) {
+              //System.out.println("run " + i + " threadid " + Thread.currentThread().getId());
+              String ch = "Hello world";
+              byte[] challenge = ch.getBytes();
+              byte[] encryptedChallenge = gmsEncrypt1.encryptData(challenge);
+
+              // receiver decrypts the message using the sender's public key
+              byte[] decryptBytes = gmsEncrypt2.decryptData(encryptedChallenge);
+
+              // now send a response
+              String response = "Hello yourself!";
+              byte[] responseBytes = response.getBytes();
+              byte[] encryptedResponse = gmsEncrypt2.encryptData(responseBytes);
+
+              // receiver decodes the response
+              byte[] decryptedResponse = gmsEncrypt1.decryptData(encryptedResponse);
+
+              Assert.assertFalse(Arrays.equals(challenge, encryptedChallenge));
+
+              Assert.assertTrue(Arrays.equals(challenge, decryptBytes));
+
+              Assert.assertFalse(Arrays.equals(responseBytes, encryptedResponse));
+
+              Assert.assertTrue(Arrays.equals(responseBytes, decryptedResponse));
+              
+              count++;
+            }
+            
+            Assert.assertEquals(runs, count);
+            
+            countDownLatch.countDown();
+          } catch (Exception e) {
+            e.printStackTrace();
+          }
+          
+        }
+      });
+    
+    countDownLatch.await();
+    executorService.shutdown();
   }