From 101fae9c14709ac577cb86b7f2c58bc98558034e Mon Sep 17 00:00:00 2001 From: Laurent Cohen Date: Sat, 5 Jan 2019 03:14:27 +0100 Subject: [PATCH] implementing issue JPPF-564 - node side changes --- .../ui/monitoring/node/NodeDataPanel.java | 3 +- .../client/balancer/ChannelWrapperLocal.java | 2 +- .../balancer/ClientExecutionManager.java | 10 +- .../client/balancer/ClientTaskBundle.java | 3 +- .../execute/AbstractExecutionManager.java | 63 +-- .../org/jppf/execute/ExecutionManager.java | 6 +- .../org/jppf/execute/NodeTaskWrapper.java | 81 ++-- .../java/org/jppf/execute/ThreadManager.java | 75 +++- .../async/AbstractAsyncExecutionManager.java | 347 +++++++++++++++ .../execute/async/AsyncExecutionManager.java | 138 ++++++ .../async/ExecutionManagerListener.java | 37 ++ .../execute/async/JobProcessingEntry.java | 89 ++++ .../org/jppf/execute/async/package-info.java | 31 +- .../java/org/jppf/nio/NioMessageReader.java | 2 +- common/src/java/org/jppf/node/Node.java | 6 + .../org/jppf/node/protocol/AbstractTask.java | 31 +- .../jppf/node/protocol/BundleParameter.java | 8 +- .../src/java/org/jppf/node/protocol/Task.java | 16 +- .../org/jppf/node/protocol/TaskBundle.java | 14 + .../protocol/TaskExecutionDispatcher.java | 16 +- .../java/org/jppf/utils/TypedProperties.java | 9 +- .../utils/configuration/JPPFProperties.java | 2 - .../i18n/JPPFProperties.properties | 4 +- demo/config/jppf-driver.properties | 1 + demo/config/log4j-driver.properties | 7 +- demo/config/log4j-node1.properties | 10 +- demo/config/node1.properties | 2 +- demo/config/samples/deadlock.properties | 10 +- demo/logging-gui.log.1 | 0 .../jppf/management/NodeConfigNotifier.java | 11 +- .../NodeConfigNotifierMBeanProvider.java | 4 +- node/src/java/org/jppf/node/AbstractNode.java | 104 ++++- .../java/org/jppf/node/NodeConnection.java | 4 +- node/src/java/org/jppf/node/NodeInternal.java | 47 +- .../jppf/node/event/JobClassPathHandler.java | 2 +- .../jppf/server/node/AbstractCommonNode.java | 202 +++++++-- .../node/AbstractNodeConnectionChecker.java | 56 --- .../org/jppf/server/node/AbstractNodeIO.java | 177 ++------ .../node/AsyncNodeExecutionManager.java | 124 ++++++ .../org/jppf/server/node/JPPFContainer.java | 108 +---- .../java/org/jppf/server/node/JPPFNode.java | 418 +++++++----------- .../server/node/NodeConnectionChecker.java | 65 --- .../server/node/NodeExecutionManager.java | 42 +- .../src/java/org/jppf/server/node/NodeIO.java | 10 +- .../node/ObjectDeserializationTask.java | 111 +++++ .../server/node/ObjectSerializationTask.java | 107 +++++ .../node/NodeListener.java | 2 +- .../org/jppf/management/JPPFDriverAdmin.java | 26 +- .../org/jppf/management/JPPFNodeAdmin.java | 16 +- .../jppf/management/NodeSelectionHelper.java | 47 +- .../ForwardingNotificationDispatcher.java | 4 +- .../ForwardingNotificationManager.java | 16 +- .../forwarding/JPPFNodeForwarding.java | 10 +- .../org/jppf/server/DriverInitializer.java | 2 +- .../org/jppf/server/debug/ServerDebug.java | 6 +- .../job/management/DriverJobManagement.java | 4 +- .../classloader/node/NodeClassNioServer.java | 2 +- .../node/WaitingNodeRequestState.java | 4 +- .../nio/heartbeat/HeartbeatContext.java | 2 +- .../nio/nodeserver/AbstractNodeContext.java | 4 +- .../nodeserver/AbstractTaskQueueChecker.java | 10 +- ...eNodeContext.java => BaseNodeContext.java} | 2 +- .../nio/nodeserver/LocalNodeMessage.java | 2 +- .../NodeConnectionCompletionListener.java | 2 +- .../nio/nodeserver/NodeContextAttributes.java | 5 +- .../nodeserver/NodeDispatchTimeoutAction.java | 4 +- .../nodeserver/NodeJMXWrapperListener.java | 4 +- .../server/nio/nodeserver/NodeNioServer.java | 12 +- .../nodeserver/NodeReservationHandler.java | 14 +- .../nio/nodeserver/NodeServerUtils.java | 4 +- .../nio/nodeserver/PeerAttributesHandler.java | 8 +- .../nio/nodeserver/RemoteNodeMessage.java | 4 +- .../nio/nodeserver/TaskQueueChecker.java | 38 +- .../nodeserver/WaitInitialBundleState.java | 2 +- ...er.java => AbstractAsyncJobScheduler.java} | 25 +- ...eueChecker.java => AsyncJobScheduler.java} | 205 ++++----- .../nodeserver/async/AsyncNodeContext.java | 69 ++- .../async/AsyncNodeContextFuture.java | 2 +- .../async/AsyncNodeMessageHandler.java | 105 +++-- .../async/AsyncNodeMessageWriter.java | 2 +- .../nodeserver/async/AsyncNodeNioServer.java | 36 +- .../node/local/AbstractLocalNodeIO.java | 2 +- .../server/node/local/AsyncLocalNodeIO.java | 14 +- .../server/node/local/JPPFLocalContainer.java | 7 +- .../jppf/server/node/local/JPPFLocalNode.java | 7 +- .../node/local/LocalClassLoaderManager.java | 2 +- .../jppf/server/node/local/LocalNodeIO.java | 10 +- .../node/remote/AbstractRemoteNode.java | 7 +- .../node/remote/JPPFRemoteContainer.java | 14 +- .../node/remote/RemoteClassLoaderManager.java | 20 +- .../remote/RemoteNodeConnectionChecker.java | 142 ------ .../jppf/server/node/remote/RemoteNodeIO.java | 30 +- .../server/protocol/ServerTaskBundleNode.java | 2 +- .../jppf/server/queue/BroadcastManager.java | 15 +- .../jppf/server/queue/JPPFPriorityQueue.java | 52 ++- tests/.classpath | 2 +- .../test/setup/AbstractNonStandardSetup.java | 35 +- .../test/org/jppf/test/setup/BaseSetup.java | 2 +- .../test/org/jppf/test/setup/BaseTest.java | 7 +- .../jppf/test/setup/common/LifeCycleTask.java | 16 +- .../config/classloader/driver.properties | 2 + .../src/resources/config/db/driver.properties | 2 + .../config/discovery/driver.properties | 2 + .../config/driver.template.properties | 2 + .../job_persistence/driver-common.properties | 2 + ...ate.properties => log4j-driver.properties} | 0 .../job_persistence_p2p/driver.properties | 2 + .../job_persistence_p2p/log4j-node.properties | 67 +++ .../job_persistence_p2p_db/driver.properties | 2 + .../log4j-node.properties | 67 +++ .../config/job_reservation/driver.properties | 2 + .../lb_persistence_client/driver.properties | 2 + ...ate.properties => log4j-driver.properties} | 0 .../driver-common.properties | 2 + ...ate.properties => log4j-driver.properties} | 0 .../driver-common.properties | 2 + .../config/localnode/driver.properties | 2 + .../config/log4j-node.template.properties | 16 +- .../resources/config/node.template.properties | 38 +- .../config/offline_node/driver.properties | 2 + .../resources/config/p2p/driver.properties | 1 + .../resources/config/p2p/driver2.properties | 30 -- .../src/resources/config/p2p/node.properties | 35 +- .../config/p2p_many/client.properties | 96 ++++ .../config/p2p_many/driver.properties | 154 +++++++ .../config/p2p_many/log4j-driver.properties | 119 +++++ .../config/p2p_many/log4j-node.properties | 75 ++++ .../resources/config/p2p_many/node.properties | 72 +++ .../config/p2p_orphan/driver.properties | 2 + .../config/persistence/driver.properties | 2 + ...ate.properties => log4j-driver.properties} | 0 .../config/provisioning/driver.properties | 2 + .../serialization/java/driver.properties | 2 + .../serialization/java_lz4/driver.properties | 2 + .../serialization/java_zlib/driver.properties | 2 + .../serialization/jppf/driver.properties | 2 + .../serialization/jppf_lz4/driver.properties | 2 + .../serialization/jppf_zlib/driver.properties | 2 + .../serialization/kryo/driver.properties | 2 + .../serialization/kryo_lz4/driver.properties | 2 + .../serialization/kryo_zlib/driver.properties | 2 + .../serialization/xstream/driver.properties | 2 + .../xstream_lz4/driver.properties | 2 + .../xstream_zlib/driver.properties | 2 + .../resources/config/ssl/driver.properties | 2 + .../resources/config/ssl2/driver.properties | 2 + .../config/ssl2_p2p/driver.properties | 2 + .../resources/config/ssl3/driver.properties | 2 + .../org/jppf/client/TestDefaultPolicies.java | 6 +- .../AbstractJobPersistenceTest.java | 2 +- .../TestDefaultDatabasePersistence.java | 1 - .../TestDefaultDatabasePersistenceAsync.java | 1 - ...stDefaultDatabasePersistenceCacheable.java | 1 - ...aultDatabasePersistenceCacheableAsync.java | 1 - ...DefaultDatabasePersistenceMultiServer.java | 1 - .../TestDefaultFilePersistence.java | 1 - .../TestDefaultFilePersistenceAsync.java | 1 - .../TestDefaultFilePersistenceCacheable.java | 3 +- ...tDefaultFilePersistenceCacheableAsync.java | 3 +- .../TestDefaultFilePersistenceP2P.java | 3 +- ...ultiServerLoadBalancerPersistenceTest.java | 13 +- ...tAsyncDatabaseLoadBalancerPersistence.java | 1 - ...lientAsyncFileLoadBalancerPersistence.java | 1 - ...ClientDatabaseLoadBalancerPersistence.java | 1 - ...TestClientFileLoadBalancerPersistence.java | 1 - ...rAsyncDatabaseLoadBalancerPersistence.java | 1 - ...riverAsyncFileLoadBalancerPersistence.java | 1 - ...DriverDatabaseLoadBalancerPersistence.java | 1 - ...TestDriverFileLoadBalancerPersistence.java | 1 - .../AbstractTestJPPFNodeForwardingMBean.java | 2 +- .../TestJPPFNodeForwardingMBean.java | 31 +- .../persistence/AbstractDatabaseSetup.java | 3 + .../TestJPPFDatasourceFactory.java | 1 + .../org/jppf/server/peer/TestManyServers.java | 120 +++++ .../org/jppf/server/peer/TestMultiServer.java | 2 - .../server/peer/TestMultiServerWithSSL.java | 1 - .../peer/TestMultiServerWithSSLSetup.java | 1 - .../jppf/server/protocol/TestJPPFJobSLA.java | 112 +---- 178 files changed, 3106 insertions(+), 1747 deletions(-) create mode 100644 common/src/java/org/jppf/execute/async/AbstractAsyncExecutionManager.java create mode 100644 common/src/java/org/jppf/execute/async/AsyncExecutionManager.java create mode 100644 common/src/java/org/jppf/execute/async/ExecutionManagerListener.java create mode 100644 common/src/java/org/jppf/execute/async/JobProcessingEntry.java rename server/src/java/org/jppf/server/node/local/LocalNodeConnectionChecker.java => common/src/java/org/jppf/execute/async/package-info.java (61%) create mode 100644 demo/logging-gui.log.1 delete mode 100644 node/src/java/org/jppf/server/node/AbstractNodeConnectionChecker.java create mode 100644 node/src/java/org/jppf/server/node/AsyncNodeExecutionManager.java delete mode 100644 node/src/java/org/jppf/server/node/NodeConnectionChecker.java create mode 100644 node/src/java/org/jppf/server/node/ObjectDeserializationTask.java create mode 100644 node/src/java/org/jppf/server/node/ObjectSerializationTask.java rename server/src/java/org/jppf/server/nio/nodeserver/{AbstractBaseNodeContext.java => BaseNodeContext.java} (98%) rename server/src/java/org/jppf/server/nio/nodeserver/async/{AbstractAsyncTaskQueueChecker.java => AbstractAsyncJobScheduler.java} (88%) rename server/src/java/org/jppf/server/nio/nodeserver/async/{AsyncTaskQueueChecker.java => AsyncJobScheduler.java} (71%) delete mode 100644 server/src/java/org/jppf/server/node/remote/RemoteNodeConnectionChecker.java rename tests/src/resources/config/job_persistence/{log4j-driver.template.properties => log4j-driver.properties} (100%) create mode 100644 tests/src/resources/config/job_persistence_p2p/log4j-node.properties create mode 100644 tests/src/resources/config/job_persistence_p2p_db/log4j-node.properties rename tests/src/resources/config/lb_persistence_client/{log4j-driver.template.properties => log4j-driver.properties} (100%) rename tests/src/resources/config/lb_persistence_driver/{log4j-driver.template.properties => log4j-driver.properties} (100%) create mode 100644 tests/src/resources/config/p2p_many/client.properties create mode 100644 tests/src/resources/config/p2p_many/driver.properties create mode 100644 tests/src/resources/config/p2p_many/log4j-driver.properties create mode 100644 tests/src/resources/config/p2p_many/log4j-node.properties create mode 100644 tests/src/resources/config/p2p_many/node.properties rename tests/src/resources/config/persistence/{log4j-driver.template.properties => log4j-driver.properties} (100%) create mode 100644 tests/src/tests/test/org/jppf/server/peer/TestManyServers.java diff --git a/admin/src/java/org/jppf/ui/monitoring/node/NodeDataPanel.java b/admin/src/java/org/jppf/ui/monitoring/node/NodeDataPanel.java index 501f992a77..f25c725bff 100644 --- a/admin/src/java/org/jppf/ui/monitoring/node/NodeDataPanel.java +++ b/admin/src/java/org/jppf/ui/monitoring/node/NodeDataPanel.java @@ -192,7 +192,8 @@ private synchronized void addNode(final TopologyDriver driverData, final Topolog final DefaultMutableTreeNode nodeNode = TopologyUtils.addNode(model, driverData, nodeData); if (nodeNode != null) { final DefaultMutableTreeNode driverNode = (DefaultMutableTreeNode) nodeNode.getParent(); - if ((driverNode.getChildCount() == 1) && !treeTable.isCollapsed(driverNode)) treeTable.expand(driverNode); + //if ((driverNode.getChildCount() == 1) && !treeTable.isCollapsed(driverNode)) treeTable.expand(driverNode); + if (driverNode.getChildCount() == 1) treeTable.expand(driverNode); } } diff --git a/client/src/java/org/jppf/client/balancer/ChannelWrapperLocal.java b/client/src/java/org/jppf/client/balancer/ChannelWrapperLocal.java index c00aacd271..680ddad556 100644 --- a/client/src/java/org/jppf/client/balancer/ChannelWrapperLocal.java +++ b/client/src/java/org/jppf/client/balancer/ChannelWrapperLocal.java @@ -77,7 +77,7 @@ public class ChannelWrapperLocal extends ChannelWrapper implements ClientConnect */ public ChannelWrapperLocal(final JPPFClient client) { this.client = client; - executionManager = new ClientExecutionManager(JPPFProperties.LOCAL_EXECUTION_THREADS); + executionManager = new ClientExecutionManager(client.getConfig(), JPPFProperties.LOCAL_EXECUTION_THREADS); priority = client.getConfig().get(JPPFProperties.LOCAL_EXECUTION_PRIORITY); systemInfo = new JPPFSystemInformation(client.getConfig(), getConnectionUuid(), true, false); managementInfo = new JPPFManagementInfo("local", "local", -1, getConnectionUuid(), JPPFManagementInfo.NODE | JPPFManagementInfo.LOCAL, false); diff --git a/client/src/java/org/jppf/client/balancer/ClientExecutionManager.java b/client/src/java/org/jppf/client/balancer/ClientExecutionManager.java index 877c4aacca..c6f230cfe3 100644 --- a/client/src/java/org/jppf/client/balancer/ClientExecutionManager.java +++ b/client/src/java/org/jppf/client/balancer/ClientExecutionManager.java @@ -45,10 +45,11 @@ public class ClientExecutionManager extends AbstractExecutionManager { /** * Initialize this execution manager. + * @param config the configuration to get the thread manager properties from. * @param nbThreadsProperty the name of the property which configures the number of threads. */ - public ClientExecutionManager(final JPPFProperty nbThreadsProperty) { - super(nbThreadsProperty); + public ClientExecutionManager(final TypedProperties config, final JPPFProperty nbThreadsProperty) { + super(config, nbThreadsProperty); } /** @@ -58,7 +59,8 @@ public ClientExecutionManager(final JPPFProperty nbThreadsProperty) { */ @Override protected void setup(final TaskBundle bundle, final List> taskList) { - taskNotificationDispatcher.setBundle(this.bundle = bundle); + this.bundle = bundle; + //taskNotificationDispatcher.setBundle(bundle); this.taskList = taskList; this.taskWrapperList = new ArrayList<>(taskList.size()); this.dataProvider = taskList.get(0).getDataProvider(); @@ -84,7 +86,7 @@ protected void cleanup() { this.dataProvider = null; usedClassLoader.dispose(); usedClassLoader = null; - taskNotificationDispatcher.setBundle(this.bundle = null); + //taskNotificationDispatcher.setBundle(this.bundle = null); this.taskList = null; this.uuidList = null; setJobCancelled(false); diff --git a/client/src/java/org/jppf/client/balancer/ClientTaskBundle.java b/client/src/java/org/jppf/client/balancer/ClientTaskBundle.java index 36069f4093..f6291fbd9a 100644 --- a/client/src/java/org/jppf/client/balancer/ClientTaskBundle.java +++ b/client/src/java/org/jppf/client/balancer/ClientTaskBundle.java @@ -230,7 +230,8 @@ public String toString() { /** * @return the id for this bundle. */ - public long getBundleId() { + @Override + public Long getBundleId() { return bundleId; } } diff --git a/common/src/java/org/jppf/execute/AbstractExecutionManager.java b/common/src/java/org/jppf/execute/AbstractExecutionManager.java index 9c355f9739..8a1b9e6711 100644 --- a/common/src/java/org/jppf/execute/AbstractExecutionManager.java +++ b/common/src/java/org/jppf/execute/AbstractExecutionManager.java @@ -53,19 +53,19 @@ public abstract class AbstractExecutionManager implements ExecutionManager { /** * The bundle whose tasks are currently being executed. */ - protected TaskBundle bundle = null; + protected TaskBundle bundle; /** * The list of tasks to execute. */ - protected List> taskList = null; + protected List> taskList; /** * The uuid path of the current bundle. */ - protected List uuidList = null; + protected List uuidList; /** - * Holds a the tasks submitted tot he executor. + * Holds the tasks submitted to the executor. */ - protected List taskWrapperList = null; + protected List taskWrapperList; /** * Dispatches tasks notifications to registered listeners. */ @@ -89,11 +89,11 @@ public abstract class AbstractExecutionManager implements ExecutionManager { /** * The class loader used to load the tasks and the classes they need from the client. */ - protected UsedClassLoader usedClassLoader = null; + protected UsedClassLoader usedClassLoader; /** * The data provider for the current job. */ - protected DataProvider dataProvider = null; + protected DataProvider dataProvider; /** * The total accumulated elapsed time of the tasks in the current bundle. */ @@ -101,48 +101,12 @@ public abstract class AbstractExecutionManager implements ExecutionManager { /** * Initialize this execution manager with the specified node. + * @param config the configuration to get the thread manager properties from. * @param nbThreadsProperty the name of the property which configures the number of threads. */ - public AbstractExecutionManager(final JPPFProperty nbThreadsProperty) { + public AbstractExecutionManager(final TypedProperties config, final JPPFProperty nbThreadsProperty) { taskNotificationDispatcher = new TaskExecutionDispatcher(getClass().getClassLoader()); - int poolSize = JPPFConfiguration.get(nbThreadsProperty); - if (poolSize <= 0) poolSize = Runtime.getRuntime().availableProcessors(); - JPPFConfiguration.set(nbThreadsProperty, poolSize); - log.info("running " + poolSize + " processing thread" + (poolSize > 1 ? "s" : "")); - threadManager = createThreadManager(poolSize); - } - - /** - * Create the thread manager instance. Default is {@link ThreadManagerThreadPool}. - * @param poolSize the initial pool size. - * @return an instance of {@link ThreadManager}. - */ - protected static ThreadManager createThreadManager(final int poolSize) { - ThreadManager result = null; - final TypedProperties config = JPPFConfiguration.getProperties(); - final String s = config.get(JPPFProperties.THREAD_MANAGER_CLASS); - if (!"default".equalsIgnoreCase(s) && !ThreadManagerThreadPool.class.getName().equals(s) && s != null) { - try { - final Class clazz = Class.forName(s); - final Object instance = ReflectionHelper.invokeConstructor(clazz, new Class[]{Integer.TYPE}, poolSize); - if (instance instanceof ThreadManager) { - result = (ThreadManager) instance; - log.info("Using custom thread manager: " + s); - } - } catch(final Exception e) { - log.error(e.getMessage(), e); - } - } - if (result == null) { - log.info("Using default thread manager"); - return new ThreadManagerThreadPool(poolSize); - } - config.set(JPPFProperties.PROCESSING_THREADS, result.getPoolSize()); - log.info("Node running " + poolSize + " processing thread" + (poolSize > 1 ? "s" : "")); - final boolean cpuTimeEnabled = result.isCpuTimeEnabled(); - config.setBoolean("cpuTimeSupported", cpuTimeEnabled); - log.info("Thread CPU time measurement is " + (cpuTimeEnabled ? "" : "not ") + "supported"); - return result; + threadManager = ThreadManager.newInstance(config, nbThreadsProperty); } @Override @@ -159,7 +123,7 @@ public void execute(final TaskBundle bundle, final List> taskList) throw for (final Task task : taskList) { if (!(task instanceof JPPFExceptionResult)) { if (task instanceof AbstractTask) ((AbstractTask) task).setExecutionDispatcher(taskNotificationDispatcher); - final NodeTaskWrapper taskWrapper = new NodeTaskWrapper(task, usedClassLoader.getClassLoader(), timeoutHandler, threadManager.isCpuTimeEnabled()); + final NodeTaskWrapper taskWrapper = new NodeTaskWrapper(task, usedClassLoader.getClassLoader(), timeoutHandler); taskWrapperList.add(taskWrapper); ecs.submit(taskWrapper, taskWrapper); count++; @@ -172,11 +136,6 @@ public void execute(final TaskBundle bundle, final List> taskList) throw final Future future = ecs.take(); if (!future.isCancelled()) { final NodeTaskWrapper taskWrapper = future.get(); - final JPPFReconnectionNotification notif = taskWrapper.getReconnectionNotification(); - if (notif != null) { - cancelAllTasks(true, false); - throw notif; - } taskEnded(taskWrapper); } } catch (final Exception e) { diff --git a/common/src/java/org/jppf/execute/ExecutionManager.java b/common/src/java/org/jppf/execute/ExecutionManager.java index d8f4f6edc8..98e8ed5fe7 100644 --- a/common/src/java/org/jppf/execute/ExecutionManager.java +++ b/common/src/java/org/jppf/execute/ExecutionManager.java @@ -57,7 +57,7 @@ public interface ExecutionManager { /** * Get the executor used by this execution manager. - * @return an ExecutorService instance. + * @return an {@code ExecutorService} instance. */ ExecutorService getExecutor(); @@ -93,13 +93,13 @@ public interface ExecutionManager { /** * Determine whether the current job has been cancelled, including before starting its execution. - * @return true if the job has been cancelled, false otherwise. + * @return {@code true} if the job has been cancelled, {@code false} otherwise. */ boolean isJobCancelled(); /** * Specify whether the current job has been cancelled, including before starting its execution. - * @param jobCancelled true if the job has been cancelled, false otherwise. + * @param jobCancelled {@code true} if the job has been cancelled, {@code false} otherwise. */ void setJobCancelled(boolean jobCancelled); diff --git a/common/src/java/org/jppf/execute/NodeTaskWrapper.java b/common/src/java/org/jppf/execute/NodeTaskWrapper.java index 766172794b..0ec74da943 100644 --- a/common/src/java/org/jppf/execute/NodeTaskWrapper.java +++ b/common/src/java/org/jppf/execute/NodeTaskWrapper.java @@ -18,8 +18,9 @@ package org.jppf.execute; import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; -import org.jppf.JPPFReconnectionNotification; +import org.jppf.execute.async.JobProcessingEntry; import org.jppf.node.protocol.*; import org.jppf.scheduling.*; import org.jppf.utils.ExceptionUtils; @@ -37,11 +38,11 @@ public class NodeTaskWrapper implements Runnable { /** * Logger for this class. */ - private static Logger log = LoggerFactory.getLogger(NodeTaskWrapper.class); + private static final Logger log = LoggerFactory.getLogger(NodeTaskWrapper.class); /** * Determines whether the debug level is enabled in the log configuration, without the cost of a method call. */ - private static boolean traceEnabled = log.isTraceEnabled(); + private static final boolean traceEnabled = log.isTraceEnabled(); /** * Timer managing the tasks timeout. */ @@ -49,19 +50,19 @@ public class NodeTaskWrapper implements Runnable { /** * Indicator whether task was cancelled; */ - private boolean cancelled = false; + private boolean cancelled; /** * Indicator whether onCancel should be called when cancelled. */ - private boolean callOnCancel = false; + private boolean callOnCancel; /** * Indicator whether task timeout. */ - private boolean timeout = false; + private boolean timeout; /** * Indicator that task was started. */ - private boolean started = false; + private boolean started; /** * The task to execute within a try/catch block. */ @@ -70,15 +71,11 @@ public class NodeTaskWrapper implements Runnable { /** * The future created by the executor service. */ - private Future future = null; - /** - * - */ - private JPPFReconnectionNotification reconnectionNotification = null; + private Future future; /** * Holds the used cpu time for this task. */ - private ExecutionInfo executionInfo = null; + private ExecutionInfo executionInfo; /** * The elapsed time for this task's execution. */ @@ -87,18 +84,37 @@ public class NodeTaskWrapper implements Runnable { * The class loader that was used to load the task class. */ private final ClassLoader taskClassLoader; + /** + * Encapsulates information about the job the task is a part of. + */ + private final JobProcessingEntry jobEntry; + /** + * Whether this task has ended. + */ + private final AtomicBoolean ended = new AtomicBoolean(false); + + /** + * Initialize this task wrapper with a specified JPPF task. + * @param task the task to execute within a try/catch block. + * @param taskClassLoader the class loader that was used to load the task class. + * @param timeoutHandler handles the timeout for this task. + */ + public NodeTaskWrapper(final Task task, final ClassLoader taskClassLoader, final JPPFScheduleHandler timeoutHandler) { + this(null, task, taskClassLoader, timeoutHandler); + } /** * Initialize this task wrapper with a specified JPPF task. + * @param jobEntry encapsulates information about the job the task is a part of. * @param task the task to execute within a try/catch block. * @param taskClassLoader the class loader that was used to load the task class. * @param timeoutHandler handles the timeout for this task. - * @param cpuTimeEnabled whether cpu time is supported/enabled. */ - public NodeTaskWrapper(final Task task, final ClassLoader taskClassLoader, final JPPFScheduleHandler timeoutHandler, final boolean cpuTimeEnabled) { + public NodeTaskWrapper(final JobProcessingEntry jobEntry, final Task task, final ClassLoader taskClassLoader, final JPPFScheduleHandler timeoutHandler) { this.task = task; this.taskClassLoader = taskClassLoader; this.timeoutHandler = timeoutHandler; + this.jobEntry = jobEntry; } /** @@ -145,8 +161,7 @@ synchronized void timeout() { */ @SuppressWarnings("unchecked") @Override - public void run() - { + public void run() { if (traceEnabled) log.trace(toString()); started = true; final long id = Thread.currentThread().getId(); @@ -158,8 +173,6 @@ public void run() Thread.currentThread().setContextClassLoader(taskClassLoader); executionInfo = CpuTimeCollector.computeExecutionInfo(id); if (!isCancelledOrTimedout()) task.run(); - } catch(final JPPFReconnectionNotification t) { - reconnectionNotification = t; } catch(final Throwable t) { task.setThrowable(t); if (t instanceof UnsatisfiedLinkError) task.setResult(ExceptionUtils.getStackTrace(t)); @@ -169,20 +182,18 @@ public void run() try { elapsedTime = System.nanoTime() - startTime; if (executionInfo != null) executionInfo = CpuTimeCollector.computeExecutionInfo(id).subtract(executionInfo); - } catch(final JPPFReconnectionNotification t) { - if (reconnectionNotification == null) reconnectionNotification = t; - } catch(@SuppressWarnings("unused") final Throwable ignore) { + } catch(final Throwable e) { + if (traceEnabled) log.trace("error in finally of {}", this, e); } try { silentTimeout(); silentCancel(); - } catch(final JPPFReconnectionNotification t) { - if (reconnectionNotification == null) reconnectionNotification = t; } catch (final Throwable t) { task.setThrowable(t); } if (task.getThrowable() instanceof InterruptedException) task.setThrowable(null); cancelTimeoutAction(); + taskEnded(); } } @@ -264,14 +275,6 @@ public void setFuture(final Future future) { this.future = future; } - /** - * Get the reconnection notification thrown by the atysk execution, if any. - * @return a {@link JPPFReconnectionNotification} or null. - */ - JPPFReconnectionNotification getReconnectionNotification() { - return reconnectionNotification; - } - /** * Remove the specified future from the pending set and notify * all threads waiting for the end of the execution. @@ -295,4 +298,18 @@ public ExecutionInfo getExecutionInfo() { public long getElapsedTime() { return elapsedTime; } + + /** + * @return the object which encapsulates information about the job the task is a part of. + */ + public JobProcessingEntry getJobEntry() { + return jobEntry; + } + + /** + * called when a task terminates. + */ + public void taskEnded() { + if (ended.compareAndSet(false, true)) jobEntry.executionManager.taskEnded(this); + } } diff --git a/common/src/java/org/jppf/execute/ThreadManager.java b/common/src/java/org/jppf/execute/ThreadManager.java index 08f80af059..7ef62168d9 100644 --- a/common/src/java/org/jppf/execute/ThreadManager.java +++ b/common/src/java/org/jppf/execute/ThreadManager.java @@ -20,13 +20,16 @@ import java.util.concurrent.*; +import org.jppf.utils.*; +import org.jppf.utils.configuration.*; +import org.slf4j.*; + /** * Interface for all thread managers. * @author Laurent Cohen * @exclude */ -public interface ThreadManager -{ +public interface ThreadManager { /** * Set the size of the node's thread pool. * @param size the size as an int. @@ -41,14 +44,14 @@ public interface ThreadManager /** * Computes the total CPU time used by the execution threads. - * @return a NodeExecutionInfo instance. + * @return a {@code NodeExecutionInfo} instance. */ ExecutionInfo computeExecutionInfo(); /** * Computes the CPU time used by thread identified by threadID. * @param threadID the thread ID. - * @return a NodeExecutionInfo instance. + * @return a {@code NodeExecutionInfo} instance. */ ExecutionInfo computeExecutionInfo(final long threadID); @@ -83,10 +86,58 @@ public interface ThreadManager */ boolean isCpuTimeEnabled(); + /** + * Create the thread manager instance. Default is {@link ThreadManagerThreadPool}. + * @param config the configuration to get the thread manager properties from. + * @param nbThreadsProperty the name of the property which configures the number of threads. + * @return an instance of {@link ThreadManager}. + */ + static ThreadManager newInstance(final TypedProperties config, JPPFProperty nbThreadsProperty) { + final Logger log = LoggerFactory.getLogger(ThreadManager.class); + ThreadManager result = null; + final int poolSize = computePoolSize(config, nbThreadsProperty); + config.set(nbThreadsProperty, poolSize); + final String s = config.get(JPPFProperties.THREAD_MANAGER_CLASS); + if (!"default".equalsIgnoreCase(s) && !ThreadManagerThreadPool.class.getName().equals(s) && s != null) { + try { + final Class clazz = Class.forName(s); + final Object instance = ReflectionHelper.invokeConstructor(clazz, new Class[]{Integer.TYPE}, poolSize); + if (instance instanceof ThreadManager) { + result = (ThreadManager) instance; + log.info("Using custom thread manager: {}", s); + } + } catch(final Exception e) { + log.error(e.getMessage(), e); + } + } + if (result == null) { + log.info("Using default thread manager"); + return new ThreadManagerThreadPool(poolSize); + } + //config.set(JPPFProperties.PROCESSING_THREADS, result.getPoolSize()); + log.info("Node running {} processing thread{}", poolSize, poolSize > 1 ? "s" : ""); + final boolean cpuTimeEnabled = result.isCpuTimeEnabled(); + config.setBoolean("cpuTimeSupported", cpuTimeEnabled); + log.info("Thread CPU time measurement is {}supported", cpuTimeEnabled ? "" : "not "); + return result; + } + + /** + * Compute a pool size based on the specified configuration and size property. + * @param config the config to read the rpoerty from. + * @param nbThreadsProperty the property that configures the pool size. + * @return the computed size. + */ + static int computePoolSize(final TypedProperties config, JPPFProperty nbThreadsProperty) { + int poolSize = config.get(nbThreadsProperty); + if (poolSize <= 0) poolSize = Runtime.getRuntime().availableProcessors(); + return poolSize; + } + /** * Use class loader in this thread manager. - * @param classLoader a ClassLoader instance. - * @return a UsedClassLoader instance. Never return null. + * @param classLoader a {@code ClassLoader} instance. + * @return a {@code UsedClassLoader} instance. Never return {@code null}. */ UsedClassLoader useClassLoader(final ClassLoader classLoader); @@ -94,25 +145,23 @@ public interface ThreadManager * Helper class for managing used class loaders. * @exclude */ - public static abstract class UsedClassLoader - { + public static abstract class UsedClassLoader { /** - * A ClassLoader instance. + * A {@code ClassLoader} instance. */ private final ClassLoader classLoader; /** * - * @param classLoader a ClassLoader instance. + * @param classLoader a {@code ClassLoader} instance. */ - protected UsedClassLoader(final ClassLoader classLoader) - { + protected UsedClassLoader(final ClassLoader classLoader) { this.classLoader = classLoader; } /** * Get a class loader instance. - * @return a ClassLoader instance. + * @return a {@code ClassLoader} instance. */ public ClassLoader getClassLoader() { return classLoader; diff --git a/common/src/java/org/jppf/execute/async/AbstractAsyncExecutionManager.java b/common/src/java/org/jppf/execute/async/AbstractAsyncExecutionManager.java new file mode 100644 index 0000000000..4cd0cd9aad --- /dev/null +++ b/common/src/java/org/jppf/execute/async/AbstractAsyncExecutionManager.java @@ -0,0 +1,347 @@ +/* + * JPPF. + * Copyright (C) 2005-2018 JPPF Team. + * http://www.jppf.org + * + * Licensed 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.jppf.execute.async; + +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.*; + +import org.jppf.JPPFReconnectionNotification; +import org.jppf.execute.*; +import org.jppf.node.protocol.*; +import org.jppf.scheduling.JPPFScheduleHandler; +import org.jppf.utils.*; +import org.jppf.utils.collections.*; +import org.jppf.utils.configuration.*; +import org.slf4j.*; + +/** + * Instances of this class manage the execution of JPPF tasks by a node. + * @author Laurent Cohen + * @author Martin JANDA + * @author Paul Woodward + * @exclude + */ +public abstract class AbstractAsyncExecutionManager implements AsyncExecutionManager { + /** + * Logger for this class. + */ + private static final Logger log = LoggerFactory.getLogger(AbstractAsyncExecutionManager.class); + /** + * Determines whether the debug level is enabled in the log configuration, without the cost of a method call. + */ + private static final boolean debugEnabled = LoggingUtils.isDebugEnabled(log); + /** + * Determines whether the trace level is enabled in the log configuration, without the cost of a method call. + */ + private static final boolean traceEnabled = log.isTraceEnabled(); + /** + * Timer managing the tasks timeout. + */ + protected final JPPFScheduleHandler timeoutHandler = new JPPFScheduleHandler("Task Timeout Timer"); + /** + * Dispatches tasks notifications to registered listeners. + */ + protected final TaskExecutionDispatcher taskNotificationDispatcher; + /** + * Determines whether the number of threads or their priority has changed. + */ + protected final AtomicBoolean configChanged = new AtomicBoolean(true); + /** + * Set if the node must reconnect to the driver. + */ + protected AtomicReference reconnectionNotification = new AtomicReference<>(null); + /** + * The thread manager that is used for execution. + */ + protected final ThreadManager threadManager; + /** + * Mapping of jobUuid + bunldeId to the corresponding {@code JobProcessingEntry} objects. + */ + protected final Map jobEntries = new HashMap<>(); + /** + * Mapping of job uuids to the ids of the bundles currently processed for this job. + */ + protected final CollectionMap jobBundleIds = new ArrayListHashMap<>(); + /** + * List of listeners to this execution manager. + */ + protected final List listeners = new CopyOnWriteArrayList<>(); + + /** + * Initialize this execution manager with the specified node. + * @param config the configuration to get the thread manager properties from. + * @param nbThreadsProperty the name of the property which configures the number of threads. + */ + public AbstractAsyncExecutionManager(final TypedProperties config, final JPPFProperty nbThreadsProperty) { + taskNotificationDispatcher = new TaskExecutionDispatcher(getClass().getClassLoader()); + threadManager = ThreadManager.newInstance(config, nbThreadsProperty); + } + + @Override + public void execute(final TaskBundle bundle, final List> taskList) throws Exception { + if ((taskList == null) || taskList.isEmpty()) return; + if (debugEnabled) log.debug("executing {} tasks of bundle {}", taskList.size(), bundle); + final JobProcessingEntry jobEntry = setup(bundle, taskList); + jobEntry.executionManager = this; + synchronized(jobEntries) { + jobEntries.put(bundle.getUuid() + bundle.getBundleId(), jobEntry); + jobBundleIds.putValue(bundle.getUuid(), bundle.getBundleId()); + } + if (!jobEntry.jobCancelled.get()) { + synchronized(jobEntry.taskWrapperList) { + if (debugEnabled) log.debug("wrapping up to {} executable tasks of bundle {}", taskList.size(), bundle); + for (final Task task : taskList) { + if (!(task instanceof JPPFExceptionResult)) { + if (task instanceof AbstractTask) ((AbstractTask) task).setExecutionDispatcher(taskNotificationDispatcher); + final NodeTaskWrapper taskWrapper = new NodeTaskWrapper(jobEntry, task, jobEntry.getClassLoader(), timeoutHandler); + jobEntry.taskWrapperList.add(taskWrapper); + jobEntry.submittedCount++; + } + } + if (debugEnabled) log.debug("submitting {} executable tasks of bundle {}", jobEntry.taskWrapperList.size(), bundle); + for (final NodeTaskWrapper taskWrapper: jobEntry.taskWrapperList) getExecutor().submit(taskWrapper, taskWrapper); + if (jobEntry.submittedCount <= 0) jobEnded(jobEntry); + } + } + } + + @Override + public void cancelAllTasks(final boolean callOnCancel, final boolean requeue) { + if (debugEnabled) log.debug("cancelling all tasks with: callOnCancel={}, requeue={}", callOnCancel, requeue); + synchronized(jobEntries) { + for (final JobProcessingEntry jobEntry: jobEntries.values()) { + if (requeue) { + synchronized(jobEntry.bundle) { + jobEntry.bundle.setRequeue(true); + jobEntry.bundle.getSLA().setSuspended(true); + } + } + if (jobEntry.taskWrapperList != null) { + synchronized(jobEntry.taskWrapperList) { + for (final NodeTaskWrapper ntw: jobEntry.taskWrapperList) cancelTask(ntw, callOnCancel); + } + } + } + } + } + + @Override + public void cancelJob(final String jobUuid, final boolean callOnCancel, final boolean requeue) { + if (debugEnabled) log.debug("cancelling all tasks with: callOnCancel={}, requeue={}, jobUuid={}", callOnCancel, requeue, jobUuid); + synchronized(jobEntries) { + final Collection bundleIds = jobBundleIds.getValues(jobUuid); + if (debugEnabled) log.debug("cancelling {} bundles for jobUuid={}", (bundleIds == null) ? 0: bundleIds.size(), jobUuid); + if (bundleIds == null) return; + final List bundleIdList = new ArrayList<>(bundleIds); + for (final long bundleId: bundleIdList) { + final JobProcessingEntry jobEntry = jobEntries.get(jobUuid + bundleId); + if (jobEntry == null) continue; + jobEntry.jobCancelled.set(true); + if (debugEnabled) log.debug("cancelling {}", jobEntry.bundle); + if (requeue) { + synchronized(jobEntry.bundle) { + jobEntry.bundle.setRequeue(true); + jobEntry.bundle.getSLA().setSuspended(true); + } + } + if (jobEntry.taskWrapperList != null) { + synchronized(jobEntry.taskWrapperList) { + for (final NodeTaskWrapper taskWrapper: jobEntry.taskWrapperList) cancelTask(taskWrapper, callOnCancel); + } + } + } + } + } + + /** + * Cancel the execution of the tasks with the specified id. + * @param taskWrapper the index of the task to cancel. + * @param callOnCancel determines whether the onCancel() callback method of each task should be invoked. + */ + private static void cancelTask(final NodeTaskWrapper taskWrapper, final boolean callOnCancel) { + if (debugEnabled) log.debug("cancelling task = {}", taskWrapper); + final Future future = taskWrapper.getFuture(); + if (!future.isDone()) { + if (debugEnabled) log.debug("calling future.cancel(true) for task = {}", taskWrapper); + taskWrapper.cancel(callOnCancel); + future.cancel(taskWrapper.getTask().isInterruptible()); + taskWrapper.cancelTimeoutAction(); + taskWrapper.taskEnded(); + } + } + + @Override + public void shutdown() { + if (debugEnabled) log.debug("closing {}", this); + getExecutor().shutdownNow(); + timeoutHandler.clear(true); + taskNotificationDispatcher.close(); + } + + /** + * Prepare this execution manager for executing the tasks of a bundle. + * @param bundle the bundle whose tasks are to be executed. + * @param taskList the list of tasks to execute. + * @return an instance of {@link JobProcessingEntry}. + */ + protected abstract JobProcessingEntry setup(final TaskBundle bundle, final List> taskList); + + /** + * Cleanup method invoked when all tasks for the current bundle have completed. + * @param jobEntry encapsulates information about the job. + */ + protected abstract void cleanup(JobProcessingEntry jobEntry); + + @Override + public void taskEnded(final NodeTaskWrapper taskWrapper) { + try { + if (traceEnabled) log.trace("task ended: {}", taskWrapper); + final long elapsedTime = taskWrapper.getElapsedTime(); + final TaskBundle bundle; + final int n, submittedCount; + final JobProcessingEntry jobEntry = taskWrapper.getJobEntry(); + synchronized(jobEntry) { + bundle = jobEntry.bundle; + jobEntry.accumulatedElapsed.addAndGet(elapsedTime); + n = jobEntry.resultCount.incrementAndGet(); + submittedCount = jobEntry.submittedCount; + } + final ExecutionInfo info = taskWrapper.getExecutionInfo(); + final long cpuTime = (info == null) ? 0L : (info.cpuTime / 1_000_000L); + final Task task = taskWrapper.getTask(); + if (traceEnabled) log.trace("sending task ended notification for {}, bundle={}", taskWrapper, bundle); + taskNotificationDispatcher.fireTaskEnded(task, bundle.getUuid(), bundle.getName(), cpuTime, elapsedTime / 1_000_000L, task.getThrowable() != null); + if (traceEnabled) log.trace("resultCount={} for {}", n, taskWrapper); + if (n >= submittedCount) jobEnded(jobEntry); + } catch (final RuntimeException e) { + log.error("error in taskEnded() for {}", taskWrapper, e); + } + } + + /** + * + * @param jobEntry the job to process. + */ + private void jobEnded(final JobProcessingEntry jobEntry) { + final TaskBundle bundle = jobEntry.bundle; + final List> taskList = jobEntry.taskList; + if (debugEnabled) log.debug("processing completion of {} tasks of job {}", taskList.size(), bundle); + synchronized(jobEntries) { + jobEntries.remove(bundle.getUuid() + bundle.getBundleId()); + jobBundleIds.removeValue(bundle.getUuid(), bundle.getBundleId()); + } + cleanup(jobEntry); + fireJobFinished(bundle, taskList, jobEntry.t); + } + + @Override + public ExecutorService getExecutor() { + return threadManager.getExecutorService(); + } + + @Override + public boolean checkConfigChanged() { + return configChanged.compareAndSet(true, false); + } + + @Override + public void triggerConfigChanged() { + configChanged.compareAndSet(false, true); + } + + @Override + public void setThreadPoolSize(final int size) { + if (size <= 0) { + log.warn("ignored attempt to set the thread pool size to 0 or less: " + size); + return; + } + final int oldSize = getThreadPoolSize(); + threadManager.setPoolSize(size); + final int newSize = getThreadPoolSize(); + if (oldSize != newSize) { + log.info("Node thread pool size changed from " + oldSize + " to " + size); + JPPFConfiguration.set(JPPFProperties.PROCESSING_THREADS, size); + triggerConfigChanged(); + } + } + + @Override + public int getThreadPoolSize() { + return threadManager.getPoolSize(); + } + + @Override + public int getThreadsPriority() { + return threadManager.getPriority(); + } + + @Override + public void updateThreadsPriority(final int newPriority) { + threadManager.setPriority(newPriority); + } + + @Override + public ThreadManager getThreadManager() { + return threadManager; + } + + @Override + public TaskExecutionDispatcher getTaskNotificationDispatcher() { + return taskNotificationDispatcher; + } + + @Override + public List getBundles(final String jobUuid) { + final List result = new ArrayList<>(); + synchronized(jobEntries) { + final Collection bundleIds = jobBundleIds.getValues(jobUuid); + if (bundleIds != null) { + for (final Long id: bundleIds) { + final JobProcessingEntry jobEntry = jobEntries.get(jobUuid + id); + if (jobEntry != null) result.add(jobEntry.bundle); + } + } + } + return result; + } + + @Override + public void addExecutionManagerListener(final ExecutionManagerListener listener) { + if (listener != null) listeners.add(listener); + } + + @Override + public void removeExecutionManagerListener(final ExecutionManagerListener listener) { + if (listener != null) listeners.remove(listener); + } + + /** + * Called when the execution of a task bundle has finished. + * @param bundle the TaskBundle which holds information on the job. + * @param tasks the tasks that were executed. + * @param t a {@link Throwable} that prevented or interrupted the job processing. + */ + protected void fireJobFinished(final TaskBundle bundle, final List> tasks, final Throwable t) { + if (debugEnabled) log.debug("sending notification to listeners for completion of {} tasks of job {}", tasks.size(), bundle); + for (final ExecutionManagerListener listener: listeners) { + if (listener != null) listener.bundleExecuted(bundle, tasks, t); + } + } +} diff --git a/common/src/java/org/jppf/execute/async/AsyncExecutionManager.java b/common/src/java/org/jppf/execute/async/AsyncExecutionManager.java new file mode 100644 index 0000000000..286dfd1a83 --- /dev/null +++ b/common/src/java/org/jppf/execute/async/AsyncExecutionManager.java @@ -0,0 +1,138 @@ +/* + * JPPF. + * Copyright (C) 2005-2018 JPPF Team. + * http://www.jppf.org + * + * Licensed 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.jppf.execute.async; + +import java.util.List; +import java.util.concurrent.ExecutorService; + +import org.jppf.execute.*; +import org.jppf.node.protocol.*; + +/** + * Instances of this interface manage the execution of JPPF tasks by a node. + * @author Laurent Cohen + * @exclude + */ +public interface AsyncExecutionManager { + /** + * Execute the specified tasks of the specified tasks bundle. + * @param bundle the bundle to which the tasks are associated. + * @param taskList the list of tasks to execute. + * @throws Exception if the execution failed. + */ + void execute(TaskBundle bundle, List> taskList) throws Exception; + + /** + * Cancel all executing or pending tasks. + * @param callOnCancel determines whether the onCancel() callback method of each task should be invoked. + * @param requeue true if the job should be requeued on the server side, false otherwise. + */ + void cancelAllTasks(boolean callOnCancel, boolean requeue); + + /** + * Cancel all executing or pending tasks for the specified job. + * @param jobUuid the uuid of the job to cancel. + * @param callOnCancel determines whether the onCancel() callback method of each task should be invoked. + * @param requeue true if the job should be requeued on the server side, false otherwise. + */ + void cancelJob(String jobUuid, boolean callOnCancel, boolean requeue); + + /** + * Shutdown this execution manager. + */ + void shutdown(); + + /** + * Get the executor used by this execution manager. + * @return an {@code ExecutorService} instance. + */ + ExecutorService getExecutor(); + + /** + * Set the size of the node's thread pool. + * @param size the size as an int. + */ + void setThreadPoolSize(int size); + + /** + * Get the size of the node's thread pool. + * @return the size as an int. + */ + int getThreadPoolSize(); + + /** + * Get the priority assigned to the execution threads. + * @return the priority as an int value. + */ + int getThreadsPriority(); + + /** + * Update the priority of all execution threads. + * @param newPriority the new priority to set. + */ + void updateThreadsPriority(int newPriority); + + /** + * Get the thread manager for this node. + * @return a {@link ThreadManager} instance. + */ + ThreadManager getThreadManager(); + + /** + * Get the object which dispatches tasks notifications to registered listeners. + * @return a {@link TaskExecutionDispatcher} instance. + */ + TaskExecutionDispatcher getTaskNotificationDispatcher(); + + /** + * Determines whether the configuration has changed and resets the flag if it has. + * @return true if the config was changed, false otherwise. + */ + boolean checkConfigChanged(); + + /** + * Trigger the configuration changed flag. + */ + void triggerConfigChanged(); + + /** + * Get a list of bundle with the specified job uuid currently being processed. + * @param jobUuid the uuid of the job for which to retrieve the bundles. + * @return a list of {@link TaskBundle} instances, possibly empty but never null. + */ + List getBundles(final String jobUuid); + + /** + * Register a listener with this execution manager. + * @param listener the listener to register. + */ + void addExecutionManagerListener(final ExecutionManagerListener listener); + + /** + * Remove a listener from the registered listeners. + * @param listener the listener to remove. + */ + void removeExecutionManagerListener(final ExecutionManagerListener listener); + + /** + * Notification that a task has finished executing. + * @param taskWrapper the task that finished its execution. + */ + void taskEnded(NodeTaskWrapper taskWrapper); +} diff --git a/common/src/java/org/jppf/execute/async/ExecutionManagerListener.java b/common/src/java/org/jppf/execute/async/ExecutionManagerListener.java new file mode 100644 index 0000000000..68c3ef5f48 --- /dev/null +++ b/common/src/java/org/jppf/execute/async/ExecutionManagerListener.java @@ -0,0 +1,37 @@ +/* + * JPPF. + * Copyright (C) 2005-2018 JPPF Team. + * http://www.jppf.org + * + * Licensed 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.jppf.execute.async; + +import java.util.List; + +import org.jppf.node.protocol.*; + +/** + * + * @author Laurent Cohen + */ +public interface ExecutionManagerListener { + /** + * Called when the execution of a task bundle has finished. + * @param bundle the TaskBundle which holds information on the job. + * @param tasks the tasks that were executed. + * @param t a {@link Throwable} that prevented or interrupted the job processing. + */ + void bundleExecuted(final TaskBundle bundle, final List> tasks, final Throwable t); +} diff --git a/common/src/java/org/jppf/execute/async/JobProcessingEntry.java b/common/src/java/org/jppf/execute/async/JobProcessingEntry.java new file mode 100644 index 0000000000..a405e9269c --- /dev/null +++ b/common/src/java/org/jppf/execute/async/JobProcessingEntry.java @@ -0,0 +1,89 @@ +/* + * JPPF. + * Copyright (C) 2005-2018 JPPF Team. + * http://www.jppf.org + * + * Licensed 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.jppf.execute.async; + +import java.util.List; +import java.util.concurrent.atomic.*; + +import org.jppf.execute.NodeTaskWrapper; +import org.jppf.execute.ThreadManager.UsedClassLoader; +import org.jppf.node.protocol.*; + +/** + * + * @author Laurent Cohen + * @exclude + */ +public class JobProcessingEntry { + /** + * The bundle whose tasks are currently being executed. + */ + public TaskBundle bundle; + /** + * The list of tasks to execute. + */ + public List> taskList; + /** + * The uuid path of the current bundle. + */ + public List uuidList; + /** + * Holds the tasks submitted to the executor. + */ + public List taskWrapperList; + /** + * Determines whether the current job has been cancelled. + */ + public AtomicBoolean jobCancelled = new AtomicBoolean(false); + /** + * The class loader used to load the tasks and the classes they need from the client. + */ + public UsedClassLoader usedClassLoader; + /** + * The data provider for the current job. + */ + public DataProvider dataProvider; + /** + * The total accumulated elapsed time of the tasks in the current bundle. + */ + public final AtomicLong accumulatedElapsed = new AtomicLong(0L); + /** + * The execution mabager that processes the job. + */ + public AsyncExecutionManager executionManager; + /** + * The number of submitted tasks. + */ + public int submittedCount; + /** + * The number of completed tasks. + */ + public final AtomicInteger resultCount = new AtomicInteger(0); + /** + * A {@link Throwable} that prevented or interrupted the job processing. + */ + public Throwable t; + + /** + * @return the class loader fot htis task bundle. + */ + public ClassLoader getClassLoader() { + return (usedClassLoader == null) ? null : usedClassLoader.getClassLoader(); + } +} diff --git a/server/src/java/org/jppf/server/node/local/LocalNodeConnectionChecker.java b/common/src/java/org/jppf/execute/async/package-info.java similarity index 61% rename from server/src/java/org/jppf/server/node/local/LocalNodeConnectionChecker.java rename to common/src/java/org/jppf/execute/async/package-info.java index de1f0d934b..a37bb0c408 100644 --- a/server/src/java/org/jppf/server/node/local/LocalNodeConnectionChecker.java +++ b/common/src/java/org/jppf/execute/async/package-info.java @@ -16,35 +16,8 @@ * limitations under the License. */ -package org.jppf.server.node.local; - -import org.jppf.server.node.AbstractNodeConnectionChecker; - /** - * This class does nothing. - * @author Laurent Cohen + * Interfaces for the new load-balancing execution on the client and server side. * @exclude */ -public class LocalNodeConnectionChecker extends AbstractNodeConnectionChecker -{ - @Override - public void start() - { - } - - @Override - public void stop() - { - } - - @Override - public void resume() - { - } - - @Override - public void suspend() - { - } - -} +package org.jppf.execute.async; diff --git a/common/src/java/org/jppf/nio/NioMessageReader.java b/common/src/java/org/jppf/nio/NioMessageReader.java index 429de678f1..b9c586869a 100644 --- a/common/src/java/org/jppf/nio/NioMessageReader.java +++ b/common/src/java/org/jppf/nio/NioMessageReader.java @@ -36,7 +36,7 @@ public abstract class NioMessageReader { */ private static final boolean debugEnabled = log.isDebugEnabled(); /** - * The server handling the connections. + * The server handling the connections.d */ protected final StatelessNioServer server; diff --git a/common/src/java/org/jppf/node/Node.java b/common/src/java/org/jppf/node/Node.java index 06653322b5..5ecb09da31 100644 --- a/common/src/java/org/jppf/node/Node.java +++ b/common/src/java/org/jppf/node/Node.java @@ -99,4 +99,10 @@ public interface Node extends Runnable { * @return a {@link JPPFManagementInfo} object. */ JPPFManagementInfo getManagementInfo(); + + /** + * Get the total number of tasks executed. + * @return the number of tasks as an int. + */ + int getExecutedTaskCount(); } diff --git a/common/src/java/org/jppf/node/protocol/AbstractTask.java b/common/src/java/org/jppf/node/protocol/AbstractTask.java index d57cbc7ddc..2d39c194f2 100644 --- a/common/src/java/org/jppf/node/protocol/AbstractTask.java +++ b/common/src/java/org/jppf/node/protocol/AbstractTask.java @@ -56,36 +56,36 @@ public class AbstractTask implements Task { /** * The result of the task execution. */ - private T result = null; + private T result; /** * The Throwable that was raised by this task's execution. */ - private Throwable throwable = null; + private Throwable throwable; /** * The provider of shared data for this task. */ - private transient DataProvider dataProvider = null; + private transient DataProvider dataProvider; /** * The task timeout schedule configuration. */ - private JPPFSchedule timeoutSchedule = null; + private JPPFSchedule timeoutSchedule; /** * Determines whether this task is executing within a node, or locally on the client side. */ - private transient boolean inNode = false; + private transient boolean inNode; /** * A user-assigned id for this task. */ - private String id = null; + private String id; /** * Dispatches notifications from this task. */ - private transient TaskExecutionDispatcher executionDisptacher = null; + private transient TaskExecutionDispatcher executionDisptacher; /** * Whether this task should be resubmitted by the server. * @since 4.1 */ - private transient boolean resubmit = false; + private transient boolean resubmit; /** * The max number of times a task can resubmit itself. */ @@ -94,6 +94,10 @@ public class AbstractTask implements Task { * The node in which this task is executing, if any. */ private transient Node node; + /** + * The job this task is a part of. + */ + private transient JPPFDistributedJob job; /** * @@ -311,4 +315,15 @@ public Task setNode(final Node node) { this.node = node; return this; } + + @Override + public JPPFDistributedJob getJob() { + return job; + } + + @Override + public Task setJob(final JPPFDistributedJob job) { + this.job = job; + return this; + } } diff --git a/common/src/java/org/jppf/node/protocol/BundleParameter.java b/common/src/java/org/jppf/node/protocol/BundleParameter.java index 56f128ef99..483ea55c6a 100644 --- a/common/src/java/org/jppf/node/protocol/BundleParameter.java +++ b/common/src/java/org/jppf/node/protocol/BundleParameter.java @@ -76,6 +76,10 @@ public enum BundleParameter { * Parameter the total accumulated task execution elapsed time in a bundle. */ NODE_BUNDLE_ELAPSED_PARAM, + /** + * Maxium number of concurrent jobs. + */ + NODE_MAX_JOBS, /** * Flag indicating whether the node is in offline mode. */ @@ -85,9 +89,9 @@ public enum BundleParameter { */ NODE_OFFLINE_OPEN_REQUEST, /** - * Id of the task bundle sent to the node. + * Id of the task bundle sent to the node or local client executor. */ - NODE_BUNDLE_ID, + TASK_BUNDLE_ID, /** * Uuid of a job executed offline. */ diff --git a/common/src/java/org/jppf/node/protocol/Task.java b/common/src/java/org/jppf/node/protocol/Task.java index edcc199dea..f7d1b1015d 100644 --- a/common/src/java/org/jppf/node/protocol/Task.java +++ b/common/src/java/org/jppf/node/protocol/Task.java @@ -226,5 +226,19 @@ public interface Task extends Runnable, Serializable, Interruptibility { * @return this task, for method chaining. * @exclude */ - public Task setNode(final Node node); + Task setNode(final Node node); + + /** + * Get the job this task is a part of. + * @return the job as an instance of {@link JPPFDistributedJob}. + */ + JPPFDistributedJob getJob(); + + /** + * set the job this task is a part of. + * @param job the job as an instance of {@link JPPFDistributedJob}. + * @return this task, for method call chaining. + * @exclude + */ + Task setJob(JPPFDistributedJob job); } diff --git a/common/src/java/org/jppf/node/protocol/TaskBundle.java b/common/src/java/org/jppf/node/protocol/TaskBundle.java index 33b7c9491b..4ef6c06834 100644 --- a/common/src/java/org/jppf/node/protocol/TaskBundle.java +++ b/common/src/java/org/jppf/node/protocol/TaskBundle.java @@ -164,4 +164,18 @@ public interface TaskBundle extends JPPFDistributedJob, Metadata { * @param handshake true if this bundle is a handshake bundle, false otherwise. */ void setHandshake(boolean handshake); + + /** + * @return the id of the task bundle sent to the node or local client executor. + */ + default Long getBundleId() { + return getParameter(BundleParameter.TASK_BUNDLE_ID); + } + + /** + * @param id the id of the task bundle sent to the node or local client executor. + */ + default void setBundleId(Long id) { + setParameter(BundleParameter.TASK_BUNDLE_ID, id); + } } diff --git a/common/src/java/org/jppf/node/protocol/TaskExecutionDispatcher.java b/common/src/java/org/jppf/node/protocol/TaskExecutionDispatcher.java index 1138da07c0..0af369774d 100644 --- a/common/src/java/org/jppf/node/protocol/TaskExecutionDispatcher.java +++ b/common/src/java/org/jppf/node/protocol/TaskExecutionDispatcher.java @@ -35,10 +35,6 @@ public class TaskExecutionDispatcher { * List of listeners to task execution events. */ private final List taskExecutionListeners = new CopyOnWriteArrayList<>(); - /** - * The bundle whose tasks are currently being executed. - */ - private TaskBundle bundle = null; /** * Class loader used to discover listeners via SPI. */ @@ -90,7 +86,9 @@ public void fireTaskEnded(final Task task, final String jobId, final String j * @param sendViaJmx if true then also send this notification via the JMX MBean, otherwise only send to local listeners. */ public void fireTaskNotification(final Task task, final Object userObject, final boolean sendViaJmx) { - final TaskExecutionEvent event = (bundle == null) ? new TaskExecutionEvent(task, null, null, userObject, sendViaJmx) + final TaskBundle bundle = (TaskBundle) task.getJob(); + final TaskExecutionEvent event = (bundle == null) + ? new TaskExecutionEvent(task, null, null, userObject, sendViaJmx) : new TaskExecutionEvent(task, bundle.getUuid(), bundle.getName(), userObject, sendViaJmx); fireEvent(event); } @@ -112,14 +110,6 @@ private void fireEvent(final TaskExecutionEvent event) { public void close() { } - /** - * Set the bundle whose tasks are currently being executed. - * @param bundle a {@link TaskBundle} instance. - */ - public void setBundle(final TaskBundle bundle) { - this.bundle = bundle; - } - /** * Register all listeners discovered via SPI. */ diff --git a/common/src/java/org/jppf/utils/TypedProperties.java b/common/src/java/org/jppf/utils/TypedProperties.java index dad882a120..6194a116ab 100644 --- a/common/src/java/org/jppf/utils/TypedProperties.java +++ b/common/src/java/org/jppf/utils/TypedProperties.java @@ -506,10 +506,11 @@ public T remove(final JPPFProperty property) { * @since 5.2 */ public boolean containsProperty(final JPPFProperty property) { - if (getProperty(property.getName()) != null) return true; - if (property.getAliases() != null) { - for (String name: property.getAliases()) { - if (getProperty(name) != null) return true; + if (containsKey(property.getName())) return true; + final String[] aliases = property.getAliases(); + if (aliases != null) { + for (final String name: aliases) { + if (containsKey(name)) return true; } } return false; diff --git a/common/src/java/org/jppf/utils/configuration/JPPFProperties.java b/common/src/java/org/jppf/utils/configuration/JPPFProperties.java index 239bbc71f2..f10cdde378 100644 --- a/common/src/java/org/jppf/utils/configuration/JPPFProperties.java +++ b/common/src/java/org/jppf/utils/configuration/JPPFProperties.java @@ -408,8 +408,6 @@ public class JPPFProperties { public static final JPPFProperty JAVA_PATH = new StringProperty("jppf.java.path", null); /** Path to the temporary config overrides properties file. */ public static final JPPFProperty CONFIG_OVERRIDES_PATH = new FileProperty("jppf.config.overrides.path", new File("config/config-overrides.properties")); - /** @exclude . */ - public static final JPPFProperty NODE_CHECK_CONNECTION = new BooleanProperty("jppf.node.check.connection", false); /** The default thickness of the scrollbars in the GUI. */ public static final JPPFProperty DEFAULT_SCROLLBAR_THICKNESS = new IntProperty("jppf.ui.default.scrollbar.thickness", 10); /** Whether a node is idle. This property is only set within a server. */ diff --git a/common/src/resources/org/jppf/utils/configuration/i18n/JPPFProperties.properties b/common/src/resources/org/jppf/utils/configuration/i18n/JPPFProperties.properties index c447b381ad..9f3ca219a0 100644 --- a/common/src/resources/org/jppf/utils/configuration/i18n/JPPFProperties.properties +++ b/common/src/resources/org/jppf/utils/configuration/i18n/JPPFProperties.properties @@ -176,8 +176,6 @@ jppf.nio.thread.ttl.doc = TTL in milliseconds for threads beyond the core numbe jppf.nio.thread.ttl.tags = common, nio jppf.node.android.doc = Whether the node is an Android node jppf.node.android.tags = node -jppf.node.check.connection.doc = internal use -jppf.node.check.connection.tags = node, internal jppf.node.class.doc = Internal use. The class of node to instantiate upon node startup. For instance Java and Android nodes use a different class jppf.node.class.tags = node, internal jppf.node.classloading.batch.period.doc = How often batched class loading requests are sent to the server @@ -189,7 +187,7 @@ jppf.node.idle.tags = driver jppf.node.management.port.doc = Node management port (to distinguish from server management port when local node is on) jppf.node.management.port.tags = driver, node jppf.node.max.jobs.doc = Maximum number of jobs that can be handled concurrently by a node -jppf.node.max.jobs.tags = node +jppf.node.max.jobs.tags = driver, node jppf.node.offline.doc = Whether the node runs in offline mode jppf.node.offline.tags = node jppf.node.provisioning.master.doc = Whether the node is a master node diff --git a/demo/config/jppf-driver.properties b/demo/config/jppf-driver.properties index 4534eddebd..b9a007a886 100644 --- a/demo/config/jppf-driver.properties +++ b/demo/config/jppf-driver.properties @@ -19,6 +19,7 @@ jppf.jmxremote.nio.servers.pool.size = 1 jppf.async.node = true +#jppf.test.job.removal = true #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # diff --git a/demo/config/log4j-driver.properties b/demo/config/log4j-driver.properties index ea53186f65..3110490cfb 100644 --- a/demo/config/log4j-driver.properties +++ b/demo/config/log4j-driver.properties @@ -90,9 +90,10 @@ log4j.logger.org.jppf.nio.PlainNioObject=INFO #log4j.logger.org.jppf.nio.SSLHandlerImpl=TRACE #log4j.logger.org.jppf.nio.SSLHandlerImpl2=TRACE #log4j.logger.org.jppf.nio.SSLNioObject=TRACE -log4j.logger.org.jppf.nio.StateTransitionTask=DEBUG +#log4j.logger.org.jppf.nio.StateTransitionTask=DEBUG #log4j.logger.org.jppf.nio.ChannelSelectorThread=DEBUG #log4j.logger.org.jppf.nio.NioServer=DEBUG +log4j.logger.org.jppf.nio.NioMessageReader=DEBUG #log4j.logger.org.jppf.nio.StatelessNioServer=TRACE #log4j.logger.org.jppf.nio.acceptor=DEBUG #log4j.logger.org.jppf.nio.acceptor.IdentifyingPeerState=DEBUG @@ -136,7 +137,7 @@ log4j.logger.org.jppf.server.nio.classloader.client=INFO #log4j.logger.org.jppf.server.nio.client.async=DEBUG #log4j.logger.org.jppf.server.nio.client.async.AsyncClientNioServer=TRACE #log4j.logger.org.jppf.server.nio.heartbeat=DEBUG -#log4j.logger.org.jppf.server.nio.nodeserver=DEBUG +log4j.logger.org.jppf.server.nio.nodeserver=DEBUG #log4j.logger.org.jppf.server.nio.nodeserver.AbstractNodeContext=DEBUG #log4j.logger.org.jppf.server.nio.nodeserver.NodeNioServer=DEBUG #log4j.logger.org.jppf.server.nio.nodeserver.PeerAttributesHandler=DEBUG @@ -165,7 +166,7 @@ log4j.logger.org.jppf.server.nio.classloader.client=INFO #log4j.logger.org.jppf.server.protocol.ServerTaskBundleClient=DEBUG #log4j.logger.org.jppf.server.protocol.ServerTaskBundleNode=DEBUG #log4j.logger.org.jppf.server.protocol.ServerTask=TRACE -#log4j.logger.org.jppf.server.queue=DEBUG +log4j.logger.org.jppf.server.queue=DEBUG #log4j.logger.org.jppf.server.queue.JPPFPriorityQueue=DEBUG #log4j.logger.org.jppf.server.queue.BroadcastJobCompletionListener=DEBUG #log4j.logger.org.jppf.server.queue.PersistenceHandler=DEBUG diff --git a/demo/config/log4j-node1.properties b/demo/config/log4j-node1.properties index 493f28dc1b..b2f3172f38 100644 --- a/demo/config/log4j-node1.properties +++ b/demo/config/log4j-node1.properties @@ -49,7 +49,7 @@ log4j.logger.org.jppf.comm.discovery=INFO #log4j.logger.org.jppf.comm.interceptor=DEBUG #log4j.logger.org.jppf.comm.recovery=DEBUG log4j.logger.org.jppf.comm.socket=INFO -#log4j.logger.org.jppf.execute=DEBUG +log4j.logger.org.jppf.execute=DEBUG #log4j.logger.org.jppf.execute.NodeTaskWrapper=TRACE #log4j.logger.org.jppf.io=DEBUG #log4j.logger.org.jppf.io.IOHelper=DEBUG @@ -64,13 +64,13 @@ log4j.logger.org.jppf.jmxremote.nio.JMXNioServer=INFO #log4j.logger.org.jppf.management.NodeStatusNotifier=TRACE #log4j.logger.org.jppf.management.JMXServerImpl=INFO #log4j.logger.org.jppf.management.diagnostics=DEBUG -log4j.logger.org.jppf.management.spi=DEBUG +#log4j.logger.org.jppf.management.spi=DEBUG #log4j.logger.org.jppf.management.spi.JPPFMBeanProviderManager=DEBUG #log4j.logger.org.jppf.nio=DEBUG log4j.logger.org.jppf.nio.PlainNioObject=INFO #log4j.logger.org.jppf.nio.SelectorSynchronizerLock=TRACE #log4j.logger.org.jppf.nio.StateTransitionTask=DEBUG -#log4j.logger.org.jppf.node=DEBUG +log4j.logger.org.jppf.node=DEBUG #log4j.logger.org.jppf.node.idle=DEBUG #log4j.logger.org.jppf.node.NodeRunner=DEBUG #log4j.logger.org.jppf.node.connection=DEBUG @@ -78,13 +78,13 @@ log4j.logger.org.jppf.nio.PlainNioObject=INFO #log4j.logger.org.jppf.node.connection.AbstractCsvConnectionStrategy=DEBUG #log4j.logger.org.jppf.node.event=DEBUG #log4j.logger.org.jppf.node.event.JobClassPathHandler=DEBUG -log4j.logger.org.jppf.node.provisioning=DEBUG +#log4j.logger.org.jppf.node.provisioning=DEBUG #log4j.logger.org.jppf.node.provisioning.SlaveNodeManager=DEBUG #log4j.logger.org.jppf.persistence=DEBUG #log4j.logger.org.jppf.process=DEBUG #log4j.logger.org.jppf.scheduling=DEBUG #log4j.logger.org.jppf.serialization=DEBUG -#log4j.logger.org.jppf.server.node=DEBUG +log4j.logger.org.jppf.server.node=DEBUG #log4j.logger.org.jppf.server.node.AbstractNodeIO=DEBUG #log4j.logger.org.jppf.server.node.JPPFContainer=DEBUG #log4j.logger.org.jppf.server.node.JPPFNode=DEBUG diff --git a/demo/config/node1.properties b/demo/config/node1.properties index 18dc3f576b..04be5ad4ae 100644 --- a/demo/config/node1.properties +++ b/demo/config/node1.properties @@ -181,7 +181,7 @@ jppf.node.provisioning.slave.path.prefix = slave_nodes/node_1_ #jppf.node.provisioning.slave.config.dir jppf.node.provisioning.slave.config.path = config/slave #jppf.node.provisioning.slave.jvm.options = -Dlog4j.configuration=config/log4j-slave.properties -jppf.node.provisioning.startup.slaves = 9 +jppf.node.provisioning.startup.slaves = 0 jppf.node.provisioning.startup.overrides.file = ${jppf.node.provisioning.slave.config.path}/node-config-overrides.properties #a.homepath = ${env.HOMEPATH} diff --git a/demo/config/samples/deadlock.properties b/demo/config/samples/deadlock.properties index dda6913689..6d10234839 100644 --- a/demo/config/samples/deadlock.properties +++ b/demo/config/samples/deadlock.properties @@ -7,12 +7,12 @@ minute = 60000 hour = 3600000 #display = $s{ java.lang.System.out.println("second = ${second}, minute = ${minute}, hour = ${hour}"); }$ -deadlock.concurrencyLimit = 20 +deadlock.concurrencyLimit = 5 deadlock.clientConnections = 1 -deadlock.streamDuration = $s{ 120 * ${second} }$ -#deadlock.nbJobs = 1000 -deadlock.tasksPerJob = 1000 -deadlock.taskDuration = 5 +#deadlock.streamDuration = $s{ 120 * ${second} }$ +deadlock.nbJobs = 10 +deadlock.tasksPerJob = 100 +deadlock.taskDuration = 100 deadlock.useCPU = false # < 0 means no provisioning deadlock.slaveNodes = -1 diff --git a/demo/logging-gui.log.1 b/demo/logging-gui.log.1 new file mode 100644 index 0000000000..e69de29bb2 diff --git a/node/src/java/org/jppf/management/NodeConfigNotifier.java b/node/src/java/org/jppf/management/NodeConfigNotifier.java index 562c183557..1402a3531f 100644 --- a/node/src/java/org/jppf/management/NodeConfigNotifier.java +++ b/node/src/java/org/jppf/management/NodeConfigNotifier.java @@ -33,22 +33,17 @@ public class NodeConfigNotifier extends NotificationBroadcasterSupport implement * Explicit serialVersionUID. */ private static final long serialVersionUID = 1L; - /** - * Singleton instance of this class. - */ - private static final NodeConfigNotifier instance = new NodeConfigNotifier(); /** * AN incrementing sequence number. */ private static final AtomicLong sequence = new AtomicLong(0L); /** - * Get the singleton instance of this class. - * @return a {@code NodeThreadsNotifier} instance. + * */ - public static NodeConfigNotifier getInstance() { - return instance; + public NodeConfigNotifier() { } + /** * Send a notification of changes in the configuration. diff --git a/node/src/java/org/jppf/management/NodeConfigNotifierMBeanProvider.java b/node/src/java/org/jppf/management/NodeConfigNotifierMBeanProvider.java index a16d80a5bf..3a06fa9bdb 100644 --- a/node/src/java/org/jppf/management/NodeConfigNotifierMBeanProvider.java +++ b/node/src/java/org/jppf/management/NodeConfigNotifierMBeanProvider.java @@ -19,7 +19,7 @@ package org.jppf.management; import org.jppf.management.spi.JPPFNodeMBeanProvider; -import org.jppf.node.Node; +import org.jppf.node.*; /** * @@ -38,6 +38,6 @@ public String getMBeanName() { @Override public Object createMBean(final Node node) { - return NodeConfigNotifier.getInstance(); + return ((NodeInternal) node).getNodeConfigNotifier(); } } diff --git a/node/src/java/org/jppf/node/AbstractNode.java b/node/src/java/org/jppf/node/AbstractNode.java index f266a40361..716c96c075 100644 --- a/node/src/java/org/jppf/node/AbstractNode.java +++ b/node/src/java/org/jppf/node/AbstractNode.java @@ -27,7 +27,8 @@ import org.jppf.management.*; import org.jppf.node.event.LifeCycleEventHandler; import org.jppf.serialization.*; -import org.jppf.utils.NetworkUtils; +import org.jppf.server.node.NodeIO; +import org.jppf.utils.*; import org.jppf.utils.concurrent.ThreadSynchronization; import org.jppf.utils.configuration.JPPFProperties; import org.slf4j.*; @@ -83,13 +84,47 @@ public abstract class AbstractNode extends ThreadSynchronization implements Node * The main node class loader. */ private AbstractJPPFClassLoader jppfClassLoader; + /** + * The object responsible for this node's I/O. + * @exclude + */ + protected NodeIO nodeIO; + /** + * The configuration of this node. + * @exclude + */ + protected final TypedProperties configuration; + /** + * Determines whether JMX management and monitoring is enabled for this node. + * @exclude + */ + protected boolean jmxEnabled; + /** + * Determines whether this node can execute .Net tasks. + * @exclude + */ + protected final boolean dotnetCapable; + /** + * Handles the firing of node life cycle events and the listeners that subscribe to these events. + * @exclude + */ + protected LifeCycleEventHandler lifeCycleEventHandler; + /** + * The jmx server that handles administration and monitoring functions for this node. + * @exclude + */ + protected JMXServer jmxServer; /** * Initialize this node. * @param uuid this node's uuid. + * @param configuration the configuration of this node. */ - public AbstractNode(final String uuid) { + public AbstractNode(final String uuid, final TypedProperties configuration) { this.uuid = uuid; + this.configuration = configuration; + jmxEnabled = configuration.get(JPPFProperties.MANAGEMENT_ENABLED); + dotnetCapable = configuration.get(JPPFProperties.DOTNET_BRIDGE_INITIALIZED); } /** @@ -101,11 +136,8 @@ public NodeConnection getNodeConnection() { return nodeConnection; } - /** - * Get the total number of tasks executed. - * @return the number of tasks as an int. - */ - public int getTaskCount() { + @Override + public int getExecutedTaskCount() { synchronized (taskCountLock) { return taskCount; } @@ -116,7 +148,7 @@ public int getTaskCount() { * @param taskCount the number of tasks as an int. * @exclude */ - public void setTaskCount(final int taskCount) { + public void setExecutedTaskCount(final int taskCount) { synchronized (taskCountLock) { this.taskCount = taskCount; } @@ -131,17 +163,6 @@ public SerializationHelper getHelper() { return helper; } - /** - * Default implementation - * @return this method always returns null. - * @see org.jppf.node.NodeInternal#getLifeCycleEventHandler() - * @exclude - */ - @Override - public LifeCycleEventHandler getLifeCycleEventHandler() { - return null; - } - /** * {@inheritDoc} *

This implementation throws a JPPFUnsupportedOperationException. It is up to subclasses to implement it. @@ -174,7 +195,6 @@ public JPPFSystemInformation getSystemInformation() { * This implementation does nothing. * @param params not used. * @return {@code null}. - * @exclude */ @Override public AbstractJPPFClassLoader resetTaskClassLoader(final Object...params) { @@ -189,6 +209,7 @@ public boolean isAndroid() { /** * Determine whether this node is currently shutting down. * @return an {@link AtomicBoolean} instance whose value is {@code true if the node is shutting down, false} otherwise. + * @exclude */ public AtomicBoolean getShuttingDown() { return shuttingDown; @@ -228,4 +249,47 @@ public JPPFManagementInfo getManagementInfo() { } return null; } + + @Override + public TypedProperties getConfiguration() { + return configuration; + } + + /** + * Determines whether JMX management and monitoring is enabled for this node. + * @return true if JMX is enabled, false otherwise. + * @exclude + */ + protected boolean isJmxEnabled() { + return jmxEnabled && !isOffline(); + } + + @Override + public boolean isDotnetCapable() { + return dotnetCapable; + } + + /** + * @exclude + */ + @Override + public LifeCycleEventHandler getLifeCycleEventHandler() { + return lifeCycleEventHandler; + } + + @Override + public boolean isMasterNode() { + return !isOffline() && (systemInformation != null) && systemInformation.getJppf().get(JPPFProperties.PROVISIONING_MASTER); + } + + @Override + public boolean isSlaveNode() { + return (systemInformation != null) && systemInformation.getJppf().get(JPPFProperties.PROVISIONING_SLAVE); + } + + @Override + public String getMasterNodeUuid() { + if (systemInformation == null) return null; + return systemInformation.getJppf().get(JPPFProperties.PROVISIONING_MASTER_UUID); + } } diff --git a/node/src/java/org/jppf/node/NodeConnection.java b/node/src/java/org/jppf/node/NodeConnection.java index 686f821b25..62f44040fa 100644 --- a/node/src/java/org/jppf/node/NodeConnection.java +++ b/node/src/java/org/jppf/node/NodeConnection.java @@ -25,8 +25,7 @@ * @author Laurent Cohen * @exclude */ -public interface NodeConnection -{ +public interface NodeConnection { /** * Initialize this connection. * @throws Exception if any error occurs. @@ -51,4 +50,3 @@ public interface NodeConnection */ C getChannel(); } - diff --git a/node/src/java/org/jppf/node/NodeInternal.java b/node/src/java/org/jppf/node/NodeInternal.java index 05ff98d6da..530247f4ca 100644 --- a/node/src/java/org/jppf/node/NodeInternal.java +++ b/node/src/java/org/jppf/node/NodeInternal.java @@ -18,43 +18,80 @@ package org.jppf.node; +import org.jppf.JPPFUnsupportedOperationException; import org.jppf.execute.ExecutionManager; -import org.jppf.management.JMXServer; +import org.jppf.execute.async.AsyncExecutionManager; +import org.jppf.management.*; import org.jppf.node.event.LifeCycleEventHandler; +import org.jppf.server.node.AbstractClassLoaderManager; /** * Internal interface for methods of a node that shoudln't be exposed in the public API. * @author Laurent Cohen - * @exclude */ public interface NodeInternal extends Node { /** * Get the connection used by this node. * @return a {@link NodeConnection} instance. + * @exclude */ NodeConnection getNodeConnection(); /** * Stop this node and release the resources it is using. + * @exclude */ void stopNode(); /** * Get the object that manages the node life cycle events. * @return a {@link LifeCycleEventHandler} instance. + * @exclude */ LifeCycleEventHandler getLifeCycleEventHandler(); /** * Get the task execution manager for this node. - * @return a NodeExecutionManager instance. + * @return a {@link ExecutionManager} instance. + * @exclude */ - ExecutionManager getExecutionManager(); + AsyncExecutionManager getExecutionManager(); /** * Get the JMX connector server associated with the node. + *

The default implementation throws a {@link JPPFUnsupportedOperationException}. It is up to concrete implementations to override it. * @return a JMXServer instance. * @throws Exception if any error occurs. + * @exclude */ - JMXServer getJmxServer() throws Exception; + default JMXServer getJmxServer() throws Exception { + throw new JPPFUnsupportedOperationException("getJmxServer() is not supported on this type of node"); + } + + /** + * Initialize this node's data channel. + * @throws Exception if an error is raised during initialization. + * @exclude + */ + void initDataChannel() throws Exception; + + /** + * Initialize this node's data channel. + * @throws Exception if an error is raised during initialization. + * @exclude + */ + void closeDataChannel() throws Exception; + + /** + * Get the service that manages the class loaders and how they are used. + * @return an {@link AbstractClassLoaderManager} instance. + * @exclude + */ + AbstractClassLoaderManager getClassLoaderManager(); + + /** + * @return the mbean which sends notifications of configuration changes. + * @exclude + */ + NodeConfigNotifier getNodeConfigNotifier(); } diff --git a/node/src/java/org/jppf/node/event/JobClassPathHandler.java b/node/src/java/org/jppf/node/event/JobClassPathHandler.java index 257d113b2f..71d8ba4e70 100644 --- a/node/src/java/org/jppf/node/event/JobClassPathHandler.java +++ b/node/src/java/org/jppf/node/event/JobClassPathHandler.java @@ -57,7 +57,7 @@ public void jobHeaderLoaded(final NodeLifeCycleEvent event) { if (node.isAndroid()) return; if (log.isTraceEnabled()) log.trace(StringUtils.printClassLoaderHierarchy(event.getTaskClassLoader())); AbstractJPPFClassLoader cl = event.getTaskClassLoader(); - if (classpath.isForceClassLoaderReset() || !classpath.isEmpty()) cl = (AbstractJPPFClassLoader) node.resetTaskClassLoader(); + if (classpath.isForceClassLoaderReset() || !classpath.isEmpty()) cl = (AbstractJPPFClassLoader) node.resetTaskClassLoader(event.getJob()); if (!classpath.isEmpty()) { for (final ClassPathElement elt: classpath) { boolean validated = false; diff --git a/node/src/java/org/jppf/server/node/AbstractCommonNode.java b/node/src/java/org/jppf/server/node/AbstractCommonNode.java index 45e981885a..88984e6f82 100644 --- a/node/src/java/org/jppf/server/node/AbstractCommonNode.java +++ b/node/src/java/org/jppf/server/node/AbstractCommonNode.java @@ -18,19 +18,26 @@ package org.jppf.server.node; +import static org.jppf.utils.configuration.JPPFProperties.MANAGEMENT_PORT_NODE; + import java.lang.reflect.*; -import java.util.List; +import java.util.*; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicBoolean; import org.jppf.JPPFReconnectionNotification; import org.jppf.classloader.AbstractJPPFClassLoader; -import org.jppf.execute.ExecutionManager; -import org.jppf.management.JMXServer; +import org.jppf.execute.ThreadManager; +import org.jppf.execute.async.AsyncExecutionManager; +import org.jppf.management.*; +import org.jppf.management.spi.JPPFMBeanProviderManager; +import org.jppf.nio.*; import org.jppf.node.*; import org.jppf.node.protocol.*; import org.jppf.startup.JPPFNodeStartupSPI; import org.jppf.utils.*; -import org.jppf.utils.concurrent.ThreadSynchronization; +import org.jppf.utils.concurrent.*; +import org.jppf.utils.configuration.*; import org.jppf.utils.hooks.*; import org.slf4j.*; @@ -59,22 +66,18 @@ public abstract class AbstractCommonNode extends AbstractNode { AtomicBoolean cacheResetFlag = new AtomicBoolean(false); /** * Flag indicating whether a node shutdown or restart has been requested. - * @since 5.0 */ final AtomicBoolean shutdownRequestFlag = new AtomicBoolean(false); /** * Flag indicating whether it is a shutdown or restart that was last requested. - * @since 5.0 */ final AtomicBoolean restart = new AtomicBoolean(false); /** * Determines whetehr the node is currently processing tasks. - * @since 5.0 */ - boolean executing = false; + boolean executing; /** * Flag indicating whether the node is suspended, i.e. it is still alive but has stopped taking on new jobs. - * @since 5.2 */ final AtomicBoolean suspended = new AtomicBoolean(false); /** @@ -83,7 +86,6 @@ public abstract class AbstractCommonNode extends AbstractNode { final ThreadSynchronization suspendedLock = new ThreadSynchronization(); /** * Flag indicating whether the node is suspended, i.e. it is still alive but has stopped taking on new jobs. - * @since 5.2 */ final AtomicBoolean reading = new AtomicBoolean(false); /** @@ -93,18 +95,32 @@ public abstract class AbstractCommonNode extends AbstractNode { /** * The task execution manager for this node. */ - ExecutionManager executionManager; + AsyncExecutionManager executionManager; + /** + * The executor for serialization and deserialization of the tasks. + */ + final ExecutorService serializationExecutor; /** * Whether this node was sdtarted from {@code NodeRunner.main()} (standalone) or not (embedded). */ boolean startedFromMain; + /** + * The default node's management MBean. + */ + private JPPFNodeAdminMBean nodeAdmin; + /** + * Manager for the MBean defined through the service provider interface. + */ + JPPFMBeanProviderManager providerManager; /** * Initialize this node. * @param uuid this node's uuid. + * @param configuration the configuration of this node. */ - public AbstractCommonNode(final String uuid) { - super(uuid); + public AbstractCommonNode(final String uuid, final TypedProperties configuration) { + super(uuid, configuration); + serializationExecutor = Executors.newFixedThreadPool(ThreadManager.computePoolSize(configuration, JPPFProperties.PROCESSING_THREADS), new JPPFThreadFactory("NodeSerializer")); } /** @@ -129,11 +145,17 @@ protected void setupBundleParameters(final TaskBundle bundle) { /** * Get the main classloader for the node. This method performs a lazy initialization of the classloader. * @return a ClassLoader used for loading the classes of the framework. + * @exclude */ public AbstractJPPFClassLoader getClassLoader() { return classLoaderManager.getClassLoader(); } + @Override + public boolean isOffline() { + return isAndroid() || getClassLoader().isOffline(); + } + /** * Set the main classloader for the node. * @param cl the class loader to set. @@ -171,6 +193,7 @@ void clearResourceCachesIfRequested() { * Request a reset of the class loaders resource caches. * This method merely sets a floag, the actual reset will * be performed at the next opportunity, when it is safe to do so. + * @exclude */ public void requestResourceCacheReset() { cacheResetFlag.compareAndSet(false, true); @@ -180,7 +203,6 @@ public void requestResourceCacheReset() { * Request that the node be shut down or restarted when it is no longer executing tasks. * @param restart {@code true} to restart the node, {@code false} to shut it down. * @return {@code true} if the node had no pending action and the request succeeded, {@code false} otherwise. - * @since 5.0 * @exclude */ public boolean requestShutdown(final boolean restart) { @@ -193,7 +215,6 @@ public boolean requestShutdown(final boolean restart) { /** * Cancel a previous deferred shutdown or restart request, if any. * @return {@code true} if the node has a pending action and it was cancelled, {@code false} otherwise. - * @since 5.0 * @exclude */ public boolean cancelShutdownRequest() { @@ -203,7 +224,6 @@ public boolean cancelShutdownRequest() { /** * Determine whether a node shurdown or restart was requested.. * @return {@code true} if a shudown or restart was requested, {@code false} otherwise. - * @since 5.0 * @exclude */ public boolean isShutdownRequested() { @@ -213,7 +233,6 @@ public boolean isShutdownRequested() { /** * Determine whether a restart or shutdown was requested. * @return {@code true} if a restart was requested, false if a {@code shutdown} was requested. - * @since 5.0 * @exclude */ public boolean isRestart() { @@ -223,7 +242,6 @@ public boolean isRestart() { /** * Determine whether the node is currently processing tasks. * @return {@code true} if the node is processing tasks, {@code false} otherwise. - * @since 5.0 * @exclude */ public boolean isExecuting() { @@ -233,7 +251,6 @@ public boolean isExecuting() { /** * Specifiy whether the node is currently processing tasks. * @param executing {@code true} to specify that the node is processing tasks, {@code false} otherwise. - * @since 5.0 * @exclude */ public void setExecuting(final boolean executing) { @@ -243,7 +260,6 @@ public void setExecuting(final boolean executing) { /** * Determine whether the node is suspended, i.e. it is still alive but has stopped taking on new jobs. * @return {@code true} if the node is suspended, {@code false} otherwise. - * @since 5.2 * @exclude */ public boolean isSuspended() { @@ -253,7 +269,6 @@ public boolean isSuspended() { /** * Set the node's suspended state, i.e. whether it should sto taking on new jobs. * @param suspended {@code true} to suspend the node, {@code false} otherwise. - * @since 5.2 * @exclude */ public void setSuspended(final boolean suspended) { @@ -261,44 +276,30 @@ public void setSuspended(final boolean suspended) { if (!suspended) suspendedLock.wakeUp(); } - /** - * Determine the node's reading state. - * @return {@code true} set the node in reading mode, {@code false} otherwise. - * @since 5.2 - * @exclude - */ - public boolean isReading() { - return reading.get(); - } - - /** - * Set the node's reading state. - * @param suspended {@code true} set the node in reading mode, {@code false} otherwise. - * @since 5.2 - * @exclude - */ - public void setReading(final boolean suspended) { - this.reading.set(suspended); - } - /** * Get the service that manages the class loaders and how they are used. * @return an {@link AbstractClassLoaderManager} instance. * @exclude */ + @Override public AbstractClassLoaderManager getClassLoaderManager() { return classLoaderManager; } + /** + * @exclude + */ @Override - public ExecutionManager getExecutionManager() { + public AsyncExecutionManager getExecutionManager() { return executionManager; } @Override public AbstractJPPFClassLoader resetTaskClassLoader(final Object...params) { - final TaskBundle bundle = executionManager.getBundle(); - if (bundle == null) return null; + if (debugEnabled) log.debug("using params = {}", Arrays.toString(params)); + if ((params == null) || (params.length <= 0)) return null; + if (!(params[0] instanceof JPPFDistributedJob)) return null; + final TaskBundle bundle = (TaskBundle) params[0]; try { final List uuidPath = bundle.getUuidPath().getList(); final boolean remoteClassLoadingDisabled = classLoaderManager.getContainer(uuidPath, params).getClassLoader().isRemoteClassLoadingDisabled(); @@ -319,7 +320,7 @@ void initStartups() { for (final HookInstance hookInstance: hook.getInstances()) { final JPPFNodeStartupSPI instance = hookInstance.getInstance(); final Method m = ReflectionUtils.getSetter(instance.getClass(), "setNode"); - if ((m != null) &&(Node.class.isAssignableFrom(m.getParameterTypes()[0]))) { + if ((m != null) && (Node.class.isAssignableFrom(m.getParameterTypes()[0]))) { try { m.invoke(instance, this); } catch (final IllegalAccessException | IllegalArgumentException | InvocationTargetException e) { @@ -345,4 +346,115 @@ public boolean isStartedFromMain() { public void setStartedFromMain(final boolean startedFromMain) { this.startedFromMain = startedFromMain; } + + /** + * @exclude + */ + @Override + public JMXServer getJmxServer() throws Exception { + synchronized(this) { + if ((jmxServer == null) || jmxServer.isStopped()) { + if (debugEnabled) log.debug("starting JMX server"); + final boolean ssl = configuration.get(JPPFProperties.SSL_ENABLED); + JPPFProperty jmxProp = null; + jmxProp = MANAGEMENT_PORT_NODE; + jmxServer = JMXServerFactory.createServer(configuration, uuid, ssl, jmxProp); + jmxServer.start(getClass().getClassLoader()); + System.out.println("JPPF Node management initialized on port " + jmxServer.getManagementPort()); + } + } + return jmxServer; + } + + /** + * Stop the jmx server. + * @throws Exception if any error occurs. + * @exclude + */ + public void stopJmxServer() throws Exception { + if (jmxServer != null) jmxServer.stop(); + } + + /** + * @exclude + */ + @Override + public synchronized void stopNode() { + if (debugEnabled) log.debug("stopping node"); + setStopped(true); + executionManager.shutdown(); + serializationExecutor.shutdownNow(); + reset(true); + } + + /** + * Shutdown and eventually restart the node. + * @param restart determines whether this node should be restarted by the node launcher. + * @exclude + */ + public void shutdown(final boolean restart) { + if (!isLocal()) { + setStopped(true); + lifeCycleEventHandler.fireNodeEnding(); + new ShutdownOrRestart(restart, startedFromMain, this).run(); + } else { + if (debugEnabled) log.debug("shutting down local node"); + stopNode(); + } + } + + /** + * Reset this node for shutdown/restart/reconnection. + * @param stopJmx true if the JMX server is to be stopped, false otherwise. + */ + void reset(final boolean stopJmx) { + if (debugEnabled) log.debug("resetting with stopJmx=" + stopJmx); + lifeCycleEventHandler.fireNodeEnding(); + lifeCycleEventHandler.removeAllListeners(); + setNodeAdmin(null); + if (stopJmx) { + try { + if (providerManager != null) providerManager.unregisterProviderMBeans(); + if (jmxServer != null) jmxServer.stop(); + final NioServer acceptor = NioHelper.removeServer(JPPFIdentifiers.ACCEPTOR_CHANNEL); + if (acceptor != null) acceptor.shutdown(); + } catch(final Exception e) { + log.error(e.getMessage(), e); + } + } + classLoaderManager.closeClassLoader(); + try { + synchronized(this) { + closeDataChannel(); + } + classLoaderManager.clearContainers(); + } catch(final Exception e) { + log.error(e.getMessage(), e); + } + } + + /** + * Get the administration and monitoring MBean for this node. + * @return a {@link JPPFNodeAdminMBean} instance. + * @exclude + */ + public synchronized JPPFNodeAdminMBean getNodeAdmin() { + return nodeAdmin; + } + + /** + * Set the administration and monitoring MBean for this node. + * @param nodeAdmin a JPPFNodeAdminMBeanm instance. + * @exclude + */ + public synchronized void setNodeAdmin(final JPPFNodeAdminMBean nodeAdmin) { + this.nodeAdmin = nodeAdmin; + } + + /** + * @return the executor for serialization and deserialization of the tasks. + */ + public ExecutorService getSerializationExecutor() { + return serializationExecutor; + } } diff --git a/node/src/java/org/jppf/server/node/AbstractNodeConnectionChecker.java b/node/src/java/org/jppf/server/node/AbstractNodeConnectionChecker.java deleted file mode 100644 index d503ab59e6..0000000000 --- a/node/src/java/org/jppf/server/node/AbstractNodeConnectionChecker.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * JPPF. - * Copyright (C) 2005-2018 JPPF Team. - * http://www.jppf.org - * - * Licensed 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.jppf.server.node; - -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * Abstract common connection checker implementation for remote and local nodes. - * @author Laurent Cohen - * @exclude - */ -public abstract class AbstractNodeConnectionChecker implements NodeConnectionChecker { - /** - * - */ - protected AtomicBoolean stopped = new AtomicBoolean(true); - /** - * - */ - protected AtomicBoolean suspended = new AtomicBoolean(true); - /** - * - */ - protected Exception exception = null; - - @Override - public boolean isStopped() { - return stopped.get(); - } - - @Override - public boolean isSuspended() { - return suspended.get(); - } - - @Override - public Exception getException() { - return exception; - } -} diff --git a/node/src/java/org/jppf/server/node/AbstractNodeIO.java b/node/src/java/org/jppf/server/node/AbstractNodeIO.java index 39a99a4200..99c4dbec51 100644 --- a/node/src/java/org/jppf/server/node/AbstractNodeIO.java +++ b/node/src/java/org/jppf/server/node/AbstractNodeIO.java @@ -22,11 +22,8 @@ import java.io.InvalidClassException; import java.util.*; -import java.util.concurrent.Callable; -import org.jppf.io.*; import org.jppf.node.protocol.*; -import org.jppf.serialization.ObjectSerializer; import org.jppf.utils.*; import org.jppf.utils.configuration.JPPFProperties; import org.jppf.utils.hooks.HookFactory; @@ -47,22 +44,10 @@ public abstract class AbstractNodeIO implements No * Determines whether the debug level is enabled in the logging configuration, without the cost of a method call. */ private static boolean debugEnabled = LoggingUtils.isDebugEnabled(log); - /** - * Determines whether the trace level is enabled in the logging configuration, without the cost of a method call. - */ - private static boolean traceEnabled = log.isTraceEnabled(); /** * The node who owns this TaskIO. */ protected final N node; - /** - * The task bundle currently being processed. - */ - protected TaskBundle currentBundle = null; - /** - * Used to serialize/deserialize tasks and data providers. - */ - protected ObjectSerializer serializer = null; /** * Initialize this TaskIO with the specified node. @@ -73,22 +58,17 @@ public AbstractNodeIO(final N node) { HookFactory.registerConfigSingleHook(JPPFProperties.SERIALIZATION_EXCEPTION_HOOK, SerializationExceptionHook.class, new DefaultSerializationExceptionHook(), getClass().getClassLoader()); } - /** - * Read a task from the socket connection, along with its header information. - * @return a pair of JPPFTaskBundle and a List of JPPFTask instances. - * @throws Exception if an error is raised while reading the task data. - */ @Override - public Pair>> readTask() throws Exception { + public Pair>> readJob() throws Exception { try { final Object[] result = readObjects(); - currentBundle = (TaskBundle) result[0]; + final TaskBundle currentBundle = (TaskBundle) result[0]; final List> taskList = new ArrayList<>(result.length - 2); if (!currentBundle.isHandshake() && (currentBundle.getParameter(NODE_EXCEPTION_PARAM) == null)) { final DataProvider dataProvider = (DataProvider) result[1]; for (int i=0; i task = (Task) result[2 + i]; - task.setDataProvider(dataProvider).setInNode(true).setNode(node); + task.setDataProvider(dataProvider).setInNode(true).setNode(node).setJob(currentBundle); taskList.add(task); } } @@ -99,31 +79,15 @@ public Pair>> readTask() throws Exception { } } - /** - * Deserialize the objects read from the socket, and reload the appropriate classes if any class change is detected.
- * A class change is triggered when an InvalidClassException is caught. Upon catching this exception, - * the class loader is reinitialized and the class are reloaded. - * @return an array of objects deserialized from the socket stream. - * @throws Exception if the classes could not be reloaded or an error occurred during deserialization. - */ - protected Object[] readObjects() throws Exception { - Object[] result = null; - boolean reload = false; + @Override + public void writeResults(final TaskBundle bundle, final List> tasks) throws Exception { try { - result = deserializeObjects(); - } catch(final IncompatibleClassChangeError err) { - reload = true; - if (debugEnabled) log.debug(err.getMessage() + "; reloading classes", err); - } catch(final InvalidClassException e) { - reload = true; - if (debugEnabled) log.debug(e.getMessage() + "; reloading classes", e); - } - if (reload) { - if (debugEnabled) log.debug("reloading classes"); - handleReload(); - result = deserializeObjects(); + bundle.setSLA(null); + bundle.setMetadata(null); + sendResults(bundle, tasks); + } finally { + postSendResults(bundle); } - return result; } /** @@ -152,26 +116,36 @@ protected Object[] readObjects() throws Exception { * @param bundle the task wrapper to send along. * @param tasks the list of tasks with their result field updated. * @throws Exception if an error occurs while writing to the socket stream. + * @since 4.2 */ - @Override - public void writeResults(final TaskBundle bundle, final List> tasks) throws Exception { - try { - bundle.setSLA(null); - bundle.setMetadata(null); - sendResults(bundle, tasks); - } finally { - postSendResults(bundle); - } - } + protected abstract void sendResults(TaskBundle bundle, List> tasks) throws Exception; /** - * Write the execution results to the socket stream. - * @param bundle the task wrapper to send along. - * @param tasks the list of tasks with their result field updated. - * @throws Exception if an error occurs while writing to the socket stream. - * @since 4.2 + * Deserialize the objects read from the socket, and reload the appropriate classes if any class change is detected.
+ * A class change is triggered when an InvalidClassException is caught. Upon catching this exception, + * the class loader is reinitialized and the class are reloaded. + * @return an array of objects deserialized from the socket stream. + * @throws Exception if the classes could not be reloaded or an error occurred during deserialization. */ - protected abstract void sendResults(TaskBundle bundle, List> tasks) throws Exception; + protected Object[] readObjects() throws Exception { + Object[] result = null; + boolean reload = false; + try { + result = deserializeObjects(); + } catch(final IncompatibleClassChangeError err) { + reload = true; + if (debugEnabled) log.debug(err.getMessage() + "; reloading classes", err); + } catch(final InvalidClassException e) { + reload = true; + if (debugEnabled) log.debug(e.getMessage() + "; reloading classes", e); + } + if (reload) { + if (debugEnabled) log.debug("reloading classes"); + handleReload(); + result = deserializeObjects(); + } + return result; + } /** * Perform some cleanup after sending the results. @@ -214,81 +188,10 @@ protected void finalizeBundleData(final TaskBundle bundle, final List> t for (int n: resubmitSet) resubmitPos[count++] = n; bundle.setParameter(BundleParameter.RESUBMIT_TASK_POSITIONS, resubmitPos); } - } - - /** - * A pairing of a list of buffers and the total length of their usable data. - * @exclude - */ - protected static class BufferList extends Pair, Integer> { - /** - * Explicit serialVersionUID. - */ - private static final long serialVersionUID = 1L; - - /** - * Initialize this pairing with the specified list of buffers and length. - * @param first the list of buffers. - * @param second the total data length. - */ - public BufferList(final List first, final Integer second) { - super(first, second); - } - } - - /** - * The goal of this class is to serialize an object before sending it back to the server, - * and catch an eventual exception. - */ - protected class ObjectSerializationTask implements Callable { - /** - * The data to send over the network connection. - */ - private final Object object; - /** - * Used to serialize the object. - */ - private final ObjectSerializer ser; - /** - * The context class loader to use. - */ - private final ClassLoader contextCL; - - /** - * Initialize this task with the specified data buffer. - * @param object the object to serialize. - * @param serializer used to serialize the object. - * @param contextCL the context class loader to use. - */ - public ObjectSerializationTask(final Object object, final ObjectSerializer serializer, final ClassLoader contextCL) { - this.object = object; - this.ser = serializer; - this.contextCL = contextCL; - } - - @Override - public DataLocation call() { - DataLocation dl = null; - final int p = (object instanceof Task) ? ((Task) object).getPosition() : -1; - try { - Thread.currentThread().setContextClassLoader(contextCL); - if (traceEnabled) log.trace("before serialization of object at position " + p); - dl = IOHelper.serializeData(object, ser); - final int size = dl.getSize(); - if (traceEnabled) log.trace("serialized object at position " + p + ", size = " + size); - } catch(final Throwable t) { - log.error(t.getMessage(), t); - try { - final JPPFExceptionResult result = (JPPFExceptionResult) HookFactory.invokeSingleHook(SerializationExceptionHook.class, "buildExceptionResult", object, t); - result.setPosition(p); - dl = IOHelper.serializeData(result, ser); - } catch(final Exception e2) { - log.error(e2.getMessage(), e2); - } - } finally { - Thread.currentThread().setContextClassLoader(contextCL); - } - return dl; + if (!node.isOffline() && node.getConfiguration().containsProperty(JPPFProperties.NODE_MAX_JOBS)) { + final int maxJobs = node.getConfiguration().get(JPPFProperties.NODE_MAX_JOBS); + if (debugEnabled) log.debug("sending node max jobs = {}", maxJobs); + bundle.setParameter(BundleParameter.NODE_MAX_JOBS, maxJobs); } } } diff --git a/node/src/java/org/jppf/server/node/AsyncNodeExecutionManager.java b/node/src/java/org/jppf/server/node/AsyncNodeExecutionManager.java new file mode 100644 index 0000000000..a78f40015c --- /dev/null +++ b/node/src/java/org/jppf/server/node/AsyncNodeExecutionManager.java @@ -0,0 +1,124 @@ +/* + * JPPF. + * Copyright (C) 2005-2018 JPPF Team. + * http://www.jppf.org + * + * Licensed 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.jppf.server.node; + +import java.util.*; + +import org.jppf.classloader.AbstractJPPFClassLoader; +import org.jppf.execute.NodeTaskWrapper; +import org.jppf.execute.async.*; +import org.jppf.node.NodeInternal; +import org.jppf.node.event.LifeCycleEventHandler; +import org.jppf.node.protocol.*; +import org.jppf.utils.*; +import org.jppf.utils.configuration.JPPFProperties; +import org.slf4j.*; + +/** + * Instances of this class manage the execution of JPPF tasks by a node. + * @author Laurent Cohen + * @author Martin JANDA + * @author Paul Woodward + * @exclude + */ +public class AsyncNodeExecutionManager extends AbstractAsyncExecutionManager { + /** + * Logger for this class. + */ + private static final Logger log = LoggerFactory.getLogger(AsyncNodeExecutionManager.class); + /** + * Determines whether the debug level is enabled in the log configuration, without the cost of a method call. + */ + private static final boolean debugEnabled = LoggingUtils.isDebugEnabled(log); + /** + * The node that uses this execution manager. + */ + private final NodeInternal node; + + /** + * Initialize this execution manager with the specified node. + * @param node the node that uses this execution manager. + */ + public AsyncNodeExecutionManager(final NodeInternal node) { + super(node.getConfiguration(), JPPFProperties.PROCESSING_THREADS); + this.node = node; + } + + @Override + protected JobProcessingEntry setup(final TaskBundle bundle, final List> taskList) { + if (debugEnabled) log.debug("setting up bundle {}", bundle); + final JobProcessingEntry jobEntry = new JobProcessingEntry(); + jobEntry.bundle = bundle; + jobEntry.taskList = taskList; + jobEntry.taskWrapperList = new ArrayList<>(taskList.size()); + jobEntry.dataProvider = taskList.get(0).getDataProvider(); + jobEntry.uuidList = bundle.getUuidPath().getList(); + ClassLoader taskClassLoader = null; + try { + taskClassLoader = node instanceof ClassLoaderProvider ? ((ClassLoaderProvider) node).getClassLoader(jobEntry.uuidList) : taskList.get(0).getTaskClassLoader(); + jobEntry.usedClassLoader = threadManager.useClassLoader(taskClassLoader); + } catch (final Exception e) { + final String msg = ExceptionUtils.getMessage(e) + " - class loader lookup failed for uuidPath=" + jobEntry.uuidList; + if (debugEnabled) log.debug(msg, e); + else log.warn(msg); + } + jobEntry.accumulatedElapsed.set(0L); + final LifeCycleEventHandler handler = node.getLifeCycleEventHandler(); + if (handler != null) handler.fireJobStarting(bundle, taskClassLoader instanceof AbstractJPPFClassLoader ? (AbstractJPPFClassLoader) taskClassLoader : null, + taskList, jobEntry.dataProvider); + if (debugEnabled) log.debug("finished setting up bundle {}", bundle); + return jobEntry; + } + + @Override + protected void cleanup(final JobProcessingEntry jobEntry) { + final TaskBundle bundle = jobEntry.bundle; + if (debugEnabled) log.debug("cleaning up bundle {}", bundle); + //jobEntry.bundle = null; + bundle.setParameter(BundleParameter.NODE_BUNDLE_ELAPSED_PARAM, jobEntry.accumulatedElapsed.get()); + final ClassLoader cl = jobEntry.getClassLoader(); + final LifeCycleEventHandler handler = node.getLifeCycleEventHandler(); + if (handler != null) handler.fireJobEnding(bundle, cl instanceof AbstractJPPFClassLoader ? (AbstractJPPFClassLoader) cl : null, jobEntry.taskList, jobEntry.dataProvider); + jobEntry.dataProvider = null; + if (jobEntry.usedClassLoader != null) jobEntry.usedClassLoader.dispose(); + jobEntry.usedClassLoader = null; + jobEntry.taskList = null; + jobEntry.uuidList = null; + jobEntry.taskWrapperList = null; + timeoutHandler.clear(); + if (debugEnabled) log.debug("cleaned up bundle {}", bundle); + } + + @Override + public void triggerConfigChanged() { + super.triggerConfigChanged(); + node.getNodeConfigNotifier().sendNotification(node.getUuid(), node.getConfiguration()); + } + + @Override + public void taskEnded(final NodeTaskWrapper taskWrapper) { + // Workaround for the Android issue https://code.google.com/p/android/issues/detail?id=211596 + final Task task = taskWrapper.getTask(); + final Throwable t = task.getThrowable(); + if (node.isAndroid() && (t instanceof ReflectiveOperationException)) { + task.setThrowable(new JPPFTaskSerializationException(t)); + } + super.taskEnded(taskWrapper); + } +} diff --git a/node/src/java/org/jppf/server/node/JPPFContainer.java b/node/src/java/org/jppf/server/node/JPPFContainer.java index b99ccc0ade..ef5d8e2bc4 100644 --- a/node/src/java/org/jppf/server/node/JPPFContainer.java +++ b/node/src/java/org/jppf/server/node/JPPFContainer.java @@ -18,15 +18,13 @@ package org.jppf.server.node; import java.util.*; -import java.util.concurrent.*; +import java.util.concurrent.ExecutorService; import java.util.concurrent.locks.*; import org.jppf.classloader.AbstractJPPFClassLoader; -import org.jppf.io.*; import org.jppf.serialization.*; -import org.jppf.utils.*; +import org.jppf.utils.LoggingUtils; import org.jppf.utils.configuration.JPPFProperties; -import org.jppf.utils.hooks.HookFactory; import org.slf4j.*; /** @@ -46,22 +44,18 @@ public abstract class JPPFContainer { * Determines whether the debug level is enabled in the logging configuration, without the cost of a method call. */ private static boolean debugEnabled = LoggingUtils.isDebugEnabled(log); - /** - * Determines whether the trace level is enabled in the logging configuration, without the cost of a method call. - */ - private static boolean traceEnabled = log.isTraceEnabled(); /** * Utility for deserialization and serialization. */ - protected SerializationHelper helper = null; + protected SerializationHelper helper; /** * Utility for deserialization and serialization. */ - protected ObjectSerializer serializer = null; + protected ObjectSerializer serializer; /** * Class loader used for dynamic loading and updating of client classes. */ - protected AbstractJPPFClassLoader classLoader = null; + protected AbstractJPPFClassLoader classLoader; /** * The unique identifier for the submitting application. */ @@ -69,11 +63,11 @@ public abstract class JPPFContainer { /** * Used to prevent parallel deserialization. */ - private Lock lock = new ReentrantLock(); + private final Lock lock = new ReentrantLock(); /** * Determines whether tasks deserialization should be sequential rather than parallel. */ - private final boolean sequentialDeserialization = JPPFConfiguration.get(JPPFProperties.SEQUENTIAL_SERIALiZATION); + private final boolean sequentialDeserialization; /** * Whether the node has access to the client that submitted the job. */ @@ -81,16 +75,18 @@ public abstract class JPPFContainer { /** * Initialize this container with a specified application uuid. + * @param node the node holding this container. * @param uuidPath the unique identifier of a submitting application. * @param classLoader the class loader for this container. * @param clientAccess whether the node has access to the client that submitted the job. * @throws Exception if an error occurs while initializing. */ - public JPPFContainer(final List uuidPath, final AbstractJPPFClassLoader classLoader, final boolean clientAccess) throws Exception { + public JPPFContainer(final AbstractCommonNode node, final List uuidPath, final AbstractJPPFClassLoader classLoader, final boolean clientAccess) throws Exception { if (debugEnabled) log.debug("new JPPFContainer with uuidPath={}, classLoader={}, clientAccess={}", uuidPath, classLoader, clientAccess); this.uuidPath = uuidPath; this.classLoader = classLoader; this.clientAccess = clientAccess; + this.sequentialDeserialization = node.getConfiguration().get(JPPFProperties.SEQUENTIAL_SERIALiZATION); init(); } @@ -163,76 +159,6 @@ public void setUuidPath(final List uuidPath) { this.uuidPath = uuidPath; } - /** - * Instances of this class are used to deserialize objects from an - * incoming message in parallel. - */ - protected class ObjectDeserializationTask implements Callable { - /** - * The data received over the network connection. - */ - private final DataLocation dl; - /** - * Index of the object to deserialize in the incoming IO message; used for debugging purposes. - */ - private final int index; - /** - * The deserialized object. - */ - private Object object; - - /** - * Initialize this task with the specified data buffer. - * @param dl the data read from the network connection, stored in a memory-sensitive location. - * @param index index of the object to deserialize in the incoming IO message; used for debugging purposes. - */ - public ObjectDeserializationTask(final DataLocation dl, final int index) { - this.dl = dl; - this.index = index; - } - - /** - * Execute this task. - * @return a deserialized object. - */ - @Override - public ObjectDeserializationTask call() { - final ClassLoader cl = Thread.currentThread().getContextClassLoader(); - try { - Thread.currentThread().setContextClassLoader(getClassLoader()); - if (traceEnabled) log.debug("deserializing object index = " + index); - if (sequentialDeserialization) lock.lock(); - try { - object = IOHelper.unwrappedData(dl, serializer); - } finally { - if (sequentialDeserialization) lock.unlock(); - } - } catch (final Throwable t) { - final String desc = (index == 0 ? "data provider" : "task at index " + index) + " could not be deserialized"; - if (debugEnabled) log.debug("{} : {}", desc, ExceptionUtils.getStackTrace(t)); - else log.error("{} : {}", desc, ExceptionUtils.getMessage(t)); - if (index > 0) object = HookFactory.invokeSingleHook(SerializationExceptionHook.class, "buildExceptionResult", desc, t); - } finally { - Thread.currentThread().setContextClassLoader(cl); - } - return this; - } - - /** - * @return the index of the object to deserialize in the incoming IO message; used for debugging purposes. - */ - public int getIndex() { - return index; - } - - /** - * @return the deserialized object. - */ - public Object getObject() { - return object; - } - } - /** * Return the utility object for serialization and deserialization. * @return an {@link ObjectSerializer} instance. @@ -240,4 +166,18 @@ public Object getObject() { public ObjectSerializer getSerializer() { return serializer; } + + /** + * @return the lock for sequential deserialization. + */ + public Lock getLock() { + return lock; + } + + /** + * @return {@code true} if sequential serialization/serialization is enabled, {@code false} otherwise. + */ + public boolean isSequentialDeserialization() { + return sequentialDeserialization; + } } diff --git a/node/src/java/org/jppf/server/node/JPPFNode.java b/node/src/java/org/jppf/server/node/JPPFNode.java index 98642aa147..4d0b783a7f 100644 --- a/node/src/java/org/jppf/server/node/JPPFNode.java +++ b/node/src/java/org/jppf/server/node/JPPFNode.java @@ -17,17 +17,15 @@ */ package org.jppf.server.node; -import static org.jppf.utils.configuration.JPPFProperties.MANAGEMENT_PORT_NODE; - import java.io.IOException; import java.util.*; +import java.util.concurrent.*; import org.jppf.*; import org.jppf.classloader.AbstractJPPFClassLoader; +import org.jppf.execute.async.ExecutionManagerListener; import org.jppf.management.*; import org.jppf.management.spi.*; -import org.jppf.nio.*; -import org.jppf.node.ShutdownOrRestart; import org.jppf.node.connection.ConnectionReason; import org.jppf.node.event.LifeCycleEventHandler; import org.jppf.node.protocol.*; @@ -36,7 +34,7 @@ import org.jppf.serialization.*; import org.jppf.ssl.SSLConfigurationException; import org.jppf.utils.*; -import org.jppf.utils.configuration.*; +import org.jppf.utils.concurrent.*; import org.slf4j.*; /** @@ -44,7 +42,7 @@ * @author Laurent Cohen * @author Domingos Creado */ -public abstract class JPPFNode extends AbstractCommonNode implements ClassLoaderProvider { +public abstract class JPPFNode extends AbstractCommonNode implements ClassLoaderProvider, ExecutionManagerListener { /** * Logger for this class. */ @@ -54,55 +52,37 @@ public abstract class JPPFNode extends AbstractCommonNode implements ClassLoader */ private static final boolean debugEnabled = LoggingUtils.isDebugEnabled(log); /** - * The object responsible for this node's I/O. - * @exclude - */ - protected NodeIO nodeIO; - /** - * Determines whether JMX management and monitoring is enabled for this node. - */ - private boolean jmxEnabled; - /** - * Determines whether this node can execute .Net tasks. - */ - private final boolean dotnetCapable; - /** - * The default node's management MBean. - */ - private JPPFNodeAdminMBean nodeAdmin; - /** - * The jmx server that handles administration and monitoring functions for this node. + * Bundle set in the JobReader or JobWriter queue when an exception occurs. */ - private static JMXServer jmxServer; + private static final Pair>> EXCEPTIONAL_BUNDLE = new Pair<>(null, null); /** - * Manager for the MBean defined through the service provider interface. + * The bundle currently processed in offline mode. */ - private JPPFMBeanProviderManager providerManager; + private Pair>> currentBundle; /** - * Handles the firing of node life cycle events and the listeners that subscribe to these events. + * The slave node manager. */ - private LifeCycleEventHandler lifeCycleEventHandler; + private final SlaveNodeManager slaveManager; /** - * The connection checker for this node. + * The mbean which sends notifications of configuration changes. */ - private NodeConnectionChecker connectionChecker; + private final NodeConfigNotifier configNotifier = new NodeConfigNotifier(); /** - * Determines whether the node connection checker should be used. + * */ - private final boolean checkConnection; + private final JobReader jobReader = new JobReader(); /** - * The bundle currently processed in offline mode. + * */ - private Pair>> currentBundle; + private final JobWriter jobWriter = new JobWriter(); /** - * The configuration of this node. - * @exclude + * */ - protected final TypedProperties configuration; + private boolean executionComplete; /** - * The slave node manager. + * */ - private final SlaveNodeManager slaveManager; + private final ThreadSynchronization offlineLock = new ThreadSynchronization(); /** * Default constructor. @@ -110,13 +90,10 @@ public abstract class JPPFNode extends AbstractCommonNode implements ClassLoader * @param configuration the configuration of this node. */ public JPPFNode(final String uuid, final TypedProperties configuration) { - super(uuid); - this.configuration = configuration; + super(uuid, configuration); if (debugEnabled) log.debug("creating node with config=\n{}", configuration); - jmxEnabled = configuration.get(JPPFProperties.MANAGEMENT_ENABLED); - dotnetCapable = configuration.get(JPPFProperties.DOTNET_BRIDGE_INITIALIZED); - checkConnection = configuration.get(JPPFProperties.NODE_CHECK_CONNECTION); - executionManager = new NodeExecutionManager(this); + executionManager = new AsyncNodeExecutionManager(this); + executionManager.addExecutionManagerListener(this); lifeCycleEventHandler = new LifeCycleEventHandler(this); updateSystemInformation(); slaveManager = new SlaveNodeManager(this); @@ -130,23 +107,21 @@ public JPPFNode(final String uuid, final TypedProperties configuration) { public void run() { setStopped(false); boolean initialized = false; - if (debugEnabled) log.debug("Start of node main loop, nodeUuid=" + uuid); + if (debugEnabled) log.debug("start of node main loop, nodeUuid=" + uuid); while (!isStopped()) { try { if (!isLocal() && getShuttingDown().get()) break; init(); if (!initialized) { - System.out.println("Node successfully initialized"); + System.out.println("node successfully initialized"); initialized = true; } perform(); } catch(final SecurityException|SSLConfigurationException e) { - if (checkConnection) connectionChecker.stop(); if (!isStopped()) reset(true); throw new JPPFError(e); } catch(final IOException e) { if (!getShuttingDown().get() && !isStopped()) log.error(e.getMessage(), e); - if (checkConnection) connectionChecker.stop(); if (!isStopped()) { reset(true); if (reconnectionNotification != null) { @@ -158,20 +133,18 @@ public void run() { } } catch(final Exception e) { log.error(e.getMessage(), e); - if (checkConnection) connectionChecker.stop(); if (!isStopped()) reset(true); } } - if (debugEnabled) log.debug("End of node main loop"); + if (debugEnabled) log.debug("end of node main loop"); } /** * Perform the main execution loop for this node. At each iteration, this method listens for a task to execute, * receives it, executes it and sends the results back. * @throws Exception if an error was raised from the underlying socket connection or the class loader. - * @exclude */ - public void perform() throws Exception { + private void perform() throws Exception { if (debugEnabled) log.debug("Start of node secondary loop"); boolean shouldInitDataChannel = false; while (!checkStopped()) { @@ -184,7 +157,8 @@ public void perform() throws Exception { shouldInitDataChannel = false; initDataChannel(); } - processNextJob(); + if (isOffline()) processNextJob(); + else processNextJobAsync(); } catch (final IOException|JPPFSuspendedNodeException e) { if (!isSuspended()) throw e; shouldInitDataChannel = true; @@ -201,35 +175,44 @@ public void perform() throws Exception { * @throws Exception if any error occurs. */ private void processNextJob() throws Exception { - final Pair>> pair = nodeIO.readTask(); + final Pair>> pair = nodeIO.readJob(); if (debugEnabled) log.debug("received bundle"); TaskBundle bundle = pair.first(); List> taskList = pair.second(); if (debugEnabled) log.debug(!bundle.isHandshake() ? "received a bundle with " + taskList.size() + " tasks" : "received a handshake bundle"); if (!bundle.isHandshake()) { - try { - if (checkConnection) connectionChecker.resume(); - executionManager.execute(bundle, taskList); - } finally { - if (checkConnection) { - connectionChecker.suspend(); - if (connectionChecker.getException() != null) throw connectionChecker.getException(); - } - } - if (isOffline()) { - currentBundle = pair; - initDataChannel(); - processNextJob(); // new handshake - } else processResults(bundle, taskList); + currentBundle = pair; + executionComplete = false; + executionManager.execute(bundle, taskList); + while (!isStopped() && !executionComplete) offlineLock.goToSleep(); + initDataChannel(); + processNextJob(); // new handshake } else { if (currentBundle != null) { - if (!isOffline()) currentBundle.first().setParameter(BundleParameter.NODE_BUNDLE_ID, bundle.getParameter(BundleParameter.NODE_BUNDLE_ID)); bundle = currentBundle.first(); taskList = currentBundle.second(); } checkInitialBundle(bundle); currentBundle = null; processResults(bundle, taskList); + } + } + + /** + * Read a job to execute or a handshake job. + * @throws Exception if any error occurs. + */ + private void processNextJobAsync() throws Exception { + final Pair>> pair = jobReader.nextJob(); + if (debugEnabled) log.debug("received bundle"); + final TaskBundle bundle = pair.first(); + final List> taskList = pair.second(); + if (debugEnabled) log.debug(!bundle.isHandshake() ? "received a bundle with " + taskList.size() + " tasks" : "received a handshake bundle"); + if (!bundle.isHandshake()) { + executionManager.execute(bundle, taskList); + } else { + checkInitialBundle(bundle); + jobWriter.putJob(bundle, taskList); if (isMasterNode()) slaveManager.handleStartup(); } } @@ -249,7 +232,7 @@ private void checkInitialBundle(final TaskBundle bundle) throws Exception { if (isSuspended()) bundle.setParameter(BundleParameter.CLOSE_COMMAND, true); if (currentBundle != null) { bundle.setParameter(BundleParameter.NODE_OFFLINE_OPEN_REQUEST, true); - bundle.setParameter(BundleParameter.NODE_BUNDLE_ID, currentBundle.first().getParameter(BundleParameter.NODE_BUNDLE_ID)); + bundle.setBundleId(currentBundle.first().getBundleId()); bundle.setParameter(BundleParameter.JOB_UUID, currentBundle.first().getUuid()); } } @@ -281,7 +264,7 @@ private void processResults(final TaskBundle bundle, final List> taskLis } nodeIO.writeResults(bundle, taskList); if ((taskList != null) && (!taskList.isEmpty())) { - if (!isJmxEnabled()) setTaskCount(getTaskCount() + taskList.size()); + if (!isJmxEnabled()) setExecutedTaskCount(getExecutedTaskCount() + taskList.size()); } if (!bundle.isHandshake()) lifeCycleEventHandler.fireBeforeNextJob(); } @@ -304,47 +287,30 @@ protected synchronized void init() throws Exception { log.error("Error registering the MBeans", e); } if (isJmxEnabled()) { - JMXServer jmxServer = null; try { - jmxServer = getJmxServer(); + getJmxServer(); } catch(final Exception e) { jmxEnabled = false; - System.out.println("JMX initialization failure - management is disabled for this node"); - System.out.println("see the log file for details"); + System.out.println("JMX initialization failure - management is disabled for this node\nsee the log file for details"); + log.error("Error creating the JMX server", e); try { if (jmxServer != null) jmxServer.stop(); } catch(final Exception e2) { log.error("Error stopping the JMX server", e2); } - jmxServer = null; - log.error("Error creating the JMX server", e); } } initStartups(); initDataChannel(); - if (checkConnection) { - connectionChecker = createConnectionChecker(); - connectionChecker.start(); - } lifeCycleEventHandler.loadListeners(); lifeCycleEventHandler.fireNodeStarting(); + if (!isOffline()) { + ThreadUtils.startDaemonThread(jobReader, "JobReader"); + ThreadUtils.startDaemonThread(jobWriter, "JobWriter"); + } if (debugEnabled) log.debug("end node initialization"); } - /** - * Initialize this node's data channel. - * @throws Exception if an error is raised during initialization. - * @exclude - */ - protected abstract void initDataChannel() throws Exception; - - /** - * Initialize this node's data channel. - * @throws Exception if an error is raised during initialization. - * @exclude - */ - public abstract void closeDataChannel() throws Exception; - /** * Get the main classloader for the node. This method performs a lazy initialization of the classloader. * @throws Exception if an error occurs while instantiating the class loader. @@ -363,193 +329,153 @@ public void initHelper() throws Exception { } /** - * Get the administration and monitoring MBean for this node. - * @return a JPPFNodeAdminMBean instance. + * Trigger the configuration changed flag. * @exclude */ - public synchronized JPPFNodeAdminMBean getNodeAdmin() { - return nodeAdmin; + public void triggerConfigChanged() { + updateSystemInformation(); + executionManager.triggerConfigChanged(); } - /** - * Set the administration and monitoring MBean for this node. - * @param nodeAdmin a JPPFNodeAdminMBeanm instance. - * @exclude - */ - public synchronized void setNodeAdmin(final JPPFNodeAdminMBean nodeAdmin) { - this.nodeAdmin = nodeAdmin; + @Override + public ClassLoader getClassLoader(final List uuidPath) throws Exception { + return getContainer(uuidPath).getClassLoader(); } /** - * Determines whether JMX management and monitoring is enabled for this node. - * @return true if JMX is enabled, false otherwise. + * Check whether this node is stopped or shutting down. If not, an unchecked {@code IllegalStateException} is thrown. + * @return {@code true} if the node is stopped or shutting down. */ - boolean isJmxEnabled() { - return jmxEnabled && !isOffline(); + private boolean checkStopped() { + if (isStopped()) throw new IllegalStateException("this node is shutting down"); + return false; } /** + * @return the slave node manager. * @exclude */ - @Override - public synchronized void stopNode() { - if (debugEnabled) log.debug("stopping node"); - setStopped(true); - executionManager.shutdown(); - reset(true); + public SlaveNodeManager getSlaveManager() { + return slaveManager; } - /** - * Shutdown and eventually restart the node. - * @param restart determines whether this node should be restarted by the node launcher. - * @exclude - */ - public void shutdown(final boolean restart) { - if (!isLocal()) { - setStopped(true); - lifeCycleEventHandler.fireNodeEnding(); - //NodeRunner.shutdown(this, restart); - new ShutdownOrRestart(restart, startedFromMain, this).run(); - } else { - if (debugEnabled) log.debug("shutting down local node"); - stopNode(); - } + @Override + public NodeConfigNotifier getNodeConfigNotifier() { + return configNotifier; } - /** - * Reset this node for shutdown/restart/reconnection. - * @param stopJmx true if the JMX server is to be stopped, false otherwise. - */ - private void reset(final boolean stopJmx) { - if (debugEnabled) log.debug("resetting with stopJmx=" + stopJmx); - lifeCycleEventHandler.fireNodeEnding(); - lifeCycleEventHandler.removeAllListeners(); - setNodeAdmin(null); - if (stopJmx) { - try { - if (providerManager != null) providerManager.unregisterProviderMBeans(); - if (jmxServer != null) jmxServer.stop(); - final NioServer acceptor = NioHelper.removeServer(JPPFIdentifiers.ACCEPTOR_CHANNEL); - if (acceptor != null) acceptor.shutdown(); - } catch(final Exception e) { - log.error(e.getMessage(), e); - } - } - classLoaderManager.closeClassLoader(); + @Override + public void bundleExecuted(final TaskBundle bundle, final List> tasks, final Throwable t) { try { - synchronized(this) { - closeDataChannel(); + if (debugEnabled) log.debug("executed {} tasks of job {}", tasks.size(), bundle); + if (isOffline()) { + executionComplete = true; + offlineLock.wakeUp(); + } else { + jobWriter.putJob(bundle, tasks); } - classLoaderManager.clearContainers(); - } catch(final Exception e) { + } catch (final Exception e) { log.error(e.getMessage(), e); } } /** - * Get the jmx server that handles administration and monitoring functions for this node. - * @return a JMXServerImpl instance. - * @throws Exception if any error occurs. + * Read the jobs from the network connection and make them available in a queue. */ - @Override - public JMXServer getJmxServer() throws Exception { - synchronized(this) { - if ((jmxServer == null) || jmxServer.isStopped()) { - if (debugEnabled) log.debug("starting JMX server"); - final boolean ssl = configuration.get(JPPFProperties.SSL_ENABLED); - JPPFProperty jmxProp = null; - jmxProp = MANAGEMENT_PORT_NODE; - jmxServer = JMXServerFactory.createServer(configuration, uuid, ssl, jmxProp); - jmxServer.start(getClass().getClassLoader()); - System.out.println("JPPF Node management initialized on port " + jmxServer.getManagementPort()); + private class JobReader extends ThreadSynchronization implements Runnable { + /** + * The queue of received jobs. + */ + private BlockingQueue>>> queue = new LinkedBlockingQueue<>(); + /** + * + */ + private Exception lastException; + + @Override + public void run() { + while (!isStopped() && !JPPFNode.this.isStopped() && !shutdownRequestFlag.get()) { + try { + queue.offer(nodeIO.readJob()); + } catch (final Exception e) { + lastException = e; + setStopped(true); + // to avoid being stuck in queue.take() when calling the nextJob() method + queue.offer(EXCEPTIONAL_BUNDLE); + break; + } } } - return jmxServer; - } - - /** - * Stop the jmx server. - * @throws Exception if any error occurs. - * @exclude - */ - public void stopJmxServer() throws Exception { - if (jmxServer != null) jmxServer.stop(); - } - - /** - * @exclude - */ - @Override - public LifeCycleEventHandler getLifeCycleEventHandler() { - return lifeCycleEventHandler; - } - - /** - * Create the connection checker for this node. - * @return an implementation of {@link NodeConnectionChecker}. - * @exclude - */ - protected abstract NodeConnectionChecker createConnectionChecker(); - - /** - * Trigger the configuration changed flag. - * @exclude - */ - public void triggerConfigChanged() { - updateSystemInformation(); - executionManager.triggerConfigChanged(); - } - - @Override - public ClassLoader getClassLoader(final List uuidPath) throws Exception { - return getContainer(uuidPath).getClassLoader(); - } - - @Override - public boolean isOffline() { - return isAndroid() || getClassLoader().isOffline(); - } - - @Override - public boolean isMasterNode() { - return !isOffline() && (systemInformation != null) && systemInformation.getJppf().get(JPPFProperties.PROVISIONING_MASTER); - } - - @Override - public boolean isSlaveNode() { - return (systemInformation != null) && systemInformation.getJppf().get(JPPFProperties.PROVISIONING_SLAVE); - } - @Override - public String getMasterNodeUuid() { - if (systemInformation == null) return null; - return systemInformation.getJppf().get(JPPFProperties.PROVISIONING_MASTER_UUID); + /** + * Get the next job from the queue, blocking if the queue is empty. + * @return a pairing of a job header and its tasks. + * @throws Exception if any error occurs. + */ + private Pair>> nextJob() throws Exception { + Pair>> result = null; + if (lastException == null) result = queue.take(); + if (lastException != null) { + queue.clear(); + final Exception e = lastException; + lastException = null; + throw e; + } + return result; + } } /** - * Check whether this node is stopped or shutting down. If not, an unchecked {@code IllegalStateException} is thrown. - * @return {@code true} if the node is stopped or shutting down. + * Get job results from a queue and send them back to the driver. */ - private boolean checkStopped() { - if (isStopped()) throw new IllegalStateException("this node is shutting down"); - return false; - } + private class JobWriter extends ThreadSynchronization implements Runnable { + /** + * The queue of received jobs. + */ + private BlockingQueue>>> queue = new LinkedBlockingQueue<>(); + /** + * + */ + private Exception lastException; - @Override - public boolean isDotnetCapable() { - return dotnetCapable; - } + @Override + public void run() { + while (!isStopped() && !JPPFNode.this.isStopped()) { + try { + final Pair>> pair = queue.take(); + processResults(pair.first(), pair.second()); + } catch (final Exception e) { + lastException = e; + setStopped(true); + break; + } + } + } - @Override - public TypedProperties getConfiguration() { - return configuration; + /** + * Put the next job results in the send queue. + * @param bundle the bundle that contains the tasks and header information. + * @param taskList the tasks results. + * @throws Exception if any error occurs. + */ + private void putJob(final TaskBundle bundle, final List> taskList) throws Exception { + if (lastException != null) { + final Exception e = lastException; + lastException = null; + throw e; + } + queue.offer(new Pair<>(bundle, taskList)); + } } /** - * @return the slave node manager. * @exclude */ - public SlaveNodeManager getSlaveManager() { - return slaveManager; + @Override + public synchronized void stopNode() { + if (debugEnabled) log.debug("stopping node"); + if (jobReader != null) jobReader.queue.clear(); + if (jobWriter != null) jobWriter.queue.clear(); + super.stopNode(); } } diff --git a/node/src/java/org/jppf/server/node/NodeConnectionChecker.java b/node/src/java/org/jppf/server/node/NodeConnectionChecker.java deleted file mode 100644 index 1955e91bdb..0000000000 --- a/node/src/java/org/jppf/server/node/NodeConnectionChecker.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * JPPF. - * Copyright (C) 2005-2018 JPPF Team. - * http://www.jppf.org - * - * Licensed 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.jppf.server.node; - -/** - * A connection checker is run asynchronusly while tasks are exected in a node and checks if the communication channel is avaialble. - * When it is no more valid, this checker propagates the resulting exception to the node for proper handling. - * @author Laurent Cohen - * @exclude - */ -public interface NodeConnectionChecker -{ - /** - * Start this checker. - */ - void start(); - - /** - * Stop this checker. - */ - void stop(); - - /** - * Determine whether this checker is stopped. - * @return true if this checker is stopped, false otherwise. - */ - boolean isStopped(); - - /** - * Resume this checker. - */ - void resume(); - - /** - * Suspend this checker. - */ - void suspend(); - - /** - * Determine whether this checker is suspended. - * @return true if this checker is suspended, false otherwise. - */ - boolean isSuspended(); - /** - * Get an eventual exception resulting from the check. - * @return an {@link Exception} if one occurred during the check, or null if none was raised. - */ - Exception getException(); -} diff --git a/node/src/java/org/jppf/server/node/NodeExecutionManager.java b/node/src/java/org/jppf/server/node/NodeExecutionManager.java index 7771a6097a..e59ec9fb60 100644 --- a/node/src/java/org/jppf/server/node/NodeExecutionManager.java +++ b/node/src/java/org/jppf/server/node/NodeExecutionManager.java @@ -22,12 +22,11 @@ import org.jppf.classloader.AbstractJPPFClassLoader; import org.jppf.execute.*; -import org.jppf.management.NodeConfigNotifier; import org.jppf.node.NodeInternal; import org.jppf.node.event.LifeCycleEventHandler; import org.jppf.node.protocol.*; import org.jppf.utils.*; -import org.jppf.utils.configuration.*; +import org.jppf.utils.configuration.JPPFProperties; import org.slf4j.*; /** @@ -56,36 +55,21 @@ public class NodeExecutionManager extends AbstractExecutionManager { * @param node the node that uses this execution manager. */ public NodeExecutionManager(final NodeInternal node) { - this(node, JPPFProperties.PROCESSING_THREADS); - } - - /** - * Initialize this execution manager with the specified node. - * @param node the node that uses this execution manager. - * @param nbThreadsProperty the name of the property which configures the number of threads. - */ - public NodeExecutionManager(final NodeInternal node, final JPPFProperty nbThreadsProperty) { - super(nbThreadsProperty); - if (node == null) throw new IllegalArgumentException("node is null"); + super(node.getConfiguration(), JPPFProperties.PROCESSING_THREADS); this.node = node; } - /** - * Prepare this execution manager for executing the tasks of a bundle. - * @param bundle the bundle whose tasks are to be executed. - * @param taskList the list of tasks to execute. - */ @Override protected void setup(final TaskBundle bundle, final List> taskList) { if (debugEnabled) log.debug("setting up bundle {}", bundle); - taskNotificationDispatcher.setBundle(this.bundle = bundle); + //taskNotificationDispatcher.setBundle(this.bundle = bundle); this.taskList = taskList; this.taskWrapperList = new ArrayList<>(taskList.size()); this.dataProvider = taskList.get(0).getDataProvider(); this.uuidList = bundle.getUuidPath().getList(); ClassLoader taskClassLoader = null; try { - taskClassLoader = node instanceof ClassLoaderProvider ? ((ClassLoaderProvider) node).getClassLoader(uuidList) : getTaskClassLoader(taskList.get(0)); + taskClassLoader = node instanceof ClassLoaderProvider ? ((ClassLoaderProvider) node).getClassLoader(uuidList) : taskList.get(0).getTaskClassLoader(); usedClassLoader = threadManager.useClassLoader(taskClassLoader); } catch (final Exception e) { final String msg = ExceptionUtils.getMessage(e) + " - class loader lookup failed for uuidPath=" + uuidList; @@ -99,9 +83,6 @@ protected void setup(final TaskBundle bundle, final List> taskList) { if (debugEnabled) log.debug("finished setting up bundle {}", bundle); } - /** - * Cleanup method invoked when all tasks for the current bundle have completed. - */ @Override protected void cleanup() { if (debugEnabled) log.debug("cleaning up bundle {}", bundle); @@ -112,7 +93,7 @@ protected void cleanup() { this.dataProvider = null; usedClassLoader.dispose(); usedClassLoader = null; - taskNotificationDispatcher.setBundle(this.bundle = null); + //taskNotificationDispatcher.setBundle(this.bundle = null); this.taskList = null; this.uuidList = null; setJobCancelled(false); @@ -121,24 +102,15 @@ protected void cleanup() { if (debugEnabled) log.debug("cleaned up bundle {}", bundle); } - /** - * Get the appropiate class loader for the specfied task. - * @param task the task from which to get the class laoder. - * @return an instance of {@link ClassLoader}. - */ - private static ClassLoader getTaskClassLoader(final Task task) { - return task.getTaskClassLoader(); - } - @Override public void triggerConfigChanged() { super.triggerConfigChanged(); - NodeConfigNotifier.getInstance().sendNotification(node.getUuid(), node.getConfiguration()); + node.getNodeConfigNotifier().sendNotification(node.getUuid(), node.getConfiguration()); } @Override protected void taskEnded(final NodeTaskWrapper taskWrapper) { - // Workaoround for the Android issue https://code.google.com/p/android/issues/detail?id=211596 + // Workaround for the Android issue https://code.google.com/p/android/issues/detail?id=211596 final Task task = taskWrapper.getTask(); final Throwable t = task.getThrowable(); if (node.isAndroid() && (t instanceof ReflectiveOperationException)) { diff --git a/node/src/java/org/jppf/server/node/NodeIO.java b/node/src/java/org/jppf/server/node/NodeIO.java index 59258ad165..8501982fec 100644 --- a/node/src/java/org/jppf/server/node/NodeIO.java +++ b/node/src/java/org/jppf/server/node/NodeIO.java @@ -28,14 +28,13 @@ * @author Laurent Cohen * @exclude */ -public interface NodeIO -{ +public interface NodeIO { /** - * Read a task from the socket connection, along with its header information. - * @return a pair of JPPFTaskBundle and a List of JPPFTask instances. + * Read a job from the socket connection, along with its header information. + * @return a pair of {@link JPPFTaskBundle} and a {@link List} of {@link Task} instances. * @throws Exception if an error is raised while reading the task data. */ - Pair>> readTask() throws Exception; + Pair>> readJob() throws Exception; /** * Write the execution results to the socket stream. @@ -44,5 +43,4 @@ public interface NodeIO * @throws Exception if an error occurs while writing to the socket stream. */ void writeResults(TaskBundle bundle, List> tasks) throws Exception; - } diff --git a/node/src/java/org/jppf/server/node/ObjectDeserializationTask.java b/node/src/java/org/jppf/server/node/ObjectDeserializationTask.java new file mode 100644 index 0000000000..cd15f0bbfb --- /dev/null +++ b/node/src/java/org/jppf/server/node/ObjectDeserializationTask.java @@ -0,0 +1,111 @@ +/* + * JPPF. + * Copyright (C) 2005-2018 JPPF Team. + * http://www.jppf.org + * + * Licensed 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.jppf.server.node; + +import java.util.concurrent.Callable; + +import org.jppf.io.*; +import org.jppf.utils.ExceptionUtils; +import org.jppf.utils.hooks.HookFactory; +import org.slf4j.*; + +/** + * Instances of this class are used to deserialize objects from an + * incoming message in parallel. + * @exclude + */ +public class ObjectDeserializationTask implements Callable { + /** + * Logger for this class. + */ + private static final Logger log = LoggerFactory.getLogger(JPPFContainer.class); + /** + * Determines whether the trace level is enabled in the logging configuration, without the cost of a method call. + */ + private static final boolean traceEnabled = log.isTraceEnabled(); + /** + * The data received over the network connection. + */ + private final DataLocation dl; + /** + * Index of the object to deserialize in the incoming IO message; used for debugging purposes. + */ + private final int index; + /** + * The deserialized object. + */ + private Object object; + /** + * + */ + private final JPPFContainer cont; + + /** + * Initialize this task with the specified data buffer. + * @param cont . + * @param dl the data read from the network connection, stored in a memory-sensitive location. + * @param index index of the object to deserialize in the incoming IO message; used for debugging purposes. + */ + public ObjectDeserializationTask(final JPPFContainer cont, final DataLocation dl, final int index) { + this.cont = cont; + this.dl = dl; + this.index = index; + } + + /** + * Execute this task. + * @return a deserialized object. + */ + @Override + public ObjectDeserializationTask call() { + final ClassLoader cl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(cont.getClassLoader()); + if (traceEnabled) log.trace("deserializing object index = " + index); + if (cont.isSequentialDeserialization()) cont.getLock().lock(); + try { + object = IOHelper.unwrappedData(dl, cont.getSerializer()); + } finally { + if (cont.isSequentialDeserialization()) cont.getLock().unlock(); + } + } catch (final Throwable t) { + final String desc = (index == 0 ? "data provider" : "task at index " + index) + " could not be deserialized"; + if (traceEnabled) log.debug("{} : {}", desc, ExceptionUtils.getStackTrace(t)); + else log.error("{} : {}", desc, ExceptionUtils.getMessage(t)); + if (index > 0) object = HookFactory.invokeSingleHook(SerializationExceptionHook.class, "buildExceptionResult", desc, t); + } finally { + Thread.currentThread().setContextClassLoader(cl); + } + return this; + } + + /** + * @return the index of the object to deserialize in the incoming IO message; used for debugging purposes. + */ + public int getIndex() { + return index; + } + + /** + * @return the deserialized object. + */ + public Object getObject() { + return object; + } +} \ No newline at end of file diff --git a/node/src/java/org/jppf/server/node/ObjectSerializationTask.java b/node/src/java/org/jppf/server/node/ObjectSerializationTask.java new file mode 100644 index 0000000000..672b3f7248 --- /dev/null +++ b/node/src/java/org/jppf/server/node/ObjectSerializationTask.java @@ -0,0 +1,107 @@ +/* + * JPPF. + * Copyright (C) 2005-2018 JPPF Team. + * http://www.jppf.org + * + * Licensed 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.jppf.server.node; + +import java.util.concurrent.Callable; + +import org.jppf.io.*; +import org.jppf.node.protocol.*; +import org.jppf.serialization.ObjectSerializer; +import org.jppf.utils.hooks.HookFactory; +import org.slf4j.*; + +/** + * The goal of this class is to serialize an object before sending it back to the server, and catch an eventual exception. + * @exclude + */ +public class ObjectSerializationTask implements Callable { + /** + * Logger for this class. + */ + private static final Logger log = LoggerFactory.getLogger(ObjectSerializationTask.class); + /** + * Determines whether the trace level is enabled in the logging configuration, without the cost of a method call. + */ + private static final boolean traceEnabled = log.isTraceEnabled(); + /** + * The data to send over the network connection. + */ + private final Object object; + /** + * Used to serialize the object. + */ + private final ObjectSerializer ser; + /** + * The context class loader to use. + */ + private final ClassLoader contextCL; + /** + * The order in which this task was submitted, if any. + */ + private final int submitOrder; + + /** + * Initialize this task with the specified data buffer. + * @param object the object to serialize. + * @param cont container used to serialize the object. + * @param submitOrder this task's submission order. + */ + public ObjectSerializationTask(final Object object, final JPPFContainer cont, final int submitOrder) { + this.object = object; + this.ser = cont.getSerializer(); + this.contextCL = cont.getClassLoader(); + this.submitOrder = submitOrder; + } + + @Override + public DataLocation call() { + DataLocation dl = null; + final boolean isTask = object instanceof Task; + final int p = isTask ? ((Task) object).getPosition() : -1; + try { + Thread.currentThread().setContextClassLoader(contextCL); + if (traceEnabled) log.trace("serializing {} at position={}, submitOrder={}, job={}", toString(object), p, submitOrder, toString(isTask ? ((Task) object).getJob() : object)); + dl = IOHelper.serializeData(object, ser); + final int size = dl.getSize(); + if (traceEnabled) log.trace("serialized {} at position={}, submitOrder={}, job={}, size={}", toString(object), p, submitOrder, toString(isTask ? ((Task) object).getJob() : object), size); + } catch(final Throwable t) { + log.error(t.getMessage(), t); + try { + final JPPFExceptionResult result = (JPPFExceptionResult) HookFactory.invokeSingleHook(SerializationExceptionHook.class, "buildExceptionResult", object, t); + result.setPosition(p); + dl = IOHelper.serializeData(result, ser); + } catch(final Exception e2) { + log.error(e2.getMessage(), e2); + } + } + return dl; + } + + /** + * @param o the object from which to get a string description. + * @return a string description of the object. + */ + private static String toString(final Object o) { + if (o instanceof JPPFDistributedJob) { + final JPPFDistributedJob job = (JPPFDistributedJob) o; + return o.getClass().getSimpleName() + "[name=" + job.getName() + ", uuid=" + job.getUuid() + ']'; + } + return (o == null) ? "null" : o.getClass().getSimpleName(); + } +} \ No newline at end of file diff --git a/samples-pack/ExtendedClassLoading/src/org/jppf/example/extendedclassloading/node/NodeListener.java b/samples-pack/ExtendedClassLoading/src/org/jppf/example/extendedclassloading/node/NodeListener.java index c424842660..3db679f9b1 100644 --- a/samples-pack/ExtendedClassLoading/src/org/jppf/example/extendedclassloading/node/NodeListener.java +++ b/samples-pack/ExtendedClassLoading/src/org/jppf/example/extendedclassloading/node/NodeListener.java @@ -115,7 +115,7 @@ public void jobHeaderLoaded(final NodeLifeCycleEvent event) { // the same client, which will now be used by the node; this will cause the old class loader to be discarded boolean isNewCl = false; if ((currentURLs != null) && (currentURLs.length > 0)) { - cl = (AbstractJPPFClassLoader) event.getNode().resetTaskClassLoader(); + cl = (AbstractJPPFClassLoader) event.getNode().resetTaskClassLoader(event.getJob()); isNewCl = true; } // add all requested jar files to the class loader's classpath diff --git a/server/src/java/org/jppf/management/JPPFDriverAdmin.java b/server/src/java/org/jppf/management/JPPFDriverAdmin.java index 9015caf797..13da5e3c0a 100644 --- a/server/src/java/org/jppf/management/JPPFDriverAdmin.java +++ b/server/src/java/org/jppf/management/JPPFDriverAdmin.java @@ -25,7 +25,7 @@ import org.jppf.load.balancer.LoadBalancingInformation; import org.jppf.load.balancer.spi.JPPFBundlerFactory; import org.jppf.server.*; -import org.jppf.server.nio.nodeserver.AbstractBaseNodeContext; +import org.jppf.server.nio.nodeserver.BaseNodeContext; import org.jppf.utils.*; import org.jppf.utils.stats.*; import org.slf4j.*; @@ -101,9 +101,9 @@ public Collection nodesInformation(final NodeSelector select @Override public Collection nodesInformation(final NodeSelector selector, final boolean includePeers) { try { - final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector, includePeers, false); + final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector, includePeers, false); final List list = new ArrayList<>(nodes.size()); - for (final AbstractBaseNodeContext context : nodes) { + for (final BaseNodeContext context : nodes) { final JPPFManagementInfo info = context.getManagementInfo(); if (info != null) list.add(info); } @@ -126,10 +126,10 @@ public Integer nbIdleNodes(final NodeSelector selector) throws Exception { @Override public Integer nbIdleNodes(final NodeSelector selector, final boolean includePeers) throws Exception { - final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector, includePeers, false); + final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector, includePeers, false); if (nodes == null) return -1; int result = 0; - for (final AbstractBaseNodeContext node: nodes) { + for (final BaseNodeContext node: nodes) { final boolean idle = node.getIdle().get(); if (idle) result++; } @@ -145,9 +145,9 @@ public Collection idleNodesInformation() throws Exception { @Override public Collection idleNodesInformation(final NodeSelector selector) { try { - final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector, false, false); + final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector, false, false); final List list = new ArrayList<>(nodes.size()); - for (final AbstractBaseNodeContext node : nodes) { + for (final BaseNodeContext node : nodes) { if (node.getIdle().get()) { final JPPFManagementInfo info = node.getManagementInfo(); if (info != null) list.add(info); @@ -260,8 +260,8 @@ private JPPFBundlerFactory getBundlerFactory() { @Override public void toggleActiveState(final NodeSelector selector) throws Exception { - final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector); - for (final AbstractBaseNodeContext node: nodes) { + final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector); + for (final BaseNodeContext node: nodes) { if (driver.isAsyncNode()) driver.getAsyncNodeNioServer().activateNode(node.getUuid(), !node.isActive()); else driver.getNodeNioServer().activateNode(node.getUuid(), !node.isActive()); } @@ -269,16 +269,16 @@ public void toggleActiveState(final NodeSelector selector) throws Exception { @Override public Map getActiveState(final NodeSelector selector) throws Exception { - final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector); + final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector); final Map result = new HashMap<>(nodes.size()); - for (final AbstractBaseNodeContext node: nodes) result.put(node.getUuid(), node.isActive()); + for (final BaseNodeContext node: nodes) result.put(node.getUuid(), node.isActive()); return result; } @Override public void setActiveState(final NodeSelector selector, final boolean active) throws Exception { - final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector); - for (final AbstractBaseNodeContext node: nodes) { + final Set> nodes = selectionHelper.getChannels(selector == null ? NodeSelector.ALL_NODES : selector); + for (final BaseNodeContext node: nodes) { if (driver.isAsyncNode()) driver.getAsyncNodeNioServer().activateNode(node.getUuid(), active); else driver.getNodeNioServer().activateNode(node.getUuid(), active); } diff --git a/server/src/java/org/jppf/management/JPPFNodeAdmin.java b/server/src/java/org/jppf/management/JPPFNodeAdmin.java index 957a76badf..863f8f2fb1 100644 --- a/server/src/java/org/jppf/management/JPPFNodeAdmin.java +++ b/server/src/java/org/jppf/management/JPPFNodeAdmin.java @@ -202,7 +202,7 @@ public void resetTaskCounter() throws Exception { @Override public synchronized void setTaskCounter(final Integer n) throws Exception { if (debugEnabled) log.debug("node tasks counter reset to " + n + " requested"); - node.setTaskCount(n); + node.setExecutedTaskCount(n); nodeState.setNbTasksExecuted(n); } @@ -264,12 +264,14 @@ public void updateConfiguration(final Map configOverrides, final */ @Override public void cancelJob(final String jobId, final Boolean requeue) throws Exception { - if (debugEnabled) log.debug("Request to cancel jobuUid = '{}', requeue = {}", jobId, requeue); - if (jobId == null) return; - if (jobId.equals(node.getExecutionManager().getCurrentJobId())) { - node.getExecutionManager().setJobCancelled(true); - node.getExecutionManager().cancelAllTasks(true, requeue); - } else if (debugEnabled) log.debug("request to cancel job with uuid = '{}' which is not currently executing", jobId); + try { + if (debugEnabled) log.debug("Request to cancel jobuUid = '{}', requeue = {}", jobId, requeue); + if (jobId == null) return; + node.getExecutionManager().cancelJob(jobId, true, requeue); + } catch (final RuntimeException e) { + log.debug("error cancelling job with uuid={}:", jobId, e); + throw e; + } } @Override diff --git a/server/src/java/org/jppf/management/NodeSelectionHelper.java b/server/src/java/org/jppf/management/NodeSelectionHelper.java index 0abe0be151..37cb566252 100644 --- a/server/src/java/org/jppf/management/NodeSelectionHelper.java +++ b/server/src/java/org/jppf/management/NodeSelectionHelper.java @@ -24,6 +24,7 @@ import org.jppf.node.policy.ExecutionPolicy; import org.jppf.server.JPPFDriver; import org.jppf.server.nio.nodeserver.*; +import org.jppf.server.nio.nodeserver.async.AsyncJobScheduler; import org.slf4j.*; /** @@ -60,13 +61,13 @@ public NodeSelectionHelper(final JPPFDriver driver) { * @return a set of {@link AbstractNodeContext} instances. * @exclude */ - public boolean isNodeAccepted(final AbstractBaseNodeContext node, final NodeSelector selector) { + public boolean isNodeAccepted(final BaseNodeContext node, final NodeSelector selector) { if (selector == null) throw new IllegalArgumentException("selector cannot be null"); if (selector instanceof AllNodesSelector) return true; if (node.isPeer()) return false; if (selector instanceof ExecutionPolicySelector) { final ExecutionPolicy policy = ((ExecutionPolicySelector) selector).getPolicy(); - TaskQueueChecker.preparePolicy(policy, null, driver.getStatistics(), 0); + AsyncJobScheduler.preparePolicy(policy, null, driver.getStatistics(), 0); return policy.evaluate(node.getSystemInformation()); } return selector.accepts(node.getManagementInfo()); @@ -75,7 +76,7 @@ public boolean isNodeAccepted(final AbstractBaseNodeContext node, final NodeS @Override public boolean isNodeAccepted(final String nodeUuid, final NodeSelector selector) { if (nodeUuid == null) throw new IllegalArgumentException("node uuid cannot be null"); - final AbstractBaseNodeContext node = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(nodeUuid) : driver.getNodeNioServer().getConnection(nodeUuid); + final BaseNodeContext node = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(nodeUuid) : driver.getNodeNioServer().getConnection(nodeUuid); if (node == null) throw new IllegalArgumentException("unknown selector type: " + selector.getClass().getName()); return isNodeAccepted(node, selector); } @@ -85,7 +86,7 @@ public boolean isNodeAccepted(final String nodeUuid, final NodeSelector selector * @param selector the node selector used as a filter. * @return a set of {@link AbstractNodeContext} instances. */ - public Set> getChannels(final NodeSelector selector) { + public Set> getChannels(final NodeSelector selector) { return getChannels(selector, false, false); } @@ -96,12 +97,12 @@ public Set> getChannels(final NodeSelector selector) * @param forForwarding whether this is for a node forwarding request, in which case only nodes with a working jmx connection are selected. * @return a set of {@link AbstractNodeContext} instances. */ - public Set> getChannels(final NodeSelector selector, final boolean includePeers, final boolean forForwarding) { + public Set> getChannels(final NodeSelector selector, final boolean includePeers, final boolean forForwarding) { if (selector == null) throw new IllegalArgumentException("selector cannot be null"); if (selector instanceof ExecutionPolicySelector) return getChannels((ExecutionPolicySelector) selector, includePeers, forForwarding); - final Set> fullSet = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getAllChannelsAsSet() : driver.getNodeNioServer().getAllChannelsAsSet(); - final Set> result = new HashSet<>(); - for (final AbstractBaseNodeContext ctx : fullSet) { + final Set> fullSet = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getAllChannelsAsSet() : driver.getNodeNioServer().getAllChannelsAsSet(); + final Set> result = new HashSet<>(); + for (final BaseNodeContext ctx : fullSet) { if (nodeAccepted(selector, ctx, includePeers, forForwarding)) result.add(ctx); } if (traceEnabled) log.trace("got {} results", result.size()); @@ -115,13 +116,13 @@ public Set> getChannels(final NodeSelector selector, * @param forForwarding whether this is for a node forwarding request, in which case only nodes with a working jmx connection are selected. * @return a {@link Set} of {@link AbstractNodeContext} instances. */ - private Set> getChannels(final ExecutionPolicySelector selector, final boolean includePeers, final boolean forForwarding) { + private Set> getChannels(final ExecutionPolicySelector selector, final boolean includePeers, final boolean forForwarding) { final ExecutionPolicy policy = selector.getPolicy(); - if (policy.getContext() == null) TaskQueueChecker.preparePolicy(policy, null, driver.getStatistics(), 0); - final Set> result = new HashSet<>(); - final List> allChannels = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getAllChannels() : driver.getNodeNioServer().getAllChannels(); - TaskQueueChecker.preparePolicy(policy, null, driver.getStatistics(), 0); - for (final AbstractBaseNodeContext context : allChannels) { + if (policy.getContext() == null) AsyncJobScheduler.preparePolicy(policy, null, driver.getStatistics(), 0); + final Set> result = new HashSet<>(); + final List> allChannels = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getAllChannels() : driver.getNodeNioServer().getAllChannels(); + AsyncJobScheduler.preparePolicy(policy, null, driver.getStatistics(), 0); + for (final BaseNodeContext context : allChannels) { if (nodeAccepted(selector, context, includePeers, forForwarding)) result.add(context); } if (traceEnabled) log.trace("got {} results", result.size()); @@ -133,7 +134,7 @@ private Set> getChannels(final ExecutionPolicySelecto * @param context the node for which to get the management info. * @return a {@link JPPFManagementInfo} instyance, or {@code null}. */ - private static JPPFManagementInfo getManagementInfo(final AbstractBaseNodeContext context) { + private static JPPFManagementInfo getManagementInfo(final BaseNodeContext context) { JPPFManagementInfo info = context.getManagementInfo(); if (info == null) { final JPPFSystemInformation sysInfo = context.getSystemInformation(); @@ -155,10 +156,10 @@ private static JPPFManagementInfo getManagementInfo(final AbstractBaseNodeContex public int getNbChannels(final NodeSelector selector, final boolean includePeers, final boolean forForwarding) { if (selector == null) throw new IllegalArgumentException("selector cannot be null"); if (selector instanceof ExecutionPolicySelector) return getNbChannels((ExecutionPolicySelector) selector, includePeers, forForwarding); - final Set> fullSet = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getAllChannelsAsSet() : driver.getNodeNioServer().getAllChannelsAsSet(); + final Set> fullSet = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getAllChannelsAsSet() : driver.getNodeNioServer().getAllChannelsAsSet(); //if (selector instanceof AllNodesSelector) return fullSet.size(); int result = 0; - for (final AbstractBaseNodeContext ctx : fullSet) { + for (final BaseNodeContext ctx : fullSet) { if (nodeAccepted(selector, ctx, includePeers, forForwarding)) result++; } return result; @@ -173,11 +174,11 @@ public int getNbChannels(final NodeSelector selector, final boolean includePeers */ private int getNbChannels(final ExecutionPolicySelector selector, final boolean includePeers, final boolean forForwarding) { final ExecutionPolicy policy = selector.getPolicy(); - if (policy.getContext() == null) TaskQueueChecker.preparePolicy(policy, null, driver.getStatistics(), 0); + if (policy.getContext() == null) AsyncJobScheduler.preparePolicy(policy, null, driver.getStatistics(), 0); int result = 0; - final List> allChannels = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getAllChannels() : driver.getNodeNioServer().getAllChannels(); - TaskQueueChecker.preparePolicy(policy, null, driver.getStatistics(), 0); - for (final AbstractBaseNodeContext context : allChannels) { + final List> allChannels = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getAllChannels() : driver.getNodeNioServer().getAllChannels(); + AsyncJobScheduler.preparePolicy(policy, null, driver.getStatistics(), 0); + for (final BaseNodeContext context : allChannels) { if (nodeAccepted(selector, context, includePeers, forForwarding)) result++; } return result; @@ -188,7 +189,7 @@ private int getNbChannels(final ExecutionPolicySelector selector, final boolean * @param ctx the context associated witht he node. * @return {@code true} if node has a working JMX connection, {@code false} otherwise. */ - private static boolean hasWorkingJmxConnection(final AbstractBaseNodeContext ctx) { + private static boolean hasWorkingJmxConnection(final BaseNodeContext ctx) { if (ctx.isPeer()) return true; final JMXNodeConnectionWrapper jmx = ctx.getJmxConnection(); return (jmx != null) && jmx.isConnected(); @@ -202,7 +203,7 @@ private static boolean hasWorkingJmxConnection(final AbstractBaseNodeContext * @param forForwarding whether this is for a node forwarding request, in which case only nodes with a working jmx connection are selected. * @return {@code true} if the node is accepted, {@code false} otherwise. */ - private static boolean nodeAccepted(final NodeSelector selector, final AbstractBaseNodeContext context, final boolean includePeers, final boolean forForwarding) { + private static boolean nodeAccepted(final NodeSelector selector, final BaseNodeContext context, final boolean includePeers, final boolean forForwarding) { if (!includePeers && context.isPeer()) return false; final boolean hasJmx = hasWorkingJmxConnection(context); if (forForwarding && !hasJmx) return false; diff --git a/server/src/java/org/jppf/management/forwarding/ForwardingNotificationDispatcher.java b/server/src/java/org/jppf/management/forwarding/ForwardingNotificationDispatcher.java index 7300db3ab1..9456bc21b5 100644 --- a/server/src/java/org/jppf/management/forwarding/ForwardingNotificationDispatcher.java +++ b/server/src/java/org/jppf/management/forwarding/ForwardingNotificationDispatcher.java @@ -52,7 +52,7 @@ class ForwardingNotificationDispatcher { /** * Connection to the node. */ - private final AbstractBaseNodeContext node; + private final BaseNodeContext node; /** * The node uuid. */ @@ -79,7 +79,7 @@ class ForwardingNotificationDispatcher { * Initialize this dispatcher with the specified node uuid. * @param node connection to the node. */ - public ForwardingNotificationDispatcher(final AbstractBaseNodeContext node) { + public ForwardingNotificationDispatcher(final BaseNodeContext node) { this.node = node; this.nodeUuid = node.getUuid(); } diff --git a/server/src/java/org/jppf/management/forwarding/ForwardingNotificationManager.java b/server/src/java/org/jppf/management/forwarding/ForwardingNotificationManager.java index 530486c846..30bc7ac4f2 100644 --- a/server/src/java/org/jppf/management/forwarding/ForwardingNotificationManager.java +++ b/server/src/java/org/jppf/management/forwarding/ForwardingNotificationManager.java @@ -109,12 +109,12 @@ public void addNotificationListener(final String listenerID, final NodeSelector private void addNotificationListener(final NotificationListenerWrapper wrapper) { if (debugEnabled) log.debug("adding notification listener {}", wrapper); final NodeSelector selector = wrapper.getSelector(); - final Set> nodes = selectionHelper.getChannels(selector); + final Set> nodes = selectionHelper.getChannels(selector); if (debugEnabled) log.debug("found {} nodes", nodes.size()); lock.lock(); try { forwardingHelper.setListener(wrapper.getListenerID(), wrapper); - for (AbstractBaseNodeContext node: nodes) addNotificationListener(node, wrapper); + for (BaseNodeContext node: nodes) addNotificationListener(node, wrapper); } finally { lock.unlock(); } @@ -125,7 +125,7 @@ private void addNotificationListener(final NotificationListenerWrapper wrapper) * @param node the node to which to add the listener. * @param wrapper the listener to add. */ - private void addNotificationListener(final AbstractBaseNodeContext node, final NotificationListenerWrapper wrapper) { + private void addNotificationListener(final BaseNodeContext node, final NotificationListenerWrapper wrapper) { final String uuid = node.getUuid(); final String mbean = wrapper.getMBeanName(); if (debugEnabled) log.debug("adding notification listener for node={} : {}", uuid, wrapper); @@ -165,13 +165,13 @@ public void removeNotificationListener(final String listenerID) throws ListenerN public void removeNotificationListener(final NotificationListenerWrapper wrapper) throws ListenerNotFoundException { if (debugEnabled) log.debug("removing notification listeners for {}", wrapper); final NodeSelector selector = wrapper.getSelector(); - final Set> nodes = forwarder.getSelectionHelper().getChannels(selector); + final Set> nodes = forwarder.getSelectionHelper().getChannels(selector); final Runnable r = new Runnable() { @Override public void run() { lock.lock(); try { - for (AbstractBaseNodeContext node: nodes) removeNotificationListener(node, wrapper); + for (BaseNodeContext node: nodes) removeNotificationListener(node, wrapper); } finally { lock.unlock(); } @@ -185,7 +185,7 @@ public void run() { * @param node the node from which to remove the listener. * @param wrapper the listener to rmeove. */ - private void removeNotificationListener(final AbstractBaseNodeContext node, final NotificationListenerWrapper wrapper) { + private void removeNotificationListener(final BaseNodeContext node, final NotificationListenerWrapper wrapper) { if (debugEnabled) log.debug("removing notification listener {} for node {}", wrapper, node); final String mbean = wrapper.getMBeanName(); final String uuid = node.getUuid(); @@ -209,7 +209,7 @@ public void nodeConnected(final NodeConnectionEvent event) { if (debugEnabled) log.debug("handling new connected node {},", info); if ((info == null) || (info.getPort() < 0) || (info.getHost() == null)) return; final String uuid = info.getUuid(); - final AbstractBaseNodeContext node = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(uuid) : driver.getNodeNioServer().getConnection(uuid); + final BaseNodeContext node = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(uuid) : driver.getNodeNioServer().getConnection(uuid); if (debugEnabled) log.debug("new connected node {}", node); if (node == null) return; lock.lock(); @@ -227,7 +227,7 @@ public void nodeDisconnected(final NodeConnectionEvent event) { final JPPFManagementInfo info = event.getNodeInformation(); if (debugEnabled) log.debug("handling disconnected node {}", info); final String uuid = info.getUuid(); - final AbstractBaseNodeContext node = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(uuid) : driver.getNodeNioServer().getConnection(uuid); + final BaseNodeContext node = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(uuid) : driver.getNodeNioServer().getConnection(uuid); if (node == null) return; final Runnable r = new Runnable() { @Override diff --git a/server/src/java/org/jppf/management/forwarding/JPPFNodeForwarding.java b/server/src/java/org/jppf/management/forwarding/JPPFNodeForwarding.java index 06eede1095..03a05e8d10 100644 --- a/server/src/java/org/jppf/management/forwarding/JPPFNodeForwarding.java +++ b/server/src/java/org/jppf/management/forwarding/JPPFNodeForwarding.java @@ -97,7 +97,7 @@ public JPPFNodeForwarding(final JPPFDriver driver) { @Override public Map forwardInvoke(final NodeSelector selector, final String name, final String methodName, final Object[] params, final String[] signature) throws Exception { - final Set> channels = selectionHelper.getChannels(selector); + final Set> channels = selectionHelper.getChannels(selector); if (debugEnabled) log.debug("invoking {}() on mbean={} for selector={} ({} channels)", new Object[] {methodName, name, selector, channels.size()}); return forward(JMXHelper.INVOKE, channels, name, methodName, params, signature); } @@ -109,13 +109,13 @@ public Map forwardInvoke(final NodeSelector selector, final Stri @Override public Map forwardGetAttribute(final NodeSelector selector, final String name, final String attribute) throws Exception { - final Set> channels = selectionHelper.getChannels(selector); + final Set> channels = selectionHelper.getChannels(selector); return forward(JMXHelper.GET_ATTRIBUTE, channels, name, attribute); } @Override public Map forwardSetAttribute(final NodeSelector selector, final String name, final String attribute, final Object value) throws Exception { - final Set> channels = selectionHelper.getChannels(selector); + final Set> channels = selectionHelper.getChannels(selector); return forward(JMXHelper.SET_ATTRIBUTE, channels, name, attribute, value); } @@ -297,13 +297,13 @@ public Map provisionSlaveNodes(final NodeSelector selector, fina * Additionally, each result may be {@code null}, in particular if the invoked method has a {@code void} return type. * @throws Exception if the invocation failed. */ - Map forward(final byte type, final Set> nodes, final String mbeanName, final String memberName, final Object...params) throws Exception { + Map forward(final byte type, final Set> nodes, final String mbeanName, final String memberName, final Object...params) throws Exception { try { final int size = nodes.size(); if (size <= 0) return Collections.emptyMap(); final ForwardCallback callback = new ForwardCallback(size); AbstractForwardingTask task; - for (final AbstractBaseNodeContext node: nodes) { + for (final BaseNodeContext node: nodes) { final JMXConnectionWrapper jmx = node.getJmxConnection(); switch(type) { case JMXHelper.INVOKE: diff --git a/server/src/java/org/jppf/server/DriverInitializer.java b/server/src/java/org/jppf/server/DriverInitializer.java index 05e067b971..ecca776b8f 100644 --- a/server/src/java/org/jppf/server/DriverInitializer.java +++ b/server/src/java/org/jppf/server/DriverInitializer.java @@ -386,7 +386,7 @@ public void handleNotification(final Notification notification, final Object han final String nodeUuid = (String) notif.getSource(); final TypedProperties nodeConfig = (TypedProperties) notif.getUserData(); if (debugEnabled) log.debug("received notification for node {}, nb threads={}", nodeUuid, nodeConfig.get(JPPFProperties.PROCESSING_THREADS)); - final AbstractBaseNodeContext node = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(nodeUuid) : driver.getNodeNioServer().getConnection(nodeUuid); + final BaseNodeContext node = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(nodeUuid) : driver.getNodeNioServer().getConnection(nodeUuid); if (node == null) return; synchronized(node.getMonitor()) { final TypedProperties oldConfig = node.getSystemInformation().getJppf(); diff --git a/server/src/java/org/jppf/server/debug/ServerDebug.java b/server/src/java/org/jppf/server/debug/ServerDebug.java index 0a89db7302..e210beb2f8 100644 --- a/server/src/java/org/jppf/server/debug/ServerDebug.java +++ b/server/src/java/org/jppf/server/debug/ServerDebug.java @@ -228,16 +228,16 @@ private Set> nodeClassLoaderSet() { * @return a set of {@link ChannelWrapper} instances. */ private Set> nodeSet() { - final List> list = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getAllChannels() : driver.getNodeNioServer().getAllChannels(); + final List> list = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getAllChannels() : driver.getNodeNioServer().getAllChannels(); final Set> set = new HashSet<>(list); return set; } @Override public String taskQueueCheckerChannels() { - final List> list = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getTaskQueueChecker().getIdleChannels() : driver.getNodeNioServer().getIdleChannels(); + final List> list = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getJobScheduler().getIdleChannels() : driver.getNodeNioServer().getIdleChannels(); final StringBuilder sb = new StringBuilder(); - for (final AbstractBaseNodeContext ctx: list) sb.append(ctx).append('\n'); + for (final BaseNodeContext ctx: list) sb.append(ctx).append('\n'); return sb.toString(); } diff --git a/server/src/java/org/jppf/server/job/management/DriverJobManagement.java b/server/src/java/org/jppf/server/job/management/DriverJobManagement.java index 8a907a76f6..0c25f28e7a 100644 --- a/server/src/java/org/jppf/server/job/management/DriverJobManagement.java +++ b/server/src/java/org/jppf/server/job/management/DriverJobManagement.java @@ -116,7 +116,7 @@ private void resumeJob(final ServerJob job) throws Exception { } if (debugEnabled) log.debug("Request to resume job '" + job.getJob().getName() + '\''); job.setSuspended(false, false); - if (driver.isAsyncNode()) driver.getAsyncNodeNioServer().getTaskQueueChecker().wakeUp(); + if (driver.isAsyncNode()) driver.getAsyncNodeNioServer().getJobScheduler().wakeUp(); else driver.getNodeNioServer().getTaskQueueChecker().wakeUp(); } @@ -347,7 +347,7 @@ public void updateJobs(final JobSelector selector, final JobSLA sla, final JobMe } job.update(driver, sla, metadata); } - if (driver.isAsyncNode()) driver.getAsyncNodeNioServer().getTaskQueueChecker().wakeUp(); + if (driver.isAsyncNode()) driver.getAsyncNodeNioServer().getJobScheduler().wakeUp(); else driver.getNodeNioServer().getTaskQueueChecker().wakeUp(); } diff --git a/server/src/java/org/jppf/server/nio/classloader/node/NodeClassNioServer.java b/server/src/java/org/jppf/server/nio/classloader/node/NodeClassNioServer.java index d261302e06..ac1399d08e 100644 --- a/server/src/java/org/jppf/server/nio/classloader/node/NodeClassNioServer.java +++ b/server/src/java/org/jppf/server/nio/classloader/node/NodeClassNioServer.java @@ -158,7 +158,7 @@ public void closeConnection(final ChannelWrapper channel) { } if (context.isPeer()) { try { - final AbstractBaseNodeContext ctx = getDriver().isAsyncNode() ? getDriver().getAsyncNodeNioServer().getConnection(uuid) : getDriver().getNodeNioServer().getConnection(uuid); + final BaseNodeContext ctx = getDriver().isAsyncNode() ? getDriver().getAsyncNodeNioServer().getConnection(uuid) : getDriver().getNodeNioServer().getConnection(uuid); if (ctx != null) ctx.handleException(null); } catch(final Exception e) { if (debugEnabled) log.debug(e.getMessage(), e); diff --git a/server/src/java/org/jppf/server/nio/classloader/node/WaitingNodeRequestState.java b/server/src/java/org/jppf/server/nio/classloader/node/WaitingNodeRequestState.java index e8f8664cfa..3d4bd91228 100644 --- a/server/src/java/org/jppf/server/nio/classloader/node/WaitingNodeRequestState.java +++ b/server/src/java/org/jppf/server/nio/classloader/node/WaitingNodeRequestState.java @@ -28,7 +28,7 @@ import org.jppf.nio.ChannelWrapper; import org.jppf.server.nio.classloader.*; import org.jppf.server.nio.classloader.client.*; -import org.jppf.server.nio.nodeserver.AbstractBaseNodeContext; +import org.jppf.server.nio.nodeserver.BaseNodeContext; import org.jppf.utils.*; import org.jppf.utils.configuration.JPPFProperties; import org.slf4j.*; @@ -85,7 +85,7 @@ public NodeClassTransition performTransition(final ChannelWrapper channel) th server.closeConnection(channel); if (context.isPeer()) { final String uuid = (String) res.getData(ResourceIdentifier.NODE_UUID); - final AbstractBaseNodeContext ctx = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(uuid) : driver.getNodeNioServer().getConnection(uuid); + final BaseNodeContext ctx = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(uuid) : driver.getNodeNioServer().getConnection(uuid); if (ctx != null) ctx.handleException(null); } return null; diff --git a/server/src/java/org/jppf/server/nio/heartbeat/HeartbeatContext.java b/server/src/java/org/jppf/server/nio/heartbeat/HeartbeatContext.java index 4649db05e5..f7d4f5413f 100644 --- a/server/src/java/org/jppf/server/nio/heartbeat/HeartbeatContext.java +++ b/server/src/java/org/jppf/server/nio/heartbeat/HeartbeatContext.java @@ -130,7 +130,7 @@ void heartbeatFailed() { if (debugEnabled) log.debug("node {} failed to respond to heartbeat messages, closing the associated node channels", this); final JPPFDriver driver = server.driver; if (server.getIdentifier() == JPPFIdentifiers.NODE_HEARTBEAT_CHANNEL) { - final AbstractBaseNodeContext nodeContext = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(uuid) : driver.getNodeNioServer().getConnection(uuid); + final BaseNodeContext nodeContext = driver.isAsyncNode() ? driver.getAsyncNodeNioServer().getConnection(uuid) : driver.getNodeNioServer().getConnection(uuid); if (nodeContext != null) { if (driver.isAsyncNode()) driver.getAsyncNodeNioServer().connectionFailed(nodeContext); driver.getNodeNioServer().connectionFailed(nodeContext.getChannel()); diff --git a/server/src/java/org/jppf/server/nio/nodeserver/AbstractNodeContext.java b/server/src/java/org/jppf/server/nio/nodeserver/AbstractNodeContext.java index d199e7ef56..c21f39efb6 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/AbstractNodeContext.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/AbstractNodeContext.java @@ -41,7 +41,7 @@ * Context associated with a channel serving tasks to a node. * @author Laurent Cohen */ -public abstract class AbstractNodeContext extends AbstractNioContext implements AbstractBaseNodeContext { +public abstract class AbstractNodeContext extends AbstractNioContext implements BaseNodeContext { /** * Logger for this class. */ @@ -170,7 +170,7 @@ void serializeBundle() throws Exception { final TaskBundle taskBundle = bundle.getJob(); final AbstractTaskBundleMessage message = newMessage(); if (!taskBundle.isHandshake()) { - taskBundle.setParameter(BundleParameter.NODE_BUNDLE_ID, bundle.getId()); + taskBundle.setBundleId(bundle.getId()); if (!isPeer()) taskBundle.removeParameter(BundleParameter.TASK_MAX_RESUBMITS); else if (bundle.getServerJob().isPersistent()) taskBundle.setParameter(BundleParameter.ALREADY_PERSISTED_P2P, true); } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/AbstractTaskQueueChecker.java b/server/src/java/org/jppf/server/nio/nodeserver/AbstractTaskQueueChecker.java index 14798512f3..8ce29a0d2e 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/AbstractTaskQueueChecker.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/AbstractTaskQueueChecker.java @@ -64,7 +64,7 @@ abstract class AbstractTaskQueueChecker extends ThreadSynchronization implements /** * The list of idle node channels. */ - final Set> idleChannels = new LinkedHashSet<>(); + final Set> idleChannels = new LinkedHashSet<>(); /** * Holds information about the execution context. */ @@ -135,7 +135,7 @@ int getNbIdleChannels() { * Add a channel to the list of idle channels. * @param channel the channel to add to the list. */ - public void addIdleChannel(final AbstractBaseNodeContext channel) { + public void addIdleChannel(final BaseNodeContext channel) { if (channel == null) { final String message = "channel is null"; log.error(message); @@ -172,7 +172,7 @@ public void addIdleChannel(final AbstractBaseNodeContext channel) { * @param channel the channel to remove from the list. * @return a reference to the removed channel. */ - AbstractBaseNodeContext removeIdleChannel(final AbstractBaseNodeContext channel) { + BaseNodeContext removeIdleChannel(final BaseNodeContext channel) { if (debugEnabled) log.debug("removing idle channel {}", channel); boolean removed = false; synchronized(idleChannels) { @@ -191,7 +191,7 @@ AbstractBaseNodeContext removeIdleChannel(final AbstractBaseNodeContext ch * Asynchronously remove a channel from the list of idle channels. * @param channel the channel to remove from the list. */ - public void removeIdleChannelAsync(final AbstractBaseNodeContext channel) { + public void removeIdleChannelAsync(final BaseNodeContext channel) { if (debugEnabled) log.debug("request to remove idle channel {}", channel); channelsExecutor.execute(() -> removeIdleChannel(channel)); } @@ -200,7 +200,7 @@ public void removeIdleChannelAsync(final AbstractBaseNodeContext channel) { * Get the list of idle channels. * @return a new copy of the underlying list of idle channels. */ - public List> getIdleChannels() { + public List> getIdleChannels() { synchronized (idleChannels) { return new ArrayList<>(idleChannels); } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/AbstractBaseNodeContext.java b/server/src/java/org/jppf/server/nio/nodeserver/BaseNodeContext.java similarity index 98% rename from server/src/java/org/jppf/server/nio/nodeserver/AbstractBaseNodeContext.java rename to server/src/java/org/jppf/server/nio/nodeserver/BaseNodeContext.java index 4e31b051ef..aacb085d67 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/AbstractBaseNodeContext.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/BaseNodeContext.java @@ -33,7 +33,7 @@ * @param the type of state. * @author Laurent Cohen */ -public interface AbstractBaseNodeContext> extends NioContext, ExecutorChannel { +public interface BaseNodeContext> extends NioContext, ExecutorChannel { /** * @return this node context's attributes. */ diff --git a/server/src/java/org/jppf/server/nio/nodeserver/LocalNodeMessage.java b/server/src/java/org/jppf/server/nio/nodeserver/LocalNodeMessage.java index a58c47ed6b..1e40ef06cc 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/LocalNodeMessage.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/LocalNodeMessage.java @@ -31,7 +31,7 @@ public class LocalNodeMessage extends AbstractTaskBundleMessage { * Build this nio message. * @param context the channel to read from or write to. */ - public LocalNodeMessage(final AbstractBaseNodeContext context) { + public LocalNodeMessage(final BaseNodeContext context) { super(context); } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/NodeConnectionCompletionListener.java b/server/src/java/org/jppf/server/nio/nodeserver/NodeConnectionCompletionListener.java index f624aa7e14..5aaef2dad7 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/NodeConnectionCompletionListener.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/NodeConnectionCompletionListener.java @@ -27,5 +27,5 @@ public interface NodeConnectionCompletionListener { * Called when a channel is connected. * @param context the connected channel. */ - void nodeConnected(AbstractBaseNodeContext context); + void nodeConnected(BaseNodeContext context); } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/NodeContextAttributes.java b/server/src/java/org/jppf/server/nio/nodeserver/NodeContextAttributes.java index 3022a86f85..f17de7d220 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/NodeContextAttributes.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/NodeContextAttributes.java @@ -99,7 +99,7 @@ public class NodeContextAttributes { /** * */ - private final AbstractBaseNodeContext context; + private final BaseNodeContext context; /** * The load-balancer persistence manager. */ @@ -127,7 +127,7 @@ public class NodeContextAttributes { * @param bundlerHandler the load-balancer persistence manager. * @param listener the listener to conneciton completion event. */ - public NodeContextAttributes(final AbstractBaseNodeContext context, final LoadBalancerPersistenceManager bundlerHandler, final NodeConnectionCompletionListener listener) { + public NodeContextAttributes(final BaseNodeContext context, final LoadBalancerPersistenceManager bundlerHandler, final NodeConnectionCompletionListener listener) { this.context = context; this.bundlerHandler = bundlerHandler; this.listener = listener; @@ -393,6 +393,7 @@ public void removeExecutionStatusListener(final ExecutorChannelStatusListener li */ public void fireExecutionStatusChanged(final ExecutorStatus oldValue, final ExecutorStatus newValue) { if (oldValue == newValue) return; + if (debugEnabled) log.debug("changing execution status from {} to {} on {}", oldValue, newValue, this); final ExecutorChannelStatusEvent event = new ExecutorChannelStatusEvent(context, oldValue, newValue); for (final ExecutorChannelStatusListener listener : executorChannelListeners) listener.executionStatusChanged(event); } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/NodeDispatchTimeoutAction.java b/server/src/java/org/jppf/server/nio/nodeserver/NodeDispatchTimeoutAction.java index 704381bdc5..9355ea1a24 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/NodeDispatchTimeoutAction.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/NodeDispatchTimeoutAction.java @@ -50,7 +50,7 @@ public class NodeDispatchTimeoutAction implements Runnable { /** * Context for the dispatch node. */ - private final AbstractBaseNodeContext context; + private final BaseNodeContext context; /** * Initialize this action with the specified server and nodBundle. @@ -58,7 +58,7 @@ public class NodeDispatchTimeoutAction implements Runnable { * @param nodeBundle the bundle sent to the node. * @param context the context for the dispatch node, may be null for an offline node. */ - public NodeDispatchTimeoutAction(final OfflineNodeHandler server, final ServerTaskBundleNode nodeBundle, final AbstractBaseNodeContext context) { + public NodeDispatchTimeoutAction(final OfflineNodeHandler server, final ServerTaskBundleNode nodeBundle, final BaseNodeContext context) { if (server == null) throw new IllegalArgumentException("server cannot be null"); if (nodeBundle == null) throw new IllegalArgumentException("node bundle cannot be null"); this.server = server; diff --git a/server/src/java/org/jppf/server/nio/nodeserver/NodeJMXWrapperListener.java b/server/src/java/org/jppf/server/nio/nodeserver/NodeJMXWrapperListener.java index 1a25ec3cf0..c1186ec552 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/NodeJMXWrapperListener.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/NodeJMXWrapperListener.java @@ -38,7 +38,7 @@ public class NodeJMXWrapperListener implements JMXWrapperListener { /** * The node context. */ - private final AbstractBaseNodeContext context; + private final BaseNodeContext context; /** * */ @@ -49,7 +49,7 @@ public class NodeJMXWrapperListener implements JMXWrapperListener { * @param context the node context. * @param listener the listener to conneciton completion event. */ - public NodeJMXWrapperListener(final AbstractBaseNodeContext context, final NodeConnectionCompletionListener listener) { + public NodeJMXWrapperListener(final BaseNodeContext context, final NodeConnectionCompletionListener listener) { this.context = context; this.listener = listener; } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/NodeNioServer.java b/server/src/java/org/jppf/server/nio/nodeserver/NodeNioServer.java index b7d6c4e246..a6d412050d 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/NodeNioServer.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/NodeNioServer.java @@ -124,9 +124,9 @@ public NodeNioServer(final JPPFDriver driver, final JPPFPriorityQueue queue, fin if (queue == null) throw new IllegalArgumentException("queue is null"); this.queue = queue; - this.queue.setCallableAllConnections(new Callable>>() { + this.queue.setCallableAllConnections(new Callable>>() { @Override - public List> call() throws Exception { + public List> call() throws Exception { return getAllChannels(); } }); @@ -356,7 +356,7 @@ public JPPFBundlerFactory getBundlerFactory() { * Get all the node connections handled by this server. * @return a list of ChannelWrapper instances. */ - public List> getAllChannels() { + public List> getAllChannels() { return new ArrayList<>(allConnections.values()); } @@ -364,7 +364,7 @@ public List> getAllChannels() { * Get all the node connections handled by this server. * @return a set of ChannelWrapper instances. */ - public Set> getAllChannelsAsSet() { + public Set> getAllChannelsAsSet() { return new HashSet<>(allConnections.values()); } @@ -372,7 +372,7 @@ public Set> getAllChannelsAsSet() { * Get the list of currently idle channels. * @return a list of AbstractNodeContext instances. */ - public List> getIdleChannels() { + public List> getIdleChannels() { return taskQueueChecker.getIdleChannels(); } @@ -427,7 +427,7 @@ public void removeAllConnections() { } @Override - public void nodeConnected(final AbstractBaseNodeContext context) { + public void nodeConnected(final BaseNodeContext context) { final JPPFManagementInfo info = context.getManagementInfo(); if (context.getChannel().isOpen()) { peerHandler.onNodeConnected(context); diff --git a/server/src/java/org/jppf/server/nio/nodeserver/NodeReservationHandler.java b/server/src/java/org/jppf/server/nio/nodeserver/NodeReservationHandler.java index 90205f5461..ae55630a8c 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/NodeReservationHandler.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/NodeReservationHandler.java @@ -96,7 +96,7 @@ public NodeReservationHandler(final JPPFDriver driver) { * @param job the job for which the node is to be reserved. * @param node the node to restart and reserve. */ - public void doReservation(final ServerJob job, final AbstractBaseNodeContext node) { + public void doReservation(final ServerJob job, final BaseNodeContext node) { if (debugEnabled) log.debug("reserving node {} for job {}", node.getUuid(), job.getUuid()); synchronized(this) { pendingMap.put(node.getUuid(), job.getUuid()); @@ -109,7 +109,7 @@ public void doReservation(final ServerJob job, final AbstractBaseNodeContext * Remove the job reservation for the specified job, if any. * @param node the node for which to remove the reservation. */ - public synchronized void removeReservation(final AbstractBaseNodeContext node) { + public synchronized void removeReservation(final BaseNodeContext node) { if (node == null) return; final String nodeUuid = node.getUuid(); if (nodeUuid == null) return; @@ -157,7 +157,7 @@ public synchronized void removeJobReservations(final String jobUuid) { * @param node the node to check. * @return the UUID of the corresponding job, or {@code null} if the node doesn't have a pending reservation. */ - public synchronized String getPendingJobUUID(final AbstractBaseNodeContext node) { + public synchronized String getPendingJobUUID(final BaseNodeContext node) { return pendingMap.get(node.getUuid()); } @@ -166,7 +166,7 @@ public synchronized String getPendingJobUUID(final AbstractBaseNodeContext no * @param node the node to check. * @return the UUID of the corresponding job, or {@code null} if the node is not reserved. */ - public synchronized String getReadyJobUUID(final AbstractBaseNodeContext node) { + public synchronized String getReadyJobUUID(final BaseNodeContext node) { return readyMap.get(node.getUuid()); } @@ -216,7 +216,7 @@ public synchronized int getNbReservedNodes(final String jobUuid) { * @param node . * @return . */ - public boolean transitionReservation(final AbstractBaseNodeContext node) { + public boolean transitionReservation(final BaseNodeContext node) { final TypedProperties config = node.getSystemInformation().getJppf(); final String reservedJobUuid = config.get(JPPFProperties.NODE_RESERVED_JOB); if (reservedJobUuid != null) { @@ -291,14 +291,14 @@ private class NodeReservationTask implements Runnable { /** * The node to restart and reserve. */ - private final AbstractBaseNodeContext node; + private final BaseNodeContext node; /** * Initialize this task with the specified node for the specified job. * @param job the job for which the node is to be reserved. * @param node the node to restart and reserve. */ - public NodeReservationTask(final ServerJob job, final AbstractBaseNodeContext node) { + public NodeReservationTask(final ServerJob job, final BaseNodeContext node) { this.job = job; this.node = node; } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/NodeServerUtils.java b/server/src/java/org/jppf/server/nio/nodeserver/NodeServerUtils.java index 0783020ce4..a63949f530 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/NodeServerUtils.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/NodeServerUtils.java @@ -87,7 +87,7 @@ public static Pair getNodeIdentifier(final JPPFBundlerFactory fa * @return a pair of string representing the clear string (keft side) and resulting unique string identifier for the node (right side). * @throws Exception if any error occurs. */ - public static Pair getNodeIdentifier(final JPPFBundlerFactory factory, final AbstractBaseNodeContext channel, final JPPFSystemInformation info) throws Exception { + public static Pair getNodeIdentifier(final JPPFBundlerFactory factory, final BaseNodeContext channel, final JPPFSystemInformation info) throws Exception { final StringBuilder sb = new StringBuilder(); final String ip = NetworkUtils.getNonLocalHostAddress(); sb.append('[').append(ip == null ? "localhost" : ip); @@ -134,7 +134,7 @@ public static String getChannelHost(final ChannelWrapper channel) throws Exce * @return the remote host name as a string. * @throws Exception if any error occurs. */ - public static String getChannelHost(final AbstractBaseNodeContext context) throws Exception { + public static String getChannelHost(final BaseNodeContext context) throws Exception { if (!context.isLocal()) { final SocketChannel ch = context.getSocketChannel(); return ((InetSocketAddress) (ch.getRemoteAddress())).getHostString(); diff --git a/server/src/java/org/jppf/server/nio/nodeserver/PeerAttributesHandler.java b/server/src/java/org/jppf/server/nio/nodeserver/PeerAttributesHandler.java index f63d3277a6..063b7ebae1 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/PeerAttributesHandler.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/PeerAttributesHandler.java @@ -104,7 +104,7 @@ public int getTotalThreads() { * @param peer the peer to update. * @param props the updated attributes. */ - private static void updatePeer(final AbstractBaseNodeContext peer, final TypedProperties props) { + private static void updatePeer(final BaseNodeContext peer, final TypedProperties props) { final int newNodes = props.getInt(PEER_TOTAL_NODES, 0); final int newThreads = props.getInt(PEER_TOTAL_THREADS, 0); final JPPFSystemInformation info = peer.getSystemInformation(); @@ -126,7 +126,7 @@ private static void updatePeer(final AbstractBaseNodeContext peer, final Type * Called when a node gets closed. * @param context the node context. */ - public void onCloseNode(final AbstractBaseNodeContext context) { + public void onCloseNode(final BaseNodeContext context) { if (!context.isPeer()) { totalNodes.decrementAndGet(); final JPPFSystemInformation sys = context.getSystemInformation(); @@ -151,7 +151,7 @@ public void onCloseNode(final AbstractBaseNodeContext context) { * Called when a node gets connected to the server. * @param context the node context. */ - public void onNodeConnected(final AbstractBaseNodeContext context) { + public void onNodeConnected(final BaseNodeContext context) { if (!context.isPeer()) { totalNodes.incrementAndGet(); final JPPFSystemInformation sys = context.getSystemInformation(); @@ -205,7 +205,7 @@ public void handleNotification(final Notification notification, final Object han if (debugEnabled) log.debug("received notification {} for {}", notification.getUserData(), handback); executor.execute(() -> { final TypedProperties props = (TypedProperties) notification.getUserData(); - updatePeer((AbstractBaseNodeContext) handback, props); + updatePeer((BaseNodeContext) handback, props); }); } } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/RemoteNodeMessage.java b/server/src/java/org/jppf/server/nio/nodeserver/RemoteNodeMessage.java index c56bfe3d90..8d5fdc9946 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/RemoteNodeMessage.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/RemoteNodeMessage.java @@ -40,7 +40,7 @@ public RemoteNodeMessage(final ChannelWrapper channel) { * Initialize this nio message with the specified sll flag. * @param context the channel to read from or write to. */ - public RemoteNodeMessage(final AbstractBaseNodeContext context) { + public RemoteNodeMessage(final BaseNodeContext context) { super(context); } @@ -50,7 +50,7 @@ public RemoteNodeMessage(final AbstractBaseNodeContext context) { */ @Override protected void afterFirstRead() throws Exception { - final AbstractBaseNodeContext context = (AbstractBaseNodeContext) this.channel; + final BaseNodeContext context = (BaseNodeContext) this.channel; bundle = (TaskBundle) IOHelper.unwrappedData(locations.get(0), context.getDriver().getSerializer()); nbObjects = bundle.getTaskCount() + 1; } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/TaskQueueChecker.java b/server/src/java/org/jppf/server/nio/nodeserver/TaskQueueChecker.java index 9615058ddf..87d2ec058f 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/TaskQueueChecker.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/TaskQueueChecker.java @@ -87,7 +87,7 @@ private boolean dispatch() { try { queue.getBroadcastManager().processPendingBroadcasts(); if (queue.isEmpty()) return false; - AbstractBaseNodeContext channel = null; + BaseNodeContext channel = null; ServerTaskBundleNode nodeBundle = null; synchronized(idleChannels) { if (idleChannels.isEmpty()) return false; @@ -152,7 +152,7 @@ private boolean dispatch() { * @return a channel for a node on which to execute the job. * @throws Exception if any error occurs. */ - private AbstractBaseNodeContext retrieveChannel(final ServerJob job) throws Exception { + private BaseNodeContext retrieveChannel(final ServerJob job) throws Exception { return checkJobState(job) ? findIdleChannelIndex(job) : null; } @@ -162,7 +162,7 @@ private AbstractBaseNodeContext retrieveChannel(final ServerJob job) throws E * @param selectedJob the job to dispatch. * @return the task bundle to dispatch to the specified node. */ - private ServerTaskBundleNode prepareJobDispatch(final AbstractBaseNodeContext channel, final ServerJob selectedJob) { + private ServerTaskBundleNode prepareJobDispatch(final BaseNodeContext channel, final ServerJob selectedJob) { if (debugEnabled) log.debug("dispatching jobUuid=" + selectedJob.getUuid() + " to node " + channel + ", nodeUuid=" + channel.getConnectionUuid()); int size = 1; try { @@ -181,7 +181,7 @@ private ServerTaskBundleNode prepareJobDispatch(final AbstractBaseNodeContext * @param nodeBundle the job to dispatch. * @throws Exception if any error occurs. */ - private static void dispatchJobToChannel(final AbstractBaseNodeContext channel, final ServerTaskBundleNode nodeBundle) throws Exception { + private static void dispatchJobToChannel(final BaseNodeContext channel, final ServerTaskBundleNode nodeBundle) throws Exception { if (debugEnabled) log.debug("dispatching {} tasks of job '{}' to node {}", nodeBundle.getTaskCount(), nodeBundle.getJob().getName(), channel.getUuid()); synchronized(channel.getMonitor()) { final Future future = channel.submit(nodeBundle); @@ -194,22 +194,22 @@ private static void dispatchJobToChannel(final AbstractBaseNodeContext channe * @param job the bundle to execute. * @return the index of an available and acceptable channel, or -1 if no channel could be found. */ - private AbstractBaseNodeContext findIdleChannelIndex(final ServerJob job) { + private BaseNodeContext findIdleChannelIndex(final ServerJob job) { final JobSLA sla = job.getJob().getSLA(); final ExecutionPolicy policy = sla.getExecutionPolicy(); final JPPFNodeConfigSpec spec = sla.getDesiredNodeConfiguration(); final TypedProperties desiredConfiguration = (spec == null) ? null : spec.getConfiguration(); if (debugEnabled && (policy != null)) log.debug("Bundle " + job + " has an execution policy:\n" + policy); - List> acceptableChannels = new ArrayList<>(idleChannels.size()); - final List> toRemove = new LinkedList<>(); + List> acceptableChannels = new ArrayList<>(idleChannels.size()); + final List> toRemove = new LinkedList<>(); final List uuidPath = job.getJob().getUuidPath().getList(); - final Iterator> iterator = idleChannels.iterator(); + final Iterator> iterator = idleChannels.iterator(); final int nbJobChannels = job.getNbChannels(); final int nbReservedNodes = reservationHandler.getNbReservedNodes(job.getUuid()); final Collection readyNodes = (spec == null) ? null : reservationHandler.getReadyNodes(job.getUuid()); if (debugEnabled) log.debug("jobUuid={}, readyNodes={}", job.getUuid(), readyNodes); while (iterator.hasNext()) { - final AbstractBaseNodeContext channel = iterator.next(); + final BaseNodeContext channel = iterator.next(); synchronized(channel.getMonitor()) { if ((channel.getExecutionStatus() != ExecutorStatus.ACTIVE) || !channel.getChannel().isOpen() || channel.isClosed() || !channel.isEnabled()) { if (debugEnabled) log.debug("channel is not opened: " + channel); @@ -261,13 +261,13 @@ private AbstractBaseNodeContext findIdleChannelIndex(final ServerJob job) { } } if (!toRemove.isEmpty()) { - for (AbstractBaseNodeContext c: toRemove) removeIdleChannelAsync(c); + for (BaseNodeContext c: toRemove) removeIdleChannelAsync(c); } //if ((desiredConfiguration != null) && !reservationHandler.hasPendingNode(job.getUuid())) acceptableChannels = filterLowestDistances(job, acceptableChannels); if (!acceptableChannels.isEmpty() && (desiredConfiguration != null)) acceptableChannels = filterLowestDistances(job, acceptableChannels); final int size = acceptableChannels.size(); if (debugEnabled) log.debug("found " + size + " acceptable channels"); - final AbstractBaseNodeContext channel = (size > 0) ? acceptableChannels.get(size > 1 ? random.nextInt(size) : 0) : null; + final BaseNodeContext channel = (size > 0) ? acceptableChannels.get(size > 1 ? random.nextInt(size) : 0) : null; return channel; } @@ -306,7 +306,7 @@ private static boolean checkJobState(final ServerJob job) { * @param job the bundle from which to get the job information. * @return true if the job can be dispatched to at least one more node, false otherwise. */ - private static boolean checkMaxNodeGroups(final AbstractBaseNodeContext currentNode, final ServerJob job) { + private static boolean checkMaxNodeGroups(final BaseNodeContext currentNode, final ServerJob job) { final JPPFManagementInfo currentInfo = currentNode.getManagementInfo(); if (currentInfo == null) return true; final String currentMasterUuid = getMasterUuid(currentInfo); @@ -347,7 +347,7 @@ else if (info.isSlaveNode()) { * @param taskBundle the job. * @param context the current node context. */ - private void updateBundler(final TaskBundle taskBundle, final AbstractBaseNodeContext context) { + private void updateBundler(final TaskBundle taskBundle, final BaseNodeContext context) { context.checkBundler(bundlerFactory, jppfContext); final Bundler ctxBundler = context.getBundler(); if (ctxBundler instanceof JobAwareness) ((JobAwareness) ctxBundler).setJob(taskBundle); @@ -374,12 +374,12 @@ private boolean checkGridPolicy(final ServerJob job) { * @param channels the list of eligible channels. * @return one or more channels with the minimum computed score. */ - private List> filterLowestDistances(final ServerJob job, final List> channels) { + private List> filterLowestDistances(final ServerJob job, final List> channels) { final JPPFNodeConfigSpec spec = job.getSLA().getDesiredNodeConfiguration(); final TypedProperties desiredConfiguration = (spec == null) ? null : spec.getConfiguration(); - final CollectionSortedMap> scoreMap = new SetSortedMap<>(); + final CollectionSortedMap> scoreMap = new SetSortedMap<>(); if (debugEnabled) log.debug("computing scores for job '{}', uuid={}", job.getName(), job.getUuid()); - for (final AbstractBaseNodeContext channel: channels) { + for (final BaseNodeContext channel: channels) { if (!channel.isLocal() && !channel.isOffline() && !channel.isPeer()) { final String reservedJobUuid = server.getNodeReservationHandler().getPendingJobUUID(channel); if ((reservedJobUuid != null) && reservedJobUuid.equals(job.getUuid())) continue; @@ -393,12 +393,12 @@ private List> filterLowestDistances(final ServerJob j } if (debugEnabled) { final CollectionMap map = new SetSortedMap<>(); - for (Map.Entry>> entry: scoreMap.entrySet()) { - for (final AbstractBaseNodeContext c: entry.getValue()) map.putValue(entry.getKey(), c.getUuid()); + for (Map.Entry>> entry: scoreMap.entrySet()) { + for (final BaseNodeContext c: entry.getValue()) map.putValue(entry.getKey(), c.getUuid()); } log.debug("computed scores: {}", map); } final int n = scoreMap.firstKey(); - return (scoreMap.isEmpty()) ? Collections.>emptyList() : new ArrayList<>(scoreMap.getValues(n)); + return (scoreMap.isEmpty()) ? Collections.>emptyList() : new ArrayList<>(scoreMap.getValues(n)); } } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/WaitInitialBundleState.java b/server/src/java/org/jppf/server/nio/nodeserver/WaitInitialBundleState.java index 1a1cef34bc..e67b9949c3 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/WaitInitialBundleState.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/WaitInitialBundleState.java @@ -195,7 +195,7 @@ private HostIP resolveHost(final ChannelWrapper channel) throws Exception { private NodeTransition processOfflineReopen(final NodeBundleResults received, final AbstractNodeContext context) throws Exception { final TaskBundle bundle = received.bundle(); final String jobUuid = bundle.getParameter(JOB_UUID); - final long id = bundle.getParameter(NODE_BUNDLE_ID); + final long id = bundle.getBundleId(); final ServerTaskBundleNode nodeBundle = server.getOfflineNodeHandler().removeNodeBundle(jobUuid, id); // if the driver was restarted, we discard the results if (nodeBundle == null) return finalizeTransition(context); diff --git a/server/src/java/org/jppf/server/nio/nodeserver/async/AbstractAsyncTaskQueueChecker.java b/server/src/java/org/jppf/server/nio/nodeserver/async/AbstractAsyncJobScheduler.java similarity index 88% rename from server/src/java/org/jppf/server/nio/nodeserver/async/AbstractAsyncTaskQueueChecker.java rename to server/src/java/org/jppf/server/nio/nodeserver/async/AbstractAsyncJobScheduler.java index 7cde800a0e..a61786bbe9 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/async/AbstractAsyncTaskQueueChecker.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/async/AbstractAsyncJobScheduler.java @@ -37,11 +37,11 @@ /** * This class ensures that idle nodes get assigned pending tasks in the queue. */ -abstract class AbstractAsyncTaskQueueChecker extends ThreadSynchronization implements Runnable { +abstract class AbstractAsyncJobScheduler extends ThreadSynchronization implements Runnable { /** * Logger for this class. */ - private static final Logger log = LoggerFactory.getLogger(AbstractAsyncTaskQueueChecker.class); + private static final Logger log = LoggerFactory.getLogger(AbstractAsyncJobScheduler.class); /** * Determines whether DEBUG logging level is enabled. */ @@ -65,7 +65,7 @@ abstract class AbstractAsyncTaskQueueChecker extends ThreadSynchronization imple /** * The list of idle node channels. */ - final Set> idleChannels = new LinkedHashSet<>(); + final Set> idleChannels = new LinkedHashSet<>(); /** * Holds information about the execution context. */ @@ -95,6 +95,10 @@ abstract class AbstractAsyncTaskQueueChecker extends ThreadSynchronization imple * The number of connected nodes below which the driver load-balances to other peer drivers. */ final int peerLoadBalanceThreshold; + /** + * Whether bias towards local node is enabled. + */ + final boolean localNodeBiasEnabled; /** * Initialize this task queue checker with the specified node server. @@ -103,7 +107,7 @@ abstract class AbstractAsyncTaskQueueChecker extends ThreadSynchronization imple * @param stats reference to the statistics. * @param bundlerFactory the load-balancer factory. */ - AbstractAsyncTaskQueueChecker(final AsyncNodeNioServer server, final JPPFPriorityQueue queue, final JPPFStatistics stats, final JPPFBundlerFactory bundlerFactory) { + AbstractAsyncJobScheduler(final AsyncNodeNioServer server, final JPPFPriorityQueue queue, final JPPFStatistics stats, final JPPFBundlerFactory bundlerFactory) { this.server = server; this.queue = queue; this.disptachtoPeersWithoutNode = server.getDriver().getConfiguration().get(JPPFProperties.PEER_ALLOW_ORPHANS); @@ -112,6 +116,7 @@ abstract class AbstractAsyncTaskQueueChecker extends ThreadSynchronization imple this.bundlerFactory = bundlerFactory; this.driverInfo = server.getDriver().getSystemInformation(); this.peerLoadBalanceThreshold = server.getDriver().getInitializer().getPeerConnectionPoolHandler().getLoadBalanceThreshold(); + this.localNodeBiasEnabled = server.getDriver().getConfiguration().get(JPPFProperties.LOCAL_NODE_BIAS); } /** @@ -136,7 +141,7 @@ int getNbIdleChannels() { * Add a channel to the list of idle channels. * @param channel the channel to add to the list. */ - public void addIdleChannel(final AbstractBaseNodeContext channel) { + public void addIdleChannel(final BaseNodeContext channel) { if (debugEnabled) log.debug("request to add idle channel {}", channel); if (channel == null) { final String message = "channel is null"; @@ -152,7 +157,7 @@ public void addIdleChannel(final AbstractBaseNodeContext channel) { if (debugEnabled) log.debug("adding idle channel {}", channel); if (!channel.isClosed()) { if (!reservationHandler.transitionReservation(channel)) reservationHandler.removeReservation(channel); - boolean added = false; + final boolean added; synchronized(idleChannels) { added = idleChannels.add(channel); } @@ -173,9 +178,9 @@ public void addIdleChannel(final AbstractBaseNodeContext channel) { * @param channel the channel to remove from the list. * @return a reference to the removed channel. */ - AbstractBaseNodeContext removeIdleChannel(final AbstractBaseNodeContext channel) { + BaseNodeContext removeIdleChannel(final BaseNodeContext channel) { if (debugEnabled) log.debug("removing idle channel {}", channel); - boolean removed = false; + final boolean removed; synchronized(idleChannels) { removed = idleChannels.remove(channel); } @@ -192,7 +197,7 @@ AbstractBaseNodeContext removeIdleChannel(final AbstractBaseNodeContext ch * Asynchronously remove a channel from the list of idle channels. * @param channel the channel to remove from the list. */ - public void removeIdleChannelAsync(final AbstractBaseNodeContext channel) { + public void removeIdleChannelAsync(final BaseNodeContext channel) { if (debugEnabled) log.debug("request to remove idle channel {}", channel); channelsExecutor.execute(() -> removeIdleChannel(channel)); } @@ -201,7 +206,7 @@ public void removeIdleChannelAsync(final AbstractBaseNodeContext channel) { * Get the list of idle channels. * @return a new copy of the underlying list of idle channels. */ - public List> getIdleChannels() { + public List> getIdleChannels() { synchronized (idleChannels) { return new ArrayList<>(idleChannels); } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncTaskQueueChecker.java b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncJobScheduler.java similarity index 71% rename from server/src/java/org/jppf/server/nio/nodeserver/async/AsyncTaskQueueChecker.java rename to server/src/java/org/jppf/server/nio/nodeserver/async/AsyncJobScheduler.java index 0b9b596c3b..e57da57a03 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncTaskQueueChecker.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncJobScheduler.java @@ -39,19 +39,15 @@ /** * This class ensures that idle nodes get assigned pending tasks in the queue. */ -public class AsyncTaskQueueChecker extends AbstractAsyncTaskQueueChecker { +public class AsyncJobScheduler extends AbstractAsyncJobScheduler { /** * Logger for this class. */ - private static final Logger log = LoggerFactory.getLogger(AsyncTaskQueueChecker.class); + private static final Logger log = LoggerFactory.getLogger(AsyncJobScheduler.class); /** * Determines whether DEBUG logging level is enabled. */ private static final boolean debugEnabled = LoggingUtils.isDebugEnabled(log); - /** - * Whether bias towards local node is enabled. - */ - private final boolean localNodeBiasEnabled; /** * Initialize this task queue checker with the specified node server. @@ -60,9 +56,8 @@ public class AsyncTaskQueueChecker extends AbstractAsyncTaskQueueChecker { * @param stats reference to the statistics. * @param bundlerFactory the load-balancer factory. */ - AsyncTaskQueueChecker(final AsyncNodeNioServer server, final JPPFPriorityQueue queue, final JPPFStatistics stats, final JPPFBundlerFactory bundlerFactory) { + AsyncJobScheduler(final AsyncNodeNioServer server, final JPPFPriorityQueue queue, final JPPFStatistics stats, final JPPFBundlerFactory bundlerFactory) { super(server, queue, stats, bundlerFactory); - this.localNodeBiasEnabled = server.getDriver().getConfiguration().get(JPPFProperties.LOCAL_NODE_BIAS); } /** @@ -72,63 +67,66 @@ public class AsyncTaskQueueChecker extends AbstractAsyncTaskQueueChecker { public void run() { if (debugEnabled) log.debug("starting {}", getClass().getSimpleName()); reservationHandler = server.getNodeReservationHandler(); - while (!isStopped()) { - if (!dispatch()) goToSleep(1000L); + try { + while (!isStopped()) { + if (!dispatch()) goToSleep(1000L); + } + } catch (final Throwable t) { + log.error("error in driver dispatch loop", t); } } /** - * Perform the assignment of tasks. + * Perform the assignment of jobs to nodes. * @return true if a job was dispatched, false otherwise. */ private boolean dispatch() { try { queue.getBroadcastManager().processPendingBroadcasts(); if (queue.isEmpty()) return false; - AbstractBaseNodeContext channel = null; + BaseNodeContext channel = null; ServerTaskBundleNode nodeBundle = null; synchronized(idleChannels) { if (idleChannels.isEmpty()) return false; final List allJobs = queue.getAllJobsFromPriorityMap(); if (debugEnabled) log.debug("there are {} idle channels and {} jobs in the queue", idleChannels.size(), allJobs.size()); try { - final Iterator it = allJobs.iterator(); - while ((channel == null) && it.hasNext() && !idleChannels.isEmpty()) { - final ServerJob job = it.next(); + final Iterator jobIterator = allJobs.iterator(); + while ((channel == null) && jobIterator.hasNext() && !idleChannels.isEmpty()) { + final ServerJob job = jobIterator.next(); final JPPFNodeConfigSpec spec = job.getSLA().getDesiredNodeConfiguration(); if (spec != null) { if ((reservationHandler.getNbReservedNodes(job.getUuid()) >= job.getSLA().getMaxNodes()) && !reservationHandler.hasReadyNode(job.getUuid())) continue; } if (!checkGridPolicy(job)) continue; - channel = retrieveChannel(job); - if (channel != null) { - synchronized(channel.getMonitor()) { - if (spec != null) { - final String readyJobUUID = reservationHandler.getReadyJobUUID(channel); - final String pendingJobUUID = reservationHandler.getPendingJobUUID(channel); - if ((pendingJobUUID == null) && (readyJobUUID == null)) { - reservationHandler.doReservation(job, channel); - channel = null; - continue; - } - } - removeIdleChannel(channel); - if (!channel.isEnabled()) { + channel = checkJobState(job) ? findIdleChannelIndex(job) : null; + if (channel == null) continue; + synchronized(channel.getMonitor()) { + if (spec != null) { + final String readyJobUUID = reservationHandler.getReadyJobUUID(channel); + final String pendingJobUUID = reservationHandler.getPendingJobUUID(channel); + if ((pendingJobUUID == null) && (readyJobUUID == null)) { + reservationHandler.doReservation(job, channel); channel = null; continue; } - nodeBundle = prepareJobDispatch(channel, job); - if (debugEnabled) log.debug("prepareJobDispatch() returned {}", nodeBundle); - if (nodeBundle != null) { - try { - dispatchJobToChannel(channel, nodeBundle); - return true; - } catch (final Exception e) { - log.error("{}\nchannel={}\njob={}\nstack trace: {}", ExceptionUtils.getMessage(e), channel, nodeBundle, ExceptionUtils.getStackTrace(e)); - channel.setClosed(false); - channel.handleException(e); - } + } + if (channel.getCurrentNbJobs() >= channel.getMaxJobs()) removeIdleChannel(channel); + if (!channel.isEnabled()) { + channel = null; + continue; + } + nodeBundle = prepareJobDispatch(channel, job); + if (debugEnabled) log.debug("prepareJobDispatch() returned {}", nodeBundle); + if (nodeBundle != null) { + try { + dispatchJobToChannel(channel, nodeBundle); + return true; + } catch (final Exception e) { + log.error("{}\nchannel={}\njob={}\nstack trace: {}", ExceptionUtils.getMessage(e), channel, nodeBundle, ExceptionUtils.getStackTrace(e)); + channel.setClosed(false); + channel.handleException(e); } } } @@ -144,23 +142,13 @@ private boolean dispatch() { return false; } - /** - * Retrieve a suitable channel for the specified job. - * @param job the job to execute. - * @return a channel for a node on which to execute the job. - * @throws Exception if any error occurs. - */ - private AbstractBaseNodeContext retrieveChannel(final ServerJob job) throws Exception { - return checkJobState(job) ? findIdleChannelIndex(job) : null; - } - /** * Prepare the specified job for the selected channel, after applying the load balancer to the job. * @param channel the node channel to prepare dispatch the job to. * @param selectedJob the job to dispatch. * @return the task bundle to dispatch to the specified node. */ - private ServerTaskBundleNode prepareJobDispatch(final AbstractBaseNodeContext channel, final ServerJob selectedJob) { + private ServerTaskBundleNode prepareJobDispatch(final BaseNodeContext channel, final ServerJob selectedJob) { if (debugEnabled) log.debug("dispatching jobUuid=" + selectedJob.getUuid() + " to node " + channel + ", nodeUuid=" + channel.getConnectionUuid()); int size = 1; try { @@ -179,12 +167,13 @@ private ServerTaskBundleNode prepareJobDispatch(final AbstractBaseNodeContext * @param nodeBundle the job to dispatch. * @throws Exception if any error occurs. */ - private static void dispatchJobToChannel(final AbstractBaseNodeContext channel, final ServerTaskBundleNode nodeBundle) throws Exception { + private static void dispatchJobToChannel(final BaseNodeContext channel, final ServerTaskBundleNode nodeBundle) throws Exception { if (debugEnabled) log.debug("dispatching {} tasks of job '{}' to node {}", nodeBundle.getTaskCount(), nodeBundle.getJob().getName(), channel.getUuid()); synchronized(channel.getMonitor()) { final Future future = channel.submit(nodeBundle); nodeBundle.jobDispatched(channel, future); } + if (debugEnabled) log.debug("dispatched {} tasks of job '{}' to node {}", nodeBundle.getTaskCount(), nodeBundle.getJob().getName(), channel.getUuid()); } /** @@ -192,22 +181,15 @@ private static void dispatchJobToChannel(final AbstractBaseNodeContext channe * @param job the bundle to execute. * @return the index of an available and acceptable channel, or -1 if no channel could be found. */ - private AbstractBaseNodeContext findIdleChannelIndex(final ServerJob job) { + private BaseNodeContext findIdleChannelIndex(final ServerJob job) { final JobSLA sla = job.getJob().getSLA(); - final ExecutionPolicy policy = sla.getExecutionPolicy(); final JPPFNodeConfigSpec spec = sla.getDesiredNodeConfiguration(); final TypedProperties desiredConfiguration = (spec == null) ? null : spec.getConfiguration(); - if (debugEnabled && (policy != null)) log.debug("Bundle " + job + " has an execution policy:\n" + policy); - List> acceptableChannels = new ArrayList<>(idleChannels.size()); - final List> toRemove = new LinkedList<>(); - final List uuidPath = job.getJob().getUuidPath().getList(); - final Iterator> iterator = idleChannels.iterator(); - final int nbJobChannels = job.getNbChannels(); - final int nbReservedNodes = reservationHandler.getNbReservedNodes(job.getUuid()); - final Collection readyNodes = (spec == null) ? null : reservationHandler.getReadyNodes(job.getUuid()); - if (debugEnabled) log.debug("jobUuid={}, readyNodes={}", job.getUuid(), readyNodes); - while (iterator.hasNext()) { - final AbstractBaseNodeContext channel = iterator.next(); + List> acceptableChannels = new ArrayList<>(idleChannels.size()); + final List> toRemove = new LinkedList<>(); + final Iterator> nodeIterator = idleChannels.iterator(); + while (nodeIterator.hasNext()) { + final BaseNodeContext channel = nodeIterator.next(); synchronized(channel.getMonitor()) { if ((channel.getExecutionStatus() != ExecutorStatus.ACTIVE) || channel.isClosed() || !channel.isEnabled()) { if (debugEnabled) log.debug("channel is not opened: {}", channel); @@ -223,11 +205,7 @@ private AbstractBaseNodeContext findIdleChannelIndex(final ServerJob job) { if (debugEnabled) log.debug("[currentNbJobs = {}] >= maxJobs = {}] for {}", channel.getCurrentNbJobs(), channel.getMaxJobs(), channel); continue; } - if (debugEnabled) log.debug("uuid path={}, node uuid={}", uuidPath, channel.getUuid()); - if (uuidPath.contains(channel.getUuid())) { - if (debugEnabled) log.debug("bundle uuid path already contains node {} : uuidPath={}, nodeUuid={}", channel, uuidPath, channel.getUuid()); - continue; - } + if (!checkUuidPath(channel, job)) continue; if (job.getBroadcastUUID() != null && !job.getBroadcastUUID().equals(channel.getUuid())) continue; final JPPFSystemInformation info = channel.getSystemInformation(); if (channel.isPeer() && !disptachtoPeersWithoutNode) { @@ -236,9 +214,11 @@ private AbstractBaseNodeContext findIdleChannelIndex(final ServerJob job) { continue; } } - if (!checkExecutionPolicy(channel, job, policy, info, nbJobChannels)) continue; + if (!checkExecutionPolicy(channel, job, sla.getExecutionPolicy(), info, job.getNbChannels())) continue; if (!checkMaxNodeGroups(channel, job)) continue; - if (!checkDesiredConfiguration(desiredConfiguration, channel, job, readyNodes, nbReservedNodes)) continue; + final Collection readyNodes = (spec == null) ? null : reservationHandler.getReadyNodes(job.getUuid()); + if (debugEnabled) log.debug("jobUuid={}, readyNodes={}", job.getUuid(), readyNodes); + if (!checkDesiredConfiguration(desiredConfiguration, channel, job, readyNodes, reservationHandler.getNbReservedNodes(job.getUuid()))) continue; if (channel.isLocal() && localNodeBiasEnabled) { // add a bias toward local node if (desiredConfiguration != null) continue; else return channel; @@ -247,15 +227,44 @@ private AbstractBaseNodeContext findIdleChannelIndex(final ServerJob job) { } } if (!toRemove.isEmpty()) { - for (final AbstractBaseNodeContext c: toRemove) removeIdleChannelAsync(c); + for (final BaseNodeContext c: toRemove) removeIdleChannelAsync(c); } if (!acceptableChannels.isEmpty() && (desiredConfiguration != null)) acceptableChannels = filterLowestDistances(job, acceptableChannels); + return selectChannel(acceptableChannels); + } + + /** + * Select a single channle from a list of channels eligible for a job. + * @param acceptableChannels the list of channels to select from. + * @return an instance of {@link BaseNodeContext}, or {@code null} if the list is empty or no channel passes the selection criteria. + */ + private BaseNodeContext selectChannel(final List> acceptableChannels) { + if (acceptableChannels.isEmpty()) return null; final int size = acceptableChannels.size(); - if (debugEnabled) log.debug("found " + size + " acceptable channels"); - final AbstractBaseNodeContext channel = (size > 0) ? acceptableChannels.get(size > 1 ? random.nextInt(size) : 0) : null; + if (debugEnabled) log.debug("found {} acceptable channels", size); + final BaseNodeContext channel = (size > 0) ? acceptableChannels.get(size > 1 ? random.nextInt(size) : 0) : null; return channel; } + /** + * Check that the job did already got through the specified channel. + * @param channel the node to check. + * @param job the job to check against. + * @return {@code true} if the check succeeds, {@code false} otherwise. + */ + private boolean checkUuidPath(final BaseNodeContext channel, final ServerJob job) { + final List uuidPath = job.getJob().getUuidPath().getList(); + if (debugEnabled) log.debug("uuid path={}, node uuid={}", uuidPath, channel.getUuid()); + final String driverUuid = server.getDriver().getUuid(); + final int index = uuidPath.indexOf(driverUuid); + if ((index >= 0) && (index != uuidPath.size() - 1)) log.warn("uuid path contains this driver's uuid {}: uuidPath={}", driverUuid, uuidPath); + if (uuidPath.contains(channel.getUuid())) { + if (debugEnabled) log.debug("bundle uuid path already contains node {} : uuidPath={}, nodeUuid={}", channel, uuidPath, channel.getUuid()); + return false; + } + return true; + } + /** * Check if the node is ready for the job's configuration. * @param config the desired node configuration @@ -265,7 +274,7 @@ private AbstractBaseNodeContext findIdleChannelIndex(final ServerJob job) { * @param nbReservedNodes number of nodes reserved for the job. * @return {@code true} if the check succeeds, {@code false} otherwise. */ - private boolean checkDesiredConfiguration(final TypedProperties config, final AbstractBaseNodeContext channel, final ServerJob job, final Collection readyNodes, final int nbReservedNodes) { + private boolean checkDesiredConfiguration(final TypedProperties config, final BaseNodeContext channel, final ServerJob job, final Collection readyNodes, final int nbReservedNodes) { if (config != null) { if (reservationHandler.getPendingJobUUID(channel) != null) return false; final String readyJobUuid = reservationHandler.getReadyJobUUID(channel); @@ -276,18 +285,6 @@ private boolean checkDesiredConfiguration(final TypedProperties config, final Ab } return true; } - /* - if (desiredConfiguration != null) { - if (reservationHandler.getPendingJobUUID(channel) != null) continue; - final String readyJobUuid = reservationHandler.getReadyJobUUID(channel); - boolean b = true; - if (readyNodes != null) { - b = readyNodes.contains(channel.getUuid()); - } - if (debugEnabled) log.debug("nodeUuid={}, readyJobUuid={}, jobUuid={}, b={}", channel.getUuid(), readyJobUuid, job.getUuid(), b); - if (!b && (nbReservedNodes >= sla.getMaxNodes())) continue; - } - */ /** * Check whether the job's execution policy, if any, matches the node. @@ -298,8 +295,9 @@ private boolean checkDesiredConfiguration(final TypedProperties config, final Ab * @param nbJobChannels the number of nodes to which the job is already dispatched, passed on to the policy context. * @return {@code true} if the policy is {@code null} or if the node matches, {@code false} otherwise. */ - private boolean checkExecutionPolicy(final AbstractBaseNodeContext channel, final ServerJob job, final ExecutionPolicy policy, final JPPFSystemInformation info, final int nbJobChannels) { + private boolean checkExecutionPolicy(final BaseNodeContext channel, final ServerJob job, final ExecutionPolicy policy, final JPPFSystemInformation info, final int nbJobChannels) { if (policy == null) return true; + if (debugEnabled) log.debug("job has an execution policy: {}\n{}", job, policy); boolean b = false; try { preparePolicy(policy, job, stats, nbJobChannels); @@ -310,19 +308,6 @@ private boolean checkExecutionPolicy(final AbstractBaseNodeContext channel, f if (debugEnabled) log.debug("rule execution is *{}* for job [name={}, uuid={}] on channel {}", b, job.getName(), job.getUuid(), channel); return b; } - /* - if (policy != null) { - boolean b = false; - try { - preparePolicy(policy, job, stats, nbJobChannels); - b = policy.evaluate(info); - } catch(final Exception ex) { - log.error("An error occurred while running the execution policy to determine node participation.", ex); - } - if (debugEnabled) log.debug("rule execution is *{}* for job [name={}, uuid={}] on channel {}", b, job.getName(), job.getUuid(), channel); - if (!b) continue; - } - */ /** * Set the parameters needed as bounded variables for scripted execution policies. @@ -359,7 +344,7 @@ private static boolean checkJobState(final ServerJob job) { * @param job the bundle from which to get the job information. * @return true if the job can be dispatched to at least one more node, false otherwise. */ - private static boolean checkMaxNodeGroups(final AbstractBaseNodeContext currentNode, final ServerJob job) { + private static boolean checkMaxNodeGroups(final BaseNodeContext currentNode, final ServerJob job) { final JPPFManagementInfo currentInfo = currentNode.getManagementInfo(); if (currentInfo == null) return true; final String currentMasterUuid = getMasterUuid(currentInfo); @@ -403,7 +388,7 @@ else if (info.isSlaveNode()) { * @param taskBundle the job. * @param context the current node context. */ - private void updateBundler(final TaskBundle taskBundle, final AbstractBaseNodeContext context) { + private void updateBundler(final TaskBundle taskBundle, final BaseNodeContext context) { context.checkBundler(bundlerFactory, jppfContext); final Bundler ctxBundler = context.getBundler(); if (ctxBundler instanceof JobAwareness) ((JobAwareness) ctxBundler).setJob(taskBundle); @@ -430,12 +415,12 @@ private boolean checkGridPolicy(final ServerJob job) { * @param channels the list of eligible channels. * @return one or more channels with the minimum computed score. */ - private List> filterLowestDistances(final ServerJob job, final List> channels) { + private List> filterLowestDistances(final ServerJob job, final List> channels) { final JPPFNodeConfigSpec spec = job.getSLA().getDesiredNodeConfiguration(); final TypedProperties desiredConfiguration = (spec == null) ? null : spec.getConfiguration(); - final CollectionSortedMap> scoreMap = new SetSortedMap<>(); + final CollectionSortedMap> scoreMap = new SetSortedMap<>(); if (debugEnabled) log.debug("computing scores for job '{}', uuid={}", job.getName(), job.getUuid()); - for (final AbstractBaseNodeContext channel: channels) { + for (final BaseNodeContext channel: channels) { if (!channel.isLocal() && !channel.isOffline() && !channel.isPeer()) { final String reservedJobUuid = server.getNodeReservationHandler().getPendingJobUUID(channel); if ((reservedJobUuid != null) && reservedJobUuid.equals(job.getUuid())) continue; @@ -449,12 +434,12 @@ private List> filterLowestDistances(final ServerJob j } if (debugEnabled) { final CollectionMap map = new SetSortedMap<>(); - for (Map.Entry>> entry: scoreMap.entrySet()) { - for (final AbstractBaseNodeContext c: entry.getValue()) map.putValue(entry.getKey(), c.getUuid()); + for (Map.Entry>> entry: scoreMap.entrySet()) { + for (final BaseNodeContext c: entry.getValue()) map.putValue(entry.getKey(), c.getUuid()); } log.debug("computed scores: {}", map); } final int n = scoreMap.firstKey(); - return (scoreMap.isEmpty()) ? Collections.>emptyList() : new ArrayList<>(scoreMap.getValues(n)); + return (scoreMap.isEmpty()) ? Collections.>emptyList() : new ArrayList<>(scoreMap.getValues(n)); } } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeContext.java b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeContext.java index 16c0f8744f..7f97c434f3 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeContext.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeContext.java @@ -23,7 +23,7 @@ import java.nio.channels.*; import java.util.*; import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.*; import org.jppf.execute.ExecutorStatus; import org.jppf.io.*; @@ -46,7 +46,7 @@ * Context or state information associated with a channel that exchanges heartbeat messages between the server and a node or client. * @author Laurent Cohen */ -public class AsyncNodeContext extends StatelessNioContext implements AbstractBaseNodeContext { +public class AsyncNodeContext extends StatelessNioContext implements BaseNodeContext { /** * Logger for this class. */ @@ -82,7 +82,15 @@ public class AsyncNodeContext extends StatelessNioContext implements AbstractBas /** * Lock to synchronize I/O on a local node. */ - private final ThreadSynchronization localNodeLock; + private final ThreadSynchronization localNodeReadLock; + /** + * Lock to synchronize I/O on a local node. + */ + private final ThreadSynchronization localNodeWriteLock; + /** + * The maximum number of concurrent jobs for this channel. + */ + private final AtomicInteger maxJobs = new AtomicInteger(0); /** * @param server the server that handles this context. @@ -94,7 +102,8 @@ public AsyncNodeContext(final AsyncNodeNioServer server, final SocketChannel soc this.driver = server.getDriver(); this.socketChannel = socketChannel; this.local = (socketChannel == null); - this.localNodeLock = local ? new ThreadSynchronization() : null; + this.localNodeReadLock = local ? new ThreadSynchronization() : null; + this.localNodeWriteLock = local ? new ThreadSynchronization() : null; this.attributes = new NodeContextAttributes(this, server.getBundlerHandler(), server); this.attributes.setDriver(server.getDriver()); } @@ -166,7 +175,7 @@ public AbstractTaskBundleMessage serializeBundle(final ServerTaskBundleNode bund bundle.checkTaskCount(); final TaskBundle taskBundle = bundle.getJob(); final AbstractTaskBundleMessage message = newMessage(); - taskBundle.setParameter(BundleParameter.NODE_BUNDLE_ID, bundle.getId()); + taskBundle.setBundleId(bundle.getId()); if (!taskBundle.isHandshake()) { if (!isPeer()) taskBundle.removeParameter(BundleParameter.TASK_MAX_RESUBMITS); else if (bundle.getServerJob().isPersistent()) taskBundle.setParameter(BundleParameter.ALREADY_PERSISTED_P2P, true); @@ -206,33 +215,31 @@ public AbstractTaskBundleMessage newMessage() { @Override public boolean readMessage() throws Exception { if (message == null) message = newMessage(); - final NioMessage msg = message; byteCount = message.getChannelReadCount(); boolean b = false; try { - b = msg.read(); + b = message.read(); } catch (final Exception e) { - updateTrafficStats((AbstractTaskBundleMessage) msg); + updateTrafficStats((AbstractTaskBundleMessage) message); throw e; } - byteCount = msg.getChannelReadCount() - byteCount; - if (b) updateTrafficStats((AbstractTaskBundleMessage) msg); + byteCount = message.getChannelReadCount() - byteCount; + if (b) updateTrafficStats((AbstractTaskBundleMessage) message); return b; } @Override public boolean writeMessage() throws Exception { - final NioMessage msg = writeMessage; - writeByteCount = msg.getChannelWriteCount(); + writeByteCount = writeMessage.getChannelWriteCount(); boolean b = false; try { - b = msg.write(); + b = writeMessage.write(); } catch (final Exception e) { - updateTrafficStats((AbstractTaskBundleMessage) msg); + updateTrafficStats((AbstractTaskBundleMessage) writeMessage); throw e; } - writeByteCount = msg.getChannelWriteCount() - writeByteCount; - if (b) updateTrafficStats((AbstractTaskBundleMessage) msg); + writeByteCount = writeMessage.getChannelWriteCount() - writeByteCount; + if (b) updateTrafficStats((AbstractTaskBundleMessage) writeMessage); return b; } @@ -328,6 +335,7 @@ public String toString() { sb.append(", ssl=").append(ssl); sb.append(", local=").append(local); sb.append(", offline=").append(isOffline()); + sb.append(", maxJobs=").append(getMaxJobs()); sb.append(", jobEntries=").append(entryMap.size()); sb.append(", sendQueue size=").append(sendQueue.size()); sb.append(", interestOps=").append(getInterestOps()); @@ -367,8 +375,9 @@ public int getCurrentNbJobs() { @Override public Future submit(final ServerTaskBundleNode nodeBundle) throws Exception { - setExecutionStatus(ExecutorStatus.EXECUTING); addJobEntry(nodeBundle); + if (debugEnabled) log.debug("submitting {} to {}", nodeBundle, this); + if (getCurrentNbJobs() >= getMaxJobs()) setExecutionStatus(ExecutorStatus.EXECUTING); nodeBundle.setOffline(isOffline()); nodeBundle.setChannel(this); nodeBundle.getJob().setExecutionStartTime(System.nanoTime()); @@ -435,9 +444,29 @@ public AsyncNodeNioServer getServer() { } /** - * @return a lock used to synchronize I/O with a local node. + * @return a lock used to synchronize input I/O with a local node. + */ + public ThreadSynchronization getLocalNodeReadLock() { + return localNodeReadLock; + } + + /** + * @return a lock used to synchronize output I/O with a local node. + */ + public ThreadSynchronization getLocalNodeWriteLock() { + return localNodeWriteLock; + } + + @Override + public int getMaxJobs() { + return maxJobs.get(); + } + + /** + * Set the maximum number of concurrent jobs for this channel. + * @param maxJobs the max number of jobs to set. */ - public ThreadSynchronization getLocalNodeLock() { - return localNodeLock; + public void setMaxJobs(final int maxJobs) { + this.maxJobs.set(maxJobs); } } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeContextFuture.java b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeContextFuture.java index 8124f0eb8f..32325d2111 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeContextFuture.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeContextFuture.java @@ -50,7 +50,7 @@ public class AsyncNodeContextFuture extends FutureTask { private static final Runnable NOOP_RUNNABLE = () -> {}; /** - * Initialize witht he specified runnable and result object. + * Initialize with the specified runnable and result object. * @param context the node context. * @param bundle the node bundle to process. */ diff --git a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeMessageHandler.java b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeMessageHandler.java index 45c0273c26..23a29428d5 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeMessageHandler.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeMessageHandler.java @@ -25,7 +25,6 @@ import java.util.*; import org.jppf.execute.ExecutorStatus; -import org.jppf.io.DataLocation; import org.jppf.job.JobReturnReason; import org.jppf.load.balancer.*; import org.jppf.management.*; @@ -63,7 +62,7 @@ public class AsyncNodeMessageHandler { protected final boolean resolveIPs; /** - * + * COnstruct this object. * @param driver reference to the driver. */ public AsyncNodeMessageHandler(final JPPFDriver driver) { @@ -109,11 +108,14 @@ public void handshakeReceived(final AsyncNodeContext context, final AbstractTask final NodeBundleResults received = context.deserializeBundle(message); if (debugEnabled) log.debug("received handshake response for channel {} : {}", context, received); final TaskBundle bundle = received.bundle(); - //final Long bundleId = bundle.getParameter(BundleParameter.NODE_BUNDLE_ID); - //context.removeJobEntry(bundle.getUuid(), bundleId); final boolean offline = bundle.getParameter(NODE_OFFLINE, false); - if (offline) context.setOffline(true); - else if (!bundle.isHandshake()) throw new IllegalStateException("handshake bundle expected."); + if (offline) { + context.setOffline(true); + context.setMaxJobs(1); + } else { + updateMaxJobs(context, bundle); + if (!bundle.isHandshake()) throw new IllegalStateException("handshake bundle expected."); + } if (debugEnabled) log.debug("read bundle for {}, bundle={}", context, bundle); final String uuid = bundle.getParameter(NODE_UUID_PARAM); context.setUuid(uuid); @@ -163,7 +165,6 @@ public void handshakeReceived(final AsyncNodeContext context, final AbstractTask } context.getServer().putConnection(context); if (bundle.getParameter(NODE_OFFLINE_OPEN_REQUEST, false)) processOfflineReopen(received, context); - else finalizeTransition(context); } /** @@ -178,14 +179,6 @@ public void resultsReceived(final AsyncNodeContext context, final AbstractTaskBu process(received, context); } - /** - * Finalize the state transition processing and return the traznsition to the next state. - * @param context the context associated with the node channel. - * @throws Exception if any error occurs. - */ - private static void finalizeTransition(final AsyncNodeContext context) throws Exception { - context.setMessage(null); - } /** * Resolve the host name for the specified channel. * @param context the channel from which to get the host information. @@ -249,13 +242,9 @@ private static void processOfflineRequest(final AsyncNodeContext context, final private void processOfflineReopen(final NodeBundleResults received, final AsyncNodeContext context) throws Exception { final TaskBundle bundle = received.bundle(); final String jobUuid = bundle.getParameter(JOB_UUID); - final long id = bundle.getParameter(NODE_BUNDLE_ID); + final long id = bundle.getBundleId(); final ServerTaskBundleNode nodeBundle = context.getServer().getOfflineNodeHandler().removeNodeBundle(jobUuid, id); - // if the driver was restarted, we discard the results - if (nodeBundle == null) { - finalizeTransition(context); - return; - } + if (nodeBundle == null) return; if (debugEnabled) log.debug(build("processing offline reopen with jobUuid=", jobUuid, ", id=", id, ", nodeBundle=", nodeBundle, ", node=", context.getChannel())); context.addJobEntry(nodeBundle); process(received, context); @@ -271,9 +260,9 @@ private void processOfflineReopen(final NodeBundleResults received, final AsyncN * @param context the channel from which the bundle was read. * @throws Exception if any error occurs. */ - public void process(final NodeBundleResults received, final AsyncNodeContext context) throws Exception { + private void process(final NodeBundleResults received, final AsyncNodeContext context) throws Exception { final TaskBundle bundle = received.first(); - final ServerTaskBundleNode nodeBundle = context.removeJobEntry(bundle.getUuid(), bundle.getParameter(BundleParameter.NODE_BUNDLE_ID)).nodeBundle; + final ServerTaskBundleNode nodeBundle = context.removeJobEntry(bundle.getUuid(), bundle.getBundleId()).nodeBundle; context.getServer().getDispatchExpirationHandler().cancelAction(ServerTaskBundleNode.makeKey(nodeBundle), false); boolean requeue = false; try { @@ -286,9 +275,12 @@ public void process(final NodeBundleResults received, final AsyncNodeContext con nodeBundle.resultsReceived(t); } if (requeue) nodeBundle.resubmit(); - // there is nothing left to do, so this instance will wait for a task bundle - // make sure the context is reset so as not to resubmit the last bundle executed by the node. - context.setMessage(null); + //context.setMessage(null); + if (!context.isOffline()) updateMaxJobs(context, bundle); + if (context.getCurrentNbJobs() < context.getMaxJobs()) { + if (debugEnabled) log.debug("updating execution status to ACTIVE for {}", context); + context.setExecutionStatus(ExecutorStatus.ACTIVE); + } } /** @@ -301,10 +293,9 @@ public void process(final NodeBundleResults received, final AsyncNodeContext con */ private boolean processResults(final AsyncNodeContext context, final NodeBundleResults received, final ServerTaskBundleNode nodeBundle) throws Exception { final TaskBundle newBundle = received.bundle(); - // if an exception prevented the node from executing the tasks or sending back the results - final Throwable t = newBundle.getParameter(NODE_EXCEPTION_PARAM); Bundler bundler = context.getBundler(); final ServerJob job = nodeBundle.getClientJob(); + final Throwable t = newBundle.getParameter(NODE_EXCEPTION_PARAM); if (t != null) { if (debugEnabled) log.debug("node " + context.getChannel() + " returned exception parameter in the header for bundle " + newBundle + " : " + ExceptionUtils.getMessage(t)); nodeBundle.setJobReturnReason(JobReturnReason.NODE_PROCESSING_ERROR); @@ -338,30 +329,38 @@ private boolean processResults(final AsyncNodeContext context, final NodeBundleR } if (count > 0) context.updateStatsUponTaskResubmit(count); } else if (debugEnabled) log.debug("bundle has expired: {}", nodeBundle); - final List data = received.data(); - if (debugEnabled) log.debug("data received: size={}, content={}", data == null ? -1 : data.size(), data); if (debugEnabled) log.debug("nodeBundle={}", nodeBundle); - final long elapsed = System.nanoTime() - nodeBundle.getJob().getExecutionStartTime(); - if (bundler == null) bundler = context.checkBundler(context.getServer().getBundlerFactory(), context.getServer().getJPPFContext()); - if (bundler instanceof BundlerEx) { - final long accumulatedTime = newBundle.getParameter(NODE_BUNDLE_ELAPSED_PARAM, -1L); - BundlerHelper.updateBundler((BundlerEx) bundler, newBundle.getTaskCount(), elapsed, accumulatedTime, elapsed - newBundle.getNodeExecutionTime()); - } else BundlerHelper.updateBundler(bundler, newBundle.getTaskCount(), elapsed); - if (debugEnabled) log.debug("updated bundler for {}", context); - context.getServer().getBundlerHandler().storeBundler(context.getNodeIdentifier(), bundler, context.getBundlerAlgorithm()); - nodeBundle.resultsReceived(data); - updateStats(newBundle.getTaskCount(), elapsed / 1_000_000L, newBundle.getNodeExecutionTime() / 1_000_000L); + bundler = updateBundlerAndStats(context, bundler, nodeBundle, newBundle); + nodeBundle.resultsReceived(received.data()); if (debugEnabled) log.debug("updated stats for {}", context); } - final boolean requeue = newBundle.isRequeue(); final JPPFSystemInformation systemInfo = newBundle.getParameter(SYSTEM_INFO_PARAM); if (systemInfo != null) { context.setNodeInfo(systemInfo, true); if (bundler instanceof ChannelAwareness) ((ChannelAwareness) bundler).setChannelConfiguration(systemInfo); } - if (debugEnabled) log.debug("updating execution status to ACTIVE for {}", context); - context.setExecutionStatus(ExecutorStatus.ACTIVE); - return requeue; + return newBundle.isRequeue(); + } + + /** + * + * @param context the context for which to update the bundler + * @param currentBundler the current bundler for this context. + * @param nodeBundle the bundle that was dispatched to the node. + * @param newBundle the header of the bundle received from the node. + * @return the updated bundler; + */ + private Bundler updateBundlerAndStats(final AsyncNodeContext context, final Bundler currentBundler, final ServerTaskBundleNode nodeBundle, final TaskBundle newBundle) { + final long elapsed = System.nanoTime() - nodeBundle.getJob().getExecutionStartTime(); + final Bundler bundler = (currentBundler == null) ? context.checkBundler(context.getServer().getBundlerFactory(), context.getServer().getJPPFContext()) : currentBundler; + if (bundler instanceof BundlerEx) { + final long accumulatedTime = newBundle.getParameter(NODE_BUNDLE_ELAPSED_PARAM, -1L); + BundlerHelper.updateBundler((BundlerEx) bundler, newBundle.getTaskCount(), elapsed, accumulatedTime, elapsed - newBundle.getNodeExecutionTime()); + } else BundlerHelper.updateBundler(bundler, newBundle.getTaskCount(), elapsed); + if (debugEnabled) log.debug("updated bundler for {}", context); + context.getServer().getBundlerHandler().storeBundler(context.getNodeIdentifier(), bundler, context.getBundlerAlgorithm()); + updateStats(newBundle.getTaskCount(), elapsed / 1_000_000L, newBundle.getNodeExecutionTime() / 1_000_000L); + return bundler; } /** @@ -377,4 +376,22 @@ private void updateStats(final int nbTasks, final long elapsed, final long elaps stats.addValues(JPPFStatisticsHelper.NODE_EXECUTION, elapsedInNode, nbTasks); stats.addValues(JPPFStatisticsHelper.TRANSPORT_TIME, elapsed - elapsedInNode, nbTasks); } + + /** + * + * @param context the context for which to update the max number of jobs. + * @param bundle the header of the bundle received from the node. + */ + private void updateMaxJobs(final AsyncNodeContext context, final TaskBundle bundle) { + final int n = context.getMaxJobs(); + final Integer newMaxJobs = bundle.getParameter(BundleParameter.NODE_MAX_JOBS); + int maxJobs = 0; + if (n <= 0) { + maxJobs = (newMaxJobs == null) ? driver.getConfiguration().get(JPPFProperties.NODE_MAX_JOBS) : newMaxJobs; + } else if (newMaxJobs != null) { + maxJobs = (newMaxJobs <= 0) ? driver.getConfiguration().get(JPPFProperties.NODE_MAX_JOBS) : newMaxJobs; + } + if (debugEnabled) log.debug("n={}, newMaxJobs={}, computed maxJobs={}, context={}", n, newMaxJobs, maxJobs, context); + if (maxJobs > 0) context.setMaxJobs(maxJobs); + } } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeMessageWriter.java b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeMessageWriter.java index 504061e567..f05d8adeea 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeMessageWriter.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeMessageWriter.java @@ -51,7 +51,7 @@ protected void postWrite(final AsyncNodeContext context, final NioMessage data) final TaskBundle header = msg.getBundle(); if (debugEnabled) log.debug("fully sent message {} for job [uuid={}, name={}, handshake={}] from context {}", data, header.getUuid(), header.getName(), header.isHandshake(), context); if (!header.isHandshake()) { - final NodeJobEntry entry = context.getJobEntry(header.getUuid(), header.getParameter(BundleParameter.NODE_BUNDLE_ID)); + final NodeJobEntry entry = context.getJobEntry(header.getUuid(), header.getBundleId()); context.getServer().getMessageHandler().bundleSent(context, entry.nodeBundle); } } diff --git a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeNioServer.java b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeNioServer.java index 5dc5189619..df2cb0cba5 100644 --- a/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeNioServer.java +++ b/server/src/java/org/jppf/server/nio/nodeserver/async/AsyncNodeNioServer.java @@ -81,7 +81,7 @@ public final class AsyncNodeNioServer extends StatelessNioServer>> callable = () -> getAllChannels(); + final Callable>> callable = () -> getAllChannels(); this.queue.setCallableAllConnections(callable); this.peerHandler = new PeerAttributesHandler(driver, Math.max(1, driver.getConfiguration().getInt("jppf.peer.handler.threads", 1))); nodeConnectionHandler = driver.getInitializer().getNodeConnectionEventHandler(); bundlerFactory = new JPPFBundlerFactory(driver.getConfiguration()); bundlerHandler = new LoadBalancerPersistenceManager(bundlerFactory); this.selectTimeout = NioConstants.DEFAULT_SELECT_TIMEOUT; - taskQueueChecker = new AsyncTaskQueueChecker(this, queue, driver.getStatistics(), bundlerFactory); + jobScheduler = new AsyncJobScheduler(this, queue, driver.getStatistics(), bundlerFactory); this.queue.addQueueListener(new QueueListenerAdapter() { @Override public void bundleAdded(final QueueEvent event) { if (debugEnabled) log.debug("received queue event {}", event); - taskQueueChecker.wakeUp(); + jobScheduler.wakeUp(); } }); initialServerJob = NodeServerUtils.createInitialServerJob(driver); nodeReservationHandler = new NodeReservationHandler(driver); - ThreadUtils.startDaemonThread(taskQueueChecker, "TaskQueueChecker"); + ThreadUtils.startDaemonThread(jobScheduler, "JobScheduler"); } @Override @@ -286,7 +286,7 @@ JPPFDriver getDriver() { * @return a {@link JPPFContext} instance. */ public JPPFContext getJPPFContext() { - return taskQueueChecker.getJPPFContext(); + return jobScheduler.getJPPFContext(); } /** @@ -323,11 +323,11 @@ public NodeReservationHandler getNodeReservationHandler() { /** * Get the task that dispatches queued jobs to available nodes. - * @return a {@link AsyncTaskQueueChecker} object. + * @return a {@link AsyncJobScheduler} object. * @exclude */ - public AsyncTaskQueueChecker getTaskQueueChecker() { - return taskQueueChecker; + public AsyncJobScheduler getJobScheduler() { + return jobScheduler; } /** @@ -348,16 +348,16 @@ public JPPFBundlerFactory getBundlerFactory() { /** * Get all the node connections handled by this server. - * @return a list of {@link AbstractBaseNodeContext} instances. + * @return a list of {@link BaseNodeContext} instances. */ - public List> getAllChannels() { + public List> getAllChannels() { return new ArrayList<>(allConnections.values()); } /** - * @return a set of {@link AbstractBaseNodeContext} instances. + * @return a set of {@link BaseNodeContext} instances. */ - public Set> getAllChannelsAsSet() { + public Set> getAllChannelsAsSet() { return new HashSet<>(allConnections.values()); } @@ -365,7 +365,7 @@ public Set> getAllChannelsAsSet() { * Called when the node failed to respond to a heartbeat message. * @param context the channel to close. */ - public void connectionFailed(final AbstractBaseNodeContext context) { + public void connectionFailed(final BaseNodeContext context) { if (context != null) { if (debugEnabled) log.debug("about to close channel = {} with uuid = {}", context, context.getUuid()); removeConnection(context.getUuid()); @@ -429,7 +429,7 @@ private void removeConnection(final AsyncNodeContext nodeContext) { if (nodeContext == null) throw new IllegalArgumentException("wrapper is null"); if (debugEnabled) log.debug("removing connection {}", nodeContext); try { - taskQueueChecker.removeIdleChannelAsync(nodeContext); + jobScheduler.removeIdleChannelAsync(nodeContext); updateConnectionStatus(nodeContext, nodeContext.getExecutionStatus(), ExecutorStatus.DISABLED); } catch(final Exception e) { if (debugEnabled) log.debug("error removing connection {} : {}", nodeContext, e); @@ -454,16 +454,16 @@ private void updateConnectionStatus(final AsyncNodeContext nodeContext, final Ex if (newStatus == null) throw new IllegalArgumentException("newStatus is null"); if (nodeContext == null || oldStatus == newStatus) return; if (debugEnabled) log.debug("updating channel status from {} to {}: {}", oldStatus, newStatus, nodeContext); - if (newStatus == ExecutorStatus.ACTIVE) taskQueueChecker.addIdleChannel(nodeContext); + if (newStatus == ExecutorStatus.ACTIVE) jobScheduler.addIdleChannel(nodeContext); else { - taskQueueChecker.removeIdleChannelAsync(nodeContext); + jobScheduler.removeIdleChannelAsync(nodeContext); if (newStatus == ExecutorStatus.FAILED || newStatus == ExecutorStatus.DISABLED) transitionManager.execute(() -> queue.getBroadcastManager().cancelBroadcastJobs(nodeContext.getUuid())); } queue.updateWorkingConnections(oldStatus, newStatus); } @Override - public void nodeConnected(final AbstractBaseNodeContext context) { + public void nodeConnected(final BaseNodeContext context) { if (debugEnabled) log.debug("node connected: {}", context); final JPPFManagementInfo info = context.getManagementInfo(); if (!context.isClosed()) { diff --git a/server/src/java/org/jppf/server/node/local/AbstractLocalNodeIO.java b/server/src/java/org/jppf/server/node/local/AbstractLocalNodeIO.java index fc7bbb7924..a75fc6dc9c 100644 --- a/server/src/java/org/jppf/server/node/local/AbstractLocalNodeIO.java +++ b/server/src/java/org/jppf/server/node/local/AbstractLocalNodeIO.java @@ -78,7 +78,7 @@ protected Object[] deserializeObjects(final TaskBundle bundle) throws Exception if (!node.isOffline() && !bundle.getSLA().isRemoteClassLoadingEnabled()) cont.getClassLoader().setRemoteClassLoadingDisabled(true); node.getLifeCycleEventHandler().fireJobHeaderLoaded(bundle, cont.getClassLoader()); cont.setCurrentMessage(currentMessage); - cont.deserializeObjects(list, 1+count, node.getExecutionManager().getExecutor()); + cont.deserializeObjects(list, 1+count, node.getSerializationExecutor()); } else { // skip null data provider } diff --git a/server/src/java/org/jppf/server/node/local/AsyncLocalNodeIO.java b/server/src/java/org/jppf/server/node/local/AsyncLocalNodeIO.java index 315c3d7221..ca33fe7d34 100644 --- a/server/src/java/org/jppf/server/node/local/AsyncLocalNodeIO.java +++ b/server/src/java/org/jppf/server/node/local/AsyncLocalNodeIO.java @@ -25,7 +25,7 @@ import org.jppf.node.protocol.*; import org.jppf.server.nio.nodeserver.LocalNodeMessage; import org.jppf.server.nio.nodeserver.async.*; -import org.jppf.server.node.JPPFContainer; +import org.jppf.server.node.*; import org.jppf.utils.LoggingUtils; import org.slf4j.*; @@ -59,7 +59,7 @@ public AsyncLocalNodeIO(final JPPFLocalNode node) { @Override protected Object[] deserializeObjects() throws Exception { Object[] result = null; - synchronized(channel.getLocalNodeLock()) { + synchronized(channel.getLocalNodeReadLock()) { if (debugEnabled) log.debug("waiting for next request"); // wait until a message has been sent by the server currentMessage = (LocalNodeMessage) channel.takeNextMessageToSend(); @@ -68,7 +68,6 @@ protected Object[] deserializeObjects() throws Exception { final DataLocation location = currentMessage.getLocations().get(0); final TaskBundle bundle = (TaskBundle) IOHelper.unwrappedData(location, node.getHelper().getSerializer()); if (debugEnabled) log.debug("got bundle " + bundle); - node.getExecutionManager().setBundle(bundle); result = deserializeObjects(bundle); if (debugEnabled) log.debug("got all data"); return result; @@ -77,19 +76,20 @@ protected Object[] deserializeObjects() throws Exception { @Override protected void sendResults(final TaskBundle bundle, final List> tasks) throws Exception { if (debugEnabled) log.debug("writing {} results for {}", tasks.size(), bundle); - final ExecutorService executor = node.getExecutionManager().getExecutor(); + final ExecutorService executor = node.getSerializationExecutor(); finalizeBundleData(bundle, tasks); final List> futureList = new ArrayList<>(tasks.size() + 1); final JPPFContainer cont = node.getContainer(bundle.getUuidPath().getList()); - futureList.add(executor.submit(new ObjectSerializationTask(bundle, cont.getSerializer(), cont.getClassLoader()))); - for (Task task : tasks) futureList.add(executor.submit(new ObjectSerializationTask(task, cont.getSerializer(), cont.getClassLoader()))); + int submitCount = 0; + futureList.add(executor.submit(new ObjectSerializationTask(bundle, cont, submitCount++))); + for (Task task : tasks) futureList.add(executor.submit(new ObjectSerializationTask(task, cont, submitCount++))); final LocalNodeMessage message = (LocalNodeMessage) channel.newMessage(); for (final Future f: futureList) { final DataLocation location = f.get(); message.addLocation(location); } message.setBundle(bundle); - synchronized(channel.getLocalNodeLock()) { + synchronized(channel.getLocalNodeWriteLock()) { if (debugEnabled) log.debug("wrote full results"); // wait until the message has been read by the server final AsyncNodeMessageHandler handler = channel.getServer().getMessageHandler(); diff --git a/server/src/java/org/jppf/server/node/local/JPPFLocalContainer.java b/server/src/java/org/jppf/server/node/local/JPPFLocalContainer.java index f03bde8c85..421adae2e9 100644 --- a/server/src/java/org/jppf/server/node/local/JPPFLocalContainer.java +++ b/server/src/java/org/jppf/server/node/local/JPPFLocalContainer.java @@ -23,7 +23,7 @@ import org.jppf.classloader.AbstractJPPFClassLoader; import org.jppf.io.*; import org.jppf.server.nio.nodeserver.LocalNodeMessage; -import org.jppf.server.node.JPPFContainer; +import org.jppf.server.node.*; import org.slf4j.*; /** @@ -49,13 +49,14 @@ public class JPPFLocalContainer extends JPPFContainer { /** * Initialize this container with a specified application uuid. + * @param node the node holding this container. * @param uuidPath the unique identifier of a submitting application. * @param classLoader the class loader for this container. * @param clientAccess whether the node has access to the client that submitted the job. * @throws Exception if an error occurs while initializing. */ - public JPPFLocalContainer(final List uuidPath, final AbstractJPPFClassLoader classLoader, final boolean clientAccess) throws Exception { - super(uuidPath, classLoader, clientAccess); + public JPPFLocalContainer(final AbstractCommonNode node, final List uuidPath, final AbstractJPPFClassLoader classLoader, final boolean clientAccess) throws Exception { + super(node, uuidPath, classLoader, clientAccess); } /** diff --git a/server/src/java/org/jppf/server/node/local/JPPFLocalNode.java b/server/src/java/org/jppf/server/node/local/JPPFLocalNode.java index 51dc8ff510..5298a45f6a 100644 --- a/server/src/java/org/jppf/server/node/local/JPPFLocalNode.java +++ b/server/src/java/org/jppf/server/node/local/JPPFLocalNode.java @@ -47,7 +47,7 @@ public JPPFLocalNode(final TypedProperties configuration, final AbstractLocalNod } @Override - protected void initDataChannel() throws Exception { + public void initDataChannel() throws Exception { if (nodeConnection.getChannel() instanceof AsyncNodeContext) nodeIO = new AsyncLocalNodeIO(this); else nodeIO = new LocalNodeIO(this); } @@ -64,11 +64,6 @@ LocalClassLoaderConnection getClassLoaderConnection() { return classLoaderConnection; } - @Override - protected NodeConnectionChecker createConnectionChecker() { - return new LocalNodeConnectionChecker(); - } - @Override public boolean isLocal() { return true; diff --git a/server/src/java/org/jppf/server/node/local/LocalClassLoaderManager.java b/server/src/java/org/jppf/server/node/local/LocalClassLoaderManager.java index 79f10cc645..173346fb29 100644 --- a/server/src/java/org/jppf/server/node/local/LocalClassLoaderManager.java +++ b/server/src/java/org/jppf/server/node/local/LocalClassLoaderManager.java @@ -52,7 +52,7 @@ public AbstractJPPFClassLoader run() { @Override protected JPPFContainer newJPPFContainer(final List uuidPath, final AbstractJPPFClassLoader cl, final boolean clientAccess) throws Exception { - return new JPPFLocalContainer(uuidPath, cl, clientAccess); + return new JPPFLocalContainer(node, uuidPath, cl, clientAccess); } @Override diff --git a/server/src/java/org/jppf/server/node/local/LocalNodeIO.java b/server/src/java/org/jppf/server/node/local/LocalNodeIO.java index 96cb52e1d4..656cad80a6 100644 --- a/server/src/java/org/jppf/server/node/local/LocalNodeIO.java +++ b/server/src/java/org/jppf/server/node/local/LocalNodeIO.java @@ -26,7 +26,7 @@ import org.jppf.io.*; import org.jppf.node.protocol.*; import org.jppf.server.nio.nodeserver.*; -import org.jppf.server.node.JPPFContainer; +import org.jppf.server.node.*; import org.jppf.utils.LoggingUtils; import org.slf4j.*; @@ -72,7 +72,6 @@ protected Object[] deserializeObjects() throws Exception { final DataLocation location = currentMessage.getLocations().get(0); final TaskBundle bundle = (TaskBundle) IOHelper.unwrappedData(location, node.getHelper().getSerializer()); if (debugEnabled) log.debug("got bundle " + bundle); - node.getExecutionManager().setBundle(bundle); result = deserializeObjects(bundle); if (debugEnabled) log.debug("got all data"); return result; @@ -81,12 +80,13 @@ protected Object[] deserializeObjects() throws Exception { @Override protected void sendResults(final TaskBundle bundle, final List> tasks) throws Exception { if (debugEnabled) log.debug("writing {} results for {}", tasks.size(), bundle); - final ExecutorService executor = node.getExecutionManager().getExecutor(); + final ExecutorService executor = node.getSerializationExecutor(); finalizeBundleData(bundle, tasks); final List> futureList = new ArrayList<>(tasks.size() + 1); final JPPFContainer cont = node.getContainer(bundle.getUuidPath().getList()); - futureList.add(executor.submit(new ObjectSerializationTask(bundle, cont.getSerializer(), cont.getClassLoader()))); - for (Task task : tasks) futureList.add(executor.submit(new ObjectSerializationTask(task, cont.getSerializer(), cont.getClassLoader()))); + int submitCount = 0; + futureList.add(executor.submit(new ObjectSerializationTask(bundle, cont, submitCount++))); + for (Task task : tasks) futureList.add(executor.submit(new ObjectSerializationTask(task, cont, submitCount++))); final LocalNodeContext ctx = channel.getChannel(); final LocalNodeMessage message = (LocalNodeMessage) ctx.newMessage(); for (final Future f: futureList) { diff --git a/server/src/java/org/jppf/server/node/remote/AbstractRemoteNode.java b/server/src/java/org/jppf/server/node/remote/AbstractRemoteNode.java index 5d59e30f3c..e00f1d9ab7 100644 --- a/server/src/java/org/jppf/server/node/remote/AbstractRemoteNode.java +++ b/server/src/java/org/jppf/server/node/remote/AbstractRemoteNode.java @@ -67,7 +67,7 @@ public AbstractRemoteNode(final String uuid, final TypedProperties configuration protected abstract void initClassLoaderManager(); @Override - protected void initDataChannel() throws Exception { + public void initDataChannel() throws Exception { (nodeConnection = new RemoteNodeConnection(connectionInfo, serializer)).init(); if (nodeIO == null) nodeIO = new RemoteNodeIO(this); if (configuration.get(JPPFProperties.RECOVERY_ENABLED)) { @@ -105,11 +105,6 @@ public void heartbeatConnectionFailed(final HeartbeatConnectionEvent event) { } } - @Override - protected NodeConnectionChecker createConnectionChecker() { - return new RemoteNodeConnectionChecker(this); - } - @Override public boolean isLocal() { return false; diff --git a/server/src/java/org/jppf/server/node/remote/JPPFRemoteContainer.java b/server/src/java/org/jppf/server/node/remote/JPPFRemoteContainer.java index e298d15a7c..74d2e70b2f 100644 --- a/server/src/java/org/jppf/server/node/remote/JPPFRemoteContainer.java +++ b/server/src/java/org/jppf/server/node/remote/JPPFRemoteContainer.java @@ -22,7 +22,7 @@ import org.jppf.classloader.AbstractJPPFClassLoader; import org.jppf.io.*; -import org.jppf.server.node.JPPFContainer; +import org.jppf.server.node.*; import org.jppf.utils.ExceptionUtils; import org.slf4j.*; @@ -49,19 +49,19 @@ public class JPPFRemoteContainer extends JPPFContainer { /** * The socket connection wrapper. */ - private RemoteNodeConnection nodeConnection = null; + private RemoteNodeConnection nodeConnection; /** * Initialize this container with a specified application uuid. - * @param nodeConnection the connection to the job server. + * @param node the node holding this container. * @param uuidPath the unique identifier of a submitting application. * @param classLoader the class loader for this container. * @param clientAccess whether the node has access to the client that submitted the job. * @throws Exception if an error occurs while initializing. */ - public JPPFRemoteContainer(final RemoteNodeConnection nodeConnection, final List uuidPath, final AbstractJPPFClassLoader classLoader, final boolean clientAccess) throws Exception { - super(uuidPath, classLoader, clientAccess); - this.nodeConnection = nodeConnection; + public JPPFRemoteContainer(final AbstractCommonNode node, final List uuidPath, final AbstractJPPFClassLoader classLoader, final boolean clientAccess) throws Exception { + super(node, uuidPath, classLoader, clientAccess); + this.nodeConnection = (RemoteNodeConnection) node.getNodeConnection(); //init(); } @@ -84,7 +84,7 @@ public int deserializeObjects(final Object[] list, final int count, final Execut for (int i = 0; i < count; i++) { final DataLocation dl = IOHelper.readData(is); if (traceEnabled) log.trace("i = " + i + ", read data size = " + dl.getSize()); - completionService.submit(new ObjectDeserializationTask(dl, i)); + completionService.submit(new ObjectDeserializationTask(this, dl, i)); } Throwable t = null; int throwableCount = 0; diff --git a/server/src/java/org/jppf/server/node/remote/RemoteClassLoaderManager.java b/server/src/java/org/jppf/server/node/remote/RemoteClassLoaderManager.java index 2a0a9c151a..1be065136f 100644 --- a/server/src/java/org/jppf/server/node/remote/RemoteClassLoaderManager.java +++ b/server/src/java/org/jppf/server/node/remote/RemoteClassLoaderManager.java @@ -57,23 +57,17 @@ protected AbstractJPPFClassLoader createClassLoader() { @Override protected JPPFContainer newJPPFContainer(final List uuidPath, final AbstractJPPFClassLoader cl, final boolean clientAccess) throws Exception { - return new JPPFRemoteContainer((RemoteNodeConnection) node.getNodeConnection(), uuidPath, cl, clientAccess); + return new JPPFRemoteContainer(node, uuidPath, cl, clientAccess); } @Override protected Callable newClassLoaderCreator(final List uuidPath, final Object...params) { - return new Callable() { - @Override - public AbstractJPPFClassLoader call() { - final PrivilegedAction pa = new PrivilegedAction() { - @Override - public AbstractJPPFClassLoader run() { - final AbstractJPPFClassLoader parent = getClassLoader(); - return new JPPFClassLoader(parent.getConnection(), parent, uuidPath); - } - }; - return AccessController.doPrivileged(pa); - } + return () -> { + final PrivilegedAction pa = () -> { + final AbstractJPPFClassLoader parent = getClassLoader(); + return new JPPFClassLoader(parent.getConnection(), parent, uuidPath); + }; + return AccessController.doPrivileged(pa); }; } } diff --git a/server/src/java/org/jppf/server/node/remote/RemoteNodeConnectionChecker.java b/server/src/java/org/jppf/server/node/remote/RemoteNodeConnectionChecker.java deleted file mode 100644 index 76c9000e8d..0000000000 --- a/server/src/java/org/jppf/server/node/remote/RemoteNodeConnectionChecker.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * JPPF. - * Copyright (C) 2005-2018 JPPF Team. - * http://www.jppf.org - * - * Licensed 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.jppf.server.node.remote; - -import java.net.SocketTimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.jppf.comm.socket.SocketWrapper; -import org.jppf.server.node.AbstractNodeConnectionChecker; -import org.jppf.utils.*; -import org.jppf.utils.concurrent.*; -import org.slf4j.*; - -/** - * Connection checker implementation for a remote node. - * @author Laurent Cohen - * @exclude - */ -public class RemoteNodeConnectionChecker extends AbstractNodeConnectionChecker { - /** - * Logger for this class. - */ - private static final Logger log = LoggerFactory.getLogger(RemoteNodeConnectionChecker.class); - /** - * Determines whether the debug level is enabled in the logging configuration, without the cost of a method call. - */ - private static final boolean debugEnabled = LoggingUtils.isDebugEnabled(log); - /** - * The node for which to check the connection. - */ - private final AbstractRemoteNode node; - /** - * The socket connection. - */ - private final SocketWrapper socketWrapper; - /** - * The thread which performs the checks. - */ - private CheckerThread checkerThread = null; - /** - * - */ - private final ThreadSynchronization suspendedLock = new ThreadSynchronization() {}; - - /** - * Initialize this checker list = new ArrayList<>(count + 2); final Object[] list = new Object[count + 2]; list[0] = bundle; try { initializeBundleData(bundle); if (debugEnabled) log.debug("bundle task count = " + count + ", handshake = " + bundle.isHandshake()); if (!bundle.isHandshake()) { - //JPPFRemoteContainer cont = (JPPFRemoteContainer) node.getContainer(bundle.getUuidPath().getList()); final boolean clientAccess = !bundle.getParameter(FROM_PERSISTENCE, false); final JPPFRemoteContainer cont = (JPPFRemoteContainer) node.getClassLoaderManager().getContainer(bundle.getUuidPath().getList(), clientAccess, (Object[]) null); cont.setNodeConnection((RemoteNodeConnection) node.getNodeConnection()); cont.getClassLoader().setRequestUuid(bundle.getUuid()); if (!node.isOffline() && !bundle.getSLA().isRemoteClassLoadingEnabled()) cont.getClassLoader().setRemoteClassLoadingDisabled(true); node.getLifeCycleEventHandler().fireJobHeaderLoaded(bundle, cont.getClassLoader()); - cont.deserializeObjects(list, 1+count, node.getExecutionManager().getExecutor()); + cont.deserializeObjects(list, 1+count, node.getSerializationExecutor()); } else getSocketWrapper().receiveBytes(0); // skip null data provider if (debugEnabled) log.debug("got all data"); @@ -126,15 +118,15 @@ protected void handleReload() throws Exception { @Override protected void sendResults(final TaskBundle bundle, final List> tasks) throws Exception { if (debugEnabled) log.debug("writing results for " + bundle); - //System.out.println("writing results for " + bundle); final SocketWrapper socketWrapper = getSocketWrapper(); if (socketWrapper == null) throw new SocketException("no connection to the server"); - final ExecutorService executor = node.getExecutionManager().getExecutor(); + final ExecutorService executor = node.getSerializationExecutor(); finalizeBundleData(bundle, tasks); final List> futureList = new ArrayList<>(tasks.size() + 1); final JPPFContainer cont = node.getContainer(bundle.getUuidPath().getList()); - futureList.add(executor.submit(new ObjectSerializationTask(bundle, cont.getSerializer(), cont.getClassLoader()))); - for (Task task : tasks) futureList.add(executor.submit(new ObjectSerializationTask(task, cont.getSerializer(), cont.getClassLoader()))); + int submitCount = 0; + futureList.add(executor.submit(new ObjectSerializationTask(bundle, cont, submitCount++))); + for (Task task : tasks) futureList.add(executor.submit(new ObjectSerializationTask(task, cont, submitCount++))); final OutputDestination dest = new SocketWrapperOutputDestination(socketWrapper); int count = 0; for (final Future f: futureList) { @@ -165,7 +157,7 @@ private void waitChannelClosed(final SocketWrapper socketWrapper) { } catch (final Error e) { if (debugEnabled) log.debug("error closing data channel: ", e); } - if (traceEnabled) log.trace("closed the data channel"); + if (debugEnabled) log.debug("closed the data channel"); } /** diff --git a/server/src/java/org/jppf/server/protocol/ServerTaskBundleNode.java b/server/src/java/org/jppf/server/protocol/ServerTaskBundleNode.java index bba7d1ae26..11a4cd0a32 100644 --- a/server/src/java/org/jppf/server/protocol/ServerTaskBundleNode.java +++ b/server/src/java/org/jppf/server/protocol/ServerTaskBundleNode.java @@ -209,7 +209,7 @@ public void resultsReceived(final Throwable throwable) { * @param exception the {@link Exception} thrown during job execution or null. */ public void taskCompleted(final Throwable exception) { - if (debugEnabled && (exception != null)) log.debug("received exception for {} : {}\ncall stack:\n{}", this, ExceptionUtils.getStackTrace(exception), ExceptionUtils.getCallStack()); + if (debugEnabled && (exception != null)) log.debug("received exception for {} :\n{}\ncall stack:\n{}", this, ExceptionUtils.getStackTrace(exception), ExceptionUtils.getCallStack()); try { job.jobReturned(this); } finally { diff --git a/server/src/java/org/jppf/server/queue/BroadcastManager.java b/server/src/java/org/jppf/server/queue/BroadcastManager.java index c5eee2553c..8b8a1970b4 100644 --- a/server/src/java/org/jppf/server/queue/BroadcastManager.java +++ b/server/src/java/org/jppf/server/queue/BroadcastManager.java @@ -28,7 +28,8 @@ import org.jppf.node.policy.*; import org.jppf.node.protocol.JobSLA; import org.jppf.queue.QueueEvent; -import org.jppf.server.nio.nodeserver.*; +import org.jppf.server.nio.nodeserver.BaseNodeContext; +import org.jppf.server.nio.nodeserver.async.AsyncJobScheduler; import org.jppf.server.protocol.*; import org.jppf.server.submission.SubmissionStatus; import org.jppf.utils.*; @@ -67,7 +68,7 @@ public class BroadcastManager { /** * Callback for getting all available connections. Used for processing broadcast jobs. */ - private Callable>> callableAllConnections = () -> Collections.emptyList(); + private Callable>> callableAllConnections = () -> Collections.emptyList(); /** * */ @@ -87,7 +88,7 @@ public class BroadcastManager { * Set the callable source for all available connections. * @param callableAllConnections a {@link Callable} instance. */ - void setCallableAllConnections(final Callable>> callableAllConnections) { + void setCallableAllConnections(final Callable>> callableAllConnections) { if (callableAllConnections == null) this.callableAllConnections = () -> Collections.emptyList(); else this.callableAllConnections = callableAllConnections; } @@ -155,7 +156,7 @@ public void cancelBroadcastJobs(final String connectionUUID) { */ public void processPendingBroadcasts() { if (nbWorkingConnections.get() <= 0) return; - List> connections; + List> connections; try { connections = callableAllConnections.call(); } catch (@SuppressWarnings("unused") final Throwable e) { @@ -174,20 +175,20 @@ public void processPendingBroadcasts() { * @param connections the list of all available connections. * @param broadcastJob the job to dispatch to connections. */ - private void processPendingBroadcast(final List> connections, final ServerJobBroadcast broadcastJob) { + private void processPendingBroadcast(final List> connections, final ServerJobBroadcast broadcastJob) { if (broadcastJob == null) throw new IllegalArgumentException("broadcastJob is null"); if (pendingBroadcasts.remove(broadcastJob.getUuid()) == null) return; final JobSLA sla = broadcastJob.getSLA(); final List jobList = new ArrayList<>(connections.size()); final Set uuidSet = new HashSet<>(); - for (final AbstractBaseNodeContext connection : connections) { + for (final BaseNodeContext connection : connections) { final ExecutorStatus status = connection.getExecutionStatus(); if (status == ExecutorStatus.ACTIVE || status == ExecutorStatus.EXECUTING) { final String uuid = connection.getUuid(); if (uuid != null && uuid.length() > 0 && uuidSet.add(uuid)) { final JPPFManagementInfo info = connection.getManagementInfo(); final ExecutionPolicy policy = sla.getExecutionPolicy(); - TaskQueueChecker.preparePolicy(policy, broadcastJob, queue.driver.getStatistics(), 0); + AsyncJobScheduler.preparePolicy(policy, broadcastJob, queue.driver.getStatistics(), 0); if ((policy != null) && !policy.evaluate(info.getSystemInfo())) { if (debugEnabled) log.debug("node uuid={} refused for broadcast {}", uuid, broadcastJob); continue; diff --git a/server/src/java/org/jppf/server/queue/JPPFPriorityQueue.java b/server/src/java/org/jppf/server/queue/JPPFPriorityQueue.java index b1f6379c3e..b7e3dd57af 100644 --- a/server/src/java/org/jppf/server/queue/JPPFPriorityQueue.java +++ b/server/src/java/org/jppf/server/queue/JPPFPriorityQueue.java @@ -22,7 +22,7 @@ import java.util.*; import java.util.concurrent.Callable; -import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.*; import org.jppf.execute.ExecutorStatus; import org.jppf.job.*; @@ -30,7 +30,7 @@ import org.jppf.queue.*; import org.jppf.server.JPPFDriver; import org.jppf.server.job.*; -import org.jppf.server.nio.nodeserver.AbstractBaseNodeContext; +import org.jppf.server.nio.nodeserver.BaseNodeContext; import org.jppf.server.protocol.*; import org.jppf.server.submission.SubmissionStatus; import org.jppf.utils.*; @@ -77,6 +77,14 @@ public class JPPFPriorityQueue extends AbstractJPPFQueue jobRemovalConditions = new HashMap<>(); + /** + * + */ + private final boolean testJobRemoval; /** * Initialize this queue. @@ -88,6 +96,7 @@ public JPPFPriorityQueue(final JPPFDriver driver, final JPPFJobManager jobManage this.jobManager = jobManager; broadcastManager = new BroadcastManager(this); this.persistenceHandler = new PersistenceHandler(this); + this.testJobRemoval = driver.getConfiguration().getBoolean("jppf.test.job.removal", false); } @Override @@ -120,7 +129,7 @@ public ServerJob addBundle(final ServerTaskBundleClient clientBundle) { done = true; } catch (final JPPFJobEndedException e) { if (debugEnabled) log.debug("caught {}, awaiting removal of {}", ExceptionUtils.getMessage(e), serverJob); - awaitJobRemoved(serverJob, true); + awaitJobRemoved(serverJob); } } if (added) { @@ -153,16 +162,20 @@ public ServerJob addBundle(final ServerTaskBundleClient clientBundle) { /** * * @param serverJob the job to remove. - * @param handleLock whther o handle the queue lock. */ - void awaitJobRemoved(final ServerJob serverJob, final boolean handleLock) { + void awaitJobRemoved(final ServerJob serverJob) { if (debugEnabled) log.debug("awaiting removal of {}", serverJob); - while (jobMap.get(serverJob.getUuid()) != null) { + final String uuid = serverJob.getUuid(); + while (jobMap.get(uuid) != null) { try { - if (handleLock) lock.unlock(); - serverJob.getRemovalCondition().goToSleep(100L); - } finally { - if (handleLock) lock.lock(); + Condition cond = jobRemovalConditions.get(uuid); + if (cond == null) { + cond = lock.newCondition(); + jobRemovalConditions.put(uuid, cond); + } + cond.await(); + } catch(final InterruptedException e) { + log.error(e.getMessage(), e); } } } @@ -175,8 +188,6 @@ void awaitJobRemoved(final ServerJob serverJob, final boolean handleLock) { private ServerJob createServerJob(final ServerTaskBundleClient clientBundle) { final TaskBundle header = clientBundle.getJob(); header.setDriverQueueTaskCount(header.getTaskCount()); - //final Lock jobLock = ConcurrentUtils.newLock("ServerJob[name=" + header.getName() + ", uuid=" + header.getUuid() + "]"); - //final Lock jobLock = ConcurrentUtils.newLock(); final Lock jobLock = ConcurrentUtils.newLock("job-" + header.getName()); final ServerJob serverJob = new ServerJob(jobLock, jobManager, header, clientBundle.getDataProvider()); serverJob.setSubmissionStatus(SubmissionStatus.PENDING); @@ -251,14 +262,25 @@ public ServerJob removeBundle(final ServerJob serverJob) { */ public ServerJob removeBundle(final ServerJob serverJob, final boolean removeFromJobMap) { if (serverJob == null) throw new IllegalArgumentException("serverJob is null"); + // TODO: remove start + if (testJobRemoval) { + try { + Thread.sleep(10L); + } catch (final Exception e) { + log.error(e.getMessage(), e); + } + } + // TODO: remove end lock.lock(); try { if (removeFromJobMap) { - if (jobMap.remove(serverJob.getUuid()) != null) { + final String uuid = serverJob.getUuid(); + if (jobMap.remove(uuid) != null) { scheduleManager.clearSchedules(serverJob.getUuid()); jobManager.jobEnded(serverJob); } - serverJob.getRemovalCondition().wakeUp(); + final Condition cond = jobRemovalConditions.remove(uuid); + if (cond != null) cond.signalAll(); } if (debugEnabled) log.debug("removing job from queue, jobName= {}, removeFromJobMap={}", serverJob.getName(), removeFromJobMap); if (priorityMap.removeValue(serverJob.getSLA().getPriority(), serverJob)) { @@ -456,7 +478,7 @@ LinkedListSortedMap getPriorityMap() { * Set the callable source for all available connections. * @param callableAllConnections a {@link Callable} instance. */ - public void setCallableAllConnections(final Callable>> callableAllConnections) { + public void setCallableAllConnections(final Callable>> callableAllConnections) { broadcastManager.setCallableAllConnections(callableAllConnections); } diff --git a/tests/.classpath b/tests/.classpath index 886f08c437..03899520fc 100644 --- a/tests/.classpath +++ b/tests/.classpath @@ -12,7 +12,7 @@ - + diff --git a/tests/src/framework/test/org/jppf/test/setup/AbstractNonStandardSetup.java b/tests/src/framework/test/org/jppf/test/setup/AbstractNonStandardSetup.java index 6d39e4129d..83843c47de 100644 --- a/tests/src/framework/test/org/jppf/test/setup/AbstractNonStandardSetup.java +++ b/tests/src/framework/test/org/jppf/test/setup/AbstractNonStandardSetup.java @@ -117,7 +117,7 @@ protected void testSimpleJob(final ExecutionPolicy policy) throws Exception { * @throws Exception if any error occurs */ protected void testSimpleJob(final ExecutionPolicy policy, final String nodePrefix) throws Exception { - System.out.printf("driver load balancing config: %s%n", BaseSetup.getJMXConnection(client).loadBalancerInformation()); + print(false, false, "driver load balancing config: %s", BaseSetup.getJMXConnection(client).loadBalancerInformation()); final int tasksPerNode = 5; final int nbNodes = getNbNodes(); final int nbTasks = tasksPerNode * nbNodes; @@ -137,7 +137,7 @@ protected void testSimpleJob(final ExecutionPolicy policy, final String nodePref assertNotNull(t.getResult()); assertEquals(BaseTestHelper.EXECUTION_SUCCESSFUL_MESSAGE, t.getResult()); } - BaseTest.printOut("%s : map = %s", name , CollectionUtils.prettyPrint(map)); + printOut("%s : map = %s", name , CollectionUtils.prettyPrint(map)); assertEquals(nbNodes, map.keySet().size()); for (int i=0; i>> awaiting 2 pools"); - final List pools = client.awaitConnectionPools(Operator.AT_LEAST, 2, Operator.AT_LEAST, 1, timeout, JPPFClientConnectionStatus.workingStatuses()); - if (pools.size() < 2) fail("timeout of " + timeout + " ms waiting for 2 pools expired"); - final List jmxList = new ArrayList<>(2); + print(false, false, ">>> awaiting %d pools", nbDrivers); + final List pools = client.awaitConnectionPools(Operator.AT_LEAST, nbDrivers, Operator.AT_LEAST, 1, timeout, JPPFClientConnectionStatus.workingStatuses()); + if (pools.size() < nbDrivers) fail("timeout of " + timeout + " ms waiting for " + nbDrivers + " pools expired"); + final List jmxList = new ArrayList<>(nbDrivers); for (final JPPFConnectionPool pool: pools) { print(false, false, ">>> awaiting JMX connection for %s", pool); final MutableReference jmx = new MutableReference<>(); timeout = maxWait - (System.currentTimeMillis() - start); if (timeout < 0L) throw new JPPFTimeoutException("execeeded maxWait timeout of " + maxWait + " ms"); - ConcurrentUtils.awaitCondition(new ConcurrentUtils.Condition() { - @Override - public boolean evaluate() { - final JMXDriverConnectionWrapper driver = pool.getJmxConnection(true); - if (driver == null) return false; - jmx.set(driver); - return true; - } + ConcurrentUtils.awaitCondition(() -> { + final JMXDriverConnectionWrapper driver = pool.getJmxConnection(true); + if (driver == null) return false; + jmx.set(driver); + return true; }, timeout, 500L, true); print(false, false, ">>> got JMX connection %s", jmx.get()); jmxList.add(jmx.get()); diff --git a/tests/src/framework/test/org/jppf/test/setup/BaseSetup.java b/tests/src/framework/test/org/jppf/test/setup/BaseSetup.java index 483efe2128..beb8af602c 100644 --- a/tests/src/framework/test/org/jppf/test/setup/BaseSetup.java +++ b/tests/src/framework/test/org/jppf/test/setup/BaseSetup.java @@ -329,7 +329,7 @@ public static void checkDriverAndNodesInitialized(final JPPFClient client, final if (!connectionMap.containsKey(pool.getDriverPort())) connectionMap.put(pool.getDriverPort(), pool); } } - if (connectionMap.size() < nbDrivers) Thread.sleep(10L); + if (connectionMap.size() < nbDrivers) Thread.sleep(100L); else allConnected = true; } final Map wrapperMap = new HashMap<>(); diff --git a/tests/src/framework/test/org/jppf/test/setup/BaseTest.java b/tests/src/framework/test/org/jppf/test/setup/BaseTest.java index 2de0b7154f..e711a912dd 100644 --- a/tests/src/framework/test/org/jppf/test/setup/BaseTest.java +++ b/tests/src/framework/test/org/jppf/test/setup/BaseTest.java @@ -304,10 +304,11 @@ public boolean accept(final File file) { /** * Check the result of a comparison and throw an {@code AssertionError} if the comparison fails (i.e. return true). * @param op the comparison operator to use. - * @param actual the actual value to test. * @param expected the expected value to test against. + * @param actual the actual value to test. */ - public static void assertCompare(final ComparisonOperator op, final long actual, final long expected) { - if (!op.evaluate(actual, expected)) Assert.fail(String.format("expected <%d> to be '%s' <%d>", actual, op, expected)); + public static void assertCompare(final ComparisonOperator op, final long expected, final long actual) { + //if (!op.evaluate(actual, expected)) Assert.fail(String.format("expected <%d> to be '%s' <%d>", actual, op, expected)); + if (!op.evaluate(actual, expected)) Assert.fail(String.format("expected: <%s %d> but was: <%d>", op, expected, actual)); } } diff --git a/tests/src/framework/test/org/jppf/test/setup/common/LifeCycleTask.java b/tests/src/framework/test/org/jppf/test/setup/common/LifeCycleTask.java index 9fb14503c7..9e170d7081 100644 --- a/tests/src/framework/test/org/jppf/test/setup/common/LifeCycleTask.java +++ b/tests/src/framework/test/org/jppf/test/setup/common/LifeCycleTask.java @@ -42,23 +42,23 @@ public class LifeCycleTask extends AbstractTask { /** * The duration of this task. */ - protected long duration = 0L; + protected long duration; /** * used to store the task's execution start time in nanoseconds. */ - protected double start = 0L; + protected double start; /** * Measures the time elapsed between the task execution start and its completion in nanoseconds. */ - protected double elapsed = 0L; + protected double elapsed; /** * Determines whether this task was cancelled. */ - protected boolean cancelled = false; + protected boolean cancelled; /** * Determines whether this task timed out. */ - protected boolean timedout = false; + protected boolean timedout; /** * Determines whether this task was executed in a node or in the client's local executor. */ @@ -66,7 +66,7 @@ public class LifeCycleTask extends AbstractTask { /** * The uuid of the node this task executes on. */ - protected String nodeUuid = null; + protected String nodeUuid; /** * Whether this task can be interrupted upon cancellation or timeout. */ @@ -74,7 +74,7 @@ public class LifeCycleTask extends AbstractTask { /** * Whether the thread running this task was interrupted upon cancellation or timeout. */ - protected boolean interrupted = false; + protected boolean interrupted; /** * A message to send via JMX. */ @@ -82,7 +82,7 @@ public class LifeCycleTask extends AbstractTask { /** * The uuid of the node obtained via {@code Task.getNode().getUuid()}. */ - protected String uuidFromNode = null; + protected String uuidFromNode; /** * Initialize this task. diff --git a/tests/src/resources/config/classloader/driver.properties b/tests/src/resources/config/classloader/driver.properties index e5416ef953..3def501990 100644 --- a/tests/src/resources/config/classloader/driver.properties +++ b/tests/src/resources/config/classloader/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/db/driver.properties b/tests/src/resources/config/db/driver.properties index 35545514b8..a07f3352ff 100644 --- a/tests/src/resources/config/db/driver.properties +++ b/tests/src/resources/config/db/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/discovery/driver.properties b/tests/src/resources/config/discovery/driver.properties index ebe3ef799d..16cfc4d641 100644 --- a/tests/src/resources/config/discovery/driver.properties +++ b/tests/src/resources/config/discovery/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/driver.template.properties b/tests/src/resources/config/driver.template.properties index f7b1b87b80..3350c18ff4 100644 --- a/tests/src/resources/config/driver.template.properties +++ b/tests/src/resources/config/driver.template.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/job_persistence/driver-common.properties b/tests/src/resources/config/job_persistence/driver-common.properties index 7edc7c5810..9f754ad308 100644 --- a/tests/src/resources/config/job_persistence/driver-common.properties +++ b/tests/src/resources/config/job_persistence/driver-common.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/job_persistence/log4j-driver.template.properties b/tests/src/resources/config/job_persistence/log4j-driver.properties similarity index 100% rename from tests/src/resources/config/job_persistence/log4j-driver.template.properties rename to tests/src/resources/config/job_persistence/log4j-driver.properties diff --git a/tests/src/resources/config/job_persistence_p2p/driver.properties b/tests/src/resources/config/job_persistence_p2p/driver.properties index 1e41e0ab57..54541ef27c 100644 --- a/tests/src/resources/config/job_persistence_p2p/driver.properties +++ b/tests/src/resources/config/job_persistence_p2p/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/job_persistence_p2p/log4j-node.properties b/tests/src/resources/config/job_persistence_p2p/log4j-node.properties new file mode 100644 index 0000000000..71ff3300f0 --- /dev/null +++ b/tests/src/resources/config/job_persistence_p2p/log4j-node.properties @@ -0,0 +1,67 @@ +#------------------------------------------------------------------------------# +# JPPF. # +# Copyright (C) 2005-2018 JPPF Team. # +# http://www.jppf.org # +# # +# Licensed 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. # +#------------------------------------------------------------------------------# + +### direct log messages to stdout ### +#log4j.appender.stdout=org.apache.log4j.ConsoleAppender +#log4j.appender.stdout.Target=System.out +#log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +#log4j.appender.stdout.layout.ConversionPattern=%d{ABSOLUTE} %5p %c{1}:%L - %m%n + +### direct messages to file jppf-node1.log ### +log4j.appender.JPPF=org.apache.log4j.FileAppender +log4j.appender.JPPF.File=expr: "node-" + $n + ".log" +log4j.appender.JPPF.Append=true +log4j.appender.JPPF.layout=org.apache.log4j.PatternLayout +log4j.appender.JPPF.layout.ConversionPattern=%d [%-5p][%-20.20t][%c.%M(%L)]: %m%n + +### set log levels - for more verbose logging change 'info' to 'debug' ### +log4j.rootLogger=INFO, JPPF +#log4j.rootLogger=DEBUG, JPPF + +#log4j.logger.org.jppf.classloader=DEBUG +log4j.logger.org.jppf.classloader.AbstractClassLoaderConnection=DEBUG +log4j.logger.org.jppf.classloader.RemoteClassLoaderConnection=DEBUG +log4j.logger.org.jppf.comm.discovery=INFO +#log4j.logger.org.jppf.comm.execute=INFO +log4j.logger.org.jppf.execute.AbstractExecutionManager=DEBUG +#log4j.logger.org.jppf.management=DEBUG +log4j.logger.org.jppf.management.JPPFNodeAdmin=DEBUG +#log4j.logger.org.jppf.management.JPPFNodeTaskMonitor=DEBUG +#log4j.logger.org.jppf.management.spi=INFO +#log4j.logger.org.jppf.management.spi.JPPFMBeanProviderManager=DEBUG +#log4j.logger.org.jppf.management.JMXServerImpl=INFO +log4j.logger.org.jppf.nio.NioServer=DEBUG +log4j.logger.org.jppf.nio.acceptor=DEBUG +log4j.logger.org.jppf.node=DEBUG +#log4j.logger.org.jppf.node.NodeRunner=DEBUG +log4j.logger.org.jppf.persistence=DEBUG +#log4j.logger.org.jppf.process.LauncherListener=DEBUG +#log4j.logger.org.jppf.serialization.JPPFSerialization$Factory=DEBUG +log4j.logger.org.jppf.server.node=DEBUG +#log4j.logger.org.jppf.server.node.remote=DEBUG +#log4j.logger.org.jppf.server.node.JPPFContainer=DEBUG +#log4j.logger.org.jppf.server.node.JPPFNode=DEBUG +#log4j.logger.org.jppf.server.node.NodeExecutionManager=DEBUG +#log4j.logger.org.jppf.server.node.NodeIO=DEBUG +#log4j.logger.org.jppf.server.node.NodeTaskWrapper=TRACE +log4j.logger.org.jppf.ssl=DEBUG +log4j.logger.org.jppf.utils.NetworkUtils=INFO +#log4j.logger.org.jppf.security=DEBUG +#log4j.logger.sample.taskcommunication=DEBUG +log4j.logger.org.jppf.utils.JPPFDefaultUncaughtExceptionHandler=DEBUG +log4j.logger.test.org.jppf.test.setup.common.NotifyingTask=DEBUG diff --git a/tests/src/resources/config/job_persistence_p2p_db/driver.properties b/tests/src/resources/config/job_persistence_p2p_db/driver.properties index 10d0d81280..f8a66d429e 100644 --- a/tests/src/resources/config/job_persistence_p2p_db/driver.properties +++ b/tests/src/resources/config/job_persistence_p2p_db/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/job_persistence_p2p_db/log4j-node.properties b/tests/src/resources/config/job_persistence_p2p_db/log4j-node.properties new file mode 100644 index 0000000000..71ff3300f0 --- /dev/null +++ b/tests/src/resources/config/job_persistence_p2p_db/log4j-node.properties @@ -0,0 +1,67 @@ +#------------------------------------------------------------------------------# +# JPPF. # +# Copyright (C) 2005-2018 JPPF Team. # +# http://www.jppf.org # +# # +# Licensed 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. # +#------------------------------------------------------------------------------# + +### direct log messages to stdout ### +#log4j.appender.stdout=org.apache.log4j.ConsoleAppender +#log4j.appender.stdout.Target=System.out +#log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +#log4j.appender.stdout.layout.ConversionPattern=%d{ABSOLUTE} %5p %c{1}:%L - %m%n + +### direct messages to file jppf-node1.log ### +log4j.appender.JPPF=org.apache.log4j.FileAppender +log4j.appender.JPPF.File=expr: "node-" + $n + ".log" +log4j.appender.JPPF.Append=true +log4j.appender.JPPF.layout=org.apache.log4j.PatternLayout +log4j.appender.JPPF.layout.ConversionPattern=%d [%-5p][%-20.20t][%c.%M(%L)]: %m%n + +### set log levels - for more verbose logging change 'info' to 'debug' ### +log4j.rootLogger=INFO, JPPF +#log4j.rootLogger=DEBUG, JPPF + +#log4j.logger.org.jppf.classloader=DEBUG +log4j.logger.org.jppf.classloader.AbstractClassLoaderConnection=DEBUG +log4j.logger.org.jppf.classloader.RemoteClassLoaderConnection=DEBUG +log4j.logger.org.jppf.comm.discovery=INFO +#log4j.logger.org.jppf.comm.execute=INFO +log4j.logger.org.jppf.execute.AbstractExecutionManager=DEBUG +#log4j.logger.org.jppf.management=DEBUG +log4j.logger.org.jppf.management.JPPFNodeAdmin=DEBUG +#log4j.logger.org.jppf.management.JPPFNodeTaskMonitor=DEBUG +#log4j.logger.org.jppf.management.spi=INFO +#log4j.logger.org.jppf.management.spi.JPPFMBeanProviderManager=DEBUG +#log4j.logger.org.jppf.management.JMXServerImpl=INFO +log4j.logger.org.jppf.nio.NioServer=DEBUG +log4j.logger.org.jppf.nio.acceptor=DEBUG +log4j.logger.org.jppf.node=DEBUG +#log4j.logger.org.jppf.node.NodeRunner=DEBUG +log4j.logger.org.jppf.persistence=DEBUG +#log4j.logger.org.jppf.process.LauncherListener=DEBUG +#log4j.logger.org.jppf.serialization.JPPFSerialization$Factory=DEBUG +log4j.logger.org.jppf.server.node=DEBUG +#log4j.logger.org.jppf.server.node.remote=DEBUG +#log4j.logger.org.jppf.server.node.JPPFContainer=DEBUG +#log4j.logger.org.jppf.server.node.JPPFNode=DEBUG +#log4j.logger.org.jppf.server.node.NodeExecutionManager=DEBUG +#log4j.logger.org.jppf.server.node.NodeIO=DEBUG +#log4j.logger.org.jppf.server.node.NodeTaskWrapper=TRACE +log4j.logger.org.jppf.ssl=DEBUG +log4j.logger.org.jppf.utils.NetworkUtils=INFO +#log4j.logger.org.jppf.security=DEBUG +#log4j.logger.sample.taskcommunication=DEBUG +log4j.logger.org.jppf.utils.JPPFDefaultUncaughtExceptionHandler=DEBUG +log4j.logger.test.org.jppf.test.setup.common.NotifyingTask=DEBUG diff --git a/tests/src/resources/config/job_reservation/driver.properties b/tests/src/resources/config/job_reservation/driver.properties index c4e53d6c23..7c6153c5a3 100644 --- a/tests/src/resources/config/job_reservation/driver.properties +++ b/tests/src/resources/config/job_reservation/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/lb_persistence_client/driver.properties b/tests/src/resources/config/lb_persistence_client/driver.properties index 915f1d144f..965aa2ea97 100644 --- a/tests/src/resources/config/lb_persistence_client/driver.properties +++ b/tests/src/resources/config/lb_persistence_client/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/lb_persistence_client/log4j-driver.template.properties b/tests/src/resources/config/lb_persistence_client/log4j-driver.properties similarity index 100% rename from tests/src/resources/config/lb_persistence_client/log4j-driver.template.properties rename to tests/src/resources/config/lb_persistence_client/log4j-driver.properties diff --git a/tests/src/resources/config/lb_persistence_driver/driver-common.properties b/tests/src/resources/config/lb_persistence_driver/driver-common.properties index 8ef4d1163a..d3526beb31 100644 --- a/tests/src/resources/config/lb_persistence_driver/driver-common.properties +++ b/tests/src/resources/config/lb_persistence_driver/driver-common.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/lb_persistence_driver/log4j-driver.template.properties b/tests/src/resources/config/lb_persistence_driver/log4j-driver.properties similarity index 100% rename from tests/src/resources/config/lb_persistence_driver/log4j-driver.template.properties rename to tests/src/resources/config/lb_persistence_driver/log4j-driver.properties diff --git a/tests/src/resources/config/lb_persistence_p2p/driver-common.properties b/tests/src/resources/config/lb_persistence_p2p/driver-common.properties index e32d71f5bd..0623597c79 100644 --- a/tests/src/resources/config/lb_persistence_p2p/driver-common.properties +++ b/tests/src/resources/config/lb_persistence_p2p/driver-common.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + jppf.server.port = expr: 11100 + $n #jppf.ssl.server.port = expr: 12100 + $n jppf.ssl.server.port = -1 diff --git a/tests/src/resources/config/localnode/driver.properties b/tests/src/resources/config/localnode/driver.properties index a4fb927d51..a0367e1f23 100644 --- a/tests/src/resources/config/localnode/driver.properties +++ b/tests/src/resources/config/localnode/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/log4j-node.template.properties b/tests/src/resources/config/log4j-node.template.properties index 60fb0f08a6..3fa06a3957 100644 --- a/tests/src/resources/config/log4j-node.template.properties +++ b/tests/src/resources/config/log4j-node.template.properties @@ -37,10 +37,10 @@ log4j.rootLogger=INFO, JPPF log4j.logger.org.jppf.classloader.AbstractClassLoaderConnection=DEBUG log4j.logger.org.jppf.classloader.RemoteClassLoaderConnection=DEBUG log4j.logger.org.jppf.comm.discovery=INFO -#log4j.logger.org.jppf.comm.execute=INFO -log4j.logger.org.jppf.execute.AbstractExecutionManager=DEBUG +log4j.logger.org.jppf.execute=TRACE +#log4j.logger.org.jppf.execute.AbstractExecutionManager=DEBUG +#log4j.logger.org.jppf.jmxremote=DEBUG #log4j.logger.org.jppf.jmxremote=DEBUG -log4j.logger.org.jppf.jmxremote=DEBUG #log4j.logger.org.jppf.management=DEBUG log4j.logger.org.jppf.management.JPPFNodeAdmin=DEBUG #log4j.logger.org.jppf.management.JPPFNodeTaskMonitor=DEBUG @@ -55,12 +55,16 @@ log4j.logger.org.jppf.node=DEBUG #log4j.logger.org.jppf.process.LauncherListener=DEBUG log4j.logger.org.jppf.serialization.JPPFSerialization$Factory=DEBUG log4j.logger.org.jppf.server.node=DEBUG -#log4j.logger.org.jppf.server.node.remote=DEBUG -#log4j.logger.org.jppf.server.node.JPPFContainer=DEBUG +#log4j.logger.org.jppf.server.node.ObjectSerializationTask=TRACE +#log4j.logger.org.jppf.server.node.ObjectDeserializationTask=TRACE #log4j.logger.org.jppf.server.node.JPPFNode=DEBUG #log4j.logger.org.jppf.server.node.NodeExecutionManager=DEBUG -#log4j.logger.org.jppf.server.node.NodeIO=DEBUG +#log4j.logger.org.jppf.server.node.AbstractNodeIO=TRACE #log4j.logger.org.jppf.server.node.NodeTaskWrapper=TRACE +#log4j.logger.org.jppf.server.node.JPPFContainer=TRACE +#log4j.logger.org.jppf.server.node.remote=TRACE +#log4j.logger.org.jppf.server.node.remote.JPPFRemoteContainer=TRACE +#log4j.logger.org.jppf.server.node.remote.RemoteNodeIO=TRACE log4j.logger.org.jppf.ssl=DEBUG log4j.logger.org.jppf.utils.NetworkUtils=INFO #log4j.logger.org.jppf.security=DEBUG diff --git a/tests/src/resources/config/node.template.properties b/tests/src/resources/config/node.template.properties index 843bfd880e..10c72caf80 100644 --- a/tests/src/resources/config/node.template.properties +++ b/tests/src/resources/config/node.template.properties @@ -19,39 +19,27 @@ jppf.debug.enabled = true #------------------------------------------------------------------------------# -# Host name, or ip address, of the host the JPPF driver is running on # -# Defaults to localhost; not used if discovery is enabled. # +# Driver connectivity # #------------------------------------------------------------------------------# jppf.server.host = localhost - -#------------------------------------------------------------------------------# -# port number for the JPPF driver # -# default value is 11111; uncomment to specify a different value # -#------------------------------------------------------------------------------# - #jppf.server.port = expr: 11100 + $n jppf.server.port = 11101 #jppf.management.enabled = false jppf.management.port = expr: 11200 + $n #------------------------------------------------------------------------------# -# SSL Settings # +# SSL Settings # #------------------------------------------------------------------------------# # enable SSL. Default is false (disabled). if enabled, only SSL connections are established #jppf.ssl.enabled = true # location of the SSL configuration on the file system -#jppf.ssl.configuration.file = classes/tests/config/ssl/ssl.properties - -# SSL configuration as an arbitrary source. Value is the fully qualified name -# of an implementation of java.util.concurrent.Callable with optional space-separated arguments -jppf.ssl.configuration.source = org.jppf.ssl.FileStoreSource classes/tests/config/ssl/ssl-node.properties +jppf.ssl.configuration.file = classes/tests/config/ssl/ssl-node.properties #------------------------------------------------------------------------------# # Enable/Disable automatic discovery of JPPF drivers. # -# default value is true; uncomment to specify a different value # #------------------------------------------------------------------------------# jppf.discovery.enabled = false @@ -61,7 +49,6 @@ jppf.discovery.enabled = false #------------------------------------------------------------------------------# # Automatic recovery: number of seconds before the first reconnection attempt. # -# default value is 1; uncomment to specify a different value # #------------------------------------------------------------------------------# #jppf.reconnect.initial.delay = 1 @@ -70,29 +57,19 @@ jppf.reconnect.max.time = 5 #------------------------------------------------------------------------------# # Processing Threads: number of threads running tasks in this node. # -# default value is 1; uncomment to specify a different value # -# blocking tasks might benefit from a number larger then CPUs # #------------------------------------------------------------------------------# jppf.processing.threads = 1 #------------------------------------------------------------------------------# -# Other JVM options added to the java command line when the node is started as # -# a subprocess. Multiple options are separated by spaces. # +# JVM options # #------------------------------------------------------------------------------# #jppf.jvm.options = -server -XX:MaxPermSize=16m -Djava.util.logging.config.file=config/logging-node1.properties jppf.jvm.options = expr: "-server -Xmx512m -Djava.util.logging.config.file=classes/tests/config/logging-node" + $n + ".properties" -#jppf.jvm.options = -server -Xmx512m -Djava.util.logging.config.file=config/logging-node1.properties -Dhazelcast.jmx=true -Dcom.sun.management.jmxremote - -# example with remote debugging options -#jppf.jvm.options = -server -Xmx512m -Djava.util.logging.config.file=config/logging-node1.properties -Xrunjdwp:transport=dt_socket,address=localhost:8000,server=y,suspend=n #------------------------------------------------------------------------------# # Idle mode enabled/disabled. In idle mode, the server ot node starts when no # -# mouse or keyboard activity has occurred since the specified tiemout, and is # -# stopped when any new activity occurs. See "jppf.idle.timeout" below. # -# Default is false (i.e. disabled). # #------------------------------------------------------------------------------# #jppf.idle.mode.enabled = true @@ -107,15 +84,8 @@ jppf.classloader.cache.size = 1 #jppf.classloader.delegation = url #jppf.thread.manager.class = org.jppf.server.node.fj.ThreadManagerForkJoin -#jppf.management.connector = rmi - -#jppf.leaks.prevention.enabled = false -#jppf.classloader.clear.thread = true - id = expr: $n -#jppf.io.buffer.size = 128 - # readable node uuid jppf.node.uuid = expr: "n" + $n diff --git a/tests/src/resources/config/offline_node/driver.properties b/tests/src/resources/config/offline_node/driver.properties index fafd2b30f3..65fbf3f506 100644 --- a/tests/src/resources/config/offline_node/driver.properties +++ b/tests/src/resources/config/offline_node/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/p2p/driver.properties b/tests/src/resources/config/p2p/driver.properties index 9489c7036d..85e29f3fa7 100644 --- a/tests/src/resources/config/p2p/driver.properties +++ b/tests/src/resources/config/p2p/driver.properties @@ -25,6 +25,7 @@ jppf.server.port = expr: 11100 + $n #jppf.ssl.server.port = expr: 12100 + $n jppf.ssl.server.port = -1 +jppf.node.max.jobs = 1 #------------------------------------------------------------------------------# # SSL Settings # diff --git a/tests/src/resources/config/p2p/driver2.properties b/tests/src/resources/config/p2p/driver2.properties index 61ec9276f4..97524130ae 100644 --- a/tests/src/resources/config/p2p/driver2.properties +++ b/tests/src/resources/config/p2p/driver2.properties @@ -129,26 +129,10 @@ jppf.load.balancing.profile.rl.maxActionRange = 10 #jppf.peer.driver2.server.host = localhost #jppf.peer.driver2.server.port = 11121 -#------------------------------------------------------------------------------# -# recovery configuration # -#------------------------------------------------------------------------------# - -jppf.recovery.enabled = false -jppf.recovery.max.retries = 3 -jppf.recovery.read.timeout = 3000 -jppf.recovery.server.port = 22222 -#jppf.recovery.reaper.pool.size = 8 -jppf.recovery.reaper.run.interval = 10000 - #------------------------------------------------------------------------------# # behavioral options # #------------------------------------------------------------------------------# -#jppf.data.transform.class = org.jppf.example.dataencryption.SecureKeyCipherTransform - -#jppf.object.stream.builder = org.jppf.serialization.XstreamObjectStreamBuilder -#jppf.object.stream.builder = org.jppf.serialization.GenericObjectStreamBuilder - #jppf.object.input.stream.class = org.jppf.serialization.JPPFObjectInputStream #jppf.object.output.stream.class = org.jppf.serialization.JPPFObjectOutputStream @@ -167,17 +151,3 @@ jppf.nio.select.timeout = 0 jppf.driver.uuid = expr: "d" + $n #jppf.peer.allow.orphans = true - -#------------------------------------------------------------------------------# -# local node configuration # -#------------------------------------------------------------------------------# - -# readable local node uuid -jppf.node.uuid = expr: "ln" + $n - -#jppf.local.node.enabled = true -#jppf.processing.threads = 4 -jppf.classloader.cache.size = 1 - -#jppf.classloader.delegation = url -#jppf.thread.manager.class = org.jppf.server.node.fj.ThreadManagerForkJoin diff --git a/tests/src/resources/config/p2p/node.properties b/tests/src/resources/config/p2p/node.properties index 28962928f3..e2d2158095 100644 --- a/tests/src/resources/config/p2p/node.properties +++ b/tests/src/resources/config/p2p/node.properties @@ -17,6 +17,7 @@ #------------------------------------------------------------------------------# jppf.debug.enabled = true +jppf.node.max.jobs = 1 #------------------------------------------------------------------------------# # Host name, or ip address, of the host the JPPF driver is running on # @@ -25,7 +26,6 @@ jppf.debug.enabled = true jppf.server.host = localhost jppf.server.port = expr: 11100 + $n -#jppf.server.port = 12101 #------------------------------------------------------------------------------# # Enabling JMX features # @@ -46,7 +46,6 @@ jppf.ssl.configuration.file = ssl/ssl-node.properties #------------------------------------------------------------------------------# # Enable/Disable automatic discovery of JPPF drivers. # -# default value is true; uncomment to specify a different value # #------------------------------------------------------------------------------# jppf.discovery.enabled = false @@ -56,7 +55,6 @@ jppf.discovery.enabled = false #------------------------------------------------------------------------------# # Automatic recovery: number of seconds before the first reconnection attempt. # -# default value is 1; uncomment to specify a different value # #------------------------------------------------------------------------------# #jppf.reconnect.initial.delay = 1 @@ -65,40 +63,16 @@ jppf.reconnect.max.time = 5 #------------------------------------------------------------------------------# # Processing Threads: number of threads running tasks in this node. # -# default value is 1; uncomment to specify a different value # -# blocking tasks might benefit from a number larger then CPUs # #------------------------------------------------------------------------------# jppf.processing.threads = 1 -#jppf.data.transform.class = org.jppf.example.dataencryption.SecureKeyCipherTransform - -#jppf.object.stream.builder = org.jppf.serialization.XstreamObjectStreamBuilder -#jppf.object.stream.builder = org.jppf.serialization.GenericObjectStreamBuilder - -#jppf.object.input.stream.class = org.jppf.serialization.JPPFObjectInputStream -#jppf.object.output.stream.class = org.jppf.serialization.JPPFObjectOutputStream - #------------------------------------------------------------------------------# -# Other JVM options added to the java command line when the node is started as # -# a subprocess. Multiple options are separated by spaces. # +# JVM options # #------------------------------------------------------------------------------# -#jppf.jvm.options = -server -XX:MaxPermSize=16m -Djava.util.logging.config.file=config/logging-node1.properties jppf.jvm.options = expr: "-server -Xmx512m -Djava.util.logging.config.file=classes/tests/config/logging-node" + $n +".properties" -#------------------------------------------------------------------------------# -# Idle mode enabled/disabled. In idle mode, the server ot node starts when no # -# mouse or keyboard activity has occurred since the specified tiemout, and is # -# stopped when any new activity occurs. See "jppf.idle.timeout" below. # -# Default is false (i.e. disabled). # -#------------------------------------------------------------------------------# - -#jppf.idle.mode.enabled = true -jppf.idle.detector.factory = org.jppf.example.idlesystem.IdleTimeDetectorFactoryImpl -jppf.idle.timeout = 6000 -jppf.idle.poll.interval = 1000 - jppf.recovery.enabled = false jppf.classloader.cache.size = 1 @@ -106,11 +80,6 @@ jppf.classloader.cache.size = 1 #jppf.classloader.delegation = url #jppf.thread.manager.class = org.jppf.server.node.fj.ThreadManagerForkJoin -#jppf.management.connector = rmi - -#jppf.leaks.prevention.enabled = false -#jppf.classloader.clear.thread = true - id = expr: $n #jppf.io.buffer.size = 128 diff --git a/tests/src/resources/config/p2p_many/client.properties b/tests/src/resources/config/p2p_many/client.properties new file mode 100644 index 0000000000..7be23ced9d --- /dev/null +++ b/tests/src/resources/config/p2p_many/client.properties @@ -0,0 +1,96 @@ +#------------------------------------------------------------------------------# +# JPPF. # +# Copyright (C) 2005-2018 JPPF Team. # +# http://www.jppf.org # +# # +# Licensed 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. # +#------------------------------------------------------------------------------# + +#------------------------------------------------------------------------------# +# SSL Settings # +#------------------------------------------------------------------------------# + +# enable SSL. Default is false (disabled). +#jppf.ssl.enabled = true +# location of the SSL configuration on the file system +jppf.ssl.configuration.file = ssl/ssl-client.properties + +#------------------------------------------------------------------------------# +# List of drivers this client may connect to # +#------------------------------------------------------------------------------# + +jppf.drivers = driver1 driver2 +driver1.jppf.server.host = localhost +driver1.jppf.server.port = 11101 +driver1.jppf.max.jobs = 1 +#driver1.jppf.management.port = 11201 +driver1.jppf.pool.size = 1 +driver2.jppf.server.host = localhost +driver2.jppf.server.port = 11102 +driver2.jppf.max.jobs = 1 +#driver2.jppf.management.port = 11202 +driver2.jppf.pool.size = 1 + +#------------------------------------------------------------------------------# +# Automatic recovery # +#------------------------------------------------------------------------------# + +#jppf.reconnect.initial.delay = 1 +jppf.reconnect.max.time = -1 +#jppf.reconnect.interval = 1 + +#------------------------------------------------------------------------------# +# Local execution # +#------------------------------------------------------------------------------# + +#jppf.local.execution.enabled = true +jppf.local.execution.threads = 4 + +#------------------------------------------------------------------------------# +# Automatic discovery of JPPF drivers. # +#------------------------------------------------------------------------------# + +jppf.discovery.enabled = false +#jppf.discovery.group = 230.0.0.1 +#jppf.discovery.port = 11111 +#jppf.pool.size = 5 + +#------------------------------------------------------------------------------# +# Load balancing # +#------------------------------------------------------------------------------# + +#jppf.load.balancing.algorithm = proportional +jppf.load.balancing.algorithm = manual + +#jppf.load.balancing.profile = proportional +jppf.load.balancing.profile = manual + +# "manual" profile +jppf.load.balancing.profile.manual.size = 5000 + +# "proportional" profile +jppf.load.balancing.profile.proportional.performanceCacheSize = 1000 +jppf.load.balancing.profile.proportional.proportionalityFactor = 1 +jppf.load.balancing.profile.proportional.initialSize = 10 +jppf.load.balancing.profile.proportional.initialMeanTime = 3e10 + +#------------------------------------------------------------------------------# +# Misc # +#------------------------------------------------------------------------------# + +#jppf.data.transform.class = org.jppf.data.transform.DESCipherTransform + +#jppf.object.stream.builder = org.jppf.serialization.XstreamObjectStreamBuilder + +# uuid of local client executor +jppf.node.uuid = local_client diff --git a/tests/src/resources/config/p2p_many/driver.properties b/tests/src/resources/config/p2p_many/driver.properties new file mode 100644 index 0000000000..f302943572 --- /dev/null +++ b/tests/src/resources/config/p2p_many/driver.properties @@ -0,0 +1,154 @@ +#------------------------------------------------------------------------------# +# JPPF. # +# Copyright (C) 2005-2018 JPPF Team. # +# http://www.jppf.org # +# # +# Licensed 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. # +#------------------------------------------------------------------------------# + +#------------------------------------------------------------------------------# +# port number to which the server listens for plain connections # +# default value is 11111; uncomment to specify a different value # +# to disable, specify a negative port number # +#------------------------------------------------------------------------------# + +jppf.server.port = expr: 11100 + $n +#jppf.ssl.server.port = expr: 12100 + $n +jppf.ssl.server.port = -1 +jppf.node.max.jobs = 1 + +#------------------------------------------------------------------------------# +# SSL Settings # +#------------------------------------------------------------------------------# + +# location of the SSL configuration on the file system +jppf.ssl.configuration.file = ssl/ssl-server.properties + +# enable secure communications with other servers; defaults to false (disabled) +#jppf.peer.ssl.enabled = true + +#------------------------------------------------------------------------------# +# enabling JMX features # +#------------------------------------------------------------------------------# + +# non-secure JMX connections; default is true (enabled) +#jppf.management.enabled = true + +# secure JMX connections via SSL/TLS; default is false (disabled) +#jppf.management.ssl.enabled = true + +#------------------------------------------------------------------------------# +# JMX management # +#------------------------------------------------------------------------------# + +#jppf.management.host = localhost +jppf.management.port = expr: 11200 + $n +#jppf.management.ssl.port = expr: 12200 + $n + +#------------------------------------------------------------------------------# +# JVM Options # +#------------------------------------------------------------------------------# + +jppf.jvm.options = expr: "-Xmx128m -server -XX:+HeapDumpOnOutOfMemoryError -Djava.util.logging.config.file=classes/tests/config/logging-driver-" + $n + ".properties" + +# with debug options +#jppf.jvm.options = expr: "-Xmx128m -server -Djava.util.logging.config.file=classes/tests/config/logging-driver-" + $n + \ +# ".properties -Xrunjdwp:transport=dt_socket,address=localhost:800" + $n + ",server=y,suspend=n" + +#------------------------------------------------------------------------------# +# server discovery # +#------------------------------------------------------------------------------# + +jppf.discovery.enabled = false +jppf.discovery.broadcast.interval = 500 +jppf.peer.discovery.enabled = false +jppf.peer.allow.orphans = true + +#jppf.peers = driver1 +jppf.peers = expr: \ + String computePeers() { \ + def s = ""; \ + for (i=1; i<=$nbDrivers; i++) { \ + if (i != $n) { \ + if (i > 1) s = s + " "; \ + s = s + "driver" + i; \ + }; \ + }; \ + return s \ + }; \ + computePeers() +#jppf.peer.driver1.server.host = localhost +jppf.peer.driver$i.server.host = $for{1; $nbDrivers; $i != $n} localhost +#jppf.peer.driver1.server.port = expr: 11100 + (($n % $nbDrivers == 1) ? 2 : 1) +jppf.peer.driver$i.server.port = $for{1; $nbDrivers; $i != $n} expr: 11100 + $i + +#------------------------------------------------------------------------------# +# the name of the load-balancing algorithm to use # +#------------------------------------------------------------------------------# + +jppf.load.balancing.algorithm = proportional +#jppf.load.balancing.algorithm = manual +#jppf.load.balancing.algorithm = nodethreads +#jppf.load.balancing.algorithm = customLoadBalancer + +jppf.load.balancing.profile = proportional + +# "manual" profile +jppf.load.balancing.profile.manual.size = 5 + +# "autotuned" profile +jppf.load.balancing.profile.autotuned.size = 5 +jppf.load.balancing.profile.autotuned.minSamplesToAnalyse = 100 +jppf.load.balancing.profile.autotuned.minSamplesToCheckConvergence = 50 +jppf.load.balancing.profile.autotuned.maxDeviation = 0.2 +jppf.load.balancing.profile.autotuned.maxGuessToStable = 50 +jppf.load.balancing.profile.autotuned.sizeRatioDeviation = 1.5 +jppf.load.balancing.profile.autotuned.decreaseRatio = 0.2 + +# "proportional" profile +jppf.load.balancing.profile.proportional.performanceCacheSize = 2000 +jppf.load.balancing.profile.proportional.proportionalityFactor = 1 +jppf.load.balancing.profile.proportional.initialSize = 5 +jppf.load.balancing.profile.proportional.initialMeanTime = 3e10 + +# "rl" profile +jppf.load.balancing.profile.rl.performanceCacheSize = 1000 +jppf.load.balancing.profile.rl.performanceVariationThreshold = 0.001 +jppf.load.balancing.profile.rl.maxActionRange = 10 + +#jppf.peers = driver2 +#jppf.peer.driver2.server.host = localhost +#jppf.peer.driver2.server.port = 11121 + +#------------------------------------------------------------------------------# +# behavioral options # +#------------------------------------------------------------------------------# + +#jppf.object.input.stream.class = org.jppf.serialization.JPPFObjectInputStream +#jppf.object.output.stream.class = org.jppf.serialization.JPPFObjectOutputStream + +#jppf.io.buffer.size = 128 + +#jppf.management.connector = rmi + +jppf.debug.enabled = true + +#jppf.nio.check.connection = false + +jppf.nio.select.timeout = 0 +#jppf.transition.thread.pool.size = 1 + +# readable driver uuid +jppf.driver.uuid = expr: "d" + $n + +#jppf.peer.allow.orphans = true diff --git a/tests/src/resources/config/p2p_many/log4j-driver.properties b/tests/src/resources/config/p2p_many/log4j-driver.properties new file mode 100644 index 0000000000..580f466850 --- /dev/null +++ b/tests/src/resources/config/p2p_many/log4j-driver.properties @@ -0,0 +1,119 @@ +#------------------------------------------------------------------------------# +# JPPF # +# Copyright (C) 2005-2018 JPPF Team. # +# http://www.jppf.org # +# # +# Licensed 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. # +#------------------------------------------------------------------------------# + +### direct log messages to stdout ### +#log4j.appender.stdout=org.apache.log4j.ConsoleAppender +#log4j.appender.stdout.Target=System.out +#log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +#log4j.appender.JPPF.layout.ConversionPattern=%d [%-5p][%c.%M(%L)]: %m\n + +### direct messages to jppf-driver.log ### +log4j.appender.JPPF=org.apache.log4j.FileAppender +log4j.appender.JPPF.File=expr: "driver-" + $n + ".log" +log4j.appender.JPPF.Append=true +log4j.appender.JPPF.BufferedIO=false +log4j.appender.JPPF.layout=org.apache.log4j.PatternLayout +log4j.appender.JPPF.layout.ConversionPattern=%d [%-5p][%-20.20t][%c.%M(%L)] %m%n + +### set log levels - for more verbose logging change 'info' to 'debug' ### + +log4j.rootLogger=INFO, JPPF +#log4j.rootLogger=DEBUG, JPPF + +#log4j.logger.org.jppf.example.loadbalancer.server=DEBUG +log4j.logger.org.jppf.classloader=INFO +log4j.logger.org.jppf.comm.discovery=INFO +#log4j.logger.org.jppf.io=DEBUG +#log4j.logger.org.jppf.io.FileDataLocation=TRACE +#log4j.logger.org.jppf.io.IOHelper=TRACE +#log4j.logger.org.jppf.jmxremote=DEBUG +#log4j.logger.org.jppf.server.job=DEBUG +log4j.logger.org.jppf.server.job.JPPFJobManager=DEBUG +#log4j.logger.org.jppf.management=DEBUG +log4j.logger.org.jppf.management.JMXConnectionWrapper=INFO +log4j.logger.org.jppf.management.JMXConnectionThread=INFO +log4j.logger.org.jppf.management.JPPFDriverAdmin=DEBUG +log4j.logger.org.jppf.management.JPPFNodeConnectionNotifier=DEBUG +log4j.logger.org.jppf.management.AbstractJMXServer=DEBUG +log4j.logger.org.jppf.management.JMXMPServer=DEBUG +log4j.logger.org.jppf.management.JPPFJMXServer=DEBUG +#log4j.logger.org.jppf.management.forwarding=DEBUG +log4j.logger.org.jppf.management.forwarding.JPPFNodeForwarding=DEBUG +#log4j.logger.org.jppf.management.forwarding.InvokeMethodTask=DEBUG +#log4j.logger.org.jppf.management.forwarding.ForwardingNotificationDispatcher=DEBUG +#log4j.logger.org.jppf.management.forwarding.ForwardingNotificationManager=DEBUG +#log4j.logger.org.jppf.nio.SimpleNioMessage=DEBUG +log4j.logger.org.jppf.nio.acceptor.AcceptorNioServer=DEBUG +#log4j.logger.org.jppf.node.policy.PolicyRuleTest=DEBUG +#log4j.logger.org.jppf.scheduling=DEBUG +log4j.logger.org.jppf.serialization.JPPFSerialization$Factory=DEBUG +#log4j.logger.org.jppf.server=DEBUG +log4j.logger.org.jppf.server.JPPFDriver=DEBUG +log4j.logger.org.jppf.server.DriverInitializer=DEBUG +#log4j.logger.org.jppf.server.NodeInformationHandler=DEBUG +#log4j.logger.org.jppf.server.event=DEBUG +log4j.logger.org.jppf.server.event.NodeConnectionEventHandler=DEBUG +#log4j.logger.org.jppf.server.job=DEBUG +#log4j.logger.org.jppf.server.job.JPPFJobManager=DEBUG +#log4j.logger.org.jppf.server.job.management=DEBUG +log4j.logger.org.jppf.server.job.management.DriverJobManagement=DEBUG +#log4j.logger.org.jppf.server.management=DEBUG +log4j.logger.org.jppf.nio=INFO +log4j.logger.org.jppf.nio.SimpleNioMessage=DEBUG +#log4j.logger.org.jppf.nio.SSLHandlerImpl=TRACE +#log4j.logger.org.jppf.nio.SSLNioObject=TRACE +#log4j.logger.org.jppf.nio.NioServer=DEBUG +log4j.logger.org.jppf.nio.NioMessageReader=DEBUG +log4j.logger.org.jppf.nio.NioMessageWriter=DEBUG +log4j.logger.org.jppf.nio.StateTransitionTask=DEBUG +#log4j.logger.org.jppf.nio.StateTransitionManager=DEBUG +#log4j.logger.org.jppf.server.nio.acceptor=DEBUG +#log4j.logger.org.jppf.server.nio.classloader=DEBUG +#log4j.logger.org.jppf.server.nio.classloader.client=DEBUG +#log4j.logger.org.jppf.server.nio.classloader.client.ClientClassNioServer=DEBUG +log4j.logger.org.jppf.server.nio.client=DEBUG +#log4j.logger.org.jppf.server.nio.client.ClientNioServer=DEBUG +log4j.logger.org.jppf.server.nio.nodeserver=DEBUG +#log4j.logger.org.jppf.server.nio.nodeserver.NodeNioServer=DEBUG +#log4j.logger.org.jppf.server.nio.nodeserver.TaskQueueChecker=INFO +#log4j.logger.org.jppf.server.nio.nodeserver.AbstractNodeContext=DEBUG +#log4j.logger.org.jppf.server.nio.nodeserver.NodeDispatchTimeoutAction=DEBUG +#log4j.logger.org.jppf.server.nio.nodeserver.SendingBundleState=DEBUG +#log4j.logger.org.jppf.server.nio.nodeserver.SendInitialBundleState=DEBUG +#log4j.logger.org.jppf.server.nio.nodeserver.WaitInitialBundleState=DEBUG +#log4j.logger.org.jppf.server.nio.nodeserver.WaitingtResultsState=DEBUG +log4j.logger.org.jppf.server.peer=DEBUG +#log4j.logger.org.jppf.server.protocol=DEBUG +#log4j.logger.org.jppf.server.protocol.ServerJob=DEBUG +#log4j.logger.org.jppf.server.queue=DEBUG +#log4j.logger.org.jppf.server.queue.JPPFPriorityQueue=DEBUG +log4j.logger.org.jppf.server.scheduler.bundle=INFO +#log4j.logger.org.jppf.server.scheduler.bundle.impl=DEBUG +#log4j.logger.org.jppf.server.scheduler.bundle.proportional.ProportionalTuneProfile=DEBUG +#log4j.logger.org.jppf.server.scheduler.bundle.simulation=DEBUG +#log4j.logger.org.jppf.server.scheduler.bundle.simulation.RLBundler=DEBUG +#log4j.logger.org.jppf.server.scheduler.bundle.simple=DEBUG +#log4j.logger.org.jppf.server.scheduler.bundle.spi=DEBUG +#log4j.logger.org.jppf.ssl=DEBUG +log4j.logger.org.jppf.utils=INFO +#log4j.logger.org.jppf.utils.BufferPool=INFO +log4j.logger.org.jppf.utils.DebugInterruptThreadFactory=DEBUG +log4j.logger.org.jppf.utils.JPPFDefaultUncaughtExceptionHandler=DEBUG +#log4j.logger.org.jppf.utils.NetworkUtils=INFO +log4j.logger.org.jppf.utils.RetryUtils=DEBUG +log4j.logger.org.jppf.serialization=DEBUG diff --git a/tests/src/resources/config/p2p_many/log4j-node.properties b/tests/src/resources/config/p2p_many/log4j-node.properties new file mode 100644 index 0000000000..e3bff82af4 --- /dev/null +++ b/tests/src/resources/config/p2p_many/log4j-node.properties @@ -0,0 +1,75 @@ +#------------------------------------------------------------------------------# +# JPPF. # +# Copyright (C) 2005-2018 JPPF Team. # +# http://www.jppf.org # +# # +# Licensed 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. # +#------------------------------------------------------------------------------# + +### direct log messages to stdout ### +#log4j.appender.stdout=org.apache.log4j.ConsoleAppender +#log4j.appender.stdout.Target=System.out +#log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +#log4j.appender.stdout.layout.ConversionPattern=%d{ABSOLUTE} %5p %c{1}:%L - %m%n + +### direct messages to file jppf-node1.log ### +log4j.appender.JPPF=org.apache.log4j.FileAppender +log4j.appender.JPPF.File=expr: "node-" + $n + ".log" +log4j.appender.JPPF.Append=true +log4j.appender.JPPF.layout=org.apache.log4j.PatternLayout +log4j.appender.JPPF.layout.ConversionPattern=%d [%-5p][%-20.20t][%c.%M(%L)]: %m%n + +### set log levels - for more verbose logging change 'info' to 'debug' ### +log4j.rootLogger=INFO, JPPF +#log4j.rootLogger=DEBUG, JPPF + +#log4j.logger.org.jppf.classloader=DEBUG +log4j.logger.org.jppf.classloader.AbstractClassLoaderConnection=DEBUG +log4j.logger.org.jppf.classloader.RemoteClassLoaderConnection=DEBUG +log4j.logger.org.jppf.comm.discovery=INFO +#log4j.logger.org.jppf.comm.execute=INFO +log4j.logger.org.jppf.execute.AbstractExecutionManager=DEBUG +#log4j.logger.org.jppf.jmxremote=DEBUG +#log4j.logger.org.jppf.management=DEBUG +log4j.logger.org.jppf.management.AbstractJMXServer=DEBUG +log4j.logger.org.jppf.management.JMXMPServer=DEBUG +log4j.logger.org.jppf.management.JPPFJMXServer=DEBUG +log4j.logger.org.jppf.management.JPPFNodeAdmin=DEBUG +#log4j.logger.org.jppf.management.JPPFNodeTaskMonitor=DEBUG +#log4j.logger.org.jppf.management.spi=INFO +#log4j.logger.org.jppf.management.spi.JPPFMBeanProviderManager=DEBUG +#log4j.logger.org.jppf.nio.SimpleNioMessage=DEBUG +#log4j.logger.org.jppf.nio.SSLHandlerImpl=TRACE +#log4j.logger.org.jppf.nio.SSLNioObject=TRACE +#log4j.logger.org.jppf.nio.acceptor.AcceptorNioServer=DEBUG +log4j.logger.org.jppf.nio.NioServer=DEBUG +log4j.logger.org.jppf.nio.acceptor=DEBUG +log4j.logger.org.jppf.node=DEBUG +#log4j.logger.org.jppf.node.NodeRunner=DEBUG +log4j.logger.org.jppf.persistence=DEBUG +#log4j.logger.org.jppf.process.LauncherListener=DEBUG +#log4j.logger.org.jppf.serialization.JPPFSerialization$Factory=DEBUG +log4j.logger.org.jppf.server.node=DEBUG +log4j.logger.org.jppf.server.node.remote=TRACE +#log4j.logger.org.jppf.server.node.JPPFContainer=DEBUG +#log4j.logger.org.jppf.server.node.JPPFNode=DEBUG +#log4j.logger.org.jppf.server.node.NodeExecutionManager=DEBUG +#log4j.logger.org.jppf.server.node.NodeIO=DEBUG +#log4j.logger.org.jppf.server.node.NodeTaskWrapper=TRACE +log4j.logger.org.jppf.ssl=DEBUG +log4j.logger.org.jppf.utils.NetworkUtils=INFO +#log4j.logger.org.jppf.security=DEBUG +#log4j.logger.sample.taskcommunication=DEBUG +log4j.logger.org.jppf.utils.JPPFDefaultUncaughtExceptionHandler=DEBUG +log4j.logger.org.jppf.utils.RetryUtils=DEBUG +log4j.logger.test.org.jppf.test.setup.common.NotifyingTask=DEBUG diff --git a/tests/src/resources/config/p2p_many/node.properties b/tests/src/resources/config/p2p_many/node.properties new file mode 100644 index 0000000000..d8d422b282 --- /dev/null +++ b/tests/src/resources/config/p2p_many/node.properties @@ -0,0 +1,72 @@ +#------------------------------------------------------------------------------# +# JPPF. # +# Copyright (C) 2005-2018 JPPF Team. # +# http://www.jppf.org # +# # +# Licensed 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. # +#------------------------------------------------------------------------------# + +jppf.debug.enabled = true + +#------------------------------------------------------------------------------# + +jppf.server.host = localhost +jppf.server.port = expr: 11100 + $n +#jppf.management.enabled = false +jppf.management.port = expr: 12300 + $n + +#------------------------------------------------------------------------------# + +#jppf.ssl.enabled = true +jppf.ssl.configuration.file = ssl/ssl-node.properties + +#------------------------------------------------------------------------------# + +jppf.discovery.enabled = false +#jppf.discovery.group = 230.0.0.1 +#jppf.discovery.port = 11111 +#jppf.discovery.timeout = 5000 + +#------------------------------------------------------------------------------# + +#jppf.reconnect.initial.delay = 1 +jppf.reconnect.max.time = 5 +#jppf.reconnect.interval = 1 + +#------------------------------------------------------------------------------# + +jppf.processing.threads = 1 + +#jppf.object.input.stream.class = org.jppf.serialization.JPPFObjectInputStream +#jppf.object.output.stream.class = org.jppf.serialization.JPPFObjectOutputStream + +#------------------------------------------------------------------------------# + +jppf.jvm.options = expr: "-server -Xmx128m -XX:+HeapDumpOnOutOfMemoryError -Djava.util.logging.config.file=classes/tests/config/logging-node" + $n +".properties" + +#------------------------------------------------------------------------------# + +#jppf.idle.mode.enabled = true +jppf.idle.detector.factory = org.jppf.example.idlesystem.IdleTimeDetectorFactoryImpl +jppf.idle.timeout = 6000 +jppf.idle.poll.interval = 1000 + +jppf.recovery.enabled = false + +jppf.classloader.cache.size = 1 + +#jppf.classloader.delegation = url +#jppf.thread.manager.class = org.jppf.server.node.fj.ThreadManagerForkJoin + +id = expr: $n +jppf.node.uuid = expr: "n" + $n diff --git a/tests/src/resources/config/p2p_orphan/driver.properties b/tests/src/resources/config/p2p_orphan/driver.properties index b2b91b2bd5..58d5be293f 100644 --- a/tests/src/resources/config/p2p_orphan/driver.properties +++ b/tests/src/resources/config/p2p_orphan/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/persistence/driver.properties b/tests/src/resources/config/persistence/driver.properties index fe0c93cd50..725a60c616 100644 --- a/tests/src/resources/config/persistence/driver.properties +++ b/tests/src/resources/config/persistence/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/persistence/log4j-driver.template.properties b/tests/src/resources/config/persistence/log4j-driver.properties similarity index 100% rename from tests/src/resources/config/persistence/log4j-driver.template.properties rename to tests/src/resources/config/persistence/log4j-driver.properties diff --git a/tests/src/resources/config/provisioning/driver.properties b/tests/src/resources/config/provisioning/driver.properties index cde4cde595..417ac8fe8c 100644 --- a/tests/src/resources/config/provisioning/driver.properties +++ b/tests/src/resources/config/provisioning/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/java/driver.properties b/tests/src/resources/config/serialization/java/driver.properties index df234bc373..2568cddbd2 100644 --- a/tests/src/resources/config/serialization/java/driver.properties +++ b/tests/src/resources/config/serialization/java/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/java_lz4/driver.properties b/tests/src/resources/config/serialization/java_lz4/driver.properties index cef51b7010..2b84edbe7a 100644 --- a/tests/src/resources/config/serialization/java_lz4/driver.properties +++ b/tests/src/resources/config/serialization/java_lz4/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/java_zlib/driver.properties b/tests/src/resources/config/serialization/java_zlib/driver.properties index 3e85ff213f..f7e2c52c43 100644 --- a/tests/src/resources/config/serialization/java_zlib/driver.properties +++ b/tests/src/resources/config/serialization/java_zlib/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/jppf/driver.properties b/tests/src/resources/config/serialization/jppf/driver.properties index c736581a12..3f463d051c 100644 --- a/tests/src/resources/config/serialization/jppf/driver.properties +++ b/tests/src/resources/config/serialization/jppf/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/jppf_lz4/driver.properties b/tests/src/resources/config/serialization/jppf_lz4/driver.properties index 2874f88ae7..805d982a48 100644 --- a/tests/src/resources/config/serialization/jppf_lz4/driver.properties +++ b/tests/src/resources/config/serialization/jppf_lz4/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/jppf_zlib/driver.properties b/tests/src/resources/config/serialization/jppf_zlib/driver.properties index 542ee6f00b..737448a12d 100644 --- a/tests/src/resources/config/serialization/jppf_zlib/driver.properties +++ b/tests/src/resources/config/serialization/jppf_zlib/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/kryo/driver.properties b/tests/src/resources/config/serialization/kryo/driver.properties index ac746b6298..dcac739dbc 100644 --- a/tests/src/resources/config/serialization/kryo/driver.properties +++ b/tests/src/resources/config/serialization/kryo/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/kryo_lz4/driver.properties b/tests/src/resources/config/serialization/kryo_lz4/driver.properties index 2171eac148..ac836dcb94 100644 --- a/tests/src/resources/config/serialization/kryo_lz4/driver.properties +++ b/tests/src/resources/config/serialization/kryo_lz4/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/kryo_zlib/driver.properties b/tests/src/resources/config/serialization/kryo_zlib/driver.properties index 457b2946d2..f4bb759bf9 100644 --- a/tests/src/resources/config/serialization/kryo_zlib/driver.properties +++ b/tests/src/resources/config/serialization/kryo_zlib/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/xstream/driver.properties b/tests/src/resources/config/serialization/xstream/driver.properties index 2a0d970bfb..d3c9a4acc1 100644 --- a/tests/src/resources/config/serialization/xstream/driver.properties +++ b/tests/src/resources/config/serialization/xstream/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/xstream_lz4/driver.properties b/tests/src/resources/config/serialization/xstream_lz4/driver.properties index 314204a4c0..9abe203f1d 100644 --- a/tests/src/resources/config/serialization/xstream_lz4/driver.properties +++ b/tests/src/resources/config/serialization/xstream_lz4/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/serialization/xstream_zlib/driver.properties b/tests/src/resources/config/serialization/xstream_zlib/driver.properties index a8646fc237..6a9d0be681 100644 --- a/tests/src/resources/config/serialization/xstream_zlib/driver.properties +++ b/tests/src/resources/config/serialization/xstream_zlib/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/ssl/driver.properties b/tests/src/resources/config/ssl/driver.properties index 2c46bcf09d..6974c2ebe9 100644 --- a/tests/src/resources/config/ssl/driver.properties +++ b/tests/src/resources/config/ssl/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/ssl2/driver.properties b/tests/src/resources/config/ssl2/driver.properties index 2102c183eb..8b9b77a315 100644 --- a/tests/src/resources/config/ssl2/driver.properties +++ b/tests/src/resources/config/ssl2/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/ssl2_p2p/driver.properties b/tests/src/resources/config/ssl2_p2p/driver.properties index bc7e7e092a..a0981dc369 100644 --- a/tests/src/resources/config/ssl2_p2p/driver.properties +++ b/tests/src/resources/config/ssl2_p2p/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/resources/config/ssl3/driver.properties b/tests/src/resources/config/ssl3/driver.properties index 4322050af7..49c811088c 100644 --- a/tests/src/resources/config/ssl3/driver.properties +++ b/tests/src/resources/config/ssl3/driver.properties @@ -16,6 +16,8 @@ # limitations under the License. # #------------------------------------------------------------------------------# +jppf.node.max.jobs = 1 + #------------------------------------------------------------------------------# # port number to which the server listens for plain connections # # default value is 11111; uncomment to specify a different value # diff --git a/tests/src/tests/test/org/jppf/client/TestDefaultPolicies.java b/tests/src/tests/test/org/jppf/client/TestDefaultPolicies.java index 16a099d3f3..a8d4f01617 100644 --- a/tests/src/tests/test/org/jppf/client/TestDefaultPolicies.java +++ b/tests/src/tests/test/org/jppf/client/TestDefaultPolicies.java @@ -207,7 +207,11 @@ private static void checkPolicyResults(final JPPFClient client, final String nod assertTrue(ta instanceof LifeCycleTask); final LifeCycleTask task = (LifeCycleTask) ta; final Throwable t = task.getThrowable(); - assertNull("task " + i + " has an exception " + t, t); + final String message = "task " + i + " has an exception; "; + if (t != null) { + print(false, false, "%s\n%s", message, ExceptionUtils.getStackTrace(t)); + fail(message + t); + } assertEquals("result of task " + i + " should be " + msg + " but is " + task.getResult(), msg, task.getResult()); assertEquals(job.getJobTasks().get(i).getId(), task.getId()); assertEquals(i, task.getPosition()); diff --git a/tests/src/tests/test/org/jppf/job/persistence/AbstractJobPersistenceTest.java b/tests/src/tests/test/org/jppf/job/persistence/AbstractJobPersistenceTest.java index 2e8b3ef71b..07c9e68afd 100644 --- a/tests/src/tests/test/org/jppf/job/persistence/AbstractJobPersistenceTest.java +++ b/tests/src/tests/test/org/jppf/job/persistence/AbstractJobPersistenceTest.java @@ -191,7 +191,7 @@ public void testSimplePersistedJobCancellation() throws Exception { * The client should recover gracefully and provide the job results without intervention. * @throws Exception if any error occurs. */ - @Test(timeout = 15000) + @Test(timeout = 10000) public void testJobAutoRecoveryOnDriverRestart() throws Exception { final int nbTasks = 20; final String method = ReflectionUtils.getCurrentMethodName(); diff --git a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistence.java b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistence.java index 0aa9a158e2..a8cb71e7f6 100644 --- a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistence.java +++ b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistence.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "job_persistence"; final TestConfiguration config = dbSetup(prefix); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_db.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceAsync.java b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceAsync.java index 05b98d8cb3..fe2c1776e8 100644 --- a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceAsync.java +++ b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceAsync.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "job_persistence"; final TestConfiguration config = dbSetup(prefix); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_async_db.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceCacheable.java b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceCacheable.java index 86a2e1ab22..8fcdd2aa67 100644 --- a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceCacheable.java +++ b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceCacheable.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "job_persistence"; final TestConfiguration config = dbSetup(prefix); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_cacheable_db.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceCacheableAsync.java b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceCacheableAsync.java index 012f5855df..19bab1fea9 100644 --- a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceCacheableAsync.java +++ b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceCacheableAsync.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "job_persistence"; final TestConfiguration config = dbSetup(prefix); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_cacheable_async_db.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceMultiServer.java b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceMultiServer.java index 74aa855d5d..f020c0bf76 100644 --- a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceMultiServer.java +++ b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultDatabasePersistenceMultiServer.java @@ -71,7 +71,6 @@ public static void setup() throws Exception { ps.executeUpdate(); } } - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.properties"; client = BaseSetup.setup(2, 2, true, true, config); } diff --git a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistence.java b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistence.java index 48bf4b9724..0e19f1fd5d 100644 --- a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistence.java +++ b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistence.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "job_persistence"; final TestConfiguration config = dbSetup(prefix, false); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_file.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceAsync.java b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceAsync.java index 1a7370d5c7..48584910a7 100644 --- a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceAsync.java +++ b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceAsync.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "job_persistence"; final TestConfiguration config = dbSetup(prefix, false); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_async_file.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceCacheable.java b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceCacheable.java index 42d1a9f0db..7f320b4f1a 100644 --- a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceCacheable.java +++ b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceCacheable.java @@ -34,9 +34,8 @@ public class TestDefaultFilePersistenceCacheable extends AbstractJobPersistenceT @BeforeClass public static void setup() throws Exception { final String prefix = "job_persistence"; - final TestConfiguration config = createConfig(prefix); + final TestConfiguration config = dbSetup(prefix, false); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_cacheable_file.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceCacheableAsync.java b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceCacheableAsync.java index b1f5660fdf..e3e1a4da61 100644 --- a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceCacheableAsync.java +++ b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceCacheableAsync.java @@ -34,9 +34,8 @@ public class TestDefaultFilePersistenceCacheableAsync extends AbstractJobPersist @BeforeClass public static void setup() throws Exception { final String prefix = "job_persistence"; - final TestConfiguration config = createConfig(prefix); + final TestConfiguration config = dbSetup(prefix, false); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_cacheable_async_file.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceP2P.java b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceP2P.java index 8059dacc35..d98efcef68 100644 --- a/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceP2P.java +++ b/tests/src/tests/test/org/jppf/job/persistence/TestDefaultFilePersistenceP2P.java @@ -47,8 +47,7 @@ public class TestDefaultFilePersistenceP2P extends AbstractDatabaseSetup { @BeforeClass public static void setup() throws Exception { final String prefix = "job_persistence_p2p"; - final TestConfiguration config = createConfig(prefix); - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.properties"; + final TestConfiguration config = dbSetup(prefix, false); client = BaseSetup.setup(2, 2, true, true, config); } diff --git a/tests/src/tests/test/org/jppf/load/balancer/persistence/AbstractMultiServerLoadBalancerPersistenceTest.java b/tests/src/tests/test/org/jppf/load/balancer/persistence/AbstractMultiServerLoadBalancerPersistenceTest.java index 6228c41754..e4520a0b57 100644 --- a/tests/src/tests/test/org/jppf/load/balancer/persistence/AbstractMultiServerLoadBalancerPersistenceTest.java +++ b/tests/src/tests/test/org/jppf/load/balancer/persistence/AbstractMultiServerLoadBalancerPersistenceTest.java @@ -64,7 +64,6 @@ static void setupConfig(final String driverConfigFile, final boolean useDB) thro final String prefix = "lb_persistence_p2p"; final TestConfiguration config = dbSetup(prefix, useDB); config.driver.jppf = "classes/tests/config/" + prefix + "/" + driverConfigFile; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.properties"; client = BaseSetup.setup(2, 2, true, false, config); checkPeers(15_000L, false, true); } @@ -178,25 +177,30 @@ public void testDifferentAlgosPerNode() throws Exception { final String[] algos = { "proportional", "autotuned", "rl2" }; for (int i=0; i>> algo=%-12s", algo); for (int j=0; j 0) job.getClientSLA().setExecutionPolicy(new Equal("jppf.driver.uuid", true, "d" + i)); + if (i > 0) job.getClientSLA().setExecutionPolicy(new Equal("jppf.driver.uuid", true, "d" + (2 - i % 2))); + print(false, false, ">>> submitting job %s", job.getName()); final List> results = client.submitJob(job); + print(false, false, ">>> checking job results"); checkJobResults(NB_TASKS, results, false); } + print(false, false, ">>> check 1"); awaitNoMorePendingOperations(mgt); final Map uuidToChannelID = new HashMap<>(); for (int i=0; i channels = mgt.listAllChannelsWithAlgorithm(algos[i]); assertNotNull(channels); if (i == 0) { - assertCompare(Operator.AT_LEAST, channels.size(), 3); + assertCompare(Operator.AT_LEAST, 3, channels.size()); } else { - assertCompare(Operator.MORE_THAN, channels.size(), 0); + assertCompare(Operator.MORE_THAN, 0, channels.size()); uuidToChannelID.put(i, channels.get(0)); } } + print(false, false, ">>> check 2"); // check that channel1 has algos[0] + algos[1] and channel2 has algos[0] + algos[2] for (final Map.Entry entry: uuidToChannelID.entrySet()) { final List channelAlgos = mgt.listAlgorithms(entry.getValue()); @@ -206,6 +210,7 @@ public void testDifferentAlgosPerNode() throws Exception { // delete algos[0] from all nodes and re-check that node1 has only algos[1] and node2 has only algos[2] mgt.deleteAlgorithm(algos[0]); awaitNoMorePendingOperations(mgt); + print(false, false, ">>> check 3"); for (final Map.Entry entry: uuidToChannelID.entrySet()) { final List channelAlgos = mgt.listAlgorithms(entry.getValue()); assertNotNull(channelAlgos); diff --git a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientAsyncDatabaseLoadBalancerPersistence.java b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientAsyncDatabaseLoadBalancerPersistence.java index b80a6f36a6..db6bc86c2d 100644 --- a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientAsyncDatabaseLoadBalancerPersistence.java +++ b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientAsyncDatabaseLoadBalancerPersistence.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "lb_persistence_client"; final TestConfiguration config = dbSetup(prefix, true); config.clientConfig = "classes/tests/config/" + prefix + "/client_async_db.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 1, true, true, config); } diff --git a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientAsyncFileLoadBalancerPersistence.java b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientAsyncFileLoadBalancerPersistence.java index 444bb0e1d7..ffe6451305 100644 --- a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientAsyncFileLoadBalancerPersistence.java +++ b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientAsyncFileLoadBalancerPersistence.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "lb_persistence_client"; final TestConfiguration config = dbSetup(prefix, false); config.clientConfig = "classes/tests/config/" + prefix + "/client_async_file.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 1, true, true, config); } diff --git a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientDatabaseLoadBalancerPersistence.java b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientDatabaseLoadBalancerPersistence.java index fe26c7b106..3b3f95a256 100644 --- a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientDatabaseLoadBalancerPersistence.java +++ b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientDatabaseLoadBalancerPersistence.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "lb_persistence_client"; final TestConfiguration config = dbSetup(prefix, true); config.clientConfig = "classes/tests/config/" + prefix + "/client_db.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 1, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientFileLoadBalancerPersistence.java b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientFileLoadBalancerPersistence.java index 6fd315645f..f225fe1f54 100644 --- a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientFileLoadBalancerPersistence.java +++ b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestClientFileLoadBalancerPersistence.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "lb_persistence_client"; final TestConfiguration config = dbSetup(prefix, false); config.clientConfig = "classes/tests/config/" + prefix + "/client_file.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 1, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverAsyncDatabaseLoadBalancerPersistence.java b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverAsyncDatabaseLoadBalancerPersistence.java index caf6e9b709..3e4485058c 100644 --- a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverAsyncDatabaseLoadBalancerPersistence.java +++ b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverAsyncDatabaseLoadBalancerPersistence.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "lb_persistence_driver"; final TestConfiguration config = dbSetup(prefix); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_async_db.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } diff --git a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverAsyncFileLoadBalancerPersistence.java b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverAsyncFileLoadBalancerPersistence.java index e41c91c99c..7acca130a4 100644 --- a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverAsyncFileLoadBalancerPersistence.java +++ b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverAsyncFileLoadBalancerPersistence.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "lb_persistence_driver"; final TestConfiguration config = dbSetup(prefix, false); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_async_file.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } diff --git a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverDatabaseLoadBalancerPersistence.java b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverDatabaseLoadBalancerPersistence.java index 9dd984adf2..11c6628349 100644 --- a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverDatabaseLoadBalancerPersistence.java +++ b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverDatabaseLoadBalancerPersistence.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "lb_persistence_driver"; final TestConfiguration config = dbSetup(prefix); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_db.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverFileLoadBalancerPersistence.java b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverFileLoadBalancerPersistence.java index cb3981dc21..a95de8617d 100644 --- a/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverFileLoadBalancerPersistence.java +++ b/tests/src/tests/test/org/jppf/load/balancer/persistence/TestDriverFileLoadBalancerPersistence.java @@ -36,7 +36,6 @@ public static void setup() throws Exception { final String prefix = "lb_persistence_driver"; final TestConfiguration config = dbSetup(prefix, false); config.driver.jppf = "classes/tests/config/" + prefix + "/driver_file.properties"; - config.driver.log4j = "classes/tests/config/" + prefix + "/log4j-driver.template.properties"; client = BaseSetup.setup(1, 2, true, true, config); } } diff --git a/tests/src/tests/test/org/jppf/management/forwarding/AbstractTestJPPFNodeForwardingMBean.java b/tests/src/tests/test/org/jppf/management/forwarding/AbstractTestJPPFNodeForwardingMBean.java index 8a01443fb8..afae1666ab 100644 --- a/tests/src/tests/test/org/jppf/management/forwarding/AbstractTestJPPFNodeForwardingMBean.java +++ b/tests/src/tests/test/org/jppf/management/forwarding/AbstractTestJPPFNodeForwardingMBean.java @@ -60,7 +60,7 @@ public abstract class AbstractTestJPPFNodeForwardingMBean extends BaseTest { public TestWatcher setup1D2N1CWatcher = new TestWatcher() { @Override protected void starting(final Description description) { - BaseTestHelper.printToServersAndNodes(client, true, true, "start of method %s()", description.getMethodName()); + BaseTestHelper.printToAll(client, false, false, true, true, false, "start of method %s()", description.getMethodName()); } }; diff --git a/tests/src/tests/test/org/jppf/management/forwarding/TestJPPFNodeForwardingMBean.java b/tests/src/tests/test/org/jppf/management/forwarding/TestJPPFNodeForwardingMBean.java index 6c7dc94bf4..4dec4ae63c 100644 --- a/tests/src/tests/test/org/jppf/management/forwarding/TestJPPFNodeForwardingMBean.java +++ b/tests/src/tests/test/org/jppf/management/forwarding/TestJPPFNodeForwardingMBean.java @@ -152,21 +152,24 @@ public void testUpdateThreadPriority() throws Exception { * @throws Exception if any error occurs. */ private static void testUpdateThreadPriority(final NodeSelector selector, final String... expectedNodes) throws Exception { - Map result = nodeForwarder.state(selector); - checkNodes(result, JPPFNodeState.class, expectedNodes); - for (final Map.Entry entry : result.entrySet()) { - final JPPFNodeState state = (JPPFNodeState) entry.getValue(); - assertEquals(Thread.NORM_PRIORITY, state.getThreadPriority()); - } - result = nodeForwarder.updateThreadsPriority(selector, Thread.MAX_PRIORITY); - checkNoException(result, expectedNodes); - result = nodeForwarder.state(selector); - checkNodes(result, JPPFNodeState.class, expectedNodes); - for (final Map.Entry entry : result.entrySet()) { - final JPPFNodeState state = (JPPFNodeState) entry.getValue(); - assertEquals(Thread.MAX_PRIORITY, state.getThreadPriority()); + try { + Map result = nodeForwarder.state(selector); + checkNodes(result, JPPFNodeState.class, expectedNodes); + for (final Map.Entry entry : result.entrySet()) { + final JPPFNodeState state = (JPPFNodeState) entry.getValue(); + assertEquals(Thread.NORM_PRIORITY, state.getThreadPriority()); + } + result = nodeForwarder.updateThreadsPriority(selector, Thread.MAX_PRIORITY); + checkNoException(result, expectedNodes); + result = nodeForwarder.state(selector); + checkNodes(result, JPPFNodeState.class, expectedNodes); + for (final Map.Entry entry : result.entrySet()) { + final JPPFNodeState state = (JPPFNodeState) entry.getValue(); + assertEquals(Thread.MAX_PRIORITY, state.getThreadPriority()); + } + } finally { + nodeForwarder.updateThreadsPriority(selector, Thread.NORM_PRIORITY); } - nodeForwarder.updateThreadsPriority(selector, Thread.NORM_PRIORITY); } /** diff --git a/tests/src/tests/test/org/jppf/persistence/AbstractDatabaseSetup.java b/tests/src/tests/test/org/jppf/persistence/AbstractDatabaseSetup.java index 7b3f42bf5c..a9a016f86e 100644 --- a/tests/src/tests/test/org/jppf/persistence/AbstractDatabaseSetup.java +++ b/tests/src/tests/test/org/jppf/persistence/AbstractDatabaseSetup.java @@ -107,6 +107,9 @@ protected static TestConfiguration dbSetup(final String prefix, final boolean us print(false, false, "table created"); } final TestConfiguration config = createConfig(prefix); + final String dir = "classes/tests/config" + (prefix == null ? "" : "/" + prefix); + config.driver.log4j = dir + "/log4j-driver.properties"; + config.node.log4j = dir + "/log4j-node.properties"; config.driver.classpath.add("lib/h2.jar"); return config; } diff --git a/tests/src/tests/test/org/jppf/persistence/TestJPPFDatasourceFactory.java b/tests/src/tests/test/org/jppf/persistence/TestJPPFDatasourceFactory.java index fa83e09ef3..f181fec404 100644 --- a/tests/src/tests/test/org/jppf/persistence/TestJPPFDatasourceFactory.java +++ b/tests/src/tests/test/org/jppf/persistence/TestJPPFDatasourceFactory.java @@ -51,6 +51,7 @@ public class TestJPPFDatasourceFactory extends AbstractDatabaseSetup { @BeforeClass public static void setup() throws Exception { final TestConfiguration config = dbSetup("persistence"); + config.driver.log4j = "classes/tests/config/persistence/log4j-driver.properties"; config.node.log4j = "classes/tests/config/persistence/log4j-node.properties"; client = BaseSetup.setup(1, 2, true, true, config); } diff --git a/tests/src/tests/test/org/jppf/server/peer/TestManyServers.java b/tests/src/tests/test/org/jppf/server/peer/TestManyServers.java new file mode 100644 index 0000000000..8b8dfb59cd --- /dev/null +++ b/tests/src/tests/test/org/jppf/server/peer/TestManyServers.java @@ -0,0 +1,120 @@ +/* + * JPPF. + * Copyright (C) 2005-2018 JPPF Team. + * http://www.jppf.org + * + * Licensed 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 test.org.jppf.server.peer; + +import static org.junit.Assert.*; + +import java.util.*; + +import org.jppf.client.*; +import org.jppf.node.protocol.Task; +import org.jppf.utils.*; +import org.jppf.utils.configuration.JPPFProperties; +import org.junit.*; +import org.junit.rules.TestWatcher; +import org.junit.runner.Description; + +import test.org.jppf.test.setup.*; +import test.org.jppf.test.setup.common.*; + +/** + * Test a multi-server topology with 2 servers, 1 node attached to each server and 1 client. + * @author Laurent Cohen + */ +public class TestManyServers extends AbstractNonStandardSetup { + /** */ + private static final long TIMEOUT = 20_000L; + /** */ + @Rule + public TestWatcher testMultiServerWatcher = new TestWatcher() { + @Override + protected void starting(final Description description) { + BaseTestHelper.printToAll(client, false, false, true, true, true, "start of method %s()", description.getMethodName()); + } + }; + + /** + * Launches 5 drivers with 1 node attached to each and start the client. + * @throws Exception if a process could not be started. + */ + public void setup() throws Exception { + try { + final int nbDrivers = 5; + print(false, false, ">>> creating test configuration"); + final TestConfiguration config = createConfig("p2p_many"); + config.driver.log4j = "classes/tests/config/p2p_many/log4j-driver.properties"; + config.node.log4j = "classes/tests/config/p2p_many/log4j-node.properties"; + print(false, false, ">>> starting grid"); + BaseSetup.setup(nbDrivers, nbDrivers, false, false, config); + print(false, false, ">>> updating client configuration"); + final String[] drivers = new String[nbDrivers]; + for (int i=0; i>> creating client"); + client = BaseSetup.createClient(null, false, config); + print(false, false, ">>> client config:\n%s", client.getConfig()); + print(false, false, ">>> checking drivers and nodes"); + awaitPeersInitialized(15_000L, nbDrivers); + print(false, false, ">>> checking peers"); + checkPeers(nbDrivers, 15_000L, false, true); + } catch (final Exception|Error e) { + e.printStackTrace(); + throw e; + } + } + + /** + * @throws Exception if any error occurs. + */ + //@Test(timeout = TIMEOUT) + @Override + public void testMultipleJobs() throws Exception { + setup(); + final int nbTasks = 15; + final int nbDrivers = BaseSetup.nbDrivers(); + final int nbJobs = 10; + client.awaitConnectionPools(Operator.AT_LEAST, nbDrivers, Operator.AT_LEAST, 1, TIMEOUT, JPPFClientConnectionStatus.workingStatuses()); + final String name = ReflectionUtils.getCurrentClassAndMethod(); + final List jobs = new ArrayList<>(nbJobs); + for (int i=1; i<=nbJobs; i++) { + final JPPFJob job = BaseTestHelper.createJob(name + '-' + i, false, false, nbTasks, LifeCycleTask.class, 1L); + //job.getClientSLA().setMaxChannels(BaseSetup.nbDrivers()); + jobs.add(job); + } + for (final JPPFJob job: jobs) client.submitJob(job); + for (final JPPFJob job: jobs) { + final List> results = job.awaitResults(); + assertNotNull(results); + assertEquals(nbTasks, results.size()); + for (final Task task: results) { + assertTrue("task = " + task, task instanceof LifeCycleTask); + final Throwable t = task.getThrowable(); + assertNull("throwable for task '" + task.getId() + "' : " + ExceptionUtils.getStackTrace(t), t); + assertNotNull(task.getResult()); + assertEquals(BaseTestHelper.EXECUTION_SUCCESSFUL_MESSAGE, task.getResult()); + } + } + } +} diff --git a/tests/src/tests/test/org/jppf/server/peer/TestMultiServer.java b/tests/src/tests/test/org/jppf/server/peer/TestMultiServer.java index baff30c0be..4d052a7a87 100644 --- a/tests/src/tests/test/org/jppf/server/peer/TestMultiServer.java +++ b/tests/src/tests/test/org/jppf/server/peer/TestMultiServer.java @@ -169,8 +169,6 @@ public void testJobMultipleChannels() throws Exception { final LoadBalancingInformation lbi = client.getLoadBalancerSettings(); final Map lbiMap = new HashMap<>(); try { - //final TypedProperties props = new TypedProperties().setInt("performanceCacheSize", 1000).setInt("proportionalityFactor", 1).setInt("initialSize", 10).setDouble("initialMeanTime", 3e10); - //client.setLoadBalancerSettings("proportional", props); client.setLoadBalancerSettings("manual", new TypedProperties().setInt("size", 10)); final TypedProperties propsDriver = new TypedProperties().setInt("size", 5); final List pools = client.awaitConnectionPools(Operator.EQUAL, 2, Operator.AT_LEAST, 1, TIMEOUT - 500, JPPFClientConnectionStatus.workingStatuses()); diff --git a/tests/src/tests/test/org/jppf/server/peer/TestMultiServerWithSSL.java b/tests/src/tests/test/org/jppf/server/peer/TestMultiServerWithSSL.java index cd11cc8033..cd7fdd59b3 100644 --- a/tests/src/tests/test/org/jppf/server/peer/TestMultiServerWithSSL.java +++ b/tests/src/tests/test/org/jppf/server/peer/TestMultiServerWithSSL.java @@ -28,7 +28,6 @@ * all setup with SSL 2-way authentication. * @author Laurent Cohen */ -//@Ignore public class TestMultiServerWithSSL extends AbstractNonStandardSetup { /** * Launches a 2 drivers with 1 node attached to each and start the client, diff --git a/tests/src/tests/test/org/jppf/server/peer/TestMultiServerWithSSLSetup.java b/tests/src/tests/test/org/jppf/server/peer/TestMultiServerWithSSLSetup.java index ad94a3cc47..19ae3c5398 100644 --- a/tests/src/tests/test/org/jppf/server/peer/TestMultiServerWithSSLSetup.java +++ b/tests/src/tests/test/org/jppf/server/peer/TestMultiServerWithSSLSetup.java @@ -27,7 +27,6 @@ * all setup with SSL 2-way authentication. * @author Laurent Cohen */ -//@Ignore public class TestMultiServerWithSSLSetup extends AbstractNonStandardSetup { /** * Test timeout. diff --git a/tests/src/tests/test/org/jppf/server/protocol/TestJPPFJobSLA.java b/tests/src/tests/test/org/jppf/server/protocol/TestJPPFJobSLA.java index affdc4ed02..ea11c762c9 100644 --- a/tests/src/tests/test/org/jppf/server/protocol/TestJPPFJobSLA.java +++ b/tests/src/tests/test/org/jppf/server/protocol/TestJPPFJobSLA.java @@ -25,19 +25,13 @@ import java.util.*; import java.util.concurrent.atomic.AtomicInteger; -import javax.management.*; - import org.jppf.client.*; import org.jppf.client.event.*; -import org.jppf.job.*; -import org.jppf.management.*; -import org.jppf.management.forwarding.JPPFNodeForwardingMBean; +import org.jppf.management.JMXDriverConnectionWrapper; import org.jppf.node.policy.*; import org.jppf.node.protocol.*; import org.jppf.scheduling.JPPFSchedule; -import org.jppf.server.job.management.DriverJobManagementMBean; import org.jppf.utils.*; -import org.jppf.utils.Operator; import org.jppf.utils.streams.StreamUtils; import org.junit.*; @@ -267,63 +261,6 @@ public void testJobExecutionPolicy() throws Exception { } } - /** - * Test that a job is only executed on one node at a time. - * @throws Exception if any error occurs. - */ - @Test(timeout=8000) - @Ignore - public void testJobMaxNodes() throws Exception { - final int nbTasks = 5 * BaseSetup.nbNodes(); - final JPPFJob job = BaseTestHelper.createJob(ReflectionUtils.getCurrentMethodName(), true, false, nbTasks, LifeCycleTask.class, 250L); - job.getSLA().setMaxNodes(1); - final List> results = client.submitJob(job); - assertNotNull(results); - assertEquals(results.size(), nbTasks); - // check that no 2 tasks were executing at the same time on different nodes - for (int i=0; i r1 = new Range<>(t1.getStart(), t1.getStart() + t1.getElapsed()); - for (int j=i+1; j r2 = new Range<>(t2.getStart(), t2.getStart() + t2.getElapsed()); - assertFalse("r1=" + r1 + ", r2=" + r2 + ", uuid1=" + t1.getNodeUuid() + ", uuid2=" + t2.getNodeUuid(), - r1.intersects(r2, false) && !t1.getNodeUuid().equals(t2.getNodeUuid())); - } - } - } - - /** - * Test that a job is executed on both nodes. - * @throws Exception if any error occurs. - */ - @Test(timeout=8000) - @Ignore - public void testJobMaxNodes2() throws Exception { - final int nbTasks = 5 * BaseSetup.nbNodes(); - final JPPFJob job = BaseTestHelper.createJob(ReflectionUtils.getCurrentMethodName(), true, false, nbTasks, LifeCycleTask.class, 250L); - job.getSLA().setMaxNodes(2); - final List> results = client.submitJob(job); - assertNotNull(results); - assertEquals(results.size(), nbTasks); - boolean found = false; - // check that at least 2 tasks were executing at the same time on different nodes - for (int i=0; i r1 = new Range<>(t1.getStart(), t1.getStart() + t1.getElapsed()); - for (int j=i+1; j r2 = new Range<>(t2.getStart(), t2.getStart() + t2.getElapsed()); - if (r1.intersects(r2) && !t1.getNodeUuid().equals(t2.getNodeUuid())) { - found = true; - break; - } - } - if (found) break; - } - assertTrue(found); - } - /** * Test that a broadcast job is executed on all nodes. * @throws Exception if any error occurs. @@ -401,53 +338,6 @@ public void testBroadcastJobNoNodeAvailable() throws Exception { } } - /** - * Test that a job is not resubmitted when the SLA flag {@code applyMaxResubmitsUponNoError} is true - * and {@code maxTaskResubmits} is set to 0. - * @throws Exception if any error occurs. - */ - @Test(timeout=10000) - @Ignore - public void testApplyMaxResubmitsUponNodeError() throws Exception { - try { - final JPPFJob job = BaseTestHelper.createJob(ReflectionUtils.getCurrentClassAndMethod(), true, false, 1, LifeCycleTask.class, 15000L); - final ExecutionPolicy n1Policy = new Equal("jppf.uuid", false, "n1"); - job.getSLA().setExecutionPolicy(n1Policy); - job.getSLA().setMaxTaskResubmits(0); - job.getSLA().setApplyMaxResubmitsUponNodeError(true); - JPPFConnectionPool pool; - while ((pool = client.getConnectionPool()) == null) Thread.sleep(10L); - final JMXDriverConnectionWrapper jmx = pool.getJmxConnection(); - final DriverJobManagementMBean jobManager = jmx.getJobManager(); - final JPPFNodeForwardingMBean forwarder = jmx.getNodeForwarder(); - // restart the node upon first dispatch of the job to this node - final NotificationListener listener = new NotificationListener() { - @Override - public synchronized void handleNotification(final Notification notification, final Object handback) { - final JobNotification jobNotif = (JobNotification) notification; - if (jobNotif.getEventType() == JobEventType.JOB_DISPATCHED) { - try { - Thread.sleep(500L); - forwarder.forwardInvoke(new UuidSelector("n1"), JPPFNodeAdminMBean.MBEAN_NAME, "restart"); - } catch (@SuppressWarnings("unused") final Exception ignore) { - //ignore.printStackTrace(); - } - } - } - }; - jobManager.addNotificationListener(listener, null, null); - final List> results = client.submitJob(job); - assertNotNull(results); - assertEquals(1, results.size()); - final LifeCycleTask task = (LifeCycleTask) results.get(0); - assertNull(task.getResult()); - assertNull(task.getThrowable()); - assertNull(task.getNodeUuid()); - } finally { - BaseSetup.checkDriverAndNodesInitialized(1, 2); - } - } - /** * Test that results are returned according to the SendNodeResultsStrategy specified in the SLA. * @throws Exception if any error occurs.