You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bs...@apache.org on 2018/03/28 23:43:01 UTC

[geode] branch feature/GEODE-4928c created (now 9a350df)

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

bschuchardt pushed a change to branch feature/GEODE-4928c
in repository https://gitbox.apache.org/repos/asf/geode.git.


      at 9a350df  GEODE-4928 DistributedLockService doesn't work as expected while the dlock grantor is initialized

This branch includes the following new commits:

     new 9a350df  GEODE-4928 DistributedLockService doesn't work as expected while the dlock grantor is initialized

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


-- 
To stop receiving notification emails like this one, please contact
bschuchardt@apache.org.

[geode] 01/01: GEODE-4928 DistributedLockService doesn't work as expected while the dlock grantor is initialized

Posted by bs...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bschuchardt pushed a commit to branch feature/GEODE-4928c
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 9a350df0a0cbae2d222df9fa99faf5fd24a66fca
Author: Bruce Schuchardt <bs...@pivotal.io>
AuthorDate: Wed Mar 28 16:42:35 2018 -0700

    GEODE-4928 DistributedLockService doesn't work as expected while the dlock grantor is initialized
    
    Wait for background transaction cleanup when there is a failure.
---
 .../distributed/internal/locks/DLockGrantor.java   |  13 ++
 .../locks/DLockLessorDepartureHandler.java         |   2 +
 .../distributed/internal/locks/DLockService.java   |   6 +-
 .../geode/internal/cache/TXFarSideCMTracker.java   |  11 +-
 .../cache/locks/TXLessorDepartureHandler.java      |  59 ++++--
 .../internal/cache/locks/TXLockServiceImpl.java    |   9 +-
 .../internal/DlockAndTxlockRegressionTest.java     | 233 +++++++++++++++++++++
 .../apache/geode/test/dunit/DUnitBlackboard.java   |   2 +-
 .../test/dunit/internal/InternalBlackboard.java    |   2 +-
 .../geode/test/dunit/standalone/DUnitLauncher.java |   2 +-
 10 files changed, 317 insertions(+), 22 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockGrantor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockGrantor.java
index d49cd62..34f2e6b 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockGrantor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockGrantor.java
@@ -461,6 +461,12 @@ public class DLockGrantor {
    * @throws LockGrantorDestroyedException if grantor is destroyed
    */
   void handleLockBatch(DLockRequestMessage request) throws InterruptedException {
+    DLockLessorDepartureHandler handler = this.dlock.getDLockLessorDepartureHandler();
+    // make sure the tx locks of departed members have been cleared so we don't have
+    // conflicts with non-existent members. This is done in a waiting-pool thread launched
+    // when the member-departure is announced.
+    handler.waitForInProcessDepartures();
+
     synchronized (this.batchLocks) { // assures serial processing
       waitWhileInitializing(); // calcWaitMillisFromNow
       if (request.checkForTimeout()) {
@@ -771,6 +777,13 @@ public class DLockGrantor {
       }
     }
     try {
+      // make sure we don't grant a dlock held by a departed member until that member's
+      // transactions are resolved
+      DLockLessorDepartureHandler dLockLessorDepartureHandler =
+          this.dlock.getDLockLessorDepartureHandler();
+      if (dLockLessorDepartureHandler != null) {
+        dLockLessorDepartureHandler.waitForInProcessDepartures();
+      }
       checkDestroyed();
       if (acquireLockPermission(request)) {
         handlePermittedLockRequest(request);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockLessorDepartureHandler.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockLessorDepartureHandler.java
index 64280b2..2178a38 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockLessorDepartureHandler.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockLessorDepartureHandler.java
@@ -28,4 +28,6 @@ public interface DLockLessorDepartureHandler {
 
   void handleDepartureOf(InternalDistributedMember owner, DLockGrantor grantor);
 
+  public void waitForInProcessDepartures() throws InterruptedException;
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java
index 3d2966b..71d6501 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java
@@ -1493,7 +1493,11 @@ public class DLockService extends DistributedLockService {
             Assert.assertTrue(isDestroyed(),
                 "Grantor reports service " + this + " is destroyed: " + name);
           } else if (processor.repliedNotGrantor() || processor.hadNoResponse()) {
-            notLockGrantorId(theLockGrantorId, 0, TimeUnit.MILLISECONDS);
+            long waitForGrantorTime = waitLimit - token.getCurrentTime();
+            if (waitForGrantorTime <= 0) {
+              waitForGrantorTime = 100;
+            }
+            notLockGrantorId(theLockGrantorId, waitForGrantorTime, TimeUnit.MILLISECONDS);
             // keepTrying is still true... loop back around
           } else if (processor.repliedNotHolder()) {
             // fix part of bug 32765 - reentrant/expiration problem
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXFarSideCMTracker.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXFarSideCMTracker.java
index 1876973..8f2093f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXFarSideCMTracker.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXFarSideCMTracker.java
@@ -140,14 +140,21 @@ public class TXFarSideCMTracker {
   public void waitForAllToProcess() throws InterruptedException {
     if (Thread.interrupted())
       throw new InterruptedException(); // wisest to do this before the synchronize below
-    // Assume that a thread interrupt is only sent in the
+    // Assume that a thread interrupt is only set in the
     // case of a shutdown, in that case we don't need to wait
-    // around any longer, propigating the interrupt is reasonable behavior
+    // around any longer, propagating the interrupt is reasonable behavior
+    boolean messageWritten = false;
     synchronized (this.txInProgress) {
       while (!this.txInProgress.isEmpty()) {
+        logger.info("Lock grantor recovery is waiting for transactions to complete: {}",
+            txInProgress);
+        messageWritten = true;
         this.txInProgress.wait();
       }
     }
+    if (messageWritten) {
+      logger.info("Wait for transactions completed");
+    }
   }
 
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/locks/TXLessorDepartureHandler.java b/geode-core/src/main/java/org/apache/geode/internal/cache/locks/TXLessorDepartureHandler.java
index c34df64..99666a5 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/locks/TXLessorDepartureHandler.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/locks/TXLessorDepartureHandler.java
@@ -36,6 +36,18 @@ import org.apache.geode.internal.logging.LogService;
 public class TXLessorDepartureHandler implements DLockLessorDepartureHandler {
   private static final Logger logger = LogService.getLogger();
 
+  private final Object stateLock = new Object();
+  private boolean processingDepartures;
+
+  @Override
+  public void waitForInProcessDepartures() throws InterruptedException {
+    synchronized (stateLock) {
+      while (processingDepartures) {
+        stateLock.wait();
+      }
+    }
+  }
+
   public void handleDepartureOf(InternalDistributedMember owner, DLockGrantor grantor) {
     // get DTLS
     TXLockService dtls = TXLockService.getDTLS();
@@ -62,7 +74,6 @@ public class TXLessorDepartureHandler implements DLockLessorDepartureHandler {
         logger.debug("{} has no active lock batches; exiting TXLessorDepartureHandler", owner);
         return;
       }
-
       sendRecoveryMsgs(dlock.getDistributionManager(), batches, owner, grantor);
     } catch (IllegalStateException e) {
       // ignore... service was destroyed
@@ -71,24 +82,42 @@ public class TXLessorDepartureHandler implements DLockLessorDepartureHandler {
 
   private void sendRecoveryMsgs(final DistributionManager dm, final DLockBatch[] batches,
       final InternalDistributedMember owner, final DLockGrantor grantor) {
-    try {
-      dm.getWaitingThreadPool().execute(new Runnable() {
-        public void run() {
-          for (int i = 0; i < batches.length; i++) {
-            TXLockBatch batch = (TXLockBatch) batches[i];
-            // send TXOriginatorDepartureMessage
-            Set participants = batch.getParticipants();
-            TXOriginatorRecoveryProcessor.sendMessage(participants, owner, batch.getTXLockId(),
-                grantor, dm);
-          }
+
+    synchronized (stateLock) {
+      processingDepartures = true;
+    }
+    Runnable recoverTx = () -> {
+      try {
+        for (int i = 0; i < batches.length; i++) {
+          TXLockBatch batch = (TXLockBatch) batches[i];
+          // send TXOriginatorDepartureMessage
+          Set participants = batch.getParticipants();
+          TXOriginatorRecoveryProcessor.sendMessage(participants, owner, batch.getTXLockId(),
+              grantor, dm);
         }
-      });
+      } finally {
+        clearProcessingDepartures();
+      }
+    };
+
+    try {
+      dm.getWaitingThreadPool().execute(recoverTx);
     } catch (RejectedExecutionException e) {
-      if (logger.isDebugEnabled()) {
-        logger.debug("Rejected sending recovery messages for departure of tx originator {}", owner,
-            e);
+      // this shouldn't happen unless we're shutting down or someone has set a size constraint
+      // on the waiting-pool using a system property
+      if (!dm.getCancelCriterion().isCancelInProgress()) {
+        logger.warn("Unable to schedule background cleanup of transactions for departed member {}."
+            + "  Performing in-line cleanup of the transactions.");
+        recoverTx.run();
       }
     }
   }
 
+  private void clearProcessingDepartures() {
+    synchronized (stateLock) {
+      processingDepartures = false;
+      stateLock.notifyAll();
+    }
+  }
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/locks/TXLockServiceImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/locks/TXLockServiceImpl.java
index 135c93a..c78c0f8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/locks/TXLockServiceImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/locks/TXLockServiceImpl.java
@@ -22,6 +22,7 @@ import java.util.Set;
 
 import org.apache.logging.log4j.Logger;
 
+import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CommitConflictException;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ReplyException;
@@ -166,9 +167,15 @@ public class TXLockServiceImpl extends TXLockService {
   public void updateParticipants(TXLockId txLockId, final Set updatedParticipants) {
     synchronized (this.txLockIdList) {
       if (!this.txLockIdList.contains(txLockId)) {
-        throw new IllegalArgumentException(
+        IllegalArgumentException e = new IllegalArgumentException(
             LocalizedStrings.TXLockServiceImpl_INVALID_TXLOCKID_NOT_FOUND_0
                 .toLocalizedString(txLockId));
+        system.getDistributionManager().getCancelCriterion().checkCancelInProgress(e);
+        Cache cache = system.getCache();
+        if (cache != null) {
+          cache.getCancelCriterion().checkCancelInProgress(e);
+        }
+        throw e;
       }
     }
     if (updatedParticipants == null) {
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/DlockAndTxlockRegressionTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/DlockAndTxlockRegressionTest.java
new file mode 100644
index 0000000..700986f
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/DlockAndTxlockRegressionTest.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.distributed.internal;
+
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.logging.log4j.Logger;
+import org.awaitility.Awaitility;
+import org.awaitility.core.ConditionTimeoutException;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.CancelException;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CommitConflictException;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.distributed.ConfigurationProperties;
+import org.apache.geode.distributed.DistributedLockService;
+import org.apache.geode.distributed.LockServiceDestroyedException;
+import org.apache.geode.internal.OSProcess;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.dunit.DistributedTestUtils;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.dunit.SerializableRunnable;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
+import org.apache.geode.test.dunit.rules.DistributedRestoreSystemProperties;
+import org.apache.geode.test.dunit.standalone.DUnitLauncher;
+import org.apache.geode.test.junit.categories.DLockTest;
+import org.apache.geode.test.junit.categories.DistributedTest;
+
+@Category({DLockTest.class, DistributedTest.class})
+public class DlockAndTxlockRegressionTest extends JUnit4CacheTestCase {
+  private static final Logger logger = LogService.getLogger();
+  public static final String TRANSACTION_COUNT = "transactionCount";
+
+  @Rule
+  public DistributedRestoreSystemProperties restoreSystemProperties =
+      new DistributedRestoreSystemProperties();
+
+  @Override
+  public Properties getDistributedSystemProperties() {
+    Properties properties = super.getDistributedSystemProperties();
+    properties.setProperty(ConfigurationProperties.DISABLE_AUTO_RECONNECT, "true");
+    properties.setProperty(ConfigurationProperties.MEMBER_TIMEOUT, "1000");
+    properties.setProperty(ConfigurationProperties.NAME,
+        "vm" + Integer.getInteger(DUnitLauncher.VM_NUM_PARAM));
+    System.getProperties().remove("gemfire.member-timeout");
+    System.getProperties().remove("gemfire.log-level");
+    return properties;
+  }
+
+  /**
+   * Distributed locks are released quickly when a server crashes but transaction locks are
+   * released in a background "pooled waiting" thread because the release involves communicating
+   * with participants of the transaction. This makes the pattern of<br>
+   * 1. get dlock,<br>
+   * 2. perform transaction<br>
+   * sometimes fail if the background cleanup takes too long. You may get the dlock but then get a
+   * CommitConflictException when committing the transaction due to lingering tx locks from the
+   * crashed server. The fix makes tx lock acquisition wait for the cleanup to finish.
+   */
+  @Test
+  public void testDLockProtectsAgainstTransactionConflict() throws Exception {
+    IgnoredException.addIgnoredException(
+        "DistributedSystemDisconnectedException|ForcedDisconnectException|Possible loss of quorum");
+    // create four nodes to perform dlock & transactions and then
+    // kill & restart each one using a forced disconnect.
+    Host host = Host.getHost(0);
+    VM[] servers = new VM[] {host.getVM(0), host.getVM(1), host.getVM(2)};
+    for (VM vm : servers) {
+      vm.invoke(() -> createCacheAndRegion());
+    }
+
+    servers[0].invoke(new SerializableRunnable() {
+      public void run() {
+        becomeLockGrantor();
+      }
+    });
+
+    AsyncInvocation[] asyncInvocations = new AsyncInvocation[servers.length];
+    for (int i = 0; i < servers.length; i++) {
+      asyncInvocations[i] = servers[i].invokeAsync(() -> performOps());
+    }
+
+    // this test uses the DUnit blackboard to coordinate actions between JVMs
+    getBlackboard().initBlackboard();
+    getBlackboard().setMailbox(TRANSACTION_COUNT, 0);
+
+    try {
+      long endTime = System.currentTimeMillis() + 180_000;
+      while (System.currentTimeMillis() < endTime) {
+        for (int i = 0; i < servers.length; i++) {
+          checkAsyncInvocations(asyncInvocations);
+
+          // clobber the current lock grantor
+          VM vm = servers[i];
+          System.out.println("TEST: killing vm " + i);
+          vm.invoke("force disconnect", () -> forceDisconnect());
+          asyncInvocations[i].join();
+          System.out.println("TEST: recreating vm " + i);
+          vm.invoke("create cache", () -> createCacheAndRegion());
+          asyncInvocations[i] = vm.invokeAsync(() -> performOps());
+
+          // move the grantor into the next VM to be clobbered
+          int nextServer = (i + 1) % (servers.length - 1);
+          logger.info("TEST: moving the lock grantor to vm " + nextServer);
+          servers[nextServer].invoke("become lock grantor", () -> becomeLockGrantor());
+
+          int txCount = getBlackboard().getMailbox(TRANSACTION_COUNT);
+          int newTxCount = txCount + 10;
+          try {
+            Awaitility.await("check for new transactions").atMost(30, TimeUnit.SECONDS)
+                .until(() -> {
+                  checkAsyncInvocations(asyncInvocations);
+                  int newCount = getBlackboard().getMailbox(TRANSACTION_COUNT);
+                  return newCount >= newTxCount;
+                });
+          } catch (ConditionTimeoutException e) {
+            for (VM server : servers) {
+              server.invoke(() -> OSProcess.printStacks(0));
+            }
+            throw e;
+          }
+        }
+      }
+
+    } finally {
+
+      for (VM vm : servers) {
+        vm.invoke(() -> forceDisconnect());
+      }
+
+      Throwable failure = null;
+      int asyncIndex = 0;
+      for (AsyncInvocation asyncInvocation : asyncInvocations) {
+        asyncInvocation.join(30000);
+        if (asyncInvocation.exceptionOccurred()) {
+          failure = asyncInvocation.getException();
+        }
+        asyncIndex++;
+      }
+      if (failure != null) {
+        throw new RuntimeException("test failed", failure);
+      }
+    }
+  }
+
+  private void checkAsyncInvocations(AsyncInvocation[] asyncInvocations) {
+    for (AsyncInvocation asyncInvocation : asyncInvocations) {
+      if (!asyncInvocation.isAlive() && asyncInvocation.exceptionOccurred()) {
+        throw new RuntimeException("", asyncInvocation.getException());
+      }
+    }
+  }
+
+  public void forceDisconnect() {
+    DistributedTestUtils.crashDistributedSystem(getCache().getDistributedSystem());
+  }
+
+  public void createCacheAndRegion() {
+    Cache cache = getCache();
+    cache.createRegionFactory(RegionShortcut.REPLICATE).setConcurrencyChecksEnabled(false)
+        .create("TestRegion");
+    DistributedLockService dlockService =
+        DistributedLockService.create("Bulldog", cache.getDistributedSystem());
+  }
+
+  public void becomeLockGrantor() {
+    DistributedLockService dlockService = DistributedLockService.getServiceNamed("Bulldog");
+    dlockService.becomeLockGrantor();
+  }
+
+  public void performOps() {
+    Cache cache = getCache();
+    Region region = cache.getRegion("TestRegion");
+    DistributedLockService dlockService = DistributedLockService.getServiceNamed("Bulldog");
+    Random random = new Random();
+
+    while (!cache.isClosed()) {
+      boolean locked = false;
+      try {
+        locked = dlockService.lock("testDLock", 500, 60_000);
+        if (!locked) {
+          // this could happen if we're starved out for 30sec by other VMs
+          continue;
+        }
+
+        cache.getCacheTransactionManager().begin();
+
+        region.put("TestKey", "TestValue" + random.nextInt(100000));
+
+        try {
+          cache.getCacheTransactionManager().commit();
+        } catch (CommitConflictException e) {
+          throw new RuntimeException("dlock failed to prevent a transaction conflict", e);
+        }
+
+        int txCount = getBlackboard().getMailbox(TRANSACTION_COUNT);
+        getBlackboard().setMailbox(TRANSACTION_COUNT, txCount + 1);
+
+      } catch (CancelException | IllegalStateException e) {
+        // okay to ignore
+      } finally {
+        if (locked) {
+          try {
+            dlockService.unlock("testDLock");
+          } catch (CancelException | IllegalStateException e) {
+            // shutting down
+          }
+        }
+      }
+    }
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/DUnitBlackboard.java b/geode-core/src/test/java/org/apache/geode/test/dunit/DUnitBlackboard.java
index ec02d4b..4bdddfe 100755
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/DUnitBlackboard.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/DUnitBlackboard.java
@@ -118,7 +118,7 @@ public class DUnitBlackboard {
   /**
    * retrieve an object from a mailbox slot
    */
-  public Object getMailbox(String boxName) {
+  public <T> T getMailbox(String boxName) {
     try {
       return blackboard.getMailbox(boxName);
     } catch (RemoteException e) {
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboard.java b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboard.java
index bc5b9b7..24abf4f 100755
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboard.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboard.java
@@ -67,7 +67,7 @@ public interface InternalBlackboard extends Remote, Serializable {
   /**
    * retrieve an object from a mailbox slot
    */
-  Object getMailbox(String boxName) throws RemoteException;
+  <T> T getMailbox(String boxName) throws RemoteException;
 
   /**
    * ping the blackboard to make sure it's there
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/standalone/DUnitLauncher.java b/geode-core/src/test/java/org/apache/geode/test/dunit/standalone/DUnitLauncher.java
index ca6a623..fa723d5 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/standalone/DUnitLauncher.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/standalone/DUnitLauncher.java
@@ -124,7 +124,7 @@ public class DUnitLauncher {
   static final String MASTER_PARAM = "DUNIT_MASTER";
 
   public static final String RMI_PORT_PARAM = GEMFIRE_PREFIX + "DUnitLauncher.RMI_PORT";
-  static final String VM_NUM_PARAM = GEMFIRE_PREFIX + "DUnitLauncher.VM_NUM";
+  public static final String VM_NUM_PARAM = GEMFIRE_PREFIX + "DUnitLauncher.VM_NUM";
   static final String VM_VERSION_PARAM = GEMFIRE_PREFIX + "DUnitLauncher.VM_VERSION";
 
   private static final String LAUNCHED_PROPERTY = GEMFIRE_PREFIX + "DUnitLauncher.LAUNCHED";

-- 
To stop receiving notification emails like this one, please contact
bschuchardt@apache.org.