GEODE-715: Move dunit.standalone under com.gemstone.gemfire.test * Move dunit.BounceResult and dunit.RemoteDUnitVMIF to dunit.standalone * Move dunit.standalone to com.gemstone.gemfire.test.dunit.standalone * Remove hydra dependencies from com.gemstone.gemfire.test.dunit.standalone except for MethExecutor
Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/c57a88c6 Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/c57a88c6 Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/c57a88c6 Branch: refs/heads/feature/GEODE-217 Commit: c57a88c6f2437a8d74f51abbe2edc3355eb1806b Parents: df6d7ff Author: Kirk Lund <kl...@pivotal.io> Authored: Mon Dec 28 15:10:27 2015 -0800 Committer: Kirk Lund <kl...@pivotal.io> Committed: Mon Jan 4 13:59:51 2016 -0800 ---------------------------------------------------------------------- .../cache/PartitionedRegionDUnitTestCase.java | 2 +- .../CompressionRegionConfigDUnitTest.java | 2 +- .../test/dunit/standalone/BounceResult.java | 36 ++ .../gemfire/test/dunit/standalone/ChildVM.java | 81 ++++ .../test/dunit/standalone/DUnitLauncher.java | 461 ++++++++++++++++++ .../test/dunit/standalone/ProcessManager.java | 259 +++++++++++ .../test/dunit/standalone/RemoteDUnitVM.java | 142 ++++++ .../test/dunit/standalone/RemoteDUnitVMIF.java | 36 ++ .../dunit/standalone/StandAloneDUnitEnv.java | 75 +++ .../test/dunit/tests/BasicDUnitTest.java | 132 ++++++ .../gemfire/test/dunit/tests/TestFailure.java | 50 ++ .../gemfire/test/dunit/tests/VMDUnitTest.java | 237 ++++++++++ .../src/test/java/dunit/BounceResult.java | 36 -- gemfire-core/src/test/java/dunit/DUnitEnv.java | 2 + gemfire-core/src/test/java/dunit/Host.java | 2 + .../src/test/java/dunit/RemoteDUnitVMIF.java | 36 -- gemfire-core/src/test/java/dunit/VM.java | 7 +- .../src/test/java/dunit/standalone/ChildVM.java | 82 ---- .../java/dunit/standalone/DUnitLauncher.java | 463 ------------------- .../java/dunit/standalone/ProcessManager.java | 261 ----------- .../java/dunit/standalone/RemoteDUnitVM.java | 144 ------ .../dunit/standalone/StandAloneDUnitEnv.java | 75 --- .../test/java/dunit/tests/BasicDUnitTest.java | 132 ------ .../src/test/java/dunit/tests/TestFailure.java | 50 -- .../src/test/java/dunit/tests/VMDUnitTest.java | 237 ---------- 25 files changed, 1520 insertions(+), 1520 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java index a4d32f2..74f201d 100644 --- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java @@ -30,10 +30,10 @@ import com.gemstone.gemfire.cache30.CacheTestCase; import com.gemstone.gemfire.internal.logging.InternalLogWriter; import com.gemstone.gemfire.internal.logging.LogWriterImpl; import com.gemstone.gemfire.internal.logging.PureLogWriter; +import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher; import dunit.Host; import dunit.SerializableRunnable; -import dunit.standalone.DUnitLauncher; /** * This class is extended by some PartitionedRegion related DUnit test cases http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java index af7d07f..1fb22c6 100644 --- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionConfigDUnitTest.java @@ -27,13 +27,13 @@ import com.gemstone.gemfire.cache.server.CacheServer; import com.gemstone.gemfire.cache30.CacheTestCase; import com.gemstone.gemfire.compression.Compressor; import com.gemstone.gemfire.compression.SnappyCompressor; +import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher; import dunit.DistributedTestCase; import dunit.Host; import dunit.SerializableCallable; import dunit.SerializableRunnable; import dunit.VM; -import dunit.standalone.DUnitLauncher; /** * Sanity checks on a number of basic cluster configurations with compression turned on. http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java new file mode 100644 index 0000000..e117004 --- /dev/null +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/BounceResult.java @@ -0,0 +1,36 @@ +/* + * 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 com.gemstone.gemfire.test.dunit.standalone; + +public class BounceResult { + private final int newPid; + private final RemoteDUnitVMIF newClient; + + public BounceResult(int newPid, RemoteDUnitVMIF newClient) { + this.newPid = newPid; + this.newClient = newClient; + } + + public int getNewPid() { + return newPid; + } + + public RemoteDUnitVMIF getNewClient() { + return newClient; + } + +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java new file mode 100644 index 0000000..49b53e8 --- /dev/null +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java @@ -0,0 +1,81 @@ +/* + * 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 com.gemstone.gemfire.test.dunit.standalone; + +import hydra.HydraRuntimeException; +import hydra.Log; + +import java.rmi.Naming; + +import org.apache.logging.log4j.Logger; + +import com.gemstone.gemfire.internal.OSProcess; +import com.gemstone.gemfire.internal.logging.LogService; +import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher.MasterRemote; + +/** + * @author dsmith + * + */ +public class ChildVM { + + private static boolean stopMainLoop = false; + + /** + * tells the main() loop to exit + */ + public static void stopVM() { + stopMainLoop = true; + } + + static { + createHydraLogWriter(); + } + + private final static Logger logger = LogService.getLogger(); + private static RemoteDUnitVM dunitVM; + + public static void main(String[] args) throws Throwable { + try { + int namingPort = Integer.getInteger(DUnitLauncher.RMI_PORT_PARAM).intValue(); + int vmNum = Integer.getInteger(DUnitLauncher.VM_NUM_PARAM).intValue(); + int pid = OSProcess.getId(); + logger.info("VM" + vmNum + " is launching" + (pid > 0? " with PID " + pid : "")); + MasterRemote holder = (MasterRemote) Naming.lookup("//localhost:" + namingPort + "/" + DUnitLauncher.MASTER_PARAM); + DUnitLauncher.init(holder); + DUnitLauncher.locatorPort = holder.getLocatorPort(); + dunitVM = new RemoteDUnitVM(); + Naming.rebind("//localhost:" + namingPort + "/vm" + vmNum, dunitVM); + holder.signalVMReady(); + //This loop is here so this VM will die even if the master is mean killed. + while (!stopMainLoop) { + holder.ping(); + Thread.sleep(1000); + } + } catch (Throwable t) { + t.printStackTrace(); + System.exit(1); + } + } + + private static void createHydraLogWriter() { + try { + Log.createLogWriter("dunit-childvm", "fine"); + } catch (HydraRuntimeException ignore) { + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java new file mode 100644 index 0000000..1358722 --- /dev/null +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java @@ -0,0 +1,461 @@ +/* + * 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 com.gemstone.gemfire.test.dunit.standalone; + +import hydra.Log; +import hydra.MethExecutorResult; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.lang.reflect.Method; +import java.net.InetAddress; +import java.net.URISyntaxException; +import java.nio.channels.FileChannel; +import java.nio.charset.Charset; +import java.rmi.AccessException; +import java.rmi.AlreadyBoundException; +import java.rmi.NotBoundException; +import java.rmi.Remote; +import java.rmi.RemoteException; +import java.rmi.registry.LocateRegistry; +import java.rmi.registry.Registry; +import java.rmi.server.UnicastRemoteObject; +import java.util.List; +import java.util.Properties; + +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.core.LoggerContext; +import org.apache.logging.log4j.core.appender.FileAppender; +import org.apache.logging.log4j.core.config.LoggerConfig; +import org.apache.logging.log4j.core.layout.PatternLayout; +import org.junit.Assert; + +import batterytest.greplogs.ExpectedStrings; +import batterytest.greplogs.LogConsumer; + +import com.gemstone.gemfire.distributed.Locator; +import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave; +import com.gemstone.gemfire.internal.AvailablePortHelper; +import com.gemstone.gemfire.internal.logging.LogService; + +import dunit.DUnitEnv; +import dunit.Host; +import dunit.SerializableCallable; +import dunit.VM; + +/** + * A class to build a fake test configuration and launch some DUnit VMS. + * + * For use within eclipse. This class completely skips hydra and just starts + * some vms directly, creating a fake test configuration + * + * Also, it's a good idea to set your working directory, because the test code + * a lot of files that it leaves around. + * + * @author dsmith + * + */ +public class DUnitLauncher { + + /** change this to use a different log level in unit tests */ + public static final String LOG_LEVEL = System.getProperty("logLevel", "info"); + + static int locatorPort; + + private static final int NUM_VMS = 4; + private static final int DEBUGGING_VM_NUM = -1; + private static final int LOCATOR_VM_NUM = -2; + + static final long STARTUP_TIMEOUT = 30 * 1000; + private static final String SUSPECT_FILENAME = "dunit_suspect.log"; + private static File DUNIT_SUSPECT_FILE; + + public static final String DUNIT_DIR = "dunit"; + public static final String WORKSPACE_DIR_PARAM = "WORKSPACE_DIR"; + public static final boolean LOCATOR_LOG_TO_DISK = Boolean.getBoolean("locatorLogToDisk"); + + static final String MASTER_PARAM = "DUNIT_MASTER"; + static final String RMI_PORT_PARAM = "gemfire.DUnitLauncher.RMI_PORT"; + static final String VM_NUM_PARAM = "gemfire.DUnitLauncher.VM_NUM"; + + private static final String LAUNCHED_PROPERTY = "gemfire.DUnitLauncher.LAUNCHED"; + + private static Master master; + + private DUnitLauncher() { + } + + private static boolean isHydra() { + try { + //TODO - this is hacky way to test for a hydra environment - see + //if there is registered test configuration object. + Class<?> clazz = Class.forName("hydra.TestConfig"); + Method getInstance = clazz.getMethod("getInstance", new Class[0]); + getInstance.invoke(null); + return true; + } catch (Exception e) { + return false; + } + } + /** + * Launch DUnit. If the unit test was launched through + * the hydra framework, leave the test alone. + */ + public static void launchIfNeeded() { + if(System.getProperties().contains(VM_NUM_PARAM)) { + //we're a dunit child vm, do nothing. + return; + } + + if(!isHydra() &&!isLaunched()) { + try { + launch(); + } catch (Exception e) { + throw new RuntimeException("Unable to launch dunit VMS", e); + } + } + } + + /** + * Test it see if the eclise dunit environment is launched. + */ + public static boolean isLaunched() { + return Boolean.getBoolean(LAUNCHED_PROPERTY); + } + + public static String getLocatorString() { + return "localhost[" + locatorPort + "]"; + } + + + private static void launch() throws URISyntaxException, AlreadyBoundException, IOException, InterruptedException, NotBoundException { +// initialize the log writer that hydra uses + Log.createLogWriter( "dunit-master", LOG_LEVEL ); + + DUNIT_SUSPECT_FILE = new File(SUSPECT_FILENAME); + DUNIT_SUSPECT_FILE.delete(); + DUNIT_SUSPECT_FILE.deleteOnExit(); + + locatorPort = AvailablePortHelper.getRandomAvailableTCPPort(); + + //create an RMI registry and add an object to share our tests config + int namingPort = AvailablePortHelper.getRandomAvailableTCPPort(); + Registry registry = LocateRegistry.createRegistry(namingPort); + + final ProcessManager processManager = new ProcessManager(namingPort, registry); + master = new Master(registry, processManager); + registry.bind(MASTER_PARAM, master); + + Runtime.getRuntime().addShutdownHook(new Thread() { + public void run() { +// System.out.println("shutting down DUnit JVMs"); +// for (int i=0; i<NUM_VMS; i++) { +// try { +// processManager.getStub(i).shutDownVM(); +// } catch (Exception e) { +// System.out.println("exception shutting down vm_"+i+": " + e); +// } +// } +// // TODO - hasLiveVMs always returns true +// System.out.print("waiting for JVMs to exit"); +// long giveUp = System.currentTimeMillis() + 5000; +// while (giveUp > System.currentTimeMillis()) { +// if (!processManager.hasLiveVMs()) { +// return; +// } +// System.out.print("."); +// System.out.flush(); +// try { +// Thread.sleep(1000); +// } catch (InterruptedException e) { +// break; +// } +// } +// System.out.println("\nkilling any remaining JVMs"); + processManager.killVMs(); + } + }); + + //Create a VM for the locator + processManager.launchVM(LOCATOR_VM_NUM); + + //Launch an initial set of VMs + for(int i=0; i < NUM_VMS; i++) { + processManager.launchVM(i); + } + + //wait for the VMS to start up + if(!processManager.waitForVMs(STARTUP_TIMEOUT)) { + throw new RuntimeException("VMs did not start up with 30 seconds"); + } + + //populate the Host class with our stubs. The tests use this host class + DUnitHost host = new DUnitHost(InetAddress.getLocalHost().getCanonicalHostName(), processManager); + host.init(registry, NUM_VMS); + + init(master); + + startLocator(registry); + } + + public static Properties getDistributedSystemProperties() { + Properties p = new Properties(); + p.setProperty("locators", getLocatorString()); + p.setProperty("mcast-port", "0"); + p.setProperty("enable-cluster-configuration", "false"); + p.setProperty("use-cluster-configuration", "false"); + p.setProperty("log-level", LOG_LEVEL); + return p; + } + + /** + * Add an appender to Log4j which sends all INFO+ messages to a separate file + * which will be used later to scan for suspect strings. The pattern of the + * messages conforms to the original log format so that hydra will be able + * to parse them. + */ + private static void addSuspectFileAppender(final String workspaceDir) { + final String suspectFilename = new File(workspaceDir, SUSPECT_FILENAME).getAbsolutePath(); + + final LoggerContext appenderContext = ((org.apache.logging.log4j.core.Logger) + LogManager.getLogger(LogService.BASE_LOGGER_NAME)).getContext(); + + final PatternLayout layout = PatternLayout.createLayout( + "[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} <%thread> tid=%tid] %message%n%throwable%n", null, null, + Charset.defaultCharset(), true, false, "", ""); + + final FileAppender fileAppender = FileAppender.createAppender(suspectFilename, "true", "false", + DUnitLauncher.class.getName(), "true", "false", "false", "0", layout, null, null, null, appenderContext.getConfiguration()); + fileAppender.start(); + + LoggerConfig loggerConfig = appenderContext.getConfiguration().getLoggerConfig(LogService.BASE_LOGGER_NAME); + loggerConfig.addAppender(fileAppender, Level.INFO, null); + } + + private static void startLocator(Registry registry) throws IOException, NotBoundException { + RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + LOCATOR_VM_NUM); + final File locatorLogFile = + LOCATOR_LOG_TO_DISK ? new File("locator-" + locatorPort + ".log") : new File(""); + MethExecutorResult result = remote.executeMethodOnObject(new SerializableCallable() { + public Object call() throws IOException { + Properties p = getDistributedSystemProperties(); + // I never want this locator to end up starting a jmx manager + // since it is part of the unit test framework + p.setProperty("jmx-manager", "false"); + //Disable the shared configuration on this locator. + //Shared configuration tests create their own locator + p.setProperty("enable-cluster-configuration", "false"); + //Tell the locator it's the first in the system for + //faster boot-up + + System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true"); + try { + Locator.startLocatorAndDS(locatorPort, locatorLogFile, p); + } finally { + System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY); + } + + return null; + } + }, "call"); + if(result.getException() != null) { + RuntimeException ex = new RuntimeException("Failed to start locator", result.getException()); + ex.printStackTrace(); + throw ex; + } + } + + public static void init(MasterRemote master) { + DUnitEnv.set(new StandAloneDUnitEnv(master)); + //fake out tests that are using a bunch of hydra stuff + String workspaceDir = System.getProperty(DUnitLauncher.WORKSPACE_DIR_PARAM) ; + workspaceDir = workspaceDir == null ? new File(".").getAbsolutePath() : workspaceDir; + + addSuspectFileAppender(workspaceDir); + + //Free off heap memory when disconnecting from the distributed system + System.setProperty("gemfire.free-off-heap-memory", "true"); + + //indicate that this CM is controlled by the eclipse dunit. + System.setProperty(LAUNCHED_PROPERTY, "true"); + } + + public static void closeAndCheckForSuspects() { + if (isLaunched()) { + final boolean skipLogMsgs = ExpectedStrings.skipLogMsgs("dunit"); + final List<?> expectedStrings = ExpectedStrings.create("dunit"); + final LogConsumer logConsumer = new LogConsumer(skipLogMsgs, expectedStrings, "log4j", 5); + + final StringBuilder suspectStringBuilder = new StringBuilder(); + + BufferedReader buffReader = null; + FileChannel fileChannel = null; + try { + fileChannel = new FileOutputStream(DUNIT_SUSPECT_FILE, true).getChannel(); + buffReader = new BufferedReader(new FileReader(DUNIT_SUSPECT_FILE)); + } catch (FileNotFoundException e) { + System.err.println("Could not find the suspect string output file: " + e); + return; + } + try { + String line; + try { + while ((line = buffReader.readLine()) != null) { + final StringBuilder builder = logConsumer.consume(line); + if (builder != null) { + suspectStringBuilder.append(builder); + } + } + } catch (IOException e) { + System.err.println("Could not read the suspect string output file: " + e); + } + + try { + fileChannel.truncate(0); + } catch (IOException e) { + System.err.println("Could not truncate the suspect string output file: " + e); + } + + } finally { + try { + buffReader.close(); + fileChannel.close(); + } catch (IOException e) { + System.err.println("Could not close the suspect string output file: " + e); + } + } + + if (suspectStringBuilder.length() != 0) { + System.err.println("Suspicious strings were written to the log during this run.\n" + + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n" + + suspectStringBuilder); + + Assert.fail("Suspicious strings were written to the log during this run.\n" + + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n" + + suspectStringBuilder); + } + } + } + + public interface MasterRemote extends Remote { + public int getLocatorPort() throws RemoteException; + public void signalVMReady() throws RemoteException; + public void ping() throws RemoteException; + public BounceResult bounce(int pid) throws RemoteException; + } + + public static class Master extends UnicastRemoteObject implements MasterRemote { + private static final long serialVersionUID = 1178600200232603119L; + + private final Registry registry; + private final ProcessManager processManager; + + + public Master(Registry registry, ProcessManager processManager) throws RemoteException { + this.processManager = processManager; + this.registry = registry; + } + + public int getLocatorPort() throws RemoteException{ + return locatorPort; + } + + public synchronized void signalVMReady() { + processManager.signalVMReady(); + } + + public void ping() { + //do nothing + } + + @Override + public BounceResult bounce(int pid) { + processManager.bounce(pid); + + try { + if(!processManager.waitForVMs(STARTUP_TIMEOUT)) { + throw new RuntimeException("VMs did not start up with 30 seconds"); + } + RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + pid); + return new BounceResult(pid, remote); + } catch (RemoteException | NotBoundException e) { + throw new RuntimeException("could not lookup name", e); + } catch (InterruptedException e) { + throw new RuntimeException("Failed waiting for VM", e); + } + } + } + + private static class DUnitHost extends Host { + private static final long serialVersionUID = -8034165624503666383L; + + private transient final VM debuggingVM; + + private transient ProcessManager processManager; + + public DUnitHost(String hostName, ProcessManager processManager) throws RemoteException { + super(hostName); + this.debuggingVM = new VM(this, -1, new RemoteDUnitVM()); + this.processManager = processManager; + } + + public void init(Registry registry, int numVMs) throws AccessException, RemoteException, NotBoundException, InterruptedException { + for(int i = 0; i < numVMs; i++) { + RemoteDUnitVMIF remote = processManager.getStub(i); + addVM(i, remote); + } + + addLocator(LOCATOR_VM_NUM, processManager.getStub(LOCATOR_VM_NUM)); + + addHost(this); + } + + @Override + public VM getVM(int n) { + + if(n == DEBUGGING_VM_NUM) { + //for ease of debugging, pass -1 to get the local VM + return debuggingVM; + } + + int oldVMCount = getVMCount(); + if(n >= oldVMCount) { + //If we don't have a VM with that number, dynamically create it. + try { + for(int i = oldVMCount; i <= n; i++) { + processManager.launchVM(i); + } + processManager.waitForVMs(STARTUP_TIMEOUT); + + for(int i = oldVMCount; i <= n; i++) { + addVM(i, processManager.getStub(i)); + } + + } catch (IOException | InterruptedException | NotBoundException e) { + throw new RuntimeException("Could not dynamically launch vm + " + n, e); + } + } + + return super.getVM(n); + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java new file mode 100644 index 0000000..7b053b6 --- /dev/null +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java @@ -0,0 +1,259 @@ +/* + * 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 com.gemstone.gemfire.test.dunit.standalone; + +import java.io.BufferedReader; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.PrintStream; +import java.lang.management.ManagementFactory; +import java.lang.management.RuntimeMXBean; +import java.rmi.AccessException; +import java.rmi.NotBoundException; +import java.rmi.RemoteException; +import java.rmi.registry.Registry; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.io.FileUtils; + +import com.gemstone.gemfire.internal.FileUtil; +import com.gemstone.gemfire.internal.logging.LogService; + +/** + * @author dsmith + * + */ +public class ProcessManager { + private int namingPort; + private Map<Integer, ProcessHolder> processes = new HashMap<Integer, ProcessHolder>(); + private File log4jConfig; + private int pendingVMs; + private Registry registry; + private int debugPort = Integer.getInteger("dunit.debug.basePort", 0); + private int suspendVM = Integer.getInteger("dunit.debug.suspendVM", -100); + + public ProcessManager(int namingPort, Registry registry) { + this.namingPort = namingPort; + this.registry = registry; + } + + public void launchVMs() throws IOException, NotBoundException { + log4jConfig = LogService.findLog4jConfigInCurrentDir(); + } + + public synchronized void launchVM(int vmNum) throws IOException { + if(processes.containsKey(vmNum)) { + throw new IllegalStateException("VM " + vmNum + " is already running."); + } + + String[] cmd = buildJavaCommand(vmNum, namingPort); + System.out.println("Executing " + Arrays.asList(cmd)); + File workingDir = getVMDir(vmNum); + try { + FileUtil.delete(workingDir); + } catch(IOException e) { + //This delete is occasionally failing on some platforms, maybe due to a lingering + //process. Allow the process to be launched anyway. + System.err.println("Unable to delete " + workingDir + ". Currently contains " + + Arrays.asList(workingDir.list())); + } + workingDir.mkdirs(); + if (log4jConfig != null) { + FileUtils.copyFileToDirectory(log4jConfig, workingDir); + } + + //TODO - delete directory contents, preferably with commons io FileUtils + Process process = Runtime.getRuntime().exec(cmd, null, workingDir); + pendingVMs++; + ProcessHolder holder = new ProcessHolder(process); + processes.put(vmNum, holder); + linkStreams(vmNum, holder, process.getErrorStream(), System.err); + linkStreams(vmNum, holder, process.getInputStream(), System.out); + } + + public static File getVMDir(int vmNum) { + return new File(DUnitLauncher.DUNIT_DIR, "vm" + vmNum); + } + + public synchronized void killVMs() { + for(ProcessHolder process : processes.values()) { + if(process != null) { + process.kill(); + } + } + } + + public synchronized boolean hasLiveVMs() { + for(ProcessHolder process : processes.values()) { + if(process != null && process.isAlive()) { + return true; + } + } + return false; + } + + public synchronized void bounce(int vmNum) { + if(!processes.containsKey(vmNum)) { + throw new IllegalStateException("No such process " + vmNum); + } + try { + ProcessHolder holder = processes.remove(vmNum); + holder.kill(); + holder.getProcess().waitFor(); + launchVM(vmNum); + } catch (InterruptedException | IOException e) { + throw new RuntimeException("Unable to restart VM " + vmNum, e); + } + } + + private void linkStreams(final int vmNum, final ProcessHolder holder, final InputStream in, final PrintStream out) { + Thread ioTransport = new Thread() { + public void run() { + BufferedReader reader = new BufferedReader(new InputStreamReader(in)); + String vmName = (vmNum==-2)? "[locator]" : "[vm_"+vmNum+"]"; + try { + String line = reader.readLine(); + while(line != null) { + if (line.length() == 0) { + out.println(); + } else { + out.print(vmName); + out.println(line); + } + line = reader.readLine(); + } + } catch(Exception e) { + if(!holder.isKilled()) { + out.println("Error transporting IO from child process"); + e.printStackTrace(out); + } + } + } + }; + + ioTransport.setDaemon(true); + ioTransport.start(); + } + + private String[] buildJavaCommand(int vmNum, int namingPort) { + String cmd = System.getProperty( "java.home" ) + File.separator + "bin" + File.separator + "java"; + String classPath = System.getProperty("java.class.path"); + //String tmpDir = System.getProperty("java.io.tmpdir"); + String agent = getAgentString(); + + String jdkDebug = ""; + if (debugPort > 0) { + jdkDebug += ",address=" + debugPort; + debugPort++; + } + + String jdkSuspend = vmNum == suspendVM ? "y" : "n"; + + return new String[] { + cmd, "-classpath", classPath, + "-D" + DUnitLauncher.RMI_PORT_PARAM + "=" + namingPort, + "-D" + DUnitLauncher.VM_NUM_PARAM + "=" + vmNum, + "-D" + DUnitLauncher.WORKSPACE_DIR_PARAM + "=" + new File(".").getAbsolutePath(), + "-DlogLevel=" + DUnitLauncher.LOG_LEVEL, + "-Djava.library.path=" + System.getProperty("java.library.path"), + "-Xrunjdwp:transport=dt_socket,server=y,suspend=" + jdkSuspend + jdkDebug, + "-XX:+HeapDumpOnOutOfMemoryError", + "-Xmx512m", + "-Dgemfire.DEFAULT_MAX_OPLOG_SIZE=10", + "-Dgemfire.disallowMcastDefaults=true", + "-ea", + agent, + "dunit.standalone.ChildVM" + }; + } + + /** + * Get the java agent passed to this process and pass it to the child VMs. + * This was added to support jacoco code coverage reports + */ + private String getAgentString() { + RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean(); + if (runtimeBean != null) { + for(String arg: runtimeBean.getInputArguments()) { + if(arg.contains("-javaagent:")) { + //HACK for gradle bug GRADLE-2859. Jacoco is passing a relative path + //That won't work when we pass this to dunit VMs in a different + //directory + arg = arg.replace("-javaagent:..", "-javaagent:" + System.getProperty("user.dir") + File.separator + ".."); + arg = arg.replace("destfile=..", "destfile=" + System.getProperty("user.dir") + File.separator + ".."); + return arg; + } + } + } + + return "-DdummyArg=true"; + } + + synchronized void signalVMReady() { + pendingVMs--; + this.notifyAll(); + } + + public synchronized boolean waitForVMs(long timeout) throws InterruptedException { + long end = System.currentTimeMillis() + timeout; + while(pendingVMs > 0) { + long remaining = end - System.currentTimeMillis(); + if(remaining <= 0) { + return false; + } + this.wait(remaining); + } + + return true; + } + + private static class ProcessHolder { + private final Process process; + private volatile boolean killed = false; + + public ProcessHolder(Process process) { + this.process = process; + } + + public void kill() { + this.killed = true; + process.destroy(); + + } + + public Process getProcess() { + return process; + } + + public boolean isKilled() { + return killed; + } + + public boolean isAlive() { + return !killed && process.isAlive(); + } + } + + public RemoteDUnitVMIF getStub(int i) throws AccessException, RemoteException, NotBoundException, InterruptedException { + waitForVMs(DUnitLauncher.STARTUP_TIMEOUT); + return (RemoteDUnitVMIF) registry.lookup("vm" + i); + } +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java new file mode 100644 index 0000000..51c6177 --- /dev/null +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java @@ -0,0 +1,142 @@ +/* + * 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 com.gemstone.gemfire.test.dunit.standalone; + +import java.rmi.RemoteException; +import java.rmi.server.UnicastRemoteObject; + +import org.apache.logging.log4j.Logger; + +import com.gemstone.gemfire.internal.logging.LogService; + +import hydra.MethExecutor; +import hydra.MethExecutorResult; + +/** + * @author dsmith + * + */ +public class RemoteDUnitVM extends UnicastRemoteObject implements RemoteDUnitVMIF { + + private static final Logger logger = LogService.getLogger(); + + public RemoteDUnitVM() throws RemoteException { + super(); + } + + /** + * Called remotely by the master controller to cause the client to execute + * the instance method on the object. Does this synchronously (does not spawn + * a thread). This method is used by the unit test framework, dunit. + * + * @param obj the object to execute the method on + * @param methodName the name of the method to execute + * @return the result of method execution + */ + public MethExecutorResult executeMethodOnObject( Object obj, String methodName ) { + String name = obj.getClass().getName() + "." + methodName + + " on object: " + obj; + logger.info("Received method: " + name); + long start = System.currentTimeMillis(); + MethExecutorResult result = MethExecutor.executeObject( obj, methodName ); + long delta = System.currentTimeMillis() - start; + logger.info( "Got result: " + result.toString().trim() + " from " + + name + " (took " + delta + " ms)"); + return result; + } + + /** + * Executes a given instance method on a given object with the given + * arguments. + */ + public MethExecutorResult executeMethodOnObject(Object obj, + String methodName, + Object[] args) { + String name = obj.getClass().getName() + "." + methodName + + (args != null ? " with " + args.length + " args": "") + + " on object: " + obj; + logger.info("Received method: " + name); + long start = System.currentTimeMillis(); + MethExecutorResult result = + MethExecutor.executeObject(obj, methodName, args); + long delta = System.currentTimeMillis() - start; + logger.info( "Got result: " + result.toString() + " from " + name + + " (took " + delta + " ms)"); + return result; + } + + /** + * Called remotely by the master controller to cause the client to execute + * the method on the class. Does this synchronously (does not spawn a thread). + * This method is used by the unit test framework, dunit. + * + * @param className the name of the class execute + * @param methodName the name of the method to execute + * @return the result of method execution + */ + public MethExecutorResult executeMethodOnClass( String className, String methodName ) { + String name = className + "." + methodName; + logger.info("Received method: " + name); + long start = System.currentTimeMillis(); + MethExecutorResult result = MethExecutor.execute( className, methodName ); + long delta = System.currentTimeMillis() - start; + logger.info( "Got result: " + result.toString() + " from " + name + + " (took " + delta + " ms)"); + + return result; + } + + /** + * Executes a given static method in a given class with the given + * arguments. + */ + public MethExecutorResult executeMethodOnClass(String className, + String methodName, + Object[] args) { + String name = className + "." + methodName + + (args != null ? " with " + args.length + " args": ""); + logger.info("Received method: " + name); + long start = System.currentTimeMillis(); + MethExecutorResult result = + MethExecutor.execute(className, methodName, args); + long delta = System.currentTimeMillis() - start; + logger.info( "Got result: " + result.toString() + " from " + name + + " (took " + delta + " ms)"); + return result; + } + + public void executeTask(int tsid, int type, int index) throws RemoteException { + throw new UnsupportedOperationException(); + + } + + public void runShutdownHook() throws RemoteException { + + } + + public void notifyDynamicActionComplete(int actionId) throws RemoteException { + throw new UnsupportedOperationException(); + + } + + public void shutDownVM() throws RemoteException { + ChildVM.stopVM(); + } + + public void disconnectVM() throws RemoteException { + } +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java new file mode 100644 index 0000000..849e2f2 --- /dev/null +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java @@ -0,0 +1,36 @@ +/* + * 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 com.gemstone.gemfire.test.dunit.standalone; + +import hydra.MethExecutorResult; + +import java.rmi.Remote; +import java.rmi.RemoteException; + +public interface RemoteDUnitVMIF extends Remote { + + MethExecutorResult executeMethodOnObject(Object o, String methodName) throws RemoteException; + + MethExecutorResult executeMethodOnObject(Object o, String methodName, + Object[] args) throws RemoteException; + + MethExecutorResult executeMethodOnClass(String name, String methodName, + Object[] args) throws RemoteException; + + void shutDownVM() throws RemoteException; + +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.java new file mode 100644 index 0000000..eef24fe --- /dev/null +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/StandAloneDUnitEnv.java @@ -0,0 +1,75 @@ +/* + * 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 com.gemstone.gemfire.test.dunit.standalone; + +import java.io.File; +import java.rmi.RemoteException; +import java.util.Properties; + +import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher.MasterRemote; + +import dunit.DUnitEnv; + +public class StandAloneDUnitEnv extends DUnitEnv { + + private MasterRemote master; + + public StandAloneDUnitEnv(MasterRemote master) { + this.master = master; + } + + @Override + public String getLocatorString() { + return DUnitLauncher.getLocatorString(); + } + + @Override + public String getLocatorAddress() { + return "localhost"; + } + + @Override + public int getLocatorPort() { + return DUnitLauncher.locatorPort; + } + + @Override + public Properties getDistributedSystemProperties() { + return DUnitLauncher.getDistributedSystemProperties(); + } + + @Override + public int getPid() { + return Integer.getInteger(DUnitLauncher.VM_NUM_PARAM, -1).intValue(); + } + + @Override + public int getVMID() { + return getPid(); + } + + @Override + public BounceResult bounce(int pid) throws RemoteException { + return master.bounce(pid); + } + + @Override + public File getWorkingDirectory(int pid) { + return ProcessManager.getVMDir(pid); + } + +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java new file mode 100644 index 0000000..76faf93 --- /dev/null +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java @@ -0,0 +1,132 @@ +/* + * 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 com.gemstone.gemfire.test.dunit.tests; + +import java.util.Properties; + +import dunit.AsyncInvocation; +import dunit.DistributedTestCase; +import dunit.Host; +import dunit.RMIException; +import dunit.VM; + +/** + * This class tests the basic functionality of the distributed unit + * test framework. + */ +public class BasicDUnitTest extends DistributedTestCase { + + public BasicDUnitTest(String name) { + super(name); + } + + //////// Test Methods + + /** + * Tests how the Hydra framework handles an error + */ + public void _testDontCatchRemoteException() { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + vm.invoke(this.getClass(), "remoteThrowException"); + } + + public void testRemoteInvocationWithException() { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + try { + vm.invoke(this.getClass(), "remoteThrowException"); + fail("Should have thrown a BasicTestException"); + + } catch (RMIException ex) { + assertTrue(ex.getCause() instanceof BasicTestException); + } + } + + static class BasicTestException extends RuntimeException { + BasicTestException() { + this("Test exception. Please ignore."); + } + + BasicTestException(String s) { + super(s); + } + } + + /** + * Accessed via reflection. DO NOT REMOVE + * + */ + protected static void remoteThrowException() { + String s = "Test exception. Please ignore."; + throw new BasicTestException(s); + } + + public void _testRemoteInvocationBoolean() { + + } + + public void testRemoteInvokeAsync() throws InterruptedException { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + String name = this.getUniqueName(); + String value = "Hello"; + + AsyncInvocation ai = + vm.invokeAsync(this.getClass(), "remoteBind", + new Object[] { name, value }); + ai.join(); + // TODO shouldn't we call fail() here? + if (ai.exceptionOccurred()) { + fail("remoteBind failed", ai.getException()); + } + + ai = vm.invokeAsync(this.getClass(), "remoteValidateBind", + new Object[] {name, value }); + ai.join(); + if (ai.exceptionOccurred()) { + fail("remoteValidateBind failed", ai.getException()); + } + } + + private static Properties bindings = new Properties(); + private static void remoteBind(String name, String s) { + new BasicDUnitTest("bogus").getSystem(); // forces connection + bindings.setProperty(name, s); + } + + private static void remoteValidateBind(String name, String expected) + { + assertEquals(expected, bindings.getProperty(name)); + } + + public void testRemoteInvokeAsyncWithException() + throws InterruptedException { + + Host host = Host.getHost(0); + VM vm = host.getVM(0); +// String name = this.getUniqueName(); +// String value = "Hello"; + + AsyncInvocation ai = + vm.invokeAsync(this.getClass(), "remoteThrowException"); + ai.join(); + assertTrue(ai.exceptionOccurred()); + Throwable ex = ai.getException(); + assertTrue(ex instanceof BasicTestException); + } +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java new file mode 100644 index 0000000..abbe229 --- /dev/null +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/TestFailure.java @@ -0,0 +1,50 @@ +/* + * 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 com.gemstone.gemfire.test.dunit.tests; + +import dunit.*; + +/** + * The tests in this class always fail. It is used when developing + * DUnit to give us an idea of how test failure are logged, etc. + * + * @author David Whitlock + * + * @since 3.0 + */ +public class TestFailure extends DistributedTestCase { + + public TestFailure(String name) { + super(name); + } + + //////// Test Methods + + public void testFailure() { + assertTrue("Test Failure", false); + } + + public void testError() { + String s = "Test Error"; + throw new Error(s); + } + + public void testHang() throws InterruptedException { + Thread.sleep(100000 * 1000); + } + +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java new file mode 100644 index 0000000..3562f86 --- /dev/null +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/VMDUnitTest.java @@ -0,0 +1,237 @@ +/* + * 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 com.gemstone.gemfire.test.dunit.tests; + +import dunit.*; + +import java.io.Serializable; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * This class tests the functionality of the {@link VM} class. + */ +public class VMDUnitTest extends DistributedTestCase { + + private static final boolean BOOLEAN_VALUE = true; + private static final byte BYTE_VALUE = (byte) 40; + private static final long LONG_VALUE = 42L; + private static final String STRING_VALUE = "BLAH BLAH BLAH"; + + public VMDUnitTest(String name) { + super(name); + } + + //////// Test Methods + + public void notestInvokeNonExistentMethod() { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + try { + vm.invoke(VMDUnitTest.class, "nonExistentMethod"); + fail("Should have thrown an RMIException"); + + } catch (RMIException ex) { + String s = "Excepted a NoSuchMethodException, got a " + + ex.getCause();; + assertTrue(s, ex.getCause() instanceof NoSuchMethodException); + } + } + + /** + * Accessed via reflection. DO NOT REMOVE + * @return + */ + protected static byte remoteByteMethod() { + return BYTE_VALUE; + } + + public void notestInvokeStaticBoolean() { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + assertEquals(BOOLEAN_VALUE, + vm.invokeBoolean(VMDUnitTest.class, "remoteBooleanMethod")); + } + + /** + * Accessed via reflection. DO NOT REMOVE + * @return + */ + protected static boolean remoteBooleanMethod() { + return BOOLEAN_VALUE; + } + + public void notestInvokeStaticBooleanNotBoolean() { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + try { + vm.invokeBoolean(VMDUnitTest.class, "remoteByteMethod"); + fail("Should have thrown an IllegalArgumentException"); + + } catch (IllegalArgumentException ex) { + + } + } + + public void notestInvokeStaticLong() { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + assertEquals(LONG_VALUE, + vm.invokeLong(VMDUnitTest.class, "remoteLongMethod")); + } + + /** + * Accessed via reflection. DO NOT REMOVE + * @return + */ + protected static long remoteLongMethod() { + return LONG_VALUE; + } + + public void notestInvokeStaticLongNotLong() { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + try { + vm.invokeLong(VMDUnitTest.class, "remoteByteMethod"); + fail("Should have thrown an IllegalArgumentException"); + + } catch (IllegalArgumentException ex) { + + } + } + + protected static class ClassWithLong implements Serializable { + public long getLong() { + return LONG_VALUE; + } + } + + protected static class ClassWithByte implements Serializable { + public byte getByte() { + return BYTE_VALUE; + } + } + + public void notestInvokeInstanceLong() { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + assertEquals(LONG_VALUE, + vm.invokeLong(new ClassWithLong(), "getLong")); + } + + public void notestInvokeInstanceLongNotLong() { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + try { + vm.invokeLong(new ClassWithByte(), "getByte"); + fail("Should have thrown an IllegalArgumentException"); + + } catch (IllegalArgumentException ex) { + + } + } + + protected static class InvokeRunnable + implements Serializable, Runnable { + + public void run() { + throw new BasicDUnitTest.BasicTestException(); + } + } + + protected static class ClassWithString implements Serializable { + public String getString() { + return STRING_VALUE; + } + } + + public void notestInvokeInstance() { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + assertEquals(STRING_VALUE, + vm.invoke(new ClassWithString(), "getString")); + } + + public void notestInvokeRunnable() { + Host host = Host.getHost(0); + VM vm = host.getVM(0); + try { + vm.invoke(new InvokeRunnable()); + fail("Should have thrown a BasicTestException"); + + } catch (RMIException ex) { + assertTrue(ex.getCause() instanceof BasicDUnitTest.BasicTestException); + } + } + + private static final AtomicInteger COUNTER = new AtomicInteger(); + public static Integer getAndIncStaticCount() { + return new Integer(COUNTER.getAndIncrement()); + } + public static Integer incrementStaticCount(Integer inc) { + return new Integer(COUNTER.addAndGet(inc.intValue())); + } + public static void incStaticCount() { + COUNTER.incrementAndGet(); + } + public static class VMTestObject implements Serializable { + private static final long serialVersionUID = 1L; + private final AtomicInteger val; + public VMTestObject(int init) { + this.val = new AtomicInteger(init); + } + public Integer get() { + return new Integer(this.val.get()); + } + public Integer incrementAndGet() { + return new Integer(this.val.incrementAndGet()); + } + public void set(Integer newVal) { + this.val.set(newVal.intValue()); + } + } + public void testReturnValue() throws Exception { + final Host host = Host.getHost(0); + final VM vm = host.getVM(0); + // Assert class static invocation works + AsyncInvocation a1 = vm.invokeAsync(getClass(), "getAndIncStaticCount"); + a1.join(); + assertEquals(new Integer(0), a1.getReturnValue()); + // Assert class static invocation with args works + a1 = vm.invokeAsync(getClass(), "incrementStaticCount", new Object[] {new Integer(2)}); + a1.join(); + assertEquals(new Integer(3), a1.getReturnValue()); + // Assert that previous values are not returned when invoking method w/ no return val + a1 = vm.invokeAsync(getClass(), "incStaticCount"); + a1.join(); + assertNull(a1.getReturnValue()); + // Assert that previous null returns are over-written + a1 = vm.invokeAsync(getClass(), "getAndIncStaticCount"); + a1.join(); + assertEquals(new Integer(4), a1.getReturnValue()); + + // Assert object method invocation works with zero arg method + final VMTestObject o = new VMTestObject(0); + a1 = vm.invokeAsync(o, "incrementAndGet", new Object[] {}); + a1.join(); + assertEquals(new Integer(1), a1.getReturnValue()); + // Assert object method invocation works with no return + a1 = vm.invokeAsync(o, "set", new Object[] {new Integer(3)}); + a1.join(); + assertNull(a1.getReturnValue()); + } +} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/BounceResult.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/dunit/BounceResult.java b/gemfire-core/src/test/java/dunit/BounceResult.java deleted file mode 100644 index d8ac186..0000000 --- a/gemfire-core/src/test/java/dunit/BounceResult.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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 dunit; - -public class BounceResult { - private final int newPid; - private final RemoteDUnitVMIF newClient; - - public BounceResult(int newPid, RemoteDUnitVMIF newClient) { - this.newPid = newPid; - this.newClient = newClient; - } - - public int getNewPid() { - return newPid; - } - - public RemoteDUnitVMIF getNewClient() { - return newClient; - } - -} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/DUnitEnv.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/dunit/DUnitEnv.java b/gemfire-core/src/test/java/dunit/DUnitEnv.java index 19c3635..54fe67f 100644 --- a/gemfire-core/src/test/java/dunit/DUnitEnv.java +++ b/gemfire-core/src/test/java/dunit/DUnitEnv.java @@ -23,6 +23,8 @@ import java.io.File; import java.rmi.RemoteException; import java.util.Properties; +import com.gemstone.gemfire.test.dunit.standalone.BounceResult; + /** * This class provides an abstraction over the environment http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/Host.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/dunit/Host.java b/gemfire-core/src/test/java/dunit/Host.java index cc41316..0c69783 100644 --- a/gemfire-core/src/test/java/dunit/Host.java +++ b/gemfire-core/src/test/java/dunit/Host.java @@ -18,6 +18,8 @@ package dunit; import java.util.*; +import com.gemstone.gemfire.test.dunit.standalone.RemoteDUnitVMIF; + /** * <P>This class represents a host on which a remote method may be * invoked. It provides access to the VMs and GemFire systems that http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/RemoteDUnitVMIF.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/dunit/RemoteDUnitVMIF.java b/gemfire-core/src/test/java/dunit/RemoteDUnitVMIF.java deleted file mode 100644 index 5dffa47..0000000 --- a/gemfire-core/src/test/java/dunit/RemoteDUnitVMIF.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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 dunit; - -import hydra.MethExecutorResult; - -import java.rmi.Remote; -import java.rmi.RemoteException; - -public interface RemoteDUnitVMIF extends Remote { - - MethExecutorResult executeMethodOnObject(Object o, String methodName) throws RemoteException; - - MethExecutorResult executeMethodOnObject(Object o, String methodName, - Object[] args) throws RemoteException; - - MethExecutorResult executeMethodOnClass(String name, String methodName, - Object[] args) throws RemoteException; - - void shutDownVM() throws RemoteException; - -} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/VM.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/dunit/VM.java b/gemfire-core/src/test/java/dunit/VM.java index 3c0a7b7..f4cde93 100644 --- a/gemfire-core/src/test/java/dunit/VM.java +++ b/gemfire-core/src/test/java/dunit/VM.java @@ -16,8 +16,6 @@ */ package dunit; -import hydra.MethExecutorResult; - import java.io.File; import java.io.PrintWriter; import java.io.StringWriter; @@ -26,6 +24,11 @@ import java.util.concurrent.Callable; //import java.util.Iterator; //import java.util.Vector; +import hydra.MethExecutorResult; + +import com.gemstone.gemfire.test.dunit.standalone.BounceResult; +import com.gemstone.gemfire.test.dunit.standalone.RemoteDUnitVMIF; + /** * This class represents a Java Virtual Machine that runs on a host. * http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/standalone/ChildVM.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/dunit/standalone/ChildVM.java b/gemfire-core/src/test/java/dunit/standalone/ChildVM.java deleted file mode 100644 index 45a236a..0000000 --- a/gemfire-core/src/test/java/dunit/standalone/ChildVM.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * 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 dunit.standalone; - -import hydra.HydraRuntimeException; -import hydra.Log; - -import java.rmi.Naming; - -import org.apache.logging.log4j.Logger; - -import com.gemstone.gemfire.internal.OSProcess; -import com.gemstone.gemfire.internal.logging.LogService; - -import dunit.standalone.DUnitLauncher.MasterRemote; - -/** - * @author dsmith - * - */ -public class ChildVM { - - private static boolean stopMainLoop = false; - - /** - * tells the main() loop to exit - */ - public static void stopVM() { - stopMainLoop = true; - } - - static { - createHydraLogWriter(); - } - - private final static Logger logger = LogService.getLogger(); - private static RemoteDUnitVM dunitVM; - - public static void main(String[] args) throws Throwable { - try { - int namingPort = Integer.getInteger(DUnitLauncher.RMI_PORT_PARAM).intValue(); - int vmNum = Integer.getInteger(DUnitLauncher.VM_NUM_PARAM).intValue(); - int pid = OSProcess.getId(); - logger.info("VM" + vmNum + " is launching" + (pid > 0? " with PID " + pid : "")); - MasterRemote holder = (MasterRemote) Naming.lookup("//localhost:" + namingPort + "/" + DUnitLauncher.MASTER_PARAM); - DUnitLauncher.init(holder); - DUnitLauncher.locatorPort = holder.getLocatorPort(); - dunitVM = new RemoteDUnitVM(); - Naming.rebind("//localhost:" + namingPort + "/vm" + vmNum, dunitVM); - holder.signalVMReady(); - //This loop is here so this VM will die even if the master is mean killed. - while (!stopMainLoop) { - holder.ping(); - Thread.sleep(1000); - } - } catch (Throwable t) { - t.printStackTrace(); - System.exit(1); - } - } - - private static void createHydraLogWriter() { - try { - Log.createLogWriter("dunit-childvm", "fine"); - } catch (HydraRuntimeException ignore) { - } - } -} http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c57a88c6/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java b/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java deleted file mode 100644 index 72c33d6..0000000 --- a/gemfire-core/src/test/java/dunit/standalone/DUnitLauncher.java +++ /dev/null @@ -1,463 +0,0 @@ -/* - * 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 dunit.standalone; - -import hydra.Log; -import hydra.MethExecutorResult; - -import java.io.BufferedReader; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.FileReader; -import java.io.IOException; -import java.lang.reflect.Method; -import java.net.InetAddress; -import java.net.URISyntaxException; -import java.nio.channels.FileChannel; -import java.nio.charset.Charset; -import java.rmi.AccessException; -import java.rmi.AlreadyBoundException; -import java.rmi.NotBoundException; -import java.rmi.Remote; -import java.rmi.RemoteException; -import java.rmi.registry.LocateRegistry; -import java.rmi.registry.Registry; -import java.rmi.server.UnicastRemoteObject; -import java.util.List; -import java.util.Properties; - -import org.apache.logging.log4j.Level; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.core.LoggerContext; -import org.apache.logging.log4j.core.appender.FileAppender; -import org.apache.logging.log4j.core.config.LoggerConfig; -import org.apache.logging.log4j.core.layout.PatternLayout; -import org.junit.Assert; - -import batterytest.greplogs.ExpectedStrings; -import batterytest.greplogs.LogConsumer; - -import com.gemstone.gemfire.distributed.Locator; -import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave; -import com.gemstone.gemfire.internal.AvailablePortHelper; -import com.gemstone.gemfire.internal.logging.LogService; - -import dunit.BounceResult; -import dunit.DUnitEnv; -import dunit.Host; -import dunit.RemoteDUnitVMIF; -import dunit.SerializableCallable; -import dunit.VM; - -/** - * A class to build a fake test configuration and launch some DUnit VMS. - * - * For use within eclipse. This class completely skips hydra and just starts - * some vms directly, creating a fake test configuration - * - * Also, it's a good idea to set your working directory, because the test code - * a lot of files that it leaves around. - * - * @author dsmith - * - */ -public class DUnitLauncher { - - /** change this to use a different log level in unit tests */ - public static final String LOG_LEVEL = System.getProperty("logLevel", "info"); - - static int locatorPort; - - private static final int NUM_VMS = 4; - private static final int DEBUGGING_VM_NUM = -1; - private static final int LOCATOR_VM_NUM = -2; - - static final long STARTUP_TIMEOUT = 30 * 1000; - private static final String SUSPECT_FILENAME = "dunit_suspect.log"; - private static File DUNIT_SUSPECT_FILE; - - public static final String DUNIT_DIR = "dunit"; - public static final String WORKSPACE_DIR_PARAM = "WORKSPACE_DIR"; - public static final boolean LOCATOR_LOG_TO_DISK = Boolean.getBoolean("locatorLogToDisk"); - - static final String MASTER_PARAM = "DUNIT_MASTER"; - static final String RMI_PORT_PARAM = "gemfire.DUnitLauncher.RMI_PORT"; - static final String VM_NUM_PARAM = "gemfire.DUnitLauncher.VM_NUM"; - - private static final String LAUNCHED_PROPERTY = "gemfire.DUnitLauncher.LAUNCHED"; - - private static Master master; - - private DUnitLauncher() { - } - - private static boolean isHydra() { - try { - //TODO - this is hacky way to test for a hydra environment - see - //if there is registered test configuration object. - Class<?> clazz = Class.forName("hydra.TestConfig"); - Method getInstance = clazz.getMethod("getInstance", new Class[0]); - getInstance.invoke(null); - return true; - } catch (Exception e) { - return false; - } - } - /** - * Launch DUnit. If the unit test was launched through - * the hydra framework, leave the test alone. - */ - public static void launchIfNeeded() { - if(System.getProperties().contains(VM_NUM_PARAM)) { - //we're a dunit child vm, do nothing. - return; - } - - if(!isHydra() &&!isLaunched()) { - try { - launch(); - } catch (Exception e) { - throw new RuntimeException("Unable to launch dunit VMS", e); - } - } - } - - /** - * Test it see if the eclise dunit environment is launched. - */ - public static boolean isLaunched() { - return Boolean.getBoolean(LAUNCHED_PROPERTY); - } - - public static String getLocatorString() { - return "localhost[" + locatorPort + "]"; - } - - - private static void launch() throws URISyntaxException, AlreadyBoundException, IOException, InterruptedException, NotBoundException { -// initialize the log writer that hydra uses - Log.createLogWriter( "dunit-master", LOG_LEVEL ); - - DUNIT_SUSPECT_FILE = new File(SUSPECT_FILENAME); - DUNIT_SUSPECT_FILE.delete(); - DUNIT_SUSPECT_FILE.deleteOnExit(); - - locatorPort = AvailablePortHelper.getRandomAvailableTCPPort(); - - //create an RMI registry and add an object to share our tests config - int namingPort = AvailablePortHelper.getRandomAvailableTCPPort(); - Registry registry = LocateRegistry.createRegistry(namingPort); - - final ProcessManager processManager = new ProcessManager(namingPort, registry); - master = new Master(registry, processManager); - registry.bind(MASTER_PARAM, master); - - Runtime.getRuntime().addShutdownHook(new Thread() { - public void run() { -// System.out.println("shutting down DUnit JVMs"); -// for (int i=0; i<NUM_VMS; i++) { -// try { -// processManager.getStub(i).shutDownVM(); -// } catch (Exception e) { -// System.out.println("exception shutting down vm_"+i+": " + e); -// } -// } -// // TODO - hasLiveVMs always returns true -// System.out.print("waiting for JVMs to exit"); -// long giveUp = System.currentTimeMillis() + 5000; -// while (giveUp > System.currentTimeMillis()) { -// if (!processManager.hasLiveVMs()) { -// return; -// } -// System.out.print("."); -// System.out.flush(); -// try { -// Thread.sleep(1000); -// } catch (InterruptedException e) { -// break; -// } -// } -// System.out.println("\nkilling any remaining JVMs"); - processManager.killVMs(); - } - }); - - //Create a VM for the locator - processManager.launchVM(LOCATOR_VM_NUM); - - //Launch an initial set of VMs - for(int i=0; i < NUM_VMS; i++) { - processManager.launchVM(i); - } - - //wait for the VMS to start up - if(!processManager.waitForVMs(STARTUP_TIMEOUT)) { - throw new RuntimeException("VMs did not start up with 30 seconds"); - } - - //populate the Host class with our stubs. The tests use this host class - DUnitHost host = new DUnitHost(InetAddress.getLocalHost().getCanonicalHostName(), processManager); - host.init(registry, NUM_VMS); - - init(master); - - startLocator(registry); - } - - public static Properties getDistributedSystemProperties() { - Properties p = new Properties(); - p.setProperty("locators", getLocatorString()); - p.setProperty("mcast-port", "0"); - p.setProperty("enable-cluster-configuration", "false"); - p.setProperty("use-cluster-configuration", "false"); - p.setProperty("log-level", LOG_LEVEL); - return p; - } - - /** - * Add an appender to Log4j which sends all INFO+ messages to a separate file - * which will be used later to scan for suspect strings. The pattern of the - * messages conforms to the original log format so that hydra will be able - * to parse them. - */ - private static void addSuspectFileAppender(final String workspaceDir) { - final String suspectFilename = new File(workspaceDir, SUSPECT_FILENAME).getAbsolutePath(); - - final LoggerContext appenderContext = ((org.apache.logging.log4j.core.Logger) - LogManager.getLogger(LogService.BASE_LOGGER_NAME)).getContext(); - - final PatternLayout layout = PatternLayout.createLayout( - "[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} <%thread> tid=%tid] %message%n%throwable%n", null, null, - Charset.defaultCharset(), true, false, "", ""); - - final FileAppender fileAppender = FileAppender.createAppender(suspectFilename, "true", "false", - DUnitLauncher.class.getName(), "true", "false", "false", "0", layout, null, null, null, appenderContext.getConfiguration()); - fileAppender.start(); - - LoggerConfig loggerConfig = appenderContext.getConfiguration().getLoggerConfig(LogService.BASE_LOGGER_NAME); - loggerConfig.addAppender(fileAppender, Level.INFO, null); - } - - private static void startLocator(Registry registry) throws IOException, NotBoundException { - RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + LOCATOR_VM_NUM); - final File locatorLogFile = - LOCATOR_LOG_TO_DISK ? new File("locator-" + locatorPort + ".log") : new File(""); - MethExecutorResult result = remote.executeMethodOnObject(new SerializableCallable() { - public Object call() throws IOException { - Properties p = getDistributedSystemProperties(); - // I never want this locator to end up starting a jmx manager - // since it is part of the unit test framework - p.setProperty("jmx-manager", "false"); - //Disable the shared configuration on this locator. - //Shared configuration tests create their own locator - p.setProperty("enable-cluster-configuration", "false"); - //Tell the locator it's the first in the system for - //faster boot-up - - System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true"); - try { - Locator.startLocatorAndDS(locatorPort, locatorLogFile, p); - } finally { - System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY); - } - - return null; - } - }, "call"); - if(result.getException() != null) { - RuntimeException ex = new RuntimeException("Failed to start locator", result.getException()); - ex.printStackTrace(); - throw ex; - } - } - - public static void init(MasterRemote master) { - DUnitEnv.set(new StandAloneDUnitEnv(master)); - //fake out tests that are using a bunch of hydra stuff - String workspaceDir = System.getProperty(DUnitLauncher.WORKSPACE_DIR_PARAM) ; - workspaceDir = workspaceDir == null ? new File(".").getAbsolutePath() : workspaceDir; - - addSuspectFileAppender(workspaceDir); - - //Free off heap memory when disconnecting from the distributed system - System.setProperty("gemfire.free-off-heap-memory", "true"); - - //indicate that this CM is controlled by the eclipse dunit. - System.setProperty(LAUNCHED_PROPERTY, "true"); - } - - public static void closeAndCheckForSuspects() { - if (isLaunched()) { - final boolean skipLogMsgs = ExpectedStrings.skipLogMsgs("dunit"); - final List<?> expectedStrings = ExpectedStrings.create("dunit"); - final LogConsumer logConsumer = new LogConsumer(skipLogMsgs, expectedStrings, "log4j", 5); - - final StringBuilder suspectStringBuilder = new StringBuilder(); - - BufferedReader buffReader = null; - FileChannel fileChannel = null; - try { - fileChannel = new FileOutputStream(DUNIT_SUSPECT_FILE, true).getChannel(); - buffReader = new BufferedReader(new FileReader(DUNIT_SUSPECT_FILE)); - } catch (FileNotFoundException e) { - System.err.println("Could not find the suspect string output file: " + e); - return; - } - try { - String line; - try { - while ((line = buffReader.readLine()) != null) { - final StringBuilder builder = logConsumer.consume(line); - if (builder != null) { - suspectStringBuilder.append(builder); - } - } - } catch (IOException e) { - System.err.println("Could not read the suspect string output file: " + e); - } - - try { - fileChannel.truncate(0); - } catch (IOException e) { - System.err.println("Could not truncate the suspect string output file: " + e); - } - - } finally { - try { - buffReader.close(); - fileChannel.close(); - } catch (IOException e) { - System.err.println("Could not close the suspect string output file: " + e); - } - } - - if (suspectStringBuilder.length() != 0) { - System.err.println("Suspicious strings were written to the log during this run.\n" - + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n" - + suspectStringBuilder); - - Assert.fail("Suspicious strings were written to the log during this run.\n" - + "Fix the strings or use DistributedTestCase.addExpectedException to ignore.\n" - + suspectStringBuilder); - } - } - } - - public interface MasterRemote extends Remote { - public int getLocatorPort() throws RemoteException; - public void signalVMReady() throws RemoteException; - public void ping() throws RemoteException; - public BounceResult bounce(int pid) throws RemoteException; - } - - public static class Master extends UnicastRemoteObject implements MasterRemote { - private static final long serialVersionUID = 1178600200232603119L; - - private final Registry registry; - private final ProcessManager processManager; - - - public Master(Registry registry, ProcessManager processManager) throws RemoteException { - this.processManager = processManager; - this.registry = registry; - } - - public int getLocatorPort() throws RemoteException{ - return locatorPort; - } - - public synchronized void signalVMReady() { - processManager.signalVMReady(); - } - - public void ping() { - //do nothing - } - - @Override - public BounceResult bounce(int pid) { - processManager.bounce(pid); - - try { - if(!processManager.waitForVMs(STARTUP_TIMEOUT)) { - throw new RuntimeException("VMs did not start up with 30 seconds"); - } - RemoteDUnitVMIF remote = (RemoteDUnitVMIF) registry.lookup("vm" + pid); - return new BounceResult(pid, remote); - } catch (RemoteException | NotBoundException e) { - throw new RuntimeException("could not lookup name", e); - } catch (InterruptedException e) { - throw new RuntimeException("Failed waiting for VM", e); - } - } - } - - private static class DUnitHost extends Host { - private static final long serialVersionUID = -8034165624503666383L; - - private transient final VM debuggingVM; - - private transient ProcessManager processManager; - - public DUnitHost(String hostName, ProcessManager processManager) throws RemoteException { - super(hostName); - this.debuggingVM = new VM(this, -1, new RemoteDUnitVM()); - this.processManager = processManager; - } - - public void init(Registry registry, int numVMs) throws AccessException, RemoteException, NotBoundException, InterruptedException { - for(int i = 0; i < numVMs; i++) { - RemoteDUnitVMIF remote = processManager.getStub(i); - addVM(i, remote); - } - - addLocator(LOCATOR_VM_NUM, processManager.getStub(LOCATOR_VM_NUM)); - - addHost(this); - } - - @Override - public VM getVM(int n) { - - if(n == DEBUGGING_VM_NUM) { - //for ease of debugging, pass -1 to get the local VM - return debuggingVM; - } - - int oldVMCount = getVMCount(); - if(n >= oldVMCount) { - //If we don't have a VM with that number, dynamically create it. - try { - for(int i = oldVMCount; i <= n; i++) { - processManager.launchVM(i); - } - processManager.waitForVMs(STARTUP_TIMEOUT); - - for(int i = oldVMCount; i <= n; i++) { - addVM(i, processManager.getStub(i)); - } - - } catch (IOException | InterruptedException | NotBoundException e) { - throw new RuntimeException("Could not dynamically launch vm + " + n, e); - } - } - - return super.getVM(n); - } - } -}