You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2017/04/28 23:25:36 UTC

geode git commit: GEODE-2840: add a DUnit test to test concurrent deploy

Repository: geode
Updated Branches:
  refs/heads/develop 953f1eebc -> 8239fbd42


GEODE-2840: add a DUnit test to test concurrent deploy


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

Branch: refs/heads/develop
Commit: 8239fbd42cdefd0e78f04bf2795e15ee9e7d8988
Parents: 953f1ee
Author: Jared Stewart <js...@pivotal.io>
Authored: Thu Apr 27 13:09:08 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri Apr 28 16:11:47 2017 -0700

----------------------------------------------------------------------
 .../org/apache/geode/internal/DeployedJar.java  |  28 +++--
 .../org/apache/geode/internal/JarDeployer.java  |   1 -
 .../cli/commands/ConcurrentDeployDUnitTest.java | 101 +++++++++++++++++++
 .../geode/test/dunit/rules/JarFileRule.java     |  80 +++++++++++++++
 .../dunit/rules/LocatorServerStartupRule.java   |   7 +-
 5 files changed, 199 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/8239fbd4/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java b/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
index f96863f..acb7d22 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
@@ -16,16 +16,23 @@ package org.apache.geode.internal;
 
 import io.github.lukehutch.fastclasspathscanner.FastClasspathScanner;
 import io.github.lukehutch.fastclasspathscanner.scanner.ScanResult;
+import org.apache.geode.cache.CacheClosedException;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.Declarable;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.pdx.internal.TypeRegistry;
+import org.apache.logging.log4j.Logger;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Modifier;
 import java.net.URL;
 import java.net.URLClassLoader;
@@ -40,17 +47,6 @@ import java.util.Properties;
 import java.util.jar.JarEntry;
 import java.util.jar.JarInputStream;
 
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.cache.CacheClosedException;
-import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.Declarable;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.execute.FunctionService;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.pdx.internal.TypeRegistry;
-
 /**
  * ClassLoader for a single JAR file.
  * 
@@ -100,11 +96,11 @@ public class DeployedJar {
 
     final byte[] fileContent = getJarContent();
     if (!Arrays.equals(fileContent, jarBytes)) {
-      throw new FileNotFoundException("JAR file: " + versionedJarFile.getAbsolutePath()
-          + ", was modified prior to obtaining a lock: " + jarName);
+      throw new IllegalStateException("JAR file: " + versionedJarFile.getAbsolutePath()
+          + ", does not have the expected content.");
     }
 
-    if (!hasValidJarContent(getJarContent())) {
+    if (!hasValidJarContent(fileContent)) {
       throw new IllegalArgumentException(
           "File does not contain valid JAR content: " + versionedJarFile.getAbsolutePath());
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/8239fbd4/geode-core/src/main/java/org/apache/geode/internal/JarDeployer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/JarDeployer.java b/geode-core/src/main/java/org/apache/geode/internal/JarDeployer.java
index a65cd0f..df3f10b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/JarDeployer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/JarDeployer.java
@@ -102,7 +102,6 @@ public class JarDeployer implements Serializable {
     }
   }
 
-
   /**
    * Get a list of all currently deployed jars.
    * 

http://git-wip-us.apache.org/repos/asf/geode/blob/8239fbd4/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConcurrentDeployDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConcurrentDeployDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConcurrentDeployDUnitTest.java
new file mode 100644
index 0000000..559440c
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConcurrentDeployDUnitTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.management.internal.cli.commands;
+
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.JarFileRule;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.LocatorStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+
+@Category(DistributedTest.class)
+public class ConcurrentDeployDUnitTest {
+
+  @Rule
+  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+
+  @Rule
+  public LocatorStarterRule locator = new LocatorStarterRule().withAutoStart();
+
+  @Rule
+  public JarFileRule jar1Rule = new JarFileRule("classOne", "jar1.jar", true);
+
+  // This is a reference used to refer to connections in VM 2 and VM 3
+  private static GfshShellConnectionRule gfsh;
+
+  private VM gfsh1, gfsh2, gfsh3;
+
+  @Test
+  public void testMultipleGfshClientToOneServer() throws Exception {
+    lsRule.startServerVM(0, locator.getPort());
+    gfsh1 = lsRule.getVM(1);
+    gfsh2 = lsRule.getVM(2);
+    gfsh3 = lsRule.getVM(3);
+
+    int locatorPort = locator.getPort();
+
+    gfsh1.invoke(() -> connectToLocator(locatorPort));
+    gfsh2.invoke(() -> connectToLocator(locatorPort));
+    gfsh3.invoke(() -> connectToLocator(locatorPort));
+
+    File jar1 = jar1Rule.getJarFile();
+    AsyncInvocation gfsh1Invocation = gfsh1.invokeAsync(() -> loopThroughDeployAndUndeploys(jar1));
+    AsyncInvocation gfsh2Invocation = gfsh2.invokeAsync(() -> loopThroughDeployAndUndeploys(jar1));
+    AsyncInvocation gfsh3Invocation = gfsh3.invokeAsync(() -> loopThroughDeployAndUndeploys(jar1));
+
+    gfsh1Invocation.await();
+    gfsh2Invocation.await();
+    gfsh3Invocation.await();
+  }
+
+  @After
+  public void after() {
+    gfsh1.invoke(() -> gfsh.close());
+    gfsh2.invoke(() -> gfsh.close());
+    gfsh3.invoke(() -> gfsh.close());
+  }
+
+  public static void connectToLocator(int locatorPort) throws Exception {
+    gfsh = new GfshShellConnectionRule();
+    gfsh.connectAndVerify(locatorPort, GfshShellConnectionRule.PortType.locator);
+  }
+
+  public static void loopThroughDeployAndUndeploys(File jar1) throws Exception {
+    int numTimesToExecute = 500;
+    String command;
+
+    for (int i = 1; i <= numTimesToExecute; i++) {
+      command = "deploy --jar=" + jar1.getAbsolutePath();
+      gfsh.executeAndVerifyCommand(command);
+
+      command = "list deployed";
+      gfsh.executeAndVerifyCommand(command);
+
+      command = "undeploy --jar=" + jar1.getName();
+      gfsh.executeAndVerifyCommand(command);
+
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/8239fbd4/geode-core/src/test/java/org/apache/geode/test/dunit/rules/JarFileRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/JarFileRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/JarFileRule.java
new file mode 100644
index 0000000..cbc5b52
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/JarFileRule.java
@@ -0,0 +1,80 @@
+/*
+ * 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.test.dunit.rules;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.geode.internal.ClassBuilder;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+
+public class JarFileRule extends ExternalResource implements Serializable {
+
+  private transient TemporaryFolder temporaryFolder = new TemporaryFolder();
+  private transient ClassBuilder classBuilder = new ClassBuilder();
+
+  private String className;
+  private String jarName;
+  private File jarFile;
+  boolean makeJarLarge;
+
+  public JarFileRule(String className, String jarName) {
+    this(className, jarName, false);
+  }
+
+  public JarFileRule(String className, String jarName, boolean makeJarLarge) {
+    this.className = className;
+    this.jarName = jarName;
+    this.makeJarLarge = makeJarLarge;
+  }
+
+  protected void before() throws IOException {
+    temporaryFolder.create();
+    this.jarFile = temporaryFolder.newFile(jarName);
+
+    if (makeJarLarge) {
+      classBuilder.writeJarFromContent(className,
+          "public class " + className + "{" + "String test = \""
+              + RandomStringUtils.randomAlphanumeric(10000) + "\";" + "String test2 = \""
+              + RandomStringUtils.randomAlphanumeric(10000) + "\";" + "String test3 = \""
+              + RandomStringUtils.randomAlphanumeric(10000) + "\";" + "String test4 = \""
+              + RandomStringUtils.randomAlphanumeric(10000) + "\";" + "}",
+          jarFile);
+    } else {
+      classBuilder.writeJarFromName(className, jarFile);
+    }
+
+  }
+
+  protected void after() {
+    temporaryFolder.delete();
+  }
+
+  public File getJarFile() {
+    assertThat(this.jarFile).exists();
+    return this.jarFile;
+  }
+
+  public String getJarName() {
+    return this.jarName;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/8239fbd4/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
index 4219d02..b86e058 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
@@ -162,13 +162,18 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
     return members[index];
   }
 
+
   /**
    * Returns the {@link Member} running inside the VM with the specified {@code index}
    */
-  public Member getMember(int index) {
+  public MemberVM getMember(int index) {
     return members[index];
   }
 
+  public VM getVM(int index) {
+    return getHost(0).getVM(index);
+  }
+
   public TemporaryFolder getTempFolder() {
     return temporaryFolder;
   }