From cfc9579fe7d9872e3bd4e0875ecfb39487f08d9d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 9 Oct 2011 16:11:21 +0200 Subject: [PATCH 001/310] Removed deprecated optimization interface and introduced advanced plugin architecture --- .../nephele/jobmanager/JobManager.java | 55 +-------- .../nephele/plugins/AbstractPluginLoader.java | 80 +++++++++++++ .../JobManagerPlugin.java} | 14 +-- .../nephele/plugins/PluginManager.java | 109 ++++++++++++++++++ .../nephele/plugins/TaskManagerPlugin.java | 20 ++++ 5 files changed, 217 insertions(+), 61 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java rename nephele/nephele-server/src/main/java/eu/stratosphere/nephele/{optimizer/Optimizer.java => plugins/JobManagerPlugin.java} (75%) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 5714f1f1d0835..16ff5362f0435 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -105,7 +105,8 @@ import eu.stratosphere.nephele.managementgraph.ManagementGraph; import eu.stratosphere.nephele.managementgraph.ManagementVertexID; import eu.stratosphere.nephele.multicast.MulticastManager; -import eu.stratosphere.nephele.optimizer.Optimizer; +import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.plugins.PluginManager; import eu.stratosphere.nephele.profiling.JobManagerProfiler; import eu.stratosphere.nephele.profiling.ProfilingUtils; import eu.stratosphere.nephele.protocols.ChannelLookupProtocol; @@ -144,8 +145,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol private final JobManagerProfiler profiler; - private final Optimizer optimizer; - private final EventCollector eventCollector; private final InputSplitManager inputSplitManager; @@ -158,6 +157,8 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol private final CheckpointDecisionCoordinator checkpointDecisionCoordinator; + private final List jobManagerPlugins; + private final int recommendedClientPollingInterval; private final ExecutorService executorService = Executors.newSingleThreadExecutor(); @@ -286,53 +287,14 @@ public JobManager(final String configDir, final String executionMode) { LOG.debug("Profiler disabled"); } - // Load optimizer if it should be used - if (GlobalConfiguration.getBoolean("jobmanager.optimizer.enable", false)) { - final String optimizerClassName = GlobalConfiguration.getString("jobmanager.optimizer.classname", null); - if (optimizerClassName == null) { - LOG.error("Cannot find class name for the optimizer"); - System.exit(FAILURERETURNCODE); - } - this.optimizer = loadOptimizer(optimizerClassName); - } else { - this.optimizer = null; - LOG.debug("Optimizer disabled"); - } + // Load the plugins + this.jobManagerPlugins = PluginManager.getJobManagerPlugins(configDir); // Add shutdown hook for clean up tasks Runtime.getRuntime().addShutdownHook(new JobManagerCleanUp(this)); } - @SuppressWarnings("unchecked") - private Optimizer loadOptimizer(String optimizerClassName) { - - final Class optimizerClass; - try { - optimizerClass = (Class) Class.forName(optimizerClassName); - } catch (ClassNotFoundException e) { - LOG.error("Cannot find class " + optimizerClassName + ": " + StringUtils.stringifyException(e)); - return null; - } - - Optimizer optimizer = null; - - try { - optimizer = optimizerClass.newInstance(); - } catch (InstantiationException e) { - LOG.error("Cannot create optimizer: " + StringUtils.stringifyException(e)); - return null; - } catch (IllegalAccessException e) { - LOG.error("Cannot create optimizer: " + StringUtils.stringifyException(e)); - return null; - } catch (IllegalArgumentException e) { - LOG.error("Cannot create optimizer: " + StringUtils.stringifyException(e)); - return null; - } - - return optimizer; - } - /** * This is the main */ @@ -507,11 +469,6 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { synchronized (eg) { - // Perform graph optimizations - if (this.optimizer != null) { - this.optimizer.optimize(eg); - } - // Check if profiling should be enabled for this job boolean profilingEnabled = false; if (this.profiler != null && job.getJobConfiguration().getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java new file mode 100644 index 0000000000000..ca6a2f13b8fa4 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java @@ -0,0 +1,80 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.plugins; + +import eu.stratosphere.nephele.configuration.Configuration; + +/** + * This abstract class must be inherited by each plugin for Nephele. It specifies how to instantiate the individual + * plugin components and provides access to the plugin environment, for example the plugin configuration. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public abstract class AbstractPluginLoader { + + /** + * The configuration for this plugin. + */ + private final Configuration pluginConfiguration; + + /** + * Protected constructor, so sub-classes are not forced to provide their own constructors. This constructor is not + * expected to ever be called. + */ + protected AbstractPluginLoader() { + this.pluginConfiguration = null; + + throw new IllegalStateException("Subclass must not call default constructor"); + } + + /** + * Constructs the plugin loader. + * + * @param pluginConfiguration + * the plugin configuration + */ + public AbstractPluginLoader(final Configuration pluginConfiguration) { + this.pluginConfiguration = pluginConfiguration; + } + + /** + * Returns the {@link Configuration} for this plugin. + * + * @return the {@link Configuration} for this plugin + */ + protected final Configuration getPluginConfiguration() { + + return this.pluginConfiguration; + } + + /** + * Loads and returns the plugin component which is supposed to run inside Nephele's {@link JobManager}. + * + * @return the {@link JobManager} plugin component or null if this plugin does not provide such a + * component. + */ + public abstract JobManagerPlugin getJobManagerPlugin(); + + /** + * Loads and returns the plugin component which is supposed to run inside Nephele's {@link TaskManager}. + * + * @return the {@link TaskManager} plugin component or null if this plugin does not provide such a + * component. + */ + public abstract TaskManagerPlugin getTaskManagerPlugin(); +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/optimizer/Optimizer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java similarity index 75% rename from nephele/nephele-server/src/main/java/eu/stratosphere/nephele/optimizer/Optimizer.java rename to nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java index 5bea9c2dd4801..9cd48289532aa 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/optimizer/Optimizer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java @@ -13,18 +13,8 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.optimizer; +package eu.stratosphere.nephele.plugins; -import eu.stratosphere.nephele.executiongraph.ExecutionGraph; - -/** - * Common interface for Nephele execution graph optimizer. - * - * @author warneke - * TODO - */ -public interface Optimizer { - - void optimize(ExecutionGraph executionGraph); +public interface JobManagerPlugin { } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java new file mode 100644 index 0000000000000..9eeaeb2fe3b19 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java @@ -0,0 +1,109 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.plugins; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +public final class PluginManager { + + /** + * The log object used to report errors and information in general. + */ + private static final Log LOG = LogFactory.getLog(PluginManager.class); + + /** + * The name of the file containing the plugin configuration. + */ + private static final String PLUGIN_CONFIG_FILE = "nephele-plugins.xml"; + + /** + * The singleton instance of this class. + */ + private static PluginManager INSTANCE = null; + + private final Map plugins; + + private PluginManager(final String configDir) { + + this.plugins = loadPlugins(); + } + + private Map loadPlugins() { + + final Map tmpPluginList = new LinkedHashMap(); + + return Collections.unmodifiableMap(tmpPluginList); + } + + private static synchronized PluginManager getInstance(final String configDir) { + + if (INSTANCE == null) { + INSTANCE = new PluginManager(configDir); + } + + return INSTANCE; + } + + private List getJobManagerPluginsInternal() { + + final List jobManagerPluginList = new ArrayList(); + + final Iterator it = this.plugins.values().iterator(); + while (it.hasNext()) { + + final JobManagerPlugin jmp = it.next().getJobManagerPlugin(); + if (jmp != null) { + jobManagerPluginList.add(jmp); + } + } + + return Collections.unmodifiableList(jobManagerPluginList); + } + + private List getTaskManagerPluginsInternal() { + + final List taskManagerPluginList = new ArrayList(); + + final Iterator it = this.plugins.values().iterator(); + while (it.hasNext()) { + + final TaskManagerPlugin jmp = it.next().getTaskManagerPlugin(); + if (jmp != null) { + taskManagerPluginList.add(jmp); + } + } + + return Collections.unmodifiableList(taskManagerPluginList); + } + + public static List getJobManagerPlugins(final String configDir) { + + return getInstance(configDir).getJobManagerPluginsInternal(); + } + + public static List getTaskManagerPlugins(final String configDir) { + + return getInstance(configDir).getTaskManagerPluginsInternal(); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java new file mode 100644 index 0000000000000..1c8b1bf7dce3d --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java @@ -0,0 +1,20 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.plugins; + +public interface TaskManagerPlugin { + +} From dcd9d29a434343b8e5456f74bf72344b7a0141bc Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 9 Oct 2011 17:34:39 +0200 Subject: [PATCH 002/310] Implemented parser for plugin configuration --- .../nephele/plugins/AbstractPluginLoader.java | 10 - .../nephele/plugins/PluginManager.java | 271 +++++++++++++++++- 2 files changed, 269 insertions(+), 12 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java index ca6a2f13b8fa4..ab7040729382c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java @@ -32,16 +32,6 @@ public abstract class AbstractPluginLoader { */ private final Configuration pluginConfiguration; - /** - * Protected constructor, so sub-classes are not forced to provide their own constructors. This constructor is not - * expected to ever be called. - */ - protected AbstractPluginLoader() { - this.pluginConfiguration = null; - - throw new IllegalStateException("Subclass must not call default constructor"); - } - /** * Constructs the plugin loader. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java index 9eeaeb2fe3b19..e276585dfa65f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java @@ -15,6 +15,10 @@ package eu.stratosphere.nephele.plugins; +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; @@ -22,9 +26,29 @@ import java.util.List; import java.util.Map; +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; +import org.w3c.dom.Text; +import org.xml.sax.SAXException; + +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.util.StringUtils; +/** + * The plugin manager is responsible for loading and managing the individual plugins. + *

+ * This class is thread-safe. + * + * @author warneke + */ public final class PluginManager { /** @@ -46,13 +70,256 @@ public final class PluginManager { private PluginManager(final String configDir) { - this.plugins = loadPlugins(); + // Check if the configuration file exists + final File configFile = new File(configDir + File.separator + PLUGIN_CONFIG_FILE); + if (configFile.exists()) { + this.plugins = loadPlugins(configFile); + } else { + this.plugins = Collections.emptyMap(); + LOG.warn("Unable to load plugins: configuration file " + configFile.getAbsolutePath() + " not found"); + } } - private Map loadPlugins() { + private String getTextChild(final Node node) { + + final NodeList nodeList = node.getChildNodes(); + if (nodeList.getLength() != 1) { + return null; + } + + final Node child = nodeList.item(0); + if (!(child instanceof Text)) { + return null; + } + final Text text = (Text) child; + + return text.getNodeValue(); + } + + @SuppressWarnings("unchecked") + private Map loadPlugins(final File configFile) { final Map tmpPluginList = new LinkedHashMap(); + final DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory.newInstance(); + // Ignore comments in the XML file + docBuilderFactory.setIgnoringComments(true); + docBuilderFactory.setNamespaceAware(true); + + try { + + final DocumentBuilder builder = docBuilderFactory.newDocumentBuilder(); + final Document doc = builder.parse(configFile); + + if (doc == null) { + LOG.error("Unable to load plugins: doc is null"); + return Collections.emptyMap(); + } + + final Element root = doc.getDocumentElement(); + if (root == null) { + LOG.error("Unable to load plugins: root is null"); + return Collections.emptyMap(); + } + + if (!"plugins".equals(root.getNodeName())) { + LOG.error("Unable to load plugins: unknown element " + root.getNodeName()); + return Collections.emptyMap(); + } + + final NodeList pluginNodes = root.getChildNodes(); + + int pluginCounter = 0; + for (int i = 0; i < pluginNodes.getLength(); ++i) { + + final Node pluginNode = pluginNodes.item(i); + + // Ignore text at this point + if (pluginNode instanceof Text) { + continue; + } + + if (!"plugin".equals(pluginNode.getNodeName())) { + LOG.error("Unable to load plugins: unknown element " + pluginNode.getNodeName()); + continue; + } + + // Increase plugin counter + ++pluginCounter; + + final NodeList pluginChildren = pluginNode.getChildNodes(); + String pluginName = null; + String pluginClass = null; + Configuration pluginConfiguration = null; + + for (int j = 0; j < pluginChildren.getLength(); ++j) { + + final Node pluginChild = pluginChildren.item(j); + + // Ignore text at this point + if (pluginChild instanceof Text) { + continue; + } + + if ("name".equals(pluginChild.getNodeName())) { + pluginName = getTextChild(pluginChild); + if (pluginName == null) { + LOG.error("Skipping plugin " + pluginCounter + + " from configuration because it does not provide a proper name"); + continue; + } + } + + if ("class".equals(pluginChild.getNodeName())) { + pluginClass = getTextChild(pluginChild); + if (pluginClass == null) { + LOG.error("Skipping plugin " + pluginCounter + + " from configuration because it does not provide a loader class"); + continue; + } + } + + if ("configuration".equals(pluginChild.getNodeName())) { + + pluginConfiguration = new Configuration(); + + final NodeList configurationNodes = pluginChild.getChildNodes(); + for (int k = 0; k < configurationNodes.getLength(); ++k) { + + final Node configurationNode = configurationNodes.item(k); + + // Ignore text at this point + if (configurationNode instanceof Text) { + continue; + } + + if (!"property".equals(configurationNode.getNodeName())) { + LOG.error("Unexpected node " + configurationNode.getNodeName() + ", skipping..."); + continue; + } + + String key = null; + String value = null; + + final NodeList properties = configurationNode.getChildNodes(); + for (int l = 0; l < properties.getLength(); ++l) { + + final Node property = properties.item(l); + + // Ignore text at this point + if (configurationNode instanceof Text) { + continue; + } + + if ("key".equals(property.getNodeName())) { + key = getTextChild(property); + if (key == null) { + LOG.warn("Skipping configuration entry for plugin " + pluginName + + " because of invalid key"); + continue; + } + } + + if ("value".equals(property.getNodeName())) { + value = getTextChild(property); + if (value == null) { + LOG.warn("Skipping configuration entry for plugin " + pluginName + + " because of invalid value"); + continue; + } + } + } + + if (key != null && value != null) { + pluginConfiguration.setString(key, value); + } + } + + } + } + + if (pluginName == null) { + LOG.error("Plugin " + pluginCounter + " does not provide a name, skipping..."); + continue; + } + + if (pluginClass == null) { + LOG.error("Plugin " + pluginCounter + " does not provide a loader class, skipping..."); + continue; + } + + if (pluginConfiguration == null) { + LOG.warn("Plugin " + pluginCounter + + " does not provide a configuration, using default configuration"); + pluginConfiguration = new Configuration(); + } + + Class loaderClass; + + try { + loaderClass = (Class) Class.forName(pluginClass); + } catch (ClassNotFoundException e) { + LOG.error("Unable to load plugin " + pluginName + ": " + StringUtils.stringifyException(e)); + continue; + } + + if (loaderClass == null) { + LOG.error("Unable to load plugin " + pluginName + ": loaderClass is null"); + continue; + } + + Constructor constructor; + try { + constructor = (Constructor) loaderClass + .getConstructor(Configuration.class); + } catch (SecurityException e) { + LOG.error("Unable to load plugin " + pluginName + ": " + StringUtils.stringifyException(e)); + continue; + } catch (NoSuchMethodException e) { + LOG.error("Unable to load plugin " + pluginName + ": " + StringUtils.stringifyException(e)); + continue; + } + + if (constructor == null) { + LOG.error("Unable to load plugin " + pluginName + ": constructor is null"); + continue; + } + + AbstractPluginLoader pluginLoader = null; + + try { + pluginLoader = constructor.newInstance(pluginConfiguration); + } catch (IllegalArgumentException e) { + LOG.error("Unable to load plugin " + pluginName + ": " + StringUtils.stringifyException(e)); + continue; + } catch (InstantiationException e) { + LOG.error("Unable to load plugin " + pluginName + ": " + StringUtils.stringifyException(e)); + continue; + } catch (IllegalAccessException e) { + LOG.error("Unable to load plugin " + pluginName + ": " + StringUtils.stringifyException(e)); + continue; + } catch (InvocationTargetException e) { + LOG.error("Unable to load plugin " + pluginName + ": " + StringUtils.stringifyException(e)); + continue; + } + + if (pluginLoader == null) { + LOG.error("Unable to load plugin " + pluginName + ": pluginLoader is null"); + continue; + } + + LOG.info("Successfully loaded plugin " + pluginName); + tmpPluginList.put(pluginName, pluginLoader); + } + + } catch (IOException e) { + LOG.error("Error while loading plugins: " + StringUtils.stringifyException(e)); + } catch (SAXException e) { + LOG.error("Error while loading plugins: " + StringUtils.stringifyException(e)); + } catch (ParserConfigurationException e) { + LOG.error("Error while loading plugins: " + StringUtils.stringifyException(e)); + } + return Collections.unmodifiableMap(tmpPluginList); } From cf35afa2ac56e2d7e9f95a95edf00de89d3ba7da Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 9 Oct 2011 18:04:17 +0200 Subject: [PATCH 003/310] Added hooks for plugins in the job manager --- .../nephele/jobmanager/JobManager.java | 35 +++++++++++++++++-- .../nephele/plugins/JobManagerPlugin.java | 33 +++++++++++++++++ 2 files changed, 66 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 16ff5362f0435..0a8743a3020bd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -343,6 +343,11 @@ public synchronized void shutdown() { } + // Stop the plugins + for (JobManagerPlugin plugin : this.jobManagerPlugins) { + plugin.shutdown(); + } + // Stop and clean up the job progress collector if (this.eventCollector != null) { this.eventCollector.shutdown(); @@ -436,7 +441,7 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { } LOG.debug("The graph of job " + job.getName() + " is acyclic"); - + // Check constrains on degree jv = job.areVertexDegreesCorrect(); if (jv != null) { @@ -456,6 +461,19 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { LOG.debug("The dependency chain for instance sharing is acyclic"); + // Allow plugins to rewrite the job graph + for(final JobManagerPlugin plugin : this.jobManagerPlugins) { + final JobGraph inputJob = job; + job = plugin.rewriteJobGraph(inputJob); + if(job == null) { + LOG.warn("Plugin " + plugin + " set job graph to null, reverting changes..."); + job = inputJob; + } + if(job != inputJob) { + LOG.debug("Plugin " + plugin + " rewrote job graph"); + } + } + // Try to create initial execution graph from job graph LOG.info("Creating initial execution graph from job graph " + job.getName()); ExecutionGraph eg = null; @@ -467,8 +485,21 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { return result; } + // Allow plugins to rewrite the execution graph + for(final JobManagerPlugin plugin : this.jobManagerPlugins) { + final ExecutionGraph inputGraph = eg; + eg = plugin.rewriteExecutionGraph(inputGraph); + if(eg == null) { + LOG.warn("Plugin " + plugin + " set execution graph to null, reverting changes..."); + eg = inputGraph; + } + if(eg != inputGraph) { + LOG.debug("Plugin " + plugin + " rewrote execution graph"); + } + } + synchronized (eg) { - + // Check if profiling should be enabled for this job boolean profilingEnabled = false; if (this.profiler != null && job.getJobConfiguration().getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java index 9cd48289532aa..78f7e7533e9e5 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java @@ -15,6 +15,39 @@ package eu.stratosphere.nephele.plugins; +import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.jobgraph.JobGraph; + +/** + * This interface must be implemented by every plugin component which is supposed to run inside Nephele's job manager. + * + * @author warneke + */ public interface JobManagerPlugin { + /** + * This method is called upon the reception of a new job graph. It gives the plugin the possibility to to rewrite + * the job graph before it is processed further. + * + * @param jobGraph + * the original job graph + * @return the rewritten job graph + */ + JobGraph rewriteJobGraph(JobGraph jobGraph); + + /** + * This method is called after the initial execution graph has been created from the user's job graph. It gives the + * plugin the possibility to rewrite the execution graph before it is processed further or to register to particular + * events. + * + * @param executionGraph + * the initial execution graph + * @return the rewritten execution graph + */ + ExecutionGraph rewriteExecutionGraph(ExecutionGraph executionGraph); + + /** + * Called by the job manager to indicate that Nephele is about to shut down. + */ + void shutdown(); } From b703fe3ab7dacc264841b48fa4c26d263382c3ac Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 9 Oct 2011 18:09:13 +0200 Subject: [PATCH 004/310] Added hooks for task manager plugins --- .../nephele/plugins/TaskManagerPlugin.java | 4 ++++ .../nephele/taskmanager/TaskManager.java | 13 +++++++++++++ 2 files changed, 17 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java index 1c8b1bf7dce3d..cae411a1be980 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java @@ -17,4 +17,8 @@ public interface TaskManagerPlugin { + /** + * Called by the task manager to indicate that Nephele is about to shut down. + */ + void shutdown(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index cf62b8f5b78f1..492fbdac87d2d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -63,6 +63,9 @@ import eu.stratosphere.nephele.ipc.Server; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.net.NetUtils; +import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.plugins.PluginManager; +import eu.stratosphere.nephele.plugins.TaskManagerPlugin; import eu.stratosphere.nephele.profiling.ProfilingUtils; import eu.stratosphere.nephele.profiling.TaskManagerProfiler; import eu.stratosphere.nephele.protocols.ChannelLookupProtocol; @@ -135,6 +138,8 @@ public class TaskManager implements TaskOperationProtocol { private final HardwareDescription hardwareDescription; + private final List taskManagerPlugins; + /** * Stores whether the task manager has already been shut down. */ @@ -305,6 +310,9 @@ public TaskManager(String configDir) throws Exception { // Initialize the I/O manager this.ioManager = new IOManager(tmpDirPath); + // Load the plugins + this.taskManagerPlugins = PluginManager.getTaskManagerPlugins(configDir); + // Add shutdown hook for clean up tasks Runtime.getRuntime().addShutdownHook(new TaskManagerCleanUp(this)); } @@ -787,6 +795,11 @@ public synchronized void shutdown() { this.memoryManager.shutdown(); } + // Shut down the plugins + for(final TaskManagerPlugin plugin: this.taskManagerPlugins) { + plugin.shutdown(); + } + this.isShutDown = true; } From 35eba7b7bdfac8ae3a8835e8f56653bff4f13d72 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 9 Oct 2011 18:15:09 +0200 Subject: [PATCH 005/310] Removed superfluous import --- .../java/eu/stratosphere/nephele/taskmanager/TaskManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 492fbdac87d2d..dabca89d0eae7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -63,7 +63,6 @@ import eu.stratosphere.nephele.ipc.Server; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.net.NetUtils; -import eu.stratosphere.nephele.plugins.JobManagerPlugin; import eu.stratosphere.nephele.plugins.PluginManager; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; import eu.stratosphere.nephele.profiling.ProfilingUtils; From bb6aad01a1b199c705f598315c255c33583a1a97 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 9 Oct 2011 18:16:22 +0200 Subject: [PATCH 006/310] Added sample plugin implementation for SCORE --- pom.xml | 1 + score/pom.xml | 68 ++++++++++++++++++ .../score/ScoreExecutionListener.java | 72 +++++++++++++++++++ .../score/ScoreJobManagerPlugin.java | 67 +++++++++++++++++ .../stratosphere/score/ScorePluginLoader.java | 65 +++++++++++++++++ .../score/ScoreTaskManagerPlugin.java | 33 +++++++++ .../stratosphere-bin/conf/nephele-plugins.xml | 30 ++++++++ 7 files changed, 336 insertions(+) create mode 100644 score/pom.xml create mode 100644 score/src/main/java/eu/stratosphere/score/ScoreExecutionListener.java create mode 100644 score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java create mode 100644 score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java create mode 100644 score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java create mode 100644 stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml diff --git a/pom.xml b/pom.xml index d813665c5b1c2..fccaf6beb7f3f 100644 --- a/pom.xml +++ b/pom.xml @@ -494,6 +494,7 @@ nephele pact build-tools + score stratosphere-dist diff --git a/score/pom.xml b/score/pom.xml new file mode 100644 index 0000000000000..d2e177507fe30 --- /dev/null +++ b/score/pom.xml @@ -0,0 +1,68 @@ + + + + 4.0.0 + + + stratosphere + eu.stratosphere + 0.2 + + + eu.stratosphere + score + 0.2 + score + + jar + + + + + eu.stratosphere + nephele-server + ${version} + + + + + + + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.7 + + + true + WARN + + + **/*TestBase*.class + + once + + + + + maven-failsafe-plugin + 2.7 + + + WARN + + always + 1 + false + + + + + + diff --git a/score/src/main/java/eu/stratosphere/score/ScoreExecutionListener.java b/score/src/main/java/eu/stratosphere/score/ScoreExecutionListener.java new file mode 100644 index 0000000000000..efeb0eda2e241 --- /dev/null +++ b/score/src/main/java/eu/stratosphere/score/ScoreExecutionListener.java @@ -0,0 +1,72 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.score; + +import eu.stratosphere.nephele.execution.ExecutionListener; +import eu.stratosphere.nephele.execution.ExecutionState; +import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; +import eu.stratosphere.nephele.executiongraph.ExecutionVertex; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.jobgraph.JobID; + +public class ScoreExecutionListener implements ExecutionListener { + + private final ExecutionVertex executionVertex; + + ScoreExecutionListener(final ExecutionVertex executionVertex) { + this.executionVertex = executionVertex; + } + + /** + * {@inheritDoc} + */ + @Override + public void executionStateChanged(final JobID jobID, final ExecutionVertexID vertexID, + final ExecutionState newExecutionState, final String optionalMessage) { + + System.out.println("SCORE received execution state update for vertex " + this.executionVertex + ": " + + newExecutionState); + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadStarted(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) { + // TODO Auto-generated method stub + + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadFinished(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) { + // TODO Auto-generated method stub + + } + + /** + * {@inheritDoc} + */ + @Override + public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID vertexID, + final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { + // TODO Auto-generated method stub + + } + +} diff --git a/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java b/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java new file mode 100644 index 0000000000000..282ee1c00e49f --- /dev/null +++ b/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java @@ -0,0 +1,67 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.score; + +import java.util.Iterator; + +import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; +import eu.stratosphere.nephele.executiongraph.ExecutionVertex; +import eu.stratosphere.nephele.jobgraph.JobGraph; +import eu.stratosphere.nephele.plugins.JobManagerPlugin; + +public final class ScoreJobManagerPlugin implements JobManagerPlugin { + + /** + * {@inheritDoc} + */ + @Override + public JobGraph rewriteJobGraph(final JobGraph jobGraph) { + + // Nothing to do here + + return jobGraph; + } + + /** + * {@inheritDoc} + */ + @Override + public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) { + + synchronized (executionGraph) { + + // Register for events + final Iterator it = new ExecutionGraphIterator(executionGraph, true); + + while (it.hasNext()) { + final ExecutionVertex vertex = it.next(); + vertex.registerExecutionListener(new ScoreExecutionListener(vertex)); + } + } + + return executionGraph; + } + + /** + * {@inheritDoc} + */ + @Override + public void shutdown() { + // TODO Auto-generated method stub + + } +} diff --git a/score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java b/score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java new file mode 100644 index 0000000000000..1582a2ff064ba --- /dev/null +++ b/score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java @@ -0,0 +1,65 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.score; + +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.plugins.AbstractPluginLoader; +import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.plugins.TaskManagerPlugin; + +/** + * A plugin loader for the SCORE (Stratosphere Continuous Re-optimization) module. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class ScorePluginLoader extends AbstractPluginLoader { + + private ScoreJobManagerPlugin jobManagerPlugin = null; + + private ScoreTaskManagerPlugin taskManagerPlugin = null; + + public ScorePluginLoader(final Configuration pluginConfiguration) { + super(pluginConfiguration); + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized JobManagerPlugin getJobManagerPlugin() { + + if (this.jobManagerPlugin == null) { + this.jobManagerPlugin = new ScoreJobManagerPlugin(); + } + + return this.jobManagerPlugin; + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized TaskManagerPlugin getTaskManagerPlugin() { + + if (this.taskManagerPlugin == null) { + this.taskManagerPlugin = new ScoreTaskManagerPlugin(getPluginConfiguration()); + } + + return this.taskManagerPlugin; + } +} diff --git a/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java b/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java new file mode 100644 index 0000000000000..7764a2d43e5cd --- /dev/null +++ b/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java @@ -0,0 +1,33 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.score; + +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.plugins.TaskManagerPlugin; + +public final class ScoreTaskManagerPlugin implements TaskManagerPlugin { + + ScoreTaskManagerPlugin(final Configuration pluginConfiguration) { + } + + /** + * {@inheritDoc} + */ + @Override + public void shutdown() { + // TODO Auto-generated method stub + } +} diff --git a/stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml b/stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml new file mode 100644 index 0000000000000..cd0ce8cbf49da --- /dev/null +++ b/stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml @@ -0,0 +1,30 @@ + + + + + SCORE + eu.stratosphere.score.ScorePluginLoader + + + testkey + testvalue + + + + + From 121fe0f6b2e0940a0e5cb7cacaa363d7541be08c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 10 Oct 2011 14:01:41 +0200 Subject: [PATCH 007/310] Fixed some merge problems --- .../scheduler/queue/QueueScheduler.java | 53 ++++++++++++++++++- .../executiongraph/ExecutionGraph.java | 23 +++++++- .../nephele/jobmanager/JobManager.java | 9 ++++ .../nephele/jobmanager/RecoveryThread.java | 28 ++-------- .../nephele/taskmanager/TaskManager.java | 2 +- 5 files changed, 86 insertions(+), 29 deletions(-) diff --git a/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java b/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java index 628cfc8268e94..2871caf365d2b 100644 --- a/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java +++ b/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java @@ -32,6 +32,7 @@ import eu.stratosphere.nephele.executiongraph.InternalJobStatus; import eu.stratosphere.nephele.executiongraph.JobStatusListener; import eu.stratosphere.nephele.instance.AllocatedResource; +import eu.stratosphere.nephele.instance.DummyInstance; import eu.stratosphere.nephele.instance.InstanceException; import eu.stratosphere.nephele.instance.InstanceManager; import eu.stratosphere.nephele.instance.InstanceRequestMap; @@ -195,8 +196,58 @@ public ExecutionGraph getExecutionGraphByID(final JobID jobID) { */ @Override public void allocatedResourcesDied(final JobID jobID, final List allocatedResources) { - // TODO Auto-generated method stub + for (final AllocatedResource allocatedResource : allocatedResources) { + + LOG.info("Resource on " + allocatedResource.getInstance().getName() + " for Job " + jobID + " died."); + // TODO (marrus) + + ExecutionGraph job = this.jobQueue.getFirst(); + synchronized (job) { + + Iterator iterator = this.jobQueue.descendingIterator(); + while (job.getJobID() != jobID) { + if (iterator.hasNext()) { + job = iterator.next(); + } else { + LOG.error("No Job with ID " + jobID + " in Queue"); + return; + } + } + List vertices = job.getVerticesAssignedToResource(allocatedResource); + Iterator vertexIter = vertices.iterator(); + while (vertexIter.hasNext()) { + ExecutionVertex vertex = vertexIter.next(); + vertex.updateExecutionState(ExecutionState.FAILED, "The Resource " + + allocatedResource.getInstance().getName() + " the Vertex " + + vertex.getEnvironment().getTaskName() + " was assigned to, died"); + if (vertex.getExecutionState() == ExecutionState.FAILED) { + job.executionStateChanged(jobID, vertex.getID(), ExecutionState.FAILED, "The Resource " + + allocatedResource.getInstance().getName() + " the Vertex " + + vertex.getEnvironment().getTaskName() + " was assigned to, died"); + return; + } + + vertex.setAllocatedResource(new AllocatedResource(DummyInstance + .createDummyInstance(allocatedResource + .getInstanceType()), allocatedResource.getInstanceType(), + null)); + } + + try { + LOG.info("Trying to allocate instance of type " + + allocatedResource.getInstanceType().getIdentifier()); + final InstanceRequestMap instanceMap = new InstanceRequestMap(); + instanceMap.setMaximumNumberOfInstances(allocatedResource.getInstanceType(), 1); + instanceMap.setMinimumNumberOfInstances(allocatedResource.getInstanceType(), 1); + this.getInstanceManager().requestInstance(jobID, job.getJobConfiguration(), instanceMap, null); + + } catch (InstanceException e) { + e.printStackTrace(); + } + job.executionStateChanged(jobID, vertices.get(0).getID(), ExecutionState.RECOVERING, null); + } + } } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index ac339c245a85f..806c036084af9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -130,7 +130,7 @@ public class ExecutionGraph implements ExecutionListener { private List executionStageListeners = new ArrayList(); private List recovering = new ArrayList(); - + /** * Private constructor used for duplicating execution vertices. * @@ -1229,10 +1229,20 @@ public synchronized void checkAndUpdateJobStatus(final ExecutionState latestStat } } } + if (latestStateChange == ExecutionState.RECOVERING) { + this.jobStatus = InternalJobStatus.RECOVERING; + return; + } if (jobHasFinishedStatus()) { this.jobStatus = InternalJobStatus.FINISHED; } break; + case RECOVERING: + if (latestStateChange == ExecutionState.RERUNNING) { + this.recovering.clear(); + this.jobStatus = InternalJobStatus.RUNNING; + } + break; case FAILING: if (jobHasFailedOrCanceledStatus()) { this.jobStatus = InternalJobStatus.FAILED; @@ -1274,6 +1284,10 @@ public synchronized void executionStateChanged(final JobID jobID, final Executio final InternalJobStatus oldStatus = this.jobStatus; + if (newExecutionState == ExecutionState.RERUNNING) { + this.recovering.remove(getVertexByID(vertexID)); + } + checkAndUpdateJobStatus(newExecutionState); if (newExecutionState == ExecutionState.FINISHED) { @@ -1291,6 +1305,11 @@ public synchronized void executionStateChanged(final JobID jobID, final Executio } } } + if (this.jobStatus == InternalJobStatus.RECOVERING){ + LOG.info("RECOVERING"); + //FIXME (marrus) see if we even need that + this.recovering.add(this.getVertexByID(vertexID)); + } if (this.jobStatus != oldStatus) { @@ -1449,7 +1468,7 @@ public void initialExecutionResourcesExhausted(final JobID jobID, final Executio // Nothing to do here } - + public List getFailedVertices() { return this.recovering; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 5714f1f1d0835..ae0e63debcf37 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -1053,6 +1053,15 @@ public void jobStatusHasChanged(final ExecutionGraph executionGraph, final Inter // Unregister job for Nephele's monitoring, optimization components, and dynamic input split assignment unregisterJob(executionGraph); } + if (newJobStatus == InternalJobStatus.RECOVERING) { + try { + RecoveryThread recoverythread = new RecoveryThread(executionGraph, this); + recoverythread.start(); + } catch (Exception e) { + e.printStackTrace(); + } + + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java index 157e6cace7200..34d4f75b64cd1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java @@ -15,43 +15,21 @@ package eu.stratosphere.nephele.jobmanager; import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.UnknownHostException; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.PriorityQueue; import java.util.Queue; -import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.configuration.ConfigConstants; -import eu.stratosphere.nephele.configuration.GlobalConfiguration; -import eu.stratosphere.nephele.discovery.DiscoveryException; -import eu.stratosphere.nephele.discovery.DiscoveryService; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.instance.AbstractInstance; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.channels.AbstractInputChannel; -import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; -import eu.stratosphere.nephele.io.compression.CompressionLevel; -import eu.stratosphere.nephele.ipc.RPC; -import eu.stratosphere.nephele.net.NetUtils; import eu.stratosphere.nephele.protocols.JobManagerProtocol; -import eu.stratosphere.nephele.taskmanager.TaskExecutionState; -import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.util.StringUtils; /** @@ -183,7 +161,7 @@ private List findRestarts(ExecutionVertex failed) { restart.add(successor); LOG.info("add " + successor.getName() + " torestart"); // totest.add(successor); - if (successor.getCheckpointState() == CheckpointState.COMPLETE) { + if (successor.getCheckpointState() == CheckpointState.PARTIAL) { this.checkpoints.remove(successor); } @@ -200,7 +178,7 @@ private List findRestarts(ExecutionVertex failed) { } for (int j = 0; j < vertex.getNumberOfPredecessors(); j++) { ExecutionVertex predecessor = vertex.getPredecessor(j); - if (predecessor.getCheckpointState() != CheckpointState.COMPLETE) { + if (predecessor.getCheckpointState() != CheckpointState.PARTIAL) { LOG.info("add " + predecessor.getName() + " torestart"); restart.add(predecessor); @@ -244,7 +222,7 @@ private List findFollowers(ExecutionVertex vertex, ArrayList checkpointsToRemove = new ArrayList(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index cf62b8f5b78f1..8c77ea14bbafd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -688,7 +688,7 @@ public void updateLibraryCache(LibraryCacheUpdate update) throws IOException { void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final Task task, final ExecutionState newExecutionState, final String optionalDescription) { - if (newExecutionState == ExecutionState.RUNNING) { + if (newExecutionState == ExecutionState.RUNNING || newExecutionState == ExecutionState.RUNNING) { // Mark task as running by putting it in the corresponding map synchronized (this.runningTasks) { this.runningTasks.put(id, task); From 06a23be65b8a7053f94fa0f53a00a01f71c26e87 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 10 Oct 2011 19:26:46 +0200 Subject: [PATCH 008/310] Enabled checkpoints by default --- .../nephele/checkpointing/CheckpointDecisionCoordinator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index ab4ab459d4c87..f98aa289416ea 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -102,7 +102,7 @@ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtil final List checkpointDecisionList = new SerializableArrayList(); synchronized (graph) { - checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), false)); //Disabled checkpoints + checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), true)); //Enabled checkpoints checkpointDecisions.put(vertex.getAllocatedResource().getInstance(), checkpointDecisionList); } From ef6105ee89dcbd20171c0ca618e52549699b1113 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 10 Oct 2011 21:15:09 +0200 Subject: [PATCH 009/310] Wrote integration test to check Nephele's fault tolerance mechanisms --- .../checkpointing/FailingJobITCase.java | 432 ++++++++++++++++++ 1 file changed, 432 insertions(+) create mode 100644 nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java new file mode 100644 index 0000000000000..8844b3fa15c67 --- /dev/null +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java @@ -0,0 +1,432 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; + +import static org.junit.Assert.fail; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import eu.stratosphere.nephele.client.JobClient; +import eu.stratosphere.nephele.client.JobExecutionException; +import eu.stratosphere.nephele.configuration.ConfigConstants; +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.configuration.GlobalConfiguration; +import eu.stratosphere.nephele.io.BipartiteDistributionPattern; +import eu.stratosphere.nephele.io.MutableRecordReader; +import eu.stratosphere.nephele.io.RecordWriter; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.compression.CompressionLevel; +import eu.stratosphere.nephele.jobgraph.JobGenericInputVertex; +import eu.stratosphere.nephele.jobgraph.JobGenericOutputVertex; +import eu.stratosphere.nephele.jobgraph.JobGraph; +import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException; +import eu.stratosphere.nephele.jobgraph.JobTaskVertex; +import eu.stratosphere.nephele.jobmanager.JobManager; +import eu.stratosphere.nephele.template.AbstractGenericInputTask; +import eu.stratosphere.nephele.template.AbstractOutputTask; +import eu.stratosphere.nephele.template.AbstractTask; +import eu.stratosphere.nephele.types.Record; +import eu.stratosphere.nephele.util.StringUtils; + +/** + * This integration test checks Nephele's fault tolerance capabilities by a series of jobs. The jobs feature different + * connecting patterns and fail at different tasks in the processing pipeline. + * + * @author warneke + */ +public class FailingJobITCase { + + /** + * The directory containing the Nephele configuration for this integration test. + */ + private static final String CONFIGURATION_DIRECTORY = "correct-conf"; + + /** + * The number of records to generate by each producer. + */ + private static final int RECORDS_TO_GENERATE = 512 * 1024; + + /** + * The size of an individual record in bytes. + */ + private static final int RECORD_SIZE = 256; + + /** + * Number of records after which the execution failure shall occur. + */ + private static final int FAILED_AFTER_RECORD = 95490; + + /** + * The degree of parallelism for the job. + */ + private static final int DEGREE_OF_PARALLELISM = 4; + + /** + * Index of the parallel subtask in which the failure shall occur. + */ + private static final int FAILURE_INDEX = 0; + + /** + * The key to access the configuration flag for a task failure. + */ + private static final String FAILURE_KEY = "failure"; + + /** + * The key to access the index of the subtask which is supposed to fail. + */ + private static final String FAILURE_INDEX_KEY = "failure.index"; + + /** + * The thread running the job manager. + */ + private static JobManagerThread jobManagerThread = null; + + /** + * The configuration for the job client; + */ + private static Configuration configuration; + + /** + * Global flag to indicate if a task has already failed once. + */ + private static final AtomicBoolean FAILED_ONCE = new AtomicBoolean(false); + + /** + * This is an auxiliary class to run the job manager thread. + * + * @author warneke + */ + private static final class JobManagerThread extends Thread { + + /** + * The job manager instance. + */ + private final JobManager jobManager; + + /** + * Constructs a new job manager thread. + * + * @param jobManager + * the job manager to run in this thread. + */ + private JobManagerThread(final JobManager jobManager) { + + this.jobManager = jobManager; + } + + /** + * {@inheritDoc} + */ + @Override + public void run() { + + // Run task loop + this.jobManager.runTaskLoop(); + + // Shut down + this.jobManager.shutdown(); + } + + /** + * Checks whether the encapsulated job manager is completely shut down. + * + * @return true if the encapsulated job manager is completely shut down, false + * otherwise + */ + public boolean isShutDown() { + + return this.jobManager.isShutDown(); + } + } + + /** + * Sets up Nephele in local mode. + */ + @BeforeClass + public static void startNephele() { + + if (jobManagerThread == null) { + + // Create the job manager + JobManager jobManager = null; + + try { + + // Try to find the correct configuration directory + final String userDir = System.getProperty("user.dir"); + String configDir = userDir + File.separator + CONFIGURATION_DIRECTORY; + if (!new File(configDir).exists()) { + configDir = userDir + "/src/test/resources/" + CONFIGURATION_DIRECTORY; + } + + final Constructor c = JobManager.class.getDeclaredConstructor(new Class[] { String.class, + String.class }); + c.setAccessible(true); + jobManager = c.newInstance(new Object[] { configDir, + new String("local") }); + + } catch (SecurityException e) { + fail(e.getMessage()); + } catch (NoSuchMethodException e) { + fail(e.getMessage()); + } catch (IllegalArgumentException e) { + fail(e.getMessage()); + } catch (InstantiationException e) { + fail(e.getMessage()); + } catch (IllegalAccessException e) { + fail(e.getMessage()); + } catch (InvocationTargetException e) { + fail(e.getMessage()); + } + + configuration = GlobalConfiguration + .getConfiguration(new String[] { ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY }); + + // Start job manager thread + if (jobManager != null) { + jobManagerThread = new JobManagerThread(jobManager); + jobManagerThread.start(); + } + + // Wait for the local task manager to arrive + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + } + } + + /** + * Shuts Nephele down. + */ + @AfterClass + public static void stopNephele() { + + if (jobManagerThread != null) { + jobManagerThread.interrupt(); + + while (!jobManagerThread.isShutDown()) { + try { + Thread.sleep(100); + } catch (InterruptedException i) { + break; + } + } + } + } + + public static class FailingJobRecord implements Record { + + private final byte[] data = new byte[RECORD_SIZE]; + + public FailingJobRecord() { + for (int i = 0; i < this.data.length; ++i) { + this.data[i] = (byte) (i % 31); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + + out.write(this.data); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + + in.readFully(this.data); + } + } + + public final static class InputTask extends AbstractGenericInputTask { + + private RecordWriter recordWriter; + + /** + * {@inheritDoc} + */ + @Override + public void registerInputOutput() { + + this.recordWriter = new RecordWriter(this, FailingJobRecord.class); + } + + /** + * {@inheritDoc} + */ + @Override + public void invoke() throws Exception { + + final FailingJobRecord record = new FailingJobRecord(); + for (int i = 0; i < RECORDS_TO_GENERATE; ++i) { + this.recordWriter.emit(record); + } + } + } + + public final static class InnerTask extends AbstractTask { + + private MutableRecordReader recordReader; + + private RecordWriter recordWriter; + + /** + * {@inheritDoc} + */ + @Override + public void registerInputOutput() { + + this.recordWriter = new RecordWriter(this, FailingJobRecord.class); + this.recordReader = new MutableRecordReader(this, + new BipartiteDistributionPattern()); + } + + /** + * {@inheritDoc} + */ + @Override + public void invoke() throws Exception { + + final FailingJobRecord record = new FailingJobRecord(); + + int count = 0; + boolean failing = (getRuntimeConfiguration().getBoolean(FAILURE_KEY, false) && (getIndexInSubtaskGroup() == getRuntimeConfiguration() + .getInteger(FAILURE_INDEX_KEY, -1))); + while (this.recordReader.next(record)) { + + this.recordWriter.emit(record); + if (count++ == FAILED_AFTER_RECORD && failing) { + if(FAILED_ONCE.compareAndSet(false, true)) { + throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " + + getIndexInSubtaskGroup()); + } + } + } + } + } + + public static final class OutputTask extends AbstractOutputTask { + + private MutableRecordReader recordReader; + + /** + * {@inheritDoc} + */ + @Override + public void registerInputOutput() { + + this.recordReader = new MutableRecordReader(this, + new BipartiteDistributionPattern()); + } + + /** + * {@inheritDoc} + */ + @Override + public void invoke() throws Exception { + + final FailingJobRecord record = new FailingJobRecord(); + while (this.recordReader.next(record)) { + // Simply consume the records + } + } + + } + + /** + * This test checks Nephele's fault tolerance capabilities by simulating a failing inner vertex. + */ + @Test + public void testFailingInternalVertex() { + + final JobGraph jobGraph = new JobGraph("Job with failing inner vertex"); + + final JobGenericInputVertex input = new JobGenericInputVertex("Input", jobGraph); + input.setInputClass(InputTask.class); + input.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + input.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobTaskVertex innerVertex1 = new JobTaskVertex("Inner vertex 1", jobGraph); + innerVertex1.setTaskClass(InnerTask.class); + innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobTaskVertex innerVertex2 = new JobTaskVertex("Inner vertex 2", jobGraph); + innerVertex2.setTaskClass(InnerTask.class); + innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + innerVertex2.getConfiguration().setBoolean(FAILURE_KEY, true); + innerVertex2.getConfiguration().setInteger(FAILURE_INDEX_KEY, FAILURE_INDEX); + + final JobTaskVertex innerVertex3 = new JobTaskVertex("Inner vertex 3", jobGraph); + innerVertex3.setTaskClass(InnerTask.class); + innerVertex3.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex3.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobGenericOutputVertex output = new JobGenericOutputVertex("Output", jobGraph); + output.setOutputClass(OutputTask.class); + output.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + output.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + // Configure instance sharing + innerVertex1.setVertexToShareInstancesWith(input); + innerVertex2.setVertexToShareInstancesWith(input); + innerVertex3.setVertexToShareInstancesWith(input); + output.setVertexToShareInstancesWith(input); + + try { + + input.connectTo(innerVertex1, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex1.connectTo(innerVertex2, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex2.connectTo(innerVertex3, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex3.connectTo(output, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + + } catch (JobGraphDefinitionException e) { + fail(StringUtils.stringifyException(e)); + } + + // Reset the FAILED_ONCE flag + FAILED_ONCE.set(false); + + // Create job client and launch job + try { + JobClient jobClient = new JobClient(jobGraph, configuration); + jobClient.submitJobAndWait(); + } catch (IOException ioe) { + fail(StringUtils.stringifyException(ioe)); + } catch (JobExecutionException e) { + fail(StringUtils.stringifyException(e)); + } + } +} From d49db9c781fb93ec7b36cf3efc623c0895665512 Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 11 Oct 2011 11:48:30 +0200 Subject: [PATCH 010/310] starting recovery for failed tasks --- .../eu/stratosphere/nephele/executiongraph/ExecutionGraph.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index 806c036084af9..14a9c93f7e1a8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -1226,6 +1226,9 @@ public synchronized void checkAndUpdateJobStatus(final ExecutionState latestStat if (vertex.getExecutionState() == ExecutionState.FAILED && !vertex.hasRetriesLeft()) { this.jobStatus = InternalJobStatus.FAILING; return; + }else{ + this.jobStatus = InternalJobStatus.RECOVERING; + return; } } } From a7a85e66ffb30154cf1b0c602b4b43f030425aad Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 11 Oct 2011 12:16:31 +0200 Subject: [PATCH 011/310] changes TaskManager methods to IORadWritable parameters --- .../nephele/instance/AbstractInstance.java | 14 ++++++++-- .../protocols/TaskOperationProtocol.java | 4 +-- .../nephele/taskmanager/TaskManager.java | 28 +++++++------------ 3 files changed, 23 insertions(+), 23 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index 893b7ed2d4852..4b148aa8a8d0c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -17,6 +17,7 @@ import java.io.IOException; import java.net.InetSocketAddress; +import java.util.Iterator; import java.util.List; import java.util.Set; @@ -39,6 +40,7 @@ import eu.stratosphere.nephele.taskmanager.TaskSubmissionWrapper; import eu.stratosphere.nephele.topology.NetworkNode; import eu.stratosphere.nephele.topology.NetworkTopology; +import eu.stratosphere.nephele.util.SerializableArrayList; /** * An abstract instance represents a resource a {@link eu.stratosphere.nephele.taskmanager.TaskManager} runs on. @@ -210,8 +212,12 @@ public synchronized List submitTasks(final List replayCheckpoints(List vertexIDs) throws IOException { + final List checkpointResultList = new SerializableArrayList(); - return getTaskManager().replayCheckpoints(vertexIDs); + for (ExecutionVertexID vertexID : vertexIDs){ + checkpointResultList.add(getTaskManager().replayCheckpoints(vertexID)); + } + return checkpointResultList; } public synchronized void propagateCheckpointDecisions(final List checkpointDecisions) @@ -245,8 +251,10 @@ public synchronized TaskCancelResult cancelTask(final ExecutionVertexID id) thro * thrown if an error occurs while transmitting the request */ public synchronized void removeCheckpoints(final List listOfVertexIDs) throws IOException { - - getTaskManager().removeCheckpoints(listOfVertexIDs); + Iterator iter = listOfVertexIDs.iterator(); + while(iter.hasNext()){ + getTaskManager().removeCheckpoints(iter.next()); + } } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index 56008aef311a0..55e279213fc50 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -107,7 +107,7 @@ List submitTasks(List tasks) */ void updateLibraryCache(LibraryCacheUpdate update) throws IOException; - List replayCheckpoints(List vertexIDs) throws IOException; + CheckpointReplayResult replayCheckpoints(ExecutionVertexID vertexID) throws IOException; void propagateCheckpointDecisions(List checkpointDecisions) throws IOException; @@ -119,7 +119,7 @@ List submitTasks(List tasks) * @throws IOException * if an error occurs during this remote procedure call */ - void removeCheckpoints(List listOfVertexIDs) throws IOException; + void removeCheckpoints(ExecutionVertexID vertexID) throws IOException; /** * Triggers the task manager write the current utilization of its read and write buffers to its logs. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 8c77ea14bbafd..aee1cd8133eab 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -575,12 +575,8 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf * {@inheritDoc} */ @Override - public List replayCheckpoints(final List vertexIDs) throws IOException { - - final List checkpointResultList = new SerializableArrayList(); - - for (final ExecutionVertexID vertexID : vertexIDs) { - + public CheckpointReplayResult replayCheckpoints(final ExecutionVertexID vertexID) throws IOException { + if (!this.checkpointManager.hasCompleteCheckpointAvailable(vertexID)) { if (this.checkpointManager.hasPartialCheckpointAvailable(vertexID)) { @@ -589,25 +585,23 @@ public List replayCheckpoints(final List listOfVertexIDs) throws IOException { + public void removeCheckpoints(ExecutionVertexID vertexID) throws IOException { - final Iterator it = listOfVertexIDs.iterator(); - while (it.hasNext()) { - this.checkpointManager.removeCheckpoint(it.next()); - } + this.checkpointManager.removeCheckpoint(vertexID); + } /** From 7ebff7724b5b774bcf7b22f747b01bbb509d5b82 Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 11 Oct 2011 12:30:14 +0200 Subject: [PATCH 012/310] removed unregistration of failed task and some debug-output --- .../stratosphere/nephele/jobmanager/RecoveryThread.java | 9 --------- .../eu/stratosphere/nephele/taskmanager/TaskManager.java | 3 +-- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java index 34d4f75b64cd1..5d8f047a8d03d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java @@ -72,7 +72,6 @@ public RecoveryThread(ExecutionGraph job, JobManager jobManager) throws Exceptio */ @Override public void run() { - LOG.info("recovery running"); if (this.failedVertices.isEmpty()) { LOG.error("No failed vertices to recover"); } @@ -85,18 +84,12 @@ public void run() { LOG.info("Staring Recovery for " + failed); List restart = findRestarts(failed); - if (restart.size() < 2) { - LOG.info("No other Vertices have to be restarted?"); - } else { - LOG.info(restart.size()); - } Iterator restartIterator = restart.iterator(); while (restartIterator.hasNext()) { ExecutionVertex vertex = restartIterator.next(); if (!vertex.equals(failed)) { - LOG.info("---------------"); LOG.info("Restarting " + vertex.getName()); final List checkpointsToReplay = new ArrayList(); checkpointsToReplay.add(vertex.getID()); @@ -106,8 +99,6 @@ public void run() { } catch (Exception e) { LOG.info("Catched Exception " + StringUtils.stringifyException(e) + "wihle restarting"); } - System.out.println("restarted " + vertex.getName()); - LOG.info("---------------"); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index aee1cd8133eab..dc165322cd635 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -689,8 +689,7 @@ void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final } } - if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED - || newExecutionState == ExecutionState.FAILED) { + if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED) { // In any of these states the task's thread will be terminated, so we remove the task from the running tasks // map From 8dd6660e4b45d38c698ba661c07c0f7f1dc70eac Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 11 Oct 2011 13:14:23 +0200 Subject: [PATCH 013/310] Revert "changes TaskManager methods to IORadWritable parameters" This reverts commit a7a85e66ffb30154cf1b0c602b4b43f030425aad --- .../nephele/instance/AbstractInstance.java | 14 ++-------- .../protocols/TaskOperationProtocol.java | 4 +-- .../nephele/taskmanager/TaskManager.java | 28 ++++++++++++------- 3 files changed, 23 insertions(+), 23 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index 4b148aa8a8d0c..893b7ed2d4852 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -17,7 +17,6 @@ import java.io.IOException; import java.net.InetSocketAddress; -import java.util.Iterator; import java.util.List; import java.util.Set; @@ -40,7 +39,6 @@ import eu.stratosphere.nephele.taskmanager.TaskSubmissionWrapper; import eu.stratosphere.nephele.topology.NetworkNode; import eu.stratosphere.nephele.topology.NetworkTopology; -import eu.stratosphere.nephele.util.SerializableArrayList; /** * An abstract instance represents a resource a {@link eu.stratosphere.nephele.taskmanager.TaskManager} runs on. @@ -212,12 +210,8 @@ public synchronized List submitTasks(final List replayCheckpoints(List vertexIDs) throws IOException { - final List checkpointResultList = new SerializableArrayList(); - for (ExecutionVertexID vertexID : vertexIDs){ - checkpointResultList.add(getTaskManager().replayCheckpoints(vertexID)); - } - return checkpointResultList; + return getTaskManager().replayCheckpoints(vertexIDs); } public synchronized void propagateCheckpointDecisions(final List checkpointDecisions) @@ -251,10 +245,8 @@ public synchronized TaskCancelResult cancelTask(final ExecutionVertexID id) thro * thrown if an error occurs while transmitting the request */ public synchronized void removeCheckpoints(final List listOfVertexIDs) throws IOException { - Iterator iter = listOfVertexIDs.iterator(); - while(iter.hasNext()){ - getTaskManager().removeCheckpoints(iter.next()); - } + + getTaskManager().removeCheckpoints(listOfVertexIDs); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index 55e279213fc50..56008aef311a0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -107,7 +107,7 @@ List submitTasks(List tasks) */ void updateLibraryCache(LibraryCacheUpdate update) throws IOException; - CheckpointReplayResult replayCheckpoints(ExecutionVertexID vertexID) throws IOException; + List replayCheckpoints(List vertexIDs) throws IOException; void propagateCheckpointDecisions(List checkpointDecisions) throws IOException; @@ -119,7 +119,7 @@ List submitTasks(List tasks) * @throws IOException * if an error occurs during this remote procedure call */ - void removeCheckpoints(ExecutionVertexID vertexID) throws IOException; + void removeCheckpoints(List listOfVertexIDs) throws IOException; /** * Triggers the task manager write the current utilization of its read and write buffers to its logs. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index dc165322cd635..1b643578929fb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -575,8 +575,12 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf * {@inheritDoc} */ @Override - public CheckpointReplayResult replayCheckpoints(final ExecutionVertexID vertexID) throws IOException { - + public List replayCheckpoints(final List vertexIDs) throws IOException { + + final List checkpointResultList = new SerializableArrayList(); + + for (final ExecutionVertexID vertexID : vertexIDs) { + if (!this.checkpointManager.hasCompleteCheckpointAvailable(vertexID)) { if (this.checkpointManager.hasPartialCheckpointAvailable(vertexID)) { @@ -585,23 +589,25 @@ public CheckpointReplayResult replayCheckpoints(final ExecutionVertexID vertexID final CheckpointReplayResult result = new CheckpointReplayResult(vertexID, ReturnCode.ERROR); result .setDescription("Checkpoint is only partial and corresponding task is no longer running"); - return result; + checkpointResultList.add(result); + continue; } } } else { final CheckpointReplayResult result = new CheckpointReplayResult(vertexID, ReturnCode.ERROR); result.setDescription("No checkpoint found"); - return result; + checkpointResultList.add(result); + continue; } } // Replay the checkpoint this.checkpointManager.replayCheckpoint(vertexID); - return new CheckpointReplayResult(vertexID, ReturnCode.SUCCESS); - + checkpointResultList.add(new CheckpointReplayResult(vertexID, ReturnCode.SUCCESS)); + } - + return checkpointResultList; } /** @@ -829,10 +835,12 @@ private void checkTaskExecution() { * {@inheritDoc} */ @Override - public void removeCheckpoints(ExecutionVertexID vertexID) throws IOException { + public void removeCheckpoints(List listOfVertexIDs) throws IOException { - this.checkpointManager.removeCheckpoint(vertexID); - + final Iterator it = listOfVertexIDs.iterator(); + while (it.hasNext()) { + this.checkpointManager.removeCheckpoint(it.next()); + } } /** From 3f85151596eb76cb193cd9f93342afbf7cc16a9e Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 11 Oct 2011 23:47:32 +0200 Subject: [PATCH 014/310] fixed issues in recovery --- .../nephele/jobmanager/RecoveryThread.java | 82 ++++++++++--------- .../nephele/taskmanager/TaskManager.java | 8 +- .../ByteBufferedChannelManager.java | 6 +- 3 files changed, 53 insertions(+), 43 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java index 5d8f047a8d03d..a3fb42b9eb24f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java @@ -24,12 +24,17 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.protocols.JobManagerProtocol; +import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; +import eu.stratosphere.nephele.util.SerializableArrayList; import eu.stratosphere.nephele.util.StringUtils; /** @@ -47,7 +52,7 @@ public class RecoveryThread extends Thread { final private JobManagerProtocol jobManager; - private List globalConsistentCheckpoint = new ArrayList(); + private List globalConsistentCheckpoint = new SerializableArrayList(); /** * Initializes RecoveryThread. @@ -75,6 +80,7 @@ public void run() { if (this.failedVertices.isEmpty()) { LOG.error("No failed vertices to recover"); } + List replayCheckpoints = new ArrayList(); Iterator vertexIter = this.failedVertices.iterator(); while (vertexIter.hasNext()) { @@ -83,15 +89,17 @@ public void run() { ExecutionVertex failed = vertexIter.next(); LOG.info("Staring Recovery for " + failed); - List restart = findRestarts(failed); - + findRestarts(failed); + + /*List restart = findRestarts(failed); + Iterator restartIterator = restart.iterator(); while (restartIterator.hasNext()) { ExecutionVertex vertex = restartIterator.next(); if (!vertex.equals(failed)) { LOG.info("Restarting " + vertex.getName()); - final List checkpointsToReplay = new ArrayList(); + final SerializableArrayList checkpointsToReplay = new SerializableArrayList(); checkpointsToReplay.add(vertex.getID()); try { @@ -102,20 +110,30 @@ public void run() { } } - - //TODO: I don't think this code is needed anymore (DW) - /*Iterator checkpointIterator = this.globalConsistentCheckpoint.iterator(); + */ + + Iterator checkpointIterator = this.globalConsistentCheckpoint.iterator(); + //replay all necessary checkpoints while (checkpointIterator.hasNext()) { - ExecutionVertex checkpoint = checkpointIterator.next(); - - AbstractInstance instance = checkpoint.getAllocatedResource().getInstance(); - - instance.recoverAll(checkpoint.getEnvironment().getOutputGate(0).getOutputChannel(0).getID()); - }*/ + ExecutionVertexID checkpoint = checkpointIterator.next(); + AbstractInstance instance = job.getVertexByID(checkpoint).getAllocatedResource().getInstance(); - LOG.info("Finished recovery for " + failed); + try { + replayCheckpoints.addAll(instance.replayCheckpoints(this.globalConsistentCheckpoint)); + } catch (IOException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + } } - + for(CheckpointReplayResult replayResult : replayCheckpoints ){ + if(replayResult.getReturnCode() == ReturnCode.ERROR){ + LOG.info("Replay of Checkpoints return Error " + replayResult.getDescription() ); + //this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); + return; + } + } + this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.RERUNNING, null); LOG.info("Recovery Finsihed"); } @@ -125,61 +143,50 @@ public void run() { * @return */ private List findRestarts(ExecutionVertex failed) { - LOG.info("in findRestarts"); ArrayList restart = new ArrayList(); Queue totest = new ArrayDeque(); ArrayList visited = new ArrayList(); - totest.add(failed); - int k = 0; - LOG.info("added totest"); ExecutionVertex vertex = failed; while (!totest.isEmpty()) { - // Add all followers - - LOG.info("in while"); - if (k != 0) { - vertex = totest.peek(); - } - LOG.info("Testing " + vertex.getName()); - k++; + + vertex = totest.peek(); totest.remove(vertex); if (!restart.contains(vertex)) { restart.add(vertex); } for (int i = 0; i < vertex.getNumberOfSuccessors(); i++) { + //all successors must be restarted ExecutionVertex successor = vertex.getSuccessor(i); restart.add(successor); - LOG.info("add " + successor.getName() + " torestart"); - // totest.add(successor); if (successor.getCheckpointState() == CheckpointState.PARTIAL) { + //these tasks will be restarted, delete checkpoints this.checkpoints.remove(successor); + this.globalConsistentCheckpoint.remove(successor.getID()); } - + //all followers must be restarted List follower = findFollowers(successor, restart); restart.addAll(follower); Iterator iter = follower.iterator(); while (iter.hasNext()) { ExecutionVertex follow = iter.next(); if (!visited.contains(follow)) { - LOG.info("add totest" + follow.getName()); totest.add(follow); } } } + //predecessors must be either checkpoints or need to be restarted too for (int j = 0; j < vertex.getNumberOfPredecessors(); j++) { ExecutionVertex predecessor = vertex.getPredecessor(j); if (predecessor.getCheckpointState() != CheckpointState.PARTIAL) { - LOG.info("add " + predecessor.getName() + " torestart"); restart.add(predecessor); if (!visited.contains(predecessor)) { totest.add(predecessor); - LOG.info("add totest" + predecessor); } } else { - if (!this.globalConsistentCheckpoint.contains(predecessor)) { - this.globalConsistentCheckpoint.add(predecessor); + if (!this.globalConsistentCheckpoint.contains(predecessor.getID())) { + this.globalConsistentCheckpoint.add(predecessor.getID()); } List follower = findFollowers(predecessor, restart); @@ -215,8 +222,9 @@ private List findFollowers(ExecutionVertex vertex, ArrayList checkpointsToRemove = new ArrayList(); + + this.globalConsistentCheckpoint.remove(successor.getID()); + final SerializableArrayList checkpointsToRemove = new SerializableArrayList(); checkpointsToRemove.add(successor.getID()); try { successor.getAllocatedResource().getInstance().removeCheckpoints(checkpointsToRemove); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 1b643578929fb..05916c7d15149 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -575,9 +575,9 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf * {@inheritDoc} */ @Override - public List replayCheckpoints(final List vertexIDs) throws IOException { + public SerializableArrayList replayCheckpoints(final List vertexIDs) throws IOException { - final List checkpointResultList = new SerializableArrayList(); + final SerializableArrayList checkpointResultList = new SerializableArrayList(); for (final ExecutionVertexID vertexID : vertexIDs) { @@ -706,7 +706,9 @@ void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final // Unregister the task (free all buffers, remove all channels, task-specific class loaders, etc... unregisterTask(id, task); } - + if (newExecutionState == ExecutionState.FAILED){ + this.runningTasks.remove(id); + } // Get lock on the jobManager object and propagate the state change synchronized (this.jobManager) { try { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 23d3e83f011c7..3919883c8be30 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -383,8 +383,8 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, private boolean processEnvelopeEnvelopeWithoutBuffer(final TransferEnvelope transferEnvelope, final TransferEnvelopeReceiverList receiverList) { - System.out.println("Received envelope without buffer with event list size " - + transferEnvelope.getEventList().size()); + //System.out.println("Received envelope without buffer with event list size " + // + transferEnvelope.getEventList().size()); // No need to copy anything final Iterator localIt = receiverList.getLocalReceivers().iterator(); @@ -496,7 +496,7 @@ public void processEnvelopeFromOutputChannel(final TransferEnvelope transferEnve public void processEnvelopeFromInputChannel(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - System.out.println("Received envelope from input channel"); + //System.out.println("Received envelope from input channel"); processEnvelope(transferEnvelope, false); } From 3208230a47b4ec75840b5bdb63f36b42b3f8792c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 12 Oct 2011 15:13:38 +0200 Subject: [PATCH 015/310] Input channels now ignore envelopes which are sent more than once --- .../bytebuffered/InputChannelContext.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java index 9980c8c1b462b..dd3369be6f14f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java @@ -47,6 +47,8 @@ final class InputChannelContext implements ChannelContext, ByteBufferedInputChan private final Queue queuedEnvelopes = new ArrayDeque(); + private int lastReceivedEnvelope = -1; + InputChannelContext(final InputGateContext inputGateContext, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final AbstractByteBufferedInputChannel byteBufferedInputChannel) { @@ -155,10 +157,26 @@ public void transferEventToOutputChannel(AbstractEvent event) throws IOException * {@inheritDoc} */ @Override - public void queueTransferEnvelope(TransferEnvelope transferEnvelope) { + public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { + + // The sequence number of the envelope to be queued + final int sequenceNumber = transferEnvelope.getSequenceNumber(); synchronized (this.queuedEnvelopes) { + + if (sequenceNumber <= this.lastReceivedEnvelope) { + + final Buffer buffer = transferEnvelope.getBuffer(); + if (buffer != null) { + buffer.recycleBuffer(); + } + + return; + } + this.queuedEnvelopes.add(transferEnvelope); + + this.lastReceivedEnvelope = sequenceNumber; } // Notify the channel about the new data From c115479a69fe0316aafd01807ff63f5d7d3d6a7c Mon Sep 17 00:00:00 2001 From: marrus Date: Thu, 13 Oct 2011 14:36:26 +0200 Subject: [PATCH 016/310] changed restarts to predecessors without checkpoints. changed recovery to one call of checkpoint replay per instance. --- .../CheckpointDecisionCoordinator.java | 30 +++- .../nephele/instance/AbstractInstance.java | 14 ++ .../nephele/jobmanager/RecoveryThread.java | 130 ++++++++++-------- .../protocols/TaskOperationProtocol.java | 10 ++ .../nephele/taskmanager/TaskManager.java | 17 ++- 5 files changed, 132 insertions(+), 69 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index f98aa289416ea..0129c875603ec 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -23,6 +23,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; @@ -91,22 +92,43 @@ public void registerJob(final ExecutionGraph executionGraph) { * the current resource utilization of the vertex */ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { - + boolean checkpointDesicion = false; LOG.info("Checkpoint decision for vertex " + vertex + " required"); + // This implementation always creates the checkpoint // TODO: Provide sensible implementation here - - // This implementation always creates the checkpoint + + + Environment ee = vertex.getEnvironment(); + double in = 0; + for(int i = 0; i < ee.getNumberOfInputGates(); i++){ + in += ee.getInputGate(i).getNumberOfInputChannels(); + } + double out = 0; + for(int i = 0; i < ee.getNumberOfOutputGates(); i++){ + out += ee.getOutputGate(i).getNumberOfOutputChannels(); + } + if( out != 0 && in/out > 1.5){ + LOG.info("vertex.getNumberOfPredecessors()/vertex.getNumberOfSuccessors() > 1.5"); + //less output-channels than input-channels + //checkpoint at this position probably saves network-traffic + checkpointDesicion = true; + }else if(true){ + //always create checkpoint for testing + checkpointDesicion = true; + } + final ExecutionGraph graph = vertex.getExecutionGraph(); final Map> checkpointDecisions = new HashMap>(); final List checkpointDecisionList = new SerializableArrayList(); synchronized (graph) { - checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), true)); //Enabled checkpoints + checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), checkpointDesicion)); checkpointDecisions.put(vertex.getAllocatedResource().getInstance(), checkpointDecisionList); } // Propagate checkpoint decisions this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); } + } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index 893b7ed2d4852..2113e7fe02556 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -39,6 +39,7 @@ import eu.stratosphere.nephele.taskmanager.TaskSubmissionWrapper; import eu.stratosphere.nephele.topology.NetworkNode; import eu.stratosphere.nephele.topology.NetworkTopology; +import eu.stratosphere.nephele.util.SerializableHashSet; /** * An abstract instance represents a resource a {@link eu.stratosphere.nephele.taskmanager.TaskManager} runs on. @@ -305,4 +306,17 @@ public synchronized void killTaskManager() throws IOException { getTaskManager().killTaskManager(); } + + /** + * @param executionVertexID + * @param serializableHashSet + * @param environment + * @param configuration + * @throws IOException + * + */ + public void restartTask(final ExecutionVertexID id, final Configuration jobConfiguration, + final Environment environment, final Set activeOutputChannels) throws IOException { + getTaskManager().restartTask(id, jobConfiguration, environment, activeOutputChannels); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java index a3fb42b9eb24f..5c479f4b7c659 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java @@ -89,53 +89,66 @@ public void run() { ExecutionVertex failed = vertexIter.next(); LOG.info("Staring Recovery for " + failed); - findRestarts(failed); + //findRestarts(failed); - /*List restart = findRestarts(failed); + List restart = findRestarts(failed); + //restart all predecessors without checkpoint Iterator restartIterator = restart.iterator(); while (restartIterator.hasNext()) { ExecutionVertex vertex = restartIterator.next(); - - if (!vertex.equals(failed)) { - LOG.info("Restarting " + vertex.getName()); - final SerializableArrayList checkpointsToReplay = new SerializableArrayList(); - checkpointsToReplay.add(vertex.getID()); - + if(vertex.getID() != failed.getID()){ try { - vertex.getAllocatedResource().getInstance().replayCheckpoints(checkpointsToReplay); - } catch (Exception e) { - LOG.info("Catched Exception " + StringUtils.stringifyException(e) + "wihle restarting"); + vertex.getAllocatedResource().getInstance().restartTask(vertex.getID(),this.job.getJobConfiguration(), vertex.getEnvironment(), vertex.constructInitialActiveOutputChannelsSet() ); + } catch (IOException e) { + e.printStackTrace(); + this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); + return; } } } - */ - - Iterator checkpointIterator = this.globalConsistentCheckpoint.iterator(); - //replay all necessary checkpoints - while (checkpointIterator.hasNext()) { - ExecutionVertexID checkpoint = checkpointIterator.next(); - AbstractInstance instance = job.getVertexByID(checkpoint).getAllocatedResource().getInstance(); + //restart failed vertex + try { + failed.getAllocatedResource().getInstance().submitTask(failed.getID(), this.job.getJobConfiguration(), failed.getEnvironment(), failed.constructInitialActiveOutputChannelsSet()); + } catch (IOException e1) { + e1.printStackTrace(); + this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); + return; + } - try { - replayCheckpoints.addAll(instance.replayCheckpoints(this.globalConsistentCheckpoint)); - } catch (IOException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } + //get list of instances of consistencheckpoints + List instances = new SerializableArrayList(); + for(ExecutionVertexID id : this.globalConsistentCheckpoint){ + AbstractInstance instance = this.job.getVertexByID(id).getAllocatedResource().getInstance(); + if(!instances.contains(instance)){ + instances.add(instance); + } + } + Iterator instanceIterator = instances.iterator(); + while(instanceIterator.hasNext()){ + //replay all necessary checkpoints + try { + + replayCheckpoints.addAll(instanceIterator.next().replayCheckpoints(this.globalConsistentCheckpoint)); + } catch (IOException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } } + + } for(CheckpointReplayResult replayResult : replayCheckpoints ){ if(replayResult.getReturnCode() == ReturnCode.ERROR){ LOG.info("Replay of Checkpoints return Error " + replayResult.getDescription() ); - //this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); + this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); return; } } this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.RERUNNING, null); - LOG.info("Recovery Finsihed"); + LOG.info("Recovery Finished"); } /** @@ -155,26 +168,26 @@ private List findRestarts(ExecutionVertex failed) { if (!restart.contains(vertex)) { restart.add(vertex); } - for (int i = 0; i < vertex.getNumberOfSuccessors(); i++) { - //all successors must be restarted - ExecutionVertex successor = vertex.getSuccessor(i); - restart.add(successor); - if (successor.getCheckpointState() == CheckpointState.PARTIAL) { - //these tasks will be restarted, delete checkpoints - this.checkpoints.remove(successor); - this.globalConsistentCheckpoint.remove(successor.getID()); - } - //all followers must be restarted - List follower = findFollowers(successor, restart); - restart.addAll(follower); - Iterator iter = follower.iterator(); - while (iter.hasNext()) { - ExecutionVertex follow = iter.next(); - if (!visited.contains(follow)) { - totest.add(follow); - } - } - } +// for (int i = 0; i < vertex.getNumberOfSuccessors(); i++) { +// //all successors must be restarted +// ExecutionVertex successor = vertex.getSuccessor(i); +// restart.add(successor); +// if (successor.getCheckpointState() == CheckpointState.PARTIAL) { +// //these tasks will be restarted, delete checkpoints +// this.checkpoints.remove(successor); +// this.globalConsistentCheckpoint.remove(successor.getID()); +// } +// //all followers must be restarted +// List follower = findFollowers(successor, restart); +// restart.addAll(follower); +// Iterator iter = follower.iterator(); +// while (iter.hasNext()) { +// ExecutionVertex follow = iter.next(); +// if (!visited.contains(follow)) { +// totest.add(follow); +// } +// } +// } //predecessors must be either checkpoints or need to be restarted too for (int j = 0; j < vertex.getNumberOfPredecessors(); j++) { ExecutionVertex predecessor = vertex.getPredecessor(j); @@ -189,27 +202,22 @@ private List findRestarts(ExecutionVertex failed) { this.globalConsistentCheckpoint.add(predecessor.getID()); } - List follower = findFollowers(predecessor, restart); - for (int i = 0; i < follower.size(); i++) { - LOG.info("add " + follower.get(i) + " torestart"); - } - restart.addAll(follower); - Iterator iter = follower.iterator(); - while (iter.hasNext()) { - ExecutionVertex follow = iter.next(); - if (!visited.contains(follow)) { - LOG.info("add totest" + follow.getName()); - totest.add(follow); - } - } + +// List follower = findFollowers(predecessor, restart); +// restart.addAll(follower); +// Iterator iter = follower.iterator(); +// while (iter.hasNext()) { +// ExecutionVertex follow = iter.next(); +// if (!visited.contains(follow)) { +// totest.add(follow); +// } +// } } } visited.add(vertex); } - LOG.info("finderestartsfinished"); - return restart; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index 56008aef311a0..f67b443a98b83 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -137,4 +137,14 @@ List submitTasks(List tasks) * throws if an error occurs during this remote procedure call */ void killTaskManager() throws IOException; + + /** + * Restarts a given Task by unregistering an submitting it + * @param executionVertexID + * @param activeOutputChannels + * @param environment + * @param jobConfiguration + * @throws IOException + */ + void restartTask(ExecutionVertexID executionVertexID, Configuration jobConfiguration, Environment environment, Set activeOutputChannels) throws IOException; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 05916c7d15149..23210d3916c97 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -695,7 +695,7 @@ void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final } } - if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED) { + if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED ||newExecutionState == ExecutionState.FAILED) { // In any of these states the task's thread will be terminated, so we remove the task from the running tasks // map @@ -706,9 +706,6 @@ void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final // Unregister the task (free all buffers, remove all channels, task-specific class loaders, etc... unregisterTask(id, task); } - if (newExecutionState == ExecutionState.FAILED){ - this.runningTasks.remove(id); - } // Get lock on the jobManager object and propagate the state change synchronized (this.jobManager) { try { @@ -882,4 +879,16 @@ public void run() { timer.schedule(timerTask, 10L); } + + /** + * {@inheritDoc} + * @throws IOException + */ + @Override + public void restartTask(ExecutionVertexID id, Configuration jobConfiguration, + Environment environment, Set activeOutputChannels) throws IOException { + unregisterTask(id, this.runningTasks.get(id)); + submitTask(id, jobConfiguration, environment, activeOutputChannels); + + } } From fa95c3d5ac5cd03c51f040544b3ed4c335dea59c Mon Sep 17 00:00:00 2001 From: marrus Date: Thu, 13 Oct 2011 14:37:44 +0200 Subject: [PATCH 017/310] fixed taskmanager mock --- .../java/eu/stratosphere/pact/testing/MockTaskManager.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pact/pact-clients/src/main/java/eu/stratosphere/pact/testing/MockTaskManager.java b/pact/pact-clients/src/main/java/eu/stratosphere/pact/testing/MockTaskManager.java index 75fea679e298a..0958871091d17 100644 --- a/pact/pact-clients/src/main/java/eu/stratosphere/pact/testing/MockTaskManager.java +++ b/pact/pact-clients/src/main/java/eu/stratosphere/pact/testing/MockTaskManager.java @@ -196,4 +196,9 @@ public void killTaskManager() throws IOException { // TODO Auto-generated method stub } + + @Override + public void restartTask(ExecutionVertexID executionVertexID, Configuration jobConfiguration, + Environment environment, Set activeOutputChannels) throws IOException { + } } From 2b7e30cb70cb3a1c1c9c2e9182ec521c03a02bdb Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Thu, 13 Oct 2011 13:58:23 +0000 Subject: [PATCH 018/310] Increased robustness of envelope routing in presence of execution failures --- .../taskmanager/bytebuffered/ByteBufferedChannelManager.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 3919883c8be30..c138bbc6aabca 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -349,7 +349,9 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, final ChannelContext cc = this.registeredChannels.get(localReceiver); if (cc == null) { - throw new IOException("Cannot find channel context for local receiver " + localReceiver); + //TODO: Make this more robust, for example by introducing a separate map for failed channel IDs. + LOG.warn("Cannot find channel context for local receiver " + localReceiver); + continue; } if (!cc.isInputChannel()) { From c08b74f1fb8aafb5f275f1a600f4eac0b1372a2c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 13 Oct 2011 14:56:39 +0000 Subject: [PATCH 019/310] Replay task now ignores partial checkpoint files which are still written at the same time --- .../CheckpointReplayManager.java | 9 ++++- .../checkpointing/CheckpointReplayTask.java | 2 +- .../checkpointing/EphemeralCheckpoint.java | 37 +++++++++++++++---- 3 files changed, 37 insertions(+), 11 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java index 70d9445ed021e..6a09f10a47d3e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java @@ -62,7 +62,12 @@ public boolean hasCompleteCheckpointAvailable(final ExecutionVertexID vertexID) public boolean hasPartialCheckpointAvailable(final ExecutionVertexID vertexID) { - final File file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_0"); + File file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_0"); + if (file.exists()) { + return true; + } + + file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_part"); if (file.exists()) { return true; } @@ -74,7 +79,7 @@ public void replayCheckpoint(final ExecutionVertexID vertexID) { final CheckpointReplayTask replayTask = new CheckpointReplayTask(vertexID, this.checkpointDirectory, this.transferEnvelopeDispatcher, hasCompleteCheckpointAvailable(vertexID)); - + replayTask.start(); LOG.info("Replaying checkpoint for vertex " + vertexID); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java index c84cffb38bdc3..cf1fe02feb682 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java @@ -84,7 +84,7 @@ private void replayCheckpoint() throws IOException, InterruptedException { return; } - if (metaDataIndex == 0 || this.isCheckpointComplete) { + if (this.isCheckpointComplete) { throw new FileNotFoundException("Cannot find meta data file " + metaDataIndex + " for checkpoint of vertex " + this.vertexID); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 998941f0cf5a4..692c11bf6a1fe 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -215,9 +215,31 @@ private void write() throws IOException, InterruptedException { } } + private boolean renameCheckpointPart(final String checkpointDir) { + + final File oldFile = new File(checkpointDir + File.separator + + CheckpointReplayManager.METADATA_PREFIX + "_" + + this.task.getVertexID() + "_part"); + + final File newFile = new File(checkpointDir + File.separator + + CheckpointReplayManager.METADATA_PREFIX + "_" + + this.task.getVertexID() + "_" + this.metaDataSuffix); + + if (!oldFile.renameTo(newFile)) { + LOG.error("Unable to rename " + oldFile.getAbsoluteFile() + " to " + newFile.getAbsoluteFile()); + return false; + } + + return true; + } + private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + final String checkpointDir = GlobalConfiguration.getString( + CheckpointReplayManager.CHECKPOINT_DIRECTORY_KEY, + CheckpointReplayManager.DEFAULT_CHECKPOINT_DIRECTORY); + final Buffer buffer = transferEnvelope.getBuffer(); if (buffer != null) { if (buffer.isBackedByMemory()) { @@ -238,6 +260,9 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro this.metaDataFileChannel.close(); this.metaDataFileChannel = null; + // Rename file + renameCheckpointPart(checkpointDir); + // Increase the meta data suffix ++this.metaDataSuffix; } @@ -245,15 +270,12 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro if (this.metaDataFileChannel == null) { - final String checkpointDir = GlobalConfiguration.getString( - CheckpointReplayManager.CHECKPOINT_DIRECTORY_KEY, - CheckpointReplayManager.DEFAULT_CHECKPOINT_DIRECTORY); if (LOG.isDebugEnabled()) { LOG.debug("Writing checkpointing meta data to directory " + checkpointDir); } final FileOutputStream fos = new FileOutputStream(checkpointDir + File.separator + CheckpointReplayManager.METADATA_PREFIX - + "_" + this.task.getVertexID() + "_" + this.metaDataSuffix); + + "_" + this.task.getVertexID() + "_part"); this.metaDataFileChannel = fos.getChannel(); } @@ -279,11 +301,10 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro if (this.metaDataFileChannel != null) { this.metaDataFileChannel.close(); - } - final String checkpointDir = GlobalConfiguration.getString( - CheckpointReplayManager.CHECKPOINT_DIRECTORY_KEY, - CheckpointReplayManager.DEFAULT_CHECKPOINT_DIRECTORY); + // Rename file + renameCheckpointPart(checkpointDir); + } new FileOutputStream(checkpointDir + File.separator + CheckpointReplayManager.METADATA_PREFIX + "_" + this.task.getVertexID() + "_final").close(); From cf9b370a6100386ef7c9535c85419f6ede808f23 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 13 Oct 2011 19:18:30 +0000 Subject: [PATCH 020/310] Fixed bug in sequence number check --- .../nephele/taskmanager/bytebuffered/InputChannelContext.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java index dd3369be6f14f..c7dc60ca23f04 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java @@ -164,7 +164,7 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { synchronized (this.queuedEnvelopes) { - if (sequenceNumber <= this.lastReceivedEnvelope) { + if (sequenceNumber != (this.lastReceivedEnvelope + 1)) { final Buffer buffer = transferEnvelope.getBuffer(); if (buffer != null) { From 0a4a043263427020078d1d5a765bbf5664518358 Mon Sep 17 00:00:00 2001 From: marrus Date: Fri, 14 Oct 2011 13:58:53 +0200 Subject: [PATCH 021/310] added User CPU to Utilization snapshot an checkpoint decision --- .../CheckpointDecisionCoordinator.java | 42 ++++++++++--------- .../ResourceUtilizationSnapshot.java | 18 +++++++- .../nephele/taskmanager/Task.java | 20 +++++++-- 3 files changed, 56 insertions(+), 24 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index 0129c875603ec..e0933e85325e8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -97,27 +97,31 @@ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtil // This implementation always creates the checkpoint // TODO: Provide sensible implementation here - - - Environment ee = vertex.getEnvironment(); - double in = 0; - for(int i = 0; i < ee.getNumberOfInputGates(); i++){ - in += ee.getInputGate(i).getNumberOfInputChannels(); - } - double out = 0; - for(int i = 0; i < ee.getNumberOfOutputGates(); i++){ - out += ee.getOutputGate(i).getNumberOfOutputChannels(); - } - if( out != 0 && in/out > 1.5){ - LOG.info("vertex.getNumberOfPredecessors()/vertex.getNumberOfSuccessors() > 1.5"); - //less output-channels than input-channels - //checkpoint at this position probably saves network-traffic - checkpointDesicion = true; - }else if(true){ - //always create checkpoint for testing + + if(rus.getUserCPU() >= 90){ + LOG.info("CPU-Bottleneck"); + //CPU bottleneck checkpointDesicion = true; + }else{ + Environment ee = vertex.getEnvironment(); + double in = 0; + for(int i = 0; i < ee.getNumberOfInputGates(); i++){ + in += ee.getInputGate(i).getNumberOfInputChannels(); + } + double out = 0; + for(int i = 0; i < ee.getNumberOfOutputGates(); i++){ + out += ee.getOutputGate(i).getNumberOfOutputChannels(); + } + if( out != 0 && in/out > 1.5){ + LOG.info("vertex.getNumberOfPredecessors()/vertex.getNumberOfSuccessors() > 1.5"); + //less output-channels than input-channels + //checkpoint at this position probably saves network-traffic + checkpointDesicion = true; + }else if(true){ + //always create checkpoint for testing + checkpointDesicion = true; + } } - final ExecutionGraph graph = vertex.getExecutionGraph(); final Map> checkpointDecisions = new HashMap>(); final List checkpointDecisionList = new SerializableArrayList(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java index 0fa5f80a7af6b..00b892debb776 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java @@ -41,8 +41,13 @@ public final class ResourceUtilizationSnapshot implements IOReadableWritable { * Stores the utilization of task's output channels at the time when the snapshot was created. */ private final Map channelUtilization; + + /** + * userCPu Time in percent + */ + private long userCPU; - public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization) { + public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU) { if (timestamp <= 0L) { throw new IllegalArgumentException("Argument timestamp must be larger than zero"); @@ -54,6 +59,7 @@ public ResourceUtilizationSnapshot(final long timestamp, final Map registeredListeners = new ConcurrentLinkedQueue(); + private long startTime; + Task(final ExecutionVertexID vertexID, final Environment environment, final TaskManager taskManager) { this.vertexID = vertexID; @@ -104,8 +110,8 @@ public void executionStateChanged(final ExecutionState newExecutionState, final */ private String getTaskName() { - return this.environment.getTaskName() + " (" + (environment.getIndexInSubtaskGroup() + 1) + "/" - + environment.getCurrentNumberOfSubtasks() + ")"; + return this.environment.getTaskName() + " (" + (this.environment.getIndexInSubtaskGroup() + 1) + "/" + + this.environment.getCurrentNumberOfSubtasks() + ")"; } /** @@ -218,6 +224,7 @@ public void startExecution() { final Thread thread = this.environment.getExecutingThread(); thread.start(); + this.startTime = System.currentTimeMillis(); } /** @@ -241,6 +248,11 @@ public void initialExecutionResourcesExhausted() { // Construct a resource utilization snapshot final long timestamp = System.currentTimeMillis(); + //Get CPU-Usertime in percent + ThreadMXBean threadBean = ManagementFactory.getThreadMXBean(); + long userCPU = (threadBean.getCurrentThreadUserTime()/NANO_TO_MILLISECONDS) * 100 / (timestamp - this.startTime); + + //collect outputChannelUtilization final Map outputChannelUtilization = new HashMap(); for (int i = 0; i < this.environment.getNumberOfOutputGates(); ++i) { @@ -251,8 +263,8 @@ public void initialExecutionResourcesExhausted() { Long.valueOf(outputChannel.getAmountOfDataTransmitted())); } } - - final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, outputChannelUtilization); + + final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, outputChannelUtilization, userCPU); // Notify the listener objects final Iterator it = this.registeredListeners.iterator(); From 74939e9846e397d5ae320334200ab968be8ad42b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 14 Oct 2011 15:54:55 +0000 Subject: [PATCH 022/310] Added two more integration tests for the fault tolerance features --- .../checkpointing/FailingJobITCase.java | 179 +++++++++++++++--- 1 file changed, 156 insertions(+), 23 deletions(-) diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java index 8844b3fa15c67..a61ec85a5a7dd 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java @@ -75,25 +75,15 @@ public class FailingJobITCase { private static final int RECORD_SIZE = 256; /** - * Number of records after which the execution failure shall occur. + * Configuration key to access the number of records after which the execution failure shall occur. */ - private static final int FAILED_AFTER_RECORD = 95490; + private static final String FAILED_AFTER_RECORD_KEY = "failure.after.record"; /** * The degree of parallelism for the job. */ private static final int DEGREE_OF_PARALLELISM = 4; - /** - * Index of the parallel subtask in which the failure shall occur. - */ - private static final int FAILURE_INDEX = 0; - - /** - * The key to access the configuration flag for a task failure. - */ - private static final String FAILURE_KEY = "failure"; - /** * The key to access the index of the subtask which is supposed to fail. */ @@ -112,8 +102,8 @@ public class FailingJobITCase { /** * Global flag to indicate if a task has already failed once. */ - private static final AtomicBoolean FAILED_ONCE = new AtomicBoolean(false); - + private static final AtomicBoolean FAILED_ONCE = new AtomicBoolean(false); + /** * This is an auxiliary class to run the job manager thread. * @@ -288,9 +278,20 @@ public void registerInputOutput() { @Override public void invoke() throws Exception { + final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + final boolean failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( + FAILURE_INDEX_KEY, -1)); + final FailingJobRecord record = new FailingJobRecord(); for (int i = 0; i < RECORDS_TO_GENERATE; ++i) { this.recordWriter.emit(record); + + if (i == failAfterRecord && failing) { + if (FAILED_ONCE.compareAndSet(false, true)) { + throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " + + getIndexInSubtaskGroup()); + } + } } } } @@ -320,14 +321,16 @@ public void invoke() throws Exception { final FailingJobRecord record = new FailingJobRecord(); + final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + int count = 0; - boolean failing = (getRuntimeConfiguration().getBoolean(FAILURE_KEY, false) && (getIndexInSubtaskGroup() == getRuntimeConfiguration() - .getInteger(FAILURE_INDEX_KEY, -1))); + final boolean failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( + FAILURE_INDEX_KEY, -1)); while (this.recordReader.next(record)) { this.recordWriter.emit(record); - if (count++ == FAILED_AFTER_RECORD && failing) { - if(FAILED_ONCE.compareAndSet(false, true)) { + if (count++ == failAfterRecord && failing) { + if (FAILED_ONCE.compareAndSet(false, true)) { throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " + getIndexInSubtaskGroup()); } @@ -347,7 +350,7 @@ public static final class OutputTask extends AbstractOutputTask { public void registerInputOutput() { this.recordReader = new MutableRecordReader(this, - new BipartiteDistributionPattern()); + new BipartiteDistributionPattern()); } /** @@ -356,9 +359,21 @@ public void registerInputOutput() { @Override public void invoke() throws Exception { + final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + + int count = 0; + final boolean failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( + FAILURE_INDEX_KEY, -1)); + final FailingJobRecord record = new FailingJobRecord(); while (this.recordReader.next(record)) { - // Simply consume the records + + if (count++ == failAfterRecord && failing) { + if (FAILED_ONCE.compareAndSet(false, true)) { + throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " + + getIndexInSubtaskGroup()); + } + } } } @@ -386,8 +401,126 @@ public void testFailingInternalVertex() { innerVertex2.setTaskClass(InnerTask.class); innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); - innerVertex2.getConfiguration().setBoolean(FAILURE_KEY, true); - innerVertex2.getConfiguration().setInteger(FAILURE_INDEX_KEY, FAILURE_INDEX); + innerVertex2.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 95490); + innerVertex2.getConfiguration().setInteger(FAILURE_INDEX_KEY, 0); + + final JobTaskVertex innerVertex3 = new JobTaskVertex("Inner vertex 3", jobGraph); + innerVertex3.setTaskClass(InnerTask.class); + innerVertex3.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex3.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobGenericOutputVertex output = new JobGenericOutputVertex("Output", jobGraph); + output.setOutputClass(OutputTask.class); + output.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + output.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + // Configure instance sharing + innerVertex1.setVertexToShareInstancesWith(input); + innerVertex2.setVertexToShareInstancesWith(input); + innerVertex3.setVertexToShareInstancesWith(input); + output.setVertexToShareInstancesWith(input); + + try { + + input.connectTo(innerVertex1, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex1.connectTo(innerVertex2, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex2.connectTo(innerVertex3, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex3.connectTo(output, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + + } catch (JobGraphDefinitionException e) { + fail(StringUtils.stringifyException(e)); + } + + // Reset the FAILED_ONCE flag + FAILED_ONCE.set(false); + + // Create job client and launch job + try { + JobClient jobClient = new JobClient(jobGraph, configuration); + jobClient.submitJobAndWait(); + } catch (IOException ioe) { + fail(StringUtils.stringifyException(ioe)); + } catch (JobExecutionException e) { + fail(StringUtils.stringifyException(e)); + } + } + + /** + * This test checks Nephele's capabilities to recover from file channel checkpoints. + */ + @Test + public void testRecoveryFromFileChannels() { + + final JobGraph jobGraph = new JobGraph("Job with file channels"); + + final JobGenericInputVertex input = new JobGenericInputVertex("Input", jobGraph); + input.setInputClass(InputTask.class); + input.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + input.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobTaskVertex innerVertex1 = new JobTaskVertex("Inner vertex 1", jobGraph); + innerVertex1.setTaskClass(InnerTask.class); + innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobGenericOutputVertex output = new JobGenericOutputVertex("Output", jobGraph); + output.setOutputClass(OutputTask.class); + output.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + output.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + output.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 153201); + output.getConfiguration().setInteger(FAILURE_INDEX_KEY, 1); + + // Configure instance sharing + innerVertex1.setVertexToShareInstancesWith(input); + output.setVertexToShareInstancesWith(input); + + try { + + input.connectTo(innerVertex1, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex1.connectTo(output, ChannelType.FILE, CompressionLevel.NO_COMPRESSION); + + } catch (JobGraphDefinitionException e) { + fail(StringUtils.stringifyException(e)); + } + + // Reset the FAILED_ONCE flag + FAILED_ONCE.set(false); + + // Create job client and launch job + try { + JobClient jobClient = new JobClient(jobGraph, configuration); + jobClient.submitJobAndWait(); + } catch (IOException ioe) { + fail(StringUtils.stringifyException(ioe)); + } catch (JobExecutionException e) { + fail(StringUtils.stringifyException(e)); + } + } + + /** + * This test checks Nephele's fault tolerance capabilities by simulating a failing input vertex. + */ + @Test + public void testFailingInputVertex() { + + final JobGraph jobGraph = new JobGraph("Job with failing inner vertex"); + + final JobGenericInputVertex input = new JobGenericInputVertex("Input", jobGraph); + input.setInputClass(InputTask.class); + input.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + input.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + input.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 804937); + input.getConfiguration().setInteger(FAILURE_INDEX_KEY, 3); + + final JobTaskVertex innerVertex1 = new JobTaskVertex("Inner vertex 1", jobGraph); + innerVertex1.setTaskClass(InnerTask.class); + innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobTaskVertex innerVertex2 = new JobTaskVertex("Inner vertex 2", jobGraph); + innerVertex2.setTaskClass(InnerTask.class); + innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); final JobTaskVertex innerVertex3 = new JobTaskVertex("Inner vertex 3", jobGraph); innerVertex3.setTaskClass(InnerTask.class); @@ -418,7 +551,7 @@ public void testFailingInternalVertex() { // Reset the FAILED_ONCE flag FAILED_ONCE.set(false); - + // Create job client and launch job try { JobClient jobClient = new JobClient(jobGraph, configuration); From c1bb8d73429c16c36166e4de9b47c219c8bead89 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 14 Oct 2011 16:11:55 +0000 Subject: [PATCH 023/310] Added another integration test to check Nephele's fault tolerance capabilities --- .../checkpointing/FailingJobITCase.java | 140 ++++++++++++++---- 1 file changed, 114 insertions(+), 26 deletions(-) diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java index a61ec85a5a7dd..48b824c377677 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java @@ -23,7 +23,8 @@ import java.io.IOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.HashSet; +import java.util.Set; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -102,7 +103,7 @@ public class FailingJobITCase { /** * Global flag to indicate if a task has already failed once. */ - private static final AtomicBoolean FAILED_ONCE = new AtomicBoolean(false); + private static final Set FAILED_ONCE = new HashSet(); /** * This is an auxiliary class to run the job manager thread. @@ -278,19 +279,21 @@ public void registerInputOutput() { @Override public void invoke() throws Exception { + boolean failing = false; + final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); - final boolean failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( - FAILURE_INDEX_KEY, -1)); + synchronized (FAILED_ONCE) { + failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( + FAILURE_INDEX_KEY, -1)) && FAILED_ONCE.add(getEnvironment().getTaskName()); + } final FailingJobRecord record = new FailingJobRecord(); for (int i = 0; i < RECORDS_TO_GENERATE; ++i) { this.recordWriter.emit(record); if (i == failAfterRecord && failing) { - if (FAILED_ONCE.compareAndSet(false, true)) { - throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " - + getIndexInSubtaskGroup()); - } + throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " + + getIndexInSubtaskGroup()); } } } @@ -321,19 +324,22 @@ public void invoke() throws Exception { final FailingJobRecord record = new FailingJobRecord(); + boolean failing = false; + final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + synchronized (FAILED_ONCE) { + failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( + FAILURE_INDEX_KEY, -1)) && FAILED_ONCE.add(getEnvironment().getTaskName()); + } int count = 0; - final boolean failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( - FAILURE_INDEX_KEY, -1)); + while (this.recordReader.next(record)) { this.recordWriter.emit(record); if (count++ == failAfterRecord && failing) { - if (FAILED_ONCE.compareAndSet(false, true)) { - throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " - + getIndexInSubtaskGroup()); - } + throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " + + getIndexInSubtaskGroup()); } } } @@ -359,20 +365,23 @@ public void registerInputOutput() { @Override public void invoke() throws Exception { + boolean failing = false; + final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + synchronized (FAILED_ONCE) { + failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( + FAILURE_INDEX_KEY, -1)) && FAILED_ONCE.add(getEnvironment().getTaskName()); + } int count = 0; - final boolean failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( - FAILURE_INDEX_KEY, -1)); final FailingJobRecord record = new FailingJobRecord(); while (this.recordReader.next(record)) { if (count++ == failAfterRecord && failing) { - if (FAILED_ONCE.compareAndSet(false, true)) { - throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " - + getIndexInSubtaskGroup()); - } + + throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " + + getIndexInSubtaskGroup()); } } } @@ -431,8 +440,10 @@ public void testFailingInternalVertex() { fail(StringUtils.stringifyException(e)); } - // Reset the FAILED_ONCE flag - FAILED_ONCE.set(false); + // Reset the FAILED_ONCE flags + synchronized (FAILED_ONCE) { + FAILED_ONCE.clear(); + } // Create job client and launch job try { @@ -483,8 +494,10 @@ public void testRecoveryFromFileChannels() { fail(StringUtils.stringifyException(e)); } - // Reset the FAILED_ONCE flag - FAILED_ONCE.set(false); + // Reset the FAILED_ONCE flags + synchronized (FAILED_ONCE) { + FAILED_ONCE.clear(); + } // Create job client and launch job try { @@ -549,8 +562,10 @@ public void testFailingInputVertex() { fail(StringUtils.stringifyException(e)); } - // Reset the FAILED_ONCE flag - FAILED_ONCE.set(false); + // Reset the FAILED_ONCE flags + synchronized (FAILED_ONCE) { + FAILED_ONCE.clear(); + } // Create job client and launch job try { @@ -562,4 +577,77 @@ public void testFailingInputVertex() { fail(StringUtils.stringifyException(e)); } } + + /** + * This test checks Nephele's fault tolerance capabilities by simulating a successively failing inner vertices. In + * particular, the test covers the situation when a vertex fails whose checkpoint is currently used for recovery + * itself. + */ + @Test + public void testSuccessivelyFailingInnerVertices() { + + final JobGraph jobGraph = new JobGraph("Job with failing inner vertex"); + + final JobGenericInputVertex input = new JobGenericInputVertex("Input", jobGraph); + input.setInputClass(InputTask.class); + input.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + input.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobTaskVertex innerVertex1 = new JobTaskVertex("Inner vertex 1", jobGraph); + innerVertex1.setTaskClass(InnerTask.class); + innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + innerVertex1.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 145613); + innerVertex1.getConfiguration().setInteger(FAILURE_INDEX_KEY, 2); + + final JobTaskVertex innerVertex2 = new JobTaskVertex("Inner vertex 2", jobGraph); + innerVertex2.setTaskClass(InnerTask.class); + innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + innerVertex2.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 95490); + innerVertex2.getConfiguration().setInteger(FAILURE_INDEX_KEY, 0); + + final JobTaskVertex innerVertex3 = new JobTaskVertex("Inner vertex 3", jobGraph); + innerVertex3.setTaskClass(InnerTask.class); + innerVertex3.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex3.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobGenericOutputVertex output = new JobGenericOutputVertex("Output", jobGraph); + output.setOutputClass(OutputTask.class); + output.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + output.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + // Configure instance sharing + innerVertex1.setVertexToShareInstancesWith(input); + innerVertex2.setVertexToShareInstancesWith(input); + innerVertex3.setVertexToShareInstancesWith(input); + output.setVertexToShareInstancesWith(input); + + try { + + input.connectTo(innerVertex1, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex1.connectTo(innerVertex2, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex2.connectTo(innerVertex3, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex3.connectTo(output, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + + } catch (JobGraphDefinitionException e) { + fail(StringUtils.stringifyException(e)); + } + + // Reset the FAILED_ONCE flags + synchronized (FAILED_ONCE) { + FAILED_ONCE.clear(); + } + + // Create job client and launch job + try { + JobClient jobClient = new JobClient(jobGraph, configuration); + jobClient.submitJobAndWait(); + } catch (IOException ioe) { + fail(StringUtils.stringifyException(ioe)); + } catch (JobExecutionException e) { + fail(StringUtils.stringifyException(e)); + } + + } } From f55a4883d166d0d4863a8a3bc4dc6936e2a54db9 Mon Sep 17 00:00:00 2001 From: marrus Date: Mon, 17 Oct 2011 11:01:53 +0200 Subject: [PATCH 024/310] changed recovery to recover from completed checkpoints --- .../checkpointing/CheckpointDecisionCoordinator.java | 12 ++---------- .../nephele/jobmanager/RecoveryThread.java | 2 +- 2 files changed, 3 insertions(+), 11 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index e0933e85325e8..0f02adc1504d2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -103,16 +103,8 @@ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtil //CPU bottleneck checkpointDesicion = true; }else{ - Environment ee = vertex.getEnvironment(); - double in = 0; - for(int i = 0; i < ee.getNumberOfInputGates(); i++){ - in += ee.getInputGate(i).getNumberOfInputChannels(); - } - double out = 0; - for(int i = 0; i < ee.getNumberOfOutputGates(); i++){ - out += ee.getOutputGate(i).getNumberOfOutputChannels(); - } - if( out != 0 && in/out > 1.5){ + + if( vertex.getNumberOfSuccessors() != 0 && vertex.getNumberOfPredecessors()*1.0/vertex.getNumberOfSuccessors() > 1.5){ LOG.info("vertex.getNumberOfPredecessors()/vertex.getNumberOfSuccessors() > 1.5"); //less output-channels than input-channels //checkpoint at this position probably saves network-traffic diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java index 5c479f4b7c659..1c900d91d5d67 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java @@ -191,7 +191,7 @@ private List findRestarts(ExecutionVertex failed) { //predecessors must be either checkpoints or need to be restarted too for (int j = 0; j < vertex.getNumberOfPredecessors(); j++) { ExecutionVertex predecessor = vertex.getPredecessor(j); - if (predecessor.getCheckpointState() != CheckpointState.PARTIAL) { + if (predecessor.getCheckpointState() != CheckpointState.PARTIAL && predecessor.getCheckpointState() != CheckpointState.COMPLETE) { restart.add(predecessor); if (!visited.contains(predecessor)) { From eba6516c46b7940f7f48bbcce2ad352310f75944 Mon Sep 17 00:00:00 2001 From: marrus Date: Mon, 17 Oct 2011 18:29:36 +0200 Subject: [PATCH 025/310] fixed recovery for successively failing tasks --- .../executiongraph/ExecutionGraph.java | 15 +++-- .../nephele/jobmanager/RecoveryThread.java | 55 ++++++++++++------- 2 files changed, 45 insertions(+), 25 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index 14a9c93f7e1a8..dac38b4343970 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -1242,8 +1242,13 @@ public synchronized void checkAndUpdateJobStatus(final ExecutionState latestStat break; case RECOVERING: if (latestStateChange == ExecutionState.RERUNNING) { - this.recovering.clear(); - this.jobStatus = InternalJobStatus.RUNNING; + if(this.recovering.isEmpty()){ + this.jobStatus = InternalJobStatus.RUNNING; + break; + } + } + if (latestStateChange == ExecutionState.FAILED){ + LOG.info("Another Failed Vertex while recovering"); } break; case FAILING: @@ -1308,10 +1313,12 @@ public synchronized void executionStateChanged(final JobID jobID, final Executio } } } - if (this.jobStatus == InternalJobStatus.RECOVERING){ + if (newExecutionState == ExecutionState.FAILED && this.jobStatus == InternalJobStatus.RECOVERING){ LOG.info("RECOVERING"); //FIXME (marrus) see if we even need that - this.recovering.add(this.getVertexByID(vertexID)); + if(!this.recovering.contains(vertexID)){ + this.recovering.add(this.getVertexByID(vertexID)); + } } if (this.jobStatus != oldStatus) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java index 1c900d91d5d67..16f34552a8f4b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java @@ -46,7 +46,7 @@ public class RecoveryThread extends Thread { final ExecutionGraph job; - final List failedVertices; + List failedVertices; private List checkpoints; @@ -62,7 +62,7 @@ public class RecoveryThread extends Thread { * @throws Exception */ public RecoveryThread(ExecutionGraph job, JobManager jobManager) throws Exception { - super(); + super("Recovery Thread"); this.job = job; this.jobManager = jobManager; this.failedVertices = new ArrayList(); @@ -80,18 +80,27 @@ public void run() { if (this.failedVertices.isEmpty()) { LOG.error("No failed vertices to recover"); } - List replayCheckpoints = new ArrayList(); - Iterator vertexIter = this.failedVertices.iterator(); + //FIXME (marrus) dirty fix + while(!this.failedVertices.isEmpty()){ + recover(this.failedVertices.remove(0)); + if(this.failedVertices.isEmpty()){ + this.failedVertices = this.job.getFailedVertices(); + } + } + LOG.info("Recovery Finished"); + } - while (vertexIter.hasNext()) { + /** + * + */ + private boolean recover(ExecutionVertex failed) { + List replayCheckpoints = new ArrayList(); - // ArrayList channels = new ArrayList(); - ExecutionVertex failed = vertexIter.next(); LOG.info("Staring Recovery for " + failed); //findRestarts(failed); - List restart = findRestarts(failed); + final List restart = findRestarts(failed); //restart all predecessors without checkpoint Iterator restartIterator = restart.iterator(); @@ -103,7 +112,7 @@ public void run() { } catch (IOException e) { e.printStackTrace(); this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); - return; + return false; } } @@ -114,7 +123,7 @@ public void run() { } catch (IOException e1) { e1.printStackTrace(); this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); - return; + return false; } //get list of instances of consistencheckpoints @@ -132,23 +141,27 @@ public void run() { replayCheckpoints.addAll(instanceIterator.next().replayCheckpoints(this.globalConsistentCheckpoint)); } catch (IOException e) { - // TODO Auto-generated catch block e.printStackTrace(); + this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); + return false; } } - - } - for(CheckpointReplayResult replayResult : replayCheckpoints ){ - if(replayResult.getReturnCode() == ReturnCode.ERROR){ - LOG.info("Replay of Checkpoints return Error " + replayResult.getDescription() ); - this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); - return; + for(CheckpointReplayResult replayResult : replayCheckpoints ){ + if(replayResult.getReturnCode() == ReturnCode.ERROR){ + LOG.info("Replay of Checkpoints return Error " + replayResult.getDescription() ); + this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); + return false; + } } - } + LOG.info("FINISHED RECOVERY for " + failed.getName()); + this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.RERUNNING, null); - this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.RERUNNING, null); - LOG.info("Recovery Finished"); + + + + + return true; } /** From 5f40ee90c1fd2d8b52a4af9743dbc1ba5d474768 Mon Sep 17 00:00:00 2001 From: marrus Date: Fri, 21 Oct 2011 10:42:15 +0200 Subject: [PATCH 026/310] fixed recovery for a repeatedly failing vertex --- .../CheckpointDecisionCoordinator.java | 12 +- .../executiongraph/ExecutionGraph.java | 15 +- .../executiongraph/ExecutionVertex.java | 14 +- .../nephele/jobmanager/JobManager.java | 2 +- .../nephele/jobmanager/RecoveryThread.java | 121 ++++------- .../checkpointing/FailingJobITCase.java | 202 +++++++++++++++++- 6 files changed, 267 insertions(+), 99 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index 0f02adc1504d2..0f92a78882d42 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -23,7 +23,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; @@ -98,18 +97,19 @@ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtil // TODO: Provide sensible implementation here - if(rus.getUserCPU() >= 90){ + if (rus.getUserCPU() >= 90) { LOG.info("CPU-Bottleneck"); //CPU bottleneck checkpointDesicion = true; - }else{ + } else { - if( vertex.getNumberOfSuccessors() != 0 && vertex.getNumberOfPredecessors()*1.0/vertex.getNumberOfSuccessors() > 1.5){ - LOG.info("vertex.getNumberOfPredecessors()/vertex.getNumberOfSuccessors() > 1.5"); + if ( vertex.getNumberOfSuccessors() != 0 + && vertex.getNumberOfPredecessors() * 1.0 / vertex.getNumberOfSuccessors() > 1.5) { + LOG.info("vertex.getNumberOfPredecessors()/ vertex.getNumberOfSuccessors() > 1.5"); //less output-channels than input-channels //checkpoint at this position probably saves network-traffic checkpointDesicion = true; - }else if(true){ + } else if (true) { //always create checkpoint for testing checkpointDesicion = true; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index dac38b4343970..ed1dd3d04952a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -1223,12 +1223,15 @@ public synchronized void checkAndUpdateJobStatus(final ExecutionState latestStat while (it.hasNext()) { final ExecutionVertex vertex = it.next(); - if (vertex.getExecutionState() == ExecutionState.FAILED && !vertex.hasRetriesLeft()) { - this.jobStatus = InternalJobStatus.FAILING; - return; - }else{ - this.jobStatus = InternalJobStatus.RECOVERING; - return; + if (vertex.getExecutionState() == ExecutionState.FAILED ) { + if (!vertex.hasRetriesLeft()){ + System.out.println(" Vertex failed finally" ); + this.jobStatus = InternalJobStatus.FAILING; + return; + }else{ + this.jobStatus = InternalJobStatus.RECOVERING; + return; + } } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index b73c9ca161386..a2f7db45d8a1c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -124,6 +124,11 @@ public final class ExecutionVertex { * The current checkpoint state of this vertex. */ private CheckpointState checkpointState = CheckpointState.NONE; + + /** + * Number of times this vertex may be restarted + */ + private int retries = 3; //TODO make this configurable /** * Create a new execution vertex and instantiates its environment. @@ -296,6 +301,9 @@ public void updateExecutionState(final ExecutionState newExecutionState, final S // Save the new execution state this.executionState = newExecutionState; + if(this.executionState == ExecutionState.FAILED){ + this.retries--; + } // Notify the listener objects final Iterator it = this.executionListeners.iterator(); while (it.hasNext()) { @@ -622,8 +630,10 @@ public String toString() { * @return true if the task has a retry attempt left, false otherwise */ public boolean hasRetriesLeft() { - // TODO: Implement me - return false; + if(this.retries < 0){ + return false; + } + return true; } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index e276a87fbbe97..ba356fe05a3c7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -1059,7 +1059,7 @@ public void jobStatusHasChanged(final ExecutionGraph executionGraph, final Inter } if (newJobStatus == InternalJobStatus.RECOVERING) { try { - RecoveryThread recoverythread = new RecoveryThread(executionGraph, this); + RecoveryThread recoverythread = new RecoveryThread(executionGraph); recoverythread.start(); } catch (Exception e) { e.printStackTrace(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java index 16f34552a8f4b..862826ce2b793 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java @@ -25,17 +25,14 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.AbstractInstance; -import eu.stratosphere.nephele.protocols.JobManagerProtocol; import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; import eu.stratosphere.nephele.util.SerializableArrayList; -import eu.stratosphere.nephele.util.StringUtils; /** * @author marrus @@ -44,32 +41,23 @@ public class RecoveryThread extends Thread { private static final Log LOG = LogFactory.getLog(RecoveryThread.class); - final ExecutionGraph job; + private final ExecutionGraph job; List failedVertices; - private List checkpoints; - - final private JobManagerProtocol jobManager; - private List globalConsistentCheckpoint = new SerializableArrayList(); /** * Initializes RecoveryThread. * - * @param job - * @param jobManager + * @param job The Job with * @throws Exception */ - public RecoveryThread(ExecutionGraph job, JobManager jobManager) throws Exception { + public RecoveryThread(final ExecutionGraph job) throws Exception { super("Recovery Thread"); this.job = job; - this.jobManager = jobManager; this.failedVertices = new ArrayList(); this.failedVertices.addAll(job.getFailedVertices()); - this.checkpoints = job.getVerticesWithCheckpoints(); - LOG.info("RecoveryThread"); - } /** @@ -81,9 +69,9 @@ public void run() { LOG.error("No failed vertices to recover"); } //FIXME (marrus) dirty fix - while(!this.failedVertices.isEmpty()){ + while (!this.failedVertices.isEmpty()) { recover(this.failedVertices.remove(0)); - if(this.failedVertices.isEmpty()){ + if (this.failedVertices.isEmpty()) { this.failedVertices = this.job.getFailedVertices(); } } @@ -93,7 +81,7 @@ public void run() { /** * */ - private boolean recover(ExecutionVertex failed) { + private boolean recover(final ExecutionVertex failed) { List replayCheckpoints = new ArrayList(); @@ -106,7 +94,7 @@ private boolean recover(ExecutionVertex failed) { Iterator restartIterator = restart.iterator(); while (restartIterator.hasNext()) { ExecutionVertex vertex = restartIterator.next(); - if(vertex.getID() != failed.getID()){ + if (vertex.getID() != failed.getID()) { try { vertex.getAllocatedResource().getInstance().restartTask(vertex.getID(),this.job.getJobConfiguration(), vertex.getEnvironment(), vertex.constructInitialActiveOutputChannelsSet() ); } catch (IOException e) { @@ -128,14 +116,14 @@ private boolean recover(ExecutionVertex failed) { //get list of instances of consistencheckpoints List instances = new SerializableArrayList(); - for(ExecutionVertexID id : this.globalConsistentCheckpoint){ + for (ExecutionVertexID id : this.globalConsistentCheckpoint) { AbstractInstance instance = this.job.getVertexByID(id).getAllocatedResource().getInstance(); - if(!instances.contains(instance)){ + if (!instances.contains(instance)) { instances.add(instance); } } Iterator instanceIterator = instances.iterator(); - while(instanceIterator.hasNext()){ + while (instanceIterator.hasNext()) { //replay all necessary checkpoints try { @@ -148,7 +136,7 @@ private boolean recover(ExecutionVertex failed) { } for(CheckpointReplayResult replayResult : replayCheckpoints ){ - if(replayResult.getReturnCode() == ReturnCode.ERROR){ + if (replayResult.getReturnCode() == ReturnCode.ERROR) { LOG.info("Replay of Checkpoints return Error " + replayResult.getDescription() ); this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); return false; @@ -156,10 +144,6 @@ private boolean recover(ExecutionVertex failed) { } LOG.info("FINISHED RECOVERY for " + failed.getName()); this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.RERUNNING, null); - - - - return true; } @@ -181,30 +165,11 @@ private List findRestarts(ExecutionVertex failed) { if (!restart.contains(vertex)) { restart.add(vertex); } -// for (int i = 0; i < vertex.getNumberOfSuccessors(); i++) { -// //all successors must be restarted -// ExecutionVertex successor = vertex.getSuccessor(i); -// restart.add(successor); -// if (successor.getCheckpointState() == CheckpointState.PARTIAL) { -// //these tasks will be restarted, delete checkpoints -// this.checkpoints.remove(successor); -// this.globalConsistentCheckpoint.remove(successor.getID()); -// } -// //all followers must be restarted -// List follower = findFollowers(successor, restart); -// restart.addAll(follower); -// Iterator iter = follower.iterator(); -// while (iter.hasNext()) { -// ExecutionVertex follow = iter.next(); -// if (!visited.contains(follow)) { -// totest.add(follow); -// } -// } -// } //predecessors must be either checkpoints or need to be restarted too for (int j = 0; j < vertex.getNumberOfPredecessors(); j++) { ExecutionVertex predecessor = vertex.getPredecessor(j); - if (predecessor.getCheckpointState() != CheckpointState.PARTIAL && predecessor.getCheckpointState() != CheckpointState.COMPLETE) { + if (predecessor.getCheckpointState() != CheckpointState.PARTIAL + && predecessor.getCheckpointState() != CheckpointState.COMPLETE) { restart.add(predecessor); if (!visited.contains(predecessor)) { @@ -215,16 +180,6 @@ private List findRestarts(ExecutionVertex failed) { this.globalConsistentCheckpoint.add(predecessor.getID()); } - -// List follower = findFollowers(predecessor, restart); -// restart.addAll(follower); -// Iterator iter = follower.iterator(); -// while (iter.hasNext()) { -// ExecutionVertex follow = iter.next(); -// if (!visited.contains(follow)) { -// totest.add(follow); -// } -// } } } @@ -234,29 +189,29 @@ private List findRestarts(ExecutionVertex failed) { return restart; } - private List findFollowers(ExecutionVertex vertex, ArrayList restart) { - ArrayList follower = new ArrayList(); - - for (int i = 0; i < vertex.getNumberOfSuccessors(); i++) { - ExecutionVertex successor = vertex.getSuccessor(i); - if (!restart.contains(successor)) { - follower.add(successor); - if (successor.getCheckpointState() == CheckpointState.PARTIAL) { - this.checkpoints.remove(successor); - - this.globalConsistentCheckpoint.remove(successor.getID()); - final SerializableArrayList checkpointsToRemove = new SerializableArrayList(); - checkpointsToRemove.add(successor.getID()); - try { - successor.getAllocatedResource().getInstance().removeCheckpoints(checkpointsToRemove); - } catch (IOException e) { - e.printStackTrace(); - } - } - } - } - - return follower; - } +// private List findFollowers(ExecutionVertex vertex, ArrayList restart) { +// ArrayList follower = new ArrayList(); +// +// for (int i = 0; i < vertex.getNumberOfSuccessors(); i++) { +// ExecutionVertex successor = vertex.getSuccessor(i); +// if (!restart.contains(successor)) { +// follower.add(successor); +// if (successor.getCheckpointState() == CheckpointState.PARTIAL) { +// this.checkpoints.remove(successor); +// +// this.globalConsistentCheckpoint.remove(successor.getID()); +// final SerializableArrayList checkpointsToRemove = new SerializableArrayList(); +// checkpointsToRemove.add(successor.getID()); +// try { +// successor.getAllocatedResource().getInstance().removeCheckpoints(checkpointsToRemove); +// } catch (IOException e) { +// e.printStackTrace(); +// } +// } +// } +// } +// +// return follower; +// } -} \ No newline at end of file +} diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java index 48b824c377677..f688ce9809a32 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java @@ -26,6 +26,7 @@ import java.util.HashSet; import java.util.Set; +import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -172,7 +173,7 @@ public static void startNephele() { if (!new File(configDir).exists()) { configDir = userDir + "/src/test/resources/" + CONFIGURATION_DIRECTORY; } - + final Constructor c = JobManager.class.getDeclaredConstructor(new Class[] { String.class, String.class }); c.setAccessible(true); @@ -344,7 +345,49 @@ public void invoke() throws Exception { } } } + public final static class RefailingInnerTask extends AbstractTask { + + private MutableRecordReader recordReader; + + private RecordWriter recordWriter; + + /** + * {@inheritDoc} + */ + @Override + public void registerInputOutput() { + + this.recordWriter = new RecordWriter(this, FailingJobRecord.class); + this.recordReader = new MutableRecordReader(this, + new BipartiteDistributionPattern()); + } + + /** + * {@inheritDoc} + */ + @Override + public void invoke() throws Exception { + final FailingJobRecord record = new FailingJobRecord(); + + boolean failing = false; + + final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger(FAILURE_INDEX_KEY, -1)); + + + int count = 0; + + while (this.recordReader.next(record)) { + + this.recordWriter.emit(record); + if (count++ == failAfterRecord && failing) { + throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " + + getIndexInSubtaskGroup()); + } + } + } + } public static final class OutputTask extends AbstractOutputTask { private MutableRecordReader recordReader; @@ -650,4 +693,161 @@ public void testSuccessivelyFailingInnerVertices() { } } + /** + * This test checks Nephele's fault tolerance capabilities by simulating a successively failing one inner vertices. + */ + @Test + public void testSuccessivelyFailingSameInnerVertices() { + + final JobGraph jobGraph = new JobGraph("Job with failing inner vertex"); + + final JobGenericInputVertex input = new JobGenericInputVertex("Input", jobGraph); + input.setInputClass(InputTask.class); + input.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + input.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobTaskVertex innerVertex1 = new JobTaskVertex("Inner vertex 1", jobGraph); + innerVertex1.setTaskClass(RefailingInnerTask.class); + innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + innerVertex1.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 145613); + innerVertex1.getConfiguration().setInteger(FAILURE_INDEX_KEY, 2); + + final JobTaskVertex innerVertex2 = new JobTaskVertex("Inner vertex 2", jobGraph); + innerVertex2.setTaskClass(InnerTask.class); + innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobTaskVertex innerVertex3 = new JobTaskVertex("Inner vertex 3", jobGraph); + innerVertex3.setTaskClass(InnerTask.class); + innerVertex3.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex3.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobGenericOutputVertex output = new JobGenericOutputVertex("Output", jobGraph); + output.setOutputClass(OutputTask.class); + output.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + output.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + // Configure instance sharing + innerVertex1.setVertexToShareInstancesWith(input); + innerVertex2.setVertexToShareInstancesWith(input); + innerVertex3.setVertexToShareInstancesWith(input); + output.setVertexToShareInstancesWith(input); + + try { + + input.connectTo(innerVertex1, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex1.connectTo(innerVertex2, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex2.connectTo(innerVertex3, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex3.connectTo(output, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + + } catch (JobGraphDefinitionException e) { + fail(StringUtils.stringifyException(e)); + } + + // Reset the FAILED_ONCE flags + synchronized (FAILED_ONCE) { + FAILED_ONCE.clear(); + } + + // Create job client and launch job + try { + JobClient jobClient = new JobClient(jobGraph, configuration); + jobClient.submitJobAndWait(); + } catch (IOException ioe) { + fail(StringUtils.stringifyException(ioe)); + } catch (JobExecutionException e) { + assert true; + } + fail("Job expected to be cancled"); + } + + /** + * This test checks Nephele's fault tolerance capabilities by simulating a successively failing one inner vertices. + */ + @Test + public void testSuccessivelyFailingSeveralInnerVertices() { + + final JobGraph jobGraph = new JobGraph("Job with failing inner vertex"); + + final JobGenericInputVertex input = new JobGenericInputVertex("Input", jobGraph); + input.setInputClass(InputTask.class); + input.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + input.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobTaskVertex innerVertex1 = new JobTaskVertex("Inner vertex 1", jobGraph); + innerVertex1.setTaskClass(RefailingInnerTask.class); + innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + innerVertex1.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 145613); + innerVertex1.getConfiguration().setInteger(FAILURE_INDEX_KEY, 2); + + final JobTaskVertex innerVertex2 = new JobTaskVertex("Inner vertex 2", jobGraph); + innerVertex2.setTaskClass(InnerTask.class); + innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + innerVertex2.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 32563); + innerVertex2.getConfiguration().setInteger(FAILURE_INDEX_KEY, 1); + + final JobTaskVertex innerVertex3 = new JobTaskVertex("Inner vertex 3", jobGraph); + innerVertex3.setTaskClass(InnerTask.class); + innerVertex3.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex3.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + innerVertex2.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 158563); + innerVertex2.getConfiguration().setInteger(FAILURE_INDEX_KEY, 0); + + final JobGenericOutputVertex output = new JobGenericOutputVertex("Output", jobGraph); + output.setOutputClass(OutputTask.class); + output.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + output.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + // Configure instance sharing + innerVertex1.setVertexToShareInstancesWith(input); + innerVertex2.setVertexToShareInstancesWith(input); + innerVertex3.setVertexToShareInstancesWith(input); + output.setVertexToShareInstancesWith(input); + + try { + + input.connectTo(innerVertex1, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex1.connectTo(innerVertex2, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex2.connectTo(innerVertex3, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex3.connectTo(output, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + + } catch (JobGraphDefinitionException e) { + fail(StringUtils.stringifyException(e)); + } + + // Reset the FAILED_ONCE flags + synchronized (FAILED_ONCE) { + FAILED_ONCE.clear(); + } + + // Create job client and launch job + try { + JobClient jobClient = new JobClient(jobGraph, configuration); + jobClient.submitJobAndWait(); + } catch (IOException ioe) { + fail(StringUtils.stringifyException(ioe)); + } catch (JobExecutionException e) { + fail(StringUtils.stringifyException(e)); + } + + } + @After public void cleanUp(){ + File file = new File("/tmp/"); + File[] files= file.listFiles(); + + for (int i = 0; i < files.length; i++) { + String name = files[i].getName(); + if (name.startsWith("fb") || name.startsWith("checkpoint_")) { + files[i].delete(); + } + } + + System.out.println("deleted"); + + + } + } From 74e1aad1ddbcb2575a424854d823d6b81f6be1a0 Mon Sep 17 00:00:00 2001 From: marrus Date: Mon, 24 Oct 2011 09:55:31 +0200 Subject: [PATCH 027/310] implemented removing of checkpoints --- .../checkpointing/CheckpointReplayManager.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java index 6a09f10a47d3e..c76dcfcec7e9b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java @@ -92,8 +92,21 @@ public void replayCheckpoint(final ExecutionVertexID vertexID) { * the vertex whose checkpoint shall be removed */ public void removeCheckpoint(final ExecutionVertexID vertexID) { - - // TODO: Implement me + File file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + + "_final"); + if (file.exists()) { + file.delete(); + return; + } + file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_0"); + if (file.exists()) { + file.delete(); + } + + file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_part"); + if (file.exists()) { + file.delete(); + } } } From 079b3d61729ef04a573b33ad195dc83136e0c788 Mon Sep 17 00:00:00 2001 From: marrus Date: Mon, 24 Oct 2011 11:32:27 +0200 Subject: [PATCH 028/310] Fixes in fault tolerance test --- .../nephele/checkpointing/FailingJobITCase.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java index f688ce9809a32..4c85fa343df0e 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java @@ -697,7 +697,7 @@ public void testSuccessivelyFailingInnerVertices() { * This test checks Nephele's fault tolerance capabilities by simulating a successively failing one inner vertices. */ @Test - public void testSuccessivelyFailingSameInnerVertices() { + public void testRepeatedlyFailingSameInnerVertex() { final JobGraph jobGraph = new JobGraph("Job with failing inner vertex"); @@ -707,6 +707,7 @@ public void testSuccessivelyFailingSameInnerVertices() { input.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); final JobTaskVertex innerVertex1 = new JobTaskVertex("Inner vertex 1", jobGraph); + //Using re-failing inner task innerVertex1.setTaskClass(RefailingInnerTask.class); innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); @@ -757,7 +758,9 @@ public void testSuccessivelyFailingSameInnerVertices() { } catch (IOException ioe) { fail(StringUtils.stringifyException(ioe)); } catch (JobExecutionException e) { - assert true; + //This is expected here + assert(e.isJobCanceledByUser() == false); + return; } fail("Job expected to be cancled"); } @@ -776,7 +779,7 @@ public void testSuccessivelyFailingSeveralInnerVertices() { input.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); final JobTaskVertex innerVertex1 = new JobTaskVertex("Inner vertex 1", jobGraph); - innerVertex1.setTaskClass(RefailingInnerTask.class); + innerVertex1.setTaskClass(InnerTask.class); innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); innerVertex1.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 145613); @@ -834,6 +837,7 @@ public void testSuccessivelyFailingSeveralInnerVertices() { } } + @After public void cleanUp(){ File file = new File("/tmp/"); File[] files= file.listFiles(); From 0a0fc1707da0bf1128908c2ceb2402fcc99f5e3b Mon Sep 17 00:00:00 2001 From: marrus Date: Wed, 26 Oct 2011 13:45:06 +0200 Subject: [PATCH 029/310] added basic networkchannel test --- .../checkpointing/FailingJobITCase.java | 64 ++++++++++++++++++- 1 file changed, 62 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java index 4c85fa343df0e..ded8c01c60d66 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java @@ -552,7 +552,66 @@ public void testRecoveryFromFileChannels() { fail(StringUtils.stringifyException(e)); } } + /** + * This test checks Nephele's capabilities to recover from network channel checkpoints. + */ + @Test + public void testRecoveryFromNetworkChannels() { + + final JobGraph jobGraph = new JobGraph("Job with Network channels"); + + final JobGenericInputVertex input = new JobGenericInputVertex("Input", jobGraph); + input.setInputClass(InputTask.class); + input.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + input.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobTaskVertex innerVertex1 = new JobTaskVertex("Inner vertex 1", jobGraph); + innerVertex1.setTaskClass(InnerTask.class); + innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobTaskVertex innerVertex2 = new JobTaskVertex("Inner vertex 2", jobGraph); + innerVertex2.setTaskClass(InnerTask.class); + innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + innerVertex2.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 95490); + innerVertex2.getConfiguration().setInteger(FAILURE_INDEX_KEY, 0); + + final JobGenericOutputVertex output = new JobGenericOutputVertex("Output", jobGraph); + output.setOutputClass(OutputTask.class); + output.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + output.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + // Configure instance sharing + innerVertex1.setVertexToShareInstancesWith(input); + innerVertex2.setVertexToShareInstancesWith(input); + output.setVertexToShareInstancesWith(input); + + try { + + input.connectTo(innerVertex1, ChannelType.NETWORK, CompressionLevel.NO_COMPRESSION); + innerVertex1.connectTo(innerVertex2, ChannelType.NETWORK, CompressionLevel.NO_COMPRESSION); + innerVertex2.connectTo(output, ChannelType.NETWORK, CompressionLevel.NO_COMPRESSION); + + } catch (JobGraphDefinitionException e) { + fail(StringUtils.stringifyException(e)); + } + + // Reset the FAILED_ONCE flags + synchronized (FAILED_ONCE) { + FAILED_ONCE.clear(); + } + // Create job client and launch job + try { + JobClient jobClient = new JobClient(jobGraph, configuration); + jobClient.submitJobAndWait(); + } catch (IOException ioe) { + fail(StringUtils.stringifyException(ioe)); + } catch (JobExecutionException e) { + fail(StringUtils.stringifyException(e)); + } + } /** * This test checks Nephele's fault tolerance capabilities by simulating a failing input vertex. */ @@ -577,6 +636,7 @@ public void testFailingInputVertex() { innerVertex2.setTaskClass(InnerTask.class); innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + final JobTaskVertex innerVertex3 = new JobTaskVertex("Inner vertex 3", jobGraph); innerVertex3.setTaskClass(InnerTask.class); @@ -699,7 +759,7 @@ public void testSuccessivelyFailingInnerVertices() { @Test public void testRepeatedlyFailingSameInnerVertex() { - final JobGraph jobGraph = new JobGraph("Job with failing inner vertex"); + final JobGraph jobGraph = new JobGraph("Job with repeatedly failing inner vertex"); final JobGenericInputVertex input = new JobGenericInputVertex("Input", jobGraph); input.setInputClass(InputTask.class); @@ -771,7 +831,7 @@ public void testRepeatedlyFailingSameInnerVertex() { @Test public void testSuccessivelyFailingSeveralInnerVertices() { - final JobGraph jobGraph = new JobGraph("Job with failing inner vertex"); + final JobGraph jobGraph = new JobGraph("Job with several successively failing inner vertex"); final JobGenericInputVertex input = new JobGenericInputVertex("Input", jobGraph); input.setInputClass(InputTask.class); From 04de7023d24794ed622fbc2df78c08bdfbaa242d Mon Sep 17 00:00:00 2001 From: marrus Date: Wed, 26 Oct 2011 18:35:16 +0200 Subject: [PATCH 030/310] implemented annotaions for tasks --- .../nephele/annotations/ForceCheckpoint.java | 30 ++++++++++++ .../nephele/annotations/Statefull.java | 31 +++++++++++++ .../nephele/annotations/Stateless.java | 31 +++++++++++++ .../nephele/annotations/TaskAnnotation.java | 16 +++++++ .../CheckpointDecisionCoordinator.java | 38 ++++++++------- .../ResourceUtilizationSnapshot.java | 46 +++++++++++++++++++ .../nephele/taskmanager/Task.java | 16 ++++++- 7 files changed, 190 insertions(+), 18 deletions(-) create mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java create mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Statefull.java create mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java create mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java new file mode 100644 index 0000000000000..f002e51643ce4 --- /dev/null +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java @@ -0,0 +1,30 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.annotations; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; +/** + * Annotation to force a checkpoint-decision. + * @author marrus + * + */ +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.TYPE) +public @interface ForceCheckpoint { + boolean checkpoint(); +} diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Statefull.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Statefull.java new file mode 100644 index 0000000000000..40a37189f9f0d --- /dev/null +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Statefull.java @@ -0,0 +1,31 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.annotations; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation indicate a state-full Task + * @author marrus + * + */ +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.TYPE) +public @interface Statefull { + +} diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java new file mode 100644 index 0000000000000..3b1800640a474 --- /dev/null +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java @@ -0,0 +1,31 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.annotations; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation indicate a state-full Task + * @author marrus + * + */ +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.TYPE) +public @interface Stateless { + +} \ No newline at end of file diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java new file mode 100644 index 0000000000000..0fabfc830d339 --- /dev/null +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java @@ -0,0 +1,16 @@ +package eu.stratosphere.nephele.annotations; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.CONSTRUCTOR) +public @interface TaskAnnotation { + + boolean statefull(); + double selektivity() default 1; + +} + diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index 0f92a78882d42..c24e41b8d2d20 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -91,35 +91,39 @@ public void registerJob(final ExecutionGraph executionGraph) { * the current resource utilization of the vertex */ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { - boolean checkpointDesicion = false; + boolean checkpointDecision = false; LOG.info("Checkpoint decision for vertex " + vertex + " required"); // This implementation always creates the checkpoint // TODO: Provide sensible implementation here + if(rus.getForced() == null){ + if (rus.getUserCPU() >= 90) { + LOG.info("CPU-Bottleneck"); + //CPU bottleneck + checkpointDecision = true; + } else { - if (rus.getUserCPU() >= 90) { - LOG.info("CPU-Bottleneck"); - //CPU bottleneck - checkpointDesicion = true; - } else { - - if ( vertex.getNumberOfSuccessors() != 0 - && vertex.getNumberOfPredecessors() * 1.0 / vertex.getNumberOfSuccessors() > 1.5) { - LOG.info("vertex.getNumberOfPredecessors()/ vertex.getNumberOfSuccessors() > 1.5"); - //less output-channels than input-channels - //checkpoint at this position probably saves network-traffic - checkpointDesicion = true; - } else if (true) { - //always create checkpoint for testing - checkpointDesicion = true; + if ( vertex.getNumberOfSuccessors() != 0 + && vertex.getNumberOfPredecessors() * 1.0 / vertex.getNumberOfSuccessors() > 1.5) { + LOG.info("vertex.getNumberOfPredecessors()/ vertex.getNumberOfSuccessors() > 1.5"); + //less output-channels than input-channels + //checkpoint at this position probably saves network-traffic + checkpointDecision = true; + } else if (true) { + //always create checkpoint for testing + checkpointDecision = true; + } } + }else{ + //checkpoint decision was forced by the user + checkpointDecision = rus.getForced(); } final ExecutionGraph graph = vertex.getExecutionGraph(); final Map> checkpointDecisions = new HashMap>(); final List checkpointDecisionList = new SerializableArrayList(); synchronized (graph) { - checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), checkpointDesicion)); + checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), checkpointDecision)); checkpointDecisions.put(vertex.getAllocatedResource().getInstance(), checkpointDecisionList); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java index 00b892debb776..c1c4aa96c1df7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.Map; +import eu.stratosphere.nephele.annotations.ForceCheckpoint; import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.io.channels.ChannelID; @@ -46,6 +47,12 @@ public final class ResourceUtilizationSnapshot implements IOReadableWritable { * userCPu Time in percent */ private long userCPU; + + /** + * The forced decision if annotated + */ + private Boolean forced; + public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU) { @@ -60,6 +67,23 @@ public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU, Boolean forced) { + + if (timestamp <= 0L) { + throw new IllegalArgumentException("Argument timestamp must be larger than zero"); + } + + if (channelUtilization == null) { + throw new IllegalArgumentException("Argument channelUtilization is null"); + } + + this.timestamp = timestamp; + this.channelUtilization = channelUtilization; + this.userCPU = userCPU; + this.forced = forced; + } public ResourceUtilizationSnapshot() { @@ -84,6 +108,15 @@ public void write(final DataOutput out) throws IOException { entry.getKey().write(out); out.writeLong(entry.getValue().longValue()); } + // Write the userCPU + out.writeLong(this.userCPU); + // Write forced decision + if(this.forced == null){ + out.writeByte(0); + }else{ + out.writeByte(1); + out.writeBoolean(this.forced); + } } /** @@ -104,6 +137,11 @@ public void read(final DataInput in) throws IOException { final Long l = Long.valueOf(in.readLong()); this.channelUtilization.put(channelID, l); } + this.userCPU = in.readLong(); + + if(in.readByte() == 1){ + this.forced = in.readBoolean(); + } } /** @@ -144,4 +182,12 @@ public long getAmountOfDataTransmitted(final ChannelID channelID) { public long getUserCPU() { return this.userCPU; } + /** + * Returns whether a decision was forced by the user + * @return + */ + public Boolean getForced() { + return forced; + } + } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index 21eba503fc9fb..5a225d895abc0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -27,6 +27,9 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.stratosphere.nephele.annotations.ForceCheckpoint; +import eu.stratosphere.nephele.annotations.Statefull; +import eu.stratosphere.nephele.annotations.Stateless; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionObserver; @@ -263,8 +266,19 @@ public void initialExecutionResourcesExhausted() { Long.valueOf(outputChannel.getAmountOfDataTransmitted())); } } + Boolean force = null; - final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, outputChannelUtilization, userCPU); + if(this.environment.getInvokable().getClass().isAnnotationPresent(Statefull.class) && !this.environment.getInvokable().getClass().isAnnotationPresent(Stateless.class) ){ + //Don't checkpoint statefull tasks + force = false; + }else{ + //look for a forced decision from the user + ForceCheckpoint forced = this.environment.getInvokable().getClass().getAnnotation(ForceCheckpoint.class); + if(forced != null){ + force = forced.checkpoint(); + } + } + final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, outputChannelUtilization, userCPU, force); // Notify the listener objects final Iterator it = this.registeredListeners.iterator(); From 975524783b90eb045123ce0898fee06cc43bf744 Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 1 Nov 2011 10:24:08 +0100 Subject: [PATCH 031/310] fixed too early closevent for large records --- .../bytebuffered/AbstractByteBufferedOutputChannel.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java index 93263e050b152..e42d09d8b9ec9 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java @@ -92,7 +92,7 @@ public abstract class AbstractByteBufferedOutputChannel extend */ private long amountOfDataTransmitted = 0L; - private static final Log LOG = LogFactory.getLog(AbstractByteBufferedInputChannel.class); + private static final Log LOG = LogFactory.getLog(AbstractByteBufferedOutputChannel.class); /** * Creates a new byte buffered output channel. @@ -148,6 +148,10 @@ public void requestClose() throws IOException, InterruptedException { if (!this.closeRequested) { this.closeRequested = true; + if (this.serializationBuffer.dataLeftFromPreviousSerialization()) { + //make sure we serialized all data before we send the close event + flush(); + } if (!isBroadcastChannel() || getChannelIndex() == 0) { transferEvent(new ByteBufferedChannelCloseEvent()); From 1f10dff889f985f89a9807e8827783367c1bfec4 Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 1 Nov 2011 10:25:47 +0100 Subject: [PATCH 032/310] changed restarting. added new test with annotations --- .../nephele/jobmanager/RecoveryThread.java | 11 +- .../nephele/taskmanager/TaskManager.java | 5 +- .../checkpointing/FailingJobITCase.java | 121 +++++++++++++++++- 3 files changed, 131 insertions(+), 6 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java index 862826ce2b793..235c75ad7dd98 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java @@ -44,7 +44,8 @@ public class RecoveryThread extends Thread { private final ExecutionGraph job; List failedVertices; - + + List recovered = new ArrayList(); private List globalConsistentCheckpoint = new SerializableArrayList(); /** @@ -70,7 +71,11 @@ public void run() { } //FIXME (marrus) dirty fix while (!this.failedVertices.isEmpty()) { - recover(this.failedVertices.remove(0)); + ExecutionVertex torecover = this.failedVertices.remove(0); + if(!this.recovered.contains(torecover)){ + recover(torecover); + this.recovered.add(torecover); + } if (this.failedVertices.isEmpty()) { this.failedVertices = this.job.getFailedVertices(); } @@ -143,7 +148,7 @@ private boolean recover(final ExecutionVertex failed) { } } LOG.info("FINISHED RECOVERY for " + failed.getName()); - this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.RERUNNING, null); + this.job.executionStateChanged(this.job.getJobID(), failed.getID(), ExecutionState.RERUNNING, null); return true; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 23210d3916c97..575768395a573 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -887,7 +887,10 @@ public void run() { @Override public void restartTask(ExecutionVertexID id, Configuration jobConfiguration, Environment environment, Set activeOutputChannels) throws IOException { - unregisterTask(id, this.runningTasks.get(id)); + Task torestart = this.runningTasks.get(id); + //torestart.markAsRestarting(); + torestart.cancelExecution(); + //unregisterTask(id,torestart); submitTask(id, jobConfiguration, environment, activeOutputChannels); } diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java index ded8c01c60d66..7b45fc8ad8203 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java @@ -31,6 +31,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import eu.stratosphere.nephele.annotations.ForceCheckpoint; import eu.stratosphere.nephele.client.JobClient; import eu.stratosphere.nephele.client.JobExecutionException; import eu.stratosphere.nephele.configuration.ConfigConstants; @@ -299,13 +300,59 @@ public void invoke() throws Exception { } } } - + @ForceCheckpoint(checkpoint = true) public final static class InnerTask extends AbstractTask { private MutableRecordReader recordReader; private RecordWriter recordWriter; + + /** + * {@inheritDoc} + */ + @Override + public void registerInputOutput() { + + this.recordWriter = new RecordWriter(this, FailingJobRecord.class); + this.recordReader = new MutableRecordReader(this, + new BipartiteDistributionPattern()); + } + + /** + * {@inheritDoc} + */ + @Override + public void invoke() throws Exception { + + final FailingJobRecord record = new FailingJobRecord(); + + boolean failing = false; + + final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + synchronized (FAILED_ONCE) { + failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( + FAILURE_INDEX_KEY, -1)) && FAILED_ONCE.add(getEnvironment().getTaskName()); + } + int count = 0; + + while (this.recordReader.next(record)) { + + this.recordWriter.emit(record); + if (count++ == failAfterRecord && failing) { + throw new RuntimeException("Runtime exception in " + getEnvironment().getTaskName() + " " + + getIndexInSubtaskGroup()); + } + } + } + } + @ForceCheckpoint(checkpoint = false) + public final static class NoCheckpointInnerTask extends AbstractTask { + + private MutableRecordReader recordReader; + + private RecordWriter recordWriter; + /** * {@inheritDoc} */ @@ -449,6 +496,7 @@ public void testFailingInternalVertex() { innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + final JobTaskVertex innerVertex2 = new JobTaskVertex("Inner vertex 2", jobGraph); innerVertex2.setTaskClass(InnerTask.class); innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); @@ -913,5 +961,74 @@ public void testSuccessivelyFailingSeveralInnerVertices() { } - + /** + * This test checks Nephele's fault tolerance capabilities by simulating a failing inner vertex without all tasks checkpointing. + */ + @Test + public void testFailingInternalVertexSomeCheckpoints() { + + final JobGraph jobGraph = new JobGraph("Job with failing inner vertex"); + + final JobGenericInputVertex input = new JobGenericInputVertex("Input", jobGraph); + input.setInputClass(InputTask.class); + input.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + input.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + final JobTaskVertex innerVertex1 = new JobTaskVertex("Inner vertex 1", jobGraph); + innerVertex1.setTaskClass(InnerTask.class); + innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + + final JobTaskVertex innerVertex2 = new JobTaskVertex("Inner vertex 2", jobGraph); + innerVertex2.setTaskClass(NoCheckpointInnerTask.class); + innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + + final JobTaskVertex innerVertex3 = new JobTaskVertex("Inner vertex 3", jobGraph); + innerVertex3.setTaskClass(InnerTask.class); + innerVertex3.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + innerVertex3.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + innerVertex3.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 95490); + innerVertex3.getConfiguration().setInteger(FAILURE_INDEX_KEY, 0); + + final JobGenericOutputVertex output = new JobGenericOutputVertex("Output", jobGraph); + output.setOutputClass(OutputTask.class); + output.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); + output.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); + + // Configure instance sharing + innerVertex1.setVertexToShareInstancesWith(input); + innerVertex2.setVertexToShareInstancesWith(input); + innerVertex3.setVertexToShareInstancesWith(input); + output.setVertexToShareInstancesWith(input); + + try { + + input.connectTo(innerVertex1, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex1.connectTo(innerVertex2, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex2.connectTo(innerVertex3, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + innerVertex3.connectTo(output, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + + } catch (JobGraphDefinitionException e) { + fail(StringUtils.stringifyException(e)); + } + + // Reset the FAILED_ONCE flags + synchronized (FAILED_ONCE) { + FAILED_ONCE.clear(); + } + + // Create job client and launch job + try { + JobClient jobClient = new JobClient(jobGraph, configuration); + jobClient.submitJobAndWait(); + } catch (IOException ioe) { + fail(StringUtils.stringifyException(ioe)); + } catch (JobExecutionException e) { + fail(StringUtils.stringifyException(e)); + } + } + } From d89f1eca9054e04bcf511a25d0e92d8aff0d4acc Mon Sep 17 00:00:00 2001 From: marrus Date: Mon, 14 Nov 2011 10:05:56 +0100 Subject: [PATCH 033/310] Added byte-counting for ByteBufferedInputChannel --- .../AbstractByteBufferedInputChannel.java | 9 +++++-- .../CheckpointDecisionCoordinator.java | 10 ++++++- .../ResourceUtilizationSnapshot.java | 27 +++++++++++++++++-- .../nephele/taskmanager/Task.java | 22 ++++++++++++--- 4 files changed, 59 insertions(+), 9 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java index fd353f0ab7c8e..56f1cfe5b53c3 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java @@ -70,6 +70,10 @@ public abstract class AbstractByteBufferedInputChannel extends private Buffer uncompressedDataBuffer = null; private IOException ioException = null; + /** + * Stores the number of bytes read through this input channel since its instantiation. + */ + private long amountOfDataTransmitted = 0L; /** * Creates a new network input channel. @@ -228,6 +232,8 @@ public void close() throws IOException, InterruptedException { private void releasedConsumedReadBuffer() { this.inputChannelBroker.releaseConsumedReadBuffer(); + // Keep track of number of bytes transmitted through this channel + this.amountOfDataTransmitted += this.uncompressedDataBuffer.size(); this.uncompressedDataBuffer = null; this.compressedDataBuffer = null; } @@ -315,7 +321,6 @@ public void activate() throws IOException, InterruptedException { @Override public long getAmountOfDataTransmitted() { - // TODO: Implement me - return 0L; + return this.amountOfDataTransmitted; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index c24e41b8d2d20..10eac34e55157 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -96,8 +96,15 @@ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtil // This implementation always creates the checkpoint // TODO: Provide sensible implementation here + if(rus.getForced() == null){ - if (rus.getUserCPU() >= 90) { + if(rus.getTotalInputAmount() != 0 && (rus.getTotalOutputAmount() * 1.0 / rus.getTotalInputAmount() > 2.0)){ + //estimated size of checkpoint + //TODO progress estimation would make sense here + checkpointDecision = false; + LOG.info("Chechpoint to large"); + } + else if (rus.getUserCPU() >= 90) { LOG.info("CPU-Bottleneck"); //CPU bottleneck checkpointDecision = true; @@ -132,3 +139,4 @@ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtil } } + diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java index c1c4aa96c1df7..7fd9bbf25f2b3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java @@ -52,6 +52,15 @@ public final class ResourceUtilizationSnapshot implements IOReadableWritable { * The forced decision if annotated */ private Boolean forced; + + /** + * amount of input bytes of all input-channels + */ + private long totalInputAmount; + /** + * amount of transmitted bytes of all output-channels + */ + private long totalOutputAmount; public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU) { @@ -69,7 +78,7 @@ public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU, Boolean forced) { + public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU, final Boolean forced, final long totalInputAmount, final long totalOutputAmount) { if (timestamp <= 0L) { throw new IllegalArgumentException("Argument timestamp must be larger than zero"); @@ -83,7 +92,8 @@ public ResourceUtilizationSnapshot(final long timestamp, final Map outputChannelUtilization = new HashMap(); - + final Map channelUtilization = new HashMap(); + long totalOutputAmount = 0; for (int i = 0; i < this.environment.getNumberOfOutputGates(); ++i) { final OutputGate outputGate = this.environment.getOutputGate(i); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); - outputChannelUtilization.put(outputChannel.getID(), + channelUtilization.put(outputChannel.getID(), Long.valueOf(outputChannel.getAmountOfDataTransmitted())); + totalOutputAmount += outputChannel.getAmountOfDataTransmitted(); + } + } + long totalInputAmount = 0; + for (int i = 0; i < this.environment.getNumberOfInputGates(); ++i) { + final InputGate inputGate = this.environment.getInputGate(i); + for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { + final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); + channelUtilization.put(inputChannel.getID(), + Long.valueOf(inputChannel.getAmountOfDataTransmitted())); + totalInputAmount += inputChannel.getAmountOfDataTransmitted(); + } } Boolean force = null; @@ -278,7 +292,7 @@ public void initialExecutionResourcesExhausted() { force = forced.checkpoint(); } } - final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, outputChannelUtilization, userCPU, force); + final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, channelUtilization, userCPU, force, totalInputAmount, totalOutputAmount); // Notify the listener objects final Iterator it = this.registeredListeners.iterator(); From 7363193cdf7236db250d5358806bfeb23d9d1a3c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 15 Nov 2011 19:10:46 +0000 Subject: [PATCH 034/310] Integrated task annotation from Mareike's branch --- .../nephele/annotations/Stateless.java | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java new file mode 100644 index 0000000000000..b99b9c579c13e --- /dev/null +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java @@ -0,0 +1,30 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.annotations; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation to indicate a stateless Nephele task. + * + * @author marrus + */ +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.TYPE) +public @interface Stateless { +} From d88e2242a3694afbbdae7a2ad3e3473d124cce4a Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 15 Nov 2011 19:39:49 +0000 Subject: [PATCH 035/310] Added convenience class to check a set of task class properties --- .../stratosphere/nephele/util/TaskUtils.java | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) create mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java new file mode 100644 index 0000000000000..24a5f9e55de52 --- /dev/null +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java @@ -0,0 +1,45 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.util; + +import eu.stratosphere.nephele.annotations.Stateless; +import eu.stratosphere.nephele.template.AbstractInvokable; + +/** + * This class implements several convenience methods to determine properties of Nephele task classes. + * + * @author warneke + */ +public class TaskUtils { + + /** + * Private constructor, so class cannot be instantiated + */ + private TaskUtils() { + } + + /** + * Checks if a task is declared to be stateless. + * + * @param taskClass + * the class to check + * @return true if the given class is declared to be stateless, false otherwise + */ + public static boolean isStateless(final Class taskClass) { + + return taskClass.isAnnotationPresent(Stateless.class); + } +} From 385614d98780f68b88bf9d7c9d574b897037e3b9 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 15 Nov 2011 19:41:24 +0000 Subject: [PATCH 036/310] Minor correction to previous commit --- .../src/main/java/eu/stratosphere/nephele/util/TaskUtils.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java index 24a5f9e55de52..408e6e8672253 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java @@ -26,7 +26,7 @@ public class TaskUtils { /** - * Private constructor, so class cannot be instantiated + * Private constructor, so class cannot be instantiated. */ private TaskUtils() { } @@ -35,7 +35,7 @@ private TaskUtils() { * Checks if a task is declared to be stateless. * * @param taskClass - * the class to check + * the class of the task to check * @return true if the given class is declared to be stateless, false otherwise */ public static boolean isStateless(final Class taskClass) { From 67d1511ba145e8fc9a859ef813151103f50d8fc3 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 17 Nov 2011 13:10:14 +0000 Subject: [PATCH 037/310] Introduced Nephele streaming plugin --- nephele/nephele-streaming/pom.xml | 29 +++++++++++++++++++ nephele/pom.xml | 1 + .../stratosphere-bin/conf/nephele-plugins.xml | 4 +-- 3 files changed, 32 insertions(+), 2 deletions(-) create mode 100644 nephele/nephele-streaming/pom.xml diff --git a/nephele/nephele-streaming/pom.xml b/nephele/nephele-streaming/pom.xml new file mode 100644 index 0000000000000..cf03276bc8e21 --- /dev/null +++ b/nephele/nephele-streaming/pom.xml @@ -0,0 +1,29 @@ + + + + 4.0.0 + + + nephele + eu.stratosphere + 0.2 + + + eu.stratosphere + nephele-streaming + 0.2 + nephele-streaming + jar + + + + + eu.stratosphere + nephele-server + ${version} + + + + + diff --git a/nephele/pom.xml b/nephele/pom.xml index e2e7a4af06365..d15ab598de44c 100644 --- a/nephele/pom.xml +++ b/nephele/pom.xml @@ -103,5 +103,6 @@ nephele-visualization nephele-server nephele-examples + nephele-streaming diff --git a/stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml b/stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml index cd0ce8cbf49da..726cb2708208d 100644 --- a/stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml +++ b/stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml @@ -17,8 +17,8 @@ --> - SCORE - eu.stratosphere.score.ScorePluginLoader + NepheleStreaming + eu.stratosphere.nephele.streaming.StreamingPluginLoader testkey From 39a0943994baa21d57df03724e10b50ec55f5f68 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 17 Nov 2011 13:56:39 +0000 Subject: [PATCH 038/310] Added JobManager and TaskManager components of streaming plugin --- .../nephele/jobmanager/JobManager.java | 2 +- .../streaming/StreamingJobManagerPlugin.java | 51 ++++++++++++++++ .../streaming/StreamingPluginLoader.java | 59 +++++++++++++++++++ .../streaming/StreamingTaskManagerPlugin.java | 31 ++++++++++ 4 files changed, 142 insertions(+), 1 deletion(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index a52eec3c4060a..477aed950678e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -403,7 +403,7 @@ public static void main(final String[] args) { } @Override - public JobSubmissionResult submitJob(final JobGraph job) throws IOException { + public JobSubmissionResult submitJob(JobGraph job) throws IOException { // First check if job is null if (job == null) { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java new file mode 100644 index 0000000000000..26c30c412ca14 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -0,0 +1,51 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.jobgraph.JobGraph; +import eu.stratosphere.nephele.plugins.JobManagerPlugin; + +public class StreamingJobManagerPlugin implements JobManagerPlugin { + + /** + * {@inheritDoc} + */ + @Override + public JobGraph rewriteJobGraph(final JobGraph jobGraph) { + // TODO Auto-generated method stub + return null; + } + + /** + * {@inheritDoc} + */ + @Override + public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) { + // TODO Auto-generated method stub + return null; + } + + /** + * {@inheritDoc} + */ + @Override + public void shutdown() { + // TODO Auto-generated method stub + + } + +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java new file mode 100644 index 0000000000000..c6ab84cb50e3c --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java @@ -0,0 +1,59 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.plugins.AbstractPluginLoader; +import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.plugins.TaskManagerPlugin; + +public class StreamingPluginLoader extends AbstractPluginLoader { + + private StreamingJobManagerPlugin jobManagerPlugin = null; + + private StreamingTaskManagerPlugin taskManagerPlugin = null; + + public StreamingPluginLoader(final Configuration pluginConfiguration) { + super(pluginConfiguration); + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized JobManagerPlugin getJobManagerPlugin() { + + if (this.jobManagerPlugin == null) { + this.jobManagerPlugin = new StreamingJobManagerPlugin(); + } + + return this.jobManagerPlugin; + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized TaskManagerPlugin getTaskManagerPlugin() { + + if (this.taskManagerPlugin == null) { + this.taskManagerPlugin = new StreamingTaskManagerPlugin(); + } + + return this.taskManagerPlugin; + } + +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java new file mode 100644 index 0000000000000..d2fd6d8d9dec4 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -0,0 +1,31 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import eu.stratosphere.nephele.plugins.TaskManagerPlugin; + +public class StreamingTaskManagerPlugin implements TaskManagerPlugin { + + /** + * {@inheritDoc} + */ + @Override + public void shutdown() { + // TODO Auto-generated method stub + + } + +} From 8d485bab5c5e1abe201e660c76564dc11476334c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 17 Nov 2011 14:54:01 +0000 Subject: [PATCH 039/310] Added taggable record type --- .../nephele/types/AbstractTaggableRecord.java | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) create mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java new file mode 100644 index 0000000000000..21436c39a91c1 --- /dev/null +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java @@ -0,0 +1,57 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.types; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public abstract class AbstractTaggableRecord implements Record { + + private byte[] tag = null; + + public void setTag(final byte[] tag) { + this.tag = tag; + } + + public byte[] getTag() { + + return this.tag; + } + + @Override + public void write(final DataOutput out) throws IOException { + + if (this.tag == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeShort((short) this.tag.length); + out.write(this.tag); + } + + } + + @Override + public void read(final DataInput in) throws IOException { + + if (in.readBoolean()) { + final short length = in.readShort(); + this.tag = new byte[length]; + in.readFully(this.tag); + } + } +} From 9defcefd9786216f467a76be3cb1146324bbc182 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 17 Nov 2011 15:30:45 +0000 Subject: [PATCH 040/310] Extended plugin interfaces --- .../nephele/types/AbstractTaggableRecord.java | 6 +++++ .../nephele/plugins/TaskManagerPlugin.java | 26 +++++++++++++++++++ .../nephele/taskmanager/TaskManager.java | 22 +++++++++++++--- .../streaming/StreamingJobManagerPlugin.java | 4 +++ .../streaming/StreamingPluginLoader.java | 4 +-- .../streaming/StreamingTaskManagerPlugin.java | 26 +++++++++++++++++++ .../score/ScoreTaskManagerPlugin.java | 21 +++++++++++++++ 7 files changed, 103 insertions(+), 6 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java index 21436c39a91c1..c7db0da4f96ed 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java @@ -32,6 +32,9 @@ public byte[] getTag() { return this.tag; } + /** + * {@inheritDoc} + */ @Override public void write(final DataOutput out) throws IOException { @@ -45,6 +48,9 @@ public void write(final DataOutput out) throws IOException { } + /** + * {@inheritDoc} + */ @Override public void read(final DataInput in) throws IOException { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java index cae411a1be980..ffb1a01f8d304 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java @@ -15,8 +15,34 @@ package eu.stratosphere.nephele.plugins; +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; + public interface TaskManagerPlugin { + /** + * Registers a new incoming task with this task manager plugin. + * + * @param id + * the ID of the vertex representing the task + * @param jobConfiguration + * the job configuration + * @param environment + * the environment of the task + */ + void registerTask(ExecutionVertexID id, Configuration jobConfiguration, Environment environment); + + /** + * Unregisters a finished, canceled, or failed task from this task manager plugin. + * + * @param id + * the ID of the vertex representing the task + * @param environment + * the environment of the task + */ + void unregisterTask(ExecutionVertexID id, Environment environment); + /** * Called by the task manager to indicate that Nephele is about to shut down. */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 3005abf64e75b..0c66c53219437 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -138,7 +138,7 @@ public class TaskManager implements TaskOperationProtocol { private final HardwareDescription hardwareDescription; private final List taskManagerPlugins; - + /** * Stores whether the task manager has already been shut down. */ @@ -310,7 +310,7 @@ public TaskManager(String configDir) throws Exception { // Load the plugins this.taskManagerPlugins = PluginManager.getTaskManagerPlugins(configDir); - + // Add shutdown hook for clean up tasks Runtime.getRuntime().addShutdownHook(new TaskManagerCleanUp(this)); } @@ -572,6 +572,13 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf } } + // Allow plugins to register their listeners for this task + if (!this.taskManagerPlugins.isEmpty()) { + for (final TaskManagerPlugin plugin : this.taskManagerPlugins) { + plugin.registerTask(id, jobConfiguration, ee); + } + } + // The environment itself will put the task into the running task map return null; @@ -641,6 +648,13 @@ private void unregisterTask(final ExecutionVertexID id, final Task task) { this.memoryManager.releaseAll(task.getEnvironment().getInvokable()); } + // Allow plugins to unregister their listeners for this task + if (!this.taskManagerPlugins.isEmpty()) { + for (final TaskManagerPlugin plugin : this.taskManagerPlugins) { + plugin.unregisterTask(id, task.getEnvironment()); + } + } + // Check if there are still vertices running that belong to the same job int numberOfVerticesBelongingToThisJob = 0; synchronized (this.runningTasks) { @@ -794,10 +808,10 @@ public synchronized void shutdown() { } // Shut down the plugins - for(final TaskManagerPlugin plugin: this.taskManagerPlugins) { + for (final TaskManagerPlugin plugin : this.taskManagerPlugins) { plugin.shutdown(); } - + this.isShutDown = true; } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 26c30c412ca14..d015c19087bca 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -15,12 +15,16 @@ package eu.stratosphere.nephele.streaming; +import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.jobgraph.JobGraph; import eu.stratosphere.nephele.plugins.JobManagerPlugin; public class StreamingJobManagerPlugin implements JobManagerPlugin { + StreamingJobManagerPlugin(final Configuration pluginConfiguration) { + } + /** * {@inheritDoc} */ diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java index c6ab84cb50e3c..9dd4337ffe10d 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java @@ -37,7 +37,7 @@ public StreamingPluginLoader(final Configuration pluginConfiguration) { public synchronized JobManagerPlugin getJobManagerPlugin() { if (this.jobManagerPlugin == null) { - this.jobManagerPlugin = new StreamingJobManagerPlugin(); + this.jobManagerPlugin = new StreamingJobManagerPlugin(getPluginConfiguration()); } return this.jobManagerPlugin; @@ -50,7 +50,7 @@ public synchronized JobManagerPlugin getJobManagerPlugin() { public synchronized TaskManagerPlugin getTaskManagerPlugin() { if (this.taskManagerPlugin == null) { - this.taskManagerPlugin = new StreamingTaskManagerPlugin(); + this.taskManagerPlugin = new StreamingTaskManagerPlugin(getPluginConfiguration()); } return this.taskManagerPlugin; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index d2fd6d8d9dec4..f5efc43d59a22 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -15,10 +15,17 @@ package eu.stratosphere.nephele.streaming; +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; public class StreamingTaskManagerPlugin implements TaskManagerPlugin { + StreamingTaskManagerPlugin(final Configuration pluginConfiguration) { + System.out.println("Task Manager plugin loaded"); + } + /** * {@inheritDoc} */ @@ -28,4 +35,23 @@ public void shutdown() { } + /** + * {@inheritDoc} + */ + @Override + public void registerTask(final ExecutionVertexID id, final Configuration jobConfiguration, + final Environment environment) { + // TODO Auto-generated method stub + + } + + /** + * {@inheritDoc} + */ + @Override + public void unregisterTask(final ExecutionVertexID id, final Environment environment) { + // TODO Auto-generated method stub + + } + } diff --git a/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java b/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java index 7764a2d43e5cd..82feb9d9bd2aa 100644 --- a/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java +++ b/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java @@ -16,6 +16,8 @@ package eu.stratosphere.score; import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; public final class ScoreTaskManagerPlugin implements TaskManagerPlugin { @@ -30,4 +32,23 @@ public final class ScoreTaskManagerPlugin implements TaskManagerPlugin { public void shutdown() { // TODO Auto-generated method stub } + + /** + * {@inheritDoc} + */ + @Override + public void registerTask(final ExecutionVertexID id, final Configuration jobConfiguration, + final Environment environment) { + // TODO Auto-generated method stub + + } + + /** + * {@inheritDoc} + */ + @Override + public void unregisterTask(final ExecutionVertexID id, final Environment environment) { + // TODO Auto-generated method stub + + } } From fc67fa6c39729a8f88fde6e18e4e1a3d71c6977b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 17 Nov 2011 16:16:52 +0000 Subject: [PATCH 041/310] Implemented streaming input and output listeners --- .../eu/stratosphere/nephele/io/InputGate.java | 10 ++++-- .../nephele/io/InputGateListener.java | 10 ++++++ .../stratosphere/nephele/io/OutputGate.java | 18 ++++++---- .../nephele/io/OutputGateListener.java | 10 ++++++ .../profiling/impl/InputGateListenerImpl.java | 10 ++++++ .../impl/OutputGateListenerImpl.java | 9 +++++ .../streaming/StreamingTaskListener.java | 33 +++++++++++++++++++ .../streaming/StreamingTaskManagerPlugin.java | 18 ++++++++-- 8 files changed, 107 insertions(+), 11 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java index b10e64158af18..6c15d45e41e87 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java @@ -352,6 +352,12 @@ record = this.getInputChannel(this.channelToReadFrom).readRecord(target); } } + if (this.inputGateListeners != null) { + for (final InputGateListener inputGateListener : this.inputGateListeners) { + inputGateListener.recordReceived(record); + } + } + return record; } @@ -427,8 +433,8 @@ public void read(DataInput in) throws IOException { AbstractInputChannel eic = null; try { final Constructor> constructor = (Constructor>) c - .getDeclaredConstructor(this.getClass(), int.class, RecordDeserializer.class, ChannelID.class, - CompressionLevel.class); + .getDeclaredConstructor(this.getClass(), int.class, RecordDeserializer.class, ChannelID.class, + CompressionLevel.class); if (constructor == null) { throw new IOException("Constructor is null!"); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGateListener.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGateListener.java index fca80760e25dd..312f1da5b7ac7 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGateListener.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGateListener.java @@ -15,6 +15,8 @@ package eu.stratosphere.nephele.io; +import eu.stratosphere.nephele.types.Record; + /** * This listener interface can be used to obtain information * about the utilization of the attached {@link InputGate}. @@ -28,4 +30,12 @@ public interface InputGateListener { * wait because none of its channels can currently deliver new data. */ void waitingForAnyChannel(); + + /** + * This method is called by the {@link InputGate} whenever it is about to pass a new record to the task. + * + * @param record + * the record which is about to be passed to the application + */ + void recordReceived(final Record record); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java index 0da6e741629a9..a8b39926f8c8a 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java @@ -73,7 +73,7 @@ public class OutputGate extends AbstractGate { * The class of the record transported through this output gate. */ private final Class type; - + /** * The listener objects registered for this output gate. */ @@ -114,7 +114,7 @@ public OutputGate(final JobID jobID, final GateID gateID, final Class inputCl this.isBroadcast = isBroadcast; this.type = inputClass; - + if (this.isBroadcast) { this.channelSelector = null; } else { @@ -134,7 +134,7 @@ public OutputGate(final JobID jobID, final GateID gateID, final Class inputCl public final Class getType() { return this.type; } - + /** * Adds a new output channel to the output gate. * @@ -329,9 +329,9 @@ public void requestClose() throws IOException, InterruptedException { */ @Override public boolean isClosed() throws IOException, InterruptedException { - + boolean allClosed = true; - + for (int i = 0; i < this.getNumberOfOutputChannels(); i++) { final AbstractOutputChannel outputChannel = this.getOutputChannel(i); if (!outputChannel.isClosed()) { @@ -362,6 +362,12 @@ public void writeRecord(T record) throws IOException, InterruptedException { throw new InterruptedException(); } + if (this.outputGateListeners != null) { + for (final OutputGateListener outputGateListener : this.outputGateListeners) { + outputGateListener.recordEmitted(record); + } + } + if (this.isBroadcast) { if (getChannelType() == ChannelType.INMEMORY) { @@ -439,7 +445,7 @@ public void read(DataInput in) throws IOException { constructor.setAccessible(true); eoc = constructor.newInstance(this, i, channelID, compressionLevel); - + } catch (InstantiationException e) { LOG.error(e); } catch (IllegalArgumentException e) { diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGateListener.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGateListener.java index 8e6a0ba0030ef..fae653edd77b5 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGateListener.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGateListener.java @@ -15,7 +15,17 @@ package eu.stratosphere.nephele.io; +import eu.stratosphere.nephele.types.Record; + public interface OutputGateListener { void channelCapacityExhausted(int channelIndex); + + /** + * This method is called to indicate that a record has just been emitted by the task. + * + * @param record + * the record which has just been emitted + */ + void recordEmitted(final Record record); } diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/InputGateListenerImpl.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/InputGateListenerImpl.java index 77562785a8e4f..101bc2b947766 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/InputGateListenerImpl.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/InputGateListenerImpl.java @@ -18,6 +18,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.InputGateListener; import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.types.Record; public class InputGateListenerImpl implements InputGateListener { @@ -61,4 +62,13 @@ public ExecutionVertexID getExecutionVertexID() { public int getGateIndex() { return this.gateIndex; } + + /** + * {@inheritDoc} + */ + @Override + public void recordReceived(final Record record) { + // Nothing to do here + + } } diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/OutputGateListenerImpl.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/OutputGateListenerImpl.java index 3715fe905ef7d..be79f53d4895e 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/OutputGateListenerImpl.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/OutputGateListenerImpl.java @@ -18,6 +18,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.OutputGateListener; import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.types.Record; public class OutputGateListenerImpl implements OutputGateListener { @@ -59,4 +60,12 @@ public int getAndResetCounter() { return retval; } + + /** + * {@inheritDoc} + */ + @Override + public void recordEmitted(final Record record) { + // Nothing to do here + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java new file mode 100644 index 0000000000000..359aa5b521c8b --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java @@ -0,0 +1,33 @@ +package eu.stratosphere.nephele.streaming; + +import eu.stratosphere.nephele.io.InputGateListener; +import eu.stratosphere.nephele.io.OutputGateListener; +import eu.stratosphere.nephele.types.Record; + +public class StreamingTaskListener implements InputGateListener, OutputGateListener { + + @Override + public void channelCapacityExhausted(int channelIndex) { + // TODO Auto-generated method stub + + } + + @Override + public void recordEmitted(Record record) { + // TODO Auto-generated method stub + + } + + @Override + public void waitingForAnyChannel() { + // TODO Auto-generated method stub + + } + + @Override + public void recordReceived(Record record) { + // TODO Auto-generated method stub + + } + +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index f5efc43d59a22..144442cbd600c 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -18,12 +18,14 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; +import eu.stratosphere.nephele.types.Record; public class StreamingTaskManagerPlugin implements TaskManagerPlugin { StreamingTaskManagerPlugin(final Configuration pluginConfiguration) { - System.out.println("Task Manager plugin loaded"); } /** @@ -41,8 +43,18 @@ public void shutdown() { @Override public void registerTask(final ExecutionVertexID id, final Configuration jobConfiguration, final Environment environment) { - // TODO Auto-generated method stub + final StreamingTaskListener listener = new StreamingTaskListener(); + + for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { + final OutputGate outputGate = environment.getOutputGate(i); + outputGate.registerOutputGateListener(listener); + } + + for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { + final InputGate inputGate = environment.getInputGate(i); + inputGate.registerInputGateListener(listener); + } } /** @@ -50,8 +62,8 @@ public void registerTask(final ExecutionVertexID id, final Configuration jobConf */ @Override public void unregisterTask(final ExecutionVertexID id, final Environment environment) { - // TODO Auto-generated method stub + // Nothing to do here } } From 06e16c12ef8170fe437a808dff147ed1338c914f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Thu, 17 Nov 2011 19:07:53 +0100 Subject: [PATCH 042/310] added javanature and builder to streaming project --- nephele/nephele-streaming/.project | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 nephele/nephele-streaming/.project diff --git a/nephele/nephele-streaming/.project b/nephele/nephele-streaming/.project new file mode 100644 index 0000000000000..7b8c938718e6b --- /dev/null +++ b/nephele/nephele-streaming/.project @@ -0,0 +1,23 @@ + + + nephele-streaming + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + From 26ebd70e6cbc07baf39ff7dfb111d317f794727d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 18 Nov 2011 13:39:29 +0000 Subject: [PATCH 043/310] Started to implement communication infrastructure for Nephele plugins --- .../stratosphere/nephele/io/AbstractID.java | 2 +- .../nephele/jobmanager/JobManager.java | 56 ++++--- .../nephele/plugins/AbstractPluginLoader.java | 47 +++++- .../plugins/JobManagerLookupService.java | 30 ++++ .../nephele/plugins/PluginCommunication.java | 32 ++++ .../nephele/plugins/PluginID.java | 43 +++++ .../nephele/plugins/PluginLookupService.java | 21 +++ .../nephele/plugins/PluginManager.java | 41 +++-- .../plugins/TaskManagerLookupService.java | 89 ++++++++++ .../PluginCommunicationProtocol.java | 33 ++++ .../nephele/taskmanager/TaskManager.java | 106 +++++++++++- .../streaming/StreamingPluginLoader.java | 21 ++- .../streaming/StreamingTaskListener.java | 152 ++++++++++++++++-- .../streaming/StreamingTaskManagerPlugin.java | 51 +++++- .../stratosphere/score/ScorePluginLoader.java | 24 ++- 15 files changed, 684 insertions(+), 64 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginCommunication.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginID.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginLookupService.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/PluginCommunicationProtocol.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java index 5e7438b94d97c..774d9881fe1fe 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java @@ -32,7 +32,7 @@ public abstract class AbstractID implements IOReadableWritable { /** * The size of the ID in byte. */ - private static final int SIZE = 16; + protected static final int SIZE = 16; /** * The buffer storing the actual ID. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 477aed950678e..1bf7d559e748c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -106,6 +106,7 @@ import eu.stratosphere.nephele.managementgraph.ManagementVertexID; import eu.stratosphere.nephele.multicast.MulticastManager; import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.plugins.PluginID; import eu.stratosphere.nephele.plugins.PluginManager; import eu.stratosphere.nephele.profiling.JobManagerProfiler; import eu.stratosphere.nephele.profiling.ProfilingUtils; @@ -157,7 +158,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol private final CheckpointDecisionCoordinator checkpointDecisionCoordinator; - private final List jobManagerPlugins; + private final Map jobManagerPlugins; private final int recommendedClientPollingInterval; @@ -340,12 +341,12 @@ public synchronized void shutdown() { // Stop the executor service if (this.executorService != null) { this.executorService.shutdown(); - } // Stop the plugins - for (JobManagerPlugin plugin : this.jobManagerPlugins) { - plugin.shutdown(); + final Iterator it = this.jobManagerPlugins.values().iterator(); + while (it.hasNext()) { + it.next().shutdown(); } // Stop and clean up the job progress collector @@ -441,7 +442,7 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { } LOG.debug("The graph of job " + job.getName() + " is acyclic"); - + // Check constrains on degree jv = job.areVertexDegreesCorrect(); if (jv != null) { @@ -462,18 +463,22 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { LOG.debug("The dependency chain for instance sharing is acyclic"); // Allow plugins to rewrite the job graph - for(final JobManagerPlugin plugin : this.jobManagerPlugins) { + Iterator it = this.jobManagerPlugins.values().iterator(); + while (it.hasNext()) { + + final JobManagerPlugin plugin = it.next(); + final JobGraph inputJob = job; job = plugin.rewriteJobGraph(inputJob); - if(job == null) { + if (job == null) { LOG.warn("Plugin " + plugin + " set job graph to null, reverting changes..."); job = inputJob; } - if(job != inputJob) { + if (job != inputJob) { LOG.debug("Plugin " + plugin + " rewrote job graph"); } } - + // Try to create initial execution graph from job graph LOG.info("Creating initial execution graph from job graph " + job.getName()); ExecutionGraph eg = null; @@ -486,19 +491,22 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { } // Allow plugins to rewrite the execution graph - for(final JobManagerPlugin plugin : this.jobManagerPlugins) { + it = this.jobManagerPlugins.values().iterator(); + while (it.hasNext()) { + + final JobManagerPlugin plugin = it.next(); + final ExecutionGraph inputGraph = eg; eg = plugin.rewriteExecutionGraph(inputGraph); - if(eg == null) { + if (eg == null) { LOG.warn("Plugin " + plugin + " set execution graph to null, reverting changes..."); eg = inputGraph; } - if(eg != inputGraph) { + if (eg != inputGraph) { LOG.debug("Plugin " + plugin + " rewrote execution graph"); } } - - + // Check if profiling should be enabled for this job boolean profilingEnabled = false; if (this.profiler != null && job.getJobConfiguration().getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) { @@ -681,7 +689,7 @@ private TaskCancelResult cancelJob(final ExecutionGraph eg) { * Cancel all nodes in the current and upper execution stages. */ final Iterator it = new ExecutionGraphIterator(eg, eg.getIndexOfCurrentExecutionStage(), - false, true); + false, true); while (it.hasNext()) { final ExecutionVertex vertex = it.next(); @@ -736,7 +744,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio final AbstractInstance assignedInstance = connectedVertex.getAllocatedResource().getInstance(); if (assignedInstance == null) { LOG.error("Cannot resolve lookup: vertex found for channel ID " + connectedChannelID - + " but no instance assigned"); + + " but no instance assigned"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } @@ -756,7 +764,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio } else { // Receiver runs on a different task manager return ConnectionInfoLookupResponse.createReceiverFoundAndReady(assignedInstance - .getInstanceConnectionInfo()); + .getInstanceConnectionInfo()); } } @@ -782,19 +790,19 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio final AbstractInstance assignedInstance = targetVertex.getAllocatedResource().getInstance(); if (assignedInstance == null) { LOG.error("Cannot resolve lookup: vertex found for channel ID " - + outputChannel.getConnectedChannelID() - + " but no instance assigned"); + + outputChannel.getConnectedChannelID() + + " but no instance assigned"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } if (assignedInstance.getInstanceConnectionInfo().equals(caller)) { // Receiver runs on the same task manager return ConnectionInfoLookupResponse.createReceiverFoundAndReady(outputChannel - .getConnectedChannelID()); + .getConnectedChannelID()); } else { // Receiver runs on a different task manager return ConnectionInfoLookupResponse.createReceiverFoundAndReady(assignedInstance - .getInstanceConnectionInfo()); + .getInstanceConnectionInfo()); } } @@ -952,7 +960,7 @@ private void removeAllCheckpoints(final ExecutionGraph executionGraph) { // Finally, trigger the removal of the checkpoints at each instance final Iterator>> it2 = instanceMap - .entrySet().iterator(); + .entrySet().iterator(); while (it2.hasNext()) { final Map.Entry> entry = it2.next(); @@ -1015,7 +1023,7 @@ public void jobStatusHasChanged(final ExecutionGraph executionGraph, final Inter } if (newJobStatus == InternalJobStatus.CANCELED || newJobStatus == InternalJobStatus.FAILED - || newJobStatus == InternalJobStatus.FINISHED) { + || newJobStatus == InternalJobStatus.FINISHED) { // Unregister job for Nephele's monitoring, optimization components, and dynamic input split assignment unregisterJob(executionGraph); } @@ -1044,7 +1052,7 @@ public void logBufferUtilization(final JobID jobID) throws IOException { if (instance instanceof DummyInstance) { LOG.error("Found instance of type DummyInstance for vertex " + vertex.getName() + " (state " - + state + ")"); + + state + ")"); continue; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java index ab7040729382c..f70c9d4dcb02c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/AbstractPluginLoader.java @@ -27,19 +27,37 @@ */ public abstract class AbstractPluginLoader { + /** + * The name of the configuration as specified in the plugin configuration file + */ + private final String pluginName; + /** * The configuration for this plugin. */ private final Configuration pluginConfiguration; + /** + * A service to lookup the location of the plugin's remote components. + */ + private final PluginLookupService pluginLookupService; + /** * Constructs the plugin loader. * + * @param pluginName + * the name of the plugin as specified in the plugin configuration file * @param pluginConfiguration * the plugin configuration + * @param pluginLookupService + * a service to lookup the location of the plugin's remote components */ - public AbstractPluginLoader(final Configuration pluginConfiguration) { + public AbstractPluginLoader(final String pluginName, final Configuration pluginConfiguration, + final PluginLookupService pluginLookupService) { + + this.pluginName = pluginName; this.pluginConfiguration = pluginConfiguration; + this.pluginLookupService = pluginLookupService; } /** @@ -52,6 +70,33 @@ protected final Configuration getPluginConfiguration() { return this.pluginConfiguration; } + /** + * Returns the name of the plugin as specified in the plugin configuration file. + * + * @return the anem of the plugin as specified in the plugin configuration file + */ + final String getPluginName() { + + return this.pluginName; + } + + /** + * Returns a service through which a plugin can determine the location of its remote components. + * + * @return a service through which a plugin can determine the location of its remote components + */ + protected final PluginLookupService getPluginLookupService() { + + return this.pluginLookupService; + } + + /** + * Returns an ID which uniquely specifies the plugin. + * + * @return an ID which uniquely specified the plugin + */ + public abstract PluginID getPluginID(); + /** * Loads and returns the plugin component which is supposed to run inside Nephele's {@link JobManager}. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java new file mode 100644 index 0000000000000..a4ca06a5f3b14 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java @@ -0,0 +1,30 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.plugins; + +public final class JobManagerLookupService implements PluginLookupService { + + /** + * {@inheritDoc} + */ + @Override + public PluginCommunication getJobManagerComponent(final PluginID pluginID) { + + throw new IllegalStateException( + "getJobManagerComponent must not be called from the plugin component running on the job manager"); + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginCommunication.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginCommunication.java new file mode 100644 index 0000000000000..d47ae8a3743ad --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginCommunication.java @@ -0,0 +1,32 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.plugins; + +import java.io.IOException; +import java.util.List; + +import eu.stratosphere.nephele.io.IOReadableWritable; + +public interface PluginCommunication { + + void sendData(IOReadableWritable data) throws IOException; + + void sendData(List listOfData) throws IOException; + + IOReadableWritable requestData(IOReadableWritable data) throws IOException; + + IOReadableWritable requestData(List listOfData) throws IOException; +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginID.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginID.java new file mode 100644 index 0000000000000..3dd105e033fad --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginID.java @@ -0,0 +1,43 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.plugins; + +import eu.stratosphere.nephele.io.AbstractID; + +/** + * This class provides IDs to uniquely identify Nephele plugins. + * + * @author warneke + */ +public final class PluginID extends AbstractID { + + private PluginID(final byte[] byteArray) { + super(byteArray); + } + + public static PluginID fromByteArray(final byte[] byteArray) { + + if (byteArray == null) { + throw new IllegalArgumentException("Argument byteArray must not be null"); + } + + if (byteArray.length != SIZE) { + throw new IllegalArgumentException("Provided byte array must have a length of " + SIZE); + } + + return new PluginID(byteArray); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginLookupService.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginLookupService.java new file mode 100644 index 0000000000000..06376c4776c88 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginLookupService.java @@ -0,0 +1,21 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.plugins; + +public interface PluginLookupService { + + PluginCommunication getJobManagerComponent(PluginID pluginID); +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java index e276585dfa65f..46d6cf53efa5e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java @@ -19,11 +19,10 @@ import java.io.IOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; -import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; import javax.xml.parsers.DocumentBuilder; @@ -332,44 +331,56 @@ private static synchronized PluginManager getInstance(final String configDir) { return INSTANCE; } - private List getJobManagerPluginsInternal() { + private Map getJobManagerPluginsInternal() { - final List jobManagerPluginList = new ArrayList(); + final Map jobManagerPluginMap = new HashMap(); final Iterator it = this.plugins.values().iterator(); while (it.hasNext()) { - final JobManagerPlugin jmp = it.next().getJobManagerPlugin(); + final AbstractPluginLoader apl = it.next(); + final PluginID pluginID = apl.getPluginID(); + final JobManagerPlugin jmp = apl.getJobManagerPlugin(); if (jmp != null) { - jobManagerPluginList.add(jmp); + if (!jobManagerPluginMap.containsKey(pluginID)) { + jobManagerPluginMap.put(pluginID, jmp); + } else { + LOG.error("Detected ID collision for plugin " + apl.getPluginName() + ", skipping it..."); + } } } - return Collections.unmodifiableList(jobManagerPluginList); + return Collections.unmodifiableMap(jobManagerPluginMap); } - private List getTaskManagerPluginsInternal() { + private Map getTaskManagerPluginsInternal() { - final List taskManagerPluginList = new ArrayList(); + final Map taskManagerPluginMap = new HashMap(); final Iterator it = this.plugins.values().iterator(); while (it.hasNext()) { - final TaskManagerPlugin jmp = it.next().getTaskManagerPlugin(); - if (jmp != null) { - taskManagerPluginList.add(jmp); + final AbstractPluginLoader apl = it.next(); + final PluginID pluginID = apl.getPluginID(); + final TaskManagerPlugin tmp = apl.getTaskManagerPlugin(); + if (tmp != null) { + if (!taskManagerPluginMap.containsKey(pluginID)) { + taskManagerPluginMap.put(pluginID, tmp); + } else { + LOG.error("Detected ID collision for plugin " + apl.getPluginName() + ", skipping it..."); + } } } - return Collections.unmodifiableList(taskManagerPluginList); + return Collections.unmodifiableMap(taskManagerPluginMap); } - public static List getJobManagerPlugins(final String configDir) { + public static Map getJobManagerPlugins(final String configDir) { return getInstance(configDir).getJobManagerPluginsInternal(); } - public static List getTaskManagerPlugins(final String configDir) { + public static Map getTaskManagerPlugins(final String configDir) { return getInstance(configDir).getTaskManagerPluginsInternal(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java new file mode 100644 index 0000000000000..85f3a2d3a289e --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java @@ -0,0 +1,89 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.plugins; + +import java.io.IOException; +import java.util.List; + +import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.taskmanager.TaskManager; + +public final class TaskManagerLookupService implements PluginLookupService { + + private final TaskManager taskManager; + + public TaskManagerLookupService(final TaskManager taskManager) { + this.taskManager = taskManager; + } + + private static final class JobManagerStub implements PluginCommunication { + + private final TaskManager taskManager; + + private final PluginID pluginID; + + public JobManagerStub(final TaskManager taskManager, final PluginID pluginID) { + this.taskManager = taskManager; + this.pluginID = pluginID; + } + + /** + * {@inheritDoc} + */ + @Override + public void sendData(final IOReadableWritable data) throws IOException { + + this.taskManager.sendData(this.pluginID, data); + } + + /** + * {@inheritDoc} + */ + @Override + public void sendData(final List listOfData) throws IOException { + + this.taskManager.sendData(this.pluginID, listOfData); + } + + /** + * {@inheritDoc} + */ + @Override + public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { + + return this.taskManager.requestData(this.pluginID, data); + } + + /** + * {@inheritDoc} + */ + @Override + public IOReadableWritable requestData(final List listOfData) throws IOException { + + return this.taskManager.requestData(this.pluginID, listOfData); + } + + } + + /** + * {@inheritDoc} + */ + @Override + public PluginCommunication getJobManagerComponent(final PluginID pluginID) { + + return new JobManagerStub(this.taskManager, pluginID); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/PluginCommunicationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/PluginCommunicationProtocol.java new file mode 100644 index 0000000000000..515cd1e6c5e1c --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/PluginCommunicationProtocol.java @@ -0,0 +1,33 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.protocols; + +import java.io.IOException; +import java.util.List; + +import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.plugins.PluginID; + +public interface PluginCommunicationProtocol extends VersionedProtocol { + + void sendData(PluginID pluginID, IOReadableWritable data) throws IOException; + + void sendData(PluginID pluginID, List listOfData) throws IOException; + + IOReadableWritable requestData(PluginID pluginID, IOReadableWritable data) throws IOException; + + IOReadableWritable requestData(PluginID pluginID, List listOfData) throws IOException; +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 0c66c53219437..2d9f489f2b4e0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -58,11 +58,13 @@ import eu.stratosphere.nephele.instance.HardwareDescription; import eu.stratosphere.nephele.instance.HardwareDescriptionFactory; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; +import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.ipc.RPC; import eu.stratosphere.nephele.ipc.Server; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.net.NetUtils; +import eu.stratosphere.nephele.plugins.PluginID; import eu.stratosphere.nephele.plugins.PluginManager; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; import eu.stratosphere.nephele.profiling.ProfilingUtils; @@ -70,6 +72,7 @@ import eu.stratosphere.nephele.protocols.ChannelLookupProtocol; import eu.stratosphere.nephele.protocols.InputSplitProviderProtocol; import eu.stratosphere.nephele.protocols.JobManagerProtocol; +import eu.stratosphere.nephele.protocols.PluginCommunicationProtocol; import eu.stratosphere.nephele.protocols.TaskOperationProtocol; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; @@ -97,6 +100,8 @@ public class TaskManager implements TaskOperationProtocol { private final ChannelLookupProtocol lookupService; + private final PluginCommunicationProtocol pluginCommunicationService; + private static final int handlerCount = 1; private final Server taskManagerServer; @@ -137,7 +142,7 @@ public class TaskManager implements TaskOperationProtocol { private final HardwareDescription hardwareDescription; - private final List taskManagerPlugins; + private final Map taskManagerPlugins; /** * Stores whether the task manager has already been shut down. @@ -235,6 +240,17 @@ public TaskManager(String configDir) throws Exception { } this.lookupService = lookupService; + // Try to create local stub for the plugin communication service + PluginCommunicationProtocol pluginCommunicationService = null; + try { + pluginCommunicationService = (PluginCommunicationProtocol) RPC.getProxy(PluginCommunicationProtocol.class, + jobManagerAddress, NetUtils.getSocketFactory()); + } catch (IOException e) { + LOG.error(StringUtils.stringifyException(e)); + throw new Exception("Failed to initialize plugin communication protocol. " + e.getMessage(), e); + } + this.pluginCommunicationService = pluginCommunicationService; + // Start local RPC server Server taskManagerServer = null; try { @@ -574,8 +590,9 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf // Allow plugins to register their listeners for this task if (!this.taskManagerPlugins.isEmpty()) { - for (final TaskManagerPlugin plugin : this.taskManagerPlugins) { - plugin.registerTask(id, jobConfiguration, ee); + final Iterator it = this.taskManagerPlugins.values().iterator(); + while (it.hasNext()) { + it.next().registerTask(id, jobConfiguration, ee); } } @@ -650,8 +667,9 @@ private void unregisterTask(final ExecutionVertexID id, final Task task) { // Allow plugins to unregister their listeners for this task if (!this.taskManagerPlugins.isEmpty()) { - for (final TaskManagerPlugin plugin : this.taskManagerPlugins) { - plugin.unregisterTask(id, task.getEnvironment()); + final Iterator it = this.taskManagerPlugins.values().iterator(); + while (it.hasNext()) { + it.next().unregisterTask(id, task.getEnvironment()); } } @@ -808,8 +826,9 @@ public synchronized void shutdown() { } // Shut down the plugins - for (final TaskManagerPlugin plugin : this.taskManagerPlugins) { - plugin.shutdown(); + final Iterator it = this.taskManagerPlugins.values().iterator(); + while (it.hasNext()) { + it.next().shutdown(); } this.isShutDown = true; @@ -906,4 +925,77 @@ public void run() { timer.schedule(timerTask, 10L); } + + /** + * Sends data from the plugin with the given ID to the respective component of the plugin running at the job + * manager. + * + * @param pluginID + * the ID of plugin + * @param data + * the data to be sent + * @throws IOException + * thrown if an I/O error occurs during the RPC call + */ + public void sendData(final PluginID pluginID, final IOReadableWritable data) throws IOException { + + synchronized (this.pluginCommunicationService) { + this.pluginCommunicationService.sendData(pluginID, data); + } + } + + /** + * Sends a list of data from the plugin with the given ID to the respective component of the plugin running at the + * job manager. + * + * @param pluginID + * the ID of plugin + * @param listOfData + * the data to be sent + * @throws IOException + * thrown if an I/O error occurs during the RPC call + */ + public void sendData(final PluginID pluginID, final List listOfData) throws IOException { + + synchronized (this.pluginCommunicationService) { + this.pluginCommunicationService.sendData(pluginID, listOfData); + } + } + + /** + * Requests data for the plugin with the given ID from the respective plugin component running at the job manager. + * + * @param pluginID + * the ID of the plugin + * @param data + * the data to specify the request + * @return the requested data + * @throws IOException + * thrown if an I/O error occurs during the RPC call + */ + public IOReadableWritable requestData(final PluginID pluginID, final IOReadableWritable data) throws IOException { + + synchronized (this.pluginCommunicationService) { + return this.pluginCommunicationService.requestData(pluginID, data); + } + } + + /** + * Requests data for the plugin with the given ID from the respective plugin component running at the job manager. + * + * @param pluginID + * the ID of the plugin + * @param listOfData + * the list of data to specify the request + * @return the requested data + * @throws IOException + * thrown if an I/O error occurs during the RPC call + */ + public IOReadableWritable requestData(final PluginID pluginID, final List listOfData) + throws IOException { + + synchronized (this.pluginCommunicationService) { + return this.pluginCommunicationService.requestData(pluginID, listOfData); + } + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java index 9dd4337ffe10d..feb0658d5443f 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java @@ -18,6 +18,8 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.plugins.AbstractPluginLoader; import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.plugins.PluginID; +import eu.stratosphere.nephele.plugins.PluginLookupService; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; public class StreamingPluginLoader extends AbstractPluginLoader { @@ -26,8 +28,14 @@ public class StreamingPluginLoader extends AbstractPluginLoader { private StreamingTaskManagerPlugin taskManagerPlugin = null; - public StreamingPluginLoader(final Configuration pluginConfiguration) { - super(pluginConfiguration); + private final PluginID pluginID; + + public StreamingPluginLoader(final String pluginName, final Configuration pluginConfiguration, + final PluginLookupService pluginLookupService) { + super(pluginName, pluginConfiguration, pluginLookupService); + + this.pluginID = PluginID.fromByteArray(new byte[] { 0x3c, 0x00, 0x00, -0x1b, 0x38, 0x4a, 0x60, -0x61, -0x25, + 0x00, 0x00, 0x16, 0x00, 0x18, 0x7f, 0x01 }); } /** @@ -56,4 +64,13 @@ public synchronized TaskManagerPlugin getTaskManagerPlugin() { return this.taskManagerPlugin; } + /** + * {@inheritDoc} + */ + @Override + public PluginID getPluginID() { + + return this.pluginID; + } + } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java index 359aa5b521c8b..00c6bea3396d9 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java @@ -1,33 +1,165 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; import eu.stratosphere.nephele.io.InputGateListener; import eu.stratosphere.nephele.io.OutputGateListener; +import eu.stratosphere.nephele.types.AbstractTaggableRecord; import eu.stratosphere.nephele.types.Record; -public class StreamingTaskListener implements InputGateListener, OutputGateListener { +public final class StreamingTaskListener implements InputGateListener, OutputGateListener { + + private final static int SIZEOFLONG = 8; + + private static enum TaskType { + INPUT, REGULAR, OUTPUT + }; + + private final TaskType taskType; + + private final int taggingInterval; + + private final int aggregationInterval; + + private byte[] tag = null; + + private int tagCounter = 0; + + private int aggregationCounter = 0; + + private double aggregatedValue = -1.0; + + static StreamingTaskListener createForInputTask(final int taggingInterval, final int aggregationInterval) { + + return new StreamingTaskListener(TaskType.INPUT, taggingInterval, aggregationInterval); + } + + static StreamingTaskListener createForRegularTask(final int aggregationInterval) { + + return new StreamingTaskListener(TaskType.REGULAR, 0, aggregationInterval); + } + + static StreamingTaskListener createForOutputTask(final int aggregationInterval) { + + return new StreamingTaskListener(TaskType.OUTPUT, 0, aggregationInterval); + } + + private StreamingTaskListener(final TaskType taskType, final int taggingInterval, final int aggregationInterval) { + + this.taskType = taskType; + this.taggingInterval = taggingInterval; + this.aggregationInterval = aggregationInterval; + } + /** + * {@inheritDoc} + */ @Override - public void channelCapacityExhausted(int channelIndex) { + public void channelCapacityExhausted(final int channelIndex) { // TODO Auto-generated method stub - + } + /** + * {@inheritDoc} + */ @Override - public void recordEmitted(Record record) { - // TODO Auto-generated method stub - + public void recordEmitted(final Record record) { + + switch (this.taskType) { + case INPUT: + if (this.tagCounter++ == this.taggingInterval) { + final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; + taggableRecord.setTag(createTag()); + this.tagCounter = 0; + } + break; + case REGULAR: + final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; + taggableRecord.setTag(this.tag); + break; + case OUTPUT: + throw new IllegalStateException("Output task emitted record"); + } + } + /** + * {@inheritDoc} + */ @Override public void waitingForAnyChannel() { // TODO Auto-generated method stub - + } + /** + * {@inheritDoc} + */ @Override - public void recordReceived(Record record) { - // TODO Auto-generated method stub - + public void recordReceived(final Record record) { + + switch (this.taskType) { + case INPUT: + throw new IllegalStateException("Input task received record"); + case REGULAR: { + final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; + this.tag = taggableRecord.getTag(); + } + break; + case OUTPUT: { + final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; + this.tag = taggableRecord.getTag(); + if (this.tag != null) { + + System.out.println(System.currentTimeMillis() - byteArrayToLong(this.tag)); + } + } + break; + } + } + private byte[] createTag() { + + if (this.tag == null) { + this.tag = new byte[SIZEOFLONG]; + } + + longToByteArray(System.currentTimeMillis(), this.tag); + + return this.tag; + } + + private static void longToByteArray(final long longToSerialize, final byte[] buffer) { + + for (int i = 0; i < SIZEOFLONG; ++i) { + final int shift = i << 3; // i * 8 + buffer[(SIZEOFLONG - 1) - i] = (byte) ((longToSerialize & (0xffL << shift)) >>> shift); + } + } + + private static long byteArrayToLong(final byte[] buffer) { + + long l = 0; + + for (int i = 0; i < SIZEOFLONG; ++i) { + l |= (buffer[(SIZEOFLONG - 1) - i] & 0xffL) << (i << 3); + } + + return l; + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index 144442cbd600c..9af3e3342dff3 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -25,7 +25,42 @@ public class StreamingTaskManagerPlugin implements TaskManagerPlugin { + /** + * Provides access to the configuration entry which defines the interval in which records shall be tagged. + */ + private static final String TAGGING_INTERVAL_KEY = "streaming.tagging.interval"; + + /** + * The default tagging interval. + */ + private static final int DEFAULT_TAGGING_INTERVAL = 10; + + /** + * Provides access to the configuration entry which defines the interval in which received tags shall be aggregated + * and sent to the job manager plugin component. + */ + private static final String AGGREGATION_INTERVAL_KEY = "streaming.aggregation.interval"; + + /** + * The default aggregation interval. + */ + private static final int DEFAULT_AGGREGATION_INTERVAL = 10; + + /** + * The tagging interval as specified in the plugin configuration. + */ + private final int taggingInterval; + + /** + * The aggregation interval as specified in the plugin configuration. + */ + private final int aggregationInterval; + StreamingTaskManagerPlugin(final Configuration pluginConfiguration) { + + this.taggingInterval = pluginConfiguration.getInteger(TAGGING_INTERVAL_KEY, DEFAULT_TAGGING_INTERVAL); + this.aggregationInterval = pluginConfiguration.getInteger(AGGREGATION_INTERVAL_KEY, + DEFAULT_AGGREGATION_INTERVAL); } /** @@ -44,7 +79,21 @@ public void shutdown() { public void registerTask(final ExecutionVertexID id, final Configuration jobConfiguration, final Environment environment) { - final StreamingTaskListener listener = new StreamingTaskListener(); + // Check if user has provided a job-specific aggregation interval + final int aggregationInterval = jobConfiguration.getInteger(AGGREGATION_INTERVAL_KEY, + this.aggregationInterval); + + StreamingTaskListener listener = null; + if (environment.getNumberOfInputGates() == 0) { + // Check if user has provided a job-specific tagging interval + final int taggingInterval = jobConfiguration.getInteger(TAGGING_INTERVAL_KEY, this.taggingInterval); + + listener = StreamingTaskListener.createForInputTask(taggingInterval, aggregationInterval); + } else if (environment.getNumberOfOutputGates() == 0) { + listener = StreamingTaskListener.createForOutputTask(aggregationInterval); + } else { + listener = StreamingTaskListener.createForRegularTask(aggregationInterval); + } for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { final OutputGate outputGate = environment.getOutputGate(i); diff --git a/score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java b/score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java index 1582a2ff064ba..2ea1cbf3427d8 100644 --- a/score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java +++ b/score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java @@ -18,6 +18,8 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.plugins.AbstractPluginLoader; import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.plugins.PluginID; +import eu.stratosphere.nephele.plugins.PluginLookupService; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; /** @@ -33,10 +35,17 @@ public final class ScorePluginLoader extends AbstractPluginLoader { private ScoreTaskManagerPlugin taskManagerPlugin = null; - public ScorePluginLoader(final Configuration pluginConfiguration) { - super(pluginConfiguration); + private final PluginID pluginID; + + public ScorePluginLoader(final String pluginName, final Configuration pluginConfiguration, + final PluginLookupService pluginLookupService) { + + super(pluginName, pluginConfiguration, pluginLookupService); + + this.pluginID = PluginID.fromByteArray(new byte[] { 0x44, -0x6f, 0x18, -0x1d, 0x00, 0x4a, 0x60, -0x61, -0x25, + 0x00, 0x00, 0x16, 0x00, 0x18, -0x01, 0x62 }); } - + /** * {@inheritDoc} */ @@ -62,4 +71,13 @@ public synchronized TaskManagerPlugin getTaskManagerPlugin() { return this.taskManagerPlugin; } + + /** + * {@inheritDoc} + */ + @Override + public PluginID getPluginID() { + + return this.pluginID; + } } From d7a45a08c4d0663467aacbc4ef4f84adcb7ffc38 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 18 Nov 2011 14:08:58 +0000 Subject: [PATCH 044/310] Finished implementation of the job manager part of the plugin communication infrastructure --- .../nephele/jobmanager/JobManager.java | 35 ++++++++++++++++- .../nephele/plugins/JobManagerPlugin.java | 2 +- .../nephele/plugins/PluginCommunication.java | 5 --- .../nephele/plugins/PluginManager.java | 32 ++++++++++----- .../plugins/TaskManagerLookupService.java | 20 ---------- .../PluginCommunicationProtocol.java | 5 --- .../nephele/taskmanager/TaskManager.java | 39 +------------------ .../streaming/StreamingJobManagerPlugin.java | 22 ++++++++++- .../score/ScoreJobManagerPlugin.java | 22 ++++++++++- 9 files changed, 100 insertions(+), 82 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 1bf7d559e748c..e97b93ffc1fcf 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -90,6 +90,7 @@ import eu.stratosphere.nephele.instance.InstanceType; import eu.stratosphere.nephele.instance.InstanceTypeDescription; import eu.stratosphere.nephele.instance.local.LocalInstanceManager; +import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; @@ -114,6 +115,7 @@ import eu.stratosphere.nephele.protocols.ExtendedManagementProtocol; import eu.stratosphere.nephele.protocols.InputSplitProviderProtocol; import eu.stratosphere.nephele.protocols.JobManagerProtocol; +import eu.stratosphere.nephele.protocols.PluginCommunicationProtocol; import eu.stratosphere.nephele.taskmanager.AbstractTaskResult; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; import eu.stratosphere.nephele.taskmanager.TaskCheckpointState; @@ -138,7 +140,8 @@ * @author warneke */ public class JobManager implements DeploymentManager, ExtendedManagementProtocol, InputSplitProviderProtocol, - JobManagerProtocol, ChannelLookupProtocol, JobStatusListener, CheckpointDecisionPropagator { + JobManagerProtocol, ChannelLookupProtocol, JobStatusListener, CheckpointDecisionPropagator, + PluginCommunicationProtocol { private static final Log LOG = LogFactory.getLog(JobManager.class); @@ -1354,4 +1357,34 @@ public void run() { } } + + /** + * {@inheritDoc} + */ + @Override + public void sendData(final PluginID pluginID, final IOReadableWritable data) throws IOException { + + final JobManagerPlugin jmp = this.jobManagerPlugins.get(pluginID); + if (jmp == null) { + LOG.error("Cannot find job manager plugin for plugin ID " + pluginID); + return; + } + + jmp.sendData(data); + } + + /** + * {@inheritDoc} + */ + @Override + public IOReadableWritable requestData(final PluginID pluginID, final IOReadableWritable data) throws IOException { + + final JobManagerPlugin jmp = this.jobManagerPlugins.get(pluginID); + if (jmp == null) { + LOG.error("Cannot find job manager plugin for plugin ID " + pluginID); + return null; + } + + return jmp.requestData(data); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java index 78f7e7533e9e5..a6e1c63876a9f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java @@ -23,7 +23,7 @@ * * @author warneke */ -public interface JobManagerPlugin { +public interface JobManagerPlugin extends PluginCommunication { /** * This method is called upon the reception of a new job graph. It gives the plugin the possibility to to rewrite diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginCommunication.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginCommunication.java index d47ae8a3743ad..c36995ebcae7a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginCommunication.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginCommunication.java @@ -16,7 +16,6 @@ package eu.stratosphere.nephele.plugins; import java.io.IOException; -import java.util.List; import eu.stratosphere.nephele.io.IOReadableWritable; @@ -24,9 +23,5 @@ public interface PluginCommunication { void sendData(IOReadableWritable data) throws IOException; - void sendData(List listOfData) throws IOException; - IOReadableWritable requestData(IOReadableWritable data) throws IOException; - - IOReadableWritable requestData(List listOfData) throws IOException; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java index 46d6cf53efa5e..f498153a9ed0b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java @@ -39,6 +39,7 @@ import org.xml.sax.SAXException; import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.taskmanager.TaskManager; import eu.stratosphere.nephele.util.StringUtils; /** @@ -67,16 +68,20 @@ public final class PluginManager { private final Map plugins; - private PluginManager(final String configDir) { + private final PluginLookupService pluginLookupService; + + private PluginManager(final String configDir, final PluginLookupService pluginLookupService) { // Check if the configuration file exists final File configFile = new File(configDir + File.separator + PLUGIN_CONFIG_FILE); if (configFile.exists()) { - this.plugins = loadPlugins(configFile); + this.plugins = loadPlugins(configFile, pluginLookupService); } else { this.plugins = Collections.emptyMap(); LOG.warn("Unable to load plugins: configuration file " + configFile.getAbsolutePath() + " not found"); } + + this.pluginLookupService = pluginLookupService; } private String getTextChild(final Node node) { @@ -96,7 +101,8 @@ private String getTextChild(final Node node) { } @SuppressWarnings("unchecked") - private Map loadPlugins(final File configFile) { + private Map loadPlugins(final File configFile, + final PluginLookupService pluginLookupService) { final Map tmpPluginList = new LinkedHashMap(); @@ -270,7 +276,7 @@ private Map loadPlugins(final File configFile) { Constructor constructor; try { constructor = (Constructor) loaderClass - .getConstructor(Configuration.class); + .getConstructor(String.class, Configuration.class, PluginLookupService.class); } catch (SecurityException e) { LOG.error("Unable to load plugin " + pluginName + ": " + StringUtils.stringifyException(e)); continue; @@ -287,7 +293,7 @@ private Map loadPlugins(final File configFile) { AbstractPluginLoader pluginLoader = null; try { - pluginLoader = constructor.newInstance(pluginConfiguration); + pluginLoader = constructor.newInstance(pluginName, pluginConfiguration, this.pluginLookupService); } catch (IllegalArgumentException e) { LOG.error("Unable to load plugin " + pluginName + ": " + StringUtils.stringifyException(e)); continue; @@ -322,10 +328,11 @@ private Map loadPlugins(final File configFile) { return Collections.unmodifiableMap(tmpPluginList); } - private static synchronized PluginManager getInstance(final String configDir) { + private static synchronized PluginManager getInstance(final String configDir, + final PluginLookupService pluginLookupService) { if (INSTANCE == null) { - INSTANCE = new PluginManager(configDir); + INSTANCE = new PluginManager(configDir, pluginLookupService); } return INSTANCE; @@ -377,11 +384,16 @@ private Map getTaskManagerPluginsInternal() { public static Map getJobManagerPlugins(final String configDir) { - return getInstance(configDir).getJobManagerPluginsInternal(); + final JobManagerLookupService lookupService = new JobManagerLookupService(); + + return getInstance(configDir, lookupService).getJobManagerPluginsInternal(); } - public static Map getTaskManagerPlugins(final String configDir) { + public static Map getTaskManagerPlugins(final TaskManager taskManager, + final String configDir) { + + final TaskManagerLookupService lookupService = new TaskManagerLookupService(taskManager); - return getInstance(configDir).getTaskManagerPluginsInternal(); + return getInstance(configDir, lookupService).getTaskManagerPluginsInternal(); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java index 85f3a2d3a289e..22cfdffdd8a30 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java @@ -16,7 +16,6 @@ package eu.stratosphere.nephele.plugins; import java.io.IOException; -import java.util.List; import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.taskmanager.TaskManager; @@ -49,15 +48,6 @@ public void sendData(final IOReadableWritable data) throws IOException { this.taskManager.sendData(this.pluginID, data); } - /** - * {@inheritDoc} - */ - @Override - public void sendData(final List listOfData) throws IOException { - - this.taskManager.sendData(this.pluginID, listOfData); - } - /** * {@inheritDoc} */ @@ -66,16 +56,6 @@ public IOReadableWritable requestData(final IOReadableWritable data) throws IOEx return this.taskManager.requestData(this.pluginID, data); } - - /** - * {@inheritDoc} - */ - @Override - public IOReadableWritable requestData(final List listOfData) throws IOException { - - return this.taskManager.requestData(this.pluginID, listOfData); - } - } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/PluginCommunicationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/PluginCommunicationProtocol.java index 515cd1e6c5e1c..34918fff0d624 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/PluginCommunicationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/PluginCommunicationProtocol.java @@ -16,7 +16,6 @@ package eu.stratosphere.nephele.protocols; import java.io.IOException; -import java.util.List; import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.plugins.PluginID; @@ -25,9 +24,5 @@ public interface PluginCommunicationProtocol extends VersionedProtocol { void sendData(PluginID pluginID, IOReadableWritable data) throws IOException; - void sendData(PluginID pluginID, List listOfData) throws IOException; - IOReadableWritable requestData(PluginID pluginID, IOReadableWritable data) throws IOException; - - IOReadableWritable requestData(PluginID pluginID, List listOfData) throws IOException; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 2d9f489f2b4e0..1b0b3492d834f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -325,7 +325,7 @@ public TaskManager(String configDir) throws Exception { this.ioManager = new IOManager(tmpDirPaths); // Load the plugins - this.taskManagerPlugins = PluginManager.getTaskManagerPlugins(configDir); + this.taskManagerPlugins = PluginManager.getTaskManagerPlugins(this, configDir); // Add shutdown hook for clean up tasks Runtime.getRuntime().addShutdownHook(new TaskManagerCleanUp(this)); @@ -944,24 +944,6 @@ public void sendData(final PluginID pluginID, final IOReadableWritable data) thr } } - /** - * Sends a list of data from the plugin with the given ID to the respective component of the plugin running at the - * job manager. - * - * @param pluginID - * the ID of plugin - * @param listOfData - * the data to be sent - * @throws IOException - * thrown if an I/O error occurs during the RPC call - */ - public void sendData(final PluginID pluginID, final List listOfData) throws IOException { - - synchronized (this.pluginCommunicationService) { - this.pluginCommunicationService.sendData(pluginID, listOfData); - } - } - /** * Requests data for the plugin with the given ID from the respective plugin component running at the job manager. * @@ -979,23 +961,4 @@ public IOReadableWritable requestData(final PluginID pluginID, final IOReadableW return this.pluginCommunicationService.requestData(pluginID, data); } } - - /** - * Requests data for the plugin with the given ID from the respective plugin component running at the job manager. - * - * @param pluginID - * the ID of the plugin - * @param listOfData - * the list of data to specify the request - * @return the requested data - * @throws IOException - * thrown if an I/O error occurs during the RPC call - */ - public IOReadableWritable requestData(final PluginID pluginID, final List listOfData) - throws IOException { - - synchronized (this.pluginCommunicationService) { - return this.pluginCommunicationService.requestData(pluginID, listOfData); - } - } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index d015c19087bca..6e7bd07373e2a 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -15,8 +15,11 @@ package eu.stratosphere.nephele.streaming; +import java.io.IOException; + import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.jobgraph.JobGraph; import eu.stratosphere.nephele.plugins.JobManagerPlugin; @@ -24,7 +27,7 @@ public class StreamingJobManagerPlugin implements JobManagerPlugin { StreamingJobManagerPlugin(final Configuration pluginConfiguration) { } - + /** * {@inheritDoc} */ @@ -52,4 +55,21 @@ public void shutdown() { } + /** + * {@inheritDoc} + */ + @Override + public void sendData(final IOReadableWritable data) throws IOException { + // TODO Auto-generated method stub + + } + + /** + * {@inheritDoc} + */ + @Override + public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { + // TODO Auto-generated method stub + return null; + } } diff --git a/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java b/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java index 282ee1c00e49f..f6a65204f6b8d 100644 --- a/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java +++ b/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java @@ -15,11 +15,13 @@ package eu.stratosphere.score; +import java.io.IOException; import java.util.Iterator; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; +import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.jobgraph.JobGraph; import eu.stratosphere.nephele.plugins.JobManagerPlugin; @@ -62,6 +64,24 @@ public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) @Override public void shutdown() { // TODO Auto-generated method stub - + + } + + /** + * {@inheritDoc} + */ + @Override + public void sendData(final IOReadableWritable data) throws IOException { + // TODO Auto-generated method stub + + } + + /** + * {@inheritDoc} + */ + @Override + public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { + // TODO Auto-generated method stub + return null; } } From 9167028b6e1ba45d933d9401c8fdd8278fdb53d8 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 18 Nov 2011 15:08:14 +0000 Subject: [PATCH 045/310] Implemented thread to asynchronously handle communication between the streaming plugin's components --- .../StreamingCommunicationThread.java | 116 ++++++++++++++++++ .../nephele/streaming/StreamingData.java | 26 ++++ .../streaming/StreamingPluginLoader.java | 3 +- .../streaming/StreamingTaskManagerPlugin.java | 13 +- 4 files changed, 155 insertions(+), 3 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingData.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java new file mode 100644 index 0000000000000..6c8cdcc294e82 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java @@ -0,0 +1,116 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import java.io.IOException; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.plugins.PluginCommunication; +import eu.stratosphere.nephele.util.StringUtils; + +/** + * This class implements a communication thread to handle communication from the task manager plugin component to the + * job manager plugin component in an asynchronous fashion. The main reason for asynchronous communication is not + * influence the processing delay by the RPC call latency. + *

+ * This class is thread-safe. + * + * @author warneke + */ +final class StreamingCommunicationThread extends Thread { + + /** + * The log object. + */ + private static final Log LOG = LogFactory.getLog(StreamingCommunicationThread.class); + + /** + * The capacity of the data queue. + */ + private static final int QUEUE_CAPACITY = 128; + + /** + * Stub object representing the job manager component of this plugin. + */ + private final PluginCommunication jobManagerComponent; + + /** + * The blocking queue which is used to asynchronously exchange data with the job manager component of this plugin. + */ + private final BlockingQueue dataQueue = new ArrayBlockingQueue(QUEUE_CAPACITY); + + /** + * Stores whether the communication thread has been requested to stop. + */ + private volatile boolean interrupted = false; + + /** + * Constructs a new streaming communication thread. + * + * @param jobManagerComponent + * the stub object for the plugin's job manager component. + */ + StreamingCommunicationThread(final PluginCommunication jobManagerComponent) { + this.jobManagerComponent = jobManagerComponent; + } + + /** + * {@inheritDoc} + */ + @Override + public void run() { + + while (!this.interrupted) { + + if (Thread.currentThread().isInterrupted()) { + break; + } + + try { + this.jobManagerComponent.sendData(this.dataQueue.take()); + } catch (InterruptedException e) { + break; + } catch (IOException ioe) { + LOG.error(StringUtils.stringifyException(ioe)); + } + } + } + + /** + * Stops the communication thread. + */ + void stopCommunicationThread() { + this.interrupted = true; + interrupt(); + } + + /** + * Sends the given data item asynchronously to the plugin's job manager component. + * + * @param data + * the data item to send to the plugin's job manager component + * @throws InterruptedException + * thrown if the thread is interrupted while waiting for the communication thread to accept the data + */ + void sendDataAsynchronously(final StreamingData data) throws InterruptedException { + + this.dataQueue.put(data); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingData.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingData.java new file mode 100644 index 0000000000000..5eaec6ae04fc0 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingData.java @@ -0,0 +1,26 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import eu.stratosphere.nephele.io.IOReadableWritable; + +/** + * This is a marker interface to identify data exchanged between the different components of the streaming plugin. + * + * @author warneke + */ +public interface StreamingData extends IOReadableWritable { +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java index feb0658d5443f..d4f50527c0e9d 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java @@ -58,7 +58,8 @@ public synchronized JobManagerPlugin getJobManagerPlugin() { public synchronized TaskManagerPlugin getTaskManagerPlugin() { if (this.taskManagerPlugin == null) { - this.taskManagerPlugin = new StreamingTaskManagerPlugin(getPluginConfiguration()); + this.taskManagerPlugin = new StreamingTaskManagerPlugin(getPluginConfiguration(), getPluginLookupService() + .getJobManagerComponent(getPluginID())); } return this.taskManagerPlugin; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index 9af3e3342dff3..19b09f2296b90 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -20,6 +20,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.plugins.PluginCommunication; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; import eu.stratosphere.nephele.types.Record; @@ -56,11 +57,19 @@ public class StreamingTaskManagerPlugin implements TaskManagerPlugin { */ private final int aggregationInterval; - StreamingTaskManagerPlugin(final Configuration pluginConfiguration) { + /** + * A special thread to asynchronously send data to the job manager component without suffering from the RPC latency. + */ + private final StreamingCommunicationThread communicationThread; + + StreamingTaskManagerPlugin(final Configuration pluginConfiguration, final PluginCommunication jobManagerComponent) { this.taggingInterval = pluginConfiguration.getInteger(TAGGING_INTERVAL_KEY, DEFAULT_TAGGING_INTERVAL); this.aggregationInterval = pluginConfiguration.getInteger(AGGREGATION_INTERVAL_KEY, DEFAULT_AGGREGATION_INTERVAL); + + this.communicationThread = new StreamingCommunicationThread(jobManagerComponent); + this.communicationThread.start(); } /** @@ -68,8 +77,8 @@ public class StreamingTaskManagerPlugin implements TaskManagerPlugin { */ @Override public void shutdown() { - // TODO Auto-generated method stub + this.communicationThread.stopCommunicationThread(); } /** From a734ce695e3a48862818d2f3aef3a3902939e9d6 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 18 Nov 2011 15:13:53 +0000 Subject: [PATCH 046/310] Improved javadoc --- .../streaming/StreamingPluginLoader.java | 29 +++++++++++++++++-- 1 file changed, 27 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java index d4f50527c0e9d..198f3660e0c84 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java @@ -22,14 +22,40 @@ import eu.stratosphere.nephele.plugins.PluginLookupService; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; -public class StreamingPluginLoader extends AbstractPluginLoader { +/** + * This class implements the loader functionality for the Nephele streaming plugin. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class StreamingPluginLoader extends AbstractPluginLoader { + /** + * The job manager component of this plugin. + */ private StreamingJobManagerPlugin jobManagerPlugin = null; + /** + * The task manager component of this plugin. + */ private StreamingTaskManagerPlugin taskManagerPlugin = null; + /** + * The ID of this plugin. + */ private final PluginID pluginID; + /** + * Constructs a loader for the Nephele streaming plugin. + * + * @param pluginName + * the name of the plugin as specified in the plugin configuration file + * @param pluginConfiguration + * the configuration of this plugin + * @param pluginLookupService + * the lookup service to locate the remote components of this plugin + */ public StreamingPluginLoader(final String pluginName, final Configuration pluginConfiguration, final PluginLookupService pluginLookupService) { super(pluginName, pluginConfiguration, pluginLookupService); @@ -73,5 +99,4 @@ public PluginID getPluginID() { return this.pluginID; } - } From a8934f8c51ec5b04444dc8b7050bd10ed493948c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 18 Nov 2011 16:15:33 +0000 Subject: [PATCH 047/310] Implemented streaming tag and integrated it --- .../nephele/types/AbstractTaggableRecord.java | 31 +++-- .../eu/stratosphere/nephele/types/Tag.java | 28 +++++ .../nephele/streaming/PathLatency.java | 109 ++++++++++++++++++ .../nephele/streaming/StreamingTag.java | 81 +++++++++++++ .../streaming/StreamingTaskListener.java | 54 ++++----- .../streaming/StreamingTaskManagerPlugin.java | 6 +- 6 files changed, 263 insertions(+), 46 deletions(-) create mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/Tag.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java index c7db0da4f96ed..b68600bbd64ca 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java @@ -19,15 +19,17 @@ import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.nephele.util.StringUtils; + public abstract class AbstractTaggableRecord implements Record { - private byte[] tag = null; + private Tag tag = null; - public void setTag(final byte[] tag) { + public void setTag(final Tag tag) { this.tag = tag; } - public byte[] getTag() { + public Tag getTag() { return this.tag; } @@ -42,22 +44,33 @@ public void write(final DataOutput out) throws IOException { out.writeBoolean(false); } else { out.writeBoolean(true); - out.writeShort((short) this.tag.length); - out.write(this.tag); + StringRecord.writeString(out, this.tag.getClass().getName()); + this.tag.write(out); } - } /** * {@inheritDoc} */ + @SuppressWarnings("unchecked") @Override public void read(final DataInput in) throws IOException { if (in.readBoolean()) { - final short length = in.readShort(); - this.tag = new byte[length]; - in.readFully(this.tag); + final String tagType = StringRecord.readString(in); + Class clazz = null; + try { + clazz = (Class) Class.forName(tagType); + } catch (ClassNotFoundException e) { + throw new IOException(StringUtils.stringifyException(e)); + } + + try { + this.tag = clazz.newInstance(); + } catch (Exception e) { + throw new IOException(StringUtils.stringifyException(e)); + } + this.tag.read(in); } } } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/Tag.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/Tag.java new file mode 100644 index 0000000000000..ce74b6af3406b --- /dev/null +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/Tag.java @@ -0,0 +1,28 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.types; + +import eu.stratosphere.nephele.io.IOReadableWritable; + +/** + * This interface specifies a tag. A tag is a piece of metadata that can be attached to {@link AbstractTaggableRecord} + * objects. + * + * @author warneke + */ +public interface Tag extends IOReadableWritable { + +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java new file mode 100644 index 0000000000000..92e08f33805d1 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java @@ -0,0 +1,109 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.jobgraph.JobID; + +public final class PathLatency implements StreamingData { + + private final JobID jobID; + + private final ExecutionVertexID sourceID; + + private final ExecutionVertexID targetID; + + private double pathLatency; + + public PathLatency(final JobID jobID, final ExecutionVertexID sourceID, final ExecutionVertexID targetID, + final double pathLatency) { + + if (jobID == null) { + throw new IllegalArgumentException("jobID must not be null"); + } + + if (sourceID == null) { + throw new IllegalArgumentException("sourceID must not be null"); + } + + if (targetID == null) { + throw new IllegalArgumentException("targetID must not be null"); + } + + this.jobID = jobID; + this.sourceID = sourceID; + this.targetID = targetID; + this.pathLatency = pathLatency; + } + + /** + * Default constructor for the deserialization of the object. + */ + public PathLatency() { + this.jobID = new JobID(); + this.sourceID = new ExecutionVertexID(); + this.targetID = new ExecutionVertexID(); + this.pathLatency = 0.0; + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + + this.jobID.write(out); + this.sourceID.write(out); + this.targetID.write(out); + out.writeDouble(this.pathLatency); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + + this.jobID.read(in); + this.sourceID.read(in); + this.targetID.read(in); + this.pathLatency = in.readDouble(); + } + + public JobID getJobID() { + + return this.jobID; + } + + public ExecutionVertexID getSourceID() { + + return this.sourceID; + } + + public ExecutionVertexID getTargetID() { + + return this.targetID; + } + + public double getPathLatency() { + + return this.pathLatency; + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java new file mode 100644 index 0000000000000..e39ecaa028b9d --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java @@ -0,0 +1,81 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.types.Tag; + +public final class StreamingTag implements Tag { + + private final ExecutionVertexID sourceID; + + private long timestamp = 0L; + + StreamingTag(final ExecutionVertexID sourceID) { + + if (sourceID == null) { + throw new IllegalArgumentException("sourceID must not be null"); + } + + this.sourceID = sourceID; + } + + /** + * Default constructor for deserialization. + */ + public StreamingTag() { + this.sourceID = new ExecutionVertexID(); + } + + public void setTimestamp(final long timestamp) { + this.timestamp = timestamp; + } + + public ExecutionVertexID getSourceID() { + + return this.sourceID; + } + + public long getTimestamp() { + + return this.timestamp; + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + // TODO Auto-generated method stub + + this.sourceID.write(out); + out.writeLong(this.timestamp); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + + this.sourceID.read(in); + this.timestamp = in.readLong(); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java index 00c6bea3396d9..fa23c2782dfca 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java @@ -15,6 +15,7 @@ package eu.stratosphere.nephele.streaming; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.InputGateListener; import eu.stratosphere.nephele.io.OutputGateListener; import eu.stratosphere.nephele.types.AbstractTaggableRecord; @@ -22,19 +23,19 @@ public final class StreamingTaskListener implements InputGateListener, OutputGateListener { - private final static int SIZEOFLONG = 8; - private static enum TaskType { INPUT, REGULAR, OUTPUT }; + private final ExecutionVertexID vertexID; + private final TaskType taskType; private final int taggingInterval; private final int aggregationInterval; - private byte[] tag = null; + private StreamingTag tag = null; private int tagCounter = 0; @@ -42,23 +43,26 @@ private static enum TaskType { private double aggregatedValue = -1.0; - static StreamingTaskListener createForInputTask(final int taggingInterval, final int aggregationInterval) { + static StreamingTaskListener createForInputTask(final ExecutionVertexID vertexID, final int taggingInterval, + final int aggregationInterval) { - return new StreamingTaskListener(TaskType.INPUT, taggingInterval, aggregationInterval); + return new StreamingTaskListener(vertexID, TaskType.INPUT, taggingInterval, aggregationInterval); } - static StreamingTaskListener createForRegularTask(final int aggregationInterval) { + static StreamingTaskListener createForRegularTask(final ExecutionVertexID vertexID, final int aggregationInterval) { - return new StreamingTaskListener(TaskType.REGULAR, 0, aggregationInterval); + return new StreamingTaskListener(vertexID, TaskType.REGULAR, 0, aggregationInterval); } - static StreamingTaskListener createForOutputTask(final int aggregationInterval) { + static StreamingTaskListener createForOutputTask(final ExecutionVertexID vertexID, final int aggregationInterval) { - return new StreamingTaskListener(TaskType.OUTPUT, 0, aggregationInterval); + return new StreamingTaskListener(vertexID, TaskType.OUTPUT, 0, aggregationInterval); } - private StreamingTaskListener(final TaskType taskType, final int taggingInterval, final int aggregationInterval) { + private StreamingTaskListener(final ExecutionVertexID vertexID, final TaskType taskType, final int taggingInterval, + final int aggregationInterval) { + this.vertexID = vertexID; this.taskType = taskType; this.taggingInterval = taggingInterval; this.aggregationInterval = aggregationInterval; @@ -117,15 +121,15 @@ public void recordReceived(final Record record) { throw new IllegalStateException("Input task received record"); case REGULAR: { final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - this.tag = taggableRecord.getTag(); + this.tag = (StreamingTag) taggableRecord.getTag(); } break; case OUTPUT: { final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - this.tag = taggableRecord.getTag(); + this.tag = (StreamingTag) taggableRecord.getTag(); if (this.tag != null) { - System.out.println(System.currentTimeMillis() - byteArrayToLong(this.tag)); + System.out.println(System.currentTimeMillis() - this.tag.getTimestamp()); } } break; @@ -133,33 +137,15 @@ public void recordReceived(final Record record) { } - private byte[] createTag() { + private StreamingTag createTag() { if (this.tag == null) { - this.tag = new byte[SIZEOFLONG]; + this.tag = new StreamingTag(this.vertexID); } - longToByteArray(System.currentTimeMillis(), this.tag); + this.tag.setTimestamp(System.currentTimeMillis()); return this.tag; } - private static void longToByteArray(final long longToSerialize, final byte[] buffer) { - - for (int i = 0; i < SIZEOFLONG; ++i) { - final int shift = i << 3; // i * 8 - buffer[(SIZEOFLONG - 1) - i] = (byte) ((longToSerialize & (0xffL << shift)) >>> shift); - } - } - - private static long byteArrayToLong(final byte[] buffer) { - - long l = 0; - - for (int i = 0; i < SIZEOFLONG; ++i) { - l |= (buffer[(SIZEOFLONG - 1) - i] & 0xffL) << (i << 3); - } - - return l; - } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index 19b09f2296b90..f8b7576f93021 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -97,11 +97,11 @@ public void registerTask(final ExecutionVertexID id, final Configuration jobConf // Check if user has provided a job-specific tagging interval final int taggingInterval = jobConfiguration.getInteger(TAGGING_INTERVAL_KEY, this.taggingInterval); - listener = StreamingTaskListener.createForInputTask(taggingInterval, aggregationInterval); + listener = StreamingTaskListener.createForInputTask(id, taggingInterval, aggregationInterval); } else if (environment.getNumberOfOutputGates() == 0) { - listener = StreamingTaskListener.createForOutputTask(aggregationInterval); + listener = StreamingTaskListener.createForOutputTask(id, aggregationInterval); } else { - listener = StreamingTaskListener.createForRegularTask(aggregationInterval); + listener = StreamingTaskListener.createForRegularTask(id, aggregationInterval); } for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { From a2651e062a82c539ee15d869f6a2f8c4fbc17074 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 18 Nov 2011 16:38:00 +0000 Subject: [PATCH 048/310] Moved instantiation of plugins to early point during the job manager instantion to prevent problems in local mode --- .../nephele/jobmanager/JobManager.java | 6 +-- .../plugins/JobManagerLookupService.java | 49 ++++++++++++++++++- .../nephele/plugins/PluginManager.java | 12 ++--- 3 files changed, 55 insertions(+), 12 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index e97b93ffc1fcf..2c04e274cf3c3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -234,6 +234,9 @@ public JobManager(final String configDir, final String executionMode) { LOG.info("Starting job manager in " + executionMode + " mode"); + // Load the plugins + this.jobManagerPlugins = PluginManager.getJobManagerPlugins(this, configDir); + // Try to load the instance manager for the given execution mode // Try to load the scheduler for the given execution mode if ("local".equals(executionMode)) { @@ -291,9 +294,6 @@ public JobManager(final String configDir, final String executionMode) { LOG.debug("Profiler disabled"); } - // Load the plugins - this.jobManagerPlugins = PluginManager.getJobManagerPlugins(configDir); - // Add shutdown hook for clean up tasks Runtime.getRuntime().addShutdownHook(new JobManagerCleanUp(this)); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java index a4ca06a5f3b14..9cb8d78e545fe 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java @@ -15,16 +15,61 @@ package eu.stratosphere.nephele.plugins; +import java.io.IOException; + +import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.protocols.PluginCommunicationProtocol; + public final class JobManagerLookupService implements PluginLookupService { + private final PluginCommunicationProtocol jobManager; + + JobManagerLookupService(final PluginCommunicationProtocol jobManager) { + this.jobManager = jobManager; + } + + private static final class JobManagerStub implements PluginCommunication { + + private final PluginCommunicationProtocol jobManager; + + private final PluginID pluginID; + + public JobManagerStub(final PluginCommunicationProtocol jobManager, final PluginID pluginID) { + this.jobManager = jobManager; + this.pluginID = pluginID; + } + + /** + * {@inheritDoc} + */ + @Override + public void sendData(final IOReadableWritable data) throws IOException { + + synchronized (this.jobManager) { + this.jobManager.sendData(this.pluginID, data); + } + + } + + /** + * {@inheritDoc} + */ + @Override + public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { + + synchronized (this.jobManager) { + return this.jobManager.requestData(this.pluginID, data); + } + } + } + /** * {@inheritDoc} */ @Override public PluginCommunication getJobManagerComponent(final PluginID pluginID) { - throw new IllegalStateException( - "getJobManagerComponent must not be called from the plugin component running on the job manager"); + return new JobManagerStub(this.jobManager, pluginID); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java index f498153a9ed0b..b4883f5c331a5 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginManager.java @@ -39,6 +39,7 @@ import org.xml.sax.SAXException; import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.protocols.PluginCommunicationProtocol; import eu.stratosphere.nephele.taskmanager.TaskManager; import eu.stratosphere.nephele.util.StringUtils; @@ -68,8 +69,6 @@ public final class PluginManager { private final Map plugins; - private final PluginLookupService pluginLookupService; - private PluginManager(final String configDir, final PluginLookupService pluginLookupService) { // Check if the configuration file exists @@ -80,8 +79,6 @@ private PluginManager(final String configDir, final PluginLookupService pluginLo this.plugins = Collections.emptyMap(); LOG.warn("Unable to load plugins: configuration file " + configFile.getAbsolutePath() + " not found"); } - - this.pluginLookupService = pluginLookupService; } private String getTextChild(final Node node) { @@ -293,7 +290,7 @@ private Map loadPlugins(final File configFile, AbstractPluginLoader pluginLoader = null; try { - pluginLoader = constructor.newInstance(pluginName, pluginConfiguration, this.pluginLookupService); + pluginLoader = constructor.newInstance(pluginName, pluginConfiguration, pluginLookupService); } catch (IllegalArgumentException e) { LOG.error("Unable to load plugin " + pluginName + ": " + StringUtils.stringifyException(e)); continue; @@ -382,9 +379,10 @@ private Map getTaskManagerPluginsInternal() { return Collections.unmodifiableMap(taskManagerPluginMap); } - public static Map getJobManagerPlugins(final String configDir) { + public static Map getJobManagerPlugins(final PluginCommunicationProtocol jobManager, + final String configDir) { - final JobManagerLookupService lookupService = new JobManagerLookupService(); + final JobManagerLookupService lookupService = new JobManagerLookupService(jobManager); return getInstance(configDir, lookupService).getJobManagerPluginsInternal(); } From 2485e7de8990a87c91263adc81342e1d65a30adf Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 18 Nov 2011 17:35:12 +0000 Subject: [PATCH 049/310] Improved javadoc --- .../nephele/streaming/PathLatency.java | 99 +++++++++++++++---- 1 file changed, 82 insertions(+), 17 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java index 92e08f33805d1..f82d824cb0bb1 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java @@ -22,34 +22,63 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; +/** + * This class stores information about the latency of a specific (sub) path from a start to an end vertex. + * + * @author warneke + */ public final class PathLatency implements StreamingData { + /** + * The ID of the job this path latency information refers to + */ private final JobID jobID; - private final ExecutionVertexID sourceID; + /** + * The ID of the vertex representing the start of the path. + */ + private final ExecutionVertexID startVertexID; - private final ExecutionVertexID targetID; + /** + * The ID of the vertex representing the end of the path. + */ + private final ExecutionVertexID endVertexID; + /** + * The path latency in milliseconds + */ private double pathLatency; - public PathLatency(final JobID jobID, final ExecutionVertexID sourceID, final ExecutionVertexID targetID, + /** + * Constructs a new path latency object. + * + * @param jobID + * the ID of the job this path latency information refers to + * @param startVertexID + * the ID of the vertex representing the start of the path + * @param endVertexID + * the ID of the vertex representing the end of the path + * @param pathLatency + * the path latency in milliseconds + */ + public PathLatency(final JobID jobID, final ExecutionVertexID startVertexID, final ExecutionVertexID endVertexID, final double pathLatency) { if (jobID == null) { throw new IllegalArgumentException("jobID must not be null"); } - if (sourceID == null) { + if (startVertexID == null) { throw new IllegalArgumentException("sourceID must not be null"); } - if (targetID == null) { + if (endVertexID == null) { throw new IllegalArgumentException("targetID must not be null"); } this.jobID = jobID; - this.sourceID = sourceID; - this.targetID = targetID; + this.startVertexID = startVertexID; + this.endVertexID = endVertexID; this.pathLatency = pathLatency; } @@ -58,8 +87,8 @@ public PathLatency(final JobID jobID, final ExecutionVertexID sourceID, final Ex */ public PathLatency() { this.jobID = new JobID(); - this.sourceID = new ExecutionVertexID(); - this.targetID = new ExecutionVertexID(); + this.startVertexID = new ExecutionVertexID(); + this.endVertexID = new ExecutionVertexID(); this.pathLatency = 0.0; } @@ -70,8 +99,8 @@ public PathLatency() { public void write(final DataOutput out) throws IOException { this.jobID.write(out); - this.sourceID.write(out); - this.targetID.write(out); + this.startVertexID.write(out); + this.endVertexID.write(out); out.writeDouble(this.pathLatency); } @@ -82,28 +111,64 @@ public void write(final DataOutput out) throws IOException { public void read(final DataInput in) throws IOException { this.jobID.read(in); - this.sourceID.read(in); - this.targetID.read(in); + this.startVertexID.read(in); + this.endVertexID.read(in); this.pathLatency = in.readDouble(); } + /** + * Returns the ID of the job this path latency information refers to. + * + * @return the ID of the job this path latency information refers to + */ public JobID getJobID() { return this.jobID; } - public ExecutionVertexID getSourceID() { + /** + * Returns the ID of the vertex representing the start of the path. + * + * @return the ID of the vertex representing the start of the path + */ + public ExecutionVertexID getStartVertexID() { - return this.sourceID; + return this.startVertexID; } - public ExecutionVertexID getTargetID() { + /** + * Returns the ID of the vertex representing the end of the path. + * + * @return the ID of the vertex representing the end of the path + */ + public ExecutionVertexID getEndVertexID() { - return this.targetID; + return this.endVertexID; } + /** + * Returns the path latency in milliseconds. + * + * @return the path latency in milliseconds + */ public double getPathLatency() { return this.pathLatency; } + + /** + * {@inheritDoc} + */ + @Override + public String toString() { + + final StringBuilder str = new StringBuilder(); + str.append(this.startVertexID.toString()); + str.append(" -> "); + str.append(this.endVertexID.toString()); + str.append(": "); + str.append(this.pathLatency); + + return str.toString(); + } } From 114712208d75132cbe5c5c2e3fb5bf80fc246372 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 18 Nov 2011 17:36:37 +0000 Subject: [PATCH 050/310] Implemented propagation of path latency information from task manager to job manager --- .../streaming/StreamingJobManagerPlugin.java | 22 ++++- .../streaming/StreamingTaskListener.java | 96 ++++++++++++++----- .../streaming/StreamingTaskManagerPlugin.java | 12 ++- 3 files changed, 103 insertions(+), 27 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 6e7bd07373e2a..f602adede306f 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -17,6 +17,9 @@ import java.io.IOException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.io.IOReadableWritable; @@ -25,6 +28,11 @@ public class StreamingJobManagerPlugin implements JobManagerPlugin { + /** + * The log object. + */ + private static final Log LOG = LogFactory.getLog(StreamingJobManagerPlugin.class); + StreamingJobManagerPlugin(final Configuration pluginConfiguration) { } @@ -60,8 +68,13 @@ public void shutdown() { */ @Override public void sendData(final IOReadableWritable data) throws IOException { - // TODO Auto-generated method stub + if (!(data instanceof StreamingData)) { + LOG.error("Received unexpected data of type " + data); + return; + } + + System.out.println(data); } /** @@ -69,7 +82,12 @@ public void sendData(final IOReadableWritable data) throws IOException { */ @Override public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { - // TODO Auto-generated method stub + + if (!(data instanceof StreamingData)) { + LOG.error("Received unexpected data of type " + data); + return null; + } + return null; } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java index fa23c2782dfca..f2b5855ef7627 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java @@ -15,18 +15,37 @@ package eu.stratosphere.nephele.streaming; +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.InputGateListener; import eu.stratosphere.nephele.io.OutputGateListener; +import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.types.AbstractTaggableRecord; import eu.stratosphere.nephele.types.Record; +import eu.stratosphere.nephele.util.StringUtils; public final class StreamingTaskListener implements InputGateListener, OutputGateListener { + /** + * The log object. + */ + private static final Log LOG = LogFactory.getLog(StreamingTaskListener.class); + private static enum TaskType { INPUT, REGULAR, OUTPUT }; + private final static double ALPHA = 0.5; + + private final StreamingCommunicationThread communicationThread; + + private final JobID jobID; + private final ExecutionVertexID vertexID; private final TaskType taskType; @@ -39,29 +58,36 @@ private static enum TaskType { private int tagCounter = 0; - private int aggregationCounter = 0; + private Map aggregationCounter = new HashMap(); - private double aggregatedValue = -1.0; + private Map aggregatedValue = new HashMap(); - static StreamingTaskListener createForInputTask(final ExecutionVertexID vertexID, final int taggingInterval, + static StreamingTaskListener createForInputTask(final StreamingCommunicationThread communicationThread, + final JobID jobID, final ExecutionVertexID vertexID, final int taggingInterval, final int aggregationInterval) { - return new StreamingTaskListener(vertexID, TaskType.INPUT, taggingInterval, aggregationInterval); + return new StreamingTaskListener(communicationThread, jobID, vertexID, TaskType.INPUT, taggingInterval, + aggregationInterval); } - static StreamingTaskListener createForRegularTask(final ExecutionVertexID vertexID, final int aggregationInterval) { + static StreamingTaskListener createForRegularTask(final StreamingCommunicationThread communicationThread, + final JobID jobID, final ExecutionVertexID vertexID, final int aggregationInterval) { - return new StreamingTaskListener(vertexID, TaskType.REGULAR, 0, aggregationInterval); + return new StreamingTaskListener(communicationThread, jobID, vertexID, TaskType.REGULAR, 0, aggregationInterval); } - static StreamingTaskListener createForOutputTask(final ExecutionVertexID vertexID, final int aggregationInterval) { + static StreamingTaskListener createForOutputTask(final StreamingCommunicationThread communicationThread, + final JobID jobID, final ExecutionVertexID vertexID, final int aggregationInterval) { - return new StreamingTaskListener(vertexID, TaskType.OUTPUT, 0, aggregationInterval); + return new StreamingTaskListener(communicationThread, jobID, vertexID, TaskType.OUTPUT, 0, aggregationInterval); } - private StreamingTaskListener(final ExecutionVertexID vertexID, final TaskType taskType, final int taggingInterval, + private StreamingTaskListener(final StreamingCommunicationThread communicationThread, final JobID jobID, + final ExecutionVertexID vertexID, final TaskType taskType, final int taggingInterval, final int aggregationInterval) { + this.communicationThread = communicationThread; + this.jobID = jobID; this.vertexID = vertexID; this.taskType = taskType; this.taggingInterval = taggingInterval; @@ -116,23 +142,49 @@ public void waitingForAnyChannel() { @Override public void recordReceived(final Record record) { - switch (this.taskType) { - case INPUT: + if (this.taskType == TaskType.INPUT) { throw new IllegalStateException("Input task received record"); - case REGULAR: { - final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - this.tag = (StreamingTag) taggableRecord.getTag(); } - break; - case OUTPUT: { - final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - this.tag = (StreamingTag) taggableRecord.getTag(); - if (this.tag != null) { - System.out.println(System.currentTimeMillis() - this.tag.getTimestamp()); + final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; + this.tag = (StreamingTag) taggableRecord.getTag(); + if (this.tag != null) { + + final long pathLatency = System.currentTimeMillis() - this.tag.getTimestamp(); + + final ExecutionVertexID sourceID = this.tag.getSourceID(); + + // Calculate moving average + Double aggregatedLatency = this.aggregatedValue.get(sourceID); + if (aggregatedLatency == null) { + aggregatedLatency = Double.valueOf(pathLatency); + } else { + aggregatedLatency = Double.valueOf((ALPHA * pathLatency) + + ((1 - ALPHA) * aggregatedLatency.doubleValue())); } - } - break; + this.aggregatedValue.put(sourceID, aggregatedLatency); + + // Check if we need to compute an event and send it to the job manager component + Integer counter = this.aggregationCounter.get(sourceID); + if (counter == null) { + counter = Integer.valueOf(0); + } + + counter = Integer.valueOf(counter.intValue() + 1); + if (counter.intValue() == this.aggregationInterval) { + + final PathLatency pl = new PathLatency(this.jobID, sourceID, this.vertexID, + aggregatedLatency.doubleValue()); + + try { + this.communicationThread.sendDataAsynchronously(pl); + } catch (InterruptedException e) { + LOG.warn(StringUtils.stringifyException(e)); + } + + counter = Integer.valueOf(0); + } + this.aggregationCounter.put(sourceID, counter); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index f8b7576f93021..dc45d4b9fe9c6 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -20,6 +20,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.plugins.PluginCommunication; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; import eu.stratosphere.nephele.types.Record; @@ -93,15 +94,20 @@ public void registerTask(final ExecutionVertexID id, final Configuration jobConf this.aggregationInterval); StreamingTaskListener listener = null; + final JobID jobID = environment.getJobID(); + if (environment.getNumberOfInputGates() == 0) { // Check if user has provided a job-specific tagging interval final int taggingInterval = jobConfiguration.getInteger(TAGGING_INTERVAL_KEY, this.taggingInterval); - listener = StreamingTaskListener.createForInputTask(id, taggingInterval, aggregationInterval); + listener = StreamingTaskListener.createForInputTask(this.communicationThread, jobID, id, taggingInterval, + aggregationInterval); } else if (environment.getNumberOfOutputGates() == 0) { - listener = StreamingTaskListener.createForOutputTask(id, aggregationInterval); + listener = StreamingTaskListener.createForOutputTask(this.communicationThread, jobID, id, + aggregationInterval); } else { - listener = StreamingTaskListener.createForRegularTask(id, aggregationInterval); + listener = StreamingTaskListener.createForRegularTask(this.communicationThread, jobID, id, + aggregationInterval); } for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { From ba452fb1a52ba72a98a07241d00522c4b12d3c87 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 18 Nov 2011 20:29:37 +0100 Subject: [PATCH 051/310] Extended abstract instance to support plugin communication --- .../nephele/instance/AbstractInstance.java | 60 +++++++++++++++++++ .../plugins/JobManagerLookupService.java | 42 ++++++++++++- .../nephele/plugins/PluginLookupService.java | 4 ++ .../plugins/TaskManagerLookupService.java | 12 +++- 4 files changed, 116 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index 893b7ed2d4852..9814c3655a8a4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -29,10 +29,13 @@ import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.ipc.RPC; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.net.NetUtils; +import eu.stratosphere.nephele.plugins.PluginID; +import eu.stratosphere.nephele.protocols.PluginCommunicationProtocol; import eu.stratosphere.nephele.protocols.TaskOperationProtocol; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult; @@ -67,6 +70,11 @@ public abstract class AbstractInstance extends NetworkNode { */ private TaskOperationProtocol taskManager = null; + /** + * Stores the RPC stub object for the instance's task manager plugin component. + */ + private PluginCommunicationProtocol taskManagerPluginComponent = null; + /** * Constructs an abstract instance object. * @@ -109,6 +117,26 @@ protected TaskOperationProtocol getTaskManager() throws IOException { return this.taskManager; } + /** + * Creates or returns the RPC stub object for the instance's task manager plugin component. + * + * @return the RPC stub object for the instance's task manager plugin component + * @throws IOException + * thrown if the RPC stub object for the task manager plugin component cannot be created + */ + protected PluginCommunicationProtocol getTaskManagerPluginComponent() throws IOException { + + if (this.taskManagerPluginComponent == null) { + + this.taskManagerPluginComponent = (PluginCommunicationProtocol) RPC.getProxy( + PluginCommunicationProtocol.class, new InetSocketAddress( + getInstanceConnectionInfo().getAddress(), getInstanceConnectionInfo().getIPCPort()), NetUtils + .getSocketFactory()); + } + + return this.taskManagerPluginComponent; + } + /** * Returns the type of the instance. * @@ -305,4 +333,36 @@ public synchronized void killTaskManager() throws IOException { getTaskManager().killTaskManager(); } + + /** + * Connects to the plugin component of this instance's task manager and sends data to the plugin with the given ID. + * + * @param pluginID + * the ID of the plugin to send data to + * @param data + * the data to send + * @throws IOException + * thrown if an error occurs while sending the data from the plugin + */ + public synchronized void sendData(final PluginID pluginID, final IOReadableWritable data) throws IOException { + + getTaskManagerPluginComponent().sendData(pluginID, data); + } + + /** + * Connects to the plugin component of this instance's task manager and requests data from the plugin with the given + * ID. + * + * @param pluginID + * the ID of the plugin to request data from + * @param data + * data to specify the request + * @return the requested data, possibly null + * @throws IOException + * thrown if an error occurs while requesting the data from the plugin + */ + public synchronized IOReadableWritable requestData(PluginID pluginID, IOReadableWritable data) throws IOException { + + return getTaskManagerPluginComponent().requestData(pluginID, data); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java index 9cb8d78e545fe..400baa7a584bd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerLookupService.java @@ -17,6 +17,7 @@ import java.io.IOException; +import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.protocols.PluginCommunicationProtocol; @@ -34,7 +35,7 @@ private static final class JobManagerStub implements PluginCommunication { private final PluginID pluginID; - public JobManagerStub(final PluginCommunicationProtocol jobManager, final PluginID pluginID) { + private JobManagerStub(final PluginCommunicationProtocol jobManager, final PluginID pluginID) { this.jobManager = jobManager; this.pluginID = pluginID; } @@ -63,6 +64,36 @@ public IOReadableWritable requestData(final IOReadableWritable data) throws IOEx } } + private final static class TaskManagerStub implements PluginCommunication { + + private final AbstractInstance instance; + + private final PluginID pluginID; + + private TaskManagerStub(final AbstractInstance instance, final PluginID pluginID) { + this.instance = instance; + this.pluginID = pluginID; + } + + /** + * {@inheritDoc} + */ + @Override + public void sendData(final IOReadableWritable data) throws IOException { + + this.instance.sendData(this.pluginID, data); + } + + /** + * {@inheritDoc} + */ + @Override + public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { + + return this.instance.requestData(this.pluginID, data); + } + } + /** * {@inheritDoc} */ @@ -72,4 +103,13 @@ public PluginCommunication getJobManagerComponent(final PluginID pluginID) { return new JobManagerStub(this.jobManager, pluginID); } + /** + * {@inheritDoc} + */ + @Override + public PluginCommunication getTaskManagerComponent(final PluginID pluginID, final AbstractInstance instance) { + + return new TaskManagerStub(instance, pluginID); + } + } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginLookupService.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginLookupService.java index 06376c4776c88..958430f325553 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginLookupService.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginLookupService.java @@ -15,7 +15,11 @@ package eu.stratosphere.nephele.plugins; +import eu.stratosphere.nephele.instance.AbstractInstance; + public interface PluginLookupService { PluginCommunication getJobManagerComponent(PluginID pluginID); + + PluginCommunication getTaskManagerComponent(PluginID pluginID, AbstractInstance instance); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java index 22cfdffdd8a30..376083ead432d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java @@ -17,6 +17,7 @@ import java.io.IOException; +import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.taskmanager.TaskManager; @@ -34,7 +35,7 @@ private static final class JobManagerStub implements PluginCommunication { private final PluginID pluginID; - public JobManagerStub(final TaskManager taskManager, final PluginID pluginID) { + private JobManagerStub(final TaskManager taskManager, final PluginID pluginID) { this.taskManager = taskManager; this.pluginID = pluginID; } @@ -66,4 +67,13 @@ public PluginCommunication getJobManagerComponent(final PluginID pluginID) { return new JobManagerStub(this.taskManager, pluginID); } + + /** + * {@inheritDoc} + */ + @Override + public PluginCommunication getTaskManagerComponent(final PluginID pluginID, final AbstractInstance instance) { + + throw new UnsupportedOperationException("getTaskManagerComponenet must not be called on this lookup service"); + } } From a169e82c9029f9d7763224902ceec1a57a9ce8a5 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 18 Nov 2011 20:53:01 +0100 Subject: [PATCH 052/310] Finished implementation of plugin communication from job manager to task maangers --- .../nephele/jobmanager/JobManager.java | 3 -- .../plugins/TaskManagerLookupService.java | 4 +- .../nephele/plugins/TaskManagerPlugin.java | 2 +- .../nephele/taskmanager/TaskManager.java | 38 +++++++++++++++++-- .../streaming/StreamingTaskManagerPlugin.java | 23 +++++++++++ .../score/ScoreTaskManagerPlugin.java | 21 ++++++++++ 6 files changed, 82 insertions(+), 9 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 2c04e274cf3c3..2e21fe4c3beaf 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -1098,9 +1098,6 @@ public void deploy(final JobID jobID, final AbstractInstance instance, return; } - // Method executionGraph field of vertex is immutable, so no need to synchronized access - final ExecutionGraph eg = verticesToBeDeployed.get(0).getExecutionGraph(); - for (final ExecutionVertex vertex : verticesToBeDeployed) { // Check vertex state diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java index 376083ead432d..755a08a6b635c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerLookupService.java @@ -46,7 +46,7 @@ private JobManagerStub(final TaskManager taskManager, final PluginID pluginID) { @Override public void sendData(final IOReadableWritable data) throws IOException { - this.taskManager.sendData(this.pluginID, data); + this.taskManager.sendDataToJobManager(this.pluginID, data); } /** @@ -55,7 +55,7 @@ public void sendData(final IOReadableWritable data) throws IOException { @Override public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { - return this.taskManager.requestData(this.pluginID, data); + return this.taskManager.requestDataFromJobManager(this.pluginID, data); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java index ffb1a01f8d304..d49ce2b8c6fd3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java @@ -19,7 +19,7 @@ import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -public interface TaskManagerPlugin { +public interface TaskManagerPlugin extends PluginCommunication { /** * Registers a new incoming task with this task manager plugin. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 1b0b3492d834f..d17a3a0c4ee93 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -90,7 +90,7 @@ * * @author warneke */ -public class TaskManager implements TaskOperationProtocol { +public class TaskManager implements TaskOperationProtocol, PluginCommunicationProtocol { private static final Log LOG = LogFactory.getLog(TaskManager.class); @@ -937,7 +937,7 @@ public void run() { * @throws IOException * thrown if an I/O error occurs during the RPC call */ - public void sendData(final PluginID pluginID, final IOReadableWritable data) throws IOException { + public void sendDataToJobManager(final PluginID pluginID, final IOReadableWritable data) throws IOException { synchronized (this.pluginCommunicationService) { this.pluginCommunicationService.sendData(pluginID, data); @@ -955,10 +955,42 @@ public void sendData(final PluginID pluginID, final IOReadableWritable data) thr * @throws IOException * thrown if an I/O error occurs during the RPC call */ - public IOReadableWritable requestData(final PluginID pluginID, final IOReadableWritable data) throws IOException { + public IOReadableWritable requestDataFromJobManager(final PluginID pluginID, final IOReadableWritable data) + throws IOException { synchronized (this.pluginCommunicationService) { return this.pluginCommunicationService.requestData(pluginID, data); } } + + /** + * {@inheritDoc} + */ + @Override + public void sendData(final PluginID pluginID, final IOReadableWritable data) throws IOException { + + final TaskManagerPlugin tmp = this.taskManagerPlugins.get(pluginID); + if (tmp == null) { + LOG.error("Cannot find task manager plugin for plugin ID " + pluginID); + return; + } + + tmp.sendData(data); + + } + + /** + * {@inheritDoc} + */ + @Override + public IOReadableWritable requestData(final PluginID pluginID, final IOReadableWritable data) throws IOException { + + final TaskManagerPlugin tmp = this.taskManagerPlugins.get(pluginID); + if (tmp == null) { + LOG.error("Cannot find task manager plugin for plugin ID " + pluginID); + return null; + } + + return tmp.requestData(data); + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index dc45d4b9fe9c6..da7751619f1de 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -15,9 +15,12 @@ package eu.stratosphere.nephele.streaming; +import java.io.IOException; + import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.jobgraph.JobID; @@ -130,4 +133,24 @@ public void unregisterTask(final ExecutionVertexID id, final Environment environ // Nothing to do here } + /** + * {@inheritDoc} + */ + @Override + public void sendData(final IOReadableWritable data) throws IOException { + + // TODO Implement me + } + + /** + * {@inheritDoc} + */ + @Override + public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { + + // TODO Implement me + + return null; + } + } diff --git a/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java b/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java index 82feb9d9bd2aa..a6541f0f537b2 100644 --- a/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java +++ b/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java @@ -15,9 +15,12 @@ package eu.stratosphere.score; +import java.io.IOException; + import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; public final class ScoreTaskManagerPlugin implements TaskManagerPlugin { @@ -51,4 +54,22 @@ public void unregisterTask(final ExecutionVertexID id, final Environment environ // TODO Auto-generated method stub } + + /** + * {@inheritDoc} + */ + @Override + public void sendData(final IOReadableWritable data) throws IOException { + // TODO Auto-generated method stub + + } + + /** + * {@inheritDoc} + */ + @Override + public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { + // TODO Auto-generated method stub + return null; + } } From 15b8a0c2e63b458bd1f00b40e553140ac3838d23 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Fri, 18 Nov 2011 23:21:17 +0100 Subject: [PATCH 053/310] some extensions to the nephele management api (untested) --- .../stratosphere/nephele/io/AbstractID.java | 17 +++++++++ .../managementgraph/ManagementEdge.java | 22 ++++++++++-- .../managementgraph/ManagementEdgeID.java | 17 +++++++++ .../managementgraph/ManagementGate.java | 20 ++++++++++- .../managementgraph/ManagementGateID.java | 14 ++++++++ .../managementgraph/ManagementGraph.java | 16 ++++++++- .../managementgraph/ManagementVertex.java | 4 +-- .../managementgraph/ManagementGraphTest.java | 36 ++++++++++++------- .../ManagementGraphFactory.java | 16 ++++++--- 9 files changed, 139 insertions(+), 23 deletions(-) create mode 100644 nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java create mode 100644 nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java index 774d9881fe1fe..3611ba19d3087 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java @@ -39,6 +39,23 @@ public abstract class AbstractID implements IOReadableWritable { */ private final byte[] bytes = new byte[SIZE]; + /** + * Creates a new id by merging the two given IDs. + * + * @param one + * The first id to use for merging + * @param two + * The second id to use for merging + */ + public AbstractID(AbstractID one, AbstractID two) { + byte[] firstBytes = one.getBytes(); + byte[] secondBytes = two.getBytes(); + + for (int i = 0; i < SIZE; i++) { + bytes[i] = (byte) (firstBytes[i] ^ secondBytes[i]); + } + } + /** * Constructs a new ID with a specific bytes value. */ diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java index d633d002fb9ba..79da066c33c56 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java @@ -26,7 +26,7 @@ * * @author warneke */ -public final class ManagementEdge { +public final class ManagementEdge extends ManagementAttachment { /** * The source of the edge referring to the output gate of an execution vertex. @@ -58,9 +58,16 @@ public final class ManagementEdge { */ private final CompressionLevel compressionLevel; + /** + * The ID of the management edge. + */ + private ManagementEdgeID managementEdgeID; + /** * Constructs a new edge object. * + * @param managementEdgeID + * the ID the new management edge * @param source * the source of the edge referring to the output gate of an execution vertex * @param sourceIndex @@ -74,9 +81,11 @@ public final class ManagementEdge { * @param compressionLevel * the compression level of the channel this edge refers to */ - public ManagementEdge(final ManagementGate source, final int sourceIndex, final ManagementGate target, + public ManagementEdge(final ManagementEdgeID managementEdgeID, + final ManagementGate source, final int sourceIndex, final ManagementGate target, final int targetIndex, final ChannelType channelType, final CompressionLevel compressionLevel) { + this.managementEdgeID = managementEdgeID; this.source = source; this.target = target; this.sourceIndex = sourceIndex; @@ -141,4 +150,13 @@ public int getSourceIndex() { public int getTargetIndex() { return this.targetIndex; } + + /** + * Returns the management edge ID. + * + * @return The management edge ID + */ + public ManagementEdgeID getManagementEdgeID() { + return managementEdgeID; + } } diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java new file mode 100644 index 0000000000000..66ef22db58475 --- /dev/null +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java @@ -0,0 +1,17 @@ +package eu.stratosphere.nephele.managementgraph; + +import eu.stratosphere.nephele.io.AbstractID; + +/** + * A management edge ID uniquely identifies a {@link ManagementEdge}. + *

+ * This class is not thread-safe. + * + * @author Bjoern Lohrmann + */ +public class ManagementEdgeID extends AbstractID { + + public ManagementEdgeID(ManagementGateID source, ManagementGateID target) { + super(source, target); + } +} diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGate.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGate.java index 4c58a1614e459..0829c3ca67371 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGate.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGate.java @@ -59,6 +59,11 @@ public final class ManagementGate extends ManagementAttachment { */ private final List backwardEdges = new ArrayList(); + /** + * The id of the management gate. + */ + private ManagementGateID gateID; + /** * Constructs a new management gate. * @@ -66,16 +71,20 @@ public final class ManagementGate extends ManagementAttachment { * the management vertex this gate belongs to * @param index * the index of this gate + * @param gateID + * The id of the new management gate * @param isInputGate * true if this gate represents an input gate in the actual execution graph, false * otherwise * @param recordType * the name of the record type transported through this gate */ - public ManagementGate(final ManagementVertex managementVertex, final int index, final boolean isInputGate, + public ManagementGate(final ManagementVertex managementVertex, final ManagementGateID gateID, + final int index, final boolean isInputGate, final String recordType) { this.isInputGate = isInputGate; this.managementVertex = managementVertex; + this.gateID = gateID; this.recordType = recordType; this.index = index; @@ -213,4 +222,13 @@ public ManagementEdge getBackwardEdge(final int index) { public ManagementVertex getVertex() { return this.managementVertex; } + + /** + * Returns the id of the management gate. + * + * @return the id of the management gate + */ + public ManagementGateID getManagementGateID() { + return gateID; + } } diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java new file mode 100644 index 0000000000000..40c7912a9e9fe --- /dev/null +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java @@ -0,0 +1,14 @@ +package eu.stratosphere.nephele.managementgraph; + +import eu.stratosphere.nephele.io.AbstractID; + +/** + * A management gate ID uniquely identifies a {@link ManagementGate}. + *

+ * This class is not thread-safe. + * + * @author Bjoern Lohrmann + */ +public class ManagementGateID extends AbstractID { + +} diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java index 346a768303301..51415013aefea 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.Deque; import java.util.HashMap; import java.util.Iterator; @@ -317,6 +319,15 @@ public ManagementVertex getOutputVertex(final int stage, final int index) { return this.stages.get(stage).getOutputManagementVertex(index); } + /** + * Returns an unmodifiable collection of all group vertices with no guarantees on their order. + * + * @return an unmodifiable collection of all group vertices with no guarantees on their order + */ + public Collection getGroupVertices() { + return Collections.unmodifiableCollection(groupVertices.values()); + } + /** * Returns a list of group vertices sorted in topological order. * @@ -468,7 +479,10 @@ public void read(final DataInput in) throws IOException { final ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class); final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); - new ManagementEdge(sourceGate, sourceIndex, targetGate, targetIndex, channelType, compressionLevel); + final ManagementEdgeID managementEdgeID = new ManagementEdgeID(sourceGate.getManagementGateID(), + targetGate.getManagementGateID()); + new ManagementEdge(managementEdgeID, sourceGate, sourceIndex, targetGate, targetIndex, channelType, + compressionLevel); } } diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java index 6094f74f47d87..f8e238be83697 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java @@ -323,7 +323,7 @@ public void read(final DataInput in) throws IOException { for (int i = 0; i < numberOfInputGates; i++) { final String recordType = StringRecord.readString(in); - new ManagementGate(this, i, true, recordType); + new ManagementGate(this, new ManagementGateID(), i, true, recordType); } // Read number of input gates and record types @@ -331,7 +331,7 @@ public void read(final DataInput in) throws IOException { for (int i = 0; i < numberOfOutputGates; i++) { final String recordType = StringRecord.readString(in); - new ManagementGate(this, i, false, recordType); + new ManagementGate(this, new ManagementGateID(), i, false, recordType); } this.instanceName = StringRecord.readString(in); diff --git a/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java b/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java index 5500197e8a2d7..3e0f8b2fba105 100644 --- a/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java +++ b/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java @@ -276,18 +276,18 @@ private static ManagementGraph constructTestManagementGraph() { "medium", CHECKPOINT_STATE, 0); // Input/output gates - final ManagementGate outputGate1_1 = new ManagementGate(vertex1_1, 0, false, RECORD_TYPE); + final ManagementGate outputGate1_1 = new ManagementGate(vertex1_1, new ManagementGateID(), 0, false, RECORD_TYPE); - final ManagementGate inputGate2_1 = new ManagementGate(vertex2_1, 0, true, RECORD_TYPE); - final ManagementGate outputGate2_1 = new ManagementGate(vertex2_1, 0, false, RECORD_TYPE); + final ManagementGate inputGate2_1 = new ManagementGate(vertex2_1, new ManagementGateID(), 0, true, RECORD_TYPE); + final ManagementGate outputGate2_1 = new ManagementGate(vertex2_1, new ManagementGateID(), 0, false, RECORD_TYPE); - final ManagementGate inputGate2_2 = new ManagementGate(vertex2_2, 0, true, RECORD_TYPE); - final ManagementGate outputGate2_2 = new ManagementGate(vertex2_2, 0, false, RECORD_TYPE); + final ManagementGate inputGate2_2 = new ManagementGate(vertex2_2, new ManagementGateID(), 0, true, RECORD_TYPE); + final ManagementGate outputGate2_2 = new ManagementGate(vertex2_2, new ManagementGateID(), 0, false, RECORD_TYPE); - final ManagementGate inputGate3_1 = new ManagementGate(vertex3_1, 0, true, RECORD_TYPE); - final ManagementGate outputGate3_1 = new ManagementGate(vertex3_1, 0, false, RECORD_TYPE); + final ManagementGate inputGate3_1 = new ManagementGate(vertex3_1, new ManagementGateID(), 0, true, RECORD_TYPE); + final ManagementGate outputGate3_1 = new ManagementGate(vertex3_1, new ManagementGateID(), 0, false, RECORD_TYPE); - final ManagementGate inputGate4_1 = new ManagementGate(vertex4_1, 0, true, RECORD_TYPE); + final ManagementGate inputGate4_1 = new ManagementGate(vertex4_1, new ManagementGateID(), 0, true, RECORD_TYPE); // Group Edges new ManagementGroupEdge(groupVertex1, 0, groupVertex2, 0, ChannelType.NETWORK, CompressionLevel.NO_COMPRESSION); @@ -295,11 +295,21 @@ private static ManagementGraph constructTestManagementGraph() { new ManagementGroupEdge(groupVertex3, 0, groupVertex4, 0, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); // Edges - new ManagementEdge(outputGate1_1, 0, inputGate2_1, 0, ChannelType.NETWORK, CompressionLevel.NO_COMPRESSION); - new ManagementEdge(outputGate1_1, 1, inputGate2_2, 0, ChannelType.NETWORK, CompressionLevel.NO_COMPRESSION); - new ManagementEdge(outputGate2_1, 0, inputGate3_1, 0, ChannelType.FILE, CompressionLevel.NO_COMPRESSION); - new ManagementEdge(outputGate2_2, 0, inputGate3_1, 1, ChannelType.FILE, CompressionLevel.NO_COMPRESSION); - new ManagementEdge(outputGate3_1, 0, inputGate4_1, 0, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + new ManagementEdge(new ManagementEdgeID(outputGate1_1.getManagementGateID(), + inputGate2_1.getManagementGateID()), outputGate1_1, 0, inputGate2_1, 0, ChannelType.NETWORK, + CompressionLevel.NO_COMPRESSION); + new ManagementEdge(new ManagementEdgeID(outputGate1_1.getManagementGateID(), + inputGate2_2.getManagementGateID()), outputGate1_1, 1, inputGate2_2, 0, ChannelType.NETWORK, + CompressionLevel.NO_COMPRESSION); + new ManagementEdge(new ManagementEdgeID(outputGate2_1.getManagementGateID(), + inputGate3_1.getManagementGateID()), outputGate2_1, 0, inputGate3_1, 0, ChannelType.FILE, + CompressionLevel.NO_COMPRESSION); + new ManagementEdge(new ManagementEdgeID(outputGate2_2.getManagementGateID(), + inputGate3_1.getManagementGateID()), outputGate2_2, 0, inputGate3_1, 1, ChannelType.FILE, + CompressionLevel.NO_COMPRESSION); + new ManagementEdge(new ManagementEdgeID(outputGate3_1.getManagementGateID(), + inputGate4_1.getManagementGateID()), outputGate3_1, 0, inputGate4_1, 0, ChannelType.INMEMORY, + CompressionLevel.NO_COMPRESSION); return graph; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java index 3f059966a972d..45de77c4f1e10 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java @@ -26,7 +26,9 @@ import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.managementgraph.ManagementEdge; +import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; import eu.stratosphere.nephele.managementgraph.ManagementGate; +import eu.stratosphere.nephele.managementgraph.ManagementGateID; import eu.stratosphere.nephele.managementgraph.ManagementGraph; import eu.stratosphere.nephele.managementgraph.ManagementGroupEdge; import eu.stratosphere.nephele.managementgraph.ManagementGroupVertex; @@ -135,14 +137,15 @@ private static void addExecutionVertices(Map outputGate = ev.getEnvironment().getOutputGate(i); - final ManagementGate managementGate = new ManagementGate(managementVertex, i, false, outputGate - .getType().toString()); + final ManagementGate managementGate = new ManagementGate(managementVertex, + new ManagementGateID(), i, false, outputGate.getType().toString()); gateMap.put(outputGate, managementGate); } for (int i = 0; i < ev.getEnvironment().getNumberOfInputGates(); i++) { final InputGate inputGate = ev.getEnvironment().getInputGate(i); - final ManagementGate managementGate = new ManagementGate(managementVertex, i, true, ""); + final ManagementGate managementGate = new ManagementGate(managementVertex, + new ManagementGateID(), i, true, ""); gateMap.put(inputGate, managementGate); } } @@ -169,7 +172,12 @@ private static void addExecutionVertices(Map inputChannel = executionGraph .getInputChannelByID(inputChannelID); final ManagementGate managementInputGate = gateMap.get(inputChannel.getInputGate()); - new ManagementEdge(manangementOutputGate, j, managementInputGate, inputChannel.getChannelIndex(), + + final ManagementEdgeID managementEdgeID = new ManagementEdgeID( + manangementOutputGate.getManagementGateID(), + managementInputGate.getManagementGateID()); + new ManagementEdge(managementEdgeID, manangementOutputGate, j, managementInputGate, + inputChannel.getChannelIndex(), inputChannel.getType(), inputChannel.getCompressionLevel()); } } From 42adabe2841483ad2465839bf47e58a7447fcca8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Fri, 18 Nov 2011 23:57:52 +0100 Subject: [PATCH 054/310] first sketch of the latency graph structures (untested) --- .../streaming/latency/EdgeLatency.java | 26 ++++ .../streaming/latency/LatencyPath.java | 85 ++++++++++++ .../streaming/latency/LatencySubgraph.java | 126 ++++++++++++++++++ .../streaming/latency/VertexLatency.java | 26 ++++ 4 files changed, 263 insertions(+) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeLatency.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeLatency.java new file mode 100644 index 0000000000000..5f88d8fac71c8 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeLatency.java @@ -0,0 +1,26 @@ +package eu.stratosphere.nephele.streaming.latency; + +import eu.stratosphere.nephele.managementgraph.ManagementEdge; + +public class EdgeLatency { + + private ManagementEdge edge; + + private long latencyInMillis; + + public EdgeLatency(ManagementEdge edge) { + this.edge = edge; + } + + public ManagementEdge getEdge() { + return edge; + } + + public long getLatencyInMillis() { + return latencyInMillis; + } + + public void setLatencyInMillis(long latencyInMillis) { + this.latencyInMillis = latencyInMillis; + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java new file mode 100644 index 0000000000000..744f9a98ca73c --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java @@ -0,0 +1,85 @@ +package eu.stratosphere.nephele.streaming.latency; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; + +import eu.stratosphere.nephele.managementgraph.ManagementEdge; +import eu.stratosphere.nephele.managementgraph.ManagementVertex; +import eu.stratosphere.nephele.managementgraph.ManagementVertexID; + +/** + * A latency path is a path through the ManagementGraph, defined by a sequence + * of {@link ManagementVertex} objects that are connected in the order in which they appear in + * the sequence. + * + * @author Bjoern Lohrmann + */ +public class LatencyPath implements Iterable { + + private LinkedList pathVertices; + + private HashMap ingoingEdges; + + private LatencySubgraph graph; + + private long pathLatencyInMillis; + + @SuppressWarnings("unchecked") + public LatencyPath(LatencyPath toClone) { + this.graph = toClone.graph; + this.pathVertices = (LinkedList) toClone.pathVertices.clone(); + this.ingoingEdges = (HashMap) toClone.ingoingEdges.clone(); + } + + public LatencyPath(LatencySubgraph graph, ManagementVertex firstVertex) { + this.graph = graph; + this.pathVertices = new LinkedList(); + this.ingoingEdges = new HashMap(); + this.pathVertices.add(firstVertex); + } + + public void appendVertex(ManagementVertex vertex, ManagementEdge ingoingEdge) { + pathVertices.add(vertex); + ingoingEdges.put(vertex.getID(), ingoingEdge); + } + + public ManagementVertex getBegin() { + return pathVertices.getFirst(); + } + + public ManagementVertex getEnd() { + return pathVertices.getFirst(); + } + + public ManagementEdge getIngoingEdge(ManagementVertex vertex) { + return ingoingEdges.get(vertex.getID()); + } + + public void removeLastVertex() { + ManagementVertex removed = pathVertices.removeLast(); + ingoingEdges.remove(removed); + } + + @Override + public Iterator iterator() { + return pathVertices.iterator(); + } + + public long refreshPathLatency() { + this.pathLatencyInMillis = 0; + for (ManagementVertex vertex : pathVertices) { + ManagementEdge ingoingEdge = ingoingEdges.get(vertex.getID()); + + if (ingoingEdge != null) { + this.pathLatencyInMillis += ((EdgeLatency) ingoingEdge.getAttachment()).getLatencyInMillis(); + } + this.pathLatencyInMillis += ((VertexLatency) vertex.getAttachment()).getLatencyInMillis(); + } + return this.pathLatencyInMillis; + } + + public long getPathLatencyInMillis() { + return this.pathLatencyInMillis; + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java new file mode 100644 index 0000000000000..0eb2f0e4de891 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java @@ -0,0 +1,126 @@ +package eu.stratosphere.nephele.streaming.latency; + +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; + +import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; +import eu.stratosphere.nephele.executiongraph.ExecutionVertex; +import eu.stratosphere.nephele.executiongraph.ManagementGraphFactory; +import eu.stratosphere.nephele.managementgraph.ManagementEdge; +import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; +import eu.stratosphere.nephele.managementgraph.ManagementGate; +import eu.stratosphere.nephele.managementgraph.ManagementGraph; +import eu.stratosphere.nephele.managementgraph.ManagementGroupVertex; +import eu.stratosphere.nephele.managementgraph.ManagementVertex; +import eu.stratosphere.nephele.managementgraph.ManagementVertexID; + +/** + * This class offers a way to find, store and compute the latencies of all possible paths between to + * {@link ExecutionGroupVertex} objects. Paths are computed on the {@link ExecutionVertex} level, not the + * {@link ExecutionGroupVertex} level, hence there may be many paths for high degrees of parallelization. + * + * @author Bjoern Lohrmann + */ +public class LatencySubgraph { + + private ManagementGroupVertex subgraphStart; + + private ManagementGroupVertex subgraphEnd; + + private List latencyPaths; + + private HashMap vertexLatencies; + + private HashMap edgeLatencies; + + public LatencySubgraph(ExecutionGraph executionGraph, ExecutionGroupVertex subgraphStart, + ExecutionGroupVertex subgraphEnd) { + + ManagementGraph managementGraph = ManagementGraphFactory.fromExecutionGraph(executionGraph); + determineAnchoringManagementGroupVertices(managementGraph, subgraphStart, subgraphEnd); + buildLatencyPaths(); + initLatenciesOnPaths(); + } + + private void initLatenciesOnPaths() { + for (LatencyPath path : latencyPaths) { + initLatenciesOnPath(path); + } + } + + private void initLatenciesOnPath(LatencyPath path) { + + for (ManagementVertex vertex : path) { + if (vertex.getAttachment() == null) { + VertexLatency vertexLatency = new VertexLatency(vertex); + vertex.setAttachment(vertexLatency); + vertexLatencies.put(vertex.getID(), vertexLatency); + } + + ManagementEdge ingoingEdge = path.getIngoingEdge(vertex); + if (ingoingEdge != null && ingoingEdge.getAttachment() == null) { + EdgeLatency edgeLatency = new EdgeLatency(ingoingEdge); + ingoingEdge.setAttachment(edgeLatency); + edgeLatencies.put(ingoingEdge.getManagementEdgeID(), edgeLatency); + } + } + } + + private void buildLatencyPaths() { + this.latencyPaths = new LinkedList(); + + for (int i = 0; i < subgraphStart.getNumberOfGroupMembers(); i++) { + ManagementVertex vertex = subgraphStart.getGroupMember(i); + LatencyPath initialPath = new LatencyPath(this, vertex); + depthFirstSearchLatencyPaths(initialPath, this.latencyPaths); + } + } + + /** + * Performs a recursive depth first search for {@link #subgraphEnd} starting at the end of the given path. + * All paths found to end in {@link #subgraphEnd} are added to foundLatencyPaths. + * + * @param path + * Initial path with at least one element to start with (will be altered during recursive search). + * @param foundLatencyPaths + * Accumulates the paths found to end at {@link #subgraphEnd} + */ + private void depthFirstSearchLatencyPaths(LatencyPath path, List foundLatencyPaths) { + ManagementVertex pathEnd = path.getEnd(); + + for (int i = 0; i < pathEnd.getNumberOfOutputGates(); i++) { + ManagementGate outputGate = pathEnd.getOutputGate(i); + + for (int j = 0; j < outputGate.getNumberOfForwardEdges(); j++) { + ManagementEdge edge = outputGate.getForwardEdge(j); + + ManagementVertex extension = edge.getTarget().getVertex(); + + path.appendVertex(extension, edge); + + if (extension.getGroupVertex() == subgraphEnd) { + foundLatencyPaths.add(new LatencyPath(path)); + } else { + depthFirstSearchLatencyPaths(path, foundLatencyPaths); + } + + path.removeLastVertex(); + } + } + } + + private void determineAnchoringManagementGroupVertices(ManagementGraph managementGraph, + ExecutionGroupVertex pathBeginExecVertex, + ExecutionGroupVertex pathEndExecVertex) { + + ManagementVertexID vertexInPathBeginGroup = pathBeginExecVertex.getGroupMember(0).getID() + .toManagementVertexID(); + this.subgraphStart = managementGraph.getVertexByID(vertexInPathBeginGroup).getGroupVertex(); + + ManagementVertexID vertexInPathEndGroup = pathEndExecVertex.getGroupMember(0).getID().toManagementVertexID(); + this.subgraphEnd = managementGraph.getVertexByID(vertexInPathEndGroup).getGroupVertex(); + } + +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java new file mode 100644 index 0000000000000..a31161039e335 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java @@ -0,0 +1,26 @@ +package eu.stratosphere.nephele.streaming.latency; + +import eu.stratosphere.nephele.managementgraph.ManagementVertex; + +public class VertexLatency { + + private ManagementVertex vertex; + + private long latencyInMillis; + + public VertexLatency(ManagementVertex vertex) { + this.vertex = vertex; + } + + public ManagementVertex getVertex() { + return vertex; + } + + public long getLatencyInMillis() { + return latencyInMillis; + } + + public void setLatencyInMillis(long latencyInMillis) { + this.latencyInMillis = latencyInMillis; + } +} From f45bdbe53a2821a3a1c37426e6381793a79629b2 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 19 Nov 2011 16:17:30 +0000 Subject: [PATCH 055/310] Refactored environment class to support encapsulation of tasks by monitor tasks --- .../nephele/execution/Environment.java | 940 +---------------- .../impl/EnvironmentListenerImpl.java | 7 +- .../checkpointing/EphemeralCheckpoint.java | 3 +- .../nephele/execution/RuntimeEnvironment.java | 974 ++++++++++++++++++ .../executiongraph/ExecutionStage.java | 4 +- .../executiongraph/ExecutionVertex.java | 13 +- .../nephele/instance/AbstractInstance.java | 5 +- .../scheduler/AbstractScheduler.java | 24 +- .../nephele/plugins/TaskManagerPlugin.java | 6 +- .../protocols/TaskOperationProtocol.java | 8 +- .../nephele/taskmanager/Task.java | 8 +- .../nephele/taskmanager/TaskManager.java | 28 +- .../taskmanager/TaskSubmissionWrapper.java | 10 +- .../ByteBufferedChannelManager.java | 32 +- .../taskmanager/bytebuffered/TaskContext.java | 6 +- .../streaming/StreamingJobManagerPlugin.java | 22 + .../streaming/StreamingTaskManagerPlugin.java | 6 +- .../runtime/test/util/MockEnvironment.java | 4 +- .../score/ScoreTaskManagerPlugin.java | 6 +- 19 files changed, 1137 insertions(+), 969 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java index 7bd58654b65ca..1dd5649649103 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java @@ -15,199 +15,111 @@ package eu.stratosphere.nephele.execution; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; -import eu.stratosphere.nephele.io.ChannelSelector; -import eu.stratosphere.nephele.io.DistributionPattern; -import eu.stratosphere.nephele.io.GateID; -import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.RecordDeserializer; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; -import eu.stratosphere.nephele.template.AbstractInvokable; import eu.stratosphere.nephele.template.InputSplitProvider; import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.types.StringRecord; -import eu.stratosphere.nephele.util.StringUtils; /** * The user code of every Nephele task runs inside an Environment object. The environment provides * important services to the task. It keeps track of setting up the communication channels and provides access to input * splits, memory manager, etc. - *

- * This class is thread-safe. * * @author warneke */ -public class Environment implements Runnable, IOReadableWritable { - - /** - * The log object used for debugging. - */ - private static final Log LOG = LogFactory.getLog(Environment.class); - - /** - * The interval to sleep in case a communication channel is not yet entirely set up (in milliseconds). - */ - private static final int SLEEPINTERVAL = 100; - - /** - * List of output gates created by the task. - */ - private final List> outputGates = new CopyOnWriteArrayList>(); - - /** - * List of input gates created by the task. - */ - private final List> inputGates = new CopyOnWriteArrayList>(); - - /** - * List of output gates which have to be rebound to a task after transferring the environment to a TaskManager. - */ - private final List> unboundOutputGates = new CopyOnWriteArrayList>(); - - /** - * List of input gates which have to be rebound to a task after transferring the environment to a TaskManager. - */ - private final List> unboundInputGates = new CopyOnWriteArrayList>(); +public interface Environment { /** - * The memory manager of the current environment (currently the one associated with the executing TaskManager). - */ - private volatile MemoryManager memoryManager; - - /** - * The io manager of the current environment (currently the one associated with the executing TaskManager). - */ - private volatile IOManager ioManager; - - /** - * Class of the task to run in this environment. - */ - private volatile Class invokableClass = null; - - /** - * Instance of the class to be run in this environment. - */ - private volatile AbstractInvokable invokable = null; - - /** - * The thread executing the task in the environment. - */ - private volatile Thread executingThread = null; - - /** - * The ID of the job this task belongs to. - */ - private volatile JobID jobID = null; - - /** - * The runtime configuration of the task encapsulated in the environment object. + * Returns the ID of the job from the original job graph. It is used by the library cache manager to find the + * required + * libraries for executing the assigned Nephele task. + * + * @return the ID of the job from the original job graph */ - private volatile Configuration runtimeConfiguration = null; + JobID getJobID(); /** - * The input split provider that can be queried for new input splits. + * Returns the runtime configuration object which was attached to the original {@link JobVertex}. + * + * @return the runtime configuration object which was attached to the original {@link JobVertex} */ - private volatile InputSplitProvider inputSplitProvider = null; + Configuration getRuntimeConfiguration(); /** - * The observer object for the task's execution. + * Returns the current number of subtasks the respective task is split into. + * + * @return the current number of subtasks the respective task is split into */ - private volatile ExecutionObserver executionObserver = null; + int getCurrentNumberOfSubtasks(); /** - * The current number of subtasks the respective task is split into. + * Returns the index of this subtask in the subtask group. + * + * @return the index of this subtask in the subtask group */ - private volatile int currentNumberOfSubtasks = 1; + int getIndexInSubtaskGroup(); /** - * The index of this subtask in the subtask group. + * Sends a notification that objects that a new user thread has been started to the execution observer. + * + * @param userThread + * the user thread which has been started */ - private volatile int indexInSubtaskGroup = 0; + void userThreadStarted(Thread userThread); /** - * The name of the task running in this environment. + * Sends a notification that a user thread has finished to the execution observer. + * + * @param userThread + * the user thread which has finished */ - private volatile String taskName; + void userThreadFinished(Thread userThread); /** - * Creates a new environment object which contains the runtime information for the encapsulated Nephele task. + * Returns the input split provider assigned to this environment. * - * @param jobID - * the ID of the original Nephele job - * @param taskName - * the name of task running in this environment - * @param invokableClass - * invokableClass the class that should be instantiated as a Nephele task - * @param runtimeConfiguration - * the configuration object which was attached to the original {@link JobVertex} + * @return the input split provider or null if no such provider has been assigned to this environment. */ - public Environment(final JobID jobID, final String taskName, - final Class invokableClass, final Configuration runtimeConfiguration) { - this.jobID = jobID; - this.taskName = taskName; - this.invokableClass = invokableClass; - this.runtimeConfiguration = runtimeConfiguration; - } + InputSplitProvider getInputSplitProvider(); /** - * Empty constructor used to deserialize the object. + * Returns the current {@link IOManager}. + * + * @return the current {@link IOManager}. */ - public Environment() { - } + IOManager getIOManager(); /** - * Returns the invokable object that represents the Nephele task. + * Returns the current {@link MemoryManager}. * - * @return the invokable object that represents the Nephele task + * @return the current {@link MemoryManager}. */ - public AbstractInvokable getInvokable() { - return this.invokable; - } + MemoryManager getMemoryManager(); /** - * Returns the ID of the job from the original job graph. It is used by the library cache manager to find the - * required - * libraries for executing the assigned Nephele task. + * Returns the name of the task running in this environment. * - * @return the ID of the job from the original job graph + * @return the name of the task running in this environment */ - public JobID getJobID() { - return this.jobID; - } + String getTaskName(); /** * Checks if the environment has unbound input gates. * * @return true if the environment has unbound input gates, false otherwise */ - public boolean hasUnboundInputGates() { - - return (this.unboundInputGates.size() > 0); - } + boolean hasUnboundInputGates(); /** * Checks if the environment has unbound output gates. * * @return true if the environment has unbound output gates, false otherwise */ - public boolean hasUnboundOutputGates() { - - return (this.unboundOutputGates.size() > 0); - } + boolean hasUnboundOutputGates(); /** * Retrieves and removes the unbound output gate with the given ID from the list of unbound output gates. @@ -216,16 +128,7 @@ public boolean hasUnboundOutputGates() { * the index of the unbound output gate * @return the unbound output gate with the given ID, or null if no such gate exists */ - public OutputGate getUnboundOutputGate(final int gateID) { - - if (this.unboundOutputGates.size() == 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("No unbound output gates"); - } - return null; - } - return this.unboundOutputGates.remove(gateID); - } + OutputGate getUnboundOutputGate(final int gateID); /** * Retrieves and removes unbound input gate with the given ID from the list of unbound input gates. @@ -234,162 +137,21 @@ public OutputGate getUnboundOutputGate(final int gateID) { * the index of the unbound input gate * @return the unbound input gate with the given ID, or null if no such gate exists */ - public InputGate getUnboundInputGate(final int gateID) { - - if (this.unboundInputGates.size() == 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("No unbound input gates"); - } - return null; - } - - return this.unboundInputGates.remove(gateID); - } + InputGate getUnboundInputGate(final int gateID); /** - * Creates a new instance of the Nephele task and registers it with its - * environment. + * Returns the number of output gates registered with this environment. * - * @throws Exception - * any exception that might be thrown by the user code during instantiation and registration of input and - * output channels - */ - public void instantiateInvokable() throws Exception { - - // Test and set, protected by synchronized block - synchronized (this) { - - if (this.invokableClass == null) { - LOG.fatal("InvokableClass is null"); - return; - } - - try { - this.invokable = this.invokableClass.newInstance(); - } catch (InstantiationException e) { - LOG.error(e); - } catch (IllegalAccessException e) { - LOG.error(e); - } - } - - this.invokable.setEnvironment(this); - this.invokable.registerInputOutput(); - - if (this.jobID == null) { - LOG.warn("jobVertexID is null"); - } - } - - /** - * {@inheritDoc} + * @return the number of output gates registered with this environment */ - @Override - public void run() { - - if (invokable == null) { - LOG.fatal("ExecutionEnvironment has no Invokable set"); - } - - // Now the actual program starts to run - changeExecutionState(ExecutionState.RUNNING, null); - - // If the task has been canceled in the mean time, do not even start it - if (this.executionObserver.isCanceled()) { - changeExecutionState(ExecutionState.CANCELED, null); - return; - } - - try { - - // Activate input channels - activateInputChannels(); - - this.invokable.invoke(); - - // Make sure, we enter the catch block when the task has been canceled - if (this.executionObserver.isCanceled()) { - throw new InterruptedException(); - } - - } catch (Exception e) { - - if (!this.executionObserver.isCanceled()) { - - // Perform clean up when the task failed and has been not canceled by the user - try { - this.invokable.cancel(); - } catch (Exception e2) { - LOG.error(StringUtils.stringifyException(e2)); - } - } - - // Release all resources that may currently be allocated by the individual channels - releaseAllChannelResources(); - - if (this.executionObserver.isCanceled()) { - changeExecutionState(ExecutionState.CANCELED, null); - } else { - changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(e)); - } - - return; - } - - // Task finished running, but there may be unconsumed output data in some of the channels - changeExecutionState(ExecutionState.FINISHING, null); - - try { - // If there is any unclosed input gate, close it and propagate close operation to corresponding output gate - closeInputGates(); - - // First, close all output gates to indicate no records will be emitted anymore - requestAllOutputGatesToClose(); - - // Wait until all input channels are closed - waitForInputChannelsToBeClosed(); - - // Now we wait until all output channels have written out their data and are closed - waitForOutputChannelsToBeClosed(); - } catch (Exception e) { - - // Release all resources that may currently be allocated by the individual channels - releaseAllChannelResources(); - - if (this.executionObserver.isCanceled()) { - changeExecutionState(ExecutionState.CANCELED, null); - } else { - changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(e)); - } - - return; - } - - // Release all resources that may currently be allocated by the individual channels - releaseAllChannelResources(); - - // Finally, switch execution state to FINISHED and report to job manager - changeExecutionState(ExecutionState.FINISHED, null); - } + int getNumberOfOutputGates(); /** - * Activates all of the task's input channels. + * Returns the number of input gates registered with this environment. * - * @throws IOException - * thrown if an I/O error occurs while transmitting one of the activation requests to the corresponding - * output channels - * @throws InterruptedException - * throws if the task is interrupted while waiting for the activation process to complete + * @return the number of input gates registered with this environment */ - private void activateInputChannels() throws IOException, InterruptedException { - - for (int i = 0; i < getNumberOfInputGates(); ++i) { - final InputGate eig = getInputGate(i); - for (int j = 0; j < eig.getNumberOfInputChannels(); ++j) { - eig.getInputChannel(j).activate(); - } - } - } + int getNumberOfInputGates(); /** * Registers an output gate with the environment. @@ -397,10 +159,7 @@ private void activateInputChannels() throws IOException, InterruptedException { * @param outputGate * the output gate to be registered with the environment */ - public void registerOutputGate(final OutputGate outputGate) { - - this.outputGates.add(outputGate); - } + void registerOutputGate(final OutputGate outputGate); /** * Registers an input gate with the environment. @@ -408,596 +167,5 @@ public void registerOutputGate(final OutputGate outputGate) { * @param inputGate * the input gate to be registered with the environment */ - public void registerInputGate(final InputGate inputGate) { - - this.inputGates.add(inputGate); - } - - /** - * Returns the number of output gates registered with this environment. - * - * @return the number of output gates registered with this environment - */ - public int getNumberOfOutputGates() { - return this.outputGates.size(); - } - - /** - * Returns the number of input gates registered with this environment. - * - * @return the number of input gates registered with this environment - */ - public int getNumberOfInputGates() { - return this.inputGates.size(); - } - - /** - * Returns the registered input gate with index pos. - * - * @param pos - * the index of the input gate to return - * @return the input gate at index pos or null if no such index exists - */ - public InputGate getInputGate(final int pos) { - if (pos < this.inputGates.size()) { - return this.inputGates.get(pos); - } - - return null; - } - - /** - * Returns the registered output gate with index pos. - * - * @param pos - * the index of the output gate to return - * @return the output gate at index pos or null if no such index exists - */ - public OutputGate getOutputGate(final int pos) { - if (pos < this.outputGates.size()) { - return this.outputGates.get(pos); - } - - return null; - } - - /** - * Returns the thread which is assigned to execute the user code. - * - * @return the thread which is assigned to execute the user code - */ - public Thread getExecutingThread() { - - synchronized (this) { - - if (this.executingThread == null) { - if (this.taskName == null) { - this.executingThread = new Thread(this); - } else { - this.executingThread = new Thread(this, this.taskName); - } - } - - return this.executingThread; - } - } - - // TODO: See if type safety can be improved here - /** - * {@inheritDoc} - */ - @SuppressWarnings("unchecked") - @Override - public void read(final DataInput in) throws IOException { - - // Read job vertex id - this.jobID = new JobID(); - this.jobID.read(in); - - // Read the task name - this.taskName = StringRecord.readString(in); - - // Read names of required jar files - final String[] requiredJarFiles = new String[in.readInt()]; - for (int i = 0; i < requiredJarFiles.length; i++) { - requiredJarFiles[i] = StringRecord.readString(in); - } - - // Now register data with the library manager - LibraryCacheManager.register(this.jobID, requiredJarFiles); - - // Get ClassLoader from Library Manager - final ClassLoader cl = LibraryCacheManager.getClassLoader(this.jobID); - - // Read the name of the invokable class; - final String invokableClassName = StringRecord.readString(in); - - if (invokableClassName == null) { - throw new IOException("invokableClassName is null"); - } - - try { - this.invokableClass = (Class) Class.forName(invokableClassName, true, cl); - } catch (ClassNotFoundException cnfe) { - throw new IOException("Class " + invokableClassName + " not found in one of the supplied jar files: " - + StringUtils.stringifyException(cnfe)); - } - - final int numOuputGates = in.readInt(); - - for (int i = 0; i < numOuputGates; i++) { - - final GateID gateID = new GateID(); - gateID.read(in); - - final String typeClassName = StringRecord.readString(in); - Class type = null; - try { - type = (Class) Class.forName(typeClassName, true, cl); - } catch (ClassNotFoundException cnfe) { - throw new IOException("Class " + typeClassName + " not found in one of the supplied jar files: " - + StringUtils.stringifyException(cnfe)); - } - - final boolean isBroadcast = in.readBoolean(); - - ChannelSelector channelSelector = null; - if (!isBroadcast) { - - final String channelSelectorClassName = StringRecord.readString(in); - try { - channelSelector = (ChannelSelector) Class.forName(channelSelectorClassName, true, - cl).newInstance(); - } catch (InstantiationException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (IllegalAccessException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (ClassNotFoundException e) { - throw new IOException(StringUtils.stringifyException(e)); - } - - channelSelector.read(in); - } - - @SuppressWarnings("rawtypes") - final OutputGate eog = new OutputGate(this.jobID, gateID, type, i, channelSelector, - isBroadcast); - eog.read(in); - this.outputGates.add(eog); - // Mark as unbound for reconnection of RecordWriter - this.unboundOutputGates.add(eog); - } - - final int numInputGates = in.readInt(); - - for (int i = 0; i < numInputGates; i++) { - - final GateID gateID = new GateID(); - gateID.read(in); - - final String deserializerClassName = StringRecord.readString(in); - RecordDeserializer recordDeserializer = null; - Class> deserializerClass = null; - try { - deserializerClass = (Class>) cl - .loadClass(deserializerClassName); - recordDeserializer = deserializerClass.newInstance(); - - } catch (ClassNotFoundException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (InstantiationException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (IllegalAccessException e) { - throw new IOException(StringUtils.stringifyException(e)); - } - - recordDeserializer.setClassLoader(cl); - recordDeserializer.read(in); - - final String distributionPatternClassName = StringRecord.readString(in); - DistributionPattern distributionPattern = null; - Class distributionPatternClass = null; - try { - distributionPatternClass = (Class) cl - .loadClass(distributionPatternClassName); - - distributionPattern = distributionPatternClass.newInstance(); - - } catch (ClassNotFoundException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (InstantiationException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (IllegalAccessException e) { - throw new IOException(StringUtils.stringifyException(e)); - } - - @SuppressWarnings("rawtypes") - final InputGate eig = new InputGate(this.jobID, gateID, recordDeserializer, i, - distributionPattern); - eig.read(in); - this.inputGates.add(eig); - // Mark as unbound for reconnection of RecordReader - this.unboundInputGates.add(eig); - } - - // The configuration object - this.runtimeConfiguration = new Configuration(); - this.runtimeConfiguration.read(in); - - // The current of number subtasks - this.currentNumberOfSubtasks = in.readInt(); - // The index in the subtask group - this.indexInSubtaskGroup = in.readInt(); - - // Finally, instantiate the invokable object - try { - instantiateInvokable(); - } catch (Exception e) { - throw new IOException(StringUtils.stringifyException(e)); - } - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - - // Write out job vertex id - if (this.jobID == null) { - throw new IOException("this.jobID is null"); - } - - this.jobID.write(out); - - // Write the task name - StringRecord.writeString(out, this.taskName); - - // Write out the names of the required jar files - final String[] requiredJarFiles = LibraryCacheManager.getRequiredJarFiles(this.jobID); - - out.writeInt(requiredJarFiles.length); - for (int i = 0; i < requiredJarFiles.length; i++) { - StringRecord.writeString(out, requiredJarFiles[i]); - } - - // Write out the name of the invokable class - if (this.invokableClass == null) { - throw new IOException("this.invokableClass is null"); - } - - StringRecord.writeString(out, this.invokableClass.getName()); - - // Output gates - out.writeInt(getNumberOfOutputGates()); - for (int i = 0; i < getNumberOfOutputGates(); i++) { - final OutputGate outputGate = getOutputGate(i); - outputGate.getGateID().write(out); - StringRecord.writeString(out, outputGate.getType().getName()); - out.writeBoolean(outputGate.isBroadcast()); - if (!outputGate.isBroadcast()) { - // Write out class name of channel selector - StringRecord.writeString(out, outputGate.getChannelSelector().getClass().getName()); - outputGate.getChannelSelector().write(out); - } - - getOutputGate(i).write(out); - } - - // Input gates - out.writeInt(getNumberOfInputGates()); - for (int i = 0; i < getNumberOfInputGates(); i++) { - final InputGate inputGate = getInputGate(i); - inputGate.getGateID().write(out); - StringRecord.writeString(out, inputGate.getRecordDeserializer().getClass().getName()); - inputGate.getRecordDeserializer().write(out); - StringRecord.writeString(out, inputGate.getDistributionPattern().getClass().getName()); - getInputGate(i).write(out); - } - - // The configuration object - this.runtimeConfiguration.write(out); - - // The current of number subtasks - out.writeInt(this.currentNumberOfSubtasks); - // The index in the subtask group - out.writeInt(this.indexInSubtaskGroup); - } - - /** - * Blocks until all output channels are closed. - * - * @throws IOException - * thrown if an error occurred while closing the output channels - * @throws InterruptedException - * thrown if the thread waiting for the channels to be closed is interrupted - */ - private void waitForOutputChannelsToBeClosed() throws IOException, InterruptedException { - - // Wait for disconnection of all output gates - while (true) { - - // Make sure, we leave this method with an InterruptedException when the task has been canceled - if (this.executionObserver.isCanceled()) { - throw new InterruptedException(); - } - - boolean allClosed = true; - for (int i = 0; i < getNumberOfOutputGates(); i++) { - final OutputGate eog = getOutputGate(i); - if (!eog.isClosed()) { - allClosed = false; - } - } - - if (allClosed) { - break; - } else { - Thread.sleep(SLEEPINTERVAL); - } - } - } - - /** - * Blocks until all input channels are closed. - * - * @throws IOException - * thrown if an error occurred while closing the input channels - * @throws InterruptedException - * thrown if the thread waiting for the channels to be closed is interrupted - */ - private void waitForInputChannelsToBeClosed() throws IOException, InterruptedException { - - // Wait for disconnection of all output gates - while (true) { - - // Make sure, we leave this method with an InterruptedException when the task has been canceled - if (this.executionObserver.isCanceled()) { - throw new InterruptedException(); - } - - boolean allClosed = true; - for (int i = 0; i < getNumberOfInputGates(); i++) { - final InputGate eig = getInputGate(i); - if (!eig.isClosed()) { - allClosed = false; - } - } - - if (allClosed) { - break; - } else { - Thread.sleep(SLEEPINTERVAL); - } - } - } - - /** - * Closes all input gates which are not already closed. - */ - private void closeInputGates() throws IOException, InterruptedException { - - for (int i = 0; i < getNumberOfInputGates(); i++) { - final InputGate eig = getInputGate(i); - // Important: close must be called on each input gate exactly once - eig.close(); - } - - } - - /** - * Requests all output gates to be closed. - */ - private void requestAllOutputGatesToClose() throws IOException, InterruptedException { - - for (int i = 0; i < getNumberOfOutputGates(); i++) { - this.getOutputGate(i).requestClose(); - } - } - - /** - * Returns a duplicate (deep copy) of this environment object. However, duplication - * does not cover the gates arrays. They must be manually reconstructed. - * - * @return a duplicate (deep copy) of this environment object - * @throws Exception - * any exception that might be thrown by the user code during instantiation and registration of input and - * output channels - */ - public Environment duplicateEnvironment() throws Exception { - - final Environment duplicatedEnvironment = new Environment(); - duplicatedEnvironment.invokableClass = this.invokableClass; - duplicatedEnvironment.jobID = this.jobID; - duplicatedEnvironment.taskName = this.taskName; - Thread tmpThread = null; - synchronized (this) { - tmpThread = this.executingThread; - } - synchronized (duplicatedEnvironment) { - duplicatedEnvironment.executingThread = tmpThread; - } - duplicatedEnvironment.runtimeConfiguration = this.runtimeConfiguration; - - // We instantiate the invokable of the new environment - duplicatedEnvironment.instantiateInvokable(); - - return duplicatedEnvironment; - } - - /** - * Returns the current {@link IOManager}. - * - * @return the current {@link IOManager}. - */ - public IOManager getIOManager() { - return this.ioManager; - } - - /** - * Sets the {@link IOManager}. - * - * @param memoryManager - * the new {@link IOManager} - */ - public void setIOManager(final IOManager ioManager) { - this.ioManager = ioManager; - } - - /** - * Returns the current {@link MemoryManager}. - * - * @return the current {@link MemoryManager}. - */ - public MemoryManager getMemoryManager() { - return this.memoryManager; - } - - /** - * Sets the {@link MemoryManager}. - * - * @param memoryManager - * the new {@link MemoryManager} - */ - public void setMemoryManager(final MemoryManager memoryManager) { - this.memoryManager = memoryManager; - } - - /** - * Returns the runtime configuration object which was attached to the original {@link JobVertex}. - * - * @return the runtime configuration object which was attached to the original {@link JobVertex} - */ - public Configuration getRuntimeConfiguration() { - return this.runtimeConfiguration; - } - - /** - * Returns the current number of subtasks the respective task is split into. - * - * @return the current number of subtasks the respective task is split into - */ - public int getCurrentNumberOfSubtasks() { - - return this.currentNumberOfSubtasks; - } - - /** - * Sets the current number of subtasks the respective task is split into. - * - * @param currentNumberOfSubtasks - * the current number of subtasks the respective task is split into - */ - public void setCurrentNumberOfSubtasks(final int currentNumberOfSubtasks) { - - this.currentNumberOfSubtasks = currentNumberOfSubtasks; - } - - /** - * Returns the index of this subtask in the subtask group. - * - * @return the index of this subtask in the subtask group - */ - public int getIndexInSubtaskGroup() { - - return this.indexInSubtaskGroup; - } - - /** - * Sets the index of this subtask in the subtask group. - * - * @param indexInSubtaskGroup - * the index of this subtask in the subtask group - */ - public void setIndexInSubtaskGroup(final int indexInSubtaskGroup) { - - this.indexInSubtaskGroup = indexInSubtaskGroup; - } - - private void changeExecutionState(final ExecutionState newExecutionState, final String optionalMessage) { - - if (this.executionObserver != null) { - this.executionObserver.executionStateChanged(newExecutionState, optionalMessage); - } - } - - /** - * Returns the name of the task running in this environment. - * - * @return the name of the task running in this environment - */ - public String getTaskName() { - - return this.taskName; - } - - /** - * Sets the execution observer for this environment. - * - * @param executionObserver - * the execution observer for this environment - */ - public void setExecutionObserver(final ExecutionObserver executionObserver) { - this.executionObserver = executionObserver; - } - - /** - * Sets the input split provider for this environment. - * - * @param inputSplitProvider - * the input split provider for this environment - */ - public void setInputSplitProvider(final InputSplitProvider inputSplitProvider) { - this.inputSplitProvider = inputSplitProvider; - } - - /** - * Returns the input split provider assigned to this environment. - * - * @return the input split provider or null if no such provider has been assigned to this environment. - */ - public InputSplitProvider getInputSplitProvider() { - return this.inputSplitProvider; - } - - /** - * Sends a notification that objects that a new user thread has been started to the execution observer. - * - * @param userThread - * the user thread which has been started - */ - public void userThreadStarted(final Thread userThread) { - - if (this.executionObserver != null) { - this.executionObserver.userThreadStarted(userThread); - } - } - - /** - * Sends a notification that a user thread has finished to the execution observer. - * - * @param userThread - * the user thread which has finished - */ - public void userThreadFinished(final Thread userThread) { - - if (this.executionObserver != null) { - this.executionObserver.userThreadFinished(userThread); - } - } - - /** - * Releases the allocated resources (particularly buffer) of input and output channels attached to this task. This - * method should only be called after the respected task has stopped running. - */ - private void releaseAllChannelResources() { - - for (int i = 0; i < getNumberOfInputGates(); i++) { - this.getInputGate(i).releaseAllChannelResources(); - } - - for (int i = 0; i < getNumberOfOutputGates(); i++) { - this.getOutputGate(i).releaseAllChannelResources(); - } - } + void registerInputGate(final InputGate inputGate); } diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java index d9803c1a7a99a..eebe12514c3fb 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java @@ -18,10 +18,10 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; @@ -31,9 +31,10 @@ public class EnvironmentListenerImpl implements ExecutionListener { private final TaskManagerProfilerImpl taskManagerProfiler; - private final Environment environment; + private final RuntimeEnvironment environment; - public EnvironmentListenerImpl(final TaskManagerProfilerImpl taskManagerProfiler, final Environment environment) { + public EnvironmentListenerImpl(final TaskManagerProfilerImpl taskManagerProfiler, + final RuntimeEnvironment environment) { this.taskManagerProfiler = taskManagerProfiler; this.environment = environment; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 998941f0cf5a4..b951f055ca587 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -33,6 +33,7 @@ import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.event.task.EventList; import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.BufferFactory; @@ -131,7 +132,7 @@ public EphemeralCheckpoint(final Task task, final boolean ephemeral) { // Determine number of output channel int nooc = 0; - final Environment environment = task.getEnvironment(); + final RuntimeEnvironment environment = task.getEnvironment(); for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { nooc += environment.getOutputGate(i).getNumberOfOutputChannels(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java new file mode 100644 index 0000000000000..5451e33bfbf1d --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -0,0 +1,974 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.execution; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; +import eu.stratosphere.nephele.io.ChannelSelector; +import eu.stratosphere.nephele.io.DistributionPattern; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RecordDeserializer; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.services.iomanager.IOManager; +import eu.stratosphere.nephele.services.memorymanager.MemoryManager; +import eu.stratosphere.nephele.template.AbstractInvokable; +import eu.stratosphere.nephele.template.InputSplitProvider; +import eu.stratosphere.nephele.types.Record; +import eu.stratosphere.nephele.types.StringRecord; +import eu.stratosphere.nephele.util.StringUtils; + +/** + * The user code of every Nephele task runs inside a RuntimeEnvironment object. The environment provides + * important services to the task. It keeps track of setting up the communication channels and provides access to input + * splits, memory manager, etc. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public class RuntimeEnvironment implements Environment, Runnable, IOReadableWritable { + + /** + * The log object used for debugging. + */ + private static final Log LOG = LogFactory.getLog(RuntimeEnvironment.class); + + /** + * The interval to sleep in case a communication channel is not yet entirely set up (in milliseconds). + */ + private static final int SLEEPINTERVAL = 100; + + /** + * List of output gates created by the task. + */ + private final List> outputGates = new CopyOnWriteArrayList>(); + + /** + * List of input gates created by the task. + */ + private final List> inputGates = new CopyOnWriteArrayList>(); + + /** + * List of output gates which have to be rebound to a task after transferring the environment to a TaskManager. + */ + private final List> unboundOutputGates = new CopyOnWriteArrayList>(); + + /** + * List of input gates which have to be rebound to a task after transferring the environment to a TaskManager. + */ + private final List> unboundInputGates = new CopyOnWriteArrayList>(); + + /** + * The memory manager of the current environment (currently the one associated with the executing TaskManager). + */ + private volatile MemoryManager memoryManager; + + /** + * The io manager of the current environment (currently the one associated with the executing TaskManager). + */ + private volatile IOManager ioManager; + + /** + * Class of the task to run in this environment. + */ + private volatile Class invokableClass = null; + + /** + * Instance of the class to be run in this environment. + */ + private volatile AbstractInvokable invokable = null; + + /** + * The thread executing the task in the environment. + */ + private volatile Thread executingThread = null; + + /** + * The ID of the job this task belongs to. + */ + private volatile JobID jobID = null; + + /** + * The runtime configuration of the task encapsulated in the environment object. + */ + private volatile Configuration runtimeConfiguration = null; + + /** + * The input split provider that can be queried for new input splits. + */ + private volatile InputSplitProvider inputSplitProvider = null; + + /** + * The observer object for the task's execution. + */ + private volatile ExecutionObserver executionObserver = null; + + /** + * The current number of subtasks the respective task is split into. + */ + private volatile int currentNumberOfSubtasks = 1; + + /** + * The index of this subtask in the subtask group. + */ + private volatile int indexInSubtaskGroup = 0; + + /** + * The name of the task running in this environment. + */ + private volatile String taskName; + + /** + * Creates a new runtime environment object which contains the runtime information for the encapsulated Nephele + * task. + * + * @param jobID + * the ID of the original Nephele job + * @param taskName + * the name of task running in this environment + * @param invokableClass + * invokableClass the class that should be instantiated as a Nephele task + * @param runtimeConfiguration + * the configuration object which was attached to the original {@link JobVertex} + */ + public RuntimeEnvironment(final JobID jobID, final String taskName, + final Class invokableClass, final Configuration runtimeConfiguration) { + this.jobID = jobID; + this.taskName = taskName; + this.invokableClass = invokableClass; + this.runtimeConfiguration = runtimeConfiguration; + } + + /** + * Empty constructor used to deserialize the object. + */ + public RuntimeEnvironment() { + } + + /** + * Returns the invokable object that represents the Nephele task. + * + * @return the invokable object that represents the Nephele task + */ + public AbstractInvokable getInvokable() { + return this.invokable; + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + return this.jobID; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasUnboundInputGates() { + + return (this.unboundInputGates.size() > 0); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasUnboundOutputGates() { + + return (this.unboundOutputGates.size() > 0); + } + + /** + * {@inheritDoc} + */ + @Override + public OutputGate getUnboundOutputGate(final int gateID) { + + if (this.unboundOutputGates.size() == 0) { + if (LOG.isDebugEnabled()) { + LOG.debug("No unbound output gates"); + } + return null; + } + return this.unboundOutputGates.remove(gateID); + } + + /** + * {@inheritDoc} + */ + @Override + public InputGate getUnboundInputGate(final int gateID) { + + if (this.unboundInputGates.size() == 0) { + if (LOG.isDebugEnabled()) { + LOG.debug("No unbound input gates"); + } + return null; + } + + return this.unboundInputGates.remove(gateID); + } + + /** + * Creates a new instance of the Nephele task and registers it with its + * environment. + * + * @throws Exception + * any exception that might be thrown by the user code during instantiation and registration of input and + * output channels + */ + public void instantiateInvokable() throws Exception { + + // Test and set, protected by synchronized block + synchronized (this) { + + if (this.invokableClass == null) { + LOG.fatal("InvokableClass is null"); + return; + } + + try { + this.invokable = this.invokableClass.newInstance(); + } catch (InstantiationException e) { + LOG.error(e); + } catch (IllegalAccessException e) { + LOG.error(e); + } + } + + this.invokable.setEnvironment(this); + this.invokable.registerInputOutput(); + + if (this.jobID == null) { + LOG.warn("jobVertexID is null"); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void run() { + + if (invokable == null) { + LOG.fatal("ExecutionEnvironment has no Invokable set"); + } + + // Now the actual program starts to run + changeExecutionState(ExecutionState.RUNNING, null); + + // If the task has been canceled in the mean time, do not even start it + if (this.executionObserver.isCanceled()) { + changeExecutionState(ExecutionState.CANCELED, null); + return; + } + + try { + + // Activate input channels + activateInputChannels(); + + this.invokable.invoke(); + + // Make sure, we enter the catch block when the task has been canceled + if (this.executionObserver.isCanceled()) { + throw new InterruptedException(); + } + + } catch (Exception e) { + + if (!this.executionObserver.isCanceled()) { + + // Perform clean up when the task failed and has been not canceled by the user + try { + this.invokable.cancel(); + } catch (Exception e2) { + LOG.error(StringUtils.stringifyException(e2)); + } + } + + // Release all resources that may currently be allocated by the individual channels + releaseAllChannelResources(); + + if (this.executionObserver.isCanceled()) { + changeExecutionState(ExecutionState.CANCELED, null); + } else { + changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(e)); + } + + return; + } + + // Task finished running, but there may be unconsumed output data in some of the channels + changeExecutionState(ExecutionState.FINISHING, null); + + try { + // If there is any unclosed input gate, close it and propagate close operation to corresponding output gate + closeInputGates(); + + // First, close all output gates to indicate no records will be emitted anymore + requestAllOutputGatesToClose(); + + // Wait until all input channels are closed + waitForInputChannelsToBeClosed(); + + // Now we wait until all output channels have written out their data and are closed + waitForOutputChannelsToBeClosed(); + } catch (Exception e) { + + // Release all resources that may currently be allocated by the individual channels + releaseAllChannelResources(); + + if (this.executionObserver.isCanceled()) { + changeExecutionState(ExecutionState.CANCELED, null); + } else { + changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(e)); + } + + return; + } + + // Release all resources that may currently be allocated by the individual channels + releaseAllChannelResources(); + + // Finally, switch execution state to FINISHED and report to job manager + changeExecutionState(ExecutionState.FINISHED, null); + } + + /** + * Activates all of the task's input channels. + * + * @throws IOException + * thrown if an I/O error occurs while transmitting one of the activation requests to the corresponding + * output channels + * @throws InterruptedException + * throws if the task is interrupted while waiting for the activation process to complete + */ + private void activateInputChannels() throws IOException, InterruptedException { + + for (int i = 0; i < getNumberOfInputGates(); ++i) { + final InputGate eig = getInputGate(i); + for (int j = 0; j < eig.getNumberOfInputChannels(); ++j) { + eig.getInputChannel(j).activate(); + } + } + } + + /** + * {@inheritDoc} + */ + public void registerOutputGate(final OutputGate outputGate) { + + this.outputGates.add(outputGate); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerInputGate(final InputGate inputGate) { + + this.inputGates.add(inputGate); + } + + /** + * {@inheritDoc} + */ + public int getNumberOfOutputGates() { + return this.outputGates.size(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfInputGates() { + return this.inputGates.size(); + } + + /** + * Returns the registered input gate with index pos. + * + * @param pos + * the index of the input gate to return + * @return the input gate at index pos or null if no such index exists + */ + public InputGate getInputGate(final int pos) { + if (pos < this.inputGates.size()) { + return this.inputGates.get(pos); + } + + return null; + } + + /** + * Returns the registered output gate with index pos. + * + * @param pos + * the index of the output gate to return + * @return the output gate at index pos or null if no such index exists + */ + public OutputGate getOutputGate(final int pos) { + if (pos < this.outputGates.size()) { + return this.outputGates.get(pos); + } + + return null; + } + + /** + * Returns the thread which is assigned to execute the user code. + * + * @return the thread which is assigned to execute the user code + */ + public Thread getExecutingThread() { + + synchronized (this) { + + if (this.executingThread == null) { + if (this.taskName == null) { + this.executingThread = new Thread(this); + } else { + this.executingThread = new Thread(this, this.taskName); + } + } + + return this.executingThread; + } + } + + // TODO: See if type safety can be improved here + /** + * {@inheritDoc} + */ + @SuppressWarnings("unchecked") + @Override + public void read(final DataInput in) throws IOException { + + // Read job vertex id + this.jobID = new JobID(); + this.jobID.read(in); + + // Read the task name + this.taskName = StringRecord.readString(in); + + // Read names of required jar files + final String[] requiredJarFiles = new String[in.readInt()]; + for (int i = 0; i < requiredJarFiles.length; i++) { + requiredJarFiles[i] = StringRecord.readString(in); + } + + // Now register data with the library manager + LibraryCacheManager.register(this.jobID, requiredJarFiles); + + // Get ClassLoader from Library Manager + final ClassLoader cl = LibraryCacheManager.getClassLoader(this.jobID); + + // Read the name of the invokable class; + final String invokableClassName = StringRecord.readString(in); + + if (invokableClassName == null) { + throw new IOException("invokableClassName is null"); + } + + try { + this.invokableClass = (Class) Class.forName(invokableClassName, true, cl); + } catch (ClassNotFoundException cnfe) { + throw new IOException("Class " + invokableClassName + " not found in one of the supplied jar files: " + + StringUtils.stringifyException(cnfe)); + } + + final int numOuputGates = in.readInt(); + + for (int i = 0; i < numOuputGates; i++) { + + final GateID gateID = new GateID(); + gateID.read(in); + + final String typeClassName = StringRecord.readString(in); + Class type = null; + try { + type = (Class) Class.forName(typeClassName, true, cl); + } catch (ClassNotFoundException cnfe) { + throw new IOException("Class " + typeClassName + " not found in one of the supplied jar files: " + + StringUtils.stringifyException(cnfe)); + } + + final boolean isBroadcast = in.readBoolean(); + + ChannelSelector channelSelector = null; + if (!isBroadcast) { + + final String channelSelectorClassName = StringRecord.readString(in); + try { + channelSelector = (ChannelSelector) Class.forName(channelSelectorClassName, true, + cl).newInstance(); + } catch (InstantiationException e) { + throw new IOException(StringUtils.stringifyException(e)); + } catch (IllegalAccessException e) { + throw new IOException(StringUtils.stringifyException(e)); + } catch (ClassNotFoundException e) { + throw new IOException(StringUtils.stringifyException(e)); + } + + channelSelector.read(in); + } + + @SuppressWarnings("rawtypes") + final OutputGate eog = new OutputGate(this.jobID, gateID, type, i, channelSelector, + isBroadcast); + eog.read(in); + this.outputGates.add(eog); + // Mark as unbound for reconnection of RecordWriter + this.unboundOutputGates.add(eog); + } + + final int numInputGates = in.readInt(); + + for (int i = 0; i < numInputGates; i++) { + + final GateID gateID = new GateID(); + gateID.read(in); + + final String deserializerClassName = StringRecord.readString(in); + RecordDeserializer recordDeserializer = null; + Class> deserializerClass = null; + try { + deserializerClass = (Class>) cl + .loadClass(deserializerClassName); + recordDeserializer = deserializerClass.newInstance(); + + } catch (ClassNotFoundException e) { + throw new IOException(StringUtils.stringifyException(e)); + } catch (InstantiationException e) { + throw new IOException(StringUtils.stringifyException(e)); + } catch (IllegalAccessException e) { + throw new IOException(StringUtils.stringifyException(e)); + } + + recordDeserializer.setClassLoader(cl); + recordDeserializer.read(in); + + final String distributionPatternClassName = StringRecord.readString(in); + DistributionPattern distributionPattern = null; + Class distributionPatternClass = null; + try { + distributionPatternClass = (Class) cl + .loadClass(distributionPatternClassName); + + distributionPattern = distributionPatternClass.newInstance(); + + } catch (ClassNotFoundException e) { + throw new IOException(StringUtils.stringifyException(e)); + } catch (InstantiationException e) { + throw new IOException(StringUtils.stringifyException(e)); + } catch (IllegalAccessException e) { + throw new IOException(StringUtils.stringifyException(e)); + } + + @SuppressWarnings("rawtypes") + final InputGate eig = new InputGate(this.jobID, gateID, recordDeserializer, i, + distributionPattern); + eig.read(in); + this.inputGates.add(eig); + // Mark as unbound for reconnection of RecordReader + this.unboundInputGates.add(eig); + } + + // The configuration object + this.runtimeConfiguration = new Configuration(); + this.runtimeConfiguration.read(in); + + // The current of number subtasks + this.currentNumberOfSubtasks = in.readInt(); + // The index in the subtask group + this.indexInSubtaskGroup = in.readInt(); + + // Finally, instantiate the invokable object + try { + instantiateInvokable(); + } catch (Exception e) { + throw new IOException(StringUtils.stringifyException(e)); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + + // Write out job vertex id + if (this.jobID == null) { + throw new IOException("this.jobID is null"); + } + + this.jobID.write(out); + + // Write the task name + StringRecord.writeString(out, this.taskName); + + // Write out the names of the required jar files + final String[] requiredJarFiles = LibraryCacheManager.getRequiredJarFiles(this.jobID); + + out.writeInt(requiredJarFiles.length); + for (int i = 0; i < requiredJarFiles.length; i++) { + StringRecord.writeString(out, requiredJarFiles[i]); + } + + // Write out the name of the invokable class + if (this.invokableClass == null) { + throw new IOException("this.invokableClass is null"); + } + + StringRecord.writeString(out, this.invokableClass.getName()); + + // Output gates + out.writeInt(getNumberOfOutputGates()); + for (int i = 0; i < getNumberOfOutputGates(); i++) { + final OutputGate outputGate = getOutputGate(i); + outputGate.getGateID().write(out); + StringRecord.writeString(out, outputGate.getType().getName()); + out.writeBoolean(outputGate.isBroadcast()); + if (!outputGate.isBroadcast()) { + // Write out class name of channel selector + StringRecord.writeString(out, outputGate.getChannelSelector().getClass().getName()); + outputGate.getChannelSelector().write(out); + } + + getOutputGate(i).write(out); + } + + // Input gates + out.writeInt(getNumberOfInputGates()); + for (int i = 0; i < getNumberOfInputGates(); i++) { + final InputGate inputGate = getInputGate(i); + inputGate.getGateID().write(out); + StringRecord.writeString(out, inputGate.getRecordDeserializer().getClass().getName()); + inputGate.getRecordDeserializer().write(out); + StringRecord.writeString(out, inputGate.getDistributionPattern().getClass().getName()); + getInputGate(i).write(out); + } + + // The configuration object + this.runtimeConfiguration.write(out); + + // The current of number subtasks + out.writeInt(this.currentNumberOfSubtasks); + // The index in the subtask group + out.writeInt(this.indexInSubtaskGroup); + } + + /** + * Blocks until all output channels are closed. + * + * @throws IOException + * thrown if an error occurred while closing the output channels + * @throws InterruptedException + * thrown if the thread waiting for the channels to be closed is interrupted + */ + private void waitForOutputChannelsToBeClosed() throws IOException, InterruptedException { + + // Wait for disconnection of all output gates + while (true) { + + // Make sure, we leave this method with an InterruptedException when the task has been canceled + if (this.executionObserver.isCanceled()) { + throw new InterruptedException(); + } + + boolean allClosed = true; + for (int i = 0; i < getNumberOfOutputGates(); i++) { + final OutputGate eog = getOutputGate(i); + if (!eog.isClosed()) { + allClosed = false; + } + } + + if (allClosed) { + break; + } else { + Thread.sleep(SLEEPINTERVAL); + } + } + } + + /** + * Blocks until all input channels are closed. + * + * @throws IOException + * thrown if an error occurred while closing the input channels + * @throws InterruptedException + * thrown if the thread waiting for the channels to be closed is interrupted + */ + private void waitForInputChannelsToBeClosed() throws IOException, InterruptedException { + + // Wait for disconnection of all output gates + while (true) { + + // Make sure, we leave this method with an InterruptedException when the task has been canceled + if (this.executionObserver.isCanceled()) { + throw new InterruptedException(); + } + + boolean allClosed = true; + for (int i = 0; i < getNumberOfInputGates(); i++) { + final InputGate eig = getInputGate(i); + if (!eig.isClosed()) { + allClosed = false; + } + } + + if (allClosed) { + break; + } else { + Thread.sleep(SLEEPINTERVAL); + } + } + } + + /** + * Closes all input gates which are not already closed. + */ + private void closeInputGates() throws IOException, InterruptedException { + + for (int i = 0; i < getNumberOfInputGates(); i++) { + final InputGate eig = getInputGate(i); + // Important: close must be called on each input gate exactly once + eig.close(); + } + + } + + /** + * Requests all output gates to be closed. + */ + private void requestAllOutputGatesToClose() throws IOException, InterruptedException { + + for (int i = 0; i < getNumberOfOutputGates(); i++) { + this.getOutputGate(i).requestClose(); + } + } + + /** + * Returns a duplicate (deep copy) of this environment object. However, duplication + * does not cover the gates arrays. They must be manually reconstructed. + * + * @return a duplicate (deep copy) of this environment object + * @throws Exception + * any exception that might be thrown by the user code during instantiation and registration of input and + * output channels + */ + public RuntimeEnvironment duplicateEnvironment() throws Exception { + + final RuntimeEnvironment duplicatedEnvironment = new RuntimeEnvironment(); + duplicatedEnvironment.invokableClass = this.invokableClass; + duplicatedEnvironment.jobID = this.jobID; + duplicatedEnvironment.taskName = this.taskName; + Thread tmpThread = null; + synchronized (this) { + tmpThread = this.executingThread; + } + synchronized (duplicatedEnvironment) { + duplicatedEnvironment.executingThread = tmpThread; + } + duplicatedEnvironment.runtimeConfiguration = this.runtimeConfiguration; + + // We instantiate the invokable of the new environment + duplicatedEnvironment.instantiateInvokable(); + + return duplicatedEnvironment; + } + + /** + * {@inheritDoc} + */ + @Override + public IOManager getIOManager() { + return this.ioManager; + } + + /** + * Sets the {@link IOManager}. + * + * @param memoryManager + * the new {@link IOManager} + */ + public void setIOManager(final IOManager ioManager) { + this.ioManager = ioManager; + } + + /** + * {@inheritDoc} + */ + @Override + public MemoryManager getMemoryManager() { + return this.memoryManager; + } + + /** + * Sets the {@link MemoryManager}. + * + * @param memoryManager + * the new {@link MemoryManager} + */ + public void setMemoryManager(final MemoryManager memoryManager) { + this.memoryManager = memoryManager; + } + + /** + * {@inheritDoc} + */ + @Override + public Configuration getRuntimeConfiguration() { + return this.runtimeConfiguration; + } + + /** + * {@inheritDoc} + */ + @Override + public int getCurrentNumberOfSubtasks() { + + return this.currentNumberOfSubtasks; + } + + /** + * Sets the current number of subtasks the respective task is split into. + * + * @param currentNumberOfSubtasks + * the current number of subtasks the respective task is split into + */ + public void setCurrentNumberOfSubtasks(final int currentNumberOfSubtasks) { + + this.currentNumberOfSubtasks = currentNumberOfSubtasks; + } + + /** + * {@inheritDoc} + */ + @Override + public int getIndexInSubtaskGroup() { + + return this.indexInSubtaskGroup; + } + + /** + * Sets the index of this subtask in the subtask group. + * + * @param indexInSubtaskGroup + * the index of this subtask in the subtask group + */ + public void setIndexInSubtaskGroup(final int indexInSubtaskGroup) { + + this.indexInSubtaskGroup = indexInSubtaskGroup; + } + + private void changeExecutionState(final ExecutionState newExecutionState, final String optionalMessage) { + + if (this.executionObserver != null) { + this.executionObserver.executionStateChanged(newExecutionState, optionalMessage); + } + } + + /** + * {@inheritDoc} + */ + @Override + public String getTaskName() { + + return this.taskName; + } + + /** + * Sets the execution observer for this environment. + * + * @param executionObserver + * the execution observer for this environment + */ + public void setExecutionObserver(final ExecutionObserver executionObserver) { + this.executionObserver = executionObserver; + } + + /** + * Sets the input split provider for this environment. + * + * @param inputSplitProvider + * the input split provider for this environment + */ + public void setInputSplitProvider(final InputSplitProvider inputSplitProvider) { + this.inputSplitProvider = inputSplitProvider; + } + + /** + * {@inheritDoc} + */ + @Override + public InputSplitProvider getInputSplitProvider() { + return this.inputSplitProvider; + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadStarted(final Thread userThread) { + + if (this.executionObserver != null) { + this.executionObserver.userThreadStarted(userThread); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadFinished(final Thread userThread) { + + if (this.executionObserver != null) { + this.executionObserver.userThreadFinished(userThread); + } + } + + /** + * Releases the allocated resources (particularly buffer) of input and output channels attached to this task. This + * method should only be called after the respected task has stopped running. + */ + private void releaseAllChannelResources() { + + for (int i = 0; i < getNumberOfInputGates(); i++) { + this.getInputGate(i).releaseAllChannelResources(); + } + + for (int i = 0; i < getNumberOfOutputGates(); i++) { + this.getOutputGate(i).releaseAllChannelResources(); + } + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java index d93bbbee73b31..2cb773fa3032a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java @@ -24,8 +24,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.instance.DummyInstance; import eu.stratosphere.nephele.instance.InstanceRequestMap; @@ -382,7 +382,7 @@ private void reconstructExecutionPipeline(final ExecutionVertex vertex, final bo vertex.setExecutionPipeline(pipeline); } - final Environment env = vertex.getEnvironment(); + final RuntimeEnvironment env = vertex.getEnvironment(); if (forward) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 06050eb5d0d38..5619b08216967 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -24,11 +24,11 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.util.StringUtils; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.execution.ExecutionStateTransition; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.instance.AllocatedResource; import eu.stratosphere.nephele.instance.AllocationID; import eu.stratosphere.nephele.io.InputGate; @@ -76,7 +76,7 @@ public final class ExecutionVertex { /** * The environment created to execute the vertex's task. */ - private final Environment environment; + private final RuntimeEnvironment environment; /** * The group vertex this vertex belongs to. @@ -148,7 +148,7 @@ public final class ExecutionVertex { */ public ExecutionVertex(final JobID jobID, final Class invokableClass, final ExecutionGraph executionGraph, final ExecutionGroupVertex groupVertex) throws Exception { - this(new ExecutionVertexID(), invokableClass, executionGraph, groupVertex, new Environment(jobID, + this(new ExecutionVertexID(), invokableClass, executionGraph, groupVertex, new RuntimeEnvironment(jobID, groupVertex.getName(), invokableClass, groupVertex.getConfiguration())); this.groupVertex.addInitialSubtask(this); @@ -177,7 +177,8 @@ public ExecutionVertex(final JobID jobID, final Class invokableClass, - final ExecutionGraph executionGraph, final ExecutionGroupVertex groupVertex, final Environment environment) { + final ExecutionGraph executionGraph, final ExecutionGroupVertex groupVertex, + final RuntimeEnvironment environment) { this.vertexID = vertexID; this.invokableClass = invokableClass; this.executionGraph = executionGraph; @@ -195,7 +196,7 @@ private ExecutionVertex(final ExecutionVertexID vertexID, final Class activeOutputChannels) throws IOException { + final Configuration jobConfiguration, final RuntimeEnvironment environment, + final Set activeOutputChannels) throws IOException { return getTaskManager().submitTask(id, jobConfiguration, environment, activeOutputChannels); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index 4821b8490ecb5..34568d8f89d93 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -26,8 +26,8 @@ import org.apache.hadoop.util.StringUtils; import eu.stratosphere.nephele.configuration.GlobalConfiguration; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; @@ -199,9 +199,9 @@ void findVerticesToBeDeployed(final ExecutionVertex vertex, if (instance instanceof DummyInstance) { LOG.error("Inconsistency: Vertex " + vertex.getName() + "(" - + vertex.getEnvironment().getIndexInSubtaskGroup() + "/" - + vertex.getEnvironment().getCurrentNumberOfSubtasks() - + ") is about to be deployed on a DummyInstance"); + + vertex.getEnvironment().getIndexInSubtaskGroup() + "/" + + vertex.getEnvironment().getCurrentNumberOfSubtasks() + + ") is about to be deployed on a DummyInstance"); } List verticesForInstance = verticesToBeDeployed.get(instance); @@ -214,7 +214,7 @@ void findVerticesToBeDeployed(final ExecutionVertex vertex, verticesForInstance.add(vertex); } - final Environment env = vertex.getEnvironment(); + final RuntimeEnvironment env = vertex.getEnvironment(); final int numberOfOutputGates = env.getNumberOfOutputGates(); for (int i = 0; i < numberOfOutputGates; ++i) { @@ -241,7 +241,7 @@ void findVerticesToBeDeployed(final ExecutionVertex vertex, for (int j = 0; j < numberOfOutputChannels; ++j) { final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); final ExecutionVertex connectedVertex = vertex.getExecutionGraph().getVertexByChannelID( - outputChannel.getConnectedChannelID()); + outputChannel.getConnectedChannelID()); findVerticesToBeDeployed(connectedVertex, verticesToBeDeployed); } } @@ -409,7 +409,7 @@ public void checkAndReleaseAllocatedResource(ExecutionGraph executionGraph, Allo } final List assignedVertices = executionGraph - .getVerticesAssignedToResource(allocatedResource); + .getVerticesAssignedToResource(allocatedResource); if (assignedVertices.isEmpty()) { return; } @@ -421,7 +421,7 @@ public void checkAndReleaseAllocatedResource(ExecutionGraph executionGraph, Allo final ExecutionState state = vertex.getExecutionState(); if (state != ExecutionState.CREATED && state != ExecutionState.FINISHED - && state != ExecutionState.FAILED && state != ExecutionState.CANCELED) { + && state != ExecutionState.FAILED && state != ExecutionState.CANCELED) { instanceCanBeReleased = false; break; @@ -432,7 +432,7 @@ public void checkAndReleaseAllocatedResource(ExecutionGraph executionGraph, Allo LOG.info("Releasing instance " + allocatedResource.getInstance()); try { getInstanceManager().releaseAllocatedResource(executionGraph.getJobID(), executionGraph - .getJobConfiguration(), allocatedResource); + .getJobConfiguration(), allocatedResource); } catch (InstanceException e) { LOG.error(StringUtils.stringifyException(e)); } @@ -465,7 +465,7 @@ protected void replayCheckpointsFromPreviousStage(final ExecutionGraph execution } final Iterator>> it = checkpointsToReplay.entrySet() - .iterator(); + .iterator(); while (it.hasNext()) { final Map.Entry> entry = it.next(); this.deploymentManager.replayCheckpoints(executionGraph.getJobID(), entry.getKey(), entry.getValue()); @@ -477,7 +477,7 @@ protected void replayCheckpointsFromPreviousStage(final ExecutionGraph execution */ @Override public void allocatedResourcesDied(final JobID jobID, final List allocatedResource) { - - //TODO: Don't forget to synchronize on stage here + + // TODO: Don't forget to synchronize on stage here } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java index ffb1a01f8d304..c9fd5f9422dcd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java @@ -16,7 +16,7 @@ package eu.stratosphere.nephele.plugins; import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; public interface TaskManagerPlugin { @@ -31,7 +31,7 @@ public interface TaskManagerPlugin { * @param environment * the environment of the task */ - void registerTask(ExecutionVertexID id, Configuration jobConfiguration, Environment environment); + void registerTask(ExecutionVertexID id, Configuration jobConfiguration, RuntimeEnvironment environment); /** * Unregisters a finished, canceled, or failed task from this task manager plugin. @@ -41,7 +41,7 @@ public interface TaskManagerPlugin { * @param environment * the environment of the task */ - void unregisterTask(ExecutionVertexID id, Environment environment); + void unregisterTask(ExecutionVertexID id, RuntimeEnvironment environment); /** * Called by the task manager to indicate that Nephele is about to shut down. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index 56008aef311a0..24846bddc2fb8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -22,7 +22,7 @@ import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate; @@ -49,15 +49,15 @@ public interface TaskOperationProtocol extends VersionedProtocol { * the ID of the corresponding execution vertex * @param jobConfiguration * the job configuration that has been attached to the original job graph - * @param ee - * the environment containing the task + * @param re + * the runtime environment containing the task * @param activeOutputChannels * the set of initially active output channels * @return the result of the task submission * @throws IOException * thrown if an error occurs during this remote procedure call */ - TaskSubmissionResult submitTask(ExecutionVertexID id, Configuration jobConfiguration, Environment ee, + TaskSubmissionResult submitTask(ExecutionVertexID id, Configuration jobConfiguration, RuntimeEnvironment re, Set activeOutputChannels) throws IOException; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index ef8e6391cfcc6..596b6ab7a8598 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -25,12 +25,12 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionObserver; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.execution.ExecutionStateTransition; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.OutputGate; @@ -50,7 +50,7 @@ public class Task implements ExecutionObserver { private final ExecutionVertexID vertexID; - private final Environment environment; + private final RuntimeEnvironment environment; private final TaskManager taskManager; @@ -66,7 +66,7 @@ public class Task implements ExecutionObserver { private Queue registeredListeners = new ConcurrentLinkedQueue(); - Task(final ExecutionVertexID vertexID, final Environment environment, final TaskManager taskManager) { + Task(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, final TaskManager taskManager) { this.vertexID = vertexID; this.environment = environment; @@ -291,7 +291,7 @@ public boolean isTerminated() { * * @return the environment associated with this task */ - public Environment getEnvironment() { + public RuntimeEnvironment getEnvironment() { return this.environment; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 1b0b3492d834f..ebadf1dd983df 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -46,9 +46,9 @@ import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.discovery.DiscoveryException; import eu.stratosphere.nephele.discovery.DiscoveryService; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse; @@ -445,19 +445,19 @@ public void run() { */ @Override public TaskSubmissionResult submitTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final Environment ee, final Set activeOutputChannels) + final RuntimeEnvironment re, final Set activeOutputChannels) throws IOException { // Register task manager components in environment - ee.setMemoryManager(this.memoryManager); - ee.setIOManager(this.ioManager); + re.setMemoryManager(this.memoryManager); + re.setIOManager(this.ioManager); // Register a new task input split provider - ee.setInputSplitProvider(new TaskInputSplitProvider(ee.getJobID(), id, this.globalInputSplitProvider)); + re.setInputSplitProvider(new TaskInputSplitProvider(re.getJobID(), id, this.globalInputSplitProvider)); // Create task object and register it with the environment - final Task task = new Task(id, ee, this); - ee.setExecutionObserver(task); + final Task task = new Task(id, re, this); + re.setExecutionObserver(task); // Register the task TaskSubmissionResult result = registerTask(id, jobConfiguration, task, activeOutputChannels); @@ -486,21 +486,21 @@ public List submitTasks(final List // Make sure all tasks are fully registered before they are started for (final TaskSubmissionWrapper tsw : tasks) { - final Environment ee = tsw.getEnvironment(); + final RuntimeEnvironment re = tsw.getEnvironment(); final ExecutionVertexID id = tsw.getVertexID(); final Configuration jobConfiguration = tsw.getConfiguration(); final Set activeOutputChannels = tsw.getActiveOutputChannels(); // Register task manager components in environment - ee.setMemoryManager(this.memoryManager); - ee.setIOManager(this.ioManager); + re.setMemoryManager(this.memoryManager); + re.setIOManager(this.ioManager); // Register a new task input split provider - ee.setInputSplitProvider(new TaskInputSplitProvider(ee.getJobID(), id, this.globalInputSplitProvider)); + re.setInputSplitProvider(new TaskInputSplitProvider(re.getJobID(), id, this.globalInputSplitProvider)); // Create task object and register it with the environment - final Task task = new Task(id, ee, this); - ee.setExecutionObserver(task); + final Task task = new Task(id, re, this); + re.setExecutionObserver(task); // Register the task TaskSubmissionResult result = registerTask(id, jobConfiguration, task, activeOutputChannels); @@ -556,7 +556,7 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf } } - final Environment ee = task.getEnvironment(); + final RuntimeEnvironment ee = task.getEnvironment(); // Check if the task has unbound input/output gates if (ee.hasUnboundInputGates() || ee.hasUnboundOutputGates()) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskSubmissionWrapper.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskSubmissionWrapper.java index cbc42b60662ce..bc15442f87dc1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskSubmissionWrapper.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskSubmissionWrapper.java @@ -21,7 +21,7 @@ import java.util.Set; import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.io.channels.ChannelID; @@ -43,7 +43,7 @@ public final class TaskSubmissionWrapper implements IOReadableWritable { /** * The task's execution environment. */ - Environment environment = null; + RuntimeEnvironment environment = null; /** * The task's configuration object. @@ -65,7 +65,7 @@ public final class TaskSubmissionWrapper implements IOReadableWritable { * @param configuration * the task's configuration */ - public TaskSubmissionWrapper(final ExecutionVertexID vertexID, final Environment environment, + public TaskSubmissionWrapper(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, final Configuration configuration, final SerializableHashSet activeOutputChannels) { if (vertexID == null) { @@ -116,7 +116,7 @@ public void read(final DataInput in) throws IOException { this.vertexID = new ExecutionVertexID(); this.vertexID.read(in); - this.environment = new Environment(); + this.environment = new RuntimeEnvironment(); this.environment.read(in); this.configuration = new Configuration(); this.configuration.read(in); @@ -139,7 +139,7 @@ public ExecutionVertexID getVertexID() { * * @return the task's execution environment */ - public Environment getEnvironment() { + public RuntimeEnvironment getEnvironment() { return this.environment; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index d20d76470eb03..398282c73474f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -28,7 +28,7 @@ import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.configuration.GlobalConfiguration; -import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; import eu.stratosphere.nephele.io.AbstractID; @@ -95,7 +95,7 @@ public final class ByteBufferedChannelManager implements TransferEnvelopeDispatc public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupService, final InstanceConnectionInfo localInstanceConnectionInfo) - throws IOException { + throws IOException { this.channelLookupService = channelLookupService; @@ -134,7 +134,7 @@ public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupServi */ public void register(final Task task, final Set activeOutputChannels) { - final Environment environment = task.getEnvironment(); + final RuntimeEnvironment environment = task.getEnvironment(); final TaskContext taskContext = new TaskContext(task, this, this.tasksWithUndecidedCheckpoints); @@ -146,7 +146,7 @@ public void register(final Task task, final Set activeOutputChannels) final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); if (!(outputChannel instanceof AbstractByteBufferedOutputChannel)) { LOG.error("Output channel " + outputChannel.getID() + "of job " + environment.getJobID() - + " is not a byte buffered output channel, skipping..."); + + " is not a byte buffered output channel, skipping..."); continue; } @@ -165,10 +165,10 @@ public void register(final Task task, final Set activeOutputChannels) final boolean isActive = activeOutputChannels.contains(bboc.getID()); LOG.info("Registering byte buffered output channel " + bboc.getID() + " (" - + (isActive ? "active" : "inactive") + ")"); + + (isActive ? "active" : "inactive") + ")"); final OutputChannelContext outputChannelContext = new OutputChannelContext(outputGateContext, bboc, - isActive, this.mergeSpilledBuffers); + isActive, this.mergeSpilledBuffers); this.registeredChannels.put(bboc.getID(), outputChannelContext); } } @@ -180,7 +180,7 @@ public void register(final Task task, final Set activeOutputChannels) final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); if (!(inputChannel instanceof AbstractByteBufferedInputChannel)) { LOG.error("Input channel " + inputChannel.getID() + "of job " + environment.getJobID() - + " is not a byte buffered input channel, skipping..."); + + " is not a byte buffered input channel, skipping..."); continue; } @@ -199,7 +199,7 @@ public void register(final Task task, final Set activeOutputChannels) LOG.info("Registering byte buffered input channel " + bbic.getID()); final InputChannelContext inputChannelContext = new InputChannelContext(inputGateContext, this, - bbic); + bbic); this.registeredChannels.put(bbic.getID(), inputChannelContext); } @@ -222,7 +222,7 @@ public void register(final Task task, final Set activeOutputChannels) */ public void unregister(final ExecutionVertexID vertexID, final Task task) { - final Environment environment = task.getEnvironment(); + final RuntimeEnvironment environment = task.getEnvironment(); for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { final OutputGate outputGate = environment.getOutputGate(i); @@ -339,7 +339,7 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, if (!cc.isInputChannel()) { throw new IOException("Local receiver " + localReceiver - + " is not an input channel, but is supposed to accept a buffer"); + + " is not an input channel, but is supposed to accept a buffer"); } cc.queueTransferEnvelope(transferEnvelope); @@ -363,7 +363,7 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, if (!cc.isInputChannel()) { throw new IOException("Local receiver " + localReceiver - + " is not an input channel, but is supposed to accept a buffer"); + + " is not an input channel, but is supposed to accept a buffer"); } final InputChannelContext inputChannelContext = (InputChannelContext) cc; @@ -439,11 +439,11 @@ private TransferEnvelopeReceiverList getReceiverList(final JobID jobID, final Ch while (true) { final ConnectionInfoLookupResponse lookupResponse = this.channelLookupService.lookupConnectionInfo( - this.localConnectionInfo, jobID, sourceChannelID); + this.localConnectionInfo, jobID, sourceChannelID); if (lookupResponse.receiverNotFound()) { throw new IOException("Cannot find task(s) waiting for data from source channel with ID " - + sourceChannelID); + + sourceChannelID); } if (lookupResponse.receiverNotReady()) { @@ -548,7 +548,7 @@ public void logBufferUtilization() { System.out.println("\tIncoming connections:"); final Iterator> it2 = this.registeredChannels.entrySet() - .iterator(); + .iterator(); while (it2.hasNext()) { @@ -561,7 +561,7 @@ public void logBufferUtilization() { final int numberOfQueuedMemoryBuffers = inputChannelContext.getNumberOfQueuedMemoryBuffers(); System.out.println("\t\t" + entry.getKey() + ": " + numberOfQueuedMemoryBuffers + " (" - + numberOfQueuedEnvelopes + ")"); + + numberOfQueuedEnvelopes + ")"); } } } @@ -586,7 +586,7 @@ public BufferProvider getBufferProvider(final JobID jobID, final ChannelID sourc if (!cc.isInputChannel()) { throw new IOException("Channel context for local receiver " + localReceiver - + " is not an input channel context"); + + " is not an input channel context"); } final InputChannelContext icc = (InputChannelContext) cc; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java index 3f1fe51e762e8..e94acb19d3d09 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java @@ -19,7 +19,7 @@ import java.util.Map; import eu.stratosphere.nephele.checkpointing.EphemeralCheckpoint; -import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.AbstractID; import eu.stratosphere.nephele.io.OutputGate; @@ -63,7 +63,7 @@ final class TaskContext implements BufferProvider, LocalBufferPoolOwner, Asynchr this.localBufferPool = new LocalBufferPool(1, false, this); this.task = task; - final Environment environment = task.getEnvironment(); + final RuntimeEnvironment environment = task.getEnvironment(); // Compute number of output input channels int nooc = 0; @@ -163,7 +163,7 @@ public void logBufferUtilization() { final int req = this.localBufferPool.getRequestedNumberOfBuffers(); final int des = this.localBufferPool.getDesignatedNumberOfBuffers(); - final Environment environment = this.task.getEnvironment(); + final RuntimeEnvironment environment = this.task.getEnvironment(); System.out.println("\t\t" + environment.getTaskName() + ": " + ava + " available, " + req + " requested, " + des + " designated"); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index f602adede306f..00471c168fd81 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -16,6 +16,7 @@ package eu.stratosphere.nephele.streaming; import java.io.IOException; +import java.util.Iterator; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -23,8 +24,14 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.jobgraph.AbstractJobInputVertex; +import eu.stratosphere.nephele.jobgraph.AbstractJobVertex; import eu.stratosphere.nephele.jobgraph.JobGraph; +import eu.stratosphere.nephele.jobgraph.JobTaskVertex; import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.streaming.wrapper.StreamingTask; +import eu.stratosphere.nephele.template.AbstractInvokable; +import eu.stratosphere.nephele.template.AbstractTask; public class StreamingJobManagerPlugin implements JobManagerPlugin { @@ -41,6 +48,21 @@ public class StreamingJobManagerPlugin implements JobManagerPlugin { */ @Override public JobGraph rewriteJobGraph(final JobGraph jobGraph) { + + // Rewrite input vertices + final Iterator inputIt = jobGraph.getInputVertices(); + + final Iterator taskIt = jobGraph.getTaskVertices(); + while (taskIt.hasNext()) { + + final JobTaskVertex taskVertex = taskIt.next(); + + final Class originalClass = taskVertex.getInvokableClass(); + + taskVertex.setTaskClass(StreamingTask.class); + taskVertex.getConfiguration().setString("origClass", originalClass.getName()); + } + // TODO Auto-generated method stub return null; } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index dc45d4b9fe9c6..2d0d2e3470299 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -16,7 +16,7 @@ package eu.stratosphere.nephele.streaming; import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; @@ -87,7 +87,7 @@ public void shutdown() { */ @Override public void registerTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final Environment environment) { + final RuntimeEnvironment environment) { // Check if user has provided a job-specific aggregation interval final int aggregationInterval = jobConfiguration.getInteger(AGGREGATION_INTERVAL_KEY, @@ -125,7 +125,7 @@ public void registerTask(final ExecutionVertexID id, final Configuration jobConf * {@inheritDoc} */ @Override - public void unregisterTask(final ExecutionVertexID id, final Environment environment) { + public void unregisterTask(final ExecutionVertexID id, final RuntimeEnvironment environment) { // Nothing to do here } diff --git a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java index ce9e6ced20347..da0d77d4f8703 100644 --- a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java +++ b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java @@ -20,7 +20,7 @@ import java.util.List; import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.io.DefaultRecordDeserializer; import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.InputGate; @@ -34,7 +34,7 @@ import eu.stratosphere.pact.common.util.MutableObjectIterator; -public class MockEnvironment extends Environment +public class MockEnvironment extends RuntimeEnvironment { private MemoryManager memManager; diff --git a/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java b/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java index 82feb9d9bd2aa..1f25ec331e96e 100644 --- a/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java +++ b/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java @@ -16,7 +16,7 @@ package eu.stratosphere.score; import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; @@ -38,7 +38,7 @@ public void shutdown() { */ @Override public void registerTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final Environment environment) { + final RuntimeEnvironment environment) { // TODO Auto-generated method stub } @@ -47,7 +47,7 @@ public void registerTask(final ExecutionVertexID id, final Configuration jobConf * {@inheritDoc} */ @Override - public void unregisterTask(final ExecutionVertexID id, final Environment environment) { + public void unregisterTask(final ExecutionVertexID id, final RuntimeEnvironment environment) { // TODO Auto-generated method stub } From 927790dce6e33c65ed2785c2d58ca1f2a6a538bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Sat, 19 Nov 2011 17:46:03 +0100 Subject: [PATCH 056/310] added .project that define a java-nature for the projects --- nephele/nephele-clustermanager/.project | 23 +++++++++++++++++++++ nephele/nephele-common/.project | 23 +++++++++++++++++++++ nephele/nephele-compression-bzip2/.project | 23 +++++++++++++++++++++ nephele/nephele-compression-lzma/.project | 23 +++++++++++++++++++++ nephele/nephele-compression-snappy/.project | 23 +++++++++++++++++++++ nephele/nephele-compression-zlib/.project | 23 +++++++++++++++++++++ nephele/nephele-ec2cloudmanager/.project | 23 +++++++++++++++++++++ nephele/nephele-examples/.project | 23 +++++++++++++++++++++ nephele/nephele-hdfs/.project | 23 +++++++++++++++++++++ nephele/nephele-management/.project | 23 +++++++++++++++++++++ nephele/nephele-profiling/.project | 23 +++++++++++++++++++++ nephele/nephele-queuescheduler/.project | 23 +++++++++++++++++++++ nephele/nephele-s3/.project | 23 +++++++++++++++++++++ nephele/nephele-server/.project | 23 +++++++++++++++++++++ nephele/nephele-visualization/.project | 23 +++++++++++++++++++++ 15 files changed, 345 insertions(+) create mode 100644 nephele/nephele-clustermanager/.project create mode 100644 nephele/nephele-common/.project create mode 100644 nephele/nephele-compression-bzip2/.project create mode 100644 nephele/nephele-compression-lzma/.project create mode 100644 nephele/nephele-compression-snappy/.project create mode 100644 nephele/nephele-compression-zlib/.project create mode 100644 nephele/nephele-ec2cloudmanager/.project create mode 100644 nephele/nephele-examples/.project create mode 100644 nephele/nephele-hdfs/.project create mode 100644 nephele/nephele-management/.project create mode 100644 nephele/nephele-profiling/.project create mode 100644 nephele/nephele-queuescheduler/.project create mode 100644 nephele/nephele-s3/.project create mode 100644 nephele/nephele-server/.project create mode 100644 nephele/nephele-visualization/.project diff --git a/nephele/nephele-clustermanager/.project b/nephele/nephele-clustermanager/.project new file mode 100644 index 0000000000000..c0c14e44c9176 --- /dev/null +++ b/nephele/nephele-clustermanager/.project @@ -0,0 +1,23 @@ + + + nephele-clustermanager + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-common/.project b/nephele/nephele-common/.project new file mode 100644 index 0000000000000..cf5422b186565 --- /dev/null +++ b/nephele/nephele-common/.project @@ -0,0 +1,23 @@ + + + nephele-common + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-compression-bzip2/.project b/nephele/nephele-compression-bzip2/.project new file mode 100644 index 0000000000000..7fbfff9d6c2d0 --- /dev/null +++ b/nephele/nephele-compression-bzip2/.project @@ -0,0 +1,23 @@ + + + nephele-compression-bzip2 + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-compression-lzma/.project b/nephele/nephele-compression-lzma/.project new file mode 100644 index 0000000000000..abe9bccf558b1 --- /dev/null +++ b/nephele/nephele-compression-lzma/.project @@ -0,0 +1,23 @@ + + + nephele-compression-lzma + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-compression-snappy/.project b/nephele/nephele-compression-snappy/.project new file mode 100644 index 0000000000000..1415e8cb355ea --- /dev/null +++ b/nephele/nephele-compression-snappy/.project @@ -0,0 +1,23 @@ + + + nephele-compression-snappy + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-compression-zlib/.project b/nephele/nephele-compression-zlib/.project new file mode 100644 index 0000000000000..2a7c2b84ae02d --- /dev/null +++ b/nephele/nephele-compression-zlib/.project @@ -0,0 +1,23 @@ + + + nephele-compression-zlib + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-ec2cloudmanager/.project b/nephele/nephele-ec2cloudmanager/.project new file mode 100644 index 0000000000000..8527f72429d74 --- /dev/null +++ b/nephele/nephele-ec2cloudmanager/.project @@ -0,0 +1,23 @@ + + + nephele-ec2cloudmanager + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-examples/.project b/nephele/nephele-examples/.project new file mode 100644 index 0000000000000..ea9c3934906dd --- /dev/null +++ b/nephele/nephele-examples/.project @@ -0,0 +1,23 @@ + + + nephele-examples + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-hdfs/.project b/nephele/nephele-hdfs/.project new file mode 100644 index 0000000000000..9a40a6e4a022a --- /dev/null +++ b/nephele/nephele-hdfs/.project @@ -0,0 +1,23 @@ + + + nephele-hdfs + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-management/.project b/nephele/nephele-management/.project new file mode 100644 index 0000000000000..a9d85cc576bbd --- /dev/null +++ b/nephele/nephele-management/.project @@ -0,0 +1,23 @@ + + + nephele-management + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-profiling/.project b/nephele/nephele-profiling/.project new file mode 100644 index 0000000000000..712a2fed53513 --- /dev/null +++ b/nephele/nephele-profiling/.project @@ -0,0 +1,23 @@ + + + nephele-profiling + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-queuescheduler/.project b/nephele/nephele-queuescheduler/.project new file mode 100644 index 0000000000000..954d5faa8bae4 --- /dev/null +++ b/nephele/nephele-queuescheduler/.project @@ -0,0 +1,23 @@ + + + nephele-queuescheduler + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-s3/.project b/nephele/nephele-s3/.project new file mode 100644 index 0000000000000..1f4962858eb7f --- /dev/null +++ b/nephele/nephele-s3/.project @@ -0,0 +1,23 @@ + + + nephele-s3 + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-server/.project b/nephele/nephele-server/.project new file mode 100644 index 0000000000000..3d87b6b42ec49 --- /dev/null +++ b/nephele/nephele-server/.project @@ -0,0 +1,23 @@ + + + nephele-server + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/nephele/nephele-visualization/.project b/nephele/nephele-visualization/.project new file mode 100644 index 0000000000000..c810a0d04b779 --- /dev/null +++ b/nephele/nephele-visualization/.project @@ -0,0 +1,23 @@ + + + nephele-visualization + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + From 22f54d332e2071ea62ebf2a1e151d627c751b05c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 19 Nov 2011 18:01:13 +0000 Subject: [PATCH 057/310] Refactored input/output gates --- .../nephele/execution/Environment.java | 24 +- .../nephele/fs/file/LocalDataInputStream.java | 21 +- .../fs/file/LocalDataOutputStream.java | 8 - .../nephele/io/AbstractRecordReader.java | 18 +- .../nephele/io/AbstractRecordWriter.java | 18 +- .../java/eu/stratosphere/nephele/io/Gate.java | 81 +++ .../eu/stratosphere/nephele/io/InputGate.java | 542 +--------------- .../stratosphere/nephele/io/OutputGate.java | 534 +--------------- .../io/channels/AbstractOutputChannel.java | 2 +- .../impl/TaskManagerProfilerImpl.java | 8 +- .../nephele/execution/RuntimeEnvironment.java | 78 ++- .../executiongraph/ExecutionGraph.java | 9 +- .../executiongraph/ExecutionStage.java | 4 +- .../executiongraph/ExecutionVertex.java | 6 +- .../ManagementGraphFactory.java | 3 +- .../stratosphere/nephele/io/AbstractGate.java | 41 +- .../nephele/io/RuntimeInputGate.java | 580 +++++++++++++++++ .../nephele/io/RuntimeOutputGate.java | 589 ++++++++++++++++++ .../profiling/TaskManagerProfiler.java | 8 +- .../nephele/taskmanager/Task.java | 4 +- 20 files changed, 1378 insertions(+), 1200 deletions(-) create mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/Gate.java rename nephele/{nephele-common => nephele-server}/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java (83%) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java index 1dd5649649103..eaf777b7275d5 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java @@ -16,8 +16,11 @@ package eu.stratosphere.nephele.execution; import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.io.ChannelSelector; +import eu.stratosphere.nephele.io.DistributionPattern; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RecordDeserializer; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; @@ -154,18 +157,23 @@ public interface Environment { int getNumberOfInputGates(); /** - * Registers an output gate with the environment. + * Creates and registers an output gate with the environment. * - * @param outputGate - * the output gate to be registered with the environment + * @param outputClass + * @param selector + * @param isBroadcast + * @return the created output gate */ - void registerOutputGate(final OutputGate outputGate); + OutputGate createAndRegisterOutputGate(Class outputClass, + ChannelSelector selector, boolean isBroadcast); /** - * Registers an input gate with the environment. + * Creates and registers an input gate with the environment. * - * @param inputGate - * the input gate to be registered with the environment + * @param deserializer + * @param distributionPattern + * @return the created input gate */ - void registerInputGate(final InputGate inputGate); + InputGate createAndRegisterInputGate(RecordDeserializer deserializer, + DistributionPattern distributionPattern); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalDataInputStream.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalDataInputStream.java index caf152e5a5139..b5bd757d1ef40 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalDataInputStream.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalDataInputStream.java @@ -34,11 +34,6 @@ public class LocalDataInputStream extends FSDataInputStream { */ private FileInputStream fis = null; - /** - * The current position in the stream. - */ - private long position = 0; - /** * Constructs a new LocalDataInputStream object from a given {@link File} object. * @@ -50,7 +45,6 @@ public class LocalDataInputStream extends FSDataInputStream { public LocalDataInputStream(final File file) throws IOException { this.fis = new FileInputStream(file); - this.position = 0; } /** @@ -60,7 +54,6 @@ public LocalDataInputStream(final File file) throws IOException { public void seek(final long desired) throws IOException { this.fis.getChannel().position(desired); - this.position = desired; } /** @@ -69,12 +62,7 @@ public void seek(final long desired) throws IOException { @Override public int read() throws IOException { - final int value = this.fis.read(); - if (value >= 0) { - this.position++; - } - - return value; + return this.fis.read(); } /** @@ -83,12 +71,7 @@ public int read() throws IOException { @Override public int read(final byte[] buffer, final int offset, final int length) throws IOException { - final int value = this.fis.read(buffer, offset, length); - if (value > 0) { - this.position += value; - } - - return value; + return this.fis.read(buffer, offset, length); } /** diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalDataOutputStream.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalDataOutputStream.java index ed9eec460f51e..539f3f9f8767c 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalDataOutputStream.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalDataOutputStream.java @@ -34,11 +34,6 @@ public class LocalDataOutputStream extends FSDataOutputStream { */ private FileOutputStream fos = null; - /** - * The current position in the output stream. - */ - private long position; - /** * Constructs a new LocalDataOutputStream object from a given {@link File} object. * @@ -50,7 +45,6 @@ public class LocalDataOutputStream extends FSDataOutputStream { public LocalDataOutputStream(final File file) throws IOException { this.fos = new FileOutputStream(file); - this.position = 0; } /** @@ -59,7 +53,6 @@ public LocalDataOutputStream(final File file) throws IOException { @Override public void write(final int b) throws IOException { fos.write(b); - position++; } /** @@ -68,7 +61,6 @@ public void write(final int b) throws IOException { @Override public void write(final byte[] b, final int off, final int len) throws IOException { fos.write(b, off, len); - position += len; // update position } /** diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java index 43be1aef3c0b4..efcc2bb793295 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java @@ -66,10 +66,8 @@ private void connectInputGate(final RecordDeserializer deserializer, final in if (this.environment.hasUnboundInputGates()) { this.inputGate = (InputGate) this.environment.getUnboundInputGate(inputGateID); } else { - this.inputGate = new InputGate(environment.getJobID(), new GateID(), deserializer, - this.environment.getNumberOfInputGates(), + this.inputGate = (InputGate) this.environment.createAndRegisterInputGate(deserializer, distributionPattern); - this.environment.registerInputGate(this.inputGate); } } @@ -102,17 +100,6 @@ public final boolean isInputChannelClosed(final int index) throws IOException, I return false; } - /** - * Registers a new listener object with the assigned input gate. - * - * @param inputGateListener - * the listener object to register - */ - public final void registerInputGateListener(final InputGateListener inputGateListener) { - - this.inputGate.registerInputGateListener(inputGateListener); - } - /** * Subscribes the listener object to receive events of the given type. * @@ -121,7 +108,8 @@ public final void registerInputGateListener(final InputGateListener inputGateLis * @param eventType * the type of event to register the listener for */ - public final void subscribeToEvent(final EventListener eventListener, final Class eventType) { + public final void subscribeToEvent(final EventListener eventListener, + final Class eventType) { // Delegate call to input gate this.inputGate.subscribeToEvent(eventListener, eventType); diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java index f3e4299ed7fad..df81b4fdb6527 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java @@ -88,9 +88,8 @@ private void connectOutputGate(Class outputClass, ChannelSelector selector this.outputGate = eog; } else { - this.outputGate = new OutputGate(environment.getJobID(), new GateID(), outputClass, - this.environment.getNumberOfOutputGates(), selector, isBroadcast); - this.environment.registerOutputGate(this.outputGate); + this.outputGate = (OutputGate) this.environment.createAndRegisterOutputGate(outputClass, selector, + isBroadcast); } } @@ -103,7 +102,7 @@ private void connectOutputGate(Class outputClass, ChannelSelector selector * @throws IOException * Thrown on an error that may happen during the transfer of the given record or a previous record. */ - public void emit(T record) throws IOException, InterruptedException { + public void emit(final T record) throws IOException, InterruptedException { // Simply pass record through to the corresponding output gate this.outputGate.writeRecord(record); @@ -118,17 +117,6 @@ public List> getOutputChannels() { return this.outputGate.getOutputChannels(); } - /** - * Registers a new listener object with the assigned output gate. - * - * @param inputGateListener - * the listener object to register - */ - public void registerOutputGateListener(OutputGateListener outputGateListener) { - - this.outputGate.registerOutputGateListener(outputGateListener); - } - // TODO (en) public OutputGate getOutputGate() { return outputGate; diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/Gate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/Gate.java new file mode 100644 index 0000000000000..a3cb5422e34a6 --- /dev/null +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/Gate.java @@ -0,0 +1,81 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.io; + +import java.io.IOException; + +import eu.stratosphere.nephele.event.task.AbstractTaskEvent; +import eu.stratosphere.nephele.event.task.EventListener; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.types.Record; + +public interface Gate { + + /** + * Subscribes the listener object to receive events of the given type. + * + * @param eventListener + * the listener object to register + * @param eventType + * the type of event to register the listener for + */ + void subscribeToEvent(EventListener eventListener, Class eventType); + + /** + * Removes the subscription for events of the given type for the listener object. + * + * @param eventListener + * the listener object to cancel the subscription for + * @param eventType + * the type of the event to cancel the subscription for + */ + void unsubscribeFromEvent(EventListener eventListener, Class eventType); + + /** + * Publishes an event. + * + * @param event + * the event to be published + * @throws IOException + * thrown if an error occurs while transmitting the event + * @throws InterruptedException + * thrown if the thread is interrupted while waiting for the event to be published + */ + void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException; + + /** + * Passes a received event on to the event notification manager so it cam ne dispatched. + * + * @param event + * the event to pass on to the notification manager + */ + void deliverEvent(AbstractTaskEvent event); + + /** + * Returns the ID of the job this gate belongs to. + * + * @return the ID of the job this gate belongs to + */ + JobID getJobID(); + + /** + * Returns the type of the input/output channels which are connected to this gate. + * + * @return the type of input/output channels which are connected to this gate + */ + ChannelType getChannelType(); +} diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java index 6c15d45e41e87..7bc36ac86a564 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java @@ -15,34 +15,10 @@ package eu.stratosphere.nephele.io; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.EOFException; import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.event.task.AbstractTaskEvent; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; -import eu.stratosphere.nephele.io.channels.bytebuffered.FileInputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkInputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryInputChannel; -import eu.stratosphere.nephele.io.compression.CompressionLevel; -import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.types.StringRecord; -import eu.stratosphere.nephele.util.ClassUtils; -import eu.stratosphere.nephele.util.EnumUtils; /** * In Nephele input gates are a specialization of general gates and connect input channels and record readers. As @@ -51,191 +27,32 @@ * input gates * can be associated with a {@link DistributionPattern} object which dictates the concrete wiring between two groups of * vertices. - *

- * This class is in general not thread-safe. * * @author warneke * @param * the type of record that can be transported through this gate */ -public class InputGate extends AbstractGate implements IOReadableWritable { - - /** - * The log object used for debugging. - */ - private static final Log LOG = LogFactory.getLog(InputGate.class); - - /** - * The deserializer used to construct records from byte streams. - */ - private final RecordDeserializer deserializer; - - /** - * The list of input channels attached to this input gate. - */ - private final ArrayList> inputChannels = new ArrayList>(); - - /** - * The distribution pattern to determine how to wire the channels. - */ - private final DistributionPattern distributionPattern; - - /** - * Queue with indices of channels that store at least one available record. - */ - private final ArrayDeque availableChannels = new ArrayDeque(); - - /** - * The listener objects registered for this input gate. - */ - private InputGateListener[] inputGateListeners = null; - - /** - * The channel to read from next. - */ - private int channelToReadFrom = -1; - - /** - * The thread which executes the task connected to the input gate. - */ - private Thread executingThread = null; - - /** - * Constructs a new input gate. - * - * @param jobID - * the ID of the job this input gate belongs to - * @param gateID - * the ID of the gate - * @param inputClass - * the class of the record that can be transported through this gate - * @param index - * the index assigned to this input gate at the {@link Environment} object - * @param distributionPattern - * the distribution pattern to determine the concrete wiring between to groups of vertices - */ - public InputGate(final JobID jobID, final GateID gateID, final RecordDeserializer deserializer, final int index, - final DistributionPattern distributionPattern) { - - super(jobID, gateID, index); - - this.deserializer = deserializer; - - this.distributionPattern = (distributionPattern != null) ? distributionPattern - : new BipartiteDistributionPattern(); - } - - /** - * Adds a new input channel to the input gate. - * - * @param inputChannel - * the input channel to be added. - */ - private void addInputChannel(AbstractInputChannel inputChannel) { - - if (!this.inputChannels.contains(inputChannel)) { - this.inputChannels.add(inputChannel); - } - } - - /** - * Removes the input channel with the given ID from the input gate if it exists. - * - * @param inputChannelID - * the ID of the channel to be removed - */ - public void removeInputChannel(ChannelID inputChannelID) { - - for (int i = 0; i < this.inputChannels.size(); i++) { - - final AbstractInputChannel inputChannel = this.inputChannels.get(i); - if (inputChannel.getID().equals(inputChannelID)) { - this.inputChannels.remove(i); - return; - } - } - - LOG.debug("Cannot find output channel with ID " + inputChannelID + " to remove"); - } - - /** - * Removes all input channels from the input gate. - */ - public void removeAllInputChannels() { - - this.inputChannels.clear(); - } - - public AbstractInputChannel replaceChannel(ChannelID oldChannelID, ChannelType newChannelType) { - - AbstractInputChannel oldInputChannel = null; - - for (int i = 0; i < this.inputChannels.size(); i++) { - final AbstractInputChannel inputChannel = this.inputChannels.get(i); - if (inputChannel.getID().equals(oldChannelID)) { - oldInputChannel = inputChannel; - break; - } - } - - if (oldInputChannel == null) { - return null; - } - - AbstractInputChannel newInputChannel = null; - - switch (newChannelType) { - case FILE: - newInputChannel = new FileInputChannel(this, oldInputChannel.getChannelIndex(), deserializer, - oldInputChannel.getID(), oldInputChannel.getCompressionLevel()); - break; - case INMEMORY: - newInputChannel = new InMemoryInputChannel(this, oldInputChannel.getChannelIndex(), deserializer, - oldInputChannel.getID(), oldInputChannel.getCompressionLevel()); - break; - case NETWORK: - newInputChannel = new NetworkInputChannel(this, oldInputChannel.getChannelIndex(), deserializer, - oldInputChannel.getID(), oldInputChannel.getCompressionLevel()); - break; - default: - LOG.error("Unknown input channel type"); - return null; - } - - newInputChannel.setConnectedChannelID(oldInputChannel.getConnectedChannelID()); - - this.inputChannels.set(newInputChannel.getChannelIndex(), newInputChannel); - - return newInputChannel; - } +public interface InputGate extends Gate { /** - * Returns the {@link DistributionPattern} associated with this input gate. + * Reads a record from one of the associated input channels. The channels are + * chosen in a way so that channels with available records are preferred. + * The operation may block until at least one of the associated input channel + * is able to provide a record. * - * @return the {@link DistributionPattern} associated with this input gate - */ - public DistributionPattern getDistributionPattern() { - return this.distributionPattern; - } - - /** - * {@inheritDoc} + * @return the record read from one of the input channels or null if all channels are already closed. + * @throws ExecutionFailureException + * thrown if an error occurred while reading the channels */ - @Override - public boolean isInputGate() { - return true; - } + T readRecord(T target) throws IOException, InterruptedException; /** * Returns the number of input channels associated with this input gate. * * @return the number of input channels associated with this input gate */ - public int getNumberOfInputChannels() { - - return this.inputChannels.size(); - } + int getNumberOfInputChannels(); /** * Returns the input channel from position pos of the gate's internal channel list. @@ -244,346 +61,11 @@ public int getNumberOfInputChannels() { * the position to retrieve the channel from * @return the channel from the given position or null if such position does not exist. */ - public AbstractInputChannel getInputChannel(int pos) { - - if (pos < this.inputChannels.size()) { - return this.inputChannels.get(pos); - } - - return null; - } - - /** - * Creates a new network input channel and assigns it to the input gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new network input channel - */ - public NetworkInputChannel createNetworkInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - - final NetworkInputChannel enic = new NetworkInputChannel(this, this.inputChannels.size(), deserializer, - channelID, compressionLevel); - addInputChannel(enic); - - return enic; - } - - /** - * Creates a new file input channel and assigns it to the input gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new file input channel - */ - public FileInputChannel createFileInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - - final FileInputChannel efic = new FileInputChannel(this, this.inputChannels.size(), deserializer, - channelID, compressionLevel); - addInputChannel(efic); - - return efic; - } - - /** - * Creates a new in-memory input channel and assigns it to the input gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new in-memory input channel - */ - public InMemoryInputChannel createInMemoryInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - - final InMemoryInputChannel eimic = new InMemoryInputChannel(this, this.inputChannels.size(), - deserializer, channelID, compressionLevel); - addInputChannel(eimic); - - return eimic; - } - - /** - * Reads a record from one of the associated input channels. The channels are - * chosen in a way so that channels with available records are preferred. - * The operation may block until at least one of the associated input channel - * is able to provide a record. - * - * @return the record read from one of the input channels or null if all channels are already closed. - * @throws ExecutionFailureException - * thrown if an error occurred while reading the channels - */ - - public T readRecord(final T target) throws IOException, InterruptedException { - - T record = null; - - if (this.executingThread == null) { - this.executingThread = Thread.currentThread(); - } - - if (this.executingThread.isInterrupted()) { - throw new InterruptedException(); - } - - while (true) { - - if (this.channelToReadFrom == -1) { - this.channelToReadFrom = waitForAnyChannelToBecomeAvailable(); - } - try { - record = this.getInputChannel(this.channelToReadFrom).readRecord(target); - } catch (EOFException e) { - // System.out.println("### Caught EOF exception at channel " + channelToReadFrom + "(" + - // this.getInputChannel(channelToReadFrom).getType().toString() + ")"); - if (this.isClosed()) { - return null; - } - } - - if (record != null) { - break; - } else { - this.channelToReadFrom = -1; - } - } - - if (this.inputGateListeners != null) { - for (final InputGateListener inputGateListener : this.inputGateListeners) { - inputGateListener.recordReceived(record); - } - } - - return record; - } + AbstractInputChannel getInputChannel(int pos); /** * Notify the gate that the channel with the given index has * at least one record available. */ - public void notifyRecordIsAvailable(int channelIndex) { - - synchronized (this.availableChannels) { - - this.availableChannels.add(Integer.valueOf(channelIndex)); - this.availableChannels.notify(); - } - } - - /** - * This method returns the index of a channel which has at least - * one record available. The method may block until at least one - * channel has become ready. - * - * @return the index of the channel which has at least one record available - */ - public int waitForAnyChannelToBecomeAvailable() throws InterruptedException { - - synchronized (this.availableChannels) { - - while (this.availableChannels.isEmpty()) { - - // notify the listener objects - if (this.inputGateListeners != null) { - for (int i = 0; i < this.inputGateListeners.length; ++i) { - this.inputGateListeners[i].waitingForAnyChannel(); - } - } - this.availableChannels.wait(); - } - - return this.availableChannels.removeFirst().intValue(); - } - } - - // TODO: See if type safety can be improved here - /** - * {@inheritDoc} - */ - @SuppressWarnings("unchecked") - @Override - public void read(DataInput in) throws IOException { - - super.read(in); - - final int numInputChannels = in.readInt(); - - for (int i = 0; i < numInputChannels; i++) { - - final ChannelID channelID = new ChannelID(); - channelID.read(in); - final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); - - final String className = StringRecord.readString(in); - Class c = null; - try { - c = ClassUtils.getRecordByName(className); - } catch (ClassNotFoundException e) { - LOG.error(e); - } - - if (c == null) { - throw new IOException("Class is null!"); - } - - AbstractInputChannel eic = null; - try { - final Constructor> constructor = (Constructor>) c - .getDeclaredConstructor(this.getClass(), int.class, RecordDeserializer.class, ChannelID.class, - CompressionLevel.class); - if (constructor == null) { - throw new IOException("Constructor is null!"); - } - constructor.setAccessible(true); - eic = constructor.newInstance(this, i, deserializer, channelID, compressionLevel); - } catch (SecurityException e) { - LOG.error(e); - } catch (NoSuchMethodException e) { - LOG.error(e); - } catch (IllegalArgumentException e) { - LOG.error(e); - } catch (InstantiationException e) { - LOG.error(e); - } catch (IllegalAccessException e) { - LOG.error(e); - } catch (InvocationTargetException e) { - LOG.error(e); - } - if (eic == null) { - throw new IOException("Created input channel is null!"); - } - - eic.read(in); - addInputChannel(eic); - } - } - - /** - * {@inheritDoc} - */ - @Override - public void write(DataOutput out) throws IOException { - - super.write(out); - - // Connected input channels - out.writeInt(this.getNumberOfInputChannels()); - for (int i = 0; i < getNumberOfInputChannels(); i++) { - getInputChannel(i).getID().write(out); - EnumUtils.writeEnum(out, getInputChannel(i).getCompressionLevel()); - StringRecord.writeString(out, getInputChannel(i).getClass().getName()); - getInputChannel(i).write(out); - } - - } - - /** - * {@inheritDoc} - */ - @Override - public boolean isClosed() throws IOException, InterruptedException { - - for (int i = 0; i < this.getNumberOfInputChannels(); i++) { - final AbstractInputChannel inputChannel = this.inputChannels.get(i); - if (!inputChannel.isClosed()) { - return false; - } - } - - return true; - } - - /** - * Immediately closes the input gate and all its input channels. The corresponding - * output channels are notified. Any remaining records in any buffers or queue is considered - * irrelevant and is discarded. - * - * @throws IOException - * thrown if an I/O error occurs while closing the gate - * @throws InterruptedException - * thrown if the thread is interrupted while waiting for the gate to be closed - */ - public void close() throws IOException, InterruptedException { - - for (int i = 0; i < this.getNumberOfInputChannels(); i++) { - final AbstractInputChannel inputChannel = this.inputChannels.get(i); - inputChannel.close(); - } - - } - - /** - * Returns the list of InputChannels that feed this RecordReader - * - * @return the list of InputChannels that feed this RecordReader - */ - public List> getInputChannels() { - return inputChannels; - } - - /** - * Registers a new listener object for this input gate. - * - * @param inputGateListener - * the listener object to register - */ - public void registerInputGateListener(InputGateListener inputGateListener) { - - if (this.inputGateListeners == null) { - this.inputGateListeners = new InputGateListener[1]; - this.inputGateListeners[0] = inputGateListener; - } else { - InputGateListener[] tmp = this.inputGateListeners; - this.inputGateListeners = new InputGateListener[tmp.length + 1]; - System.arraycopy(tmp, 0, this.inputGateListeners, 0, tmp.length); - this.inputGateListeners[tmp.length] = inputGateListener; - } - } - - /** - * {@inheritDoc} - */ - @Override - public String toString() { - return "Input " + super.toString(); - } - - /** - * {@inheritDoc} - */ - @Override - public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException { - - // Copy event to all connected channels - final Iterator> it = this.inputChannels.iterator(); - while (it.hasNext()) { - it.next().transferEvent(event); - } - } - - /** - * Returns the {@link RecordDeserializer} used by this input gate. - * - * @return the {@link RecordDeserializer} used by this input gate - */ - public RecordDeserializer getRecordDeserializer() { - - return this.deserializer; - } - - /** - * {@inheritDoc} - */ - @Override - public void releaseAllChannelResources() { - - final Iterator> it = this.inputChannels.iterator(); - while (it.hasNext()) { - it.next().releaseResources(); - } - } + void notifyRecordIsAvailable(int channelIndex); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java index a8b39926f8c8a..7add6680f0cec 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java @@ -15,332 +15,29 @@ package eu.stratosphere.nephele.io; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.util.ArrayList; -import java.util.Iterator; +import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.event.task.AbstractTaskEvent; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; -import eu.stratosphere.nephele.io.channels.bytebuffered.FileOutputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkOutputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryOutputChannel; -import eu.stratosphere.nephele.io.compression.CompressionLevel; -import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.types.StringRecord; -import eu.stratosphere.nephele.util.ClassUtils; -import eu.stratosphere.nephele.util.EnumUtils; /** * In Nephele output gates are a specialization of general gates and connect * record writers and output channels. As channels, output gates are always * parameterized to a specific type of record which they can transport. - *

- * This class is in general not thread-safe. * * @author warneke * @param * the type of record that can be transported through this gate */ -public class OutputGate extends AbstractGate { - - /** - * The log object used for debugging. - */ - private static final Log LOG = LogFactory.getLog(OutputGate.class); - - /** - * The list of output channels attached to this gate. - */ - private final ArrayList> outputChannels = new ArrayList>(); - - /** - * Channel selector to determine which channel is supposed receive the next record. - */ - private final ChannelSelector channelSelector; - - /** - * The class of the record transported through this output gate. - */ - private final Class type; - - /** - * The listener objects registered for this output gate. - */ - private OutputGateListener[] outputGateListeners = null; - - /** - * The thread which executes the task connected to the output gate. - */ - private Thread executingThread = null; - - /** - * Stores whether all records passed to this output gate shall be transmitted through all connected output channels. - */ - private final boolean isBroadcast; - - /** - * Constructs a new output gate. - * - * @param jobID - * the ID of the job this input gate belongs to - * @param gateID - * the ID of the gate - * @param inputClass - * the class of the record that can be transported through this - * gate - * @param index - * the index assigned to this output gate at the {@link Environment} object - * @param channelSelector - * the channel selector to be used for this output gate - * @param isBroadcast - * true if every records passed to this output gate shall be transmitted through all connected - * output channels, false otherwise - */ - public OutputGate(final JobID jobID, final GateID gateID, final Class inputClass, final int index, - final ChannelSelector channelSelector, final boolean isBroadcast) { - - super(jobID, gateID, index); - - this.isBroadcast = isBroadcast; - this.type = inputClass; - - if (this.isBroadcast) { - this.channelSelector = null; - } else { - if (channelSelector == null) { - this.channelSelector = new DefaultChannelSelector(); - } else { - this.channelSelector = channelSelector; - } - } - } +public interface OutputGate extends Gate { /** * Returns the type of record that can be transported through this gate. * * @return the type of record that can be transported through this gate */ - public final Class getType() { - return this.type; - } - - /** - * Adds a new output channel to the output gate. - * - * @param outputChannel - * the output channel to be added. - */ - private void addOutputChannel(AbstractOutputChannel outputChannel) { - if (!this.outputChannels.contains(outputChannel)) { - this.outputChannels.add(outputChannel); - } - } - - /** - * Removes the output channel with the given ID from the output gate if it - * exists. - * - * @param outputChannelID - * the ID of the channel to be removed - */ - public void removeOutputChannel(ChannelID outputChannelID) { - - for (int i = 0; i < this.outputChannels.size(); i++) { - - final AbstractOutputChannel outputChannel = this.outputChannels.get(i); - if (outputChannel.getID().equals(outputChannelID)) { - this.outputChannels.remove(i); - return; - } - } - - LOG.debug("Cannot find output channel with ID " + outputChannelID + " to remove"); - } - - /** - * Removes all output channels from the output gate. - */ - public void removeAllOutputChannels() { - - this.outputChannels.clear(); - } - - public AbstractOutputChannel replaceChannel(ChannelID oldChannelID, ChannelType newChannelType, - boolean followsPushModel) { - - AbstractOutputChannel oldOutputChannel = null; - - for (int i = 0; i < this.outputChannels.size(); i++) { - final AbstractOutputChannel outputChannel = this.outputChannels.get(i); - if (outputChannel.getID().equals(oldChannelID)) { - oldOutputChannel = outputChannel; - break; - } - } - - if (oldOutputChannel == null) { - return null; - } - - AbstractOutputChannel newOutputChannel = null; - - switch (newChannelType) { - case FILE: - newOutputChannel = new FileOutputChannel(this, oldOutputChannel.getChannelIndex(), oldOutputChannel - .getID(), oldOutputChannel.getCompressionLevel()); - break; - case INMEMORY: - newOutputChannel = new InMemoryOutputChannel(this, oldOutputChannel.getChannelIndex(), oldOutputChannel - .getID(), oldOutputChannel.getCompressionLevel()); - break; - case NETWORK: - newOutputChannel = new NetworkOutputChannel(this, oldOutputChannel.getChannelIndex(), oldOutputChannel - .getID(), oldOutputChannel.getCompressionLevel()); - break; - default: - return null; - } - - newOutputChannel.setConnectedChannelID(oldOutputChannel.getConnectedChannelID()); - - this.outputChannels.set(newOutputChannel.getChannelIndex(), newOutputChannel); - - return newOutputChannel; - } - - /** - * {@inheritDoc} - */ - @Override - public boolean isInputGate() { - - return false; - } - - /** - * Returns the number of output channels associated with this output gate. - * - * @return the number of output channels associated with this output gate - */ - public int getNumberOfOutputChannels() { - - return this.outputChannels.size(); - } - - /** - * Returns the output channel from position pos of the gate's - * internal channel list. - * - * @param pos - * the position to retrieve the channel from - * @return the channel from the given position or null if such - * position does not exist. - */ - public AbstractOutputChannel getOutputChannel(int pos) { - - if (pos < this.outputChannels.size()) - return this.outputChannels.get(pos); - else - return null; - } - - /** - * Creates a new network output channel and assigns it to the output gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new network output channel - */ - public NetworkOutputChannel createNetworkOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - - final NetworkOutputChannel enoc = new NetworkOutputChannel(this, this.outputChannels.size(), channelID, - compressionLevel); - addOutputChannel(enoc); - - return enoc; - } - - /** - * Creates a new file output channel and assigns it to the output gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new file output channel - */ - public FileOutputChannel createFileOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - - final FileOutputChannel efoc = new FileOutputChannel(this, this.outputChannels.size(), channelID, - compressionLevel); - addOutputChannel(efoc); - - return efoc; - } - - /** - * Creates a new in-memory output channel and assigns it to the output gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new in-memory output channel - */ - public InMemoryOutputChannel createInMemoryOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - - final InMemoryOutputChannel einoc = new InMemoryOutputChannel(this, this.outputChannels.size(), - channelID, compressionLevel); - addOutputChannel(einoc); - - return einoc; - } - - /** - * Requests the output gate to closed. This means the application will send - * no records through this gate anymore. - * - * @throws IOException - * @throws InterruptedException - */ - public void requestClose() throws IOException, InterruptedException { - // Close all output channels - for (int i = 0; i < this.getNumberOfOutputChannels(); i++) { - final AbstractOutputChannel outputChannel = this.getOutputChannel(i); - outputChannel.requestClose(); - } - } - - /** - * {@inheritDoc} - */ - @Override - public boolean isClosed() throws IOException, InterruptedException { - - boolean allClosed = true; - - for (int i = 0; i < this.getNumberOfOutputChannels(); i++) { - final AbstractOutputChannel outputChannel = this.getOutputChannel(i); - if (!outputChannel.isClosed()) { - allClosed = false; - } - } - - return allClosed; - } + Class getType(); /** * Writes a record to one of the associated output channels. Currently, the @@ -352,190 +49,14 @@ public boolean isClosed() throws IOException, InterruptedException { * @throws IOException * thrown if any error occurs during channel I/O */ - public void writeRecord(T record) throws IOException, InterruptedException { - - if (this.executingThread == null) { - this.executingThread = Thread.currentThread(); - } - - if (this.executingThread.isInterrupted()) { - throw new InterruptedException(); - } - - if (this.outputGateListeners != null) { - for (final OutputGateListener outputGateListener : this.outputGateListeners) { - outputGateListener.recordEmitted(record); - } - } - - if (this.isBroadcast) { - - if (getChannelType() == ChannelType.INMEMORY) { - - final int numberOfOutputChannels = this.outputChannels.size(); - for (int i = 0; i < numberOfOutputChannels; ++i) { - this.outputChannels.get(i).writeRecord(record); - } - - } else { - - // Use optimization for byte buffered channels - this.outputChannels.get(0).writeRecord(record); - } - - } else { - - // Non-broadcast gate, use channel selector to select output channels - final int numberOfOutputChannels = this.outputChannels.size(); - final int[] selectedOutputChannels = this.channelSelector.selectChannels(record, numberOfOutputChannels); - - if (selectedOutputChannels == null) { - return; - } - - for (int i = 0; i < selectedOutputChannels.length; ++i) { - - if (selectedOutputChannels[i] < numberOfOutputChannels) { - final AbstractOutputChannel outputChannel = this.outputChannels.get(selectedOutputChannels[i]); - outputChannel.writeRecord(record); - } - } - } - } - - // TODO: See if type safety can be improved here - /** - * {@inheritDoc} - */ - @SuppressWarnings("unchecked") - public void read(DataInput in) throws IOException { - - super.read(in); - - final int numOutputChannels = in.readInt(); - - final Class[] parameters = { this.getClass(), int.class, ChannelID.class, CompressionLevel.class }; - - for (int i = 0; i < numOutputChannels; i++) { - - final ChannelID channelID = new ChannelID(); - channelID.read(in); - final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); - final String className = StringRecord.readString(in); - Class c = null; - try { - c = ClassUtils.getRecordByName(className); - } catch (ClassNotFoundException e) { - LOG.error(e); - } - - if (c == null) { - throw new IOException("Class is null!"); - } - - AbstractOutputChannel eoc = null; - try { - final Constructor> constructor = (Constructor>) c - .getDeclaredConstructor(parameters); - - if (constructor == null) { - throw new IOException("Constructor is null!"); - } - - constructor.setAccessible(true); - - eoc = constructor.newInstance(this, i, channelID, compressionLevel); - - } catch (InstantiationException e) { - LOG.error(e); - } catch (IllegalArgumentException e) { - LOG.error(e); - } catch (IllegalAccessException e) { - LOG.error(e); - } catch (InvocationTargetException e) { - LOG.error(e); - } catch (SecurityException e) { - LOG.error(e); - } catch (NoSuchMethodException e) { - LOG.error(e); - } - - if (eoc == null) { - throw new IOException("Created output channel is null!"); - } - - eoc.read(in); - addOutputChannel(eoc); - } - } - - /** - * {@inheritDoc} - */ - public void write(DataOutput out) throws IOException { - - super.write(out); - - // Output channels - out.writeInt(this.getNumberOfOutputChannels()); - - for (int i = 0; i < getNumberOfOutputChannels(); i++) { - getOutputChannel(i).getID().write(out); - EnumUtils.writeEnum(out, getOutputChannel(i).getCompressionLevel()); - StringRecord.writeString(out, getOutputChannel(i).getClass().getName()); - getOutputChannel(i).write(out); - } - - } + void writeRecord(T record) throws IOException, InterruptedException; /** * Returns all the OutputChannels connected to this gate * * @return the list of OutputChannels connected to this RecordWriter */ - public ArrayList> getOutputChannels() { - return this.outputChannels; - } - - /** - * Registers a new listener object for this output gate. - * - * @param outputGateListener - * the listener object to register - */ - public void registerOutputGateListener(OutputGateListener outputGateListener) { - - if (this.outputGateListeners == null) { - this.outputGateListeners = new OutputGateListener[1]; - this.outputGateListeners[0] = outputGateListener; - } else { - OutputGateListener[] tmp = this.outputGateListeners; - this.outputGateListeners = new OutputGateListener[tmp.length + 1]; - System.arraycopy(tmp, 0, this.outputGateListeners, 0, tmp.length); - this.outputGateListeners[tmp.length] = outputGateListener; - } - } - - /** - * {@inheritDoc} - */ - @Override - public String toString() { - return "Output " + super.toString(); - } - - /** - * {@inheritDoc} - */ - @Override - public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException { - - // Copy event to all connected channels - final Iterator> it = this.outputChannels.iterator(); - while (it.hasNext()) { - it.next().transferEvent(event); - } - } + List> getOutputChannels(); /** * Flushes all connected output channels. @@ -545,13 +66,7 @@ public void publishEvent(AbstractTaskEvent event) throws IOException, Interrupte * @throws InterruptedException * thrown if the thread is interrupted while waiting for the data to be flushed */ - public void flush() throws IOException, InterruptedException { - // Flush all connected channels - final Iterator> it = this.outputChannels.iterator(); - while (it.hasNext()) { - it.next().flush(); - } - } + void flush() throws IOException, InterruptedException; /** * This method is called by one of the attached output channel when its @@ -560,15 +75,7 @@ public void flush() throws IOException, InterruptedException { * @param channelIndex * the index of the exhausted output channel. */ - public void channelCapacityExhausted(int channelIndex) { - - // notify the listener objects - if (this.outputGateListeners != null) { - for (int i = 0; i < this.outputGateListeners.length; ++i) { - this.outputGateListeners[i].channelCapacityExhausted(channelIndex); - } - } - } + void channelCapacityExhausted(int channelIndex); /** * Checks if this output gate operates in broadcast mode, i.e. all records passed to it are transferred through all @@ -576,31 +83,12 @@ public void channelCapacityExhausted(int channelIndex) { * * @return true if this output gate operates in broadcast mode, false otherwise */ - public boolean isBroadcast() { - - return this.isBroadcast; - } + boolean isBroadcast(); /** - * Returns the output gate's channel selector. + * Returns the number of output channels associated with this output gate. * - * @return the output gate's channel selector or null if the gate operates in broadcast mode - */ - public ChannelSelector getChannelSelector() { - - return this.channelSelector; - } - - /** - * {@inheritDoc} + * @return the number of output channels associated with this output gate */ - @Override - public void releaseAllChannelResources() { - - final Iterator> it = this.outputChannels.iterator(); - - while (it.hasNext()) { - it.next().releaseResources(); - } - } + int getNumberOfOutputChannels(); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java index 4be69ef69d095..6ad4a70323ec2 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java @@ -41,7 +41,7 @@ public abstract class AbstractOutputChannel extends AbstractChannel { private OutputGate outputGate = null; - + /** * Creates a new output channel object. * diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java index fc5d8346f2876..7ff86201fd533 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java @@ -36,6 +36,8 @@ import eu.stratosphere.nephele.instance.InstanceConnectionInfo; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeInputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.ipc.RPC; import eu.stratosphere.nephele.net.NetUtils; import eu.stratosphere.nephele.profiling.ProfilingException; @@ -73,7 +75,7 @@ public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerPro private final Map, OutputGateListenerImpl> monitoredOutputGates = new HashMap, OutputGateListenerImpl>(); public TaskManagerProfilerImpl(InetAddress jobManagerAddress, InstanceConnectionInfo instanceConnectionInfo) - throws ProfilingException { + throws ProfilingException { // Create RPC stub for communication with job manager's profiling component. final InetSocketAddress profilingAddress = new InetSocketAddress(jobManagerAddress, GlobalConfiguration @@ -120,7 +122,7 @@ public void registerExecutionListener(final Task task, final Configuration jobCo @Override public void registerInputGateListener(ExecutionVertexID id, Configuration jobConfiguration, - InputGate inputGate) { + RuntimeInputGate inputGate) { synchronized (this.monitoredInputGates) { @@ -133,7 +135,7 @@ public void registerInputGateListener(ExecutionVertexID id, Configuration jobCon @Override public void registerOutputGateListener(ExecutionVertexID id, Configuration jobConfiguration, - OutputGate outputGate) { + RuntimeOutputGate outputGate) { synchronized (this.monitoredOutputGates) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 5451e33bfbf1d..3336bf552d9ad 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -33,6 +33,8 @@ import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RecordDeserializer; +import eu.stratosphere.nephele.io.RuntimeInputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; @@ -66,22 +68,22 @@ public class RuntimeEnvironment implements Environment, Runnable, IOReadableWrit /** * List of output gates created by the task. */ - private final List> outputGates = new CopyOnWriteArrayList>(); + private final List> outputGates = new CopyOnWriteArrayList>(); /** * List of input gates created by the task. */ - private final List> inputGates = new CopyOnWriteArrayList>(); + private final List> inputGates = new CopyOnWriteArrayList>(); /** * List of output gates which have to be rebound to a task after transferring the environment to a TaskManager. */ - private final List> unboundOutputGates = new CopyOnWriteArrayList>(); + private final List> unboundOutputGates = new CopyOnWriteArrayList>(); /** * List of input gates which have to be rebound to a task after transferring the environment to a TaskManager. */ - private final List> unboundInputGates = new CopyOnWriteArrayList>(); + private final List> unboundInputGates = new CopyOnWriteArrayList>(); /** * The memory manager of the current environment (currently the one associated with the executing TaskManager). @@ -384,18 +386,34 @@ private void activateInputChannels() throws IOException, InterruptedException { /** * {@inheritDoc} */ - public void registerOutputGate(final OutputGate outputGate) { + @Override + public OutputGate createAndRegisterOutputGate(final Class outputClass, + final ChannelSelector selector, final boolean isBroadcast) { + + @SuppressWarnings({ "unchecked", "rawtypes" }) + final RuntimeOutputGate rog = (RuntimeOutputGate) new RuntimeOutputGate( + getJobID(), new GateID(), outputClass, getNumberOfOutputGates(), selector, isBroadcast); - this.outputGates.add(outputGate); + this.outputGates.add(rog); + + return rog; } /** * {@inheritDoc} */ @Override - public void registerInputGate(final InputGate inputGate) { + public InputGate createAndRegisterInputGate( + final RecordDeserializer deserializer, + final DistributionPattern distributionPattern) { + + @SuppressWarnings({ "unchecked", "rawtypes" }) + final RuntimeInputGate rig = (RuntimeInputGate) new RuntimeInputGate( + getJobID(), new GateID(), deserializer, getNumberOfInputGates(), distributionPattern); - this.inputGates.add(inputGate); + this.inputGates.add(rig); + + return rig; } /** @@ -435,7 +453,7 @@ public InputGate getInputGate(final int pos) { * the index of the output gate to return * @return the output gate at index pos or null if no such index exists */ - public OutputGate getOutputGate(final int pos) { + public RuntimeOutputGate getOutputGate(final int pos) { if (pos < this.outputGates.size()) { return this.outputGates.get(pos); } @@ -542,8 +560,8 @@ public void read(final DataInput in) throws IOException { } @SuppressWarnings("rawtypes") - final OutputGate eog = new OutputGate(this.jobID, gateID, type, i, channelSelector, - isBroadcast); + final RuntimeOutputGate eog = new RuntimeOutputGate(this.jobID, gateID, type, i, + channelSelector, isBroadcast); eog.read(in); this.outputGates.add(eog); // Mark as unbound for reconnection of RecordWriter @@ -594,8 +612,8 @@ public void read(final DataInput in) throws IOException { } @SuppressWarnings("rawtypes") - final InputGate eig = new InputGate(this.jobID, gateID, recordDeserializer, i, - distributionPattern); + final RuntimeInputGate eig = new RuntimeInputGate(this.jobID, gateID, recordDeserializer, + i, distributionPattern); eig.read(in); this.inputGates.add(eig); // Mark as unbound for reconnection of RecordReader @@ -651,9 +669,9 @@ public void write(final DataOutput out) throws IOException { StringRecord.writeString(out, this.invokableClass.getName()); // Output gates - out.writeInt(getNumberOfOutputGates()); - for (int i = 0; i < getNumberOfOutputGates(); i++) { - final OutputGate outputGate = getOutputGate(i); + out.writeInt(this.outputGates.size()); + for (int i = 0; i < this.outputGates.size(); i++) { + final RuntimeOutputGate outputGate = this.outputGates.get(i); outputGate.getGateID().write(out); StringRecord.writeString(out, outputGate.getType().getName()); out.writeBoolean(outputGate.isBroadcast()); @@ -663,18 +681,18 @@ public void write(final DataOutput out) throws IOException { outputGate.getChannelSelector().write(out); } - getOutputGate(i).write(out); + outputGate.write(out); } // Input gates out.writeInt(getNumberOfInputGates()); for (int i = 0; i < getNumberOfInputGates(); i++) { - final InputGate inputGate = getInputGate(i); + final RuntimeInputGate inputGate = this.inputGates.get(i); inputGate.getGateID().write(out); StringRecord.writeString(out, inputGate.getRecordDeserializer().getClass().getName()); inputGate.getRecordDeserializer().write(out); StringRecord.writeString(out, inputGate.getDistributionPattern().getClass().getName()); - getInputGate(i).write(out); + inputGate.write(out); } // The configuration object @@ -706,8 +724,8 @@ private void waitForOutputChannelsToBeClosed() throws IOException, InterruptedEx boolean allClosed = true; for (int i = 0; i < getNumberOfOutputGates(); i++) { - final OutputGate eog = getOutputGate(i); - if (!eog.isClosed()) { + final RuntimeOutputGate rog = this.outputGates.get(i); + if (!rog.isClosed()) { allClosed = false; } } @@ -740,7 +758,7 @@ private void waitForInputChannelsToBeClosed() throws IOException, InterruptedExc boolean allClosed = true; for (int i = 0; i < getNumberOfInputGates(); i++) { - final InputGate eig = getInputGate(i); + final RuntimeInputGate eig = this.inputGates.get(i); if (!eig.isClosed()) { allClosed = false; } @@ -759,8 +777,8 @@ private void waitForInputChannelsToBeClosed() throws IOException, InterruptedExc */ private void closeInputGates() throws IOException, InterruptedException { - for (int i = 0; i < getNumberOfInputGates(); i++) { - final InputGate eig = getInputGate(i); + for (int i = 0; i < this.inputGates.size(); i++) { + final RuntimeInputGate eig = this.inputGates.get(i); // Important: close must be called on each input gate exactly once eig.close(); } @@ -772,8 +790,8 @@ private void closeInputGates() throws IOException, InterruptedException { */ private void requestAllOutputGatesToClose() throws IOException, InterruptedException { - for (int i = 0; i < getNumberOfOutputGates(); i++) { - this.getOutputGate(i).requestClose(); + for (int i = 0; i < this.outputGates.size(); i++) { + this.outputGates.get(i).requestClose(); } } @@ -963,12 +981,12 @@ public void userThreadFinished(final Thread userThread) { */ private void releaseAllChannelResources() { - for (int i = 0; i < getNumberOfInputGates(); i++) { - this.getInputGate(i).releaseAllChannelResources(); + for (int i = 0; i < this.inputGates.size(); i++) { + this.inputGates.get(i).releaseAllChannelResources(); } - for (int i = 0; i < getNumberOfOutputGates(); i++) { - this.getOutputGate(i).releaseAllChannelResources(); + for (int i = 0; i < this.outputGates.size(); i++) { + this.outputGates.get(i).releaseAllChannelResources(); } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index c76c5c30c7cad..19578d706573d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -39,6 +39,7 @@ import eu.stratosphere.nephele.instance.InstanceType; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; @@ -391,7 +392,7 @@ void unwire(final ExecutionGroupVertex source, final int indexOfOutputGate, fina for (int i = 0; i < source.getCurrentNumberOfGroupMembers(); i++) { final ExecutionVertex sourceVertex = source.getGroupMember(i); - final OutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( + final RuntimeOutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( indexOfOutputGate); if (outputGate == null) { throw new GraphConversionException("unwire: " + sourceVertex.getName() @@ -411,7 +412,7 @@ void unwire(final ExecutionGroupVertex source, final int indexOfOutputGate, fina for (int i = 0; i < target.getCurrentNumberOfGroupMembers(); i++) { final ExecutionVertex targetVertex = target.getGroupMember(i); - final InputGate inputGate = targetVertex.getEnvironment().getInputGate(indexOfInputGate); + final RuntimeInputGate inputGate = targetVertex.getEnvironment().getInputGate(indexOfInputGate); if (inputGate == null) { throw new GraphConversionException("unwire: " + targetVertex.getName() + " has no input gate with index " + indexOfInputGate); @@ -622,11 +623,11 @@ private ExecutionVertex createVertex(final AbstractJobVertex jobVertex, final In if (ev.getEnvironment().getInvokable() instanceof AbstractInputTask) { try { inputSplits = ((AbstractInputTask) ev.getEnvironment().getInvokable()). - computeInputSplits(jobVertex.getNumberOfSubtasks()); + computeInputSplits(jobVertex.getNumberOfSubtasks()); } catch (Exception e) { throw new GraphConversionException("Cannot compute input splits for " + groupVertex.getName() + ": " - + StringUtils.stringifyException(e)); + + StringUtils.stringifyException(e)); } } else { throw new GraphConversionException( diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java index 2cb773fa3032a..851528a04d6ba 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java @@ -31,7 +31,7 @@ import eu.stratosphere.nephele.instance.InstanceRequestMap; import eu.stratosphere.nephele.instance.InstanceType; import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelType; @@ -389,7 +389,7 @@ private void reconstructExecutionPipeline(final ExecutionVertex vertex, final bo final int numberOfOutputGates = env.getNumberOfOutputGates(); for (int i = 0; i < numberOfOutputGates; ++i) { - final OutputGate outputGate = env.getOutputGate(i); + final RuntimeOutputGate outputGate = env.getOutputGate(i); final ChannelType channelType = outputGate.getChannelType(); final int numberOfOutputChannels = outputGate.getNumberOfOutputChannels(); for (int j = 0; j < numberOfOutputChannels; ++j) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 5619b08216967..f3c31daf1082b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -32,7 +32,7 @@ import eu.stratosphere.nephele.instance.AllocatedResource; import eu.stratosphere.nephele.instance.AllocationID; import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; @@ -477,7 +477,7 @@ public ExecutionVertex getSuccessor(int index) { for (int i = 0; i < this.environment.getNumberOfOutputGates(); i++) { - final OutputGate outputGate = this.environment.getOutputGate(i); + final RuntimeOutputGate outputGate = this.environment.getOutputGate(i); if (index >= 0 && index < outputGate.getNumberOfOutputChannels()) { @@ -522,7 +522,7 @@ public SerializableHashSet constructInitialActiveOutputChannelsSet() final int numberOfOutputGates = this.environment.getNumberOfOutputGates(); for (int i = 0; i < numberOfOutputGates; ++i) { - final OutputGate outputGate = this.environment.getOutputGate(i); + final RuntimeOutputGate outputGate = this.environment.getOutputGate(i); final ChannelType channelType = outputGate.getChannelType(); final int numberOfOutputChannels = outputGate.getNumberOfOutputChannels(); for (int j = 0; j < numberOfOutputChannels; ++j) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java index 3f059966a972d..d190c5b7c7747 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java @@ -22,6 +22,7 @@ import eu.stratosphere.nephele.io.AbstractGate; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; @@ -134,7 +135,7 @@ private static void addExecutionVertices(Map outputGate = ev.getEnvironment().getOutputGate(i); + final RuntimeOutputGate outputGate = ev.getEnvironment().getOutputGate(i); final ManagementGate managementGate = new ManagementGate(managementVertex, i, false, outputGate .getType().toString()); gateMap.put(outputGate, managementGate); diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java similarity index 83% rename from nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java rename to nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java index a85762e44a4d5..d0faf41e591a1 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java @@ -39,7 +39,7 @@ * @param * the record type to be transported from this gate */ -public abstract class AbstractGate implements IOReadableWritable { +public abstract class AbstractGate implements Gate, IOReadableWritable { /** * The ID of the job this gate belongs to. @@ -128,26 +128,18 @@ public String toString() { } /** - * Subscribes the listener object to receive events of the given type. - * - * @param eventListener - * the listener object to register - * @param eventType - * the type of event to register the listener for + * {@inheritDoc} */ + @Override public void subscribeToEvent(EventListener eventListener, Class eventType) { this.eventNotificationManager.subscribeToEvent(eventListener, eventType); } /** - * Removes the subscription for events of the given type for the listener object. - * - * @param eventListener - * the listener object to cancel the subscription for - * @param eventType - * the type of the event to cancel the subscription for + * {@inheritDoc} */ + @Override public final void unsubscribeFromEvent(final EventListener eventListener, final Class eventType) { @@ -155,28 +147,14 @@ public final void unsubscribeFromEvent(final EventListener eventListener, } /** - * Passes a received event on to the event notification manager so it cam ne dispatched. - * - * @param event - * the event to pass on to the notification manager + * {@inheritDoc} */ + @Override public final void deliverEvent(final AbstractTaskEvent event) { this.eventNotificationManager.deliverEvent((AbstractTaskEvent) event); } - /** - * Publishes an event. - * - * @param event - * the event to be published - * @throws IOException - * thrown if an error occurs while transmitting the event - * @throws InterruptedException - * thrown if the thread is interrupted while waiting for the event to be published - */ - public abstract void publishEvent(final AbstractTaskEvent event) throws IOException, InterruptedException; - /** * Sets the type of the input/output channels which are connected to this gate. * @@ -217,10 +195,9 @@ public void write(final DataOutput out) throws IOException { } /** - * Returns the ID of the job this gate belongs to. - * - * @return the ID of the job this gate belongs to + * {@inheritDoc} */ + @Override public JobID getJobID() { return this.jobID; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java new file mode 100644 index 0000000000000..9495615a72a7c --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java @@ -0,0 +1,580 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.EOFException; +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.event.task.AbstractTaskEvent; +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.io.channels.AbstractInputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.channels.bytebuffered.FileInputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkInputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryInputChannel; +import eu.stratosphere.nephele.io.compression.CompressionLevel; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.types.Record; +import eu.stratosphere.nephele.types.StringRecord; +import eu.stratosphere.nephele.util.ClassUtils; +import eu.stratosphere.nephele.util.EnumUtils; + +/** + * In Nephele input gates are a specialization of general gates and connect input channels and record readers. As + * channels, input + * gates are always parameterized to a specific type of record which they can transport. In contrast to output gates + * input gates + * can be associated with a {@link DistributionPattern} object which dictates the concrete wiring between two groups of + * vertices. + *

+ * This class is in general not thread-safe. + * + * @author warneke + * @param + * the type of record that can be transported through this gate + */ +public final class RuntimeInputGate extends AbstractGate implements InputGate, + IOReadableWritable { + + /** + * The log object used for debugging. + */ + private static final Log LOG = LogFactory.getLog(InputGate.class); + + /** + * The deserializer used to construct records from byte streams. + */ + private final RecordDeserializer deserializer; + + /** + * The list of input channels attached to this input gate. + */ + private final ArrayList> inputChannels = new ArrayList>(); + + /** + * The distribution pattern to determine how to wire the channels. + */ + private final DistributionPattern distributionPattern; + + /** + * Queue with indices of channels that store at least one available record. + */ + private final ArrayDeque availableChannels = new ArrayDeque(); + + /** + * The listener objects registered for this input gate. + */ + private InputGateListener[] inputGateListeners = null; + + /** + * The channel to read from next. + */ + private int channelToReadFrom = -1; + + /** + * The thread which executes the task connected to the input gate. + */ + private Thread executingThread = null; + + /** + * Constructs a new runtime input gate. + * + * @param jobID + * the ID of the job this input gate belongs to + * @param gateID + * the ID of the gate + * @param inputClass + * the class of the record that can be transported through this gate + * @param index + * the index assigned to this input gate at the {@link Environment} object + * @param distributionPattern + * the distribution pattern to determine the concrete wiring between to groups of vertices + */ + public RuntimeInputGate(final JobID jobID, final GateID gateID, final RecordDeserializer deserializer, + final int index, + final DistributionPattern distributionPattern) { + + super(jobID, gateID, index); + + this.deserializer = deserializer; + + this.distributionPattern = (distributionPattern != null) ? distributionPattern + : new BipartiteDistributionPattern(); + } + + /** + * Adds a new input channel to the input gate. + * + * @param inputChannel + * the input channel to be added. + */ + private void addInputChannel(AbstractInputChannel inputChannel) { + + if (!this.inputChannels.contains(inputChannel)) { + this.inputChannels.add(inputChannel); + } + } + + /** + * Removes the input channel with the given ID from the input gate if it exists. + * + * @param inputChannelID + * the ID of the channel to be removed + */ + public void removeInputChannel(ChannelID inputChannelID) { + + for (int i = 0; i < this.inputChannels.size(); i++) { + + final AbstractInputChannel inputChannel = this.inputChannels.get(i); + if (inputChannel.getID().equals(inputChannelID)) { + this.inputChannels.remove(i); + return; + } + } + + LOG.debug("Cannot find output channel with ID " + inputChannelID + " to remove"); + } + + /** + * Removes all input channels from the input gate. + */ + public void removeAllInputChannels() { + + this.inputChannels.clear(); + } + + public AbstractInputChannel replaceChannel(ChannelID oldChannelID, ChannelType newChannelType) { + + AbstractInputChannel oldInputChannel = null; + + for (int i = 0; i < this.inputChannels.size(); i++) { + final AbstractInputChannel inputChannel = this.inputChannels.get(i); + if (inputChannel.getID().equals(oldChannelID)) { + oldInputChannel = inputChannel; + break; + } + } + + if (oldInputChannel == null) { + return null; + } + + AbstractInputChannel newInputChannel = null; + + switch (newChannelType) { + case FILE: + newInputChannel = new FileInputChannel(this, oldInputChannel.getChannelIndex(), deserializer, + oldInputChannel.getID(), oldInputChannel.getCompressionLevel()); + break; + case INMEMORY: + newInputChannel = new InMemoryInputChannel(this, oldInputChannel.getChannelIndex(), deserializer, + oldInputChannel.getID(), oldInputChannel.getCompressionLevel()); + break; + case NETWORK: + newInputChannel = new NetworkInputChannel(this, oldInputChannel.getChannelIndex(), deserializer, + oldInputChannel.getID(), oldInputChannel.getCompressionLevel()); + break; + default: + LOG.error("Unknown input channel type"); + return null; + } + + newInputChannel.setConnectedChannelID(oldInputChannel.getConnectedChannelID()); + + this.inputChannels.set(newInputChannel.getChannelIndex(), newInputChannel); + + return newInputChannel; + } + + /** + * Returns the {@link DistributionPattern} associated with this input gate. + * + * @return the {@link DistributionPattern} associated with this input gate + */ + public DistributionPattern getDistributionPattern() { + return this.distributionPattern; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isInputGate() { + + return true; + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfInputChannels() { + + return this.inputChannels.size(); + } + + /** + * {@inheritDoc} + */ + @Override + public AbstractInputChannel getInputChannel(int pos) { + + if (pos < this.inputChannels.size()) { + return this.inputChannels.get(pos); + } + + return null; + } + + /** + * Creates a new network input channel and assigns it to the input gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new network input channel + */ + public NetworkInputChannel createNetworkInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + + final NetworkInputChannel enic = new NetworkInputChannel(this, this.inputChannels.size(), deserializer, + channelID, compressionLevel); + addInputChannel(enic); + + return enic; + } + + /** + * Creates a new file input channel and assigns it to the input gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new file input channel + */ + public FileInputChannel createFileInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + + final FileInputChannel efic = new FileInputChannel(this, this.inputChannels.size(), deserializer, + channelID, compressionLevel); + addInputChannel(efic); + + return efic; + } + + /** + * Creates a new in-memory input channel and assigns it to the input gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new in-memory input channel + */ + public InMemoryInputChannel createInMemoryInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + + final InMemoryInputChannel eimic = new InMemoryInputChannel(this, this.inputChannels.size(), + deserializer, channelID, compressionLevel); + addInputChannel(eimic); + + return eimic; + } + + /** + * {@inheritDoc} + */ + @Override + public T readRecord(final T target) throws IOException, InterruptedException { + + T record = null; + + if (this.executingThread == null) { + this.executingThread = Thread.currentThread(); + } + + if (this.executingThread.isInterrupted()) { + throw new InterruptedException(); + } + + while (true) { + + if (this.channelToReadFrom == -1) { + this.channelToReadFrom = waitForAnyChannelToBecomeAvailable(); + } + try { + record = this.getInputChannel(this.channelToReadFrom).readRecord(target); + } catch (EOFException e) { + // System.out.println("### Caught EOF exception at channel " + channelToReadFrom + "(" + + // this.getInputChannel(channelToReadFrom).getType().toString() + ")"); + if (this.isClosed()) { + return null; + } + } + + if (record != null) { + break; + } else { + this.channelToReadFrom = -1; + } + } + + if (this.inputGateListeners != null) { + for (final InputGateListener inputGateListener : this.inputGateListeners) { + inputGateListener.recordReceived(record); + } + } + + return record; + } + + /** + * {@inheritDoc} + */ + @Override + public void notifyRecordIsAvailable(final int channelIndex) { + + synchronized (this.availableChannels) { + + this.availableChannels.add(Integer.valueOf(channelIndex)); + this.availableChannels.notify(); + } + } + + /** + * This method returns the index of a channel which has at least + * one record available. The method may block until at least one + * channel has become ready. + * + * @return the index of the channel which has at least one record available + */ + public int waitForAnyChannelToBecomeAvailable() throws InterruptedException { + + synchronized (this.availableChannels) { + + while (this.availableChannels.isEmpty()) { + + // notify the listener objects + if (this.inputGateListeners != null) { + for (int i = 0; i < this.inputGateListeners.length; ++i) { + this.inputGateListeners[i].waitingForAnyChannel(); + } + } + this.availableChannels.wait(); + } + + return this.availableChannels.removeFirst().intValue(); + } + } + + // TODO: See if type safety can be improved here + /** + * {@inheritDoc} + */ + @SuppressWarnings("unchecked") + @Override + public void read(DataInput in) throws IOException { + + super.read(in); + + final int numInputChannels = in.readInt(); + + for (int i = 0; i < numInputChannels; i++) { + + final ChannelID channelID = new ChannelID(); + channelID.read(in); + final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); + + final String className = StringRecord.readString(in); + Class c = null; + try { + c = ClassUtils.getRecordByName(className); + } catch (ClassNotFoundException e) { + LOG.error(e); + } + + if (c == null) { + throw new IOException("Class is null!"); + } + + AbstractInputChannel eic = null; + try { + final Constructor> constructor = (Constructor>) c + .getDeclaredConstructor(this.getClass(), int.class, RecordDeserializer.class, ChannelID.class, + CompressionLevel.class); + if (constructor == null) { + throw new IOException("Constructor is null!"); + } + constructor.setAccessible(true); + eic = constructor.newInstance(this, i, deserializer, channelID, compressionLevel); + } catch (SecurityException e) { + LOG.error(e); + } catch (NoSuchMethodException e) { + LOG.error(e); + } catch (IllegalArgumentException e) { + LOG.error(e); + } catch (InstantiationException e) { + LOG.error(e); + } catch (IllegalAccessException e) { + LOG.error(e); + } catch (InvocationTargetException e) { + LOG.error(e); + } + if (eic == null) { + throw new IOException("Created input channel is null!"); + } + + eic.read(in); + addInputChannel(eic); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void write(DataOutput out) throws IOException { + + super.write(out); + + // Connected input channels + out.writeInt(this.getNumberOfInputChannels()); + for (int i = 0; i < getNumberOfInputChannels(); i++) { + getInputChannel(i).getID().write(out); + EnumUtils.writeEnum(out, getInputChannel(i).getCompressionLevel()); + StringRecord.writeString(out, getInputChannel(i).getClass().getName()); + getInputChannel(i).write(out); + } + + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isClosed() throws IOException, InterruptedException { + + for (int i = 0; i < this.getNumberOfInputChannels(); i++) { + final AbstractInputChannel inputChannel = this.inputChannels.get(i); + if (!inputChannel.isClosed()) { + return false; + } + } + + return true; + } + + /** + * Immediately closes the input gate and all its input channels. The corresponding + * output channels are notified. Any remaining records in any buffers or queue is considered + * irrelevant and is discarded. + * + * @throws IOException + * thrown if an I/O error occurs while closing the gate + * @throws InterruptedException + * thrown if the thread is interrupted while waiting for the gate to be closed + */ + public void close() throws IOException, InterruptedException { + + for (int i = 0; i < this.getNumberOfInputChannels(); i++) { + final AbstractInputChannel inputChannel = this.inputChannels.get(i); + inputChannel.close(); + } + + } + + /** + * Returns the list of InputChannels that feed this RecordReader + * + * @return the list of InputChannels that feed this RecordReader + */ + public List> getInputChannels() { + return inputChannels; + } + + /** + * Registers a new listener object for this input gate. + * + * @param inputGateListener + * the listener object to register + */ + public void registerInputGateListener(InputGateListener inputGateListener) { + + if (this.inputGateListeners == null) { + this.inputGateListeners = new InputGateListener[1]; + this.inputGateListeners[0] = inputGateListener; + } else { + InputGateListener[] tmp = this.inputGateListeners; + this.inputGateListeners = new InputGateListener[tmp.length + 1]; + System.arraycopy(tmp, 0, this.inputGateListeners, 0, tmp.length); + this.inputGateListeners[tmp.length] = inputGateListener; + } + } + + /** + * {@inheritDoc} + */ + @Override + public String toString() { + return "Input " + super.toString(); + } + + /** + * {@inheritDoc} + */ + @Override + public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException { + + // Copy event to all connected channels + final Iterator> it = this.inputChannels.iterator(); + while (it.hasNext()) { + it.next().transferEvent(event); + } + } + + /** + * Returns the {@link RecordDeserializer} used by this input gate. + * + * @return the {@link RecordDeserializer} used by this input gate + */ + public RecordDeserializer getRecordDeserializer() { + + return this.deserializer; + } + + /** + * {@inheritDoc} + */ + @Override + public void releaseAllChannelResources() { + + final Iterator> it = this.inputChannels.iterator(); + while (it.hasNext()) { + it.next().releaseResources(); + } + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java new file mode 100644 index 0000000000000..14ef3da11ceb5 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java @@ -0,0 +1,589 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.event.task.AbstractTaskEvent; +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.channels.bytebuffered.FileOutputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkOutputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryOutputChannel; +import eu.stratosphere.nephele.io.compression.CompressionLevel; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.types.Record; +import eu.stratosphere.nephele.types.StringRecord; +import eu.stratosphere.nephele.util.ClassUtils; +import eu.stratosphere.nephele.util.EnumUtils; + +/** + * In Nephele output gates are a specialization of general gates and connect + * record writers and output channels. As channels, output gates are always + * parameterized to a specific type of record which they can transport. + *

+ * This class is in general not thread-safe. + * + * @author warneke + * @param + * the type of record that can be transported through this gate + */ +public class RuntimeOutputGate extends AbstractGate implements OutputGate { + + /** + * The log object used for debugging. + */ + private static final Log LOG = LogFactory.getLog(OutputGate.class); + + /** + * The list of output channels attached to this gate. + */ + private final ArrayList> outputChannels = new ArrayList>(); + + /** + * Channel selector to determine which channel is supposed receive the next record. + */ + private final ChannelSelector channelSelector; + + /** + * The class of the record transported through this output gate. + */ + private final Class type; + + /** + * The listener objects registered for this output gate. + */ + private OutputGateListener[] outputGateListeners = null; + + /** + * The thread which executes the task connected to the output gate. + */ + private Thread executingThread = null; + + /** + * Stores whether all records passed to this output gate shall be transmitted through all connected output channels. + */ + private final boolean isBroadcast; + + /** + * Constructs a new runtime output gate. + * + * @param jobID + * the ID of the job this input gate belongs to + * @param gateID + * the ID of the gate + * @param inputClass + * the class of the record that can be transported through this + * gate + * @param index + * the index assigned to this output gate at the {@link Environment} object + * @param channelSelector + * the channel selector to be used for this output gate + * @param isBroadcast + * true if every records passed to this output gate shall be transmitted through all connected + * output channels, false otherwise + */ + public RuntimeOutputGate(final JobID jobID, final GateID gateID, final Class inputClass, final int index, + final ChannelSelector channelSelector, final boolean isBroadcast) { + + super(jobID, gateID, index); + + this.isBroadcast = isBroadcast; + this.type = inputClass; + + if (this.isBroadcast) { + this.channelSelector = null; + } else { + if (channelSelector == null) { + this.channelSelector = new DefaultChannelSelector(); + } else { + this.channelSelector = channelSelector; + } + } + } + + /** + * {@inheritDoc} + */ + @Override + public final Class getType() { + return this.type; + } + + /** + * Adds a new output channel to the output gate. + * + * @param outputChannel + * the output channel to be added. + */ + private void addOutputChannel(AbstractOutputChannel outputChannel) { + if (!this.outputChannels.contains(outputChannel)) { + this.outputChannels.add(outputChannel); + } + } + + /** + * Removes the output channel with the given ID from the output gate if it + * exists. + * + * @param outputChannelID + * the ID of the channel to be removed + */ + public void removeOutputChannel(ChannelID outputChannelID) { + + for (int i = 0; i < this.outputChannels.size(); i++) { + + final AbstractOutputChannel outputChannel = this.outputChannels.get(i); + if (outputChannel.getID().equals(outputChannelID)) { + this.outputChannels.remove(i); + return; + } + } + + LOG.debug("Cannot find output channel with ID " + outputChannelID + " to remove"); + } + + /** + * Removes all output channels from the output gate. + */ + public void removeAllOutputChannels() { + + this.outputChannels.clear(); + } + + public AbstractOutputChannel replaceChannel(ChannelID oldChannelID, ChannelType newChannelType, + boolean followsPushModel) { + + AbstractOutputChannel oldOutputChannel = null; + + for (int i = 0; i < this.outputChannels.size(); i++) { + final AbstractOutputChannel outputChannel = this.outputChannels.get(i); + if (outputChannel.getID().equals(oldChannelID)) { + oldOutputChannel = outputChannel; + break; + } + } + + if (oldOutputChannel == null) { + return null; + } + + AbstractOutputChannel newOutputChannel = null; + + switch (newChannelType) { + case FILE: + newOutputChannel = new FileOutputChannel(this, oldOutputChannel.getChannelIndex(), oldOutputChannel + .getID(), oldOutputChannel.getCompressionLevel()); + break; + case INMEMORY: + newOutputChannel = new InMemoryOutputChannel(this, oldOutputChannel.getChannelIndex(), oldOutputChannel + .getID(), oldOutputChannel.getCompressionLevel()); + break; + case NETWORK: + newOutputChannel = new NetworkOutputChannel(this, oldOutputChannel.getChannelIndex(), oldOutputChannel + .getID(), oldOutputChannel.getCompressionLevel()); + break; + default: + return null; + } + + newOutputChannel.setConnectedChannelID(oldOutputChannel.getConnectedChannelID()); + + this.outputChannels.set(newOutputChannel.getChannelIndex(), newOutputChannel); + + return newOutputChannel; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isInputGate() { + + return false; + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfOutputChannels() { + + return this.outputChannels.size(); + } + + /** + * Returns the output channel from position pos of the gate's + * internal channel list. + * + * @param pos + * the position to retrieve the channel from + * @return the channel from the given position or null if such + * position does not exist. + */ + public AbstractOutputChannel getOutputChannel(int pos) { + + if (pos < this.outputChannels.size()) + return this.outputChannels.get(pos); + else + return null; + } + + /** + * Creates a new network output channel and assigns it to the output gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new network output channel + */ + public NetworkOutputChannel createNetworkOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + + final NetworkOutputChannel enoc = new NetworkOutputChannel(this, this.outputChannels.size(), channelID, + compressionLevel); + addOutputChannel(enoc); + + return enoc; + } + + /** + * Creates a new file output channel and assigns it to the output gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new file output channel + */ + public FileOutputChannel createFileOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + + final FileOutputChannel efoc = new FileOutputChannel(this, this.outputChannels.size(), channelID, + compressionLevel); + addOutputChannel(efoc); + + return efoc; + } + + /** + * Creates a new in-memory output channel and assigns it to the output gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new in-memory output channel + */ + public InMemoryOutputChannel createInMemoryOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + + final InMemoryOutputChannel einoc = new InMemoryOutputChannel(this, this.outputChannels.size(), + channelID, compressionLevel); + addOutputChannel(einoc); + + return einoc; + } + + /** + * Requests the output gate to closed. This means the application will send + * no records through this gate anymore. + * + * @throws IOException + * @throws InterruptedException + */ + public void requestClose() throws IOException, InterruptedException { + // Close all output channels + for (int i = 0; i < this.getNumberOfOutputChannels(); i++) { + final AbstractOutputChannel outputChannel = this.getOutputChannel(i); + outputChannel.requestClose(); + } + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isClosed() throws IOException, InterruptedException { + + boolean allClosed = true; + + for (int i = 0; i < this.getNumberOfOutputChannels(); i++) { + final AbstractOutputChannel outputChannel = this.getOutputChannel(i); + if (!outputChannel.isClosed()) { + allClosed = false; + } + } + + return allClosed; + } + + /** + * {@inheritDoc} + */ + @Override + public void writeRecord(final T record) throws IOException, InterruptedException { + + if (this.executingThread == null) { + this.executingThread = Thread.currentThread(); + } + + if (this.executingThread.isInterrupted()) { + throw new InterruptedException(); + } + + if (this.outputGateListeners != null) { + for (final OutputGateListener outputGateListener : this.outputGateListeners) { + outputGateListener.recordEmitted(record); + } + } + + if (this.isBroadcast) { + + if (getChannelType() == ChannelType.INMEMORY) { + + final int numberOfOutputChannels = this.outputChannels.size(); + for (int i = 0; i < numberOfOutputChannels; ++i) { + this.outputChannels.get(i).writeRecord(record); + } + + } else { + + // Use optimization for byte buffered channels + this.outputChannels.get(0).writeRecord(record); + } + + } else { + + // Non-broadcast gate, use channel selector to select output channels + final int numberOfOutputChannels = this.outputChannels.size(); + final int[] selectedOutputChannels = this.channelSelector.selectChannels(record, numberOfOutputChannels); + + if (selectedOutputChannels == null) { + return; + } + + for (int i = 0; i < selectedOutputChannels.length; ++i) { + + if (selectedOutputChannels[i] < numberOfOutputChannels) { + final AbstractOutputChannel outputChannel = this.outputChannels.get(selectedOutputChannels[i]); + outputChannel.writeRecord(record); + } + } + } + } + + // TODO: See if type safety can be improved here + /** + * {@inheritDoc} + */ + @SuppressWarnings("unchecked") + public void read(DataInput in) throws IOException { + + super.read(in); + + final int numOutputChannels = in.readInt(); + + final Class[] parameters = { this.getClass(), int.class, ChannelID.class, CompressionLevel.class }; + + for (int i = 0; i < numOutputChannels; i++) { + + final ChannelID channelID = new ChannelID(); + channelID.read(in); + final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); + final String className = StringRecord.readString(in); + Class c = null; + try { + c = ClassUtils.getRecordByName(className); + } catch (ClassNotFoundException e) { + LOG.error(e); + } + + if (c == null) { + throw new IOException("Class is null!"); + } + + AbstractOutputChannel eoc = null; + try { + final Constructor> constructor = (Constructor>) c + .getDeclaredConstructor(parameters); + + if (constructor == null) { + throw new IOException("Constructor is null!"); + } + + constructor.setAccessible(true); + + eoc = constructor.newInstance(this, i, channelID, compressionLevel); + + } catch (InstantiationException e) { + LOG.error(e); + } catch (IllegalArgumentException e) { + LOG.error(e); + } catch (IllegalAccessException e) { + LOG.error(e); + } catch (InvocationTargetException e) { + LOG.error(e); + } catch (SecurityException e) { + LOG.error(e); + } catch (NoSuchMethodException e) { + LOG.error(e); + } + + if (eoc == null) { + throw new IOException("Created output channel is null!"); + } + + eoc.read(in); + addOutputChannel(eoc); + } + } + + /** + * {@inheritDoc} + */ + public void write(DataOutput out) throws IOException { + + super.write(out); + + // Output channels + out.writeInt(this.getNumberOfOutputChannels()); + + for (int i = 0; i < getNumberOfOutputChannels(); i++) { + getOutputChannel(i).getID().write(out); + EnumUtils.writeEnum(out, getOutputChannel(i).getCompressionLevel()); + StringRecord.writeString(out, getOutputChannel(i).getClass().getName()); + getOutputChannel(i).write(out); + } + + } + + /** + * {@inheritDoc} + */ + @Override + public List> getOutputChannels() { + return this.outputChannels; + } + + /** + * Registers a new listener object for this output gate. + * + * @param outputGateListener + * the listener object to register + */ + public void registerOutputGateListener(OutputGateListener outputGateListener) { + + if (this.outputGateListeners == null) { + this.outputGateListeners = new OutputGateListener[1]; + this.outputGateListeners[0] = outputGateListener; + } else { + OutputGateListener[] tmp = this.outputGateListeners; + this.outputGateListeners = new OutputGateListener[tmp.length + 1]; + System.arraycopy(tmp, 0, this.outputGateListeners, 0, tmp.length); + this.outputGateListeners[tmp.length] = outputGateListener; + } + } + + /** + * {@inheritDoc} + */ + @Override + public String toString() { + return "Output " + super.toString(); + } + + /** + * {@inheritDoc} + */ + @Override + public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException { + + // Copy event to all connected channels + final Iterator> it = this.outputChannels.iterator(); + while (it.hasNext()) { + it.next().transferEvent(event); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void flush() throws IOException, InterruptedException { + // Flush all connected channels + final Iterator> it = this.outputChannels.iterator(); + while (it.hasNext()) { + it.next().flush(); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void channelCapacityExhausted(final int channelIndex) { + + // notify the listener objects + if (this.outputGateListeners != null) { + for (int i = 0; i < this.outputGateListeners.length; ++i) { + this.outputGateListeners[i].channelCapacityExhausted(channelIndex); + } + } + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isBroadcast() { + + return this.isBroadcast; + } + + /** + * Returns the output gate's channel selector. + * + * @return the output gate's channel selector or null if the gate operates in broadcast mode + */ + public ChannelSelector getChannelSelector() { + + return this.channelSelector; + } + + /** + * {@inheritDoc} + */ + @Override + public void releaseAllChannelResources() { + + final Iterator> it = this.outputChannels.iterator(); + + while (it.hasNext()) { + it.next().releaseResources(); + } + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java index 54b857801ad0d..00fccb98aeeaa 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java @@ -18,10 +18,10 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.InputGateListener; -import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.OutputGateListener; +import eu.stratosphere.nephele.io.RuntimeInputGate; import eu.stratosphere.nephele.taskmanager.Task; import eu.stratosphere.nephele.types.Record; @@ -54,7 +54,7 @@ public interface TaskManagerProfiler { * the input gate to register a {@link InputGateListener} object for */ void registerInputGateListener(ExecutionVertexID id, Configuration jobConfiguration, - InputGate inputGate); + RuntimeInputGate inputGate); /** * Registers a {@link OutputGateListener} object for the given output gate. @@ -67,7 +67,7 @@ void registerInputGateListener(ExecutionVertexID id, Configuration jobConfigurat * the output gate to register a {@link InputGateListener} object for */ void registerOutputGateListener(ExecutionVertexID id, Configuration jobConfiguration, - OutputGate outputGate); + RuntimeOutputGate outputGate); /** * Unregisters all previously register {@link ExecutionListener} objects for diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index 596b6ab7a8598..626414c9499ae 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -33,7 +33,7 @@ import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.jobgraph.JobID; @@ -244,7 +244,7 @@ public void initialExecutionResourcesExhausted() { final Map outputChannelUtilization = new HashMap(); for (int i = 0; i < this.environment.getNumberOfOutputGates(); ++i) { - final OutputGate outputGate = this.environment.getOutputGate(i); + final RuntimeOutputGate outputGate = this.environment.getOutputGate(i); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); outputChannelUtilization.put(outputChannel.getID(), From a525864411bc578940ed9f0bb258001787e6c708 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 19 Nov 2011 18:10:10 +0000 Subject: [PATCH 058/310] Changed calculation of path latencies --- .../nephele/streaming/StreamingTaskListener.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java index f2b5855ef7627..2690ae155f502 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java @@ -119,7 +119,12 @@ public void recordEmitted(final Record record) { break; case REGULAR: final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - taggableRecord.setTag(this.tag); + if(this.tag == null) { + taggableRecord.setTag(null); + } else { + this.tag = createTag(); + taggableRecord.setTag(this.tag); + } break; case OUTPUT: throw new IllegalStateException("Output task emitted record"); @@ -191,10 +196,7 @@ public void recordReceived(final Record record) { private StreamingTag createTag() { - if (this.tag == null) { - this.tag = new StreamingTag(this.vertexID); - } - + this.tag = new StreamingTag(this.vertexID); this.tag.setTimestamp(System.currentTimeMillis()); return this.tag; From b105defcdbcfcd5161cb3e59f7f3a1191fee80f2 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 19 Nov 2011 18:47:17 +0000 Subject: [PATCH 059/310] Finished refactoring of input and output gates --- .../stratosphere/nephele/io/OutputGate.java | 7 ++++ .../nephele/execution/RuntimeEnvironment.java | 2 +- .../executiongraph/ExecutionGraph.java | 27 +++++++------- .../ManagementGraphFactory.java | 9 +++-- .../nephele/io/RuntimeInputGate.java | 4 +-- .../nephele/io/RuntimeOutputGate.java | 7 ++-- .../scheduler/AbstractScheduler.java | 4 +-- .../ByteBufferedChannelManager.java | 12 +++---- .../streaming/StreamingJobManagerPlugin.java | 4 +-- .../streaming/StreamingTaskManagerPlugin.java | 8 ++--- .../runtime/test/util/MockEnvironment.java | 35 ++++++++++--------- 11 files changed, 62 insertions(+), 57 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java index 7add6680f0cec..20d0c08a7dc79 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java @@ -91,4 +91,11 @@ public interface OutputGate extends Gate { * @return the number of output channels associated with this output gate */ int getNumberOfOutputChannels(); + + /** + * Returns the output gate's channel selector. + * + * @return the output gate's channel selector or null if the gate operates in broadcast mode + */ + ChannelSelector getChannelSelector(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 3336bf552d9ad..068477d535ff4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -438,7 +438,7 @@ public int getNumberOfInputGates() { * the index of the input gate to return * @return the input gate at index pos or null if no such index exists */ - public InputGate getInputGate(final int pos) { + public RuntimeInputGate getInputGate(final int pos) { if (pos < this.inputGates.size()) { return this.inputGates.get(pos); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index 19578d706573d..171cc1f3a0039 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -37,8 +37,7 @@ import eu.stratosphere.nephele.instance.DummyInstance; import eu.stratosphere.nephele.instance.InstanceManager; import eu.stratosphere.nephele.instance.InstanceType; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeInputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; @@ -326,7 +325,7 @@ private void createInitialChannels(final AbstractJobVertex jobVertex, final AbstractJobVertex target = edge.getConnectedVertex(); // find output gate of execution vertex - final OutputGate eog = ev.getEnvironment().getOutputGate(j); + final RuntimeOutputGate eog = ev.getEnvironment().getOutputGate(j); if (eog == null) { throw new GraphConversionException("Cannot retrieve output gate " + j + " from vertex " + jobVertex.getName()); @@ -337,7 +336,7 @@ private void createInitialChannels(final AbstractJobVertex jobVertex, throw new GraphConversionException("Cannot find mapping for vertex " + target.getName()); } - final InputGate eig = executionTarget.getEnvironment().getInputGate( + final RuntimeInputGate eig = executionTarget.getEnvironment().getInputGate( edge.getIndexOfInputGate()); if (eig == null) { throw new GraphConversionException("Cannot retrieve input gate " + edge.getIndexOfInputGate() @@ -412,7 +411,8 @@ void unwire(final ExecutionGroupVertex source, final int indexOfOutputGate, fina for (int i = 0; i < target.getCurrentNumberOfGroupMembers(); i++) { final ExecutionVertex targetVertex = target.getGroupMember(i); - final RuntimeInputGate inputGate = targetVertex.getEnvironment().getInputGate(indexOfInputGate); + final RuntimeInputGate inputGate = targetVertex.getEnvironment().getInputGate( + indexOfInputGate); if (inputGate == null) { throw new GraphConversionException("unwire: " + targetVertex.getName() + " has no input gate with index " + indexOfInputGate); @@ -436,7 +436,7 @@ void wire(final ExecutionGroupVertex source, final int indexOfOutputGate, final for (int i = 0; i < source.getCurrentNumberOfGroupMembers(); i++) { final ExecutionVertex sourceVertex = source.getGroupMember(i); - final OutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( + final RuntimeOutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( indexOfOutputGate); if (outputGate == null) { throw new GraphConversionException("wire: " + sourceVertex.getName() @@ -450,7 +450,7 @@ void wire(final ExecutionGroupVertex source, final int indexOfOutputGate, final for (int j = 0; j < target.getCurrentNumberOfGroupMembers(); j++) { final ExecutionVertex targetVertex = target.getGroupMember(j); - final InputGate inputGate = targetVertex.getEnvironment().getInputGate( + final RuntimeInputGate inputGate = targetVertex.getEnvironment().getInputGate( indexOfInputGate); if (inputGate == null) { throw new GraphConversionException("wire: " + targetVertex.getName() @@ -477,10 +477,9 @@ void wire(final ExecutionGroupVertex source, final int indexOfOutputGate, final } - private void createChannel(final ExecutionVertex source, final OutputGate outputGate, - final ExecutionVertex target, final InputGate inputGate, final ChannelType channelType, - final CompressionLevel compressionLevel) - throws GraphConversionException { + private void createChannel(final ExecutionVertex source, final RuntimeOutputGate outputGate, + final ExecutionVertex target, final RuntimeInputGate inputGate, + final ChannelType channelType, final CompressionLevel compressionLevel) throws GraphConversionException { AbstractOutputChannel outputChannel; AbstractInputChannel inputChannel; @@ -1072,7 +1071,7 @@ public void repairInstanceAssignment() { for (int i = 0; i < sourceVertex.getEnvironment().getNumberOfOutputGates(); i++) { - final OutputGate outputGate = sourceVertex.getEnvironment().getOutputGate(i); + final RuntimeOutputGate outputGate = sourceVertex.getEnvironment().getOutputGate(i); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); j++) { final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); final ChannelType channelType = outputChannel.getType(); @@ -1092,7 +1091,7 @@ public void repairInstanceAssignment() { for (int i = 0; i < targetVertex.getEnvironment().getNumberOfInputGates(); i++) { - final InputGate inputGate = targetVertex.getEnvironment().getInputGate(i); + final RuntimeInputGate inputGate = targetVertex.getEnvironment().getInputGate(i); for (int j = 0; j < inputGate.getNumberOfInputChannels(); j++) { final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); final ChannelType channelType = inputChannel.getType(); @@ -1120,7 +1119,7 @@ public ChannelType getChannelType(final ExecutionVertex sourceVertex, final Exec final ExecutionGroupEdge edge = edges.get(0); // Now lets see if these two concrete subtasks are connected - final OutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( + final RuntimeOutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( edge.getIndexOfOutputGate()); for (int i = 0; i < outputGate.getNumberOfOutputChannels(); i++) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java index d190c5b7c7747..a995103a2b7c5 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java @@ -20,8 +20,7 @@ import java.util.Map; import eu.stratosphere.nephele.io.AbstractGate; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeInputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; @@ -142,7 +141,7 @@ private static void addExecutionVertices(Map inputGate = ev.getEnvironment().getInputGate(i); + final RuntimeInputGate inputGate = ev.getEnvironment().getInputGate(i); final ManagementGate managementGate = new ManagementGate(managementVertex, i, true, ""); gateMap.put(inputGate, managementGate); } @@ -157,8 +156,8 @@ private static void addExecutionVertices(Map outputGate = (OutputGate) source.getEnvironment() - .getOutputGate(i); + final RuntimeOutputGate outputGate = (RuntimeOutputGate) source + .getEnvironment().getOutputGate(i); final ManagementGate manangementOutputGate = gateMap.get(outputGate); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); j++) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java index 9495615a72a7c..b46e87fccf905 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java @@ -58,7 +58,7 @@ * @param * the type of record that can be transported through this gate */ -public final class RuntimeInputGate extends AbstractGate implements InputGate, +public class RuntimeInputGate extends AbstractGate implements InputGate, IOReadableWritable { /** @@ -424,7 +424,7 @@ public void read(DataInput in) throws IOException { AbstractInputChannel eic = null; try { final Constructor> constructor = (Constructor>) c - .getDeclaredConstructor(this.getClass(), int.class, RecordDeserializer.class, ChannelID.class, + .getDeclaredConstructor(InputGate.class, int.class, RecordDeserializer.class, ChannelID.class, CompressionLevel.class); if (constructor == null) { throw new IOException("Constructor is null!"); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java index 14ef3da11ceb5..d776750213aeb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java @@ -407,7 +407,7 @@ public void read(DataInput in) throws IOException { final int numOutputChannels = in.readInt(); - final Class[] parameters = { this.getClass(), int.class, ChannelID.class, CompressionLevel.class }; + final Class[] parameters = { OutputGate.class, int.class, ChannelID.class, CompressionLevel.class }; for (int i = 0; i < numOutputChannels; i++) { @@ -565,10 +565,9 @@ public boolean isBroadcast() { } /** - * Returns the output gate's channel selector. - * - * @return the output gate's channel selector or null if the gate operates in broadcast mode + * {@inheritDoc} */ + @Override public ChannelSelector getChannelSelector() { return this.channelSelector; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index 34568d8f89d93..91b1686f84c8d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -43,7 +43,7 @@ import eu.stratosphere.nephele.instance.InstanceManager; import eu.stratosphere.nephele.instance.InstanceRequestMap; import eu.stratosphere.nephele.instance.InstanceType; -import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobmanager.DeploymentManager; @@ -218,7 +218,7 @@ void findVerticesToBeDeployed(final ExecutionVertex vertex, final int numberOfOutputGates = env.getNumberOfOutputGates(); for (int i = 0; i < numberOfOutputGates; ++i) { - final OutputGate outputGate = env.getOutputGate(i); + final RuntimeOutputGate outputGate = env.getOutputGate(i); boolean deployTarget; switch (outputGate.getChannelType()) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 398282c73474f..10974a54db980 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -32,8 +32,8 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; import eu.stratosphere.nephele.io.AbstractID; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeInputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractChannel; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; @@ -139,7 +139,7 @@ public void register(final Task task, final Set activeOutputChannels) final TaskContext taskContext = new TaskContext(task, this, this.tasksWithUndecidedCheckpoints); for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - final OutputGate outputGate = environment.getOutputGate(i); + final RuntimeOutputGate outputGate = environment.getOutputGate(i); final OutputGateContext outputGateContext = new OutputGateContext(taskContext, outputGate.getChannelType(), outputGate.getIndex()); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { @@ -174,7 +174,7 @@ public void register(final Task task, final Set activeOutputChannels) } for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { - final InputGate inputGate = environment.getInputGate(i); + final RuntimeInputGate inputGate = environment.getInputGate(i); final InputGateContext inputGateContext = new InputGateContext(inputGate.getNumberOfInputChannels()); for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); @@ -225,7 +225,7 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { final RuntimeEnvironment environment = task.getEnvironment(); for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - final OutputGate outputGate = environment.getOutputGate(i); + final RuntimeOutputGate outputGate = environment.getOutputGate(i); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); this.registeredChannels.remove(outputChannel.getID()); @@ -234,7 +234,7 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { } for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { - final InputGate inputGate = environment.getInputGate(i); + final RuntimeInputGate inputGate = environment.getInputGate(i); for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); this.registeredChannels.remove(inputChannel.getID()); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 00471c168fd81..e1f9841ebe91b 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -50,7 +50,7 @@ public class StreamingJobManagerPlugin implements JobManagerPlugin { public JobGraph rewriteJobGraph(final JobGraph jobGraph) { // Rewrite input vertices - final Iterator inputIt = jobGraph.getInputVertices(); + /*final Iterator inputIt = jobGraph.getInputVertices(); final Iterator taskIt = jobGraph.getTaskVertices(); while (taskIt.hasNext()) { @@ -61,7 +61,7 @@ public JobGraph rewriteJobGraph(final JobGraph jobGraph) { taskVertex.setTaskClass(StreamingTask.class); taskVertex.getConfiguration().setString("origClass", originalClass.getName()); - } + }*/ // TODO Auto-generated method stub return null; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index 4e483a901ece2..1814fceb0960f 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -21,8 +21,8 @@ import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.IOReadableWritable; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeInputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.plugins.PluginCommunication; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; @@ -114,12 +114,12 @@ public void registerTask(final ExecutionVertexID id, final Configuration jobConf } for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - final OutputGate outputGate = environment.getOutputGate(i); + final RuntimeOutputGate outputGate = environment.getOutputGate(i); outputGate.registerOutputGateListener(listener); } for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { - final InputGate inputGate = environment.getInputGate(i); + final RuntimeInputGate inputGate = environment.getInputGate(i); inputGate.registerInputGateListener(listener); } } diff --git a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java index da0d77d4f8703..d8308e6c0c0bb 100644 --- a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java +++ b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java @@ -24,7 +24,8 @@ import eu.stratosphere.nephele.io.DefaultRecordDeserializer; import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RuntimeInputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; @@ -33,7 +34,6 @@ import eu.stratosphere.pact.common.type.PactRecord; import eu.stratosphere.pact.common.util.MutableObjectIterator; - public class MockEnvironment extends RuntimeEnvironment { private MemoryManager memManager; @@ -42,16 +42,17 @@ public class MockEnvironment extends RuntimeEnvironment private Configuration config; - private List> inputs; - private List> outputs; - + private List> inputs; + + private List> outputs; + private final JobID jobId = new JobID(); public MockEnvironment(long memorySize) { this.config = new Configuration(); - this.inputs = new ArrayList>(); - this.outputs = new ArrayList>(); - + this.inputs = new ArrayList>(); + this.outputs = new ArrayList>(); + this.memManager = new DefaultMemoryManager(memorySize); this.ioManager = new IOManager(System.getProperty("java.io.tmpdir")); } @@ -60,7 +61,7 @@ public void addInput(MutableObjectIterator inputIterator) { int id = inputs.size(); inputs.add(new MockInputGate(id, inputIterator)); } - + public void addOutput(List outputList) { int id = outputs.size(); outputs.add(new MockOutputGate(id, outputList)); @@ -100,12 +101,12 @@ public MemoryManager getMemoryManager() { public IOManager getIOManager() { return this.ioManager; } - + public JobID getJobID() { return this.jobId; } - private static class MockInputGate extends InputGate { + private static class MockInputGate extends RuntimeInputGate { private MutableObjectIterator it; @@ -116,7 +117,7 @@ public MockInputGate(int id, MutableObjectIterator it) { @Override public PactRecord readRecord(PactRecord target) throws IOException, InterruptedException { - + if (it.next(target)) { return target; } else { @@ -124,13 +125,13 @@ public PactRecord readRecord(PactRecord target) throws IOException, InterruptedE } } } - - private static class MockOutputGate extends OutputGate { - + + private static class MockOutputGate extends RuntimeOutputGate { + private List out; - + public MockOutputGate(int index, List outList) { - super(new JobID(), new GateID(), PactRecord.class, index, null ,false); + super(new JobID(), new GateID(), PactRecord.class, index, null, false); this.out = outList; } From d85b8f06a7fc66bf8e319c930f568c455a78255d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Sat, 19 Nov 2011 20:42:51 +0100 Subject: [PATCH 060/310] integration between latency model and jobmanager plugin (untested) --- .../streaming/AbstractStreamingData.java | 71 +++++++++++++++++++ .../nephele/streaming/PathLatency.java | 28 ++------ .../StreamingCommunicationThread.java | 4 +- .../nephele/streaming/StreamingData.java | 26 ------- .../streaming/StreamingJobManagerPlugin.java | 55 +++++++++++--- .../streaming/latency/LatencyModel.java | 27 +++++++ .../latency/LatencyOptimizerThread.java | 41 +++++++++++ .../streaming/latency/LatencyPath.java | 24 +++++++ 8 files changed, 217 insertions(+), 59 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/AbstractStreamingData.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingData.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/AbstractStreamingData.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/AbstractStreamingData.java new file mode 100644 index 0000000000000..a6ecaaf99f5f1 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/AbstractStreamingData.java @@ -0,0 +1,71 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.jobgraph.JobID; + +/** + * Abstract base class to be used to for exchanging data between the different + * components of the streaming plugin. + * + * @author warneke + */ +public abstract class AbstractStreamingData implements IOReadableWritable { + + /** + * The ID of the job this piece of streaming data refers to + */ + private final JobID jobID; + + public AbstractStreamingData(JobID jobID) { + if (jobID == null) { + throw new IllegalArgumentException("jobID must not be null"); + } + + this.jobID = jobID; + } + + /** + * Returns the ID of the job this path latency information refers to. + * + * @return the ID of the job this path latency information refers to + */ + public JobID getJobID() { + + return this.jobID; + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + this.jobID.write(out); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + this.jobID.read(in); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java index f82d824cb0bb1..f90a98987216b 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java @@ -27,12 +27,7 @@ * * @author warneke */ -public final class PathLatency implements StreamingData { - - /** - * The ID of the job this path latency information refers to - */ - private final JobID jobID; +public final class PathLatency extends AbstractStreamingData { /** * The ID of the vertex representing the start of the path. @@ -64,9 +59,7 @@ public final class PathLatency implements StreamingData { public PathLatency(final JobID jobID, final ExecutionVertexID startVertexID, final ExecutionVertexID endVertexID, final double pathLatency) { - if (jobID == null) { - throw new IllegalArgumentException("jobID must not be null"); - } + super(jobID); if (startVertexID == null) { throw new IllegalArgumentException("sourceID must not be null"); @@ -76,7 +69,6 @@ public PathLatency(final JobID jobID, final ExecutionVertexID startVertexID, fin throw new IllegalArgumentException("targetID must not be null"); } - this.jobID = jobID; this.startVertexID = startVertexID; this.endVertexID = endVertexID; this.pathLatency = pathLatency; @@ -86,7 +78,7 @@ public PathLatency(final JobID jobID, final ExecutionVertexID startVertexID, fin * Default constructor for the deserialization of the object. */ public PathLatency() { - this.jobID = new JobID(); + super(new JobID()); this.startVertexID = new ExecutionVertexID(); this.endVertexID = new ExecutionVertexID(); this.pathLatency = 0.0; @@ -97,8 +89,7 @@ public PathLatency() { */ @Override public void write(final DataOutput out) throws IOException { - - this.jobID.write(out); + super.write(out); this.startVertexID.write(out); this.endVertexID.write(out); out.writeDouble(this.pathLatency); @@ -109,22 +100,13 @@ public void write(final DataOutput out) throws IOException { */ @Override public void read(final DataInput in) throws IOException { - - this.jobID.read(in); + super.read(in); this.startVertexID.read(in); this.endVertexID.read(in); this.pathLatency = in.readDouble(); } - /** - * Returns the ID of the job this path latency information refers to. - * - * @return the ID of the job this path latency information refers to - */ - public JobID getJobID() { - return this.jobID; - } /** * Returns the ID of the vertex representing the start of the path. diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java index 6c8cdcc294e82..b6af5d0f2b58e 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java @@ -54,7 +54,7 @@ final class StreamingCommunicationThread extends Thread { /** * The blocking queue which is used to asynchronously exchange data with the job manager component of this plugin. */ - private final BlockingQueue dataQueue = new ArrayBlockingQueue(QUEUE_CAPACITY); + private final BlockingQueue dataQueue = new ArrayBlockingQueue(QUEUE_CAPACITY); /** * Stores whether the communication thread has been requested to stop. @@ -109,7 +109,7 @@ void stopCommunicationThread() { * @throws InterruptedException * thrown if the thread is interrupted while waiting for the communication thread to accept the data */ - void sendDataAsynchronously(final StreamingData data) throws InterruptedException { + void sendDataAsynchronously(final AbstractStreamingData data) throws InterruptedException { this.dataQueue.put(data); } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingData.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingData.java deleted file mode 100644 index 5eaec6ae04fc0..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingData.java +++ /dev/null @@ -1,26 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming; - -import eu.stratosphere.nephele.io.IOReadableWritable; - -/** - * This is a marker interface to identify data exchanged between the different components of the streaming plugin. - * - * @author warneke - */ -public interface StreamingData extends IOReadableWritable { -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index f602adede306f..ccaca6985dae1 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -16,17 +16,25 @@ package eu.stratosphere.nephele.streaming; import java.io.IOException; +import java.util.Iterator; +import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.executiongraph.InternalJobStatus; +import eu.stratosphere.nephele.executiongraph.JobStatusListener; import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.jobgraph.JobGraph; +import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.streaming.latency.LatencyOptimizerThread; -public class StreamingJobManagerPlugin implements JobManagerPlugin { +public class StreamingJobManagerPlugin implements JobManagerPlugin, JobStatusListener { + + private ConcurrentHashMap latencyOptimizerThreads = new ConcurrentHashMap(); /** * The log object. @@ -50,8 +58,11 @@ public JobGraph rewriteJobGraph(final JobGraph jobGraph) { */ @Override public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) { - // TODO Auto-generated method stub - return null; + JobID jobId = executionGraph.getJobID(); + LatencyOptimizerThread optimizerThread = new LatencyOptimizerThread(executionGraph); + latencyOptimizerThreads.put(jobId, optimizerThread); + optimizerThread.start(); + return executionGraph; } /** @@ -59,8 +70,18 @@ public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) */ @Override public void shutdown() { - // TODO Auto-generated method stub + shutdownLatencyOptimizerThreads(); + } + private void shutdownLatencyOptimizerThreads() { + Iterator iter = latencyOptimizerThreads.values().iterator(); + while (iter.hasNext()) { + LatencyOptimizerThread thread = iter.next(); + thread.interrupt(); + + // also removes jobID mappings from underlying map + iter.remove(); + } } /** @@ -69,12 +90,14 @@ public void shutdown() { @Override public void sendData(final IOReadableWritable data) throws IOException { - if (!(data instanceof StreamingData)) { + if (!(data instanceof AbstractStreamingData)) { LOG.error("Received unexpected data of type " + data); return; } - - System.out.println(data); + + AbstractStreamingData streamingData = (AbstractStreamingData) data; + LatencyOptimizerThread optimizerThread = latencyOptimizerThreads.get(streamingData.getJobID()); + optimizerThread.handOffStreamingData(streamingData); } /** @@ -83,11 +106,27 @@ public void sendData(final IOReadableWritable data) throws IOException { @Override public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { - if (!(data instanceof StreamingData)) { + if (!(data instanceof AbstractStreamingData)) { LOG.error("Received unexpected data of type " + data); return null; } return null; } + + @Override + public void jobStatusHasChanged(ExecutionGraph executionGraph, + InternalJobStatus newJobStatus, + String optionalMessage) { + + if (newJobStatus == InternalJobStatus.FAILED + || newJobStatus == InternalJobStatus.CANCELED + || newJobStatus == InternalJobStatus.FINISHED) { + + LatencyOptimizerThread optimizerThread = latencyOptimizerThreads.remove(executionGraph.getJobID()); + if (optimizerThread != null) { + optimizerThread.interrupt(); + } + } + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java new file mode 100644 index 0000000000000..69c8d041ba6d1 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java @@ -0,0 +1,27 @@ +package eu.stratosphere.nephele.streaming.latency; + +import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; +import eu.stratosphere.nephele.streaming.PathLatency; + +public class LatencyModel { + + private ExecutionGraph executionGraph; + + private LatencySubgraph latencySubgraph; + + public LatencyModel(ExecutionGraph executionGraph) { + this.executionGraph = executionGraph; + + // FIXME naive implementation until we can annotate the job + // subgraphStart and subgraphEnd should be derived from the annotations + ExecutionGroupVertex subgraphStart = this.executionGraph.getInputVertex(0).getGroupVertex(); + ExecutionGroupVertex subgraphEnd = this.executionGraph.getOutputVertex(0).getGroupVertex(); + + this.latencySubgraph = new LatencySubgraph(executionGraph, subgraphStart, subgraphEnd); + } + + public void refreshEdgeLatency(PathLatency pathLatency) { + + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java new file mode 100644 index 0000000000000..57658e1679fd0 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java @@ -0,0 +1,41 @@ +package eu.stratosphere.nephele.streaming.latency; + +import java.util.concurrent.LinkedBlockingQueue; + +import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.streaming.AbstractStreamingData; +import eu.stratosphere.nephele.streaming.PathLatency; + +public class LatencyOptimizerThread extends Thread { + + private LinkedBlockingQueue streamingDataQueue; + + private ExecutionGraph executionGraph; + + private LatencyModel latencyModel; + + public LatencyOptimizerThread(ExecutionGraph executionGraph) { + this.latencyModel = new LatencyModel(executionGraph); + this.streamingDataQueue = new LinkedBlockingQueue(); + } + + public void run() { + + try { + while (!interrupted()) { + AbstractStreamingData streamingData = streamingDataQueue.take(); + + if (streamingData instanceof PathLatency) { + latencyModel.refreshEdgeLatency((PathLatency) streamingData); + } + + } + + } catch (InterruptedException e) { + } + } + + public void handOffStreamingData(AbstractStreamingData data) { + streamingDataQueue.add(data); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java index 744f9a98ca73c..ffba8734a857e 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java @@ -61,6 +61,30 @@ public void removeLastVertex() { ingoingEdges.remove(removed); } + /** + * Returns whether we have latency values for all parts (vertices and edges) of this + * path. + * + * @return Whether we have latency values for all parts of this path + */ + public boolean isActive() { + // FIXME inefficient, naive implementation. This may need to be precomputed. + + for (ManagementVertex vertex : pathVertices) { + if (((VertexLatency) vertex.getAttachment()).getLatencyInMillis() == -1) { + return false; + } + } + + for (ManagementEdge edge : ingoingEdges.values()) { + if (((EdgeLatency) edge.getAttachment()).getLatencyInMillis() == -1) { + return false; + } + } + + return true; + } + @Override public Iterator iterator() { return pathVertices.iterator(); From bcdd97dfbb8df67f7a4e5a69d0a3a096075ba0ba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Sat, 19 Nov 2011 22:59:39 +0100 Subject: [PATCH 061/310] * fixed some bugs (endless recursion in latency path construction, some NPEs) * added some toString() functions --- .../ManagementGroupVertex.java | 5 ++++ .../managementgraph/ManagementVertex.java | 5 ++++ .../streaming/StreamingJobManagerPlugin.java | 10 ++++---- .../streaming/latency/LatencyModel.java | 7 +++++- .../latency/LatencyOptimizerThread.java | 23 +++++++++++++------ .../streaming/latency/LatencyPath.java | 19 ++++++++++++++- .../streaming/latency/LatencySubgraph.java | 9 ++++++-- 7 files changed, 62 insertions(+), 16 deletions(-) diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java index da68ea6e2588a..ba12247dbcc01 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java @@ -415,4 +415,9 @@ public List getPredecessors() { return predecessors; } + + @Override + public String toString() { + return String.format("ManagementGroupVertex(%s)", getName()); + } } diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java index f8e238be83697..5805cc6c146e6 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java @@ -370,4 +370,9 @@ public void write(final DataOutput out) throws IOException { StringRecord.writeString(out, this.instanceType); StringRecord.writeString(out, this.checkpointState); } + + @Override + public String toString() { + return String.format("ManagementVertex(%s_%d)", getGroupVertex().getName(), indexInGroup); + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index ccaca6985dae1..4900bacb72419 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -34,13 +34,13 @@ public class StreamingJobManagerPlugin implements JobManagerPlugin, JobStatusListener { - private ConcurrentHashMap latencyOptimizerThreads = new ConcurrentHashMap(); - /** * The log object. */ private static final Log LOG = LogFactory.getLog(StreamingJobManagerPlugin.class); + private ConcurrentHashMap latencyOptimizerThreads = new ConcurrentHashMap(); + StreamingJobManagerPlugin(final Configuration pluginConfiguration) { } @@ -78,7 +78,7 @@ private void shutdownLatencyOptimizerThreads() { while (iter.hasNext()) { LatencyOptimizerThread thread = iter.next(); thread.interrupt(); - + // also removes jobID mappings from underlying map iter.remove(); } @@ -94,7 +94,7 @@ public void sendData(final IOReadableWritable data) throws IOException { LOG.error("Received unexpected data of type " + data); return; } - + AbstractStreamingData streamingData = (AbstractStreamingData) data; LatencyOptimizerThread optimizerThread = latencyOptimizerThreads.get(streamingData.getJobID()); optimizerThread.handOffStreamingData(streamingData); @@ -118,7 +118,7 @@ public IOReadableWritable requestData(final IOReadableWritable data) throws IOEx public void jobStatusHasChanged(ExecutionGraph executionGraph, InternalJobStatus newJobStatus, String optionalMessage) { - + if (newJobStatus == InternalJobStatus.FAILED || newJobStatus == InternalJobStatus.CANCELED || newJobStatus == InternalJobStatus.FINISHED) { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java index 69c8d041ba6d1..548fceb264d66 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java @@ -1,11 +1,16 @@ package eu.stratosphere.nephele.streaming.latency; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; import eu.stratosphere.nephele.streaming.PathLatency; public class LatencyModel { + private static Log LOG = LogFactory.getLog(LatencyModel.class); + private ExecutionGraph executionGraph; private LatencySubgraph latencySubgraph; @@ -22,6 +27,6 @@ public LatencyModel(ExecutionGraph executionGraph) { } public void refreshEdgeLatency(PathLatency pathLatency) { - + LOG.info("Received " + pathLatency); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java index 57658e1679fd0..cebca038a5063 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java @@ -2,29 +2,36 @@ import java.util.concurrent.LinkedBlockingQueue; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.streaming.AbstractStreamingData; import eu.stratosphere.nephele.streaming.PathLatency; public class LatencyOptimizerThread extends Thread { - + + private Log LOG = LogFactory.getLog(LatencyOptimizerThread.class); + private LinkedBlockingQueue streamingDataQueue; - + private ExecutionGraph executionGraph; - + private LatencyModel latencyModel; - + public LatencyOptimizerThread(ExecutionGraph executionGraph) { + this.executionGraph = executionGraph; this.latencyModel = new LatencyModel(executionGraph); this.streamingDataQueue = new LinkedBlockingQueue(); } - + public void run() { + LOG.info("Started optimizer thread for job " + executionGraph.getJobName()); try { while (!interrupted()) { AbstractStreamingData streamingData = streamingDataQueue.take(); - + if (streamingData instanceof PathLatency) { latencyModel.refreshEdgeLatency((PathLatency) streamingData); } @@ -33,8 +40,10 @@ public void run() { } catch (InterruptedException e) { } + + LOG.info("Stopped optimizer thread for job " + executionGraph.getJobName()); } - + public void handOffStreamingData(AbstractStreamingData data) { streamingDataQueue.add(data); } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java index ffba8734a857e..a980321b19db4 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java @@ -49,7 +49,7 @@ public ManagementVertex getBegin() { } public ManagementVertex getEnd() { - return pathVertices.getFirst(); + return pathVertices.getLast(); } public ManagementEdge getIngoingEdge(ManagementVertex vertex) { @@ -106,4 +106,21 @@ public long refreshPathLatency() { public long getPathLatencyInMillis() { return this.pathLatencyInMillis; } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("LatencyPath["); + ManagementVertex previous = null; + for (ManagementVertex vertex : pathVertices) { + if (previous != null) { + builder.append("->"); + } + builder.append(vertex); + previous = vertex; + } + builder.append("]"); + + return builder.toString(); + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java index 0eb2f0e4de891..ea10dc2183aeb 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java @@ -4,6 +4,9 @@ import java.util.LinkedList; import java.util.List; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; @@ -24,6 +27,8 @@ * @author Bjoern Lohrmann */ public class LatencySubgraph { + + private static Log LOG = LogFactory.getLog(LatencySubgraph.class); private ManagementGroupVertex subgraphStart; @@ -31,9 +36,9 @@ public class LatencySubgraph { private List latencyPaths; - private HashMap vertexLatencies; + private HashMap vertexLatencies = new HashMap(); - private HashMap edgeLatencies; + private HashMap edgeLatencies = new HashMap(); public LatencySubgraph(ExecutionGraph executionGraph, ExecutionGroupVertex subgraphStart, ExecutionGroupVertex subgraphEnd) { From 49d2f9c12c777b30c10fb0a1b8cd7c111128b347 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Sun, 20 Nov 2011 00:41:29 +0100 Subject: [PATCH 062/310] simplifications to management graph structures due to multigraph-DAG problem (management graph and streaming plugin cannot deal with multigraph-DAGs currently) --- .../managementgraph/ManagementEdgeID.java | 11 +++++++++- .../managementgraph/ManagementGraph.java | 4 ++-- .../managementgraph/ManagementVertex.java | 2 +- .../managementgraph/ManagementGraphTest.java | 20 +++++++++---------- .../ManagementGraphFactory.java | 4 ++-- 5 files changed, 25 insertions(+), 16 deletions(-) diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java index 66ef22db58475..7f130e93ab51a 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java @@ -11,7 +11,16 @@ */ public class ManagementEdgeID extends AbstractID { - public ManagementEdgeID(ManagementGateID source, ManagementGateID target) { + /** + * A ManagementEdgeID is derived from a pair of #{@link ManagementVertexID}s. + * Note that this only works for simple DAGs that are not multi-graphs. + * FIXME: use ManagementGateID to make the management graph fully multi-graph capable. This + * means we have to construct management edge IDs from gate IDs + * + * @param source + * @param target + */ + public ManagementEdgeID(ManagementVertexID source, ManagementVertexID target) { super(source, target); } } diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java index 51415013aefea..8d8b1d86dd922 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java @@ -479,8 +479,8 @@ public void read(final DataInput in) throws IOException { final ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class); final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); - final ManagementEdgeID managementEdgeID = new ManagementEdgeID(sourceGate.getManagementGateID(), - targetGate.getManagementGateID()); + final ManagementEdgeID managementEdgeID = new ManagementEdgeID(sourceVertex.getID(), + targetVertex.getID()); new ManagementEdge(managementEdgeID, sourceGate, sourceIndex, targetGate, targetIndex, channelType, compressionLevel); } diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java index 5805cc6c146e6..e398a19bf0322 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java @@ -373,6 +373,6 @@ public void write(final DataOutput out) throws IOException { @Override public String toString() { - return String.format("ManagementVertex(%s_%d)", getGroupVertex().getName(), indexInGroup); + return String.format("%s_%d", getGroupVertex().getName(), indexInGroup); } } diff --git a/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java b/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java index 3e0f8b2fba105..e60694bcc700b 100644 --- a/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java +++ b/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java @@ -295,20 +295,20 @@ private static ManagementGraph constructTestManagementGraph() { new ManagementGroupEdge(groupVertex3, 0, groupVertex4, 0, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); // Edges - new ManagementEdge(new ManagementEdgeID(outputGate1_1.getManagementGateID(), - inputGate2_1.getManagementGateID()), outputGate1_1, 0, inputGate2_1, 0, ChannelType.NETWORK, + new ManagementEdge(new ManagementEdgeID(vertex1_1.getID(), + vertex2_1.getID()), outputGate1_1, 0, inputGate2_1, 0, ChannelType.NETWORK, CompressionLevel.NO_COMPRESSION); - new ManagementEdge(new ManagementEdgeID(outputGate1_1.getManagementGateID(), - inputGate2_2.getManagementGateID()), outputGate1_1, 1, inputGate2_2, 0, ChannelType.NETWORK, + new ManagementEdge(new ManagementEdgeID(vertex1_1.getID(), + vertex2_2.getID()), outputGate1_1, 1, inputGate2_2, 0, ChannelType.NETWORK, CompressionLevel.NO_COMPRESSION); - new ManagementEdge(new ManagementEdgeID(outputGate2_1.getManagementGateID(), - inputGate3_1.getManagementGateID()), outputGate2_1, 0, inputGate3_1, 0, ChannelType.FILE, + new ManagementEdge(new ManagementEdgeID(vertex2_1.getID(), + vertex3_1.getID()), outputGate2_1, 0, inputGate3_1, 0, ChannelType.FILE, CompressionLevel.NO_COMPRESSION); - new ManagementEdge(new ManagementEdgeID(outputGate2_2.getManagementGateID(), - inputGate3_1.getManagementGateID()), outputGate2_2, 0, inputGate3_1, 1, ChannelType.FILE, + new ManagementEdge(new ManagementEdgeID(vertex2_2.getID(), + vertex3_1.getID()), outputGate2_2, 0, inputGate3_1, 1, ChannelType.FILE, CompressionLevel.NO_COMPRESSION); - new ManagementEdge(new ManagementEdgeID(outputGate3_1.getManagementGateID(), - inputGate4_1.getManagementGateID()), outputGate3_1, 0, inputGate4_1, 0, ChannelType.INMEMORY, + new ManagementEdge(new ManagementEdgeID(vertex3_1.getID(), + vertex4_1.getID()), outputGate3_1, 0, inputGate4_1, 0, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); return graph; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java index 45de77c4f1e10..5e4803bf6ceea 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java @@ -174,8 +174,8 @@ private static void addExecutionVertices(Map Date: Sun, 20 Nov 2011 00:42:32 +0100 Subject: [PATCH 063/310] latency values from task managers are now written into latency model --- .../{PathLatency.java => ChannelLatency.java} | 6 +-- .../streaming/StreamingTaskListener.java | 2 +- .../streaming/latency/EdgeLatency.java | 7 ++-- .../streaming/latency/LatencyModel.java | 39 ++++++++++++++----- .../latency/LatencyOptimizerThread.java | 6 +-- .../streaming/latency/LatencyPath.java | 37 +++++++++++++++--- .../streaming/latency/LatencySubgraph.java | 12 ++++++ .../streaming/latency/VertexLatency.java | 12 ++++-- 8 files changed, 93 insertions(+), 28 deletions(-) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{PathLatency.java => ChannelLatency.java} (94%) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/ChannelLatency.java similarity index 94% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/ChannelLatency.java index f90a98987216b..a164731898968 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/PathLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/ChannelLatency.java @@ -27,7 +27,7 @@ * * @author warneke */ -public final class PathLatency extends AbstractStreamingData { +public final class ChannelLatency extends AbstractStreamingData { /** * The ID of the vertex representing the start of the path. @@ -56,7 +56,7 @@ public final class PathLatency extends AbstractStreamingData { * @param pathLatency * the path latency in milliseconds */ - public PathLatency(final JobID jobID, final ExecutionVertexID startVertexID, final ExecutionVertexID endVertexID, + public ChannelLatency(final JobID jobID, final ExecutionVertexID startVertexID, final ExecutionVertexID endVertexID, final double pathLatency) { super(jobID); @@ -77,7 +77,7 @@ public PathLatency(final JobID jobID, final ExecutionVertexID startVertexID, fin /** * Default constructor for the deserialization of the object. */ - public PathLatency() { + public ChannelLatency() { super(new JobID()); this.startVertexID = new ExecutionVertexID(); this.endVertexID = new ExecutionVertexID(); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java index 2690ae155f502..cfc8d6c7d251a 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java @@ -178,7 +178,7 @@ public void recordReceived(final Record record) { counter = Integer.valueOf(counter.intValue() + 1); if (counter.intValue() == this.aggregationInterval) { - final PathLatency pl = new PathLatency(this.jobID, sourceID, this.vertexID, + final ChannelLatency pl = new ChannelLatency(this.jobID, sourceID, this.vertexID, aggregatedLatency.doubleValue()); try { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeLatency.java index 5f88d8fac71c8..f2994f1e1dc4c 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeLatency.java @@ -6,21 +6,22 @@ public class EdgeLatency { private ManagementEdge edge; - private long latencyInMillis; + private double latencyInMillis; public EdgeLatency(ManagementEdge edge) { this.edge = edge; + this.latencyInMillis = -1; } public ManagementEdge getEdge() { return edge; } - public long getLatencyInMillis() { + public double getLatencyInMillis() { return latencyInMillis; } - public void setLatencyInMillis(long latencyInMillis) { + public void setLatencyInMillis(double latencyInMillis) { this.latencyInMillis = latencyInMillis; } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java index 548fceb264d66..f235094878e20 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java @@ -5,28 +5,47 @@ import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; -import eu.stratosphere.nephele.streaming.PathLatency; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; +import eu.stratosphere.nephele.streaming.ChannelLatency; public class LatencyModel { - - private static Log LOG = LogFactory.getLog(LatencyModel.class); - + + private static Log LOG = LogFactory.getLog(LatencyModel.class); + private ExecutionGraph executionGraph; - + private LatencySubgraph latencySubgraph; public LatencyModel(ExecutionGraph executionGraph) { this.executionGraph = executionGraph; - + // FIXME naive implementation until we can annotate the job // subgraphStart and subgraphEnd should be derived from the annotations ExecutionGroupVertex subgraphStart = this.executionGraph.getInputVertex(0).getGroupVertex(); ExecutionGroupVertex subgraphEnd = this.executionGraph.getOutputVertex(0).getGroupVertex(); - + this.latencySubgraph = new LatencySubgraph(executionGraph, subgraphStart, subgraphEnd); } - - public void refreshEdgeLatency(PathLatency pathLatency) { - LOG.info("Received " + pathLatency); + + public void refreshEdgeLatency(ChannelLatency latency) { + ExecutionVertexID startID = latency.getStartVertexID(); + ExecutionVertexID endID = latency.getEndVertexID(); + + if (startID.equals(endID)) { + System.out.println("new vertex latency"); + VertexLatency vertexLatency = latencySubgraph.getVertexLatency(startID.toManagementVertexID()); + vertexLatency.setLatencyInMillis(latency.getPathLatency()); + } else { + System.out.println("new edge latency"); + ManagementEdgeID edgeID = new ManagementEdgeID(latency.getStartVertexID().toManagementVertexID(), + latency.getEndVertexID().toManagementVertexID()); + + EdgeLatency edgeLatency = latencySubgraph.getEdgeLatency(edgeID); + edgeLatency.setLatencyInMillis(latency.getPathLatency()); + for (LatencyPath path : latencySubgraph.getLatencyPaths()) { + path.dumpLatencies(); + } + } } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java index cebca038a5063..9baba1be92330 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java @@ -7,7 +7,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.streaming.AbstractStreamingData; -import eu.stratosphere.nephele.streaming.PathLatency; +import eu.stratosphere.nephele.streaming.ChannelLatency; public class LatencyOptimizerThread extends Thread { @@ -32,8 +32,8 @@ public void run() { while (!interrupted()) { AbstractStreamingData streamingData = streamingDataQueue.take(); - if (streamingData instanceof PathLatency) { - latencyModel.refreshEdgeLatency((PathLatency) streamingData); + if (streamingData instanceof ChannelLatency) { + latencyModel.refreshEdgeLatency((ChannelLatency) streamingData); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java index a980321b19db4..59fdea6f3628b 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java @@ -23,7 +23,7 @@ public class LatencyPath implements Iterable { private LatencySubgraph graph; - private long pathLatencyInMillis; + private double pathLatencyInMillis; @SuppressWarnings("unchecked") public LatencyPath(LatencyPath toClone) { @@ -90,20 +90,30 @@ public Iterator iterator() { return pathVertices.iterator(); } - public long refreshPathLatency() { + public double refreshPathLatency() { this.pathLatencyInMillis = 0; for (ManagementVertex vertex : pathVertices) { ManagementEdge ingoingEdge = ingoingEdges.get(vertex.getID()); if (ingoingEdge != null) { - this.pathLatencyInMillis += ((EdgeLatency) ingoingEdge.getAttachment()).getLatencyInMillis(); + double edgeLatency = ((EdgeLatency) ingoingEdge.getAttachment()).getLatencyInMillis(); + if (edgeLatency == -1) { + throw new IllegalStateException("Edge latency has not yet been initialized: " + edgeLatency); + } + + this.pathLatencyInMillis += edgeLatency; + } + + double vertexLatency = ((VertexLatency) vertex.getAttachment()).getLatencyInMillis(); + if (vertexLatency == -1) { + throw new IllegalStateException("Vertex latency has not yet been initialized: " + vertexLatency); } - this.pathLatencyInMillis += ((VertexLatency) vertex.getAttachment()).getLatencyInMillis(); + this.pathLatencyInMillis += vertexLatency; } return this.pathLatencyInMillis; } - public long getPathLatencyInMillis() { + public double getPathLatencyInMillis() { return this.pathLatencyInMillis; } @@ -123,4 +133,21 @@ public String toString() { return builder.toString(); } + + public void dumpLatencies() { + + for (ManagementVertex vertex : pathVertices) { + ManagementEdge ingoing = ingoingEdges.get(vertex.getID()); + + if (ingoing != null) { + System.out.printf("---edge(%.03f)---%s(%.03f)\n", + ((EdgeLatency) ingoing.getAttachment()).getLatencyInMillis(), + vertex, + ((VertexLatency) vertex.getAttachment()).getLatencyInMillis()); + } else { + System.out.printf("%s(%.03f)\n", vertex, + ((VertexLatency) vertex.getAttachment()).getLatencyInMillis()); + } + } + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java index ea10dc2183aeb..9e06adca0e100 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java @@ -128,4 +128,16 @@ private void determineAnchoringManagementGroupVertices(ManagementGraph managemen this.subgraphEnd = managementGraph.getVertexByID(vertexInPathEndGroup).getGroupVertex(); } + public EdgeLatency getEdgeLatency(ManagementEdgeID edgeID) { + return edgeLatencies.get(edgeID); + } + + public VertexLatency getVertexLatency(ManagementVertexID managementVertexID) { + return vertexLatencies.get(managementVertexID); + } + + public List getLatencyPaths() { + return latencyPaths; + } + } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java index a31161039e335..dfe7b639f6694 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java @@ -6,21 +6,27 @@ public class VertexLatency { private ManagementVertex vertex; - private long latencyInMillis; + private double latencyInMillis; public VertexLatency(ManagementVertex vertex) { this.vertex = vertex; + this.latencyInMillis = -1; } public ManagementVertex getVertex() { return vertex; } - public long getLatencyInMillis() { + public double getLatencyInMillis() { return latencyInMillis; } - public void setLatencyInMillis(long latencyInMillis) { + public void setLatencyInMillis(double latencyInMillis) { this.latencyInMillis = latencyInMillis; } + + @Override + public String toString() { + return String.format("VertexLatency[%s|%.03f]", vertex.toString(), latencyInMillis); + } } From 8812d00e4ec0cf1dd44ffd44c8221db39454f71c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 14:36:01 +0000 Subject: [PATCH 064/310] Refactored input and output gates to support plugins --- .../nephele/execution/Environment.java | 57 ++-- .../nephele/io/AbstractRecordReader.java | 12 +- .../nephele/io/AbstractRecordWriter.java | 18 +- .../java/eu/stratosphere/nephele/io/Gate.java | 46 +++ .../eu/stratosphere/nephele/io/InputGate.java | 81 +++++ .../stratosphere/nephele/io/OutputGate.java | 71 ++++ .../impl/TaskManagerProfilerImpl.java | 16 +- .../nephele/execution/RuntimeEnvironment.java | 312 +++++++++--------- .../executiongraph/ExecutionGraph.java | 26 +- .../executiongraph/ExecutionStage.java | 3 +- .../executiongraph/ExecutionVertex.java | 5 +- .../ManagementGraphFactory.java | 11 +- .../stratosphere/nephele/io/AbstractGate.java | 67 +--- .../nephele/io/RuntimeInputGate.java | 151 ++------- .../nephele/io/RuntimeOutputGate.java | 143 +------- .../scheduler/AbstractScheduler.java | 3 +- .../profiling/TaskManagerProfiler.java | 8 +- .../nephele/taskmanager/Task.java | 3 +- .../nephele/taskmanager/TaskManager.java | 8 - .../ByteBufferedChannelManager.java | 10 +- .../nephele/streaming/InputGateWrapper.java | 216 ++++++++++++ .../nephele/streaming/OutputGateWrapper.java | 253 ++++++++++++++ .../streaming/StreamingJobManagerPlugin.java | 11 +- .../streaming/StreamingTaskManagerPlugin.java | 6 +- .../nephele/streaming/TaskWrapper.java | 207 ++++++++++++ .../runtime/test/util/MockEnvironment.java | 22 -- 26 files changed, 1170 insertions(+), 596 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/InputGateWrapper.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/OutputGateWrapper.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskWrapper.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java index eaf777b7275d5..1f2525ca2773d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java @@ -18,6 +18,7 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.io.ChannelSelector; import eu.stratosphere.nephele.io.DistributionPattern; +import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RecordDeserializer; @@ -111,36 +112,18 @@ public interface Environment { String getTaskName(); /** - * Checks if the environment has unbound input gates. + * Returns the next unbound input gate ID or null if no such ID exists * - * @return true if the environment has unbound input gates, false otherwise + * @return the next unbound input gate ID or null if no such ID exists */ - boolean hasUnboundInputGates(); + GateID getNextUnboundInputGateID(); /** - * Checks if the environment has unbound output gates. + * Returns the next unbound output gate ID or null if no such ID exists * - * @return true if the environment has unbound output gates, false otherwise + * @return the next unbound output gate ID or null if no such ID exists */ - boolean hasUnboundOutputGates(); - - /** - * Retrieves and removes the unbound output gate with the given ID from the list of unbound output gates. - * - * @param gateID - * the index of the unbound output gate - * @return the unbound output gate with the given ID, or null if no such gate exists - */ - OutputGate getUnboundOutputGate(final int gateID); - - /** - * Retrieves and removes unbound input gate with the given ID from the list of unbound input gates. - * - * @param gateID - * the index of the unbound input gate - * @return the unbound input gate with the given ID, or null if no such gate exists - */ - InputGate getUnboundInputGate(final int gateID); + GateID getNextUnboundOutputGateID(); /** * Returns the number of output gates registered with this environment. @@ -157,23 +140,41 @@ public interface Environment { int getNumberOfInputGates(); /** - * Creates and registers an output gate with the environment. + * Creates an output gate. * + * @param gateID * @param outputClass * @param selector * @param isBroadcast * @return the created output gate */ - OutputGate createAndRegisterOutputGate(Class outputClass, + OutputGate createOutputGate(GateID gateID, Class outputClass, ChannelSelector selector, boolean isBroadcast); /** - * Creates and registers an input gate with the environment. + * Creates an input gate. * + * @param gateID * @param deserializer * @param distributionPattern * @return the created input gate */ - InputGate createAndRegisterInputGate(RecordDeserializer deserializer, + InputGate createInputGate(GateID gateID, RecordDeserializer deserializer, DistributionPattern distributionPattern); + + /** + * Registers an output gate with this environment. + * + * @param outputGate + * the output gate to be registered + */ + void registerOutputGate(OutputGate outputGate); + + /** + * Registers an input gate with this environment. + * + * @param inputGate + * the input gate to be registered + */ + void registerInputGate(InputGate inputGate); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java index efcc2bb793295..2312f938b2edb 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java @@ -62,13 +62,13 @@ protected AbstractRecordReader(final AbstractInvokable invokable, final RecordDe private void connectInputGate(final RecordDeserializer deserializer, final int inputGateID, final DistributionPattern distributionPattern) { - // See if there are any unbound input gates left we can connect to - if (this.environment.hasUnboundInputGates()) { - this.inputGate = (InputGate) this.environment.getUnboundInputGate(inputGateID); - } else { - this.inputGate = (InputGate) this.environment.createAndRegisterInputGate(deserializer, - distributionPattern); + GateID gateID = this.environment.getNextUnboundInputGateID(); + if (gateID == null) { + gateID = new GateID(); } + + this.inputGate = (InputGate) this.environment.createInputGate(gateID, deserializer, distributionPattern); + this.environment.registerInputGate(this.inputGate); } /** diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java index df81b4fdb6527..852562045fab4 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java @@ -79,18 +79,13 @@ public AbstractRecordWriter(AbstractInvokable invokable, Class outputClass, C @SuppressWarnings("unchecked") private void connectOutputGate(Class outputClass, ChannelSelector selector, boolean isBroadcast) { - // See if there are any unbound input gates left we can connect to - if (this.environment.hasUnboundOutputGates()) { - final OutputGate eog = (OutputGate) this.environment.getUnboundOutputGate(0); - if (!outputClass.equals(eog.getType())) { - throw new RuntimeException("Unbound input gate found, but types do not match!"); - } - - this.outputGate = eog; - } else { - this.outputGate = (OutputGate) this.environment.createAndRegisterOutputGate(outputClass, selector, - isBroadcast); + GateID gateID = this.environment.getNextUnboundOutputGateID(); + if (gateID == null) { + gateID = new GateID(); } + + this.outputGate = (OutputGate) this.environment.createOutputGate(gateID, outputClass, selector, isBroadcast); + this.environment.registerOutputGate(this.outputGate); } /** @@ -118,6 +113,7 @@ public List> getOutputChannels() { } // TODO (en) + @Deprecated public OutputGate getOutputGate() { return outputGate; } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/Gate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/Gate.java index a3cb5422e34a6..ffe16b740ff7b 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/Gate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/Gate.java @@ -25,6 +25,13 @@ public interface Gate { + /** + * Returns the index that has been assigned to the gate upon initialization. + * + * @return the index that has been assigned to the gate upon initialization. + */ + int getIndex(); + /** * Subscribes the listener object to receive events of the given type. * @@ -78,4 +85,43 @@ public interface Gate { * @return the type of input/output channels which are connected to this gate */ ChannelType getChannelType(); + + /** + * Returns the ID of the gate. + * + * @return the ID of the gate + */ + GateID getGateID(); + + /** + * Releases the allocated resources (particularly buffer) of all channels attached to this gate. This method + * should only be called after the respected task has stopped running. + */ + void releaseAllChannelResources(); + + /** + * Checks if the gate is closed. The gate is closed if all this associated channels are closed. + * + * @return true if the gate is closed, false otherwise + * @throws IOException + * thrown if any error occurred while closing the gate + * @throws InterruptedException + * thrown if the gate is interrupted while waiting for this operation to complete + */ + boolean isClosed() throws IOException, InterruptedException; + + /** + * Checks if the considered gate is an input gate. + * + * @return true if the considered gate is an input gate, false if it is an output gate + */ + boolean isInputGate(); + + /** + * Sets the type of the input/output channels which are connected to this gate. + * + * @param channelType + * the type of input/output channels which are connected to this gate + */ + void setChannelType(ChannelType channelType); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java index 7bc36ac86a564..3f97d50cee95d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java @@ -18,6 +18,11 @@ import java.io.IOException; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.bytebuffered.FileInputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryInputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkInputChannel; +import eu.stratosphere.nephele.io.compression.CompressionLevel; import eu.stratosphere.nephele.types.Record; /** @@ -68,4 +73,80 @@ public interface InputGate extends Gate { * at least one record available. */ void notifyRecordIsAvailable(int channelIndex); + + /** + * Activates all of the task's input channels. + * + * @throws IOException + * thrown if an I/O error occurs while transmitting one of the activation requests to the corresponding + * output channels + * @throws InterruptedException + * throws if the task is interrupted while waiting for the activation process to complete + */ + void activateInputChannels() throws IOException, InterruptedException; + + /** + * Immediately closes the input gate and all its input channels. The corresponding + * output channels are notified. Any remaining records in any buffers or queue is considered + * irrelevant and is discarded. + * + * @throws IOException + * thrown if an I/O error occurs while closing the gate + * @throws InterruptedException + * thrown if the thread is interrupted while waiting for the gate to be closed + */ + void close() throws IOException, InterruptedException; + + /** + * Registers a new listener object for this input gate. + * + * @param inputGateListener + * the listener object to register + */ + void registerInputGateListener(InputGateListener inputGateListener); + + /** + * Returns the {@link DistributionPattern} associated with this input gate. + * + * @return the {@link DistributionPattern} associated with this input gate + */ + DistributionPattern getDistributionPattern(); + + /** + * Creates a new network input channel and assigns it to the input gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new network input channel + */ + NetworkInputChannel createNetworkInputChannel(ChannelID channelID, CompressionLevel compressionLevel); + + /** + * Creates a new file input channel and assigns it to the input gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new file input channel + */ + FileInputChannel createFileInputChannel(ChannelID channelID, CompressionLevel compressionLevel); + + /** + * Creates a new in-memory input channel and assigns it to the input gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new in-memory input channel + */ + InMemoryInputChannel createInMemoryInputChannel(ChannelID channelID, CompressionLevel compressionLevel); + + /** + * Removes all input channels from the input gate. + */ + void removeAllInputChannels(); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java index 20d0c08a7dc79..56c096df36821 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java @@ -19,6 +19,11 @@ import java.util.List; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.bytebuffered.FileOutputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryOutputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkOutputChannel; +import eu.stratosphere.nephele.io.compression.CompressionLevel; import eu.stratosphere.nephele.types.Record; /** @@ -92,10 +97,76 @@ public interface OutputGate extends Gate { */ int getNumberOfOutputChannels(); + /** + * Returns the output channel from position pos of the gate's + * internal channel list. + * + * @param pos + * the position to retrieve the channel from + * @return the channel from the given position or null if such + * position does not exist. + */ + AbstractOutputChannel getOutputChannel(int pos); + /** * Returns the output gate's channel selector. * * @return the output gate's channel selector or null if the gate operates in broadcast mode */ ChannelSelector getChannelSelector(); + + /** + * Requests the output gate to closed. This means the application will send + * no records through this gate anymore. + * + * @throws IOException + * @throws InterruptedException + */ + void requestClose() throws IOException, InterruptedException; + + /** + * Removes all output channels from the output gate. + */ + void removeAllOutputChannels(); + + /** + * Creates a new network output channel and assigns it to the output gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new network output channel + */ + NetworkOutputChannel createNetworkOutputChannel(ChannelID channelID, CompressionLevel compressionLevel); + + /** + * Creates a new file output channel and assigns it to the output gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new file output channel + */ + FileOutputChannel createFileOutputChannel(ChannelID channelID, CompressionLevel compressionLevel); + + /** + * Creates a new in-memory output channel and assigns it to the output gate. + * + * @param channelID + * the channel ID to assign to the new channel, null to generate a new ID + * @param compressionLevel + * the level of compression to be used for this channel + * @return the new in-memory output channel + */ + InMemoryOutputChannel createInMemoryOutputChannel(ChannelID channelID, CompressionLevel compressionLevel); + + /** + * Registers a new listener object for this output gate. + * + * @param outputGateListener + * the listener object to register + */ + void registerOutputGateListener(OutputGateListener outputGateListener); } diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java index 7ff86201fd533..b13a5f4628de3 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java @@ -36,8 +36,6 @@ import eu.stratosphere.nephele.instance.InstanceConnectionInfo; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.RuntimeInputGate; -import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.ipc.RPC; import eu.stratosphere.nephele.net.NetUtils; import eu.stratosphere.nephele.profiling.ProfilingException; @@ -120,9 +118,12 @@ public void registerExecutionListener(final Task task, final Configuration jobCo task.registerExecutionListener(new EnvironmentListenerImpl(this, task.getEnvironment())); } + /** + * {@inheritDoc} + */ @Override - public void registerInputGateListener(ExecutionVertexID id, Configuration jobConfiguration, - RuntimeInputGate inputGate) { + public void registerInputGateListener(final ExecutionVertexID id, final Configuration jobConfiguration, + InputGate inputGate) { synchronized (this.monitoredInputGates) { @@ -133,9 +134,12 @@ public void registerInputGateListener(ExecutionVertexID id, Configuration jobCon } } + /** + * {@inheritDoc} + */ @Override - public void registerOutputGateListener(ExecutionVertexID id, Configuration jobConfiguration, - RuntimeOutputGate outputGate) { + public void registerOutputGateListener(final ExecutionVertexID id, final Configuration jobConfiguration, + final OutputGate outputGate) { synchronized (this.monitoredOutputGates) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 068477d535ff4..7162bc61d9696 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -18,7 +18,9 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.ArrayDeque; import java.util.List; +import java.util.Queue; import java.util.concurrent.CopyOnWriteArrayList; import org.apache.commons.logging.Log; @@ -35,6 +37,11 @@ import eu.stratosphere.nephele.io.RecordDeserializer; import eu.stratosphere.nephele.io.RuntimeInputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; +import eu.stratosphere.nephele.io.channels.AbstractInputChannel; +import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.compression.CompressionLevel; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; @@ -42,6 +49,7 @@ import eu.stratosphere.nephele.template.InputSplitProvider; import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.types.StringRecord; +import eu.stratosphere.nephele.util.EnumUtils; import eu.stratosphere.nephele.util.StringUtils; /** @@ -68,22 +76,24 @@ public class RuntimeEnvironment implements Environment, Runnable, IOReadableWrit /** * List of output gates created by the task. */ - private final List> outputGates = new CopyOnWriteArrayList>(); + private final List> outputGates = new CopyOnWriteArrayList>(); /** * List of input gates created by the task. */ - private final List> inputGates = new CopyOnWriteArrayList>(); + private final List> inputGates = new CopyOnWriteArrayList>(); /** - * List of output gates which have to be rebound to a task after transferring the environment to a TaskManager. + * Queue of unbound output gate IDs which are required for deserializing an environment in the course of an RPC + * call. */ - private final List> unboundOutputGates = new CopyOnWriteArrayList>(); + private final Queue unboundOutputGateIDs = new ArrayDeque(); /** - * List of input gates which have to be rebound to a task after transferring the environment to a TaskManager. + * Queue of unbound input gate IDs which are required for deserializing an environment in the course of an RPC + * call. */ - private final List> unboundInputGates = new CopyOnWriteArrayList>(); + private final Queue unboundInputGateIDs = new ArrayDeque(); /** * The memory manager of the current environment (currently the one associated with the executing TaskManager). @@ -193,49 +203,17 @@ public JobID getJobID() { * {@inheritDoc} */ @Override - public boolean hasUnboundInputGates() { + public GateID getNextUnboundInputGateID() { - return (this.unboundInputGates.size() > 0); + return this.unboundInputGateIDs.poll(); } /** * {@inheritDoc} */ - @Override - public boolean hasUnboundOutputGates() { - - return (this.unboundOutputGates.size() > 0); - } - - /** - * {@inheritDoc} - */ - @Override - public OutputGate getUnboundOutputGate(final int gateID) { - - if (this.unboundOutputGates.size() == 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("No unbound output gates"); - } - return null; - } - return this.unboundOutputGates.remove(gateID); - } - - /** - * {@inheritDoc} - */ - @Override - public InputGate getUnboundInputGate(final int gateID) { - - if (this.unboundInputGates.size() == 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("No unbound input gates"); - } - return null; - } + public GateID getNextUnboundOutputGateID() { - return this.unboundInputGates.remove(gateID); + return this.unboundOutputGateIDs.poll(); } /** @@ -376,10 +354,7 @@ public void run() { private void activateInputChannels() throws IOException, InterruptedException { for (int i = 0; i < getNumberOfInputGates(); ++i) { - final InputGate eig = getInputGate(i); - for (int j = 0; j < eig.getNumberOfInputChannels(); ++j) { - eig.getInputChannel(j).activate(); - } + this.inputGates.get(i).activateInputChannels(); } } @@ -387,14 +362,12 @@ private void activateInputChannels() throws IOException, InterruptedException { * {@inheritDoc} */ @Override - public OutputGate createAndRegisterOutputGate(final Class outputClass, + public OutputGate createOutputGate(final GateID gateID, Class outputClass, final ChannelSelector selector, final boolean isBroadcast) { @SuppressWarnings({ "unchecked", "rawtypes" }) final RuntimeOutputGate rog = (RuntimeOutputGate) new RuntimeOutputGate( - getJobID(), new GateID(), outputClass, getNumberOfOutputGates(), selector, isBroadcast); - - this.outputGates.add(rog); + getJobID(), gateID, outputClass, getNumberOfOutputGates(), selector, isBroadcast); return rog; } @@ -403,19 +376,28 @@ public OutputGate createAndRegisterOutputGate(final Class createAndRegisterInputGate( - final RecordDeserializer deserializer, - final DistributionPattern distributionPattern) { + public InputGate createInputGate(final GateID gateID, + final RecordDeserializer deserializer, final DistributionPattern distributionPattern) { @SuppressWarnings({ "unchecked", "rawtypes" }) final RuntimeInputGate rig = (RuntimeInputGate) new RuntimeInputGate( - getJobID(), new GateID(), deserializer, getNumberOfInputGates(), distributionPattern); - - this.inputGates.add(rig); + getJobID(), gateID, deserializer, getNumberOfInputGates(), distributionPattern); return rig; } + @Override + public void registerOutputGate(OutputGate outputGate) { + + this.outputGates.add(outputGate); + } + + @Override + public void registerInputGate(InputGate inputGate) { + + this.inputGates.add(inputGate); + } + /** * {@inheritDoc} */ @@ -438,7 +420,7 @@ public int getNumberOfInputGates() { * the index of the input gate to return * @return the input gate at index pos or null if no such index exists */ - public RuntimeInputGate getInputGate(final int pos) { + public InputGate getInputGate(final int pos) { if (pos < this.inputGates.size()) { return this.inputGates.get(pos); } @@ -453,7 +435,7 @@ public RuntimeInputGate getInputGate(final int pos) { * the index of the output gate to return * @return the output gate at index pos or null if no such index exists */ - public RuntimeOutputGate getOutputGate(final int pos) { + public OutputGate getOutputGate(final int pos) { if (pos < this.outputGates.size()) { return this.outputGates.get(pos); } @@ -529,43 +511,7 @@ public void read(final DataInput in) throws IOException { final GateID gateID = new GateID(); gateID.read(in); - - final String typeClassName = StringRecord.readString(in); - Class type = null; - try { - type = (Class) Class.forName(typeClassName, true, cl); - } catch (ClassNotFoundException cnfe) { - throw new IOException("Class " + typeClassName + " not found in one of the supplied jar files: " - + StringUtils.stringifyException(cnfe)); - } - - final boolean isBroadcast = in.readBoolean(); - - ChannelSelector channelSelector = null; - if (!isBroadcast) { - - final String channelSelectorClassName = StringRecord.readString(in); - try { - channelSelector = (ChannelSelector) Class.forName(channelSelectorClassName, true, - cl).newInstance(); - } catch (InstantiationException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (IllegalAccessException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (ClassNotFoundException e) { - throw new IOException(StringUtils.stringifyException(e)); - } - - channelSelector.read(in); - } - - @SuppressWarnings("rawtypes") - final RuntimeOutputGate eog = new RuntimeOutputGate(this.jobID, gateID, type, i, - channelSelector, isBroadcast); - eog.read(in); - this.outputGates.add(eog); - // Mark as unbound for reconnection of RecordWriter - this.unboundOutputGates.add(eog); + this.unboundOutputGateIDs.add(gateID); } final int numInputGates = in.readInt(); @@ -574,50 +520,7 @@ public void read(final DataInput in) throws IOException { final GateID gateID = new GateID(); gateID.read(in); - - final String deserializerClassName = StringRecord.readString(in); - RecordDeserializer recordDeserializer = null; - Class> deserializerClass = null; - try { - deserializerClass = (Class>) cl - .loadClass(deserializerClassName); - recordDeserializer = deserializerClass.newInstance(); - - } catch (ClassNotFoundException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (InstantiationException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (IllegalAccessException e) { - throw new IOException(StringUtils.stringifyException(e)); - } - - recordDeserializer.setClassLoader(cl); - recordDeserializer.read(in); - - final String distributionPatternClassName = StringRecord.readString(in); - DistributionPattern distributionPattern = null; - Class distributionPatternClass = null; - try { - distributionPatternClass = (Class) cl - .loadClass(distributionPatternClassName); - - distributionPattern = distributionPatternClass.newInstance(); - - } catch (ClassNotFoundException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (InstantiationException e) { - throw new IOException(StringUtils.stringifyException(e)); - } catch (IllegalAccessException e) { - throw new IOException(StringUtils.stringifyException(e)); - } - - @SuppressWarnings("rawtypes") - final RuntimeInputGate eig = new RuntimeInputGate(this.jobID, gateID, recordDeserializer, - i, distributionPattern); - eig.read(in); - this.inputGates.add(eig); - // Mark as unbound for reconnection of RecordReader - this.unboundInputGates.add(eig); + this.unboundInputGateIDs.add(gateID); } // The configuration object @@ -635,6 +538,74 @@ public void read(final DataInput in) throws IOException { } catch (Exception e) { throw new IOException(StringUtils.stringifyException(e)); } + + // Output channels + for (int i = 0; i < numOuputGates; ++i) { + final OutputGate outputGate = this.outputGates.get(i); + final int numberOfOutputChannels = in.readInt(); + for (int j = 0; j < numberOfOutputChannels; ++j) { + final ChannelID channelID = new ChannelID(); + channelID.read(in); + final ChannelID connectedChannelID = new ChannelID(); + connectedChannelID.read(in); + final ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class); + final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); + + AbstractOutputChannel outputChannel = null; + + switch (channelType) { + case INMEMORY: + outputChannel = outputGate.createInMemoryOutputChannel(channelID, compressionLevel); + break; + case NETWORK: + outputChannel = outputGate.createNetworkOutputChannel(channelID, compressionLevel); + break; + case FILE: + outputChannel = outputGate.createFileOutputChannel(channelID, compressionLevel); + break; + } + + if (outputChannel == null) { + throw new IOException("Unable to create output channel for channel ID " + channelID); + } + + outputChannel.setConnectedChannelID(connectedChannelID); + } + } + + // Input channels + for (int i = 0; i < numInputGates; ++i) { + final InputGate inputGate = this.inputGates.get(i); + final int numberOfInputChannels = in.readInt(); + for (int j = 0; j < numberOfInputChannels; ++j) { + final ChannelID channelID = new ChannelID(); + channelID.read(in); + final ChannelID connectedChannelID = new ChannelID(); + connectedChannelID.read(in); + final ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class); + final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); + + AbstractInputChannel inputChannel = null; + + switch (channelType) { + case INMEMORY: + inputChannel = inputGate.createInMemoryInputChannel(channelID, compressionLevel); + break; + case NETWORK: + inputChannel = inputGate.createNetworkInputChannel(channelID, compressionLevel); + break; + case FILE: + inputChannel = inputGate.createFileInputChannel(channelID, compressionLevel); + break; + } + + if (inputChannel == null) { + throw new IOException("Unable to create output channel for channel ID " + channelID); + } + + inputChannel.setConnectedChannelID(connectedChannelID); + } + } } /** @@ -669,30 +640,19 @@ public void write(final DataOutput out) throws IOException { StringRecord.writeString(out, this.invokableClass.getName()); // Output gates - out.writeInt(this.outputGates.size()); - for (int i = 0; i < this.outputGates.size(); i++) { - final RuntimeOutputGate outputGate = this.outputGates.get(i); + final int numberOfOutputGates = this.outputGates.size(); + out.writeInt(numberOfOutputGates); + for (int i = 0; i < numberOfOutputGates; ++i) { + final OutputGate outputGate = this.outputGates.get(i); outputGate.getGateID().write(out); - StringRecord.writeString(out, outputGate.getType().getName()); - out.writeBoolean(outputGate.isBroadcast()); - if (!outputGate.isBroadcast()) { - // Write out class name of channel selector - StringRecord.writeString(out, outputGate.getChannelSelector().getClass().getName()); - outputGate.getChannelSelector().write(out); - } - - outputGate.write(out); } // Input gates - out.writeInt(getNumberOfInputGates()); - for (int i = 0; i < getNumberOfInputGates(); i++) { - final RuntimeInputGate inputGate = this.inputGates.get(i); + final int numberOfInputGates = this.inputGates.size(); + out.writeInt(numberOfInputGates); + for (int i = 0; i < numberOfInputGates; i++) { + final InputGate inputGate = this.inputGates.get(i); inputGate.getGateID().write(out); - StringRecord.writeString(out, inputGate.getRecordDeserializer().getClass().getName()); - inputGate.getRecordDeserializer().write(out); - StringRecord.writeString(out, inputGate.getDistributionPattern().getClass().getName()); - inputGate.write(out); } // The configuration object @@ -702,6 +662,34 @@ public void write(final DataOutput out) throws IOException { out.writeInt(this.currentNumberOfSubtasks); // The index in the subtask group out.writeInt(this.indexInSubtaskGroup); + + // Output channels + for (int i = 0; i < numberOfOutputGates; ++i) { + final OutputGate outputGate = this.outputGates.get(i); + final int numberOfOutputChannels = outputGate.getNumberOfOutputChannels(); + out.writeInt(numberOfOutputChannels); + for (int j = 0; j < numberOfOutputChannels; ++j) { + final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); + outputChannel.getID().write(out); + outputChannel.getConnectedChannelID().write(out); + EnumUtils.writeEnum(out, outputChannel.getType()); + EnumUtils.writeEnum(out, outputChannel.getCompressionLevel()); + } + } + + // Input channels + for (int i = 0; i < numberOfInputGates; ++i) { + final InputGate inputGate = this.inputGates.get(i); + final int numberOfInputChannels = inputGate.getNumberOfInputChannels(); + out.writeInt(numberOfInputChannels); + for (int j = 0; j < numberOfInputChannels; ++j) { + final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); + inputChannel.getID().write(out); + inputChannel.getConnectedChannelID().write(out); + EnumUtils.writeEnum(out, inputChannel.getType()); + EnumUtils.writeEnum(out, inputChannel.getCompressionLevel()); + } + } } /** @@ -724,8 +712,8 @@ private void waitForOutputChannelsToBeClosed() throws IOException, InterruptedEx boolean allClosed = true; for (int i = 0; i < getNumberOfOutputGates(); i++) { - final RuntimeOutputGate rog = this.outputGates.get(i); - if (!rog.isClosed()) { + final OutputGate og = this.outputGates.get(i); + if (!og.isClosed()) { allClosed = false; } } @@ -758,7 +746,7 @@ private void waitForInputChannelsToBeClosed() throws IOException, InterruptedExc boolean allClosed = true; for (int i = 0; i < getNumberOfInputGates(); i++) { - final RuntimeInputGate eig = this.inputGates.get(i); + final InputGate eig = this.inputGates.get(i); if (!eig.isClosed()) { allClosed = false; } @@ -778,7 +766,7 @@ private void waitForInputChannelsToBeClosed() throws IOException, InterruptedExc private void closeInputGates() throws IOException, InterruptedException { for (int i = 0; i < this.inputGates.size(); i++) { - final RuntimeInputGate eig = this.inputGates.get(i); + final InputGate eig = this.inputGates.get(i); // Important: close must be called on each input gate exactly once eig.close(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index 171cc1f3a0039..ff3ea84070212 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -37,8 +37,8 @@ import eu.stratosphere.nephele.instance.DummyInstance; import eu.stratosphere.nephele.instance.InstanceManager; import eu.stratosphere.nephele.instance.InstanceType; -import eu.stratosphere.nephele.io.RuntimeInputGate; -import eu.stratosphere.nephele.io.RuntimeOutputGate; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; @@ -325,7 +325,7 @@ private void createInitialChannels(final AbstractJobVertex jobVertex, final AbstractJobVertex target = edge.getConnectedVertex(); // find output gate of execution vertex - final RuntimeOutputGate eog = ev.getEnvironment().getOutputGate(j); + final OutputGate eog = ev.getEnvironment().getOutputGate(j); if (eog == null) { throw new GraphConversionException("Cannot retrieve output gate " + j + " from vertex " + jobVertex.getName()); @@ -336,7 +336,7 @@ private void createInitialChannels(final AbstractJobVertex jobVertex, throw new GraphConversionException("Cannot find mapping for vertex " + target.getName()); } - final RuntimeInputGate eig = executionTarget.getEnvironment().getInputGate( + final InputGate eig = executionTarget.getEnvironment().getInputGate( edge.getIndexOfInputGate()); if (eig == null) { throw new GraphConversionException("Cannot retrieve input gate " + edge.getIndexOfInputGate() @@ -391,7 +391,7 @@ void unwire(final ExecutionGroupVertex source, final int indexOfOutputGate, fina for (int i = 0; i < source.getCurrentNumberOfGroupMembers(); i++) { final ExecutionVertex sourceVertex = source.getGroupMember(i); - final RuntimeOutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( + final OutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( indexOfOutputGate); if (outputGate == null) { throw new GraphConversionException("unwire: " + sourceVertex.getName() @@ -411,7 +411,7 @@ void unwire(final ExecutionGroupVertex source, final int indexOfOutputGate, fina for (int i = 0; i < target.getCurrentNumberOfGroupMembers(); i++) { final ExecutionVertex targetVertex = target.getGroupMember(i); - final RuntimeInputGate inputGate = targetVertex.getEnvironment().getInputGate( + final InputGate inputGate = targetVertex.getEnvironment().getInputGate( indexOfInputGate); if (inputGate == null) { throw new GraphConversionException("unwire: " + targetVertex.getName() @@ -436,7 +436,7 @@ void wire(final ExecutionGroupVertex source, final int indexOfOutputGate, final for (int i = 0; i < source.getCurrentNumberOfGroupMembers(); i++) { final ExecutionVertex sourceVertex = source.getGroupMember(i); - final RuntimeOutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( + final OutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( indexOfOutputGate); if (outputGate == null) { throw new GraphConversionException("wire: " + sourceVertex.getName() @@ -450,7 +450,7 @@ void wire(final ExecutionGroupVertex source, final int indexOfOutputGate, final for (int j = 0; j < target.getCurrentNumberOfGroupMembers(); j++) { final ExecutionVertex targetVertex = target.getGroupMember(j); - final RuntimeInputGate inputGate = targetVertex.getEnvironment().getInputGate( + final InputGate inputGate = targetVertex.getEnvironment().getInputGate( indexOfInputGate); if (inputGate == null) { throw new GraphConversionException("wire: " + targetVertex.getName() @@ -477,8 +477,8 @@ void wire(final ExecutionGroupVertex source, final int indexOfOutputGate, final } - private void createChannel(final ExecutionVertex source, final RuntimeOutputGate outputGate, - final ExecutionVertex target, final RuntimeInputGate inputGate, + private void createChannel(final ExecutionVertex source, final OutputGate outputGate, + final ExecutionVertex target, final InputGate inputGate, final ChannelType channelType, final CompressionLevel compressionLevel) throws GraphConversionException { AbstractOutputChannel outputChannel; @@ -1071,7 +1071,7 @@ public void repairInstanceAssignment() { for (int i = 0; i < sourceVertex.getEnvironment().getNumberOfOutputGates(); i++) { - final RuntimeOutputGate outputGate = sourceVertex.getEnvironment().getOutputGate(i); + final OutputGate outputGate = sourceVertex.getEnvironment().getOutputGate(i); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); j++) { final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); final ChannelType channelType = outputChannel.getType(); @@ -1091,7 +1091,7 @@ public void repairInstanceAssignment() { for (int i = 0; i < targetVertex.getEnvironment().getNumberOfInputGates(); i++) { - final RuntimeInputGate inputGate = targetVertex.getEnvironment().getInputGate(i); + final InputGate inputGate = targetVertex.getEnvironment().getInputGate(i); for (int j = 0; j < inputGate.getNumberOfInputChannels(); j++) { final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); final ChannelType channelType = inputChannel.getType(); @@ -1119,7 +1119,7 @@ public ChannelType getChannelType(final ExecutionVertex sourceVertex, final Exec final ExecutionGroupEdge edge = edges.get(0); // Now lets see if these two concrete subtasks are connected - final RuntimeOutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( + final OutputGate outputGate = sourceVertex.getEnvironment().getOutputGate( edge.getIndexOfOutputGate()); for (int i = 0; i < outputGate.getNumberOfOutputChannels(); i++) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java index 851528a04d6ba..894509f58acf8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java @@ -31,6 +31,7 @@ import eu.stratosphere.nephele.instance.InstanceRequestMap; import eu.stratosphere.nephele.instance.InstanceType; import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; @@ -389,7 +390,7 @@ private void reconstructExecutionPipeline(final ExecutionVertex vertex, final bo final int numberOfOutputGates = env.getNumberOfOutputGates(); for (int i = 0; i < numberOfOutputGates; ++i) { - final RuntimeOutputGate outputGate = env.getOutputGate(i); + final OutputGate outputGate = env.getOutputGate(i); final ChannelType channelType = outputGate.getChannelType(); final int numberOfOutputChannels = outputGate.getNumberOfOutputChannels(); for (int j = 0; j < numberOfOutputChannels; ++j) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index f3c31daf1082b..014bb3b02838c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -32,6 +32,7 @@ import eu.stratosphere.nephele.instance.AllocatedResource; import eu.stratosphere.nephele.instance.AllocationID; import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; @@ -477,7 +478,7 @@ public ExecutionVertex getSuccessor(int index) { for (int i = 0; i < this.environment.getNumberOfOutputGates(); i++) { - final RuntimeOutputGate outputGate = this.environment.getOutputGate(i); + final OutputGate outputGate = this.environment.getOutputGate(i); if (index >= 0 && index < outputGate.getNumberOfOutputChannels()) { @@ -522,7 +523,7 @@ public SerializableHashSet constructInitialActiveOutputChannelsSet() final int numberOfOutputGates = this.environment.getNumberOfOutputGates(); for (int i = 0; i < numberOfOutputGates; ++i) { - final RuntimeOutputGate outputGate = this.environment.getOutputGate(i); + final OutputGate outputGate = this.environment.getOutputGate(i); final ChannelType channelType = outputGate.getChannelType(); final int numberOfOutputChannels = outputGate.getNumberOfOutputChannels(); for (int j = 0; j < numberOfOutputChannels; ++j) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java index a995103a2b7c5..ab7ed2a29eff2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java @@ -19,8 +19,9 @@ import java.util.Iterator; import java.util.Map; -import eu.stratosphere.nephele.io.AbstractGate; -import eu.stratosphere.nephele.io.RuntimeInputGate; +import eu.stratosphere.nephele.io.Gate; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; @@ -118,7 +119,7 @@ private static void addExecutionVertices(Map vertexMap = new HashMap(); - final Map, ManagementGate> gateMap = new HashMap, ManagementGate>(); + final Map, ManagementGate> gateMap = new HashMap, ManagementGate>(); while (iterator.hasNext()) { @@ -134,14 +135,14 @@ private static void addExecutionVertices(Map outputGate = ev.getEnvironment().getOutputGate(i); + final OutputGate outputGate = ev.getEnvironment().getOutputGate(i); final ManagementGate managementGate = new ManagementGate(managementVertex, i, false, outputGate .getType().toString()); gateMap.put(outputGate, managementGate); } for (int i = 0; i < ev.getEnvironment().getNumberOfInputGates(); i++) { - final RuntimeInputGate inputGate = ev.getEnvironment().getInputGate(i); + final InputGate inputGate = ev.getEnvironment().getInputGate(i); final ManagementGate managementGate = new ManagementGate(managementVertex, i, true, ""); gateMap.put(inputGate, managementGate); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java index d0faf41e591a1..0240f2481ae11 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java @@ -15,17 +15,12 @@ package eu.stratosphere.nephele.io; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - import eu.stratosphere.nephele.event.task.AbstractTaskEvent; import eu.stratosphere.nephele.event.task.EventListener; import eu.stratosphere.nephele.event.task.EventNotificationManager; import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.util.EnumUtils; /** * In Nephele a gate represents the connection between a user program and the processing framework. A gate @@ -39,7 +34,7 @@ * @param * the record type to be transported from this gate */ -public abstract class AbstractGate implements Gate, IOReadableWritable { +public abstract class AbstractGate implements Gate { /** * The ID of the job this gate belongs to. @@ -83,10 +78,9 @@ protected AbstractGate(final JobID jobID, final GateID gateID, final int index) } /** - * Returns the index that has been assigned to the gate upon initialization. - * - * @return the index that has been assigned to the gate upon initialization. + * {@inheritDoc} */ + @Override public final int getIndex() { return this.index; } @@ -100,24 +94,6 @@ protected final EventNotificationManager getEventNotificationManager() { return this.eventNotificationManager; } - /** - * Checks if the gate is closed. The gate is closed if alls this associated channels are closed. - * - * @return true if the gate is closed, false otherwise - * @throws IOException - * thrown if any error occurred while closing the gate - * @throws InterruptedException - * thrown if the gate is interrupted while waiting for this operation to complete - */ - public abstract boolean isClosed() throws IOException, InterruptedException; - - /** - * Checks if the considered gate is an input gate. - * - * @return true if the considered gate is an input gate, false if it is an output gate - */ - public abstract boolean isInputGate(); - /** * {@inheritDoc} */ @@ -131,7 +107,7 @@ public String toString() { * {@inheritDoc} */ @Override - public void subscribeToEvent(EventListener eventListener, Class eventType) { + public final void subscribeToEvent(EventListener eventListener, Class eventType) { this.eventNotificationManager.subscribeToEvent(eventListener, eventType); } @@ -156,11 +132,9 @@ public final void deliverEvent(final AbstractTaskEvent event) { } /** - * Sets the type of the input/output channels which are connected to this gate. - * - * @param channelType - * the type of input/output channels which are connected to this gate + * {@inheritDoc} */ + @Override public final void setChannelType(final ChannelType channelType) { this.channelType = channelType; @@ -176,24 +150,6 @@ public final ChannelType getChannelType() { return this.channelType; } - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - - this.channelType = EnumUtils.readEnum(in, ChannelType.class); - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - - EnumUtils.writeEnum(out, this.channelType); - } - /** * {@inheritDoc} */ @@ -204,18 +160,11 @@ public JobID getJobID() { } /** - * Returns the ID of the gate. - * - * @return the ID of the gate + * {@inheritDoc} */ + @Override public GateID getGateID() { return this.gateID; } - - /** - * Releases the allocated resources (particularly buffer) of all channels attached to this gate. This method - * should only be called after the respected task has stopped running. - */ - public abstract void releaseAllChannelResources(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java index b46e87fccf905..2c25d34abdaf3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java @@ -15,12 +15,8 @@ package eu.stratosphere.nephele.io; -import java.io.DataInput; -import java.io.DataOutput; import java.io.EOFException; import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Iterator; @@ -40,9 +36,6 @@ import eu.stratosphere.nephele.io.compression.CompressionLevel; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.types.StringRecord; -import eu.stratosphere.nephele.util.ClassUtils; -import eu.stratosphere.nephele.util.EnumUtils; /** * In Nephele input gates are a specialization of general gates and connect input channels and record readers. As @@ -58,8 +51,7 @@ * @param * the type of record that can be transported through this gate */ -public class RuntimeInputGate extends AbstractGate implements InputGate, - IOReadableWritable { +public class RuntimeInputGate extends AbstractGate implements InputGate { /** * The log object used for debugging. @@ -161,8 +153,9 @@ public void removeInputChannel(ChannelID inputChannelID) { } /** - * Removes all input channels from the input gate. + * {@inheritDoc} */ + @Override public void removeAllInputChannels() { this.inputChannels.clear(); @@ -212,10 +205,9 @@ public AbstractInputChannel replaceChannel(ChannelID oldChannelID, ChannelTyp } /** - * Returns the {@link DistributionPattern} associated with this input gate. - * - * @return the {@link DistributionPattern} associated with this input gate + * {@inheritDoc} */ + @Override public DistributionPattern getDistributionPattern() { return this.distributionPattern; } @@ -252,15 +244,11 @@ public AbstractInputChannel getInputChannel(int pos) { } /** - * Creates a new network input channel and assigns it to the input gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new network input channel + * {@inheritDoc} */ - public NetworkInputChannel createNetworkInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + @Override + public NetworkInputChannel createNetworkInputChannel(final ChannelID channelID, + final CompressionLevel compressionLevel) { final NetworkInputChannel enic = new NetworkInputChannel(this, this.inputChannels.size(), deserializer, channelID, compressionLevel); @@ -270,15 +258,10 @@ public NetworkInputChannel createNetworkInputChannel(ChannelID channelID, Com } /** - * Creates a new file input channel and assigns it to the input gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new file input channel + * {@inheritDoc} */ - public FileInputChannel createFileInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + @Override + public FileInputChannel createFileInputChannel(final ChannelID channelID, final CompressionLevel compressionLevel) { final FileInputChannel efic = new FileInputChannel(this, this.inputChannels.size(), deserializer, channelID, compressionLevel); @@ -288,15 +271,11 @@ public FileInputChannel createFileInputChannel(ChannelID channelID, Compressi } /** - * Creates a new in-memory input channel and assigns it to the input gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new in-memory input channel + * {@inheritDoc} */ - public InMemoryInputChannel createInMemoryInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + @Override + public InMemoryInputChannel createInMemoryInputChannel(final ChannelID channelID, + final CompressionLevel compressionLevel) { final InMemoryInputChannel eimic = new InMemoryInputChannel(this, this.inputChannels.size(), deserializer, channelID, compressionLevel); @@ -391,87 +370,6 @@ public int waitForAnyChannelToBecomeAvailable() throws InterruptedException { } } - // TODO: See if type safety can be improved here - /** - * {@inheritDoc} - */ - @SuppressWarnings("unchecked") - @Override - public void read(DataInput in) throws IOException { - - super.read(in); - - final int numInputChannels = in.readInt(); - - for (int i = 0; i < numInputChannels; i++) { - - final ChannelID channelID = new ChannelID(); - channelID.read(in); - final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); - - final String className = StringRecord.readString(in); - Class c = null; - try { - c = ClassUtils.getRecordByName(className); - } catch (ClassNotFoundException e) { - LOG.error(e); - } - - if (c == null) { - throw new IOException("Class is null!"); - } - - AbstractInputChannel eic = null; - try { - final Constructor> constructor = (Constructor>) c - .getDeclaredConstructor(InputGate.class, int.class, RecordDeserializer.class, ChannelID.class, - CompressionLevel.class); - if (constructor == null) { - throw new IOException("Constructor is null!"); - } - constructor.setAccessible(true); - eic = constructor.newInstance(this, i, deserializer, channelID, compressionLevel); - } catch (SecurityException e) { - LOG.error(e); - } catch (NoSuchMethodException e) { - LOG.error(e); - } catch (IllegalArgumentException e) { - LOG.error(e); - } catch (InstantiationException e) { - LOG.error(e); - } catch (IllegalAccessException e) { - LOG.error(e); - } catch (InvocationTargetException e) { - LOG.error(e); - } - if (eic == null) { - throw new IOException("Created input channel is null!"); - } - - eic.read(in); - addInputChannel(eic); - } - } - - /** - * {@inheritDoc} - */ - @Override - public void write(DataOutput out) throws IOException { - - super.write(out); - - // Connected input channels - out.writeInt(this.getNumberOfInputChannels()); - for (int i = 0; i < getNumberOfInputChannels(); i++) { - getInputChannel(i).getID().write(out); - EnumUtils.writeEnum(out, getInputChannel(i).getCompressionLevel()); - StringRecord.writeString(out, getInputChannel(i).getClass().getName()); - getInputChannel(i).write(out); - } - - } - /** * {@inheritDoc} */ @@ -489,15 +387,9 @@ public boolean isClosed() throws IOException, InterruptedException { } /** - * Immediately closes the input gate and all its input channels. The corresponding - * output channels are notified. Any remaining records in any buffers or queue is considered - * irrelevant and is discarded. - * - * @throws IOException - * thrown if an I/O error occurs while closing the gate - * @throws InterruptedException - * thrown if the thread is interrupted while waiting for the gate to be closed + * {@inheritDoc} */ + @Override public void close() throws IOException, InterruptedException { for (int i = 0; i < this.getNumberOfInputChannels(); i++) { @@ -512,6 +404,7 @@ public void close() throws IOException, InterruptedException { * * @return the list of InputChannels that feed this RecordReader */ + @Deprecated public List> getInputChannels() { return inputChannels; } @@ -577,4 +470,10 @@ public void releaseAllChannelResources() { it.next().releaseResources(); } } + + @Override + public void activateInputChannels() throws IOException, InterruptedException { + // TODO Auto-generated method stub + + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java index d776750213aeb..040e9a0ac88e6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java @@ -15,11 +15,7 @@ package eu.stratosphere.nephele.io; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -38,9 +34,6 @@ import eu.stratosphere.nephele.io.compression.CompressionLevel; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.types.StringRecord; -import eu.stratosphere.nephele.util.ClassUtils; -import eu.stratosphere.nephele.util.EnumUtils; /** * In Nephele output gates are a specialization of general gates and connect @@ -255,15 +248,11 @@ public AbstractOutputChannel getOutputChannel(int pos) { } /** - * Creates a new network output channel and assigns it to the output gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new network output channel + * {@inheritDoc} */ - public NetworkOutputChannel createNetworkOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + @Override + public NetworkOutputChannel createNetworkOutputChannel(final ChannelID channelID, + final CompressionLevel compressionLevel) { final NetworkOutputChannel enoc = new NetworkOutputChannel(this, this.outputChannels.size(), channelID, compressionLevel); @@ -273,15 +262,11 @@ public NetworkOutputChannel createNetworkOutputChannel(ChannelID channelID, C } /** - * Creates a new file output channel and assigns it to the output gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new file output channel + * {@inheritDoc} */ - public FileOutputChannel createFileOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + @Override + public FileOutputChannel createFileOutputChannel(final ChannelID channelID, + final CompressionLevel compressionLevel) { final FileOutputChannel efoc = new FileOutputChannel(this, this.outputChannels.size(), channelID, compressionLevel); @@ -291,15 +276,11 @@ public FileOutputChannel createFileOutputChannel(ChannelID channelID, Compres } /** - * Creates a new in-memory output channel and assigns it to the output gate. - * - * @param channelID - * the channel ID to assign to the new channel, null to generate a new ID - * @param compressionLevel - * the level of compression to be used for this channel - * @return the new in-memory output channel + * {@inheritDoc} */ - public InMemoryOutputChannel createInMemoryOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + @Override + public InMemoryOutputChannel createInMemoryOutputChannel(final ChannelID channelID, + final CompressionLevel compressionLevel) { final InMemoryOutputChannel einoc = new InMemoryOutputChannel(this, this.outputChannels.size(), channelID, compressionLevel); @@ -309,12 +290,9 @@ public InMemoryOutputChannel createInMemoryOutputChannel(ChannelID channelID, } /** - * Requests the output gate to closed. This means the application will send - * no records through this gate anymore. - * - * @throws IOException - * @throws InterruptedException + * {@inheritDoc} */ + @Override public void requestClose() throws IOException, InterruptedException { // Close all output channels for (int i = 0; i < this.getNumberOfOutputChannels(); i++) { @@ -396,91 +374,6 @@ public void writeRecord(final T record) throws IOException, InterruptedException } } - // TODO: See if type safety can be improved here - /** - * {@inheritDoc} - */ - @SuppressWarnings("unchecked") - public void read(DataInput in) throws IOException { - - super.read(in); - - final int numOutputChannels = in.readInt(); - - final Class[] parameters = { OutputGate.class, int.class, ChannelID.class, CompressionLevel.class }; - - for (int i = 0; i < numOutputChannels; i++) { - - final ChannelID channelID = new ChannelID(); - channelID.read(in); - final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); - final String className = StringRecord.readString(in); - Class c = null; - try { - c = ClassUtils.getRecordByName(className); - } catch (ClassNotFoundException e) { - LOG.error(e); - } - - if (c == null) { - throw new IOException("Class is null!"); - } - - AbstractOutputChannel eoc = null; - try { - final Constructor> constructor = (Constructor>) c - .getDeclaredConstructor(parameters); - - if (constructor == null) { - throw new IOException("Constructor is null!"); - } - - constructor.setAccessible(true); - - eoc = constructor.newInstance(this, i, channelID, compressionLevel); - - } catch (InstantiationException e) { - LOG.error(e); - } catch (IllegalArgumentException e) { - LOG.error(e); - } catch (IllegalAccessException e) { - LOG.error(e); - } catch (InvocationTargetException e) { - LOG.error(e); - } catch (SecurityException e) { - LOG.error(e); - } catch (NoSuchMethodException e) { - LOG.error(e); - } - - if (eoc == null) { - throw new IOException("Created output channel is null!"); - } - - eoc.read(in); - addOutputChannel(eoc); - } - } - - /** - * {@inheritDoc} - */ - public void write(DataOutput out) throws IOException { - - super.write(out); - - // Output channels - out.writeInt(this.getNumberOfOutputChannels()); - - for (int i = 0; i < getNumberOfOutputChannels(); i++) { - getOutputChannel(i).getID().write(out); - EnumUtils.writeEnum(out, getOutputChannel(i).getCompressionLevel()); - StringRecord.writeString(out, getOutputChannel(i).getClass().getName()); - getOutputChannel(i).write(out); - } - - } - /** * {@inheritDoc} */ @@ -490,12 +383,10 @@ public List> getOutputChannels() { } /** - * Registers a new listener object for this output gate. - * - * @param outputGateListener - * the listener object to register + * {@inheritDoc} */ - public void registerOutputGateListener(OutputGateListener outputGateListener) { + @Override + public void registerOutputGateListener(final OutputGateListener outputGateListener) { if (this.outputGateListeners == null) { this.outputGateListeners = new OutputGateListener[1]; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index 91b1686f84c8d..22efd12f4abdd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -43,6 +43,7 @@ import eu.stratosphere.nephele.instance.InstanceManager; import eu.stratosphere.nephele.instance.InstanceRequestMap; import eu.stratosphere.nephele.instance.InstanceType; +import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.jobgraph.JobID; @@ -218,7 +219,7 @@ void findVerticesToBeDeployed(final ExecutionVertex vertex, final int numberOfOutputGates = env.getNumberOfOutputGates(); for (int i = 0; i < numberOfOutputGates; ++i) { - final RuntimeOutputGate outputGate = env.getOutputGate(i); + final OutputGate outputGate = env.getOutputGate(i); boolean deployTarget; switch (outputGate.getChannelType()) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java index 00fccb98aeeaa..54b857801ad0d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java @@ -18,10 +18,10 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.InputGateListener; -import eu.stratosphere.nephele.io.RuntimeOutputGate; +import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.OutputGateListener; -import eu.stratosphere.nephele.io.RuntimeInputGate; import eu.stratosphere.nephele.taskmanager.Task; import eu.stratosphere.nephele.types.Record; @@ -54,7 +54,7 @@ public interface TaskManagerProfiler { * the input gate to register a {@link InputGateListener} object for */ void registerInputGateListener(ExecutionVertexID id, Configuration jobConfiguration, - RuntimeInputGate inputGate); + InputGate inputGate); /** * Registers a {@link OutputGateListener} object for the given output gate. @@ -67,7 +67,7 @@ void registerInputGateListener(ExecutionVertexID id, Configuration jobConfigurat * the output gate to register a {@link InputGateListener} object for */ void registerOutputGateListener(ExecutionVertexID id, Configuration jobConfiguration, - RuntimeOutputGate outputGate); + OutputGate outputGate); /** * Unregisters all previously register {@link ExecutionListener} objects for diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index 626414c9499ae..a17432ac36362 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -33,6 +33,7 @@ import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; @@ -244,7 +245,7 @@ public void initialExecutionResourcesExhausted() { final Map outputChannelUtilization = new HashMap(); for (int i = 0; i < this.environment.getNumberOfOutputGates(); ++i) { - final RuntimeOutputGate outputGate = this.environment.getOutputGate(i); + final OutputGate outputGate = this.environment.getOutputGate(i); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); outputChannelUtilization.put(outputChannel.getID(), diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 13ec268a21e47..c28148493a54c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -558,14 +558,6 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf final RuntimeEnvironment ee = task.getEnvironment(); - // Check if the task has unbound input/output gates - if (ee.hasUnboundInputGates() || ee.hasUnboundOutputGates()) { - LOG.debug("Task with ID " + id + " has unbound gates"); - TaskSubmissionResult result = new TaskSubmissionResult(id, AbstractTaskResult.ReturnCode.ERROR); - result.setDescription("Task with ID " + id + " has unbound gates"); - return result; - } - // Register the task with the byte buffered channel manager this.byteBufferedChannelManager.register(task, activeOutputChannels); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 10974a54db980..cc40f54ef4128 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -32,6 +32,8 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; import eu.stratosphere.nephele.io.AbstractID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RuntimeInputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractChannel; @@ -139,7 +141,7 @@ public void register(final Task task, final Set activeOutputChannels) final TaskContext taskContext = new TaskContext(task, this, this.tasksWithUndecidedCheckpoints); for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - final RuntimeOutputGate outputGate = environment.getOutputGate(i); + final OutputGate outputGate = environment.getOutputGate(i); final OutputGateContext outputGateContext = new OutputGateContext(taskContext, outputGate.getChannelType(), outputGate.getIndex()); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { @@ -174,7 +176,7 @@ public void register(final Task task, final Set activeOutputChannels) } for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { - final RuntimeInputGate inputGate = environment.getInputGate(i); + final InputGate inputGate = environment.getInputGate(i); final InputGateContext inputGateContext = new InputGateContext(inputGate.getNumberOfInputChannels()); for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); @@ -225,7 +227,7 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { final RuntimeEnvironment environment = task.getEnvironment(); for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - final RuntimeOutputGate outputGate = environment.getOutputGate(i); + final OutputGate outputGate = environment.getOutputGate(i); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); this.registeredChannels.remove(outputChannel.getID()); @@ -234,7 +236,7 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { } for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { - final RuntimeInputGate inputGate = environment.getInputGate(i); + final InputGate inputGate = environment.getInputGate(i); for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); this.registeredChannels.remove(inputChannel.getID()); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/InputGateWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/InputGateWrapper.java new file mode 100644 index 0000000000000..f407bf9603f34 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/InputGateWrapper.java @@ -0,0 +1,216 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import java.io.IOException; + +import eu.stratosphere.nephele.event.task.AbstractTaskEvent; +import eu.stratosphere.nephele.event.task.EventListener; +import eu.stratosphere.nephele.io.DistributionPattern; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.InputGateListener; +import eu.stratosphere.nephele.io.channels.AbstractInputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.channels.bytebuffered.FileInputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryInputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkInputChannel; +import eu.stratosphere.nephele.io.compression.CompressionLevel; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.types.Record; + +public final class InputGateWrapper implements InputGate { + + private final TaskWrapper taskWrapper; + + private final InputGate wrappedInputGate; + + InputGateWrapper(final TaskWrapper taskWrapper, final InputGate wrappedInputGate) { + + this.taskWrapper = taskWrapper; + this.wrappedInputGate = wrappedInputGate; + } + + /** + * {@inheritDoc} + */ + @Override + public void publishEvent(final AbstractTaskEvent event) throws IOException, InterruptedException { + + this.wrappedInputGate.publishEvent(event); + } + + @Override + public T readRecord(final T target) throws IOException, InterruptedException { + + // TODO Auto-generated method stub + return null; + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfInputChannels() { + + return this.wrappedInputGate.getNumberOfInputChannels(); + } + + /** + * {@inheritDoc} + */ + @Override + public void notifyRecordIsAvailable(final int channelIndex) { + + throw new IllegalStateException("notifyRecordIsAvailable is called on InputGateWrapper"); + } + + /** + * {@inheritDoc} + */ + @Override + public void subscribeToEvent(final EventListener eventListener, final Class eventType) { + + this.wrappedInputGate.subscribeToEvent(eventListener, eventType); + } + + /** + * {@inheritDoc} + */ + @Override + public void unsubscribeFromEvent(EventListener eventListener, Class eventType) { + + this.wrappedInputGate.unsubscribeFromEvent(eventListener, eventType); + } + + /** + * {@inheritDoc} + */ + @Override + public void deliverEvent(final AbstractTaskEvent event) { + + throw new IllegalStateException("deliverEvent is called on InputGateWrapper"); + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + + return this.wrappedInputGate.getJobID(); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelType getChannelType() { + + return this.wrappedInputGate.getChannelType(); + } + + @Override + public GateID getGateID() { + // TODO Auto-generated method stub + return null; + } + + @Override + public void releaseAllChannelResources() { + // TODO Auto-generated method stub + + } + + @Override + public boolean isClosed() throws IOException, InterruptedException { + // TODO Auto-generated method stub + return false; + } + + @Override + public boolean isInputGate() { + // TODO Auto-generated method stub + return false; + } + + @Override + public void activateInputChannels() throws IOException, InterruptedException { + // TODO Auto-generated method stub + + } + + @Override + public AbstractInputChannel getInputChannel(int pos) { + // TODO Auto-generated method stub + return null; + } + + @Override + public void close() throws IOException, InterruptedException { + // TODO Auto-generated method stub + + } + + @Override + public void registerInputGateListener(InputGateListener inputGateListener) { + // TODO Auto-generated method stub + + } + + @Override + public void setChannelType(ChannelType channelType) { + // TODO Auto-generated method stub + + } + + @Override + public DistributionPattern getDistributionPattern() { + // TODO Auto-generated method stub + return null; + } + + @Override + public NetworkInputChannel createNetworkInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + // TODO Auto-generated method stub + return null; + } + + @Override + public FileInputChannel createFileInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + // TODO Auto-generated method stub + return null; + } + + @Override + public InMemoryInputChannel createInMemoryInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + // TODO Auto-generated method stub + return null; + } + + @Override + public int getIndex() { + // TODO Auto-generated method stub + return 0; + } + + @Override + public void removeAllInputChannels() { + // TODO Auto-generated method stub + + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/OutputGateWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/OutputGateWrapper.java new file mode 100644 index 0000000000000..3a1609af64707 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/OutputGateWrapper.java @@ -0,0 +1,253 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import java.io.IOException; +import java.util.List; + +import eu.stratosphere.nephele.event.task.AbstractTaskEvent; +import eu.stratosphere.nephele.event.task.EventListener; +import eu.stratosphere.nephele.io.ChannelSelector; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.OutputGateListener; +import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.channels.bytebuffered.FileOutputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryOutputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkOutputChannel; +import eu.stratosphere.nephele.io.compression.CompressionLevel; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.types.Record; + +public final class OutputGateWrapper implements OutputGate { + + private final TaskWrapper taskWrapper; + + private final OutputGate wrappedOutputGate; + + OutputGateWrapper(final TaskWrapper taskWrapper, final OutputGate wrappedOutputGate) { + + this.taskWrapper = taskWrapper; + this.wrappedOutputGate = wrappedOutputGate; + } + + /** + * {@inheritDoc} + */ + @Override + public void subscribeToEvent(final EventListener eventListener, final Class eventType) { + + this.wrappedOutputGate.subscribeToEvent(eventListener, eventType); + } + + /** + * {@inheritDoc} + */ + @Override + public void unsubscribeFromEvent(final EventListener eventListener, + final Class eventType) { + + this.wrappedOutputGate.unsubscribeFromEvent(eventListener, eventType); + } + + /** + * {@inheritDoc} + */ + @Override + public void publishEvent(final AbstractTaskEvent event) throws IOException, InterruptedException { + + this.wrappedOutputGate.publishEvent(event); + } + + /** + * {@inheritDoc} + */ + @Override + public void deliverEvent(final AbstractTaskEvent event) { + + throw new IllegalStateException("deliverEvent is called on OutputGateWrapper"); + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + + return this.wrappedOutputGate.getJobID(); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelType getChannelType() { + + return this.wrappedOutputGate.getChannelType(); + } + + /** + * {@inheritDoc} + */ + @Override + public Class getType() { + + return this.wrappedOutputGate.getType(); + } + + /** + * {@inheritDoc} + */ + @Override + public void writeRecord(final T record) throws IOException, InterruptedException { + // TODO Auto-generated method stub + this.wrappedOutputGate.writeRecord(record); + } + + /** + * {@inheritDoc} + */ + @Override + public List> getOutputChannels() { + + return this.wrappedOutputGate.getOutputChannels(); + } + + /** + * {@inheritDoc} + */ + @Override + public void flush() throws IOException, InterruptedException { + + this.wrappedOutputGate.flush(); + } + + /** + * {@inheritDoc} + */ + @Override + public void channelCapacityExhausted(final int channelIndex) { + + this.wrappedOutputGate.channelCapacityExhausted(channelIndex); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isBroadcast() { + + return this.wrappedOutputGate.isBroadcast(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfOutputChannels() { + + return this.wrappedOutputGate.getNumberOfOutputChannels(); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelSelector getChannelSelector() { + + return this.wrappedOutputGate.getChannelSelector(); + } + + @Override + public GateID getGateID() { + // TODO Auto-generated method stub + return null; + } + + @Override + public void releaseAllChannelResources() { + // TODO Auto-generated method stub + + } + + @Override + public boolean isClosed() throws IOException, InterruptedException { + // TODO Auto-generated method stub + return false; + } + + @Override + public boolean isInputGate() { + // TODO Auto-generated method stub + return false; + } + + @Override + public AbstractOutputChannel getOutputChannel(int pos) { + // TODO Auto-generated method stub + return null; + } + + @Override + public void requestClose() throws IOException, InterruptedException { + // TODO Auto-generated method stub + + } + + @Override + public void removeAllOutputChannels() { + // TODO Auto-generated method stub + + } + + @Override + public void setChannelType(ChannelType channelType) { + // TODO Auto-generated method stub + + } + + @Override + public NetworkOutputChannel createNetworkOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + // TODO Auto-generated method stub + return null; + } + + @Override + public FileOutputChannel createFileOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + // TODO Auto-generated method stub + return null; + } + + @Override + public InMemoryOutputChannel createInMemoryOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { + // TODO Auto-generated method stub + return null; + } + + @Override + public int getIndex() { + // TODO Auto-generated method stub + return 0; + } + + @Override + public void registerOutputGateListener(OutputGateListener outputGateListener) { + // TODO Auto-generated method stub + + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index e1f9841ebe91b..5f970bfaff545 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -16,7 +16,6 @@ package eu.stratosphere.nephele.streaming; import java.io.IOException; -import java.util.Iterator; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -24,14 +23,8 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.io.IOReadableWritable; -import eu.stratosphere.nephele.jobgraph.AbstractJobInputVertex; -import eu.stratosphere.nephele.jobgraph.AbstractJobVertex; import eu.stratosphere.nephele.jobgraph.JobGraph; -import eu.stratosphere.nephele.jobgraph.JobTaskVertex; import eu.stratosphere.nephele.plugins.JobManagerPlugin; -import eu.stratosphere.nephele.streaming.wrapper.StreamingTask; -import eu.stratosphere.nephele.template.AbstractInvokable; -import eu.stratosphere.nephele.template.AbstractTask; public class StreamingJobManagerPlugin implements JobManagerPlugin { @@ -59,8 +52,8 @@ public JobGraph rewriteJobGraph(final JobGraph jobGraph) { final Class originalClass = taskVertex.getInvokableClass(); - taskVertex.setTaskClass(StreamingTask.class); - taskVertex.getConfiguration().setString("origClass", originalClass.getName()); + taskVertex.setTaskClass(TaskWrapper.class); + taskVertex.getConfiguration().setString(TaskWrapper.WRAPPED_CLASS_KEY, originalClass.getName()); }*/ // TODO Auto-generated method stub diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index 1814fceb0960f..d1c9f76ae329b 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -21,6 +21,8 @@ import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RuntimeInputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.jobgraph.JobID; @@ -114,12 +116,12 @@ public void registerTask(final ExecutionVertexID id, final Configuration jobConf } for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - final RuntimeOutputGate outputGate = environment.getOutputGate(i); + final OutputGate outputGate = environment.getOutputGate(i); outputGate.registerOutputGateListener(listener); } for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { - final RuntimeInputGate inputGate = environment.getInputGate(i); + final InputGate inputGate = environment.getInputGate(i); inputGate.registerInputGateListener(listener); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskWrapper.java new file mode 100644 index 0000000000000..1d0a5d025df73 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskWrapper.java @@ -0,0 +1,207 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; +import eu.stratosphere.nephele.io.ChannelSelector; +import eu.stratosphere.nephele.io.DistributionPattern; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RecordDeserializer; +import eu.stratosphere.nephele.io.RuntimeInputGate; +import eu.stratosphere.nephele.io.RuntimeOutputGate; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.services.iomanager.IOManager; +import eu.stratosphere.nephele.services.memorymanager.MemoryManager; +import eu.stratosphere.nephele.template.AbstractInvokable; +import eu.stratosphere.nephele.template.AbstractTask; +import eu.stratosphere.nephele.template.InputSplitProvider; +import eu.stratosphere.nephele.types.Record; +import eu.stratosphere.nephele.util.StringUtils; + +public final class TaskWrapper extends AbstractTask implements Environment { + + static final String WRAPPED_CLASS_KEY = "streaming.class.name"; + + private AbstractInvokable wrappedInvokable = null; + + private synchronized AbstractInvokable getWrappedInvokable() { + + if (this.wrappedInvokable != null) { + return this.wrappedInvokable; + } + + final Configuration conf = getEnvironment().getRuntimeConfiguration(); + final JobID jobID = getEnvironment().getJobID(); + final String className = conf.getString(WRAPPED_CLASS_KEY, null); + if (className == null) { + throw new IllegalStateException("Cannot find name of wrapped class"); + } + + try { + final ClassLoader cl = LibraryCacheManager.getClassLoader(jobID); + + @SuppressWarnings("unchecked") + final Class invokableClass = (Class) Class + .forName(className, true, cl); + + this.wrappedInvokable = invokableClass.newInstance(); + } catch (Exception e) { + throw new RuntimeException(StringUtils.stringifyException(e)); + } + + this.wrappedInvokable.setEnvironment(this); + + return this.wrappedInvokable; + } + + /** + * {@inheritDoc} + */ + @Override + public void registerInputOutput() { + + getWrappedInvokable().registerInputOutput(); + } + + /** + * {@inheritDoc} + */ + @Override + public void invoke() throws Exception { + + getWrappedInvokable().invoke(); + + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + + return getEnvironment().getJobID(); + } + + /** + * {@inheritDoc} + */ + @Override + public InputSplitProvider getInputSplitProvider() { + + return getEnvironment().getInputSplitProvider(); + } + + /** + * {@inheritDoc} + */ + @Override + public IOManager getIOManager() { + + return getEnvironment().getIOManager(); + } + + /** + * {@inheritDoc} + */ + @Override + public MemoryManager getMemoryManager() { + + return getEnvironment().getMemoryManager(); + } + + /** + * {@inheritDoc} + */ + @Override + public String getTaskName() { + + return getEnvironment().getTaskName(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfOutputGates() { + + return getEnvironment().getNumberOfOutputGates(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfInputGates() { + + return getEnvironment().getNumberOfInputGates(); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerOutputGate(OutputGate outputGate) { + + getEnvironment().registerOutputGate(outputGate); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerInputGate(InputGate inputGate) { + + getEnvironment().registerInputGate(inputGate); + } + + /** + * {@inheritDoc} + */ + @Override + public OutputGate createOutputGate(final GateID gateID, + final Class outputClass, final ChannelSelector selector, + final boolean isBroadcast) { + + return getEnvironment().createOutputGate(gateID, outputClass, selector, isBroadcast); + } + + /** + * {@inheritDoc} + */ + @Override + public InputGate createInputGate(final GateID gateID, + final RecordDeserializer deserializer, + final DistributionPattern distributionPattern) { + + return getEnvironment().createInputGate(gateID, deserializer, distributionPattern); + } + + @Override + public GateID getNextUnboundInputGateID() { + // TODO Auto-generated method stub + return null; + } + + @Override + public GateID getNextUnboundOutputGateID() { + // TODO Auto-generated method stub + return null; + } +} diff --git a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java index d8308e6c0c0bb..29356e0376255 100644 --- a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java +++ b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java @@ -23,14 +23,12 @@ import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.io.DefaultRecordDeserializer; import eu.stratosphere.nephele.io.GateID; -import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.RuntimeInputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager; -import eu.stratosphere.nephele.types.Record; import eu.stratosphere.pact.common.type.PactRecord; import eu.stratosphere.pact.common.util.MutableObjectIterator; @@ -72,26 +70,6 @@ public Configuration getRuntimeConfiguration() { return this.config; } - @Override - public boolean hasUnboundInputGates() { - return this.inputs.size() > 0 ? true : false; - } - - @Override - public boolean hasUnboundOutputGates() { - return this.outputs.size() > 0 ? true : false; - } - - @Override - public InputGate getUnboundInputGate(int gateID) { - return inputs.remove(gateID); - } - - @Override - public eu.stratosphere.nephele.io.OutputGate getUnboundOutputGate(int gateID) { - return outputs.remove(gateID); - } - @Override public MemoryManager getMemoryManager() { return this.memManager; From d0b38e4d55fc71b3f795bf15c31997ebc6feeda9 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 15:24:08 +0000 Subject: [PATCH 065/310] Added computation of task latency and propagation thereof to the job manager --- .../nephele/streaming/ChannelLatency.java | 92 +++++++------- .../streaming/StreamingTaskListener.java | 42 ++++++- .../nephele/streaming/TaskLatency.java | 112 ++++++++++++++++++ .../streaming/latency/LatencyModel.java | 12 +- .../latency/LatencyOptimizerThread.java | 5 + 5 files changed, 204 insertions(+), 59 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskLatency.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/ChannelLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/ChannelLatency.java index a164731898968..20d327e69bafa 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/ChannelLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/ChannelLatency.java @@ -23,55 +23,55 @@ import eu.stratosphere.nephele.jobgraph.JobID; /** - * This class stores information about the latency of a specific (sub) path from a start to an end vertex. + * This class stores information about the latency of a specific channel from a source to a sink vertex. * * @author warneke */ public final class ChannelLatency extends AbstractStreamingData { /** - * The ID of the vertex representing the start of the path. + * The ID of the vertex representing the source of the channel. */ - private final ExecutionVertexID startVertexID; + private final ExecutionVertexID sourceVertexID; /** - * The ID of the vertex representing the end of the path. + * The ID of the vertex representing the sink of the channel. */ - private final ExecutionVertexID endVertexID; + private final ExecutionVertexID sinkVertexID; /** - * The path latency in milliseconds + * The channel latency in milliseconds */ - private double pathLatency; + private double channelLatency; /** * Constructs a new path latency object. * * @param jobID - * the ID of the job this path latency information refers to - * @param startVertexID - * the ID of the vertex representing the start of the path - * @param endVertexID - * the ID of the vertex representing the end of the path + * the ID of the job this channel latency information refers to + * @param sourceVertexID + * the ID of the vertex representing the source of the channel + * @param sinkVertexID + * the ID of the vertex representing the sink of the channel * @param pathLatency * the path latency in milliseconds */ - public ChannelLatency(final JobID jobID, final ExecutionVertexID startVertexID, final ExecutionVertexID endVertexID, - final double pathLatency) { + public ChannelLatency(final JobID jobID, final ExecutionVertexID sourceVertexID, + final ExecutionVertexID sinkVertexID, final double channelLatency) { super(jobID); - if (startVertexID == null) { - throw new IllegalArgumentException("sourceID must not be null"); + if (sourceVertexID == null) { + throw new IllegalArgumentException("sourceVertexID must not be null"); } - if (endVertexID == null) { - throw new IllegalArgumentException("targetID must not be null"); + if (sinkVertexID == null) { + throw new IllegalArgumentException("sinkVertexID must not be null"); } - this.startVertexID = startVertexID; - this.endVertexID = endVertexID; - this.pathLatency = pathLatency; + this.sourceVertexID = sourceVertexID; + this.sinkVertexID = sinkVertexID; + this.channelLatency = channelLatency; } /** @@ -79,9 +79,9 @@ public ChannelLatency(final JobID jobID, final ExecutionVertexID startVertexID, */ public ChannelLatency() { super(new JobID()); - this.startVertexID = new ExecutionVertexID(); - this.endVertexID = new ExecutionVertexID(); - this.pathLatency = 0.0; + this.sourceVertexID = new ExecutionVertexID(); + this.sinkVertexID = new ExecutionVertexID(); + this.channelLatency = 0.0; } /** @@ -90,9 +90,9 @@ public ChannelLatency() { @Override public void write(final DataOutput out) throws IOException { super.write(out); - this.startVertexID.write(out); - this.endVertexID.write(out); - out.writeDouble(this.pathLatency); + this.sourceVertexID.write(out); + this.sinkVertexID.write(out); + out.writeDouble(this.channelLatency); } /** @@ -101,41 +101,39 @@ public void write(final DataOutput out) throws IOException { @Override public void read(final DataInput in) throws IOException { super.read(in); - this.startVertexID.read(in); - this.endVertexID.read(in); - this.pathLatency = in.readDouble(); + this.sourceVertexID.read(in); + this.sinkVertexID.read(in); + this.channelLatency = in.readDouble(); } - - /** - * Returns the ID of the vertex representing the start of the path. + * Returns the ID of the vertex representing the source of the channel. * - * @return the ID of the vertex representing the start of the path + * @return the ID of the vertex representing the source of the channel */ - public ExecutionVertexID getStartVertexID() { + public ExecutionVertexID getSourceVertexID() { - return this.startVertexID; + return this.sourceVertexID; } /** - * Returns the ID of the vertex representing the end of the path. + * Returns the ID of the vertex representing the sink of the channel. * - * @return the ID of the vertex representing the end of the path + * @return the ID of the vertex representing the sink of the channel */ - public ExecutionVertexID getEndVertexID() { + public ExecutionVertexID getSinkVertexID() { - return this.endVertexID; + return this.sinkVertexID; } /** - * Returns the path latency in milliseconds. + * Returns the channel latency in milliseconds. * - * @return the path latency in milliseconds + * @return the channel latency in milliseconds */ - public double getPathLatency() { + public double getChannelLatency() { - return this.pathLatency; + return this.channelLatency; } /** @@ -145,11 +143,11 @@ public double getPathLatency() { public String toString() { final StringBuilder str = new StringBuilder(); - str.append(this.startVertexID.toString()); + str.append(this.sourceVertexID.toString()); str.append(" -> "); - str.append(this.endVertexID.toString()); + str.append(this.sinkVertexID.toString()); str.append(": "); - str.append(this.pathLatency); + str.append(this.channelLatency); return str.toString(); } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java index cfc8d6c7d251a..c10d335fde9f7 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java @@ -58,6 +58,8 @@ private static enum TaskType { private int tagCounter = 0; + private long lastTimestamp = -1L; + private Map aggregationCounter = new HashMap(); private Map aggregatedValue = new HashMap(); @@ -112,18 +114,31 @@ public void recordEmitted(final Record record) { switch (this.taskType) { case INPUT: if (this.tagCounter++ == this.taggingInterval) { + final long timestamp = System.currentTimeMillis(); final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - taggableRecord.setTag(createTag()); + taggableRecord.setTag(createTag(timestamp)); + if (this.lastTimestamp > 0) { + final long taskLatency = (timestamp - this.lastTimestamp) / this.taggingInterval; + try { + this.communicationThread.sendDataAsynchronously(new TaskLatency(this.jobID, this.vertexID, + taskLatency)); + } catch (InterruptedException e) { + LOG.error(StringUtils.stringifyException(e)); + } + } + this.lastTimestamp = timestamp; this.tagCounter = 0; } break; case REGULAR: final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - if(this.tag == null) { + if (this.tag == null) { taggableRecord.setTag(null); } else { - this.tag = createTag(); + final long timestamp = System.currentTimeMillis(); + this.tag = createTag(timestamp); taggableRecord.setTag(this.tag); + this.lastTimestamp = timestamp; } break; case OUTPUT: @@ -155,7 +170,22 @@ public void recordReceived(final Record record) { this.tag = (StreamingTag) taggableRecord.getTag(); if (this.tag != null) { - final long pathLatency = System.currentTimeMillis() - this.tag.getTimestamp(); + final long timestamp = System.currentTimeMillis(); + if (this.lastTimestamp > 0) { + try { + this.communicationThread.sendDataAsynchronously(new TaskLatency(jobID, vertexID, timestamp + - this.lastTimestamp)); + } catch (InterruptedException e) { + LOG.error(StringUtils.stringifyException(e)); + } + if (this.taskType == TaskType.REGULAR) { + this.lastTimestamp = -1L; + } else { + this.lastTimestamp = timestamp; + } + } + + final long pathLatency = timestamp - this.tag.getTimestamp(); final ExecutionVertexID sourceID = this.tag.getSourceID(); @@ -194,10 +224,10 @@ public void recordReceived(final Record record) { } - private StreamingTag createTag() { + private StreamingTag createTag(final long timestamp) { this.tag = new StreamingTag(this.vertexID); - this.tag.setTimestamp(System.currentTimeMillis()); + this.tag.setTimestamp(timestamp); return this.tag; } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskLatency.java new file mode 100644 index 0000000000000..a6f0e4740b40a --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskLatency.java @@ -0,0 +1,112 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.jobgraph.JobID; + +/** + * This class stores information about the latency of a specific (sub) path from a start to an end vertex. + * + * @author warneke + */ +public final class TaskLatency extends AbstractStreamingData { + + /** + * The ID of the vertex this task latency information refers to + */ + private final ExecutionVertexID vertexID; + + /** + * The task latency in milliseconds + */ + private double taskLatency; + + /** + * Constructs a new task latency object. + * + * @param jobID + * the ID of the job this path latency information refers to + * @param vertexID + * the ID of the vertex this task latency information refers to + * @param taskLatency + * the task latency in milliseconds + */ + public TaskLatency(final JobID jobID, final ExecutionVertexID vertexID, final double taskLatency) { + + super(jobID); + + if (vertexID == null) { + throw new IllegalArgumentException("vertexID must not be null"); + } + + this.vertexID = vertexID; + this.taskLatency = taskLatency; + } + + /** + * Default constructor for the deserialization of the object. + */ + public TaskLatency() { + super(new JobID()); + this.vertexID = new ExecutionVertexID(); + this.taskLatency = 0.0; + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + super.write(out); + this.vertexID.write(out); + out.writeDouble(this.taskLatency); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + super.read(in); + this.vertexID.read(in); + this.taskLatency = in.readDouble(); + } + + /** + * Returns the ID of the vertex this task latency information refers to. + * + * @return the ID of the vertex this task latency information refers to + */ + public ExecutionVertexID getVertexID() { + + return this.vertexID; + } + + /** + * Returns the task latency in milliseconds. + * + * @return the task latency in milliseconds + */ + public double getTaskLatency() { + + return this.taskLatency; + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java index f235094878e20..3a8f88c2d770e 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java @@ -29,20 +29,20 @@ public LatencyModel(ExecutionGraph executionGraph) { } public void refreshEdgeLatency(ChannelLatency latency) { - ExecutionVertexID startID = latency.getStartVertexID(); - ExecutionVertexID endID = latency.getEndVertexID(); + ExecutionVertexID startID = latency.getSourceVertexID(); + ExecutionVertexID endID = latency.getSinkVertexID(); if (startID.equals(endID)) { System.out.println("new vertex latency"); VertexLatency vertexLatency = latencySubgraph.getVertexLatency(startID.toManagementVertexID()); - vertexLatency.setLatencyInMillis(latency.getPathLatency()); + vertexLatency.setLatencyInMillis(latency.getChannelLatency()); } else { System.out.println("new edge latency"); - ManagementEdgeID edgeID = new ManagementEdgeID(latency.getStartVertexID().toManagementVertexID(), - latency.getEndVertexID().toManagementVertexID()); + ManagementEdgeID edgeID = new ManagementEdgeID(latency.getSourceVertexID().toManagementVertexID(), + latency.getSinkVertexID().toManagementVertexID()); EdgeLatency edgeLatency = latencySubgraph.getEdgeLatency(edgeID); - edgeLatency.setLatencyInMillis(latency.getPathLatency()); + edgeLatency.setLatencyInMillis(latency.getChannelLatency()); for (LatencyPath path : latencySubgraph.getLatencyPaths()) { path.dumpLatencies(); } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java index 9baba1be92330..2536c98707e2d 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java @@ -8,6 +8,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.streaming.AbstractStreamingData; import eu.stratosphere.nephele.streaming.ChannelLatency; +import eu.stratosphere.nephele.streaming.TaskLatency; public class LatencyOptimizerThread extends Thread { @@ -34,7 +35,11 @@ public void run() { if (streamingData instanceof ChannelLatency) { latencyModel.refreshEdgeLatency((ChannelLatency) streamingData); + } else if(streamingData instanceof TaskLatency) { + TaskLatency tl = (TaskLatency) streamingData; + System.out.println("Task latency: " + tl.getTaskLatency()); } + } From 7a2cae991400f7632bc77f31a6de7ed6af3c9ced Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 15:53:46 +0000 Subject: [PATCH 066/310] Fixed several POM file-related warnings reported by the eclipse maven plugin --- nephele/nephele-clustermanager/pom.xml | 2 -- nephele/nephele-common/pom.xml | 2 -- nephele/nephele-ec2cloudmanager/pom.xml | 3 --- nephele/nephele-examples/pom.xml | 2 -- nephele/nephele-hdfs/pom.xml | 2 -- nephele/nephele-management/pom.xml | 2 -- nephele/nephele-profiling/pom.xml | 2 -- nephele/nephele-queuescheduler/pom.xml | 2 -- nephele/nephele-s3/pom.xml | 2 -- nephele/nephele-server/pom.xml | 2 -- nephele/nephele-streaming/pom.xml | 2 -- nephele/nephele-visualization/pom.xml | 2 -- nephele/pom.xml | 2 -- pact/pact-clients/pom.xml | 2 -- pact/pact-common/pom.xml | 2 -- pact/pact-compiler/pom.xml | 2 -- pact/pact-examples/pom.xml | 2 -- pact/pact-runtime/pom.xml | 2 -- pact/pact-tests/pom.xml | 2 -- pact/pom.xml | 2 -- score/pom.xml | 2 -- stratosphere-dist/pom.xml | 2 -- 22 files changed, 45 deletions(-) diff --git a/nephele/nephele-clustermanager/pom.xml b/nephele/nephele-clustermanager/pom.xml index 4d609a34ffa05..d7102040d6a0c 100644 --- a/nephele/nephele-clustermanager/pom.xml +++ b/nephele/nephele-clustermanager/pom.xml @@ -8,10 +8,8 @@ 4.0.0 - eu.stratosphere nephele-clustermanager nephele-clustermanager - 0.2 http://maven.apache.org diff --git a/nephele/nephele-common/pom.xml b/nephele/nephele-common/pom.xml index 0cd1d9c8c2e17..5174a87943eb3 100644 --- a/nephele/nephele-common/pom.xml +++ b/nephele/nephele-common/pom.xml @@ -9,9 +9,7 @@ 0.2 - eu.stratosphere nephele-common - 0.2 nephele-common jar diff --git a/nephele/nephele-ec2cloudmanager/pom.xml b/nephele/nephele-ec2cloudmanager/pom.xml index 302d076d6d6d6..a211d06d17c17 100644 --- a/nephele/nephele-ec2cloudmanager/pom.xml +++ b/nephele/nephele-ec2cloudmanager/pom.xml @@ -10,13 +10,10 @@ 0.2 - eu.stratosphere nephele-ec2cloudmanager nephele-ec2cloudmanager jar - 0.2 - nephele-server diff --git a/nephele/nephele-examples/pom.xml b/nephele/nephele-examples/pom.xml index f7ded8c979934..5ea7963bbde42 100644 --- a/nephele/nephele-examples/pom.xml +++ b/nephele/nephele-examples/pom.xml @@ -10,9 +10,7 @@ 0.2 - eu.stratosphere nephele-examples - 0.2 nephele-examples jar diff --git a/nephele/nephele-hdfs/pom.xml b/nephele/nephele-hdfs/pom.xml index d5af8274b73a2..f4eda194ec4b0 100644 --- a/nephele/nephele-hdfs/pom.xml +++ b/nephele/nephele-hdfs/pom.xml @@ -10,9 +10,7 @@ 0.2 - eu.stratosphere nephele-hdfs - 0.2 nephele-hdfs jar diff --git a/nephele/nephele-management/pom.xml b/nephele/nephele-management/pom.xml index d07c07f32ce38..943e3f2d80ba7 100644 --- a/nephele/nephele-management/pom.xml +++ b/nephele/nephele-management/pom.xml @@ -11,9 +11,7 @@ 0.2 - eu.stratosphere nephele-management - 0.2 nephele-management jar diff --git a/nephele/nephele-profiling/pom.xml b/nephele/nephele-profiling/pom.xml index e6ba97b996d47..eaaa0ad8e05fc 100644 --- a/nephele/nephele-profiling/pom.xml +++ b/nephele/nephele-profiling/pom.xml @@ -11,9 +11,7 @@ 0.2 - eu.stratosphere nephele-profiling - 0.2 nephele-profiling jar diff --git a/nephele/nephele-queuescheduler/pom.xml b/nephele/nephele-queuescheduler/pom.xml index 86e94c22c6641..eec489066d122 100644 --- a/nephele/nephele-queuescheduler/pom.xml +++ b/nephele/nephele-queuescheduler/pom.xml @@ -6,10 +6,8 @@ 0.2 4.0.0 - eu.stratosphere nephele-queuescheduler nephele-queuescheduler - 0.2 http://maven.apache.org diff --git a/nephele/nephele-s3/pom.xml b/nephele/nephele-s3/pom.xml index 06f4623609de8..7d193e008f556 100644 --- a/nephele/nephele-s3/pom.xml +++ b/nephele/nephele-s3/pom.xml @@ -10,9 +10,7 @@ 0.2 - eu.stratosphere nephele-s3 - 0.2 nephele-s3 jar diff --git a/nephele/nephele-server/pom.xml b/nephele/nephele-server/pom.xml index 2842a722cf08c..c15cd0104114c 100644 --- a/nephele/nephele-server/pom.xml +++ b/nephele/nephele-server/pom.xml @@ -9,9 +9,7 @@ 0.2 - eu.stratosphere nephele-server - 0.2 nephele-server jar diff --git a/nephele/nephele-streaming/pom.xml b/nephele/nephele-streaming/pom.xml index cf03276bc8e21..431b139fb11b7 100644 --- a/nephele/nephele-streaming/pom.xml +++ b/nephele/nephele-streaming/pom.xml @@ -10,9 +10,7 @@ 0.2 - eu.stratosphere nephele-streaming - 0.2 nephele-streaming jar diff --git a/nephele/nephele-visualization/pom.xml b/nephele/nephele-visualization/pom.xml index 689d71c6a49d5..d12106f8077a4 100644 --- a/nephele/nephele-visualization/pom.xml +++ b/nephele/nephele-visualization/pom.xml @@ -11,9 +11,7 @@ 0.2 - eu.stratosphere nephele-visualization - 0.2 nephele-visualization jar diff --git a/nephele/pom.xml b/nephele/pom.xml index d15ab598de44c..84ec69a862671 100644 --- a/nephele/pom.xml +++ b/nephele/pom.xml @@ -9,12 +9,10 @@ 0.2 - eu.stratosphere nephele pom - 0.2 nephele http://maven.apache.org diff --git a/pact/pact-clients/pom.xml b/pact/pact-clients/pom.xml index dc14926dd4506..be4d4bca3734a 100644 --- a/pact/pact-clients/pom.xml +++ b/pact/pact-clients/pom.xml @@ -10,9 +10,7 @@ 0.2 - eu.stratosphere pact-clients - 0.2 pact-clients jar diff --git a/pact/pact-common/pom.xml b/pact/pact-common/pom.xml index a11aaa9c5c972..a87f6faee16ac 100644 --- a/pact/pact-common/pom.xml +++ b/pact/pact-common/pom.xml @@ -9,9 +9,7 @@ 0.2 - eu.stratosphere pact-common - 0.2 pact-common jar diff --git a/pact/pact-compiler/pom.xml b/pact/pact-compiler/pom.xml index a0b9767f1070e..1646a73403f1d 100644 --- a/pact/pact-compiler/pom.xml +++ b/pact/pact-compiler/pom.xml @@ -10,9 +10,7 @@ 0.2 - eu.stratosphere pact-compiler - 0.2 pact-compiler jar diff --git a/pact/pact-examples/pom.xml b/pact/pact-examples/pom.xml index a9a45ecc388fe..5c21d1281e84a 100644 --- a/pact/pact-examples/pom.xml +++ b/pact/pact-examples/pom.xml @@ -10,9 +10,7 @@ 0.2 - eu.stratosphere pact-examples - 0.2 pact-examples jar diff --git a/pact/pact-runtime/pom.xml b/pact/pact-runtime/pom.xml index 81c20e2d6ad8f..3d20753a887bf 100644 --- a/pact/pact-runtime/pom.xml +++ b/pact/pact-runtime/pom.xml @@ -10,9 +10,7 @@ 0.2 - eu.stratosphere pact-runtime - 0.2 pact-runtime jar diff --git a/pact/pact-tests/pom.xml b/pact/pact-tests/pom.xml index 2962c29d8674a..225fbecb635ca 100644 --- a/pact/pact-tests/pom.xml +++ b/pact/pact-tests/pom.xml @@ -9,9 +9,7 @@ 0.2 - eu.stratosphere pact-tests - 0.2 pact-tests jar diff --git a/pact/pom.xml b/pact/pom.xml index 7cf7edddcba22..ce91616409707 100644 --- a/pact/pom.xml +++ b/pact/pom.xml @@ -9,12 +9,10 @@ 0.2 - eu.stratosphere pact pom - 0.2 pact 2009 diff --git a/score/pom.xml b/score/pom.xml index d2e177507fe30..c55ba9a9322f4 100644 --- a/score/pom.xml +++ b/score/pom.xml @@ -9,9 +9,7 @@ 0.2 - eu.stratosphere score - 0.2 score jar diff --git a/stratosphere-dist/pom.xml b/stratosphere-dist/pom.xml index 37e55eda9f13c..b20ee22147067 100644 --- a/stratosphere-dist/pom.xml +++ b/stratosphere-dist/pom.xml @@ -10,9 +10,7 @@ 0.2 - eu.stratosphere stratosphere-dist - 0.2 stratosphere-dist pom From 0d50ae23e8302729f544af8a53b0937c3d7f196c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 16:32:22 +0000 Subject: [PATCH 067/310] Added default wrapper for input and output gates --- .../wrapper/AbstractInputGateWrapper.java | 296 ++++++++++++++++ .../wrapper/AbstractOutputGateWrapper.java | 324 ++++++++++++++++++ 2 files changed, 620 insertions(+) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java new file mode 100644 index 0000000000000..267f272240705 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java @@ -0,0 +1,296 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.plugins.wrapper; + +import java.io.IOException; + +import eu.stratosphere.nephele.event.task.AbstractTaskEvent; +import eu.stratosphere.nephele.event.task.EventListener; +import eu.stratosphere.nephele.io.DistributionPattern; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.InputGateListener; +import eu.stratosphere.nephele.io.channels.AbstractInputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.channels.bytebuffered.FileInputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryInputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkInputChannel; +import eu.stratosphere.nephele.io.compression.CompressionLevel; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.types.Record; + +/** + * This class provides an abstract base class for an input gate wrapper. An input gate wrapper can be used by a plugin + * to wrap an input gate and intercept particular method calls. The default implementation of this abstract base class + * simply forwards every method call to the encapsulated input gate. + * + * @author warneke + * @param + * the type of record transported through this input gate + */ +public abstract class AbstractInputGateWrapper implements InputGate { + + /** + * The wrapped input gate. + */ + private final InputGate wrappedInputGate; + + /** + * Constructs a new abstract input gate wrapper. + * + * @param wrappedInputGate + * the input gate to be wrapped + */ + public AbstractInputGateWrapper(final InputGate wrappedInputGate) { + + if (wrappedInputGate == null) { + throw new IllegalArgumentException("Argument wrappedInputGate must not be null"); + } + + this.wrappedInputGate = wrappedInputGate; + } + + /** + * Returns the wrapped input gate. + * + * @return the wrapped input gate + */ + protected InputGate getWrappedInputGate() { + + return this.wrappedInputGate; + } + + /** + * {@inheritDoc} + */ + @Override + public int getIndex() { + + return this.wrappedInputGate.getIndex(); + } + + /** + * {@inheritDoc} + */ + @Override + public void subscribeToEvent(final EventListener eventListener, final Class eventType) { + + this.wrappedInputGate.subscribeToEvent(eventListener, eventType); + + } + + /** + * {@inheritDoc} + */ + @Override + public void unsubscribeFromEvent(final EventListener eventListener, + final Class eventType) { + + this.wrappedInputGate.unsubscribeFromEvent(eventListener, eventType); + } + + /** + * {@inheritDoc} + */ + @Override + public void publishEvent(final AbstractTaskEvent event) throws IOException, InterruptedException { + + this.wrappedInputGate.publishEvent(event); + } + + /** + * {@inheritDoc} + */ + @Override + public void deliverEvent(AbstractTaskEvent event) { + + this.wrappedInputGate.deliverEvent(event); + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + + return this.wrappedInputGate.getJobID(); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelType getChannelType() { + + return this.wrappedInputGate.getChannelType(); + } + + /** + * {@inheritDoc} + */ + @Override + public GateID getGateID() { + + return this.wrappedInputGate.getGateID(); + } + + /** + * {@inheritDoc} + */ + @Override + public void releaseAllChannelResources() { + + this.wrappedInputGate.releaseAllChannelResources(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isClosed() throws IOException, InterruptedException { + + return this.wrappedInputGate.isClosed(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isInputGate() { + + return this.wrappedInputGate.isInputGate(); + } + + /** + * {@inheritDoc} + */ + @Override + public void setChannelType(final ChannelType channelType) { + + this.wrappedInputGate.setChannelType(channelType); + } + + /** + * {@inheritDoc} + */ + @Override + public T readRecord(final T target) throws IOException, InterruptedException { + + return this.wrappedInputGate.readRecord(target); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfInputChannels() { + + return this.wrappedInputGate.getNumberOfInputChannels(); + } + + /** + * {@inheritDoc} + */ + @Override + public AbstractInputChannel getInputChannel(final int pos) { + + return this.wrappedInputGate.getInputChannel(pos); + } + + /** + * {@inheritDoc} + */ + @Override + public void notifyRecordIsAvailable(final int channelIndex) { + + this.wrappedInputGate.notifyRecordIsAvailable(channelIndex); + } + + /** + * {@inheritDoc} + */ + @Override + public void activateInputChannels() throws IOException, InterruptedException { + + this.wrappedInputGate.activateInputChannels(); + } + + /** + * {@inheritDoc} + */ + @Override + public void close() throws IOException, InterruptedException { + + this.wrappedInputGate.close(); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerInputGateListener(final InputGateListener inputGateListener) { + + this.wrappedInputGate.registerInputGateListener(inputGateListener); + } + + /** + * {@inheritDoc} + */ + @Override + public DistributionPattern getDistributionPattern() { + + return this.wrappedInputGate.getDistributionPattern(); + } + + /** + * {@inheritDoc} + */ + @Override + public NetworkInputChannel createNetworkInputChannel(final ChannelID channelID, + final CompressionLevel compressionLevel) { + + return this.wrappedInputGate.createNetworkInputChannel(channelID, compressionLevel); + } + + /** + * {@inheritDoc} + */ + @Override + public FileInputChannel createFileInputChannel(final ChannelID channelID, final CompressionLevel compressionLevel) { + + return this.wrappedInputGate.createFileInputChannel(channelID, compressionLevel); + } + + /** + * {@inheritDoc} + */ + @Override + public InMemoryInputChannel createInMemoryInputChannel(final ChannelID channelID, + final CompressionLevel compressionLevel) { + + return this.wrappedInputGate.createInMemoryInputChannel(channelID, compressionLevel); + } + + /** + * {@inheritDoc} + */ + @Override + public void removeAllInputChannels() { + + this.wrappedInputGate.removeAllInputChannels(); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java new file mode 100644 index 0000000000000..51956d6620e8a --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java @@ -0,0 +1,324 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.plugins.wrapper; + +import java.io.IOException; +import java.util.List; + +import eu.stratosphere.nephele.event.task.AbstractTaskEvent; +import eu.stratosphere.nephele.event.task.EventListener; +import eu.stratosphere.nephele.io.ChannelSelector; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.OutputGateListener; +import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.channels.bytebuffered.FileOutputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryOutputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkOutputChannel; +import eu.stratosphere.nephele.io.compression.CompressionLevel; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.types.Record; + +/** + * This class provides an abstract base class for an output gate wrapper. An output gate wrapper can be used by a plugin + * to wrap an output gate and intercept particular method calls. The default implementation of this abstract base class + * simply forwards every method call to the encapsulated output gate. + * + * @author warneke + * @param + * the type of record transported through this output gate + */ +public abstract class AbstractOutputGateWrapper implements OutputGate { + + /** + * The wrapped output gate. + */ + private final OutputGate wrappedOutputGate; + + /** + * Constructs a new abstract output gate wrapper. + * + * @param wrappedOutputGate + * the output gate to be wrapped + */ + public AbstractOutputGateWrapper(final OutputGate wrappedOutputGate) { + + if (wrappedOutputGate == null) { + throw new IllegalArgumentException("Argument wrappedOutputGate must not be null"); + } + + this.wrappedOutputGate = wrappedOutputGate; + } + + /** + * Returns the wrapped output gate. + * + * @return the wrapped output gate + */ + protected OutputGate getWrappedOutputGate() { + + return this.wrappedOutputGate; + } + + /** + * {@inheritDoc} + */ + @Override + public int getIndex() { + + return this.wrappedOutputGate.getIndex(); + } + + /** + * {@inheritDoc} + */ + @Override + public void subscribeToEvent(final EventListener eventListener, final Class eventType) { + + this.wrappedOutputGate.subscribeToEvent(eventListener, eventType); + } + + /** + * {@inheritDoc} + */ + @Override + public void unsubscribeFromEvent(final EventListener eventListener, + final Class eventType) { + + this.wrappedOutputGate.unsubscribeFromEvent(eventListener, eventType); + } + + /** + * {@inheritDoc} + */ + @Override + public void publishEvent(final AbstractTaskEvent event) throws IOException, InterruptedException { + + this.wrappedOutputGate.publishEvent(event); + } + + /** + * {@inheritDoc} + */ + @Override + public void deliverEvent(final AbstractTaskEvent event) { + + this.wrappedOutputGate.deliverEvent(event); + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + + return this.wrappedOutputGate.getJobID(); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelType getChannelType() { + + return this.wrappedOutputGate.getChannelType(); + } + + /** + * {@inheritDoc} + */ + @Override + public GateID getGateID() { + + return this.wrappedOutputGate.getGateID(); + } + + /** + * {@inheritDoc} + */ + @Override + public void releaseAllChannelResources() { + + this.wrappedOutputGate.releaseAllChannelResources(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isClosed() throws IOException, InterruptedException { + + return this.wrappedOutputGate.isClosed(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isInputGate() { + + return this.wrappedOutputGate.isInputGate(); + } + + /** + * {@inheritDoc} + */ + @Override + public void setChannelType(final ChannelType channelType) { + + this.wrappedOutputGate.setChannelType(channelType); + } + + /** + * {@inheritDoc} + */ + @Override + public Class getType() { + + return this.wrappedOutputGate.getType(); + } + + /** + * {@inheritDoc} + */ + @Override + public void writeRecord(final T record) throws IOException, InterruptedException { + + this.wrappedOutputGate.writeRecord(record); + } + + /** + * {@inheritDoc} + */ + @Override + public List> getOutputChannels() { + + return this.wrappedOutputGate.getOutputChannels(); + } + + /** + * {@inheritDoc} + */ + @Override + public void flush() throws IOException, InterruptedException { + + this.wrappedOutputGate.flush(); + } + + /** + * {@inheritDoc} + */ + @Override + public void channelCapacityExhausted(final int channelIndex) { + + this.wrappedOutputGate.channelCapacityExhausted(channelIndex); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isBroadcast() { + + return this.wrappedOutputGate.isBroadcast(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfOutputChannels() { + + return this.wrappedOutputGate.getNumberOfOutputChannels(); + } + + /** + * {@inheritDoc} + */ + @Override + public AbstractOutputChannel getOutputChannel(final int pos) { + + return this.wrappedOutputGate.getOutputChannel(pos); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelSelector getChannelSelector() { + + return this.wrappedOutputGate.getChannelSelector(); + } + + /** + * {@inheritDoc} + */ + @Override + public void requestClose() throws IOException, InterruptedException { + + this.wrappedOutputGate.requestClose(); + } + + /** + * {@inheritDoc} + */ + @Override + public void removeAllOutputChannels() { + + this.wrappedOutputGate.removeAllOutputChannels(); + } + + /** + * {@inheritDoc} + */ + @Override + public NetworkOutputChannel createNetworkOutputChannel(final ChannelID channelID, + final CompressionLevel compressionLevel) { + + return this.wrappedOutputGate.createNetworkOutputChannel(channelID, compressionLevel); + } + + /** + * {@inheritDoc} + */ + @Override + public FileOutputChannel createFileOutputChannel(final ChannelID channelID, + final CompressionLevel compressionLevel) { + + return this.wrappedOutputGate.createFileOutputChannel(channelID, compressionLevel); + } + + /** + * {@inheritDoc} + */ + @Override + public InMemoryOutputChannel createInMemoryOutputChannel(final ChannelID channelID, + final CompressionLevel compressionLevel) { + + return this.wrappedOutputGate.createInMemoryOutputChannel(channelID, compressionLevel); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerOutputGateListener(final OutputGateListener outputGateListener) { + + this.wrappedOutputGate.registerOutputGateListener(outputGateListener); + } +} From 70e705c1267800e8e63f59846fb461be873f6c04 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 16:53:05 +0000 Subject: [PATCH 068/310] Added default implementation for environment wrapper --- .../wrapper/AbstractEnvironmentWrapper.java | 233 ++++++++++++++++++ 1 file changed, 233 insertions(+) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractEnvironmentWrapper.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractEnvironmentWrapper.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractEnvironmentWrapper.java new file mode 100644 index 0000000000000..df661429dc2dc --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractEnvironmentWrapper.java @@ -0,0 +1,233 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.plugins.wrapper; + +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.io.ChannelSelector; +import eu.stratosphere.nephele.io.DistributionPattern; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RecordDeserializer; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.services.iomanager.IOManager; +import eu.stratosphere.nephele.services.memorymanager.MemoryManager; +import eu.stratosphere.nephele.template.InputSplitProvider; +import eu.stratosphere.nephele.types.Record; + +/** + * This class provides an abstract base class for an environment wrapper. An environment wrapper can be used by a plugin + * to wrap a task's environment and intercept particular method calls. The default implementation of this abstract base + * class simply forwards every method call to the encapsulated environment. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public abstract class AbstractEnvironmentWrapper implements Environment { + + private final Environment wrappedEnvironment; + + /** + * Constructs a new abstract environment wrapper. + * + * @param wrappedEnvironment + * the environment to be wrapped + */ + public AbstractEnvironmentWrapper(final Environment wrappedEnvironment) { + + if (wrappedEnvironment == null) { + throw new IllegalArgumentException("Argument wrappedEnvironment must not be null"); + } + + this.wrappedEnvironment = wrappedEnvironment; + } + + /** + * Returns the wrapped environment. + * + * @return the wrapped environment + */ + protected Environment getWrappedEnvironment() { + + return this.wrappedEnvironment; + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + + return this.wrappedEnvironment.getJobID(); + } + + /** + * {@inheritDoc} + */ + @Override + public Configuration getRuntimeConfiguration() { + + return this.wrappedEnvironment.getRuntimeConfiguration(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getCurrentNumberOfSubtasks() { + + return this.wrappedEnvironment.getCurrentNumberOfSubtasks(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getIndexInSubtaskGroup() { + + return this.wrappedEnvironment.getIndexInSubtaskGroup(); + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadStarted(final Thread userThread) { + + this.wrappedEnvironment.userThreadStarted(userThread); + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadFinished(final Thread userThread) { + + this.wrappedEnvironment.userThreadStarted(userThread); + } + + /** + * {@inheritDoc} + */ + @Override + public InputSplitProvider getInputSplitProvider() { + + return this.wrappedEnvironment.getInputSplitProvider(); + } + + /** + * {@inheritDoc} + */ + @Override + public IOManager getIOManager() { + + return this.wrappedEnvironment.getIOManager(); + } + + /** + * {@inheritDoc} + */ + @Override + public MemoryManager getMemoryManager() { + + return this.wrappedEnvironment.getMemoryManager(); + } + + /** + * {@inheritDoc} + */ + @Override + public String getTaskName() { + return this.wrappedEnvironment.getTaskName(); + } + + /** + * {@inheritDoc} + */ + @Override + public GateID getNextUnboundInputGateID() { + + return this.wrappedEnvironment.getNextUnboundInputGateID(); + } + + /** + * {@inheritDoc} + */ + @Override + public GateID getNextUnboundOutputGateID() { + + return this.wrappedEnvironment.getNextUnboundOutputGateID(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfOutputGates() { + + return this.wrappedEnvironment.getNumberOfOutputGates(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfInputGates() { + + return this.wrappedEnvironment.getNumberOfInputGates(); + } + + /** + * {@inheritDoc} + */ + @Override + public OutputGate createOutputGate(final GateID gateID, + final Class outputClass, final ChannelSelector selector, + final boolean isBroadcast) { + + return this.wrappedEnvironment.createOutputGate(gateID, outputClass, selector, isBroadcast); + } + + /** + * {@inheritDoc} + */ + @Override + public InputGate createInputGate(final GateID gateID, + final RecordDeserializer deserializer, final DistributionPattern distributionPattern) { + + return this.wrappedEnvironment.createInputGate(gateID, deserializer, distributionPattern); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerOutputGate(final OutputGate outputGate) { + + this.wrappedEnvironment.registerOutputGate(outputGate); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerInputGate(final InputGate inputGate) { + + this.wrappedEnvironment.registerInputGate(inputGate); + } +} From 3a2aaf10d8da7eadae0a9cfb9dd297c74266394d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 17:18:38 +0000 Subject: [PATCH 069/310] Removed deprecated methods from input and output gate listener interfaces --- .../eu/stratosphere/nephele/io/InputGateListener.java | 10 ---------- .../eu/stratosphere/nephele/io/OutputGateListener.java | 10 ---------- .../nephele/profiling/impl/InputGateListenerImpl.java | 10 ---------- .../nephele/profiling/impl/OutputGateListenerImpl.java | 9 --------- .../eu/stratosphere/nephele/io/RuntimeInputGate.java | 6 ------ .../eu/stratosphere/nephele/io/RuntimeOutputGate.java | 6 ------ 6 files changed, 51 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGateListener.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGateListener.java index 312f1da5b7ac7..fca80760e25dd 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGateListener.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGateListener.java @@ -15,8 +15,6 @@ package eu.stratosphere.nephele.io; -import eu.stratosphere.nephele.types.Record; - /** * This listener interface can be used to obtain information * about the utilization of the attached {@link InputGate}. @@ -30,12 +28,4 @@ public interface InputGateListener { * wait because none of its channels can currently deliver new data. */ void waitingForAnyChannel(); - - /** - * This method is called by the {@link InputGate} whenever it is about to pass a new record to the task. - * - * @param record - * the record which is about to be passed to the application - */ - void recordReceived(final Record record); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGateListener.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGateListener.java index fae653edd77b5..8e6a0ba0030ef 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGateListener.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGateListener.java @@ -15,17 +15,7 @@ package eu.stratosphere.nephele.io; -import eu.stratosphere.nephele.types.Record; - public interface OutputGateListener { void channelCapacityExhausted(int channelIndex); - - /** - * This method is called to indicate that a record has just been emitted by the task. - * - * @param record - * the record which has just been emitted - */ - void recordEmitted(final Record record); } diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/InputGateListenerImpl.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/InputGateListenerImpl.java index 101bc2b947766..77562785a8e4f 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/InputGateListenerImpl.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/InputGateListenerImpl.java @@ -18,7 +18,6 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.InputGateListener; import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.types.Record; public class InputGateListenerImpl implements InputGateListener { @@ -62,13 +61,4 @@ public ExecutionVertexID getExecutionVertexID() { public int getGateIndex() { return this.gateIndex; } - - /** - * {@inheritDoc} - */ - @Override - public void recordReceived(final Record record) { - // Nothing to do here - - } } diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/OutputGateListenerImpl.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/OutputGateListenerImpl.java index be79f53d4895e..3715fe905ef7d 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/OutputGateListenerImpl.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/OutputGateListenerImpl.java @@ -18,7 +18,6 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.OutputGateListener; import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.types.Record; public class OutputGateListenerImpl implements OutputGateListener { @@ -60,12 +59,4 @@ public int getAndResetCounter() { return retval; } - - /** - * {@inheritDoc} - */ - @Override - public void recordEmitted(final Record record) { - // Nothing to do here - } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java index 2c25d34abdaf3..540a8bd451b20 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java @@ -322,12 +322,6 @@ record = this.getInputChannel(this.channelToReadFrom).readRecord(target); } } - if (this.inputGateListeners != null) { - for (final InputGateListener inputGateListener : this.inputGateListeners) { - inputGateListener.recordReceived(record); - } - } - return record; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java index 040e9a0ac88e6..1af2e3ad9cebc 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java @@ -333,12 +333,6 @@ public void writeRecord(final T record) throws IOException, InterruptedException throw new InterruptedException(); } - if (this.outputGateListeners != null) { - for (final OutputGateListener outputGateListener : this.outputGateListeners) { - outputGateListener.recordEmitted(record); - } - } - if (this.isBroadcast) { if (getChannelType() == ChannelType.INMEMORY) { From 9e3c1033a8f82f89648fff7b95353f505a516528 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 17:19:30 +0000 Subject: [PATCH 070/310] Improved javadoc --- .../nephele/plugins/wrapper/AbstractInputGateWrapper.java | 2 ++ .../nephele/plugins/wrapper/AbstractOutputGateWrapper.java | 2 ++ 2 files changed, 4 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java index 267f272240705..ba7b0c82e3770 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java @@ -37,6 +37,8 @@ * This class provides an abstract base class for an input gate wrapper. An input gate wrapper can be used by a plugin * to wrap an input gate and intercept particular method calls. The default implementation of this abstract base class * simply forwards every method call to the encapsulated input gate. + *

+ * This class is thread-safe. * * @author warneke * @param diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java index 51956d6620e8a..95d7f50260266 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java @@ -38,6 +38,8 @@ * This class provides an abstract base class for an output gate wrapper. An output gate wrapper can be used by a plugin * to wrap an output gate and intercept particular method calls. The default implementation of this abstract base class * simply forwards every method call to the encapsulated output gate. + *

+ * This class is thread-safe. * * @author warneke * @param From 8b96769abfee79cd3c241867cd5f51e6b489966c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 17:27:57 +0000 Subject: [PATCH 071/310] Cleaned up ChannelSelector interface --- .../nephele/io/ChannelSelector.java | 2 +- .../nephele/io/DefaultChannelSelector.java | 20 --- .../pact/runtime/task/AbstractPactTask.java | 2 +- .../pact/runtime/task/DataSourceTask.java | 2 +- .../pact/runtime/task/util/OutputEmitter.java | 134 ++++-------------- .../pact/runtime/io/OutputEmitterTest.java | 11 +- 6 files changed, 35 insertions(+), 136 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.java index 7618befaad3e7..4720e77383f26 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.java @@ -26,7 +26,7 @@ * @param * the type of record which is sent through the attached output gate */ -public interface ChannelSelector extends IOReadableWritable { +public interface ChannelSelector { /** * Called to determine to which attached {@link AbstractOutputChannel} objects the given record shall be forwarded. diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java index 504c87c89130e..6abc5a10be7f1 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java @@ -15,10 +15,6 @@ package eu.stratosphere.nephele.io; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - import eu.stratosphere.nephele.types.Record; /** @@ -53,20 +49,4 @@ public int[] selectChannels(final T record, final int numberOfOutpuChannels) { return this.nextChannelToSendTo; } - - /** - * {@inheritDoc} - */ - @Override - public void write(DataOutput out) throws IOException { - // Nothing to do here - } - - /** - * {@inheritDoc} - */ - @Override - public void read(DataInput in) throws IOException { - // Nothing to do here - } } diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java index 307e446258045..b7fe287e89494 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java @@ -323,7 +323,7 @@ protected void initOutputs() OutputEmitter oe = (keyPositions == null || keyClasses == null) ? new OutputEmitter(strategy) : - new OutputEmitter(strategy, jobId, keyPositions, keyClasses); + new OutputEmitter(strategy, keyPositions, keyClasses); // create writer RecordWriter writer= new RecordWriter(this, PactRecord.class, oe); diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java index 463a1cc4a1b7c..7e4e4697c99fd 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java @@ -249,7 +249,7 @@ protected void initOutputs() OutputEmitter oe = (keyPositions == null || keyClasses == null) ? new OutputEmitter(strategy) : - new OutputEmitter(strategy, jobId, keyPositions, keyClasses); + new OutputEmitter(strategy, keyPositions, keyClasses); // create writer RecordWriter writer= new RecordWriter(this, PactRecord.class, oe); diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/OutputEmitter.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/OutputEmitter.java index fc0737a8d8780..c8774be0780b9 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/OutputEmitter.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/OutputEmitter.java @@ -15,13 +15,7 @@ package eu.stratosphere.pact.runtime.task.util; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.io.ChannelSelector; -import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.pact.common.type.Key; import eu.stratosphere.pact.common.type.PactRecord; @@ -52,20 +46,18 @@ public enum ShipStrategy { // ------------------------------------------------------------------------ private static final byte[] DEFAULT_SALT = new byte[] { 17, 31, 47, 51, 83, 1 }; - - private ShipStrategy strategy; // the shipping strategy used by this output emitter - - private int[] channels; // the reused array defining target channels - - private int nextChannelToSendTo = 0; // counter to go over channels round robin - + + private ShipStrategy strategy; // the shipping strategy used by this output emitter + + private int[] channels; // the reused array defining target channels + + private int nextChannelToSendTo = 0; // counter to go over channels round robin + private Class[] keyClasses; - + private int[] keyPositions; - - private final byte[] salt; // the salt used to randomize the hash values - - private JobID jobId; // the job ID is necessary to obtain the class loader + + private final byte[] salt; // the salt used to randomize the hash values private PartitionFunction partitionFunction; @@ -91,23 +83,23 @@ public OutputEmitter(ShipStrategy strategy) { this.strategy = strategy; this.salt = DEFAULT_SALT; - } - - public OutputEmitter(ShipStrategy strategy, JobID jobId, int[] keyPositions, Class[] keyTypes) + } + + public OutputEmitter(ShipStrategy strategy, int[] keyPositions, Class[] keyTypes) { - this(strategy, jobId, DEFAULT_SALT, keyPositions, keyTypes); + this(strategy, DEFAULT_SALT, keyPositions, keyTypes); } - - public OutputEmitter(ShipStrategy strategy, JobID jobId, byte[] salt , int[] keyPositions, Class[] keyTypes) + + public OutputEmitter(ShipStrategy strategy, byte[] salt, int[] keyPositions, + Class[] keyTypes) { - if (strategy == null | jobId == null | salt == null | keyPositions == null | keyTypes == null) { + if (strategy == null | salt == null | keyPositions == null | keyTypes == null) { throw new NullPointerException(); } this.strategy = strategy; this.salt = salt; this.keyPositions = keyPositions; this.keyClasses = keyTypes; - this.jobId = jobId; } // ------------------------------------------------------------------------ @@ -135,9 +127,10 @@ public final int[] selectChannels(PactRecord record, int numberOfChannels) throw new UnsupportedOperationException("Unsupported distribution strategy: " + strategy.name()); } } - + /** * Set the partition function that is used for range partitioning + * * @param func */ public void setPartitionFunction(PartitionFunction func) { @@ -147,7 +140,7 @@ public void setPartitionFunction(PartitionFunction func) { private int[] partition_range(PactRecord record, int numberOfChannels) { return partitionFunction.selectChannels(record, numberOfChannels); } - + // -------------------------------------------------------------------------------------------- private final int[] robin(int numberOfChannels) @@ -155,10 +148,10 @@ private final int[] robin(int numberOfChannels) if (this.channels == null || this.channels.length != 1) { this.channels = new int[1]; } - + int nextChannel = nextChannelToSendTo + 1; nextChannel = nextChannel < numberOfChannels ? nextChannel : 0; - + this.nextChannelToSendTo = nextChannel; this.channels[0] = nextChannel; return this.channels; @@ -180,93 +173,18 @@ private final int[] hashPartitionDefault(PactRecord record, int numberOfChannels if (channels == null || channels.length != 1) { channels = new int[1]; } - + int hash = 0; for (int i = 0; i < this.keyPositions.length; i++) { final Key k = record.getField(this.keyPositions[i], this.keyClasses[i]); hash ^= (1315423911 ^ ((1315423911 << 5) + k.hashCode() + (1315423911 >> 2))); } - + for (int i = 0; i < salt.length; i++) { hash ^= ((hash << 5) + salt[i] + (hash >> 2)); } - + this.channels[0] = (hash < 0) ? -hash % numberOfChannels : hash % numberOfChannels; return this.channels; } - - // ------------------------------------------------------------------------ - // Serialization - // ------------------------------------------------------------------------ - - /* - * (non-Javadoc) - * @see eu.stratosphere.nephele.io.IOReadableWritable#read(java.io.DataInput) - */ - @Override - public void read(DataInput in) throws IOException - { - // strategy - this.strategy = ShipStrategy.valueOf(in.readUTF()); - - // check whether further parameters come - final boolean keyParameterized = in.readBoolean(); - - if (keyParameterized) { - // read the jobID to find the classloader - this.jobId = new JobID(); - this.jobId.read(in); - final ClassLoader loader = LibraryCacheManager.getClassLoader(this.jobId); - - // read the number of keys and key positions - int numKeys = in.readInt(); - this.keyPositions = new int[numKeys]; - for (int i = 0; i < numKeys; i++) { - this.keyPositions[i] = in.readInt(); - } - - // read the key types - @SuppressWarnings("unchecked") - Class[] classes = (Class[]) new Class[numKeys]; - try { - for (int i = 0; i < numKeys; i++) { - String className = in.readUTF(); - classes[i] = Class.forName(className, true, loader).asSubclass(Key.class); - } - } - catch (Exception e) { - throw new RuntimeException("Output Emmitter is unable to load the classes that describe the key types: " - + e.getMessage(), e); - } - this.keyClasses = classes; - } - } - - /* - * (non-Javadoc) - * @see eu.stratosphere.nephele.io.IOReadableWritable#write(java.io.DataOutput) - */ - @Override - public void write(DataOutput out) throws IOException - { - out.writeUTF(strategy.name()); - - if (this.keyClasses != null) { - // write additional info - out.writeBoolean(true); - this.jobId.write(out); - - // write number of keys, key positions and key types - out.writeInt(this.keyClasses.length); - for (int i = 0; i < this.keyPositions.length; i++) { - out.writeInt(this.keyPositions[i]); - } - for (int i = 0; i < this.keyClasses.length; i++) { - out.writeUTF(this.keyClasses[i].getName()); - } - } - else { - out.writeBoolean(false); - } - } } diff --git a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/io/OutputEmitterTest.java b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/io/OutputEmitterTest.java index 80eeed8c9963d..15bcfca8bad69 100644 --- a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/io/OutputEmitterTest.java +++ b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/io/OutputEmitterTest.java @@ -19,7 +19,6 @@ import org.junit.Test; -import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.pact.common.type.PactRecord; import eu.stratosphere.pact.common.type.base.PactInteger; import eu.stratosphere.pact.common.type.base.PactString; @@ -32,14 +31,15 @@ public class OutputEmitterTest extends TestCase { public static void testPartitioning() { @SuppressWarnings("unchecked") - OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new JobID(), new int[] {0}, new Class[] {PactInteger.class}); + OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new int[] { 0 }, + new Class[] { PactInteger.class }); int[] hit = new int[100]; for (int i = 0; i < 1000000; i++) { PactInteger k = new PactInteger(i); PactRecord rec = new PactRecord(k); - + hit[oe1.selectChannels(rec, hit.length)[0]]++; } @@ -49,14 +49,15 @@ public static void testPartitioning() { } @SuppressWarnings("unchecked") - OutputEmitter oe2 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new JobID(), new int[] {0}, new Class[] {PactInteger.class}); + OutputEmitter oe2 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new int[] { 0 }, + new Class[] { PactInteger.class }); hit = new int[10]; for (int i = 0; i < 1000; i++) { PactString k = new PactString(i + ""); PactRecord rec = new PactRecord(k); - + hit[oe2.selectChannels(rec, hit.length)[0]]++; } From 4dfcb183b7b3a17fdfbf7af12faeb751ff8a2c0d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 18:04:04 +0000 Subject: [PATCH 072/310] Both task and job configuration are now accessible at the task managers at runtime --- .../nephele/execution/Environment.java | 13 +++-- .../nephele/jobgraph/JobFileInputVertex.java | 2 +- .../template/AbstractFileInputTask.java | 2 +- .../template/AbstractFileOutputTask.java | 2 +- .../nephele/template/AbstractInvokable.java | 22 ++++++--- .../io/library/FileLineReadWriteTest.java | 2 +- .../example/broadcast/BroadcastConsumer.java | 8 ++-- .../example/broadcast/BroadcastProducer.java | 2 +- .../nephele/execution/RuntimeEnvironment.java | 47 ++++++++++++++----- .../executiongraph/ExecutionGraph.java | 10 ++-- .../executiongraph/ExecutionVertex.java | 9 ++-- .../wrapper/AbstractEnvironmentWrapper.java | 13 ++++- .../pact/runtime/task/AbstractPactTask.java | 2 +- .../pact/runtime/task/DataSinkTask.java | 4 +- .../pact/runtime/task/DataSourceTask.java | 2 +- .../pact/runtime/task/PartitionTask.java | 2 +- .../runtime/test/util/MockEnvironment.java | 2 +- .../pact/runtime/test/util/TaskTestBase.java | 20 ++++---- 18 files changed, 109 insertions(+), 55 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java index 1f2525ca2773d..efe45c34d876f 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java @@ -47,11 +47,18 @@ public interface Environment { JobID getJobID(); /** - * Returns the runtime configuration object which was attached to the original {@link JobVertex}. + * Returns the task configuration object which was attached to the original {@link JobVertex}. * - * @return the runtime configuration object which was attached to the original {@link JobVertex} + * @return the task configuration object which was attached to the original {@link JobVertex} */ - Configuration getRuntimeConfiguration(); + Configuration getTaskConfiguration(); + + /** + * Returns the job configuration object which was attached to the original {@link JobGraph}. + * + * @return the job configuration object which was attached to the original {@link JobGraph} + */ + Configuration getJobConfiguration(); /** * Returns the current number of subtasks the respective task is split into. diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileInputVertex.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileInputVertex.java index 545ee03153700..d117b308f92c3 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileInputVertex.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/jobgraph/JobFileInputVertex.java @@ -173,7 +173,7 @@ public void checkConfiguration(final AbstractInvokable invokable) throws Illegal } // register the path in the configuration - invokable.getRuntimeConfiguration() + invokable.getTaskConfiguration() .setString(AbstractFileInputTask.INPUT_PATH_CONFIG_KEY, this.path.toString()); // Finally, see if the task itself has a valid configuration diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractFileInputTask.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractFileInputTask.java index af3df12a2a392..4b0a4b552a390 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractFileInputTask.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractFileInputTask.java @@ -62,7 +62,7 @@ public Iterator getFileInputSplits() { @Override public FileInputSplit[] computeInputSplits(final int minNumSplits) throws IOException { - final String pathURI = getRuntimeConfiguration().getString(INPUT_PATH_CONFIG_KEY, null); + final String pathURI = getTaskConfiguration().getString(INPUT_PATH_CONFIG_KEY, null); if (pathURI == null) { throw new IOException("The path to the file was not found in the runtime configuration."); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractFileOutputTask.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractFileOutputTask.java index 0b2402e606137..0a394ae56ab96 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractFileOutputTask.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractFileOutputTask.java @@ -35,7 +35,7 @@ public abstract class AbstractFileOutputTask extends AbstractOutputTask { public Path getFileOutputPath() { // TODO: This is a quick workaround, problem can be solved in a more generic way - final Configuration conf = getEnvironment().getRuntimeConfiguration(); + final Configuration conf = getEnvironment().getTaskConfiguration(); final String outputPath = conf.getString("outputPath", null); diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java index 67931165799f4..23330721d7c6f 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/template/AbstractInvokable.java @@ -59,11 +59,11 @@ public final void setEnvironment(final Environment environment) { * * @return the environment of this task or null if the environment has not yet been set */ - //TODO: This method should be final + // TODO: This method should be final public Environment getEnvironment() { return this.environment; } - + /** * Overwrite this method to implement task specific checks if the * respective task has been configured properly. @@ -119,13 +119,23 @@ public final int getIndexInSubtaskGroup() { } /** - * Returns the runtime configuration object which was attached to the original {@link JobVertex}. + * Returns the task configuration object which was attached to the original {@link JobVertex}. + * + * @return the task configuration object which was attached to the original {@link JobVertex} + */ + public final Configuration getTaskConfiguration() { + + return this.environment.getTaskConfiguration(); + } + + /** + * Returns the job configuration object which was attached to the original {@link JobGraph}. * - * @return the runtime configuration object which was attached to the original {@link JobVertex} + * @return the job configuration object which was attached to the original {@link JobGraph} */ - public final Configuration getRuntimeConfiguration() { + public final Configuration getJobConfiguration() { - return this.environment.getRuntimeConfiguration(); + return this.environment.getJobConfiguration(); } /** diff --git a/nephele/nephele-common/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java b/nephele/nephele-common/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java index 3bf60da723035..80516a2da1b77 100644 --- a/nephele/nephele-common/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java +++ b/nephele/nephele-common/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java @@ -101,7 +101,7 @@ public void testReadWrite() throws Exception { FileLineWriter writer = new FileLineWriter(); Whitebox.setInternalState(writer, "environment", this.environment); Whitebox.setInternalState(writer, "input", this.recordReader); - when(this.environment.getRuntimeConfiguration()).thenReturn(this.conf); + when(this.environment.getTaskConfiguration()).thenReturn(this.conf); when(this.conf.getString("outputPath", null)).thenReturn(this.file.toURI().toString()); when(this.recordReader.hasNext()).thenReturn(true, true, true, false); diff --git a/nephele/nephele-examples/src/main/java/eu/stratosphere/nephele/example/broadcast/BroadcastConsumer.java b/nephele/nephele-examples/src/main/java/eu/stratosphere/nephele/example/broadcast/BroadcastConsumer.java index 4d7038139a9e0..440e232d52570 100644 --- a/nephele/nephele-examples/src/main/java/eu/stratosphere/nephele/example/broadcast/BroadcastConsumer.java +++ b/nephele/nephele-examples/src/main/java/eu/stratosphere/nephele/example/broadcast/BroadcastConsumer.java @@ -100,10 +100,10 @@ public void invoke() throws Exception { */ private String getFilename() { - final String outputPath = getRuntimeConfiguration().getString(OUTPUT_PATH_KEY, ""); - final String instanceType = getRuntimeConfiguration().getString(INSTANCE_TYPE_KEY, "unknown"); - final String topologyTree = getRuntimeConfiguration().getString(TOPOLOGY_TREE_KEY, "unknown"); - final int numberOfRecords = getRuntimeConfiguration().getInteger(BroadcastProducer.NUMBER_OF_RECORDS_KEY, 0); + final String outputPath = getTaskConfiguration().getString(OUTPUT_PATH_KEY, ""); + final String instanceType = getTaskConfiguration().getString(INSTANCE_TYPE_KEY, "unknown"); + final String topologyTree = getTaskConfiguration().getString(TOPOLOGY_TREE_KEY, "unknown"); + final int numberOfRecords = getTaskConfiguration().getInteger(BroadcastProducer.NUMBER_OF_RECORDS_KEY, 0); return outputPath + File.separator + "latency_" + instanceType + "_" + topologyTree + "_" + getCurrentNumberOfSubtasks() + "_" + numberOfRecords + "_" + getIndexInSubtaskGroup() + "_" diff --git a/nephele/nephele-examples/src/main/java/eu/stratosphere/nephele/example/broadcast/BroadcastProducer.java b/nephele/nephele-examples/src/main/java/eu/stratosphere/nephele/example/broadcast/BroadcastProducer.java index aaa32a3661b7a..1ac6f6506155d 100644 --- a/nephele/nephele-examples/src/main/java/eu/stratosphere/nephele/example/broadcast/BroadcastProducer.java +++ b/nephele/nephele-examples/src/main/java/eu/stratosphere/nephele/example/broadcast/BroadcastProducer.java @@ -71,7 +71,7 @@ public void registerInputOutput() { public void invoke() throws Exception { // Determine number of records to emit - final int numberOfRecordsToEmit = getRuntimeConfiguration().getInteger(NUMBER_OF_RECORDS_KEY, + final int numberOfRecordsToEmit = getTaskConfiguration().getInteger(NUMBER_OF_RECORDS_KEY, DEFAULT_NUMBER_OF_RECORDS); // Create and prepare record diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 7162bc61d9696..30b8d6deff045 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -126,9 +126,14 @@ public class RuntimeEnvironment implements Environment, Runnable, IOReadableWrit private volatile JobID jobID = null; /** - * The runtime configuration of the task encapsulated in the environment object. + * The task configuration encapsulated in the environment object. */ - private volatile Configuration runtimeConfiguration = null; + private volatile Configuration taskConfiguration = null; + + /** + * The job configuration encapsulated in the environment object. + */ + private volatile Configuration jobConfiguration = null; /** * The input split provider that can be queried for new input splits. @@ -165,15 +170,19 @@ public class RuntimeEnvironment implements Environment, Runnable, IOReadableWrit * the name of task running in this environment * @param invokableClass * invokableClass the class that should be instantiated as a Nephele task - * @param runtimeConfiguration + * @param taskConfiguration * the configuration object which was attached to the original {@link JobVertex} + * @param jobConfiguration + * the configuration object which was attached to the original {@link JobGraph} */ public RuntimeEnvironment(final JobID jobID, final String taskName, - final Class invokableClass, final Configuration runtimeConfiguration) { + final Class invokableClass, final Configuration taskConfiguration, + final Configuration jobConfiguration) { this.jobID = jobID; this.taskName = taskName; this.invokableClass = invokableClass; - this.runtimeConfiguration = runtimeConfiguration; + this.taskConfiguration = taskConfiguration; + this.jobConfiguration = jobConfiguration; } /** @@ -523,9 +532,11 @@ public void read(final DataInput in) throws IOException { this.unboundInputGateIDs.add(gateID); } - // The configuration object - this.runtimeConfiguration = new Configuration(); - this.runtimeConfiguration.read(in); + // The configuration objects + this.taskConfiguration = new Configuration(); + this.taskConfiguration.read(in); + this.jobConfiguration = new Configuration(); + this.jobConfiguration.read(in); // The current of number subtasks this.currentNumberOfSubtasks = in.readInt(); @@ -655,8 +666,9 @@ public void write(final DataOutput out) throws IOException { inputGate.getGateID().write(out); } - // The configuration object - this.runtimeConfiguration.write(out); + // The configuration objects + this.taskConfiguration.write(out); + this.jobConfiguration.write(out); // The current of number subtasks out.writeInt(this.currentNumberOfSubtasks); @@ -805,7 +817,8 @@ public RuntimeEnvironment duplicateEnvironment() throws Exception { synchronized (duplicatedEnvironment) { duplicatedEnvironment.executingThread = tmpThread; } - duplicatedEnvironment.runtimeConfiguration = this.runtimeConfiguration; + duplicatedEnvironment.taskConfiguration = this.taskConfiguration; + duplicatedEnvironment.jobConfiguration = this.jobConfiguration; // We instantiate the invokable of the new environment duplicatedEnvironment.instantiateInvokable(); @@ -853,8 +866,16 @@ public void setMemoryManager(final MemoryManager memoryManager) { * {@inheritDoc} */ @Override - public Configuration getRuntimeConfiguration() { - return this.runtimeConfiguration; + public Configuration getTaskConfiguration() { + return this.taskConfiguration; + } + + /** + * {@inheritDoc} + */ + @Override + public Configuration getJobConfiguration() { + return this.jobConfiguration; } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index ff3ea84070212..376f98d553218 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -271,7 +271,8 @@ private void constructExecutionGraph(final JobGraph jobGraph, final InstanceMana // Convert job vertices to execution vertices and initialize them final AbstractJobVertex[] all = jobGraph.getAllJobVertices(); for (int i = 0; i < all.length; i++) { - final ExecutionVertex createdVertex = createVertex(all[i], instanceManager, initialExecutionStage); + final ExecutionVertex createdVertex = createVertex(all[i], instanceManager, initialExecutionStage, + jobGraph.getJobConfiguration()); temporaryVertexMap.put(all[i], createdVertex); temporaryGroupVertexMap.put(all[i], createdVertex.getGroupVertex()); } @@ -520,12 +521,15 @@ private void createChannel(final ExecutionVertex source, final OutputGate invokableClass, - final ExecutionGraph executionGraph, final ExecutionGroupVertex groupVertex) throws Exception { + final ExecutionGraph executionGraph, final ExecutionGroupVertex groupVertex, + final Configuration jobConfiguration) throws Exception { this(new ExecutionVertexID(), invokableClass, executionGraph, groupVertex, new RuntimeEnvironment(jobID, - groupVertex.getName(), invokableClass, groupVertex.getConfiguration())); + groupVertex.getName(), invokableClass, groupVertex.getConfiguration(), jobConfiguration)); this.groupVertex.addInitialSubtask(this); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractEnvironmentWrapper.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractEnvironmentWrapper.java index df661429dc2dc..61238a1d740a6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractEnvironmentWrapper.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractEnvironmentWrapper.java @@ -80,9 +80,18 @@ public JobID getJobID() { * {@inheritDoc} */ @Override - public Configuration getRuntimeConfiguration() { + public Configuration getTaskConfiguration() { - return this.wrappedEnvironment.getRuntimeConfiguration(); + return this.wrappedEnvironment.getTaskConfiguration(); + } + + /** + * {@inheritDoc} + */ + @Override + public Configuration getJobConfiguration() { + + return this.wrappedEnvironment.getJobConfiguration(); } /** diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java index b7fe287e89494..9be2f6d31edfe 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java @@ -227,7 +227,7 @@ public void cancel() throws Exception protected void initConfigAndStub(Class stubSuperClass) { // obtain task configuration (including stub parameters) - this.config = new TaskConfig(getRuntimeConfiguration()); + this.config = new TaskConfig(getTaskConfiguration()); // obtain stub implementation class try { diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java index 2d0a8f4bc0825..dddf6f9f489ea 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java @@ -182,7 +182,7 @@ public void cancel() throws Exception private void initOutputFormat() { // obtain task configuration (including stub parameters) - this.config = new TaskConfig(getRuntimeConfiguration()); + this.config = new TaskConfig(getTaskConfiguration()); // obtain stub implementation class try { @@ -290,7 +290,7 @@ public int getMaximumNumberOfSubtasks() catch (FileNotFoundException fnfex) { // The exception is thrown if the requested file/directory does not exist. // if the degree of parallelism is > 1, we create a directory for this path - int dop = getRuntimeConfiguration().getInteger(DEGREE_OF_PARALLELISM_KEY, -1); + int dop = getTaskConfiguration().getInteger(DEGREE_OF_PARALLELISM_KEY, -1); if (dop == 1) { // a none existing file and a degree of parallelism that is one return 1; diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java index 7e4e4697c99fd..63842d82b616e 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java @@ -183,7 +183,7 @@ public void cancel() throws Exception private void initInputFormat() { // obtain task configuration (including stub parameters) - this.config = new TaskConfig(getRuntimeConfiguration()); + this.config = new TaskConfig(getTaskConfiguration()); // obtain stub implementation class try { diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/PartitionTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/PartitionTask.java index 4f379aa68c921..393554523f614 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/PartitionTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/PartitionTask.java @@ -85,7 +85,7 @@ public Class getStubType() { @Override public void prepare() throws Exception { // obtain task configuration (including stub parameters) - config = new TaskConfig(getRuntimeConfiguration()); + config = new TaskConfig(getTaskConfiguration()); order = Order.valueOf(config.getStubParameters().getString(GLOBAL_PARTITIONING_ORDER, "")); usesSample = config.getStubParameters().getBoolean(PARTITION_BY_SAMPLING, true); diff --git a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java index 29356e0376255..0665dd86482d3 100644 --- a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java +++ b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java @@ -66,7 +66,7 @@ public void addOutput(List outputList) { } @Override - public Configuration getRuntimeConfiguration() { + public Configuration getTaskConfiguration() { return this.config; } diff --git a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java index 0ac6e01e59213..011e9eac5abef 100644 --- a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java +++ b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java @@ -71,24 +71,24 @@ public void initEnvironment(long memorySize) { public void addInput(MutableObjectIterator input) { this.mockEnv.addInput(input); - new TaskConfig(mockEnv.getRuntimeConfiguration()).addInputShipStrategy(ShipStrategy.FORWARD); + new TaskConfig(mockEnv.getTaskConfiguration()).addInputShipStrategy(ShipStrategy.FORWARD); } public void addOutput(List output) { this.mockEnv.addOutput(output); - new TaskConfig(mockEnv.getRuntimeConfiguration()).addOutputShipStrategy(ShipStrategy.FORWARD); + new TaskConfig(mockEnv.getTaskConfiguration()).addOutputShipStrategy(ShipStrategy.FORWARD); } public TaskConfig getTaskConfig() { - return new TaskConfig(mockEnv.getRuntimeConfiguration()); + return new TaskConfig(mockEnv.getTaskConfiguration()); } - + public Configuration getConfiguration() { - return mockEnv.getRuntimeConfiguration(); + return mockEnv.getTaskConfiguration(); } public void registerTask(AbstractTask task, Class stubClass) { - new TaskConfig(mockEnv.getRuntimeConfiguration()).setStubClass(stubClass); + new TaskConfig(mockEnv.getTaskConfiguration()).setStubClass(stubClass); task.setEnvironment(mockEnv); task.registerInputOutput(); } @@ -101,11 +101,11 @@ public void registerTask(AbstractTask task) { public void registerFileOutputTask(AbstractOutputTask outTask, Class stubClass, String outPath) { - TaskConfig dsConfig = new TaskConfig(mockEnv.getRuntimeConfiguration()); - + TaskConfig dsConfig = new TaskConfig(mockEnv.getTaskConfiguration()); + dsConfig.setStubClass(stubClass); dsConfig.setStubParameter(FileOutputFormat.FILE_PARAMETER_KEY, outPath); - + outTask.setEnvironment(mockEnv); outTask.registerInputOutput(); } @@ -113,7 +113,7 @@ public void registerFileOutputTask(AbstractOutputTask outTask, public void registerFileInputTask(AbstractInputTask inTask, Class stubClass, String inPath, String delimiter) { - TaskConfig dsConfig = new TaskConfig(mockEnv.getRuntimeConfiguration()); + TaskConfig dsConfig = new TaskConfig(mockEnv.getTaskConfiguration()); dsConfig.setStubClass(stubClass); dsConfig.setStubParameter(FileInputFormat.FILE_PARAMETER_KEY, inPath); dsConfig.setStubParameter(DelimitedInputFormat.RECORD_DELIMITER, delimiter); From 53a61c26dcc66ba3e376798b5b81f8b7c42d872c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 18:10:20 +0000 Subject: [PATCH 073/310] Added new wrapper classes for Nephele streaming plugin --- .../streaming/StreamingEnvironment.java | 70 +++++++++++++++++++ .../nephele/streaming/StreamingInputGate.java | 27 +++++++ .../streaming/StreamingOutputGate.java | 42 +++++++++++ 3 files changed, 139 insertions(+) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingEnvironment.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingInputGate.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingOutputGate.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingEnvironment.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingEnvironment.java new file mode 100644 index 0000000000000..b1da4b890c143 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingEnvironment.java @@ -0,0 +1,70 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.io.ChannelSelector; +import eu.stratosphere.nephele.io.DistributionPattern; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RecordDeserializer; +import eu.stratosphere.nephele.plugins.wrapper.AbstractEnvironmentWrapper; +import eu.stratosphere.nephele.types.Record; + +/** + * A streaming environment wraps the created input and output gates in special {@link StreamingInputGate} and + * {@link StreamingOutputGate} objects to intercept particular methods calls necessary for the statistics collection. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class StreamingEnvironment extends AbstractEnvironmentWrapper { + + public StreamingEnvironment(final Environment wrappedEnvironment) { + super(wrappedEnvironment); + } + + /** + * {@inheritDoc} + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + @Override + public OutputGate createOutputGate(final GateID gateID, + final Class outputClass, final ChannelSelector selector, + final boolean isBroadcast) { + + final OutputGate outputGate = getWrappedEnvironment().createOutputGate(gateID, outputClass, + selector, isBroadcast); + + return new StreamingOutputGate(outputGate); + } + + /** + * {@inheritDoc} + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + @Override + public InputGate createInputGate(final GateID gateID, + final RecordDeserializer deserializer, final DistributionPattern distributionPattern) { + + final InputGate inputGate = getWrappedEnvironment().createInputGate(gateID, deserializer, + distributionPattern); + + return new StreamingInputGate(inputGate); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingInputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingInputGate.java new file mode 100644 index 0000000000000..6a01601d42b72 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingInputGate.java @@ -0,0 +1,27 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.plugins.wrapper.AbstractInputGateWrapper; +import eu.stratosphere.nephele.types.Record; + +public final class StreamingInputGate extends AbstractInputGateWrapper { + + StreamingInputGate(final InputGate wrappedInputGate) { + super(wrappedInputGate); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingOutputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingOutputGate.java new file mode 100644 index 0000000000000..9154aab94ee22 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingOutputGate.java @@ -0,0 +1,42 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming; + +import java.io.IOException; + +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.plugins.wrapper.AbstractOutputGateWrapper; +import eu.stratosphere.nephele.types.Record; + +public final class StreamingOutputGate extends AbstractOutputGateWrapper { + + StreamingOutputGate(final OutputGate wrappedOutputGate) { + super(wrappedOutputGate); + + System.out.println("STREAMING CREATED"); + } + + /** + * {@inheritDoc} + */ + @Override + public void writeRecord(final T record) throws IOException, InterruptedException { + + System.out.println("DADADADADADADAD"); + + getWrappedOutputGate().writeRecord(record); + } +} From c42372b06b267c266bda1a94b3bf9143c4e2cca5 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 18:12:38 +0000 Subject: [PATCH 074/310] Worked on streaming plugin --- .../nephele/streaming/InputGateWrapper.java | 216 --------------- .../nephele/streaming/OutputGateWrapper.java | 253 ------------------ .../streaming/StreamingJobManagerPlugin.java | 7 +- .../streaming/StreamingTaskListener.java | 2 - .../streaming/StreamingTaskManagerPlugin.java | 28 +- .../nephele/streaming/TaskWrapper.java | 134 +--------- 6 files changed, 10 insertions(+), 630 deletions(-) delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/InputGateWrapper.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/OutputGateWrapper.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/InputGateWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/InputGateWrapper.java deleted file mode 100644 index f407bf9603f34..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/InputGateWrapper.java +++ /dev/null @@ -1,216 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming; - -import java.io.IOException; - -import eu.stratosphere.nephele.event.task.AbstractTaskEvent; -import eu.stratosphere.nephele.event.task.EventListener; -import eu.stratosphere.nephele.io.DistributionPattern; -import eu.stratosphere.nephele.io.GateID; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.InputGateListener; -import eu.stratosphere.nephele.io.channels.AbstractInputChannel; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; -import eu.stratosphere.nephele.io.channels.bytebuffered.FileInputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryInputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkInputChannel; -import eu.stratosphere.nephele.io.compression.CompressionLevel; -import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.types.Record; - -public final class InputGateWrapper implements InputGate { - - private final TaskWrapper taskWrapper; - - private final InputGate wrappedInputGate; - - InputGateWrapper(final TaskWrapper taskWrapper, final InputGate wrappedInputGate) { - - this.taskWrapper = taskWrapper; - this.wrappedInputGate = wrappedInputGate; - } - - /** - * {@inheritDoc} - */ - @Override - public void publishEvent(final AbstractTaskEvent event) throws IOException, InterruptedException { - - this.wrappedInputGate.publishEvent(event); - } - - @Override - public T readRecord(final T target) throws IOException, InterruptedException { - - // TODO Auto-generated method stub - return null; - } - - /** - * {@inheritDoc} - */ - @Override - public int getNumberOfInputChannels() { - - return this.wrappedInputGate.getNumberOfInputChannels(); - } - - /** - * {@inheritDoc} - */ - @Override - public void notifyRecordIsAvailable(final int channelIndex) { - - throw new IllegalStateException("notifyRecordIsAvailable is called on InputGateWrapper"); - } - - /** - * {@inheritDoc} - */ - @Override - public void subscribeToEvent(final EventListener eventListener, final Class eventType) { - - this.wrappedInputGate.subscribeToEvent(eventListener, eventType); - } - - /** - * {@inheritDoc} - */ - @Override - public void unsubscribeFromEvent(EventListener eventListener, Class eventType) { - - this.wrappedInputGate.unsubscribeFromEvent(eventListener, eventType); - } - - /** - * {@inheritDoc} - */ - @Override - public void deliverEvent(final AbstractTaskEvent event) { - - throw new IllegalStateException("deliverEvent is called on InputGateWrapper"); - } - - /** - * {@inheritDoc} - */ - @Override - public JobID getJobID() { - - return this.wrappedInputGate.getJobID(); - } - - /** - * {@inheritDoc} - */ - @Override - public ChannelType getChannelType() { - - return this.wrappedInputGate.getChannelType(); - } - - @Override - public GateID getGateID() { - // TODO Auto-generated method stub - return null; - } - - @Override - public void releaseAllChannelResources() { - // TODO Auto-generated method stub - - } - - @Override - public boolean isClosed() throws IOException, InterruptedException { - // TODO Auto-generated method stub - return false; - } - - @Override - public boolean isInputGate() { - // TODO Auto-generated method stub - return false; - } - - @Override - public void activateInputChannels() throws IOException, InterruptedException { - // TODO Auto-generated method stub - - } - - @Override - public AbstractInputChannel getInputChannel(int pos) { - // TODO Auto-generated method stub - return null; - } - - @Override - public void close() throws IOException, InterruptedException { - // TODO Auto-generated method stub - - } - - @Override - public void registerInputGateListener(InputGateListener inputGateListener) { - // TODO Auto-generated method stub - - } - - @Override - public void setChannelType(ChannelType channelType) { - // TODO Auto-generated method stub - - } - - @Override - public DistributionPattern getDistributionPattern() { - // TODO Auto-generated method stub - return null; - } - - @Override - public NetworkInputChannel createNetworkInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - // TODO Auto-generated method stub - return null; - } - - @Override - public FileInputChannel createFileInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - // TODO Auto-generated method stub - return null; - } - - @Override - public InMemoryInputChannel createInMemoryInputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - // TODO Auto-generated method stub - return null; - } - - @Override - public int getIndex() { - // TODO Auto-generated method stub - return 0; - } - - @Override - public void removeAllInputChannels() { - // TODO Auto-generated method stub - - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/OutputGateWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/OutputGateWrapper.java deleted file mode 100644 index 3a1609af64707..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/OutputGateWrapper.java +++ /dev/null @@ -1,253 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming; - -import java.io.IOException; -import java.util.List; - -import eu.stratosphere.nephele.event.task.AbstractTaskEvent; -import eu.stratosphere.nephele.event.task.EventListener; -import eu.stratosphere.nephele.io.ChannelSelector; -import eu.stratosphere.nephele.io.GateID; -import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.OutputGateListener; -import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; -import eu.stratosphere.nephele.io.channels.bytebuffered.FileOutputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryOutputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkOutputChannel; -import eu.stratosphere.nephele.io.compression.CompressionLevel; -import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.types.Record; - -public final class OutputGateWrapper implements OutputGate { - - private final TaskWrapper taskWrapper; - - private final OutputGate wrappedOutputGate; - - OutputGateWrapper(final TaskWrapper taskWrapper, final OutputGate wrappedOutputGate) { - - this.taskWrapper = taskWrapper; - this.wrappedOutputGate = wrappedOutputGate; - } - - /** - * {@inheritDoc} - */ - @Override - public void subscribeToEvent(final EventListener eventListener, final Class eventType) { - - this.wrappedOutputGate.subscribeToEvent(eventListener, eventType); - } - - /** - * {@inheritDoc} - */ - @Override - public void unsubscribeFromEvent(final EventListener eventListener, - final Class eventType) { - - this.wrappedOutputGate.unsubscribeFromEvent(eventListener, eventType); - } - - /** - * {@inheritDoc} - */ - @Override - public void publishEvent(final AbstractTaskEvent event) throws IOException, InterruptedException { - - this.wrappedOutputGate.publishEvent(event); - } - - /** - * {@inheritDoc} - */ - @Override - public void deliverEvent(final AbstractTaskEvent event) { - - throw new IllegalStateException("deliverEvent is called on OutputGateWrapper"); - } - - /** - * {@inheritDoc} - */ - @Override - public JobID getJobID() { - - return this.wrappedOutputGate.getJobID(); - } - - /** - * {@inheritDoc} - */ - @Override - public ChannelType getChannelType() { - - return this.wrappedOutputGate.getChannelType(); - } - - /** - * {@inheritDoc} - */ - @Override - public Class getType() { - - return this.wrappedOutputGate.getType(); - } - - /** - * {@inheritDoc} - */ - @Override - public void writeRecord(final T record) throws IOException, InterruptedException { - // TODO Auto-generated method stub - this.wrappedOutputGate.writeRecord(record); - } - - /** - * {@inheritDoc} - */ - @Override - public List> getOutputChannels() { - - return this.wrappedOutputGate.getOutputChannels(); - } - - /** - * {@inheritDoc} - */ - @Override - public void flush() throws IOException, InterruptedException { - - this.wrappedOutputGate.flush(); - } - - /** - * {@inheritDoc} - */ - @Override - public void channelCapacityExhausted(final int channelIndex) { - - this.wrappedOutputGate.channelCapacityExhausted(channelIndex); - } - - /** - * {@inheritDoc} - */ - @Override - public boolean isBroadcast() { - - return this.wrappedOutputGate.isBroadcast(); - } - - /** - * {@inheritDoc} - */ - @Override - public int getNumberOfOutputChannels() { - - return this.wrappedOutputGate.getNumberOfOutputChannels(); - } - - /** - * {@inheritDoc} - */ - @Override - public ChannelSelector getChannelSelector() { - - return this.wrappedOutputGate.getChannelSelector(); - } - - @Override - public GateID getGateID() { - // TODO Auto-generated method stub - return null; - } - - @Override - public void releaseAllChannelResources() { - // TODO Auto-generated method stub - - } - - @Override - public boolean isClosed() throws IOException, InterruptedException { - // TODO Auto-generated method stub - return false; - } - - @Override - public boolean isInputGate() { - // TODO Auto-generated method stub - return false; - } - - @Override - public AbstractOutputChannel getOutputChannel(int pos) { - // TODO Auto-generated method stub - return null; - } - - @Override - public void requestClose() throws IOException, InterruptedException { - // TODO Auto-generated method stub - - } - - @Override - public void removeAllOutputChannels() { - // TODO Auto-generated method stub - - } - - @Override - public void setChannelType(ChannelType channelType) { - // TODO Auto-generated method stub - - } - - @Override - public NetworkOutputChannel createNetworkOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - // TODO Auto-generated method stub - return null; - } - - @Override - public FileOutputChannel createFileOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - // TODO Auto-generated method stub - return null; - } - - @Override - public InMemoryOutputChannel createInMemoryOutputChannel(ChannelID channelID, CompressionLevel compressionLevel) { - // TODO Auto-generated method stub - return null; - } - - @Override - public int getIndex() { - // TODO Auto-generated method stub - return 0; - } - - @Override - public void registerOutputGateListener(OutputGateListener outputGateListener) { - // TODO Auto-generated method stub - - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 5f970bfaff545..03c233a9186ce 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -16,6 +16,7 @@ package eu.stratosphere.nephele.streaming; import java.io.IOException; +import java.util.Iterator; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -24,7 +25,9 @@ import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.jobgraph.JobGraph; +import eu.stratosphere.nephele.jobgraph.JobTaskVertex; import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.template.AbstractInvokable; public class StreamingJobManagerPlugin implements JobManagerPlugin { @@ -43,7 +46,7 @@ public class StreamingJobManagerPlugin implements JobManagerPlugin { public JobGraph rewriteJobGraph(final JobGraph jobGraph) { // Rewrite input vertices - /*final Iterator inputIt = jobGraph.getInputVertices(); + //final Iterator inputIt = jobGraph.getInputVertices(); final Iterator taskIt = jobGraph.getTaskVertices(); while (taskIt.hasNext()) { @@ -54,7 +57,7 @@ public JobGraph rewriteJobGraph(final JobGraph jobGraph) { taskVertex.setTaskClass(TaskWrapper.class); taskVertex.getConfiguration().setString(TaskWrapper.WRAPPED_CLASS_KEY, originalClass.getName()); - }*/ + } // TODO Auto-generated method stub return null; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java index 2690ae155f502..38a93582b62eb 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java @@ -106,7 +106,6 @@ public void channelCapacityExhausted(final int channelIndex) { /** * {@inheritDoc} */ - @Override public void recordEmitted(final Record record) { switch (this.taskType) { @@ -144,7 +143,6 @@ public void waitingForAnyChannel() { /** * {@inheritDoc} */ - @Override public void recordReceived(final Record record) { if (this.taskType == TaskType.INPUT) { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index d1c9f76ae329b..d8bb2ea5e1ab3 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -98,32 +98,8 @@ public void registerTask(final ExecutionVertexID id, final Configuration jobConf final int aggregationInterval = jobConfiguration.getInteger(AGGREGATION_INTERVAL_KEY, this.aggregationInterval); - StreamingTaskListener listener = null; - final JobID jobID = environment.getJobID(); - - if (environment.getNumberOfInputGates() == 0) { - // Check if user has provided a job-specific tagging interval - final int taggingInterval = jobConfiguration.getInteger(TAGGING_INTERVAL_KEY, this.taggingInterval); - - listener = StreamingTaskListener.createForInputTask(this.communicationThread, jobID, id, taggingInterval, - aggregationInterval); - } else if (environment.getNumberOfOutputGates() == 0) { - listener = StreamingTaskListener.createForOutputTask(this.communicationThread, jobID, id, - aggregationInterval); - } else { - listener = StreamingTaskListener.createForRegularTask(this.communicationThread, jobID, id, - aggregationInterval); - } - - for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - final OutputGate outputGate = environment.getOutputGate(i); - outputGate.registerOutputGateListener(listener); - } - - for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { - final InputGate inputGate = environment.getInputGate(i); - inputGate.registerInputGateListener(listener); - } + final int taggingInterval = jobConfiguration.getInteger(TAGGING_INTERVAL_KEY, this.taggingInterval); + } /** diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskWrapper.java index 1d0a5d025df73..823d4e41e42cc 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskWrapper.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskWrapper.java @@ -16,26 +16,13 @@ package eu.stratosphere.nephele.streaming; import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; -import eu.stratosphere.nephele.io.ChannelSelector; -import eu.stratosphere.nephele.io.DistributionPattern; -import eu.stratosphere.nephele.io.GateID; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.RecordDeserializer; -import eu.stratosphere.nephele.io.RuntimeInputGate; -import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.services.iomanager.IOManager; -import eu.stratosphere.nephele.services.memorymanager.MemoryManager; import eu.stratosphere.nephele.template.AbstractInvokable; import eu.stratosphere.nephele.template.AbstractTask; -import eu.stratosphere.nephele.template.InputSplitProvider; -import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.util.StringUtils; -public final class TaskWrapper extends AbstractTask implements Environment { +public final class TaskWrapper extends AbstractTask { static final String WRAPPED_CLASS_KEY = "streaming.class.name"; @@ -47,7 +34,7 @@ private synchronized AbstractInvokable getWrappedInvokable() { return this.wrappedInvokable; } - final Configuration conf = getEnvironment().getRuntimeConfiguration(); + final Configuration conf = getEnvironment().getTaskConfiguration(); final JobID jobID = getEnvironment().getJobID(); final String className = conf.getString(WRAPPED_CLASS_KEY, null); if (className == null) { @@ -66,7 +53,7 @@ private synchronized AbstractInvokable getWrappedInvokable() { throw new RuntimeException(StringUtils.stringifyException(e)); } - this.wrappedInvokable.setEnvironment(this); + this.wrappedInvokable.setEnvironment(new StreamingEnvironment(getEnvironment())); return this.wrappedInvokable; } @@ -89,119 +76,4 @@ public void invoke() throws Exception { getWrappedInvokable().invoke(); } - - /** - * {@inheritDoc} - */ - @Override - public JobID getJobID() { - - return getEnvironment().getJobID(); - } - - /** - * {@inheritDoc} - */ - @Override - public InputSplitProvider getInputSplitProvider() { - - return getEnvironment().getInputSplitProvider(); - } - - /** - * {@inheritDoc} - */ - @Override - public IOManager getIOManager() { - - return getEnvironment().getIOManager(); - } - - /** - * {@inheritDoc} - */ - @Override - public MemoryManager getMemoryManager() { - - return getEnvironment().getMemoryManager(); - } - - /** - * {@inheritDoc} - */ - @Override - public String getTaskName() { - - return getEnvironment().getTaskName(); - } - - /** - * {@inheritDoc} - */ - @Override - public int getNumberOfOutputGates() { - - return getEnvironment().getNumberOfOutputGates(); - } - - /** - * {@inheritDoc} - */ - @Override - public int getNumberOfInputGates() { - - return getEnvironment().getNumberOfInputGates(); - } - - /** - * {@inheritDoc} - */ - @Override - public void registerOutputGate(OutputGate outputGate) { - - getEnvironment().registerOutputGate(outputGate); - } - - /** - * {@inheritDoc} - */ - @Override - public void registerInputGate(InputGate inputGate) { - - getEnvironment().registerInputGate(inputGate); - } - - /** - * {@inheritDoc} - */ - @Override - public OutputGate createOutputGate(final GateID gateID, - final Class outputClass, final ChannelSelector selector, - final boolean isBroadcast) { - - return getEnvironment().createOutputGate(gateID, outputClass, selector, isBroadcast); - } - - /** - * {@inheritDoc} - */ - @Override - public InputGate createInputGate(final GateID gateID, - final RecordDeserializer deserializer, - final DistributionPattern distributionPattern) { - - return getEnvironment().createInputGate(gateID, deserializer, distributionPattern); - } - - @Override - public GateID getNextUnboundInputGateID() { - // TODO Auto-generated method stub - return null; - } - - @Override - public GateID getNextUnboundOutputGateID() { - // TODO Auto-generated method stub - return null; - } } From 50178859d3cbafd576f62bf8216b145b842b95d9 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 19:42:12 +0000 Subject: [PATCH 075/310] Fixed problem with inconsistent view on input/output gates --- .../eu/stratosphere/nephele/io/InputGate.java | 21 +++++++++++++------ .../stratosphere/nephele/io/OutputGate.java | 21 +++++++++++++------ .../nephele/execution/RuntimeEnvironment.java | 19 ++++++++++------- .../executiongraph/ExecutionGraph.java | 13 ++++++------ .../ManagementGraphFactory.java | 6 ++---- .../nephele/io/RuntimeInputGate.java | 19 +++++++++-------- .../nephele/io/RuntimeOutputGate.java | 17 ++++++++------- .../wrapper/AbstractInputGateWrapper.java | 13 ++++++------ .../wrapper/AbstractOutputGateWrapper.java | 15 ++++++------- 9 files changed, 85 insertions(+), 59 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java index 3f97d50cee95d..aa8ba3345c9e3 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java @@ -113,37 +113,46 @@ public interface InputGate extends Gate { DistributionPattern getDistributionPattern(); /** - * Creates a new network input channel and assigns it to the input gate. + * Creates a new network input channel and assigns it to the given input gate. * + * @param inputGate + * the input gate the channel shall be assigned to * @param channelID * the channel ID to assign to the new channel, null to generate a new ID * @param compressionLevel * the level of compression to be used for this channel * @return the new network input channel */ - NetworkInputChannel createNetworkInputChannel(ChannelID channelID, CompressionLevel compressionLevel); + NetworkInputChannel createNetworkInputChannel(InputGate inputGate, ChannelID channelID, + CompressionLevel compressionLevel); /** - * Creates a new file input channel and assigns it to the input gate. + * Creates a new file input channel and assigns it to the given input gate. * + * @param inputGate + * the input gate the channel shall be assigned to * @param channelID * the channel ID to assign to the new channel, null to generate a new ID * @param compressionLevel * the level of compression to be used for this channel * @return the new file input channel */ - FileInputChannel createFileInputChannel(ChannelID channelID, CompressionLevel compressionLevel); + FileInputChannel createFileInputChannel(InputGate inputGate, ChannelID channelID, + CompressionLevel compressionLevel); /** - * Creates a new in-memory input channel and assigns it to the input gate. + * Creates a new in-memory input channel and assigns it to the given input gate. * + * @param inputGate + * the input gate the channel shall be assigned to * @param channelID * the channel ID to assign to the new channel, null to generate a new ID * @param compressionLevel * the level of compression to be used for this channel * @return the new in-memory input channel */ - InMemoryInputChannel createInMemoryInputChannel(ChannelID channelID, CompressionLevel compressionLevel); + InMemoryInputChannel createInMemoryInputChannel(InputGate inputGate, ChannelID channelID, + CompressionLevel compressionLevel); /** * Removes all input channels from the input gate. diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java index 56c096df36821..26068246a779e 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java @@ -130,37 +130,46 @@ public interface OutputGate extends Gate { void removeAllOutputChannels(); /** - * Creates a new network output channel and assigns it to the output gate. + * Creates a new network output channel and assigns it to the given output gate. * + * @param outputGate + * the output gate the channel shall be assigned to * @param channelID * the channel ID to assign to the new channel, null to generate a new ID * @param compressionLevel * the level of compression to be used for this channel * @return the new network output channel */ - NetworkOutputChannel createNetworkOutputChannel(ChannelID channelID, CompressionLevel compressionLevel); + NetworkOutputChannel createNetworkOutputChannel(OutputGate outputGate, ChannelID channelID, + CompressionLevel compressionLevel); /** - * Creates a new file output channel and assigns it to the output gate. + * Creates a new file output channel and assigns it to the given output gate. * + * @param outputGate + * the output gate the channel shall be assigned to * @param channelID * the channel ID to assign to the new channel, null to generate a new ID * @param compressionLevel * the level of compression to be used for this channel * @return the new file output channel */ - FileOutputChannel createFileOutputChannel(ChannelID channelID, CompressionLevel compressionLevel); + FileOutputChannel createFileOutputChannel(OutputGate outputGate, ChannelID channelID, + CompressionLevel compressionLevel); /** - * Creates a new in-memory output channel and assigns it to the output gate. + * Creates a new in-memory output channel and assigns it to the given output gate. * + * @param outputGate + * the output gate the channel shall be assigned to * @param channelID * the channel ID to assign to the new channel, null to generate a new ID * @param compressionLevel * the level of compression to be used for this channel * @return the new in-memory output channel */ - InMemoryOutputChannel createInMemoryOutputChannel(ChannelID channelID, CompressionLevel compressionLevel); + InMemoryOutputChannel createInMemoryOutputChannel(OutputGate outputGate, ChannelID channelID, + CompressionLevel compressionLevel); /** * Registers a new listener object for this output gate. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 30b8d6deff045..04ef4a49da909 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -477,7 +477,7 @@ public Thread getExecutingThread() { /** * {@inheritDoc} */ - @SuppressWarnings("unchecked") + @SuppressWarnings({ "unchecked", "rawtypes" }) @Override public void read(final DataInput in) throws IOException { @@ -566,13 +566,16 @@ public void read(final DataInput in) throws IOException { switch (channelType) { case INMEMORY: - outputChannel = outputGate.createInMemoryOutputChannel(channelID, compressionLevel); + outputChannel = outputGate.createInMemoryOutputChannel((OutputGate) outputGate, channelID, + compressionLevel); break; case NETWORK: - outputChannel = outputGate.createNetworkOutputChannel(channelID, compressionLevel); + outputChannel = outputGate.createNetworkOutputChannel((OutputGate) outputGate, channelID, + compressionLevel); break; case FILE: - outputChannel = outputGate.createFileOutputChannel(channelID, compressionLevel); + outputChannel = outputGate.createFileOutputChannel((OutputGate) outputGate, channelID, + compressionLevel); break; } @@ -600,13 +603,15 @@ public void read(final DataInput in) throws IOException { switch (channelType) { case INMEMORY: - inputChannel = inputGate.createInMemoryInputChannel(channelID, compressionLevel); + inputChannel = inputGate.createInMemoryInputChannel((InputGate) inputGate, channelID, + compressionLevel); break; case NETWORK: - inputChannel = inputGate.createNetworkInputChannel(channelID, compressionLevel); + inputChannel = inputGate.createNetworkInputChannel((InputGate) inputGate, channelID, + compressionLevel); break; case FILE: - inputChannel = inputGate.createFileInputChannel(channelID, compressionLevel); + inputChannel = inputGate.createFileInputChannel((InputGate) inputGate, channelID, compressionLevel); break; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index 376f98d553218..c28109031b034 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -478,6 +478,7 @@ void wire(final ExecutionGroupVertex source, final int indexOfOutputGate, final } + @SuppressWarnings({ "unchecked", "rawtypes" }) private void createChannel(final ExecutionVertex source, final OutputGate outputGate, final ExecutionVertex target, final InputGate inputGate, final ChannelType channelType, final CompressionLevel compressionLevel) throws GraphConversionException { @@ -487,16 +488,16 @@ private void createChannel(final ExecutionVertex source, final OutputGate outputGate = ev.getEnvironment().getOutputGate(i); - final ManagementGate managementGate = new ManagementGate(managementVertex, + final ManagementGate managementGate = new ManagementGate(managementVertex, new ManagementGateID(), i, false, outputGate.getType().toString()); gateMap.put(outputGate, managementGate); } @@ -160,8 +159,7 @@ private static void addExecutionVertices(Map outputGate = (RuntimeOutputGate) source - .getEnvironment().getOutputGate(i); + final OutputGate outputGate = source.getEnvironment().getOutputGate(i); final ManagementGate manangementOutputGate = gateMap.get(outputGate); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); j++) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java index 540a8bd451b20..43a10ee8c0bdb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java @@ -247,11 +247,11 @@ public AbstractInputChannel getInputChannel(int pos) { * {@inheritDoc} */ @Override - public NetworkInputChannel createNetworkInputChannel(final ChannelID channelID, + public NetworkInputChannel createNetworkInputChannel(final InputGate inputGate, final ChannelID channelID, final CompressionLevel compressionLevel) { - final NetworkInputChannel enic = new NetworkInputChannel(this, this.inputChannels.size(), deserializer, - channelID, compressionLevel); + final NetworkInputChannel enic = new NetworkInputChannel(inputGate, this.inputChannels.size(), + this.deserializer, channelID, compressionLevel); addInputChannel(enic); return enic; @@ -261,10 +261,11 @@ public NetworkInputChannel createNetworkInputChannel(final ChannelID channelI * {@inheritDoc} */ @Override - public FileInputChannel createFileInputChannel(final ChannelID channelID, final CompressionLevel compressionLevel) { + public FileInputChannel createFileInputChannel(final InputGate inputGate, final ChannelID channelID, + final CompressionLevel compressionLevel) { - final FileInputChannel efic = new FileInputChannel(this, this.inputChannels.size(), deserializer, - channelID, compressionLevel); + final FileInputChannel efic = new FileInputChannel(inputGate, this.inputChannels.size(), + this.deserializer, channelID, compressionLevel); addInputChannel(efic); return efic; @@ -274,11 +275,11 @@ public FileInputChannel createFileInputChannel(final ChannelID channelID, fin * {@inheritDoc} */ @Override - public InMemoryInputChannel createInMemoryInputChannel(final ChannelID channelID, + public InMemoryInputChannel createInMemoryInputChannel(final InputGate inputGate, final ChannelID channelID, final CompressionLevel compressionLevel) { - final InMemoryInputChannel eimic = new InMemoryInputChannel(this, this.inputChannels.size(), - deserializer, channelID, compressionLevel); + final InMemoryInputChannel eimic = new InMemoryInputChannel(inputGate, this.inputChannels.size(), + this.deserializer, channelID, compressionLevel); addInputChannel(eimic); return eimic; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java index 1af2e3ad9cebc..0dd9866473e6a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java @@ -251,10 +251,11 @@ public AbstractOutputChannel getOutputChannel(int pos) { * {@inheritDoc} */ @Override - public NetworkOutputChannel createNetworkOutputChannel(final ChannelID channelID, - final CompressionLevel compressionLevel) { + public NetworkOutputChannel createNetworkOutputChannel(final OutputGate outputGate, + final ChannelID channelID, final CompressionLevel compressionLevel) { - final NetworkOutputChannel enoc = new NetworkOutputChannel(this, this.outputChannels.size(), channelID, + final NetworkOutputChannel enoc = new NetworkOutputChannel(outputGate, this.outputChannels.size(), + channelID, compressionLevel); addOutputChannel(enoc); @@ -265,10 +266,10 @@ public NetworkOutputChannel createNetworkOutputChannel(final ChannelID channe * {@inheritDoc} */ @Override - public FileOutputChannel createFileOutputChannel(final ChannelID channelID, + public FileOutputChannel createFileOutputChannel(final OutputGate outputGate, final ChannelID channelID, final CompressionLevel compressionLevel) { - final FileOutputChannel efoc = new FileOutputChannel(this, this.outputChannels.size(), channelID, + final FileOutputChannel efoc = new FileOutputChannel(outputGate, this.outputChannels.size(), channelID, compressionLevel); addOutputChannel(efoc); @@ -279,10 +280,10 @@ public FileOutputChannel createFileOutputChannel(final ChannelID channelID, * {@inheritDoc} */ @Override - public InMemoryOutputChannel createInMemoryOutputChannel(final ChannelID channelID, - final CompressionLevel compressionLevel) { + public InMemoryOutputChannel createInMemoryOutputChannel(final OutputGate outputGate, + final ChannelID channelID, final CompressionLevel compressionLevel) { - final InMemoryOutputChannel einoc = new InMemoryOutputChannel(this, this.outputChannels.size(), + final InMemoryOutputChannel einoc = new InMemoryOutputChannel(outputGate, this.outputChannels.size(), channelID, compressionLevel); addOutputChannel(einoc); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java index ba7b0c82e3770..9d9a1f2b04d71 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractInputGateWrapper.java @@ -262,29 +262,30 @@ public DistributionPattern getDistributionPattern() { * {@inheritDoc} */ @Override - public NetworkInputChannel createNetworkInputChannel(final ChannelID channelID, + public NetworkInputChannel createNetworkInputChannel(final InputGate inputGate, final ChannelID channelID, final CompressionLevel compressionLevel) { - return this.wrappedInputGate.createNetworkInputChannel(channelID, compressionLevel); + return this.wrappedInputGate.createNetworkInputChannel(inputGate, channelID, compressionLevel); } /** * {@inheritDoc} */ @Override - public FileInputChannel createFileInputChannel(final ChannelID channelID, final CompressionLevel compressionLevel) { + public FileInputChannel createFileInputChannel(final InputGate inputGate, final ChannelID channelID, + final CompressionLevel compressionLevel) { - return this.wrappedInputGate.createFileInputChannel(channelID, compressionLevel); + return this.wrappedInputGate.createFileInputChannel(inputGate, channelID, compressionLevel); } /** * {@inheritDoc} */ @Override - public InMemoryInputChannel createInMemoryInputChannel(final ChannelID channelID, + public InMemoryInputChannel createInMemoryInputChannel(final InputGate inputGate, final ChannelID channelID, final CompressionLevel compressionLevel) { - return this.wrappedInputGate.createInMemoryInputChannel(channelID, compressionLevel); + return this.wrappedInputGate.createInMemoryInputChannel(inputGate, channelID, compressionLevel); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java index 95d7f50260266..1a79bcc9aac09 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java @@ -289,30 +289,31 @@ public void removeAllOutputChannels() { * {@inheritDoc} */ @Override - public NetworkOutputChannel createNetworkOutputChannel(final ChannelID channelID, + public NetworkOutputChannel createNetworkOutputChannel(final OutputGate outputGate, + final ChannelID channelID, final CompressionLevel compressionLevel) { - return this.wrappedOutputGate.createNetworkOutputChannel(channelID, compressionLevel); + return this.wrappedOutputGate.createNetworkOutputChannel(outputGate, channelID, compressionLevel); } /** * {@inheritDoc} */ @Override - public FileOutputChannel createFileOutputChannel(final ChannelID channelID, + public FileOutputChannel createFileOutputChannel(final OutputGate outputGate, final ChannelID channelID, final CompressionLevel compressionLevel) { - return this.wrappedOutputGate.createFileOutputChannel(channelID, compressionLevel); + return this.wrappedOutputGate.createFileOutputChannel(outputGate, channelID, compressionLevel); } /** * {@inheritDoc} */ @Override - public InMemoryOutputChannel createInMemoryOutputChannel(final ChannelID channelID, - final CompressionLevel compressionLevel) { + public InMemoryOutputChannel createInMemoryOutputChannel(final OutputGate outputGate, + final ChannelID channelID, final CompressionLevel compressionLevel) { - return this.wrappedOutputGate.createInMemoryOutputChannel(channelID, compressionLevel); + return this.wrappedOutputGate.createInMemoryOutputChannel(outputGate, channelID, compressionLevel); } /** From 341b8a28818482c129689dd382aa148fb6ec9b4b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 19:44:49 +0000 Subject: [PATCH 076/310] Removed superfluous import --- .../eu/stratosphere/nephele/executiongraph/ExecutionStage.java | 1 - 1 file changed, 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java index 894509f58acf8..2cb773fa3032a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java @@ -32,7 +32,6 @@ import eu.stratosphere.nephele.instance.InstanceType; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelType; From 336b3345d441f2b435c28d008558dbba7d68e7e7 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 20 Nov 2011 20:03:13 +0000 Subject: [PATCH 077/310] Finished rewrite of JobGraph by Nephele streaming plugin --- .../streaming/StreamingJobManagerPlugin.java | 69 +++++++++++++++---- .../{ => wrapper}/StreamingEnvironment.java | 10 ++- .../wrapper/StreamingFileInputWrapper.java | 49 +++++++++++++ .../wrapper/StreamingFileOutputWrapper.java | 49 +++++++++++++ .../{ => wrapper}/StreamingInputGate.java | 2 +- .../wrapper/StreamingInputWrapper.java | 49 +++++++++++++ .../{ => wrapper}/StreamingOutputGate.java | 2 +- .../wrapper/StreamingOutputWrapper.java | 49 +++++++++++++ .../wrapper/StreamingTaskWrapper.java | 49 +++++++++++++ .../WrapperUtils.java} | 67 +++++++++--------- 10 files changed, 344 insertions(+), 51 deletions(-) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{ => wrapper}/StreamingEnvironment.java (90%) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileInputWrapper.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileOutputWrapper.java rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{ => wrapper}/StreamingInputGate.java (96%) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputWrapper.java rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{ => wrapper}/StreamingOutputGate.java (96%) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputWrapper.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingTaskWrapper.java rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{TaskWrapper.java => wrapper/WrapperUtils.java} (55%) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 8ad6c6f27afe7..169b63ae5c2d3 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -17,10 +17,8 @@ import java.io.IOException; import java.util.Iterator; -<<<<<<< HEAD + import java.util.concurrent.ConcurrentHashMap; -======= ->>>>>>> streaming_test import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -30,16 +28,24 @@ import eu.stratosphere.nephele.executiongraph.InternalJobStatus; import eu.stratosphere.nephele.executiongraph.JobStatusListener; import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.jobgraph.AbstractJobInputVertex; +import eu.stratosphere.nephele.jobgraph.AbstractJobOutputVertex; +import eu.stratosphere.nephele.jobgraph.JobFileInputVertex; +import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex; import eu.stratosphere.nephele.jobgraph.JobGraph; -<<<<<<< HEAD import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.jobgraph.JobInputVertex; +import eu.stratosphere.nephele.jobgraph.JobOutputVertex; import eu.stratosphere.nephele.plugins.JobManagerPlugin; import eu.stratosphere.nephele.streaming.latency.LatencyOptimizerThread; -======= +import eu.stratosphere.nephele.streaming.wrapper.StreamingFileInputWrapper; +import eu.stratosphere.nephele.streaming.wrapper.StreamingFileOutputWrapper; +import eu.stratosphere.nephele.streaming.wrapper.StreamingInputWrapper; +import eu.stratosphere.nephele.streaming.wrapper.StreamingOutputWrapper; +import eu.stratosphere.nephele.streaming.wrapper.StreamingTaskWrapper; +import eu.stratosphere.nephele.streaming.wrapper.WrapperUtils; import eu.stratosphere.nephele.jobgraph.JobTaskVertex; -import eu.stratosphere.nephele.plugins.JobManagerPlugin; import eu.stratosphere.nephele.template.AbstractInvokable; ->>>>>>> streaming_test public class StreamingJobManagerPlugin implements JobManagerPlugin, JobStatusListener { @@ -60,21 +66,58 @@ public class StreamingJobManagerPlugin implements JobManagerPlugin, JobStatusLis public JobGraph rewriteJobGraph(final JobGraph jobGraph) { // Rewrite input vertices - //final Iterator inputIt = jobGraph.getInputVertices(); + final Iterator inputIt = jobGraph.getInputVertices(); + while (inputIt.hasNext()) { + + final AbstractJobInputVertex abstractInputVertex = inputIt.next(); + final Class originalClass = abstractInputVertex.getInvokableClass(); + + if (abstractInputVertex instanceof JobFileInputVertex) { + final JobFileInputVertex fileInputVertex = (JobFileInputVertex) abstractInputVertex; + fileInputVertex.setFileInputClass(StreamingFileInputWrapper.class); + } else if (abstractInputVertex instanceof JobInputVertex) { + final JobInputVertex inputVertex = (JobInputVertex) abstractInputVertex; + inputVertex.setInputClass(StreamingInputWrapper.class); + } else { + LOG.warn("Cannot wrap input task of type " + originalClass + ", skipping..."); + continue; + } + + abstractInputVertex.getConfiguration().setString(WrapperUtils.WRAPPED_CLASS_KEY, originalClass.getName()); + } + // Rewrite the task vertices final Iterator taskIt = jobGraph.getTaskVertices(); while (taskIt.hasNext()) { final JobTaskVertex taskVertex = taskIt.next(); - final Class originalClass = taskVertex.getInvokableClass(); + taskVertex.setTaskClass(StreamingTaskWrapper.class); + taskVertex.getConfiguration().setString(WrapperUtils.WRAPPED_CLASS_KEY, originalClass.getName()); + } - taskVertex.setTaskClass(TaskWrapper.class); - taskVertex.getConfiguration().setString(TaskWrapper.WRAPPED_CLASS_KEY, originalClass.getName()); + // Rewrite the output vertices + final Iterator outputIt = jobGraph.getOutputVertices(); + while (outputIt.hasNext()) { + + final AbstractJobOutputVertex abstractOutputVertex = outputIt.next(); + final Class originalClass = abstractOutputVertex.getInvokableClass(); + + if (abstractOutputVertex instanceof JobFileOutputVertex) { + final JobFileOutputVertex fileOutputVertex = (JobFileOutputVertex) abstractOutputVertex; + fileOutputVertex.setFileOutputClass(StreamingFileOutputWrapper.class); + } else if (abstractOutputVertex instanceof JobOutputVertex) { + final JobOutputVertex outputVertex = (JobOutputVertex) abstractOutputVertex; + outputVertex.setOutputClass(StreamingOutputWrapper.class); + } else { + LOG.warn("Cannot wrap output task of type " + originalClass + ", skipping..."); + continue; + } + + abstractOutputVertex.getConfiguration().setString(WrapperUtils.WRAPPED_CLASS_KEY, originalClass.getName()); } - // TODO Auto-generated method stub - return null; + return jobGraph; } /** diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingEnvironment.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingEnvironment.java similarity index 90% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingEnvironment.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingEnvironment.java index b1da4b890c143..34a758a7dc3fb 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingEnvironment.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingEnvironment.java @@ -13,7 +13,7 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming; +package eu.stratosphere.nephele.streaming.wrapper; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.io.ChannelSelector; @@ -35,7 +35,13 @@ */ public final class StreamingEnvironment extends AbstractEnvironmentWrapper { - public StreamingEnvironment(final Environment wrappedEnvironment) { + /** + * Constructs a new straming environment + * + * @param wrappedEnvironment + * the environment to be encapsulated by this streaming environment + */ + StreamingEnvironment(final Environment wrappedEnvironment) { super(wrappedEnvironment); } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileInputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileInputWrapper.java new file mode 100644 index 0000000000000..4fadce928fe9d --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileInputWrapper.java @@ -0,0 +1,49 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.wrapper; + +import eu.stratosphere.nephele.template.AbstractFileInputTask; +import eu.stratosphere.nephele.template.AbstractInvokable; + +/** + * This class provides a wrapper for Nephele tasks of the type {@link AbstractFileInputTask}. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class StreamingFileInputWrapper extends AbstractFileInputTask { + + private volatile AbstractInvokable wrappedInvokable = null; + + /** + * {@inheritDoc} + */ + @Override + public void registerInputOutput() { + this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment()); + this.wrappedInvokable.registerInputOutput(); + } + + /** + * {@inheritDoc} + */ + @Override + public void invoke() throws Exception { + + this.wrappedInvokable.invoke(); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileOutputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileOutputWrapper.java new file mode 100644 index 0000000000000..45e5f529f9df6 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileOutputWrapper.java @@ -0,0 +1,49 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.wrapper; + +import eu.stratosphere.nephele.template.AbstractFileOutputTask; +import eu.stratosphere.nephele.template.AbstractInvokable; + +/** + * This class provides a wrapper for Nephele tasks of the type {@link AbstractFileOutputTask}. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class StreamingFileOutputWrapper extends AbstractFileOutputTask { + + private volatile AbstractInvokable wrappedInvokable = null; + + /** + * {@inheritDoc} + */ + @Override + public void registerInputOutput() { + this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment()); + this.wrappedInvokable.registerInputOutput(); + } + + /** + * {@inheritDoc} + */ + @Override + public void invoke() throws Exception { + + this.wrappedInvokable.invoke(); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingInputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputGate.java similarity index 96% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingInputGate.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputGate.java index 6a01601d42b72..a1753f7595066 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingInputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputGate.java @@ -13,7 +13,7 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming; +package eu.stratosphere.nephele.streaming.wrapper; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.plugins.wrapper.AbstractInputGateWrapper; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputWrapper.java new file mode 100644 index 0000000000000..a0aec1e93ee0a --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputWrapper.java @@ -0,0 +1,49 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.wrapper; + +import eu.stratosphere.nephele.template.AbstractGenericInputTask; +import eu.stratosphere.nephele.template.AbstractInvokable; + +/** + * This class provides a wrapper for Nephele tasks of the type {@link AbstractGenericInputTask}. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class StreamingInputWrapper extends AbstractGenericInputTask { + + private volatile AbstractInvokable wrappedInvokable = null; + + /** + * {@inheritDoc} + */ + @Override + public void registerInputOutput() { + this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment()); + this.wrappedInvokable.registerInputOutput(); + } + + /** + * {@inheritDoc} + */ + @Override + public void invoke() throws Exception { + + this.wrappedInvokable.invoke(); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingOutputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputGate.java similarity index 96% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingOutputGate.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputGate.java index 9154aab94ee22..ee070e266a50d 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingOutputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputGate.java @@ -13,7 +13,7 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming; +package eu.stratosphere.nephele.streaming.wrapper; import java.io.IOException; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputWrapper.java new file mode 100644 index 0000000000000..f9ddfb4ca84ec --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputWrapper.java @@ -0,0 +1,49 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.wrapper; + +import eu.stratosphere.nephele.template.AbstractInvokable; +import eu.stratosphere.nephele.template.AbstractOutputTask; + +/** + * This class provides a wrapper for Nephele tasks of the type {@link AbstractOutputTask}. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class StreamingOutputWrapper extends AbstractOutputTask { + + private volatile AbstractInvokable wrappedInvokable = null; + + /** + * {@inheritDoc} + */ + @Override + public void registerInputOutput() { + this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment()); + this.wrappedInvokable.registerInputOutput(); + } + + /** + * {@inheritDoc} + */ + @Override + public void invoke() throws Exception { + + this.wrappedInvokable.invoke(); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingTaskWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingTaskWrapper.java new file mode 100644 index 0000000000000..eef3b2d759eee --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingTaskWrapper.java @@ -0,0 +1,49 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.wrapper; + +import eu.stratosphere.nephele.template.AbstractInvokable; +import eu.stratosphere.nephele.template.AbstractTask; + +/** + * This class provides a wrapper for Nephele tasks of the type {@link AbstractTask}. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class StreamingTaskWrapper extends AbstractTask { + + private volatile AbstractInvokable wrappedInvokable = null; + + /** + * {@inheritDoc} + */ + @Override + public void registerInputOutput() { + this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment()); + this.wrappedInvokable.registerInputOutput(); + } + + /** + * {@inheritDoc} + */ + @Override + public void invoke() throws Exception { + + this.wrappedInvokable.invoke(); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/WrapperUtils.java similarity index 55% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskWrapper.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/WrapperUtils.java index 823d4e41e42cc..4700c05532996 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskWrapper.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/WrapperUtils.java @@ -13,30 +13,48 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming; +package eu.stratosphere.nephele.streaming.wrapper; import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.template.AbstractInvokable; -import eu.stratosphere.nephele.template.AbstractTask; import eu.stratosphere.nephele.util.StringUtils; -public final class TaskWrapper extends AbstractTask { +/** + * This class contains convenience methods to access wrapped Nephele task classes. + * + * @author warneke + */ +public final class WrapperUtils { - static final String WRAPPED_CLASS_KEY = "streaming.class.name"; + /** + * The configuration key to access the name of the wrapped class from the task configuration. + */ + public static final String WRAPPED_CLASS_KEY = "streaming.class.name"; - private AbstractInvokable wrappedInvokable = null; + /** + * Private constructor so class cannot be instantiated. + */ + private WrapperUtils() { + } - private synchronized AbstractInvokable getWrappedInvokable() { + /** + * Retrieves the name of the original class from the task configuration, loads the class, creates an instances of + * it, and finally wraps the given environment in an {@link StreamingEnvironment} object. + * + * @param environment + * the original environment + * @return an instance of the wrapped invokable class + */ + static AbstractInvokable getWrappedInvokable(final Environment environment) { - if (this.wrappedInvokable != null) { - return this.wrappedInvokable; - } + AbstractInvokable wrappedInvokable = null; - final Configuration conf = getEnvironment().getTaskConfiguration(); - final JobID jobID = getEnvironment().getJobID(); - final String className = conf.getString(WRAPPED_CLASS_KEY, null); + final Configuration taskConfiguration = environment.getTaskConfiguration(); + final JobID jobID = environment.getJobID(); + final String className = taskConfiguration.getString(WRAPPED_CLASS_KEY, null); if (className == null) { throw new IllegalStateException("Cannot find name of wrapped class"); } @@ -48,32 +66,13 @@ private synchronized AbstractInvokable getWrappedInvokable() { final Class invokableClass = (Class) Class .forName(className, true, cl); - this.wrappedInvokable = invokableClass.newInstance(); + wrappedInvokable = invokableClass.newInstance(); } catch (Exception e) { throw new RuntimeException(StringUtils.stringifyException(e)); } - this.wrappedInvokable.setEnvironment(new StreamingEnvironment(getEnvironment())); - - return this.wrappedInvokable; - } - - /** - * {@inheritDoc} - */ - @Override - public void registerInputOutput() { - - getWrappedInvokable().registerInputOutput(); - } - - /** - * {@inheritDoc} - */ - @Override - public void invoke() throws Exception { - - getWrappedInvokable().invoke(); + wrappedInvokable.setEnvironment(new StreamingEnvironment(environment)); + return wrappedInvokable; } } From baa563e9b8ac030db3fb1db5aed500c48a0a2c7b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 21 Nov 2011 20:44:06 +0000 Subject: [PATCH 078/310] Worked on task manager implementation of Nephele streaming plugin --- .../StreamingCommunicationThread.java | 8 +- .../streaming/StreamingJobManagerPlugin.java | 13 +- .../nephele/streaming/StreamingTag.java | 2 +- .../streaming/StreamingTaskListener.java | 233 ------------------ .../streaming/StreamingTaskManagerPlugin.java | 51 +++- .../streaming/latency/LatencyModel.java | 2 +- .../latency/LatencyOptimizerThread.java | 6 +- .../streaming/listeners/StreamListener.java | 182 ++++++++++++++ .../listeners/StreamListenerContext.java | 140 +++++++++++ .../{ => types}/AbstractStreamingData.java | 2 +- .../streaming/{ => types}/ChannelLatency.java | 2 +- .../streaming/{ => types}/TaskLatency.java | 2 +- .../StreamingEnvironment.java | 17 +- .../StreamingFileInputWrapper.java | 16 +- .../StreamingFileOutputWrapper.java | 16 +- .../StreamingInputGate.java | 13 +- .../StreamingInputWrapper.java | 16 +- .../StreamingOutputGate.java | 46 +++- .../StreamingOutputWrapper.java | 16 +- .../StreamingTaskWrapper.java | 16 +- .../{wrapper => wrappers}/WrapperUtils.java | 21 +- 21 files changed, 536 insertions(+), 284 deletions(-) delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{ => types}/AbstractStreamingData.java (97%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{ => types}/ChannelLatency.java (98%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{ => types}/TaskLatency.java (98%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{wrapper => wrappers}/StreamingEnvironment.java (83%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{wrapper => wrappers}/StreamingFileInputWrapper.java (80%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{wrapper => wrappers}/StreamingFileOutputWrapper.java (80%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{wrapper => wrappers}/StreamingInputGate.java (73%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{wrapper => wrappers}/StreamingInputWrapper.java (80%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{wrapper => wrappers}/StreamingOutputGate.java (50%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{wrapper => wrappers}/StreamingOutputWrapper.java (80%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{wrapper => wrappers}/StreamingTaskWrapper.java (80%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{wrapper => wrappers}/WrapperUtils.java (82%) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java index b6af5d0f2b58e..55a17c4eb97a2 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java @@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.plugins.PluginCommunication; +import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; import eu.stratosphere.nephele.util.StringUtils; /** @@ -34,7 +35,7 @@ * * @author warneke */ -final class StreamingCommunicationThread extends Thread { +public final class StreamingCommunicationThread extends Thread { /** * The log object. @@ -54,7 +55,8 @@ final class StreamingCommunicationThread extends Thread { /** * The blocking queue which is used to asynchronously exchange data with the job manager component of this plugin. */ - private final BlockingQueue dataQueue = new ArrayBlockingQueue(QUEUE_CAPACITY); + private final BlockingQueue dataQueue = new ArrayBlockingQueue( + QUEUE_CAPACITY); /** * Stores whether the communication thread has been requested to stop. @@ -109,7 +111,7 @@ void stopCommunicationThread() { * @throws InterruptedException * thrown if the thread is interrupted while waiting for the communication thread to accept the data */ - void sendDataAsynchronously(final AbstractStreamingData data) throws InterruptedException { + public void sendDataAsynchronously(final AbstractStreamingData data) throws InterruptedException { this.dataQueue.put(data); } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 169b63ae5c2d3..744a1a4acea2f 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -38,12 +38,13 @@ import eu.stratosphere.nephele.jobgraph.JobOutputVertex; import eu.stratosphere.nephele.plugins.JobManagerPlugin; import eu.stratosphere.nephele.streaming.latency.LatencyOptimizerThread; -import eu.stratosphere.nephele.streaming.wrapper.StreamingFileInputWrapper; -import eu.stratosphere.nephele.streaming.wrapper.StreamingFileOutputWrapper; -import eu.stratosphere.nephele.streaming.wrapper.StreamingInputWrapper; -import eu.stratosphere.nephele.streaming.wrapper.StreamingOutputWrapper; -import eu.stratosphere.nephele.streaming.wrapper.StreamingTaskWrapper; -import eu.stratosphere.nephele.streaming.wrapper.WrapperUtils; +import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; +import eu.stratosphere.nephele.streaming.wrappers.StreamingFileInputWrapper; +import eu.stratosphere.nephele.streaming.wrappers.StreamingFileOutputWrapper; +import eu.stratosphere.nephele.streaming.wrappers.StreamingInputWrapper; +import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputWrapper; +import eu.stratosphere.nephele.streaming.wrappers.StreamingTaskWrapper; +import eu.stratosphere.nephele.streaming.wrappers.WrapperUtils; import eu.stratosphere.nephele.jobgraph.JobTaskVertex; import eu.stratosphere.nephele.template.AbstractInvokable; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java index e39ecaa028b9d..a3f91eb915269 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java @@ -28,7 +28,7 @@ public final class StreamingTag implements Tag { private long timestamp = 0L; - StreamingTag(final ExecutionVertexID sourceID) { + public StreamingTag(final ExecutionVertexID sourceID) { if (sourceID == null) { throw new IllegalArgumentException("sourceID must not be null"); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java deleted file mode 100644 index a8b95845f014b..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskListener.java +++ /dev/null @@ -1,233 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.InputGateListener; -import eu.stratosphere.nephele.io.OutputGateListener; -import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.types.AbstractTaggableRecord; -import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.util.StringUtils; - -public final class StreamingTaskListener implements InputGateListener, OutputGateListener { - - /** - * The log object. - */ - private static final Log LOG = LogFactory.getLog(StreamingTaskListener.class); - - private static enum TaskType { - INPUT, REGULAR, OUTPUT - }; - - private final static double ALPHA = 0.5; - - private final StreamingCommunicationThread communicationThread; - - private final JobID jobID; - - private final ExecutionVertexID vertexID; - - private final TaskType taskType; - - private final int taggingInterval; - - private final int aggregationInterval; - - private StreamingTag tag = null; - - private int tagCounter = 0; - - private long lastTimestamp = -1L; - - private Map aggregationCounter = new HashMap(); - - private Map aggregatedValue = new HashMap(); - - static StreamingTaskListener createForInputTask(final StreamingCommunicationThread communicationThread, - final JobID jobID, final ExecutionVertexID vertexID, final int taggingInterval, - final int aggregationInterval) { - - return new StreamingTaskListener(communicationThread, jobID, vertexID, TaskType.INPUT, taggingInterval, - aggregationInterval); - } - - static StreamingTaskListener createForRegularTask(final StreamingCommunicationThread communicationThread, - final JobID jobID, final ExecutionVertexID vertexID, final int aggregationInterval) { - - return new StreamingTaskListener(communicationThread, jobID, vertexID, TaskType.REGULAR, 0, aggregationInterval); - } - - static StreamingTaskListener createForOutputTask(final StreamingCommunicationThread communicationThread, - final JobID jobID, final ExecutionVertexID vertexID, final int aggregationInterval) { - - return new StreamingTaskListener(communicationThread, jobID, vertexID, TaskType.OUTPUT, 0, aggregationInterval); - } - - private StreamingTaskListener(final StreamingCommunicationThread communicationThread, final JobID jobID, - final ExecutionVertexID vertexID, final TaskType taskType, final int taggingInterval, - final int aggregationInterval) { - - this.communicationThread = communicationThread; - this.jobID = jobID; - this.vertexID = vertexID; - this.taskType = taskType; - this.taggingInterval = taggingInterval; - this.aggregationInterval = aggregationInterval; - } - - /** - * {@inheritDoc} - */ - @Override - public void channelCapacityExhausted(final int channelIndex) { - // TODO Auto-generated method stub - - } - - /** - * {@inheritDoc} - */ - public void recordEmitted(final Record record) { - - switch (this.taskType) { - case INPUT: - if (this.tagCounter++ == this.taggingInterval) { - final long timestamp = System.currentTimeMillis(); - final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - taggableRecord.setTag(createTag(timestamp)); - if (this.lastTimestamp > 0) { - final long taskLatency = (timestamp - this.lastTimestamp) / this.taggingInterval; - try { - this.communicationThread.sendDataAsynchronously(new TaskLatency(this.jobID, this.vertexID, - taskLatency)); - } catch (InterruptedException e) { - LOG.error(StringUtils.stringifyException(e)); - } - } - this.lastTimestamp = timestamp; - this.tagCounter = 0; - } - break; - case REGULAR: - final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - if (this.tag == null) { - taggableRecord.setTag(null); - } else { - final long timestamp = System.currentTimeMillis(); - this.tag = createTag(timestamp); - taggableRecord.setTag(this.tag); - this.lastTimestamp = timestamp; - } - break; - case OUTPUT: - throw new IllegalStateException("Output task emitted record"); - } - - } - - /** - * {@inheritDoc} - */ - @Override - public void waitingForAnyChannel() { - // TODO Auto-generated method stub - - } - - /** - * {@inheritDoc} - */ - public void recordReceived(final Record record) { - - if (this.taskType == TaskType.INPUT) { - throw new IllegalStateException("Input task received record"); - } - - final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - this.tag = (StreamingTag) taggableRecord.getTag(); - if (this.tag != null) { - - final long timestamp = System.currentTimeMillis(); - if (this.lastTimestamp > 0) { - try { - this.communicationThread.sendDataAsynchronously(new TaskLatency(jobID, vertexID, timestamp - - this.lastTimestamp)); - } catch (InterruptedException e) { - LOG.error(StringUtils.stringifyException(e)); - } - if (this.taskType == TaskType.REGULAR) { - this.lastTimestamp = -1L; - } else { - this.lastTimestamp = timestamp; - } - } - - final long pathLatency = timestamp - this.tag.getTimestamp(); - - final ExecutionVertexID sourceID = this.tag.getSourceID(); - - // Calculate moving average - Double aggregatedLatency = this.aggregatedValue.get(sourceID); - if (aggregatedLatency == null) { - aggregatedLatency = Double.valueOf(pathLatency); - } else { - aggregatedLatency = Double.valueOf((ALPHA * pathLatency) - + ((1 - ALPHA) * aggregatedLatency.doubleValue())); - } - this.aggregatedValue.put(sourceID, aggregatedLatency); - - // Check if we need to compute an event and send it to the job manager component - Integer counter = this.aggregationCounter.get(sourceID); - if (counter == null) { - counter = Integer.valueOf(0); - } - - counter = Integer.valueOf(counter.intValue() + 1); - if (counter.intValue() == this.aggregationInterval) { - - final ChannelLatency pl = new ChannelLatency(this.jobID, sourceID, this.vertexID, - aggregatedLatency.doubleValue()); - - try { - this.communicationThread.sendDataAsynchronously(pl); - } catch (InterruptedException e) { - LOG.warn(StringUtils.stringifyException(e)); - } - - counter = Integer.valueOf(0); - } - this.aggregationCounter.put(sourceID, counter); - } - - } - - private StreamingTag createTag(final long timestamp) { - - this.tag = new StreamingTag(this.vertexID); - this.tag.setTimestamp(timestamp); - - return this.tag; - } - -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index d8bb2ea5e1ab3..0141d47942368 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -16,19 +16,17 @@ package eu.stratosphere.nephele.streaming; import java.io.IOException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.IOReadableWritable; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.RuntimeInputGate; -import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.plugins.PluginCommunication; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; -import eu.stratosphere.nephele.types.Record; +import eu.stratosphere.nephele.streaming.listeners.StreamListenerContext; public class StreamingTaskManagerPlugin implements TaskManagerPlugin { @@ -53,6 +51,16 @@ public class StreamingTaskManagerPlugin implements TaskManagerPlugin { */ private static final int DEFAULT_AGGREGATION_INTERVAL = 10; + /** + * Stores the instance of the streaming task manager plugin. + */ + private static volatile StreamingTaskManagerPlugin INSTANCE = null; + + /** + * Map storing the listener context objects for the individual stream listners. + */ + private final ConcurrentMap listenerContexts = new ConcurrentHashMap(); + /** * The tagging interval as specified in the plugin configuration. */ @@ -76,6 +84,17 @@ public class StreamingTaskManagerPlugin implements TaskManagerPlugin { this.communicationThread = new StreamingCommunicationThread(jobManagerComponent); this.communicationThread.start(); + + INSTANCE = this; + } + + public static StreamListenerContext getStreamingListenerContext(final String listenerKey) { + + if (INSTANCE == null) { + throw new IllegalStateException("StreamingTaskManagerPlugin has not been initialized"); + } + + return INSTANCE.listenerContexts.get(listenerKey); } /** @@ -99,7 +118,25 @@ public void registerTask(final ExecutionVertexID id, final Configuration jobConf this.aggregationInterval); final int taggingInterval = jobConfiguration.getInteger(TAGGING_INTERVAL_KEY, this.taggingInterval); - + + final String idAsString = id.toString(); + + environment.getTaskConfiguration().setString(StreamListenerContext.CONTEXT_CONFIGURATION_KEY, idAsString); + + final JobID jobID = environment.getJobID(); + StreamListenerContext listenerContext = null; + if (environment.getNumberOfInputGates() == 0) { + listenerContext = StreamListenerContext.createForInputTask(jobID, id, this.communicationThread, + aggregationInterval, taggingInterval); + } else if (environment.getNumberOfOutputGates() == 0) { + listenerContext = StreamListenerContext.createForOutputTask(jobID, id, this.communicationThread, + aggregationInterval); + } else { + listenerContext = StreamListenerContext.createForRegularTask(jobID, id, this.communicationThread, + aggregationInterval); + } + + this.listenerContexts.putIfAbsent(idAsString, listenerContext); } /** @@ -108,7 +145,7 @@ public void registerTask(final ExecutionVertexID id, final Configuration jobConf @Override public void unregisterTask(final ExecutionVertexID id, final RuntimeEnvironment environment) { - // Nothing to do here + this.listenerContexts.remove(id.toString()); } /** diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java index 3a8f88c2d770e..67ecfc77979cb 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java @@ -7,7 +7,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; -import eu.stratosphere.nephele.streaming.ChannelLatency; +import eu.stratosphere.nephele.streaming.types.ChannelLatency; public class LatencyModel { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java index 2536c98707e2d..7610fc13765a1 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java @@ -6,9 +6,9 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.streaming.AbstractStreamingData; -import eu.stratosphere.nephele.streaming.ChannelLatency; -import eu.stratosphere.nephele.streaming.TaskLatency; +import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; +import eu.stratosphere.nephele.streaming.types.ChannelLatency; +import eu.stratosphere.nephele.streaming.types.TaskLatency; public class LatencyOptimizerThread extends Thread { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java new file mode 100644 index 0000000000000..22e526df519ec --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java @@ -0,0 +1,182 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.listeners; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.streaming.StreamingTag; +import eu.stratosphere.nephele.streaming.StreamingTaskManagerPlugin; +import eu.stratosphere.nephele.streaming.types.TaskLatency; +import eu.stratosphere.nephele.types.AbstractTaggableRecord; +import eu.stratosphere.nephele.types.Record; +import eu.stratosphere.nephele.util.StringUtils; + +public final class StreamListener { + + /** + * The log object. + */ + private static final Log LOG = LogFactory.getLog(StreamListener.class); + + private final static double ALPHA = 0.5; + + private final Configuration taskConfiguration; + + private StreamListenerContext listenerContext = null; + + private StreamingTag tag = null; + + private int tagCounter = 0; + + private long lastTimestamp = -1L; + + private Map aggregationCounter = new HashMap(); + + private Map aggregatedValue = new HashMap(); + + public StreamListener(final Configuration taskConfiguration) { + + if (taskConfiguration == null) { + throw new IllegalArgumentException("Argument taskConfiguration must not be null"); + } + + this.taskConfiguration = taskConfiguration; + } + + /** + * Initializes the stream listener by retrieving the listener context from the task manager plugin. + */ + public void init() { + + final String listenerKey = this.taskConfiguration.getString(StreamListenerContext.CONTEXT_CONFIGURATION_KEY, + null); + + if (listenerKey == null) { + throw new RuntimeException("Stream listener is unable to retrieve context key"); + } + + this.listenerContext = StreamingTaskManagerPlugin.getStreamingListenerContext(listenerKey); + } + + public long recordEmitted(final Record record) { + + long timestamp = -1L; + + // Input vertex + if (this.listenerContext.isInputVertex()) { + final int taggingInterval = this.listenerContext.getTaggingInterval(); + if (this.tagCounter++ == taggingInterval) { + timestamp = System.currentTimeMillis(); + final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; + taggableRecord.setTag(createTag(timestamp)); + if (this.lastTimestamp > 0L) { + final long taskLatency = (timestamp - this.lastTimestamp) / taggingInterval; + try { + this.listenerContext.sendDataAsynchronously(new TaskLatency(this.listenerContext.getJobID(), + this.listenerContext.getVertexID(), taskLatency)); + } catch (InterruptedException e) { + LOG.error(StringUtils.stringifyException(e)); + } + } + this.lastTimestamp = timestamp; + this.tagCounter = 0; + } + } else { + final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; + if (this.tag == null) { + taggableRecord.setTag(null); + } else { + timestamp = System.currentTimeMillis(); + this.tag = createTag(timestamp); + taggableRecord.setTag(this.tag); + this.lastTimestamp = timestamp; + } + } + + return timestamp; + } + + /** + * {@inheritDoc} + */ + public void recordReceived(final Record record) { + + /* + * if (this.taskType == TaskType.INPUT) { + * throw new IllegalStateException("Input task received record"); + * } + * final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; + * this.tag = (StreamingTag) taggableRecord.getTag(); + * if (this.tag != null) { + * final long timestamp = System.currentTimeMillis(); + * if (this.lastTimestamp > 0) { + * try { + * this.communicationThread.sendDataAsynchronously(new TaskLatency(jobID, vertexID, timestamp + * - this.lastTimestamp)); + * } catch (InterruptedException e) { + * LOG.error(StringUtils.stringifyException(e)); + * } + * if (this.taskType == TaskType.REGULAR) { + * this.lastTimestamp = -1L; + * } else { + * this.lastTimestamp = timestamp; + * } + * } + * final long pathLatency = timestamp - this.tag.getTimestamp(); + * final ExecutionVertexID sourceID = this.tag.getSourceID(); + * // Calculate moving average + * Double aggregatedLatency = this.aggregatedValue.get(sourceID); + * if (aggregatedLatency == null) { + * aggregatedLatency = Double.valueOf(pathLatency); + * } else { + * aggregatedLatency = Double.valueOf((ALPHA * pathLatency) + * + ((1 - ALPHA) * aggregatedLatency.doubleValue())); + * } + * this.aggregatedValue.put(sourceID, aggregatedLatency); + * // Check if we need to compute an event and send it to the job manager component + * Integer counter = this.aggregationCounter.get(sourceID); + * if (counter == null) { + * counter = Integer.valueOf(0); + * } + * counter = Integer.valueOf(counter.intValue() + 1); + * if (counter.intValue() == this.aggregationInterval) { + * final ChannelLatency pl = new ChannelLatency(this.jobID, sourceID, this.vertexID, + * aggregatedLatency.doubleValue()); + * try { + * this.communicationThread.sendDataAsynchronously(pl); + * } catch (InterruptedException e) { + * LOG.warn(StringUtils.stringifyException(e)); + * } + * counter = Integer.valueOf(0); + * } + * this.aggregationCounter.put(sourceID, counter); + * } + */ + + } + + private StreamingTag createTag(final long timestamp) { + this.tag = new StreamingTag(this.listenerContext.getVertexID()); + this.tag.setTimestamp(timestamp); + return this.tag; + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java new file mode 100644 index 0000000000000..c860836c71fd3 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java @@ -0,0 +1,140 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.listeners; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.streaming.StreamingCommunicationThread; +import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; + +public final class StreamListenerContext { + + public static final String CONTEXT_CONFIGURATION_KEY = "streaming.listener.context"; + + private static enum TaskType { + INPUT, REGULAR, OUTPUT + }; + + private final JobID jobID; + + private final ExecutionVertexID vertexID; + + private final StreamingCommunicationThread communicationThread; + + private final TaskType taskType; + + private final int aggregationInterval; + + private final int taggingInterval; + + private StreamListenerContext(final JobID jobID, final ExecutionVertexID vertexID, + final StreamingCommunicationThread communicationThread, final TaskType taskType, + final int aggregationInterval, final int taggingInterval) { + + if (jobID == null) { + throw new IllegalArgumentException("Parameter jobID must not be null"); + } + + if (vertexID == null) { + throw new IllegalArgumentException("Parameter vertexID must not be null"); + } + + if (communicationThread == null) { + throw new IllegalArgumentException("Parameter communicationThread must not be null"); + } + + if (taskType == null) { + throw new IllegalArgumentException("Parameter taskType must not be null"); + } + + if (aggregationInterval <= 0) { + throw new IllegalArgumentException("Parameter aggregationInterval must be greater than zero"); + } + + if (taggingInterval <= 0 && taskType == TaskType.INPUT) { + throw new IllegalArgumentException("Parameter taggingInterval must be greater than zero"); + } + + this.jobID = jobID; + this.vertexID = vertexID; + this.communicationThread = communicationThread; + this.taskType = taskType; + this.aggregationInterval = aggregationInterval; + this.taggingInterval = taggingInterval; + } + + public static StreamListenerContext createForInputTask(final JobID jobID, final ExecutionVertexID vertexID, + final StreamingCommunicationThread communicationThread, final int aggregationInterval, + final int taggingInterval) { + + return new StreamListenerContext(jobID, vertexID, communicationThread, TaskType.INPUT, aggregationInterval, + taggingInterval); + } + + public static StreamListenerContext createForRegularTask(final JobID jobID, final ExecutionVertexID vertexID, + final StreamingCommunicationThread communicationThread, final int aggregationInterval) { + + return new StreamListenerContext(jobID, vertexID, communicationThread, TaskType.REGULAR, aggregationInterval, + -1); + } + + public static StreamListenerContext createForOutputTask(final JobID jobID, final ExecutionVertexID vertexID, + final StreamingCommunicationThread communicationThread, final int aggregationInterval) { + + return new StreamListenerContext(jobID, vertexID, communicationThread, TaskType.OUTPUT, aggregationInterval, + -1); + } + + boolean isInputVertex() { + + return (this.taskType == TaskType.INPUT); + } + + boolean isOutputVertex() { + + return (this.taskType == TaskType.OUTPUT); + } + + boolean isRegularVertex() { + + return (this.taskType == TaskType.REGULAR); + } + + JobID getJobID() { + + return this.jobID; + } + + ExecutionVertexID getVertexID() { + + return this.vertexID; + } + + int getTaggingInterval() { + + return this.taggingInterval; + } + + int getAggregationInterval() { + + return this.aggregationInterval; + } + + void sendDataAsynchronously(final AbstractStreamingData data) throws InterruptedException { + + this.communicationThread.sendDataAsynchronously(data); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/AbstractStreamingData.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/AbstractStreamingData.java similarity index 97% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/AbstractStreamingData.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/AbstractStreamingData.java index a6ecaaf99f5f1..032c6df1ed30c 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/AbstractStreamingData.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/AbstractStreamingData.java @@ -13,7 +13,7 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming; +package eu.stratosphere.nephele.streaming.types; import java.io.DataInput; import java.io.DataOutput; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/ChannelLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelLatency.java similarity index 98% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/ChannelLatency.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelLatency.java index 20d327e69bafa..32efd82811cda 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/ChannelLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelLatency.java @@ -13,7 +13,7 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming; +package eu.stratosphere.nephele.streaming.types; import java.io.DataInput; import java.io.DataOutput; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/TaskLatency.java similarity index 98% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskLatency.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/TaskLatency.java index a6f0e4740b40a..18afb543c6011 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/TaskLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/TaskLatency.java @@ -13,7 +13,7 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming; +package eu.stratosphere.nephele.streaming.types; import java.io.DataInput; import java.io.DataOutput; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingEnvironment.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java similarity index 83% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingEnvironment.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java index 34a758a7dc3fb..baa2788ac0ab8 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingEnvironment.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java @@ -13,7 +13,7 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming.wrapper; +package eu.stratosphere.nephele.streaming.wrappers; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.io.ChannelSelector; @@ -23,6 +23,7 @@ import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RecordDeserializer; import eu.stratosphere.nephele.plugins.wrapper.AbstractEnvironmentWrapper; +import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.types.Record; /** @@ -35,14 +36,20 @@ */ public final class StreamingEnvironment extends AbstractEnvironmentWrapper { + private final StreamListener streamListener; + /** - * Constructs a new straming environment + * Constructs a new streaming environment * * @param wrappedEnvironment * the environment to be encapsulated by this streaming environment + * @param streamListener + * the stream listener */ - StreamingEnvironment(final Environment wrappedEnvironment) { + StreamingEnvironment(final Environment wrappedEnvironment, final StreamListener streamListener) { super(wrappedEnvironment); + + this.streamListener = streamListener; } /** @@ -57,7 +64,7 @@ public OutputGate createOutputGate(final GateID gateID, final OutputGate outputGate = getWrappedEnvironment().createOutputGate(gateID, outputClass, selector, isBroadcast); - return new StreamingOutputGate(outputGate); + return new StreamingOutputGate(outputGate, this.streamListener); } /** @@ -71,6 +78,6 @@ public InputGate createInputGate(final GateID gateID, final InputGate inputGate = getWrappedEnvironment().createInputGate(gateID, deserializer, distributionPattern); - return new StreamingInputGate(inputGate); + return new StreamingInputGate(inputGate, this.streamListener); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileInputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileInputWrapper.java similarity index 80% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileInputWrapper.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileInputWrapper.java index 4fadce928fe9d..f4c3d4a2ce734 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileInputWrapper.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileInputWrapper.java @@ -13,8 +13,9 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming.wrapper; +package eu.stratosphere.nephele.streaming.wrappers; +import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.template.AbstractFileInputTask; import eu.stratosphere.nephele.template.AbstractInvokable; @@ -27,14 +28,24 @@ */ public final class StreamingFileInputWrapper extends AbstractFileInputTask { + /** + * The wrapped task. + */ private volatile AbstractInvokable wrappedInvokable = null; + /** + * The stream listener object. + */ + private volatile StreamListener streamListener = null; + /** * {@inheritDoc} */ @Override public void registerInputOutput() { - this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment()); + + this.streamListener = WrapperUtils.createStreamListener(getEnvironment()); + this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment(), this.streamListener); this.wrappedInvokable.registerInputOutput(); } @@ -44,6 +55,7 @@ public void registerInputOutput() { @Override public void invoke() throws Exception { + this.streamListener.init(); this.wrappedInvokable.invoke(); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileOutputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileOutputWrapper.java similarity index 80% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileOutputWrapper.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileOutputWrapper.java index 45e5f529f9df6..4db0e2b64ea0d 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingFileOutputWrapper.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileOutputWrapper.java @@ -13,8 +13,9 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming.wrapper; +package eu.stratosphere.nephele.streaming.wrappers; +import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.template.AbstractFileOutputTask; import eu.stratosphere.nephele.template.AbstractInvokable; @@ -27,14 +28,24 @@ */ public final class StreamingFileOutputWrapper extends AbstractFileOutputTask { + /** + * The wrapped task. + */ private volatile AbstractInvokable wrappedInvokable = null; + /** + * The stream listener object. + */ + private volatile StreamListener streamListener = null; + /** * {@inheritDoc} */ @Override public void registerInputOutput() { - this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment()); + + this.streamListener = WrapperUtils.createStreamListener(getEnvironment()); + this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment(), this.streamListener); this.wrappedInvokable.registerInputOutput(); } @@ -44,6 +55,7 @@ public void registerInputOutput() { @Override public void invoke() throws Exception { + this.streamListener.init(); this.wrappedInvokable.invoke(); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java similarity index 73% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputGate.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java index a1753f7595066..47fe7855bc4e5 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java @@ -13,15 +13,24 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming.wrapper; +package eu.stratosphere.nephele.streaming.wrappers; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.plugins.wrapper.AbstractInputGateWrapper; +import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.types.Record; public final class StreamingInputGate extends AbstractInputGateWrapper { - StreamingInputGate(final InputGate wrappedInputGate) { + private final StreamListener streamListener; + + StreamingInputGate(final InputGate wrappedInputGate, final StreamListener streamListener) { super(wrappedInputGate); + + if (streamListener == null) { + throw new IllegalArgumentException("Argument streamListener must not be null"); + } + + this.streamListener = streamListener; } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputWrapper.java similarity index 80% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputWrapper.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputWrapper.java index a0aec1e93ee0a..308320762996f 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingInputWrapper.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputWrapper.java @@ -13,8 +13,9 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming.wrapper; +package eu.stratosphere.nephele.streaming.wrappers; +import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.template.AbstractGenericInputTask; import eu.stratosphere.nephele.template.AbstractInvokable; @@ -27,14 +28,24 @@ */ public final class StreamingInputWrapper extends AbstractGenericInputTask { + /** + * The wrapped task. + */ private volatile AbstractInvokable wrappedInvokable = null; + /** + * The stream listener object. + */ + private volatile StreamListener streamListener = null; + /** * {@inheritDoc} */ @Override public void registerInputOutput() { - this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment()); + + this.streamListener = WrapperUtils.createStreamListener(getEnvironment()); + this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment(), this.streamListener); this.wrappedInvokable.registerInputOutput(); } @@ -44,6 +55,7 @@ public void registerInputOutput() { @Override public void invoke() throws Exception { + this.streamListener.init(); this.wrappedInvokable.invoke(); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java similarity index 50% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputGate.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java index ee070e266a50d..b86adf7196464 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java @@ -13,30 +13,62 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming.wrapper; +package eu.stratosphere.nephele.streaming.wrappers; import java.io.IOException; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.plugins.wrapper.AbstractOutputGateWrapper; +import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.types.Record; public final class StreamingOutputGate extends AbstractOutputGateWrapper { - StreamingOutputGate(final OutputGate wrappedOutputGate) { + private final StreamListener streamListener; + + private long lastTimestamp = -1L; + + private long[] lastSentBytes = null; + + StreamingOutputGate(final OutputGate wrappedOutputGate, final StreamListener streamListener) { super(wrappedOutputGate); - - System.out.println("STREAMING CREATED"); + + if (streamListener == null) { + throw new IllegalArgumentException("Argument streamListener must not be null"); + } + + this.streamListener = streamListener; } - + /** * {@inheritDoc} */ @Override public void writeRecord(final T record) throws IOException, InterruptedException { - System.out.println("DADADADADADADAD"); - + final long timestamp = this.streamListener.recordEmitted(record); + if (timestamp >= 0) { + + final int numberOfOutputChannels = getNumberOfOutputChannels(); + + if (this.lastTimestamp < 0) { + // Initialize array and fill it + this.lastSentBytes = new long[numberOfOutputChannels]; + for (int i = 0; i < numberOfOutputChannels; ++i) { + this.lastSentBytes[i] = getOutputChannel(i).getAmountOfDataTransmitted(); + } + } else { + for (int i = 0; i < numberOfOutputChannels; ++i) { + final long amountOfDataTransmitted = getOutputChannel(i).getAmountOfDataTransmitted(); + final long dataDiff = amountOfDataTransmitted - this.lastSentBytes[i]; + this.lastSentBytes[i] = amountOfDataTransmitted; + System.out.println("Data diff " + dataDiff); + } + } + + this.lastTimestamp = timestamp; + } + getWrappedOutputGate().writeRecord(record); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputWrapper.java similarity index 80% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputWrapper.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputWrapper.java index f9ddfb4ca84ec..e73c8b9b36a85 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingOutputWrapper.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputWrapper.java @@ -13,8 +13,9 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming.wrapper; +package eu.stratosphere.nephele.streaming.wrappers; +import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.template.AbstractInvokable; import eu.stratosphere.nephele.template.AbstractOutputTask; @@ -27,14 +28,24 @@ */ public final class StreamingOutputWrapper extends AbstractOutputTask { + /** + * The wrapped task. + */ private volatile AbstractInvokable wrappedInvokable = null; + /** + * The stream listener object. + */ + private volatile StreamListener streamListener = null; + /** * {@inheritDoc} */ @Override public void registerInputOutput() { - this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment()); + + this.streamListener = WrapperUtils.createStreamListener(getEnvironment()); + this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment(), this.streamListener); this.wrappedInvokable.registerInputOutput(); } @@ -44,6 +55,7 @@ public void registerInputOutput() { @Override public void invoke() throws Exception { + this.streamListener.init(); this.wrappedInvokable.invoke(); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingTaskWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingTaskWrapper.java similarity index 80% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingTaskWrapper.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingTaskWrapper.java index eef3b2d759eee..dba4d69adfd24 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/StreamingTaskWrapper.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingTaskWrapper.java @@ -13,8 +13,9 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming.wrapper; +package eu.stratosphere.nephele.streaming.wrappers; +import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.template.AbstractInvokable; import eu.stratosphere.nephele.template.AbstractTask; @@ -27,14 +28,24 @@ */ public final class StreamingTaskWrapper extends AbstractTask { + /** + * The wrapped task. + */ private volatile AbstractInvokable wrappedInvokable = null; + /** + * The stream listener object. + */ + private volatile StreamListener streamListener = null; + /** * {@inheritDoc} */ @Override public void registerInputOutput() { - this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment()); + + this.streamListener = WrapperUtils.createStreamListener(getEnvironment()); + this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment(), this.streamListener); this.wrappedInvokable.registerInputOutput(); } @@ -44,6 +55,7 @@ public void registerInputOutput() { @Override public void invoke() throws Exception { + this.streamListener.init(); this.wrappedInvokable.invoke(); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/WrapperUtils.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/WrapperUtils.java similarity index 82% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/WrapperUtils.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/WrapperUtils.java index 4700c05532996..eb43b4dbc1496 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrapper/WrapperUtils.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/WrapperUtils.java @@ -13,12 +13,13 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.streaming.wrapper; +package eu.stratosphere.nephele.streaming.wrappers; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.template.AbstractInvokable; import eu.stratosphere.nephele.util.StringUtils; @@ -46,9 +47,11 @@ private WrapperUtils() { * * @param environment * the original environment + * @param streamListener + * the stream listener object * @return an instance of the wrapped invokable class */ - static AbstractInvokable getWrappedInvokable(final Environment environment) { + static AbstractInvokable getWrappedInvokable(final Environment environment, final StreamListener streamListener) { AbstractInvokable wrappedInvokable = null; @@ -71,8 +74,20 @@ static AbstractInvokable getWrappedInvokable(final Environment environment) { throw new RuntimeException(StringUtils.stringifyException(e)); } - wrappedInvokable.setEnvironment(new StreamingEnvironment(environment)); + wrappedInvokable.setEnvironment(new StreamingEnvironment(environment, streamListener)); return wrappedInvokable; } + + /** + * Creates and configures a new stream listener. + * + * @param environment + * the environment for the newly created stream listener + * @return the configured stream listener + */ + static StreamListener createStreamListener(final Environment environment) { + + return new StreamListener(environment.getTaskConfiguration()); + } } From 11a76db19b729d2a9abc167d245133ec5b6e4b35 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 22 Nov 2011 10:43:37 +0000 Subject: [PATCH 079/310] Finished implementation of channel throughput measurements --- .../streaming/listeners/StreamListener.java | 12 ++ .../streaming/types/ChannelThroughput.java | 144 ++++++++++++++++++ .../wrappers/StreamingOutputGate.java | 8 +- 3 files changed, 162 insertions(+), 2 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelThroughput.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java index 22e526df519ec..adf3d9ebd35db 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java @@ -23,8 +23,10 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.streaming.StreamingTag; import eu.stratosphere.nephele.streaming.StreamingTaskManagerPlugin; +import eu.stratosphere.nephele.streaming.types.ChannelThroughput; import eu.stratosphere.nephele.streaming.types.TaskLatency; import eu.stratosphere.nephele.types.AbstractTaggableRecord; import eu.stratosphere.nephele.types.Record; @@ -174,6 +176,16 @@ public void recordReceived(final Record record) { } + public void reportChannelThroughput(final ChannelID sourceChannelID, final double throughput) { + + try { + this.listenerContext.sendDataAsynchronously(new ChannelThroughput(this.listenerContext.getJobID(), + this.listenerContext.getVertexID(), sourceChannelID, throughput)); + } catch (InterruptedException e) { + LOG.error(StringUtils.stringifyException(e)); + } + } + private StreamingTag createTag(final long timestamp) { this.tag = new StreamingTag(this.listenerContext.getVertexID()); this.tag.setTimestamp(timestamp); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelThroughput.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelThroughput.java new file mode 100644 index 0000000000000..de6450373637d --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelThroughput.java @@ -0,0 +1,144 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.types; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.jobgraph.JobID; + +/** + * This class stores information about the throughput of a specific output channel. + * + * @author warneke + */ +public final class ChannelThroughput extends AbstractStreamingData { + + /** + * The ID of the vertex which is connected to this output channel. + */ + private final ExecutionVertexID vertexID; + + /** + * The ID of the output channel. + */ + private final ChannelID sourceChannelID; + + /** + * The throughput in MBit/s. + */ + private double throughput; + + /** + * Constructs a new channel throughput object. + * + * @param jobID + * the ID of the job this channel throughput object belongs to + * @param vertexID + * the ID of the vertex which is connected to this output channel + * @param sourceChannelID + * the ID of the output channel + * @param throughput + * the throughput in MBit/s + */ + public ChannelThroughput(final JobID jobID, final ExecutionVertexID vertexID, final ChannelID sourceChannelID, + final double throughput) { + super(jobID); + + if (vertexID == null) { + throw new IllegalArgumentException("Argument vertexID must not be null"); + } + + if (sourceChannelID == null) { + throw new IllegalArgumentException("Argument sourceChannelID must not be null"); + } + + if (throughput < 0.0) { + throw new IllegalArgumentException("Argument throughput must not be positive"); + } + + this.vertexID = vertexID; + this.sourceChannelID = sourceChannelID; + this.throughput = throughput; + } + + /** + * Default constructor for deserialization. + */ + public ChannelThroughput() { + super(new JobID()); + this.vertexID = new ExecutionVertexID(); + this.sourceChannelID = new ChannelID(); + this.throughput = 0.0; + } + + /** + * The ID of the vertex which is connected to the output channel. + * + * @return the ID of the vertex which is connected to the output channel + */ + public ExecutionVertexID getVertexID() { + + return this.vertexID; + } + + /** + * The ID of the output channel. + * + * @return the ID of the output channel. + */ + public ChannelID getSourceChannelID() { + + return this.sourceChannelID; + } + + /** + * Returns the measured throughput for the channel in MBit/s. + * + * @return the measured throughput in MBit/s. + */ + public double getThroughput() { + + return this.throughput; + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + super.write(out); + + this.vertexID.write(out); + this.sourceChannelID.write(out); + out.writeDouble(this.throughput); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + super.read(in); + + this.vertexID.read(in); + this.sourceChannelID.read(in); + this.throughput = in.readDouble(); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java index b86adf7196464..15fac1b7f5e72 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java @@ -18,6 +18,7 @@ import java.io.IOException; import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.plugins.wrapper.AbstractOutputGateWrapper; import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.types.Record; @@ -59,10 +60,13 @@ public void writeRecord(final T record) throws IOException, InterruptedException } } else { for (int i = 0; i < numberOfOutputChannels; ++i) { - final long amountOfDataTransmitted = getOutputChannel(i).getAmountOfDataTransmitted(); + final AbstractOutputChannel outputChannel = getOutputChannel(i); + final long amountOfDataTransmitted = outputChannel.getAmountOfDataTransmitted(); final long dataDiff = amountOfDataTransmitted - this.lastSentBytes[i]; this.lastSentBytes[i] = amountOfDataTransmitted; - System.out.println("Data diff " + dataDiff); + final long timeDiff = timestamp - this.lastTimestamp; + final double throughput = (double) (1000 * 8 * dataDiff) / (double) (1024 * 1024 * timeDiff); + this.streamListener.reportChannelThroughput(outputChannel.getID(), throughput); } } From 1a6e5d77c5be16815767f571bf9d1c562ef03e51 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 22 Nov 2011 14:13:47 +0000 Subject: [PATCH 080/310] Fixed stream listener implementation --- .../streaming/listeners/StreamListener.java | 99 +++++++++---------- .../wrappers/StreamingInputGate.java | 15 +++ 2 files changed, 63 insertions(+), 51 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java index adf3d9ebd35db..acbfc183037cd 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java @@ -26,6 +26,7 @@ import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.streaming.StreamingTag; import eu.stratosphere.nephele.streaming.StreamingTaskManagerPlugin; +import eu.stratosphere.nephele.streaming.types.ChannelLatency; import eu.stratosphere.nephele.streaming.types.ChannelThroughput; import eu.stratosphere.nephele.streaming.types.TaskLatency; import eu.stratosphere.nephele.types.AbstractTaggableRecord; @@ -122,57 +123,53 @@ public long recordEmitted(final Record record) { */ public void recordReceived(final Record record) { - /* - * if (this.taskType == TaskType.INPUT) { - * throw new IllegalStateException("Input task received record"); - * } - * final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - * this.tag = (StreamingTag) taggableRecord.getTag(); - * if (this.tag != null) { - * final long timestamp = System.currentTimeMillis(); - * if (this.lastTimestamp > 0) { - * try { - * this.communicationThread.sendDataAsynchronously(new TaskLatency(jobID, vertexID, timestamp - * - this.lastTimestamp)); - * } catch (InterruptedException e) { - * LOG.error(StringUtils.stringifyException(e)); - * } - * if (this.taskType == TaskType.REGULAR) { - * this.lastTimestamp = -1L; - * } else { - * this.lastTimestamp = timestamp; - * } - * } - * final long pathLatency = timestamp - this.tag.getTimestamp(); - * final ExecutionVertexID sourceID = this.tag.getSourceID(); - * // Calculate moving average - * Double aggregatedLatency = this.aggregatedValue.get(sourceID); - * if (aggregatedLatency == null) { - * aggregatedLatency = Double.valueOf(pathLatency); - * } else { - * aggregatedLatency = Double.valueOf((ALPHA * pathLatency) - * + ((1 - ALPHA) * aggregatedLatency.doubleValue())); - * } - * this.aggregatedValue.put(sourceID, aggregatedLatency); - * // Check if we need to compute an event and send it to the job manager component - * Integer counter = this.aggregationCounter.get(sourceID); - * if (counter == null) { - * counter = Integer.valueOf(0); - * } - * counter = Integer.valueOf(counter.intValue() + 1); - * if (counter.intValue() == this.aggregationInterval) { - * final ChannelLatency pl = new ChannelLatency(this.jobID, sourceID, this.vertexID, - * aggregatedLatency.doubleValue()); - * try { - * this.communicationThread.sendDataAsynchronously(pl); - * } catch (InterruptedException e) { - * LOG.warn(StringUtils.stringifyException(e)); - * } - * counter = Integer.valueOf(0); - * } - * this.aggregationCounter.put(sourceID, counter); - * } - */ + final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; + this.tag = (StreamingTag) taggableRecord.getTag(); + if (this.tag != null) { + final long timestamp = System.currentTimeMillis(); + if (this.lastTimestamp > 0) { + final TaskLatency tl = new TaskLatency(this.listenerContext.getJobID(), + this.listenerContext.getVertexID(), timestamp - this.lastTimestamp); + try { + this.listenerContext.sendDataAsynchronously(tl); + } catch (InterruptedException e) { + LOG.error(StringUtils.stringifyException(e)); + } + if (this.listenerContext.isRegularVertex()) { + this.lastTimestamp = -1L; + } else { + this.lastTimestamp = timestamp; + } + } + final long pathLatency = timestamp - this.tag.getTimestamp(); + final ExecutionVertexID sourceID = this.tag.getSourceID(); + // Calculate moving average + Double aggregatedLatency = this.aggregatedValue.get(sourceID); + if (aggregatedLatency == null) { + aggregatedLatency = Double.valueOf(pathLatency); + } else { + aggregatedLatency = Double.valueOf((ALPHA * pathLatency) + + ((1 - ALPHA) * aggregatedLatency.doubleValue())); + } + this.aggregatedValue.put(sourceID, aggregatedLatency); + // Check if we need to compute an event and send it to the job manager component + Integer counter = this.aggregationCounter.get(sourceID); + if (counter == null) { + counter = Integer.valueOf(0); + } + counter = Integer.valueOf(counter.intValue() + 1); + if (counter.intValue() == this.listenerContext.getAggregationInterval()) { + final ChannelLatency pl = new ChannelLatency(this.listenerContext.getJobID(), sourceID, + this.listenerContext.getVertexID(), aggregatedLatency.doubleValue()); + try { + this.listenerContext.sendDataAsynchronously(pl); + } catch (InterruptedException e) { + LOG.warn(StringUtils.stringifyException(e)); + } + counter = Integer.valueOf(0); + } + this.aggregationCounter.put(sourceID, counter); + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java index 47fe7855bc4e5..c6aeb669b5c33 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java @@ -15,6 +15,8 @@ package eu.stratosphere.nephele.streaming.wrappers; +import java.io.IOException; + import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.plugins.wrapper.AbstractInputGateWrapper; import eu.stratosphere.nephele.streaming.listeners.StreamListener; @@ -33,4 +35,17 @@ public final class StreamingInputGate extends AbstractInputGat this.streamListener = streamListener; } + + /** + * {@inheritDoc} + */ + @Override + public T readRecord(final T target) throws IOException, InterruptedException { + + final T retVal = getWrappedInputGate().readRecord(target); + + this.streamListener.recordReceived(retVal); + + return retVal; + } } From e95dfcb5007fa3d0130182133424a44ec5cf0aab Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 22 Nov 2011 15:10:16 +0000 Subject: [PATCH 081/310] Simplified stream listener logic --- .../streaming/listeners/StreamListener.java | 114 ++++++++---------- 1 file changed, 49 insertions(+), 65 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java index acbfc183037cd..e77e7179cbf35 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java @@ -15,15 +15,13 @@ package eu.stratosphere.nephele.streaming.listeners; -import java.util.HashMap; -import java.util.Map; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.streaming.StreamingTag; import eu.stratosphere.nephele.streaming.StreamingTaskManagerPlugin; import eu.stratosphere.nephele.streaming.types.ChannelLatency; @@ -40,22 +38,17 @@ public final class StreamListener { */ private static final Log LOG = LogFactory.getLog(StreamListener.class); - private final static double ALPHA = 0.5; - private final Configuration taskConfiguration; private StreamListenerContext listenerContext = null; - private StreamingTag tag = null; - private int tagCounter = 0; + /** + * Indicates the time of the last received tagged incoming record + */ private long lastTimestamp = -1L; - private Map aggregationCounter = new HashMap(); - - private Map aggregatedValue = new HashMap(); - public StreamListener(final Configuration taskConfiguration) { if (taskConfiguration == null) { @@ -86,10 +79,13 @@ public long recordEmitted(final Record record) { // Input vertex if (this.listenerContext.isInputVertex()) { + final int taggingInterval = this.listenerContext.getTaggingInterval(); + final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; + + // Tag every record and calculate task latency if (this.tagCounter++ == taggingInterval) { timestamp = System.currentTimeMillis(); - final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; taggableRecord.setTag(createTag(timestamp)); if (this.lastTimestamp > 0L) { final long taskLatency = (timestamp - this.lastTimestamp) / taggingInterval; @@ -102,16 +98,32 @@ public long recordEmitted(final Record record) { } this.lastTimestamp = timestamp; this.tagCounter = 0; + } else { + taggableRecord.setTag(null); } + } else { + final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - if (this.tag == null) { - taggableRecord.setTag(null); - } else { + + if(this.lastTimestamp > 0L) { + timestamp = System.currentTimeMillis(); - this.tag = createTag(timestamp); - taggableRecord.setTag(this.tag); - this.lastTimestamp = timestamp; + taggableRecord.setTag(createTag(timestamp)); + final JobID jobID = this.listenerContext.getJobID(); + final ExecutionVertexID vertexID = this.listenerContext.getVertexID(); + + // Calculate task latency + final TaskLatency tl = new TaskLatency(jobID, vertexID, timestamp - this.lastTimestamp); + try { + this.listenerContext.sendDataAsynchronously(tl); + } catch (InterruptedException e) { + LOG.error(StringUtils.stringifyException(e)); + } + + this.lastTimestamp = -1L; + } else { + taggableRecord.setTag(null); } } @@ -124,53 +136,25 @@ public long recordEmitted(final Record record) { public void recordReceived(final Record record) { final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - this.tag = (StreamingTag) taggableRecord.getTag(); - if (this.tag != null) { + final StreamingTag tag = (StreamingTag) taggableRecord.getTag(); + if(tag != null) { + + final long timestamp = System.currentTimeMillis(); - if (this.lastTimestamp > 0) { - final TaskLatency tl = new TaskLatency(this.listenerContext.getJobID(), - this.listenerContext.getVertexID(), timestamp - this.lastTimestamp); - try { - this.listenerContext.sendDataAsynchronously(tl); - } catch (InterruptedException e) { - LOG.error(StringUtils.stringifyException(e)); - } - if (this.listenerContext.isRegularVertex()) { - this.lastTimestamp = -1L; - } else { - this.lastTimestamp = timestamp; - } - } - final long pathLatency = timestamp - this.tag.getTimestamp(); - final ExecutionVertexID sourceID = this.tag.getSourceID(); - // Calculate moving average - Double aggregatedLatency = this.aggregatedValue.get(sourceID); - if (aggregatedLatency == null) { - aggregatedLatency = Double.valueOf(pathLatency); - } else { - aggregatedLatency = Double.valueOf((ALPHA * pathLatency) - + ((1 - ALPHA) * aggregatedLatency.doubleValue())); + final JobID jobID = this.listenerContext.getJobID(); + final ExecutionVertexID vertexID = this.listenerContext.getVertexID(); + + // Calculate channel latency + final ChannelLatency cl = new ChannelLatency(jobID, tag.getSourceID(), vertexID, timestamp + - tag.getTimestamp()); + try { + this.listenerContext.sendDataAsynchronously(cl); + } catch (InterruptedException e) { + LOG.warn(StringUtils.stringifyException(e)); } - this.aggregatedValue.put(sourceID, aggregatedLatency); - // Check if we need to compute an event and send it to the job manager component - Integer counter = this.aggregationCounter.get(sourceID); - if (counter == null) { - counter = Integer.valueOf(0); - } - counter = Integer.valueOf(counter.intValue() + 1); - if (counter.intValue() == this.listenerContext.getAggregationInterval()) { - final ChannelLatency pl = new ChannelLatency(this.listenerContext.getJobID(), sourceID, - this.listenerContext.getVertexID(), aggregatedLatency.doubleValue()); - try { - this.listenerContext.sendDataAsynchronously(pl); - } catch (InterruptedException e) { - LOG.warn(StringUtils.stringifyException(e)); - } - counter = Integer.valueOf(0); - } - this.aggregationCounter.put(sourceID, counter); + + this.lastTimestamp = timestamp; } - } public void reportChannelThroughput(final ChannelID sourceChannelID, final double throughput) { @@ -184,8 +168,8 @@ public void reportChannelThroughput(final ChannelID sourceChannelID, final doubl } private StreamingTag createTag(final long timestamp) { - this.tag = new StreamingTag(this.listenerContext.getVertexID()); - this.tag.setTimestamp(timestamp); - return this.tag; + StreamingTag tag = new StreamingTag(this.listenerContext.getVertexID()); + tag.setTimestamp(timestamp); + return tag; } } From 9098c562c673c08c48a569a3acace44e7564547d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Tue, 22 Nov 2011 20:42:55 +0100 Subject: [PATCH 082/310] merge with warneke/streaming --- .../streaming/latency/LatencyModel.java | 58 ++++++++++++------- .../latency/LatencyOptimizerThread.java | 5 +- .../streaming/latency/LatencySubgraph.java | 5 +- 3 files changed, 40 insertions(+), 28 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java index 67ecfc77979cb..2d6f125239879 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java @@ -1,17 +1,16 @@ package eu.stratosphere.nephele.streaming.latency; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; +import eu.stratosphere.nephele.managementgraph.ManagementVertexID; import eu.stratosphere.nephele.streaming.types.ChannelLatency; +import eu.stratosphere.nephele.streaming.types.TaskLatency; + public class LatencyModel { - private static Log LOG = LogFactory.getLog(LatencyModel.class); + // private static Log LOG = LogFactory.getLog(LatencyModel.class); private ExecutionGraph executionGraph; @@ -28,21 +27,40 @@ public LatencyModel(ExecutionGraph executionGraph) { this.latencySubgraph = new LatencySubgraph(executionGraph, subgraphStart, subgraphEnd); } - public void refreshEdgeLatency(ChannelLatency latency) { - ExecutionVertexID startID = latency.getSourceVertexID(); - ExecutionVertexID endID = latency.getSinkVertexID(); - - if (startID.equals(endID)) { - System.out.println("new vertex latency"); - VertexLatency vertexLatency = latencySubgraph.getVertexLatency(startID.toManagementVertexID()); - vertexLatency.setLatencyInMillis(latency.getChannelLatency()); - } else { - System.out.println("new edge latency"); - ManagementEdgeID edgeID = new ManagementEdgeID(latency.getSourceVertexID().toManagementVertexID(), - latency.getSinkVertexID().toManagementVertexID()); - - EdgeLatency edgeLatency = latencySubgraph.getEdgeLatency(edgeID); - edgeLatency.setLatencyInMillis(latency.getChannelLatency()); + public int i = 0; + + public void refreshEdgeLatency(ChannelLatency channelLatency) { + + // FIXME: this is done to prevent an NPE caused by another bug that causes identical + // vertex ids inside the channel latency + ManagementVertexID startID = channelLatency.getSourceVertexID().toManagementVertexID(); + ManagementVertexID stopID = channelLatency.getSinkVertexID().toManagementVertexID(); + if(startID.equals(stopID)) { + return; + } + + ManagementEdgeID edgeID = new ManagementEdgeID(channelLatency.getSourceVertexID().toManagementVertexID(), + channelLatency.getSinkVertexID().toManagementVertexID()); + + EdgeLatency edgeLatency = latencySubgraph.getEdgeLatency(edgeID); + edgeLatency.setLatencyInMillis(channelLatency.getChannelLatency()); + + i++; + + if(i % 20 == 0) { + for (LatencyPath path : latencySubgraph.getLatencyPaths()) { + path.dumpLatencies(); + } + } + } + + public void refreshTaskLatency(TaskLatency taskLatency) { + VertexLatency vertexLatency = latencySubgraph + .getVertexLatency(taskLatency.getVertexID().toManagementVertexID()); + vertexLatency.setLatencyInMillis(taskLatency.getTaskLatency()); + i++; + + if(i % 20 == 0) { for (LatencyPath path : latencySubgraph.getLatencyPaths()) { path.dumpLatencies(); } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java index 7610fc13765a1..dc72eba7f823f 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java @@ -36,11 +36,8 @@ public void run() { if (streamingData instanceof ChannelLatency) { latencyModel.refreshEdgeLatency((ChannelLatency) streamingData); } else if(streamingData instanceof TaskLatency) { - TaskLatency tl = (TaskLatency) streamingData; - System.out.println("Task latency: " + tl.getTaskLatency()); + latencyModel.refreshTaskLatency((TaskLatency) streamingData); } - - } } catch (InterruptedException e) { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java index 9e06adca0e100..5bde4c90f89bb 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java @@ -4,9 +4,6 @@ import java.util.LinkedList; import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; @@ -28,7 +25,7 @@ */ public class LatencySubgraph { - private static Log LOG = LogFactory.getLog(LatencySubgraph.class); + //private static Log LOG = LogFactory.getLog(LatencySubgraph.class); private ManagementGroupVertex subgraphStart; From e790e48fb41eb47a18208214f6c330e7940c4fad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Tue, 22 Nov 2011 22:59:53 +0100 Subject: [PATCH 083/310] refactored ManagementEdge identification scheme --- .../stratosphere/nephele/io/AbstractID.java | 17 ------ .../managementgraph/ManagementEdge.java | 41 ++++++++++----- .../managementgraph/ManagementEdgeID.java | 20 ++++--- .../managementgraph/ManagementGraph.java | 16 ++++-- .../managementgraph/ManagementGraphTest.java | 25 ++++----- .../ManagementGraphFactory.java | 10 ++-- ...eLatency.java => EdgeCharacteristics.java} | 15 +++++- .../streaming/latency/LatencyModel.java | 52 ++++++++++--------- .../latency/LatencyOptimizerThread.java | 3 ++ .../streaming/latency/LatencyPath.java | 6 +-- .../streaming/latency/LatencySubgraph.java | 21 ++++---- .../streaming/listeners/StreamListener.java | 3 +- 12 files changed, 128 insertions(+), 101 deletions(-) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/{EdgeLatency.java => EdgeCharacteristics.java} (60%) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java index 3611ba19d3087..774d9881fe1fe 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java @@ -39,23 +39,6 @@ public abstract class AbstractID implements IOReadableWritable { */ private final byte[] bytes = new byte[SIZE]; - /** - * Creates a new id by merging the two given IDs. - * - * @param one - * The first id to use for merging - * @param two - * The second id to use for merging - */ - public AbstractID(AbstractID one, AbstractID two) { - byte[] firstBytes = one.getBytes(); - byte[] secondBytes = two.getBytes(); - - for (int i = 0; i < SIZE; i++) { - bytes[i] = (byte) (firstBytes[i] ^ secondBytes[i]); - } - } - /** * Constructs a new ID with a specific bytes value. */ diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java index 79da066c33c56..7dfd29073536e 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java @@ -59,15 +59,22 @@ public final class ManagementEdge extends ManagementAttachment { private final CompressionLevel compressionLevel; /** - * The ID of the management edge. + * The source ID of the management edge. */ - private ManagementEdgeID managementEdgeID; + private final ManagementEdgeID sourceEdgeID; + + /** + * The target ID of the management edge. + */ + private final ManagementEdgeID targetEdgeID; /** * Constructs a new edge object. * - * @param managementEdgeID - * the ID the new management edge + * @param sourceEdgeID + * source ID of the management edge + * @param targetEdgeID + * target ID of the management edge * @param source * the source of the edge referring to the output gate of an execution vertex * @param sourceIndex @@ -81,11 +88,12 @@ public final class ManagementEdge extends ManagementAttachment { * @param compressionLevel * the compression level of the channel this edge refers to */ - public ManagementEdge(final ManagementEdgeID managementEdgeID, - final ManagementGate source, final int sourceIndex, final ManagementGate target, - final int targetIndex, + public ManagementEdge(final ManagementEdgeID sourceEdgeID, final ManagementEdgeID targetEdgeID, + final ManagementGate source, final int sourceIndex, final ManagementGate target, final int targetIndex, final ChannelType channelType, final CompressionLevel compressionLevel) { - this.managementEdgeID = managementEdgeID; + + this.sourceEdgeID = sourceEdgeID; + this.targetEdgeID = targetEdgeID; this.source = source; this.target = target; this.sourceIndex = sourceIndex; @@ -152,11 +160,20 @@ public int getTargetIndex() { } /** - * Returns the management edge ID. + * Returns the source ID of the edge. + * + * @return The source ID of the edge + */ + public ManagementEdgeID getSourceEdgeID() { + return sourceEdgeID; + } + + /** + * Returns the target ID of the edge. * - * @return The management edge ID + * @return The target ID of the edge */ - public ManagementEdgeID getManagementEdgeID() { - return managementEdgeID; + public ManagementEdgeID getTargetEdgeID() { + return targetEdgeID; } } diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java index 7f130e93ab51a..ac9ad9add6b0d 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java @@ -1,6 +1,7 @@ package eu.stratosphere.nephele.managementgraph; import eu.stratosphere.nephele.io.AbstractID; +import eu.stratosphere.nephele.io.channels.ChannelID; /** * A management edge ID uniquely identifies a {@link ManagementEdge}. @@ -12,15 +13,20 @@ public class ManagementEdgeID extends AbstractID { /** - * A ManagementEdgeID is derived from a pair of #{@link ManagementVertexID}s. - * Note that this only works for simple DAGs that are not multi-graphs. - * FIXME: use ManagementGateID to make the management graph fully multi-graph capable. This - * means we have to construct management edge IDs from gate IDs + * Initializes ManagementEdgeID. + */ + ManagementEdgeID() { + } + + /** + * A ManagementEdgeID is derived from the #{@link ChannelID} of the corresponding + * output channel in the execution graph. * * @param source - * @param target + * ID of the corresponding output channel */ - public ManagementEdgeID(ManagementVertexID source, ManagementVertexID target) { - super(source, target); + public ManagementEdgeID(ChannelID source) { + super(); + this.setID(source); } } diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java index 8d8b1d86dd922..a49526a0408b7 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java @@ -468,6 +468,12 @@ public void read(final DataInput in) throws IOException { final ManagementGate sourceGate = sourceVertex.getOutputGate(j); int numberOfForwardEdges = in.readInt(); for (int k = 0; k < numberOfForwardEdges; k++) { + final ManagementEdgeID sourceEdgeID = new ManagementEdgeID(); + sourceEdgeID.read(in); + + final ManagementEdgeID targetEdgeID = new ManagementEdgeID(); + targetEdgeID.read(in); + final ManagementVertexID targetID = new ManagementVertexID(); targetID.read(in); final ManagementVertex targetVertex = getVertexByID(targetID); @@ -479,10 +485,8 @@ public void read(final DataInput in) throws IOException { final ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class); final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); - final ManagementEdgeID managementEdgeID = new ManagementEdgeID(sourceVertex.getID(), - targetVertex.getID()); - new ManagementEdge(managementEdgeID, sourceGate, sourceIndex, targetGate, targetIndex, channelType, - compressionLevel); + new ManagementEdge(sourceEdgeID, targetEdgeID, sourceGate, sourceIndex, targetGate, targetIndex, + channelType, compressionLevel); } } @@ -547,6 +551,10 @@ public void write(final DataOutput out) throws IOException { out.writeInt(outputGate.getNumberOfForwardEdges()); for (int j = 0; j < outputGate.getNumberOfForwardEdges(); j++) { final ManagementEdge edge = outputGate.getForwardEdge(j); + + edge.getSourceEdgeID().write(out); + edge.getTargetEdgeID().write(out); + // This identifies the target gate edge.getTarget().getVertex().getID().write(out); out.writeInt(edge.getTarget().getIndex()); diff --git a/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java b/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java index e60694bcc700b..4b594808c1f5f 100644 --- a/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java +++ b/nephele/nephele-management/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java @@ -295,21 +295,16 @@ private static ManagementGraph constructTestManagementGraph() { new ManagementGroupEdge(groupVertex3, 0, groupVertex4, 0, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); // Edges - new ManagementEdge(new ManagementEdgeID(vertex1_1.getID(), - vertex2_1.getID()), outputGate1_1, 0, inputGate2_1, 0, ChannelType.NETWORK, - CompressionLevel.NO_COMPRESSION); - new ManagementEdge(new ManagementEdgeID(vertex1_1.getID(), - vertex2_2.getID()), outputGate1_1, 1, inputGate2_2, 0, ChannelType.NETWORK, - CompressionLevel.NO_COMPRESSION); - new ManagementEdge(new ManagementEdgeID(vertex2_1.getID(), - vertex3_1.getID()), outputGate2_1, 0, inputGate3_1, 0, ChannelType.FILE, - CompressionLevel.NO_COMPRESSION); - new ManagementEdge(new ManagementEdgeID(vertex2_2.getID(), - vertex3_1.getID()), outputGate2_2, 0, inputGate3_1, 1, ChannelType.FILE, - CompressionLevel.NO_COMPRESSION); - new ManagementEdge(new ManagementEdgeID(vertex3_1.getID(), - vertex4_1.getID()), outputGate3_1, 0, inputGate4_1, 0, ChannelType.INMEMORY, - CompressionLevel.NO_COMPRESSION); + new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate1_1, 0, inputGate2_1, 0, + ChannelType.NETWORK, CompressionLevel.NO_COMPRESSION); + new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate1_1, 1, inputGate2_2, 0, + ChannelType.NETWORK, CompressionLevel.NO_COMPRESSION); + new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate2_1, 0, inputGate3_1, 0, + ChannelType.FILE, CompressionLevel.NO_COMPRESSION); + new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate2_2, 0, inputGate3_1, 1, + ChannelType.FILE, CompressionLevel.NO_COMPRESSION); + new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate3_1, 0, inputGate4_1, 0, + ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); return graph; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java index ede8cd1ebfe36..b58eadd49d932 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java @@ -172,12 +172,10 @@ private static void addExecutionVertices(Map vertexLatencies = new HashMap(); - private HashMap edgeLatencies = new HashMap(); + private HashMap edgeCharacteristics = new HashMap(); public LatencySubgraph(ExecutionGraph executionGraph, ExecutionGroupVertex subgraphStart, ExecutionGroupVertex subgraphEnd) { @@ -63,9 +63,10 @@ private void initLatenciesOnPath(LatencyPath path) { ManagementEdge ingoingEdge = path.getIngoingEdge(vertex); if (ingoingEdge != null && ingoingEdge.getAttachment() == null) { - EdgeLatency edgeLatency = new EdgeLatency(ingoingEdge); - ingoingEdge.setAttachment(edgeLatency); - edgeLatencies.put(ingoingEdge.getManagementEdgeID(), edgeLatency); + EdgeCharacteristics characteristics = new EdgeCharacteristics(ingoingEdge); + ingoingEdge.setAttachment(characteristics); + edgeCharacteristics.put(ingoingEdge.getSourceEdgeID(), characteristics); + edgeCharacteristics.put(ingoingEdge.getTargetEdgeID(), characteristics); } } } @@ -125,14 +126,14 @@ private void determineAnchoringManagementGroupVertices(ManagementGraph managemen this.subgraphEnd = managementGraph.getVertexByID(vertexInPathEndGroup).getGroupVertex(); } - public EdgeLatency getEdgeLatency(ManagementEdgeID edgeID) { - return edgeLatencies.get(edgeID); + public EdgeCharacteristics getEdgeCharacteristicsBySourceEdgeID(ManagementEdgeID sourceEdgeID) { + return edgeCharacteristics.get(sourceEdgeID); } - + public VertexLatency getVertexLatency(ManagementVertexID managementVertexID) { return vertexLatencies.get(managementVertexID); } - + public List getLatencyPaths() { return latencyPaths; } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java index e77e7179cbf35..a7eef4a1f9129 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java @@ -106,7 +106,7 @@ public long recordEmitted(final Record record) { final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - if(this.lastTimestamp > 0L) { + if(this.lastTimestamp >= 0L) { timestamp = System.currentTimeMillis(); taggableRecord.setTag(createTag(timestamp)); @@ -142,6 +142,7 @@ public void recordReceived(final Record record) { final long timestamp = System.currentTimeMillis(); final JobID jobID = this.listenerContext.getJobID(); + final ExecutionVertexID vertexID = this.listenerContext.getVertexID(); // Calculate channel latency From 2693043dca539ac369159f8ea761ef87693973ac Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Tue, 22 Nov 2011 23:00:45 +0100 Subject: [PATCH 084/310] addendum to last commit --- .../nephele/streaming/latency/LatencyOptimizerThread.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java index 973cc89e16b3a..0eef740403d55 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java @@ -39,7 +39,7 @@ public void run() { } else if(streamingData instanceof TaskLatency) { latencyModel.refreshTaskLatency((TaskLatency) streamingData); } else if (streamingData instanceof ChannelThroughput) { - + latencyModel.refreshChannelThroughput((ChannelThroughput) streamingData); } } From 0ab7e9b754c8ee16f3ba78372c7bcaba84c937a8 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 23 Nov 2011 10:19:15 +0000 Subject: [PATCH 085/310] Implemented mapping from receiver vertex to source channel IDs --- .../streaming/latency/LatencySubgraph.java | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java index be3ba00c774d8..2f3363e8cea7b 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java @@ -1,6 +1,7 @@ package eu.stratosphere.nephele.streaming.latency; import java.util.HashMap; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; @@ -12,6 +13,7 @@ import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; import eu.stratosphere.nephele.managementgraph.ManagementGate; import eu.stratosphere.nephele.managementgraph.ManagementGraph; +import eu.stratosphere.nephele.managementgraph.ManagementGraphIterator; import eu.stratosphere.nephele.managementgraph.ManagementGroupVertex; import eu.stratosphere.nephele.managementgraph.ManagementVertex; import eu.stratosphere.nephele.managementgraph.ManagementVertexID; @@ -37,6 +39,8 @@ public class LatencySubgraph { private HashMap edgeCharacteristics = new HashMap(); + private HashMap receiverVertexToSourceEdgeIDMap = new HashMap(); + public LatencySubgraph(ExecutionGraph executionGraph, ExecutionGroupVertex subgraphStart, ExecutionGroupVertex subgraphEnd) { @@ -44,6 +48,7 @@ public LatencySubgraph(ExecutionGraph executionGraph, ExecutionGroupVertex subgr determineAnchoringManagementGroupVertices(managementGraph, subgraphStart, subgraphEnd); buildLatencyPaths(); initLatenciesOnPaths(); + initReceiverVertexToSourceEdgeIDMap(managementGraph); } private void initLatenciesOnPaths() { @@ -71,6 +76,25 @@ private void initLatenciesOnPath(LatencyPath path) { } } + private void initReceiverVertexToSourceEdgeIDMap(final ManagementGraph managementGraph) { + + final Iterator it = new ManagementGraphIterator(managementGraph, true); + while (it.hasNext()) { + + final ManagementVertex source = it.next(); + final int numberOfOutputGates = source.getNumberOfOutputGates(); + for (int i = 0; i < numberOfOutputGates; ++i) { + final ManagementGate outputGate = source.getOutputGate(i); + final int numberOfOutgoingEdges = outputGate.getNumberOfForwardEdges(); + for (int j = 0; j < numberOfOutgoingEdges; ++j) { + final ManagementEdge edge = outputGate.getForwardEdge(j); + final ManagementVertex receiver = edge.getTarget().getVertex(); + this.receiverVertexToSourceEdgeIDMap.put(receiver.getID(), edge.getSourceEdgeID()); + } + } + } + } + private void buildLatencyPaths() { this.latencyPaths = new LinkedList(); From 67b3c6ab1ea90ef7dda04518a57008522dcb0f14 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Wed, 23 Nov 2011 15:52:15 +0100 Subject: [PATCH 086/310] added value statistics (median, min, max, avg) to stream profiling data model --- .../latency/EdgeCharacteristics.java | 30 ++-- .../streaming/latency/LatencyModel.java | 6 +- .../streaming/latency/ProfilingValue.java | 49 ++++++ .../latency/ProfilingValueStatistic.java | 106 +++++++++++++ .../streaming/latency/VertexLatency.java | 19 ++- .../latency/ProfilingValueStatisticTest.java | 139 ++++++++++++++++++ 6 files changed, 329 insertions(+), 20 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValue.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatistic.java create mode 100644 nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeCharacteristics.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeCharacteristics.java index 9bab014fd4eee..9f1481e6a9e0b 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeCharacteristics.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeCharacteristics.java @@ -6,13 +6,14 @@ public class EdgeCharacteristics { private ManagementEdge edge; - private double latencyInMillis; + private ProfilingValueStatistic latencyInMillisStatistic; - private double throughputInMbit; + private ProfilingValueStatistic throughputInMbitStatistic; public EdgeCharacteristics(ManagementEdge edge) { this.edge = edge; - this.latencyInMillis = -1; + this.latencyInMillisStatistic = new ProfilingValueStatistic(20); + this.throughputInMbitStatistic = new ProfilingValueStatistic(20); } public ManagementEdge getEdge() { @@ -20,19 +21,28 @@ public ManagementEdge getEdge() { } public double getLatencyInMillis() { - return latencyInMillis; + if (latencyInMillisStatistic.hasValues()) { + return latencyInMillisStatistic.getMedianValue(); + } else { + return -1; + } } - public void setLatencyInMillis(double latencyInMillis) { - this.latencyInMillis = latencyInMillis; + public void addLatencyMeasurement(long timestamp, double latencyInMillis) { + ProfilingValue value = new ProfilingValue(latencyInMillis, timestamp); + this.latencyInMillisStatistic.addValue(value); } public double getThroughputInMbit() { - return throughputInMbit; + if (throughputInMbitStatistic.hasValues()) { + return throughputInMbitStatistic.getMedianValue(); + } else { + return -1; + } } - public void setThroughputInMbit(double throughputInMbit) { - this.throughputInMbit = throughputInMbit; + public void addThroughputMeasurement(long timestamp, double throughputInMbit) { + ProfilingValue value = new ProfilingValue(throughputInMbit, timestamp); + this.throughputInMbitStatistic.addValue(value); } - } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java index d7f21e9971380..867e12513db16 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java @@ -47,7 +47,7 @@ public void refreshEdgeLatency(ChannelLatency channelLatency) { public void refreshTaskLatency(TaskLatency taskLatency) { VertexLatency vertexLatency = latencySubgraph .getVertexLatency(taskLatency.getVertexID().toManagementVertexID()); - vertexLatency.setLatencyInMillis(taskLatency.getTaskLatency()); + vertexLatency.addLatencyMeasurement(System.currentTimeMillis(), taskLatency.getTaskLatency()); i++; if (i % 20 == 0) { @@ -60,8 +60,8 @@ public void refreshTaskLatency(TaskLatency taskLatency) { public void refreshChannelThroughput(ChannelThroughput channelThroughput) { ManagementEdgeID edgeID = new ManagementEdgeID(channelThroughput.getSourceChannelID()); EdgeCharacteristics edgeCharaceristics = latencySubgraph.getEdgeCharacteristicsBySourceEdgeID(edgeID); - edgeCharaceristics.setThroughputInMbit(channelThroughput.getThroughput()); - + edgeCharaceristics.addThroughputMeasurement(System.currentTimeMillis(), channelThroughput.getThroughput()); + i++; if (i % 20 == 0) { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValue.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValue.java new file mode 100644 index 0000000000000..70ea67affe267 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValue.java @@ -0,0 +1,49 @@ +package eu.stratosphere.nephele.streaming.latency; + +public class ProfilingValue implements Comparable { + + private double value; + + private long timestamp; + + public ProfilingValue(double value, long timestamp) { + this.value = value; + this.timestamp = timestamp; + } + + public double getValue() { + return value; + } + + public void setValue(double value) { + this.value = value; + } + + public long getTimestamp() { + return timestamp; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + /** + * Sorts first by value and then by timestamp. + */ + @Override + public int compareTo(ProfilingValue other) { + if (this.value > other.value) { + return 1; + } else if (this.value < other.value) { + return -1; + } else { + if (this.timestamp > other.timestamp) { + return 1; + } else if (this.timestamp < other.timestamp) { + return -1; + } else { + return 0; + } + } + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatistic.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatistic.java new file mode 100644 index 0000000000000..b1c2d9f8537d5 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatistic.java @@ -0,0 +1,106 @@ +package eu.stratosphere.nephele.streaming.latency; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedList; + +public class ProfilingValueStatistic { + + private ArrayList sortedByValue; + + private LinkedList sortedByTimestamp; + + private int valueArraySize; + + private int noOfStoredValues; + + private double sumOfValues; + + public ProfilingValueStatistic(int valueSetSize) { + this.sortedByTimestamp = new LinkedList(); + this.sortedByValue = new ArrayList(); + this.valueArraySize = valueSetSize; + this.noOfStoredValues = 0; + this.sumOfValues = 0; + } + + public void addValue(ProfilingValue value) { + ProfilingValue droppedValue = insertIntoSortedByTimestamp(value); + + if (droppedValue != null) { + removeFromSortedByValue(droppedValue); + noOfStoredValues--; + sumOfValues -= droppedValue.getValue(); + } + + insertIntoSortedByValue(value); + noOfStoredValues++; + sumOfValues += value.getValue(); + } + + private ProfilingValue insertIntoSortedByTimestamp(ProfilingValue value) { + if (!sortedByTimestamp.isEmpty() && sortedByTimestamp.getLast().getTimestamp() > value.getTimestamp()) { + throw new IllegalArgumentException("Trying to add stale profiling values. This should not happen."); + } + sortedByTimestamp.add(value); + + if (noOfStoredValues >= valueArraySize) { + return sortedByTimestamp.removeFirst(); + } else { + return null; + } + } + + protected void insertIntoSortedByValue(ProfilingValue value) { + int insertionIndex = Collections.binarySearch(sortedByValue, value); + if (insertionIndex > 0) { + throw new IllegalArgumentException("Profiling already in list. This should not happen."); + } else { + insertionIndex = -(insertionIndex + 1); + sortedByValue.add(insertionIndex, value); + } + } + + protected void removeFromSortedByValue(ProfilingValue toRemove) { + int removeIndex = Collections.binarySearch(sortedByValue, toRemove); + if (removeIndex < 0) { + throw new IllegalArgumentException("Trying to drop inexistant profiling value. This should not happen."); + } + sortedByValue.remove(removeIndex); + } + + public double getMedianValue() { + if (noOfStoredValues == 0) { + throw new RuntimeException("Cannot calculate median of empty value set"); + } + + int medianIndex = noOfStoredValues / 2; + return sortedByValue.get(medianIndex).getValue(); + } + + public double getMaxValue() { + if (noOfStoredValues == 0) { + throw new RuntimeException("Cannot calculate the max value of empty value set"); + } + return sortedByValue.get(noOfStoredValues - 1).getValue(); + } + + public double getMinValue() { + if (noOfStoredValues == 0) { + throw new RuntimeException("Cannot calculate the min value of empty value set"); + } + return sortedByValue.get(0).getValue(); + } + + public double getArithmeticMean() { + if (noOfStoredValues == 0) { + throw new RuntimeException("Cannot calculate the arithmetic mean of empty value set"); + } + + return sumOfValues / noOfStoredValues; + } + + public boolean hasValues() { + return noOfStoredValues > 0; + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java index dfe7b639f6694..7967432171221 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java @@ -6,11 +6,11 @@ public class VertexLatency { private ManagementVertex vertex; - private double latencyInMillis; + private ProfilingValueStatistic latencyStatistics; public VertexLatency(ManagementVertex vertex) { this.vertex = vertex; - this.latencyInMillis = -1; + this.latencyStatistics = new ProfilingValueStatistic(20); } public ManagementVertex getVertex() { @@ -18,15 +18,20 @@ public ManagementVertex getVertex() { } public double getLatencyInMillis() { - return latencyInMillis; + if (latencyStatistics.hasValues()) { + return latencyStatistics.getMedianValue(); + } else { + return -1; + } } - public void setLatencyInMillis(double latencyInMillis) { - this.latencyInMillis = latencyInMillis; + public void addLatencyMeasurement(long timestamp, double latencyInMillis) { + ProfilingValue value = new ProfilingValue(latencyInMillis, timestamp); + latencyStatistics.addValue(value); } - + @Override public String toString() { - return String.format("VertexLatency[%s|%.03f]", vertex.toString(), latencyInMillis); + return String.format("VertexLatency[%s|%.03f]", vertex.toString(), getLatencyInMillis()); } } diff --git a/nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java b/nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java new file mode 100644 index 0000000000000..b245e97cc3d27 --- /dev/null +++ b/nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java @@ -0,0 +1,139 @@ +package eu.stratosphere.nephele.streaming.latency; + +import static org.junit.Assert.assertTrue; + +import org.junit.Before; +import org.junit.Test; + +public class ProfilingValueStatisticTest { + + private ProfilingValueStatistic valueStatistic; + + @Before + public void setup() { + valueStatistic = new ProfilingValueStatistic(7); + } + + @Test + public void testValueSorted() { + valueStatistic.addValue(createProfilingValue(1, 1)); + valueStatistic.addValue(createProfilingValue(2, 3)); + valueStatistic.addValue(createProfilingValue(3, 7)); + valueStatistic.addValue(createProfilingValue(4, 8)); + valueStatistic.addValue(createProfilingValue(5, 21)); + valueStatistic.addValue(createProfilingValue(6, 35)); + valueStatistic.addValue(createProfilingValue(7, 41)); + + assertTrue(valueStatistic.getMedianValue() == 8); + assertTrue(valueStatistic.getMinValue() == 1); + assertTrue(valueStatistic.getMaxValue() == 41); + assertTrue((valueStatistic.getArithmeticMean() - 16.5714) < 0.0001); + } + + private ProfilingValue createProfilingValue(long timestamp, double value) { + return new ProfilingValue(value, timestamp); + } + + @Test + public void testAddValueUnsorted() { + valueStatistic.addValue(createProfilingValue(1, 7)); + valueStatistic.addValue(createProfilingValue(2, 15)); + valueStatistic.addValue(createProfilingValue(3, 13)); + valueStatistic.addValue(createProfilingValue(4, 1)); + valueStatistic.addValue(createProfilingValue(5, 5)); + valueStatistic.addValue(createProfilingValue(6, 7.5)); + valueStatistic.addValue(createProfilingValue(7, 8)); + + assertTrue(valueStatistic.getMedianValue() == 7.5); + assertTrue(valueStatistic.getMinValue() == 1); + assertTrue(valueStatistic.getMaxValue() == 15); + assertTrue((valueStatistic.getArithmeticMean() - 8.0714) < 0.0001); + } + + @Test + public void testAddValueReverseSorted() { + valueStatistic.addValue(createProfilingValue(1, 18)); + valueStatistic.addValue(createProfilingValue(2, 15)); + valueStatistic.addValue(createProfilingValue(3, 13)); + valueStatistic.addValue(createProfilingValue(4, 10)); + valueStatistic.addValue(createProfilingValue(5, 9)); + valueStatistic.addValue(createProfilingValue(6, 8)); + valueStatistic.addValue(createProfilingValue(7, 7)); + + assertTrue(valueStatistic.getMedianValue() == 10); + assertTrue(valueStatistic.getMinValue() == 7); + assertTrue(valueStatistic.getMaxValue() == 18); + assertTrue((valueStatistic.getArithmeticMean() - 11.4285) < 0.0001); + } + + @Test + public void testAddValueOverfullUnsorted() { + valueStatistic.addValue(createProfilingValue(1, 7)); + valueStatistic.addValue(createProfilingValue(2, 15)); + valueStatistic.addValue(createProfilingValue(3, 13)); + valueStatistic.addValue(createProfilingValue(4, 1)); + valueStatistic.addValue(createProfilingValue(5, 7.5)); + valueStatistic.addValue(createProfilingValue(6, 5)); + valueStatistic.addValue(createProfilingValue(7, 18)); + valueStatistic.addValue(createProfilingValue(8, 13)); + valueStatistic.addValue(createProfilingValue(9, 10)); + valueStatistic.addValue(createProfilingValue(10, 8)); + + assertTrue(valueStatistic.getMedianValue() == 8); + assertTrue(valueStatistic.getMinValue() == 1); + assertTrue(valueStatistic.getMaxValue() == 18); + assertTrue((valueStatistic.getArithmeticMean() - 8.9285) < 0.0001); + } + + @Test + public void testAddValueOverfullSorted() { + valueStatistic.addValue(createProfilingValue(1, 1)); + valueStatistic.addValue(createProfilingValue(2, 2)); + valueStatistic.addValue(createProfilingValue(3, 3)); + valueStatistic.addValue(createProfilingValue(4, 4)); + valueStatistic.addValue(createProfilingValue(5, 5)); + valueStatistic.addValue(createProfilingValue(6, 6)); + valueStatistic.addValue(createProfilingValue(7, 7)); + valueStatistic.addValue(createProfilingValue(8, 8)); + valueStatistic.addValue(createProfilingValue(9, 9)); + valueStatistic.addValue(createProfilingValue(10, 10)); + + assertTrue(valueStatistic.getMedianValue() == 7); + assertTrue(valueStatistic.getMinValue() == 4); + assertTrue(valueStatistic.getMaxValue() == 10); + assertTrue(valueStatistic.getArithmeticMean() == 7); + } + + @Test + public void testGetMedianUnderfull() { + valueStatistic.addValue(createProfilingValue(1, 18)); + valueStatistic.addValue(createProfilingValue(2, 15)); + assertTrue(valueStatistic.getMedianValue() == 18); + + valueStatistic.addValue(createProfilingValue(3, 17)); + assertTrue(valueStatistic.getMedianValue() == 17); + } + + @Test + public void testGetMinMaxUnderfull() { + valueStatistic.addValue(createProfilingValue(1, 18)); + valueStatistic.addValue(createProfilingValue(2, 15)); + assertTrue(valueStatistic.getMinValue() == 15); + assertTrue(valueStatistic.getMaxValue() == 18); + + valueStatistic.addValue(createProfilingValue(3, 17)); + assertTrue(valueStatistic.getMinValue() == 15); + assertTrue(valueStatistic.getMaxValue() == 18); + + } + + @Test + public void testGetArithmeticMeanUnderfull() { + valueStatistic.addValue(createProfilingValue(1, 18)); + valueStatistic.addValue(createProfilingValue(2, 15)); + assertTrue(valueStatistic.getArithmeticMean() == 16.5); + + valueStatistic.addValue(createProfilingValue(3, 17)); + assertTrue((valueStatistic.getArithmeticMean() - 16.6666) < 0.0001); + } +} From a99608388d1c3a758b96df9dd672c5e69c9b06d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Wed, 23 Nov 2011 16:03:08 +0100 Subject: [PATCH 087/310] added processing of task latency back in --- .../streaming/latency/LatencyModel.java | 26 +++++++------------ .../streaming/latency/LatencySubgraph.java | 4 +++ 2 files changed, 14 insertions(+), 16 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java index 867e12513db16..0284f26e4d874 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java @@ -26,22 +26,13 @@ public LatencyModel(ExecutionGraph executionGraph) { this.latencySubgraph = new LatencySubgraph(executionGraph, subgraphStart, subgraphEnd); } - public int i = 0; - public void refreshEdgeLatency(ChannelLatency channelLatency) { -// ManagementEdgeID edgeID = new ManagementEdgeID(channelLatency.getSourceVertexID().toManagementVertexID(), -// channelLatency.getSinkVertexID().toManagementVertexID()); -// -// EdgeCharacteristics edgeLatency = latencySubgraph.getEdgeLatency(edgeID); -// edgeLatency.setLatencyInMillis(channelLatency.getChannelLatency()); -// -// i++; -// -// if (i % 20 == 0) { -// for (LatencyPath path : latencySubgraph.getLatencyPaths()) { -// path.dumpLatencies(); -// } -// } + + ManagementEdgeID sourceEdgeID = latencySubgraph.getEdgeByReceiverVertexID(channelLatency.getSinkVertexID() + .toManagementVertexID()); + + EdgeCharacteristics edgeLatency = latencySubgraph.getEdgeCharacteristicsBySourceEdgeID(sourceEdgeID); + edgeLatency.addLatencyMeasurement(System.currentTimeMillis(), channelLatency.getChannelLatency()); } public void refreshTaskLatency(TaskLatency taskLatency) { @@ -57,13 +48,16 @@ public void refreshTaskLatency(TaskLatency taskLatency) { } } + // FIXME this should be removed later on + public int i = 0; + public void refreshChannelThroughput(ChannelThroughput channelThroughput) { ManagementEdgeID edgeID = new ManagementEdgeID(channelThroughput.getSourceChannelID()); EdgeCharacteristics edgeCharaceristics = latencySubgraph.getEdgeCharacteristicsBySourceEdgeID(edgeID); edgeCharaceristics.addThroughputMeasurement(System.currentTimeMillis(), channelThroughput.getThroughput()); + // FIXME this should be removed later on i++; - if (i % 20 == 0) { for (LatencyPath path : latencySubgraph.getLatencyPaths()) { path.dumpLatencies(); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java index 2f3363e8cea7b..adab0de0bb68a 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java @@ -149,6 +149,10 @@ private void determineAnchoringManagementGroupVertices(ManagementGraph managemen ManagementVertexID vertexInPathEndGroup = pathEndExecVertex.getGroupMember(0).getID().toManagementVertexID(); this.subgraphEnd = managementGraph.getVertexByID(vertexInPathEndGroup).getGroupVertex(); } + + public ManagementEdgeID getEdgeByReceiverVertexID(ManagementVertexID receiverVertexID) { + return receiverVertexToSourceEdgeIDMap.get(receiverVertexID); + } public EdgeCharacteristics getEdgeCharacteristicsBySourceEdgeID(ManagementEdgeID sourceEdgeID) { return edgeCharacteristics.get(sourceEdgeID); From 13173414dd3d91177f4fac176517a8c164596785 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 23 Nov 2011 19:06:39 +0000 Subject: [PATCH 088/310] Implemented method to adjust channel buffer size limits from job manager --- .../AbstractByteBufferedOutputChannel.java | 12 +++ .../ByteBufferedOutputChannelBroker.java | 8 ++ .../managementgraph/ManagementEdgeID.java | 28 ++++++ .../nephele/taskmanager/Task.java | 1 - .../bytebuffered/OutputChannelContext.java | 25 ++++- .../streaming/StreamingJobManagerPlugin.java | 34 ++++++- .../streaming/StreamingPluginLoader.java | 2 +- .../actions/BufferSizeLimitAction.java | 95 +++++++++++++++++++ .../latency/LatencyOptimizerThread.java | 31 ++++-- .../latency/ProfilingValueStatistic.java | 9 +- 10 files changed, 228 insertions(+), 17 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java index 93263e050b152..1e22afb222cea 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java @@ -427,4 +427,16 @@ public long getAmountOfDataTransmitted() { return this.amountOfDataTransmitted; } + + /** + * Limits the size of the buffer this channel will write its records to before passing them on to the framework. + * + * @param bufferSize + * the new limit for the by + */ + public void limitBufferSize(final int bufferSize) { + + // Delegate call to the assigned output channel broker + this.outputChannelBroker.limitBufferSize(bufferSize); + } } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java index a565dc58f9ee0..a4d7dead0fd4e 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java @@ -67,4 +67,12 @@ public interface ByteBufferedOutputChannelBroker { * thrown if an I/O error occurs while transfering the event */ void transferEventToInputChannel(AbstractEvent event) throws IOException, InterruptedException; + + /** + * Limits the size of the buffer the output channel is allowed to write into. + * + * @param bufferSize + * the new buffer size in bytes + */ + void limitBufferSize(int bufferSize); } diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java index ac9ad9add6b0d..d0b2a62fe90c2 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.managementgraph; import eu.stratosphere.nephele.io.AbstractID; @@ -29,4 +44,17 @@ public ManagementEdgeID(ChannelID source) { super(); this.setID(source); } + + /** + * Converts the management edge ID into a {@link ChannelID}. + * + * @return the corresponding channelID. + */ + public ChannelID toChannelID() { + + final ChannelID channelID = new ChannelID(); + channelID.setID(this); + + return channelID; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index a17432ac36362..596b6ab7a8598 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -34,7 +34,6 @@ import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.jobgraph.JobID; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java index c1774287dcfc0..e32c6482fb2f3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java @@ -76,6 +76,11 @@ final class OutputChannelContext implements ByteBufferedOutputChannelBroker, Cha */ private boolean spillingQueueAlreadyFlushed = false; + /** + * The size of the buffer that is offered to the assigned output channel in bytes. + */ + private int bufferSize; + OutputChannelContext(final OutputGateContext outputGateContext, final AbstractByteBufferedOutputChannel byteBufferedOutputChannel, final boolean isReceiverRunning, final boolean mergeSpilledBuffers) { @@ -93,6 +98,9 @@ final class OutputChannelContext implements ByteBufferedOutputChannelBroker, Cha if (!isReceiverRunning) { this.outputGateContext.registerInactiveOutputChannel(this); } + + // Set the buffer size to the largest possible value by default + this.bufferSize = this.outputGateContext.getMaximumBufferSize(); } /** @@ -214,8 +222,7 @@ public void transferEventToInputChannel(final AbstractEvent event) throws IOExce */ private int calculateBufferSize() { - // TODO: Include latency considerations - return this.outputGateContext.getMaximumBufferSize(); + return this.bufferSize; } /** @@ -374,4 +381,18 @@ long spillQueueWithOutgoingEnvelopes() throws IOException { return this.queuedOutgoingEnvelopes.spillSynchronouslyIncludingHead(); } + + /** + * {@inheritDoc} + */ + @Override + public void limitBufferSize(final int bufferSize) { + + if (bufferSize > this.outputGateContext.getMaximumBufferSize()) { + throw new IllegalArgumentException("Buffer size limit must not be larger than " + + this.outputGateContext.getMaximumBufferSize() + " bytes"); + } + + this.bufferSize = bufferSize; + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 744a1a4acea2f..b12fa18fc47f2 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -25,9 +25,13 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.executiongraph.ExecutionVertex; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.executiongraph.InternalJobStatus; import eu.stratosphere.nephele.executiongraph.JobStatusListener; +import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.jobgraph.AbstractJobInputVertex; import eu.stratosphere.nephele.jobgraph.AbstractJobOutputVertex; import eu.stratosphere.nephele.jobgraph.JobFileInputVertex; @@ -37,6 +41,8 @@ import eu.stratosphere.nephele.jobgraph.JobInputVertex; import eu.stratosphere.nephele.jobgraph.JobOutputVertex; import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.plugins.PluginID; +import eu.stratosphere.nephele.streaming.actions.BufferSizeLimitAction; import eu.stratosphere.nephele.streaming.latency.LatencyOptimizerThread; import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; import eu.stratosphere.nephele.streaming.wrappers.StreamingFileInputWrapper; @@ -47,6 +53,7 @@ import eu.stratosphere.nephele.streaming.wrappers.WrapperUtils; import eu.stratosphere.nephele.jobgraph.JobTaskVertex; import eu.stratosphere.nephele.template.AbstractInvokable; +import eu.stratosphere.nephele.util.StringUtils; public class StreamingJobManagerPlugin implements JobManagerPlugin, JobStatusListener { @@ -55,9 +62,12 @@ public class StreamingJobManagerPlugin implements JobManagerPlugin, JobStatusLis */ private static final Log LOG = LogFactory.getLog(StreamingJobManagerPlugin.class); + private final PluginID pluginID; + private ConcurrentHashMap latencyOptimizerThreads = new ConcurrentHashMap(); - StreamingJobManagerPlugin(final Configuration pluginConfiguration) { + StreamingJobManagerPlugin(final PluginID pluginID, final Configuration pluginConfiguration) { + this.pluginID = pluginID; } /** @@ -126,8 +136,9 @@ public JobGraph rewriteJobGraph(final JobGraph jobGraph) { */ @Override public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) { + JobID jobId = executionGraph.getJobID(); - LatencyOptimizerThread optimizerThread = new LatencyOptimizerThread(executionGraph); + LatencyOptimizerThread optimizerThread = new LatencyOptimizerThread(this, executionGraph); latencyOptimizerThreads.put(jobId, optimizerThread); optimizerThread.start(); return executionGraph; @@ -197,4 +208,23 @@ public void jobStatusHasChanged(ExecutionGraph executionGraph, } } } + + public void limitBufferSize(final ExecutionVertex vertex, final ChannelID sourceChannelID, final int bufferSize) { + + final JobID jobID = vertex.getExecutionGraph().getJobID(); + final ExecutionVertexID vertexID = vertex.getID(); + + final AbstractInstance instance = vertex.getAllocatedResource().getInstance(); + if (instance == null) { + LOG.error(vertex + " has no instance assigned"); + return; + } + + final BufferSizeLimitAction bsla = new BufferSizeLimitAction(jobID, vertexID, sourceChannelID, bufferSize); + try { + instance.sendData(this.pluginID, bsla); + } catch (IOException e) { + LOG.error(StringUtils.stringifyException(e)); + } + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java index 198f3660e0c84..77a3e1ef140e1 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java @@ -71,7 +71,7 @@ public StreamingPluginLoader(final String pluginName, final Configuration plugin public synchronized JobManagerPlugin getJobManagerPlugin() { if (this.jobManagerPlugin == null) { - this.jobManagerPlugin = new StreamingJobManagerPlugin(getPluginConfiguration()); + this.jobManagerPlugin = new StreamingJobManagerPlugin(this.pluginID, getPluginConfiguration()); } return this.jobManagerPlugin; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java new file mode 100644 index 0000000000000..af7c20cf6913c --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java @@ -0,0 +1,95 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.actions; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.jobgraph.JobID; + +public final class BufferSizeLimitAction implements IOReadableWritable { + + private final JobID jobID; + + private final ExecutionVertexID vertexID; + + private final ChannelID sourceChannelID; + + private int bufferSize; + + public BufferSizeLimitAction(final JobID jobID, final ExecutionVertexID vertexID, final ChannelID sourceChannelID, + final int bufferSize) { + + if (jobID == null) { + throw new IllegalArgumentException("Argument jobID must not be null"); + } + + if (vertexID == null) { + throw new IllegalArgumentException("Argument vertexID must not be null"); + } + + if (sourceChannelID == null) { + throw new IllegalArgumentException("Argument sourceChannelID must not be null"); + } + + if (bufferSize <= 0) { + throw new IllegalArgumentException("Argument bufferSize must be greather than zero"); + } + + this.jobID = jobID; + this.vertexID = vertexID; + this.sourceChannelID = sourceChannelID; + this.bufferSize = bufferSize; + } + + /** + * Default constructor for deserialization. + */ + public BufferSizeLimitAction() { + this.jobID = new JobID(); + this.vertexID = new ExecutionVertexID(); + this.sourceChannelID = new ChannelID(); + this.bufferSize = 0; + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + + this.jobID.write(out); + this.vertexID.write(out); + this.sourceChannelID.write(out); + out.writeInt(this.bufferSize); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + + this.jobID.read(in); + this.vertexID.read(in); + this.sourceChannelID.read(in); + this.bufferSize = in.readInt(); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java index 0eef740403d55..9ff70e67d8c38 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java @@ -6,6 +6,10 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.executiongraph.ExecutionVertex; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; +import eu.stratosphere.nephele.streaming.StreamingJobManagerPlugin; import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; import eu.stratosphere.nephele.streaming.types.ChannelLatency; import eu.stratosphere.nephele.streaming.types.ChannelThroughput; @@ -15,13 +19,16 @@ public class LatencyOptimizerThread extends Thread { private Log LOG = LogFactory.getLog(LatencyOptimizerThread.class); - private LinkedBlockingQueue streamingDataQueue; + private final LinkedBlockingQueue streamingDataQueue; - private ExecutionGraph executionGraph; + private final StreamingJobManagerPlugin jobManagerPlugin; - private LatencyModel latencyModel; + private final ExecutionGraph executionGraph; - public LatencyOptimizerThread(ExecutionGraph executionGraph) { + private final LatencyModel latencyModel; + + public LatencyOptimizerThread(StreamingJobManagerPlugin jobManagerPlugin, ExecutionGraph executionGraph) { + this.jobManagerPlugin = jobManagerPlugin; this.executionGraph = executionGraph; this.latencyModel = new LatencyModel(executionGraph); this.streamingDataQueue = new LinkedBlockingQueue(); @@ -36,8 +43,8 @@ public void run() { if (streamingData instanceof ChannelLatency) { latencyModel.refreshEdgeLatency((ChannelLatency) streamingData); - } else if(streamingData instanceof TaskLatency) { - latencyModel.refreshTaskLatency((TaskLatency) streamingData); + } else if (streamingData instanceof TaskLatency) { + latencyModel.refreshTaskLatency((TaskLatency) streamingData); } else if (streamingData instanceof ChannelThroughput) { latencyModel.refreshChannelThroughput((ChannelThroughput) streamingData); } @@ -52,4 +59,16 @@ public void run() { public void handOffStreamingData(AbstractStreamingData data) { streamingDataQueue.add(data); } + + public void limitBufferSize(ManagementEdgeID sourceEdgeID, int bufferSize) { + + final ChannelID sourceChannelID = sourceEdgeID.toChannelID(); + final ExecutionVertex vertex = this.executionGraph.getVertexByChannelID(sourceChannelID); + if (vertex == null) { + LOG.error("Cannot find vertex to channel ID " + vertex); + return; + } + + this.jobManagerPlugin.limitBufferSize(vertex, sourceChannelID, bufferSize); + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatistic.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatistic.java index b1c2d9f8537d5..fbee6d0163ff7 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatistic.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatistic.java @@ -23,7 +23,7 @@ public ProfilingValueStatistic(int valueSetSize) { this.noOfStoredValues = 0; this.sumOfValues = 0; } - + public void addValue(ProfilingValue value) { ProfilingValue droppedValue = insertIntoSortedByTimestamp(value); @@ -53,12 +53,11 @@ private ProfilingValue insertIntoSortedByTimestamp(ProfilingValue value) { protected void insertIntoSortedByValue(ProfilingValue value) { int insertionIndex = Collections.binarySearch(sortedByValue, value); - if (insertionIndex > 0) { - throw new IllegalArgumentException("Profiling already in list. This should not happen."); - } else { + if (insertionIndex < 0) { insertionIndex = -(insertionIndex + 1); - sortedByValue.add(insertionIndex, value); } + + sortedByValue.add(insertionIndex, value); } protected void removeFromSortedByValue(ProfilingValue toRemove) { From 0a7a2aa5e21354d45386e160209dee3c72baf230 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 23 Nov 2011 20:48:25 +0000 Subject: [PATCH 089/310] Refactored actions the Nephele streaming plugin can take to achieve latency and throughput goals --- .../streaming/actions/AbstractAction.java | 113 ++++++++++++++++++ .../actions/BufferSizeLimitAction.java | 68 +++++++---- 2 files changed, 159 insertions(+), 22 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java new file mode 100644 index 0000000000000..0b98841946afc --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java @@ -0,0 +1,113 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.actions; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.jobgraph.JobID; + +/** + * This class implements an abstract base class for actions the job manager component of the Nephele streaming plugin + * can initiate to achieve particular latency or throughput goals. + * + * @author warneke + */ +public abstract class AbstractAction implements IOReadableWritable { + + /** + * The ID of the job the initiated action applies to. + */ + private final JobID jobID; + + /** + * The ID of the vertex the initiated action applies to. + */ + private final ExecutionVertexID vertexID; + + /** + * Constructs a new abstract action object. + * + * @param jobID + * the ID of the job the initiated action applies to + * @param vertexID + * the ID of the vertex the initiated action applies to + */ + AbstractAction(final JobID jobID, final ExecutionVertexID vertexID) { + + if (jobID == null) { + throw new IllegalArgumentException("Argument jobID must not be null"); + } + + if (vertexID == null) { + throw new IllegalArgumentException("Argument vertexID must not be null"); + } + + this.jobID = jobID; + this.vertexID = vertexID; + } + + /** + * Default constructor required for deserialization. + */ + AbstractAction() { + this.jobID = new JobID(); + this.vertexID = new ExecutionVertexID(); + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + + this.jobID.write(out); + this.vertexID.write(out); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + + this.jobID.read(in); + this.vertexID.read(in); + } + + /** + * Returns the ID of the job the initiated action applies to. + * + * @return the ID of the job the initiated action applies to + */ + public JobID getJobID() { + + return this.jobID; + } + + /** + * Returns the ID of the vertex the initiated action applies to. + * + * @return the ID of the vertex the initiated action applies to + */ + public ExecutionVertexID getVertexID() { + + return this.vertexID; + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java index af7c20cf6913c..6a21e0e7ccefa 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java @@ -20,30 +20,41 @@ import java.io.IOException; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.jobgraph.JobID; -public final class BufferSizeLimitAction implements IOReadableWritable { - - private final JobID jobID; - - private final ExecutionVertexID vertexID; +/** + * This class implements an action to limit the buffer size of a particular output channel. + * + * @author warneke + */ +public final class BufferSizeLimitAction extends AbstractAction { + /** + * The ID of the output channel whose buffer size shall be limited. + */ private final ChannelID sourceChannelID; + /** + * The new buffer size in bytes. + */ private int bufferSize; + /** + * Constructs a new buffer size limit action object. + * + * @param jobID + * the ID of the job the action applies to + * @param vertexID + * the ID of the vertex the action applies to + * @param sourceChannelID + * the ID of the output channel whose buffer size shall be limited + * @param bufferSize + * the new buffer size in bytes + */ public BufferSizeLimitAction(final JobID jobID, final ExecutionVertexID vertexID, final ChannelID sourceChannelID, final int bufferSize) { - - if (jobID == null) { - throw new IllegalArgumentException("Argument jobID must not be null"); - } - - if (vertexID == null) { - throw new IllegalArgumentException("Argument vertexID must not be null"); - } + super(jobID, vertexID); if (sourceChannelID == null) { throw new IllegalArgumentException("Argument sourceChannelID must not be null"); @@ -53,8 +64,6 @@ public BufferSizeLimitAction(final JobID jobID, final ExecutionVertexID vertexID throw new IllegalArgumentException("Argument bufferSize must be greather than zero"); } - this.jobID = jobID; - this.vertexID = vertexID; this.sourceChannelID = sourceChannelID; this.bufferSize = bufferSize; } @@ -63,8 +72,7 @@ public BufferSizeLimitAction(final JobID jobID, final ExecutionVertexID vertexID * Default constructor for deserialization. */ public BufferSizeLimitAction() { - this.jobID = new JobID(); - this.vertexID = new ExecutionVertexID(); + super(); this.sourceChannelID = new ChannelID(); this.bufferSize = 0; } @@ -75,8 +83,6 @@ public BufferSizeLimitAction() { @Override public void write(final DataOutput out) throws IOException { - this.jobID.write(out); - this.vertexID.write(out); this.sourceChannelID.write(out); out.writeInt(this.bufferSize); } @@ -87,9 +93,27 @@ public void write(final DataOutput out) throws IOException { @Override public void read(final DataInput in) throws IOException { - this.jobID.read(in); - this.vertexID.read(in); this.sourceChannelID.read(in); this.bufferSize = in.readInt(); } + + /** + * Returns the ID of the output channel whose buffer size shall be limited. + * + * @return the ID of the output channel whose buffer size shall be limited + */ + public ChannelID getSourceChannelID() { + + return this.sourceChannelID; + } + + /** + * Returns the new buffer size in bytes. + * + * @return the new buffer size in bytes + */ + public int getBufferSize() { + + return this.bufferSize; + } } From bbf35d6a380ce67ae6311f675050cf5a7fe330ae Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 23 Nov 2011 20:51:14 +0000 Subject: [PATCH 090/310] Fixed deserialization problems with the plugin ID --- .../nephele/plugins/PluginID.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginID.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginID.java index 3dd105e033fad..322dced0461c6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginID.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/PluginID.java @@ -24,10 +24,29 @@ */ public final class PluginID extends AbstractID { + /** + * Constructs a new plugin ID from the given byte array. + * + * @param byteArray + * the byte array to construct the plugin ID from + */ private PluginID(final byte[] byteArray) { super(byteArray); } + /** + * Default constructor required for the deserialization. + */ + public PluginID() { + super(); + } + + /** + * Constructs a new plugin ID from the given byte array. + * + * @param byteArray + * the byte array to construct the plugin ID from + */ public static PluginID fromByteArray(final byte[] byteArray) { if (byteArray == null) { From 8b2accdaffcb16b3ba27ff8d3405e71ae9a3fa8d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 23 Nov 2011 21:37:41 +0000 Subject: [PATCH 091/310] Implemented queue for pending actions inside the task manager component of the Nephele streaming plugin --- .../streaming/StreamingTaskManagerPlugin.java | 25 ++++++++++++++++++- .../listeners/StreamListenerContext.java | 18 +++++++++++++ 2 files changed, 42 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index 0141d47942368..dc8f4a0e54a9e 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -19,6 +19,9 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; @@ -26,10 +29,16 @@ import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.plugins.PluginCommunication; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; +import eu.stratosphere.nephele.streaming.actions.AbstractAction; import eu.stratosphere.nephele.streaming.listeners.StreamListenerContext; public class StreamingTaskManagerPlugin implements TaskManagerPlugin { + /** + * The log object. + */ + private static final Log LOG = LogFactory.getLog(StreamingTaskManagerPlugin.class); + /** * Provides access to the configuration entry which defines the interval in which records shall be tagged. */ @@ -154,7 +163,21 @@ public void unregisterTask(final ExecutionVertexID id, final RuntimeEnvironment @Override public void sendData(final IOReadableWritable data) throws IOException { - // TODO Implement me + if (!(data instanceof AbstractAction)) { + LOG.error("Received data is of unknown type " + data.getClass()); + return; + } + + final AbstractAction action = (AbstractAction) data; + final StreamListenerContext listenerContext = this.listenerContexts.get(action.getVertexID().toString()); + + if (listenerContext == null) { + LOG.error("Cannot find listener context for vertex with ID " + action.getVertexID()); + return; + } + + // Queue the action and return + listenerContext.queuePendingAction(action); } /** diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java index c860836c71fd3..911c614b0475b 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java @@ -15,9 +15,13 @@ package eu.stratosphere.nephele.streaming.listeners; +import java.util.ArrayDeque; +import java.util.Queue; + import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.streaming.StreamingCommunicationThread; +import eu.stratosphere.nephele.streaming.actions.AbstractAction; import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; public final class StreamListenerContext { @@ -28,6 +32,8 @@ private static enum TaskType { INPUT, REGULAR, OUTPUT }; + private final Queue pendingActions = new ArrayDeque(); + private final JobID jobID; private final ExecutionVertexID vertexID; @@ -137,4 +143,16 @@ void sendDataAsynchronously(final AbstractStreamingData data) throws Interrupted this.communicationThread.sendDataAsynchronously(data); } + + public void queuePendingAction(final AbstractAction action) { + + synchronized (this.pendingActions) { + this.pendingActions.add(action); + } + } + + Queue getPendingActionsQueue() { + + return this.pendingActions; + } } From 50892e3f40bc0e1d943b1c5789f8985981a64cf1 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 24 Nov 2011 12:12:45 +0000 Subject: [PATCH 092/310] Fixed serialization problem with the buffer size limit action --- .../nephele/streaming/actions/BufferSizeLimitAction.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java index 6a21e0e7ccefa..81ab86c3702c3 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java @@ -83,6 +83,8 @@ public BufferSizeLimitAction() { @Override public void write(final DataOutput out) throws IOException { + super.write(out); + this.sourceChannelID.write(out); out.writeInt(this.bufferSize); } @@ -93,6 +95,8 @@ public void write(final DataOutput out) throws IOException { @Override public void read(final DataInput in) throws IOException { + super.read(in); + this.sourceChannelID.read(in); this.bufferSize = in.readInt(); } From d5c490c5c89e3999fe5a3b6d343b194479764b0f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 24 Nov 2011 13:00:26 +0000 Subject: [PATCH 093/310] Finished implementation to adapt buffer limits during runtime --- .../streaming/listeners/StreamListener.java | 104 ++++++++++++++++-- .../wrappers/StreamingOutputGate.java | 2 + 2 files changed, 95 insertions(+), 11 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java index a7eef4a1f9129..9b290aaf61bac 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java @@ -15,18 +15,30 @@ package eu.stratosphere.nephele.streaming.listeners; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Queue; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.streaming.StreamingTag; import eu.stratosphere.nephele.streaming.StreamingTaskManagerPlugin; +import eu.stratosphere.nephele.streaming.actions.AbstractAction; +import eu.stratosphere.nephele.streaming.actions.BufferSizeLimitAction; import eu.stratosphere.nephele.streaming.types.ChannelLatency; import eu.stratosphere.nephele.streaming.types.ChannelThroughput; import eu.stratosphere.nephele.streaming.types.TaskLatency; +import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; import eu.stratosphere.nephele.types.AbstractTaggableRecord; import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.util.StringUtils; @@ -44,6 +56,10 @@ public final class StreamListener { private int tagCounter = 0; + private Map> outputGateMap = new HashMap>(); + + private Map> outputChannelMap; + /** * Indicates the time of the last received tagged incoming record */ @@ -71,6 +87,20 @@ public void init() { } this.listenerContext = StreamingTaskManagerPlugin.getStreamingListenerContext(listenerKey); + + final Map> tmpMap = new HashMap>(); + + final Iterator> it = this.outputGateMap.values().iterator(); + while (it.hasNext()) { + final StreamingOutputGate outputGate = it.next(); + final int numberOfOutputChannels = outputGate.getNumberOfOutputChannels(); + for (int i = 0; i < numberOfOutputChannels; ++i) { + final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(i); + tmpMap.put(outputChannel.getID(), outputChannel); + } + } + + this.outputChannelMap = Collections.unmodifiableMap(tmpMap); } public long recordEmitted(final Record record) { @@ -98,6 +128,9 @@ public long recordEmitted(final Record record) { } this.lastTimestamp = timestamp; this.tagCounter = 0; + + // Finally, check for pending actions + checkForPendingActions(); } else { taggableRecord.setTag(null); } @@ -105,14 +138,14 @@ public long recordEmitted(final Record record) { } else { final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - - if(this.lastTimestamp >= 0L) { - + + if (this.lastTimestamp >= 0L) { + timestamp = System.currentTimeMillis(); taggableRecord.setTag(createTag(timestamp)); final JobID jobID = this.listenerContext.getJobID(); final ExecutionVertexID vertexID = this.listenerContext.getVertexID(); - + // Calculate task latency final TaskLatency tl = new TaskLatency(jobID, vertexID, timestamp - this.lastTimestamp); try { @@ -120,8 +153,11 @@ public long recordEmitted(final Record record) { } catch (InterruptedException e) { LOG.error(StringUtils.stringifyException(e)); } - + this.lastTimestamp = -1L; + + // Finally, check for pending actions + checkForPendingActions(); } else { taggableRecord.setTag(null); } @@ -137,14 +173,13 @@ public void recordReceived(final Record record) { final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; final StreamingTag tag = (StreamingTag) taggableRecord.getTag(); - if(tag != null) { - - + if (tag != null) { + final long timestamp = System.currentTimeMillis(); final JobID jobID = this.listenerContext.getJobID(); - + final ExecutionVertexID vertexID = this.listenerContext.getVertexID(); - + // Calculate channel latency final ChannelLatency cl = new ChannelLatency(jobID, tag.getSourceID(), vertexID, timestamp - tag.getTimestamp()); @@ -153,7 +188,7 @@ public void recordReceived(final Record record) { } catch (InterruptedException e) { LOG.warn(StringUtils.stringifyException(e)); } - + this.lastTimestamp = timestamp; } } @@ -173,4 +208,51 @@ private StreamingTag createTag(final long timestamp) { tag.setTimestamp(timestamp); return tag; } + + private void checkForPendingActions() { + + final Queue pendingActions = this.listenerContext.getPendingActionsQueue(); + + synchronized (pendingActions) { + + while (!pendingActions.isEmpty()) { + + final AbstractAction action = pendingActions.poll(); + + if (action instanceof BufferSizeLimitAction) { + limitBufferSize((BufferSizeLimitAction) action); + } else { + LOG.error("Ignoring unknown action of type " + action.getClass()); + } + } + } + } + + private void limitBufferSize(final BufferSizeLimitAction bsla) { + + final ChannelID sourceChannelID = bsla.getSourceChannelID(); + final int bufferSize = bsla.getBufferSize(); + + final AbstractOutputChannel outputChannel = this.outputChannelMap.get(sourceChannelID); + if (outputChannel == null) { + LOG.error("Cannot find output channel with ID " + sourceChannelID); + return; + } + + if (!(outputChannel instanceof AbstractByteBufferedOutputChannel)) { + LOG.error("Output channel with ID " + sourceChannelID + " is not a byte-buffered channel"); + return; + } + + final AbstractByteBufferedOutputChannel byteBufferedOutputChannel = + (AbstractByteBufferedOutputChannel) outputChannel; + + LOG.info("Setting buffer size limit of output channel " + sourceChannelID + " to " + bufferSize + " bytes"); + byteBufferedOutputChannel.limitBufferSize(bufferSize); + } + + public void registerOutputGate(final StreamingOutputGate outputGate) { + + this.outputGateMap.put(outputGate.getGateID(), outputGate); + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java index 15fac1b7f5e72..587f965467209 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java @@ -38,6 +38,8 @@ public final class StreamingOutputGate extends AbstractOutputG throw new IllegalArgumentException("Argument streamListener must not be null"); } + streamListener.registerOutputGate(this); + this.streamListener = streamListener; } From cce6834859e70d5eaa964916c9bbe9294a2c01cc Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 24 Nov 2011 18:20:09 +0000 Subject: [PATCH 094/310] Implemented callback to calculate the latency introduced by the output buffers --- .../java/eu/stratosphere/nephele/io/OutputGate.java | 9 +++++++++ .../AbstractByteBufferedOutputChannel.java | 3 +++ .../eu/stratosphere/nephele/io/RuntimeOutputGate.java | 9 +++++++++ .../plugins/wrapper/AbstractOutputGateWrapper.java | 9 +++++++++ .../streaming/wrappers/StreamingOutputGate.java | 10 ++++++++++ 5 files changed, 40 insertions(+) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java index 26068246a779e..ca1c1ba49a27d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java @@ -178,4 +178,13 @@ InMemoryOutputChannel createInMemoryOutputChannel(OutputGate outputGate, C * the listener object to register */ void registerOutputGateListener(OutputGateListener outputGateListener); + + /** + * This method is called by one of the attached output channels to indicate that the buffer which is internally used + * to store written records has been forwarded to the next step in the processing pipeline. + * + * @param channelID + * the ID of the output channel which has forwarded its buffer + */ + void outputBufferSent(ChannelID channelID); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java index 1e22afb222cea..e7e3bd917dd8d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java @@ -200,6 +200,9 @@ private void releaseWriteBuffers() throws IOException, InterruptedException { this.outputChannelBroker.releaseWriteBuffers(); this.compressedDataBuffer = null; this.uncompressedDataBuffer = null; + + // Notify the output gate to enable statistics collection by plugins + getOutputGate().outputBufferSent(getID()); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java index 0dd9866473e6a..c843cca1955ba 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java @@ -471,4 +471,13 @@ public void releaseAllChannelResources() { it.next().releaseResources(); } } + + /** + * {@inheritDoc} + */ + @Override + public void outputBufferSent(final ChannelID channelID) { + + // Nothing to do here + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java index 1a79bcc9aac09..3e77ad26a2652 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/wrapper/AbstractOutputGateWrapper.java @@ -324,4 +324,13 @@ public void registerOutputGateListener(final OutputGateListener outputGateListen this.wrappedOutputGate.registerOutputGateListener(outputGateListener); } + + /** + * {@inheritDoc} + */ + @Override + public void outputBufferSent(final ChannelID channelID) { + + this.wrappedOutputGate.outputBufferSent(channelID); + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java index 587f965467209..225330b621e3a 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java @@ -19,6 +19,7 @@ import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.plugins.wrapper.AbstractOutputGateWrapper; import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.types.Record; @@ -77,4 +78,13 @@ public void writeRecord(final T record) throws IOException, InterruptedException getWrappedOutputGate().writeRecord(record); } + + /** + * {@inheritDoc} + */ + @Override + public void outputBufferSent(final ChannelID channelID) { + + getWrappedOutputGate().outputBufferSent(channelID); + } } From fcad7e0f62183601752b3283672f3be86fa7c5b0 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 24 Nov 2011 18:46:04 +0000 Subject: [PATCH 095/310] Introduced new type for output buffer latency --- .../types/AbstractStreamingData.java | 8 ++ .../streaming/types/OutputBufferLatency.java | 101 ++++++++++++++++++ 2 files changed, 109 insertions(+) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/AbstractStreamingData.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/AbstractStreamingData.java index 032c6df1ed30c..6919ffa4abe3d 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/AbstractStreamingData.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/AbstractStreamingData.java @@ -43,6 +43,14 @@ public AbstractStreamingData(JobID jobID) { this.jobID = jobID; } + /** + * Default constructor required for deserilization. + */ + public AbstractStreamingData() { + + this.jobID = new JobID(); + } + /** * Returns the ID of the job this path latency information refers to. * diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java new file mode 100644 index 0000000000000..cc0363c33347f --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java @@ -0,0 +1,101 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.types; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.jobgraph.JobID; + +public final class OutputBufferLatency extends AbstractStreamingData { + + private final ExecutionVertexID vertexID; + + private final ChannelID sourceChannelID; + + private int bufferLatency; + + public OutputBufferLatency(final JobID jobID, final ExecutionVertexID vertexID, final ChannelID sourceChannelID, + final int bufferLatency) { + super(jobID); + + if (vertexID == null) { + throw new IllegalArgumentException("Argument vertexID must not be null"); + } + + if (sourceChannelID == null) { + throw new IllegalArgumentException("Argument sourceChannelID must not be null"); + } + + if (bufferLatency <= 0) { + throw new IllegalArgumentException("Argument bufferLatency must be greater than zero"); + } + + this.vertexID = vertexID; + this.sourceChannelID = sourceChannelID; + this.bufferLatency = bufferLatency; + } + + public OutputBufferLatency() { + super(); + + this.vertexID = new ExecutionVertexID(); + this.sourceChannelID = new ChannelID(); + this.bufferLatency = 0; + } + + public ExecutionVertexID getVertexID() { + + return this.vertexID; + } + + public ChannelID getSourceChannelID() { + + return this.sourceChannelID; + } + + public int getBufferLatency() { + + return this.bufferLatency; + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + super.write(out); + + this.vertexID.write(out); + this.sourceChannelID.write(out); + out.writeInt(this.bufferLatency); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + super.read(in); + + this.vertexID.read(in); + this.sourceChannelID.read(in); + this.bufferLatency = in.readInt(); + } +} From affd2e1cf4868b42b4741ccd393099727a4ea671 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 24 Nov 2011 19:02:52 +0000 Subject: [PATCH 096/310] Implemented propagation of buffer latencies to job manager component of the Nephele streaming plugin --- .../streaming/listeners/StreamListener.java | 11 ++++++++ .../streaming/types/OutputBufferLatency.java | 5 ++-- .../wrappers/StreamingOutputGate.java | 28 ++++++++++++++++--- 3 files changed, 38 insertions(+), 6 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java index 9b290aaf61bac..211b84f55942a 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java @@ -37,6 +37,7 @@ import eu.stratosphere.nephele.streaming.actions.BufferSizeLimitAction; import eu.stratosphere.nephele.streaming.types.ChannelLatency; import eu.stratosphere.nephele.streaming.types.ChannelThroughput; +import eu.stratosphere.nephele.streaming.types.OutputBufferLatency; import eu.stratosphere.nephele.streaming.types.TaskLatency; import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; import eu.stratosphere.nephele.types.AbstractTaggableRecord; @@ -203,6 +204,16 @@ public void reportChannelThroughput(final ChannelID sourceChannelID, final doubl } } + public void reportBufferLatency(final ChannelID sourceChannelID, final int bufferLatency) { + + try { + this.listenerContext.sendDataAsynchronously(new OutputBufferLatency(this.listenerContext.getJobID(), + this.listenerContext.getVertexID(), sourceChannelID, bufferLatency)); + } catch (InterruptedException e) { + LOG.error(StringUtils.stringifyException(e)); + } + } + private StreamingTag createTag(final long timestamp) { StreamingTag tag = new StreamingTag(this.listenerContext.getVertexID()); tag.setTimestamp(timestamp); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java index cc0363c33347f..4ff9e9ac2bfd2 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java @@ -43,8 +43,9 @@ public OutputBufferLatency(final JobID jobID, final ExecutionVertexID vertexID, throw new IllegalArgumentException("Argument sourceChannelID must not be null"); } - if (bufferLatency <= 0) { - throw new IllegalArgumentException("Argument bufferLatency must be greater than zero"); + if (bufferLatency < 0) { + throw new IllegalArgumentException("Argument bufferLatency must be greater than or equal to zero but is " + + bufferLatency); } this.vertexID = vertexID; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java index 225330b621e3a..183f117bf9851 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java @@ -26,12 +26,18 @@ public final class StreamingOutputGate extends AbstractOutputGateWrapper { + private static final int BUFFER_LATENCY_REPORT_INTERVAL = 10; + private final StreamListener streamListener; - private long lastTimestamp = -1L; + private long lastThroughputTimestamp = -1L; + + private long lastBufferLatencyTimestamp = -1L; private long[] lastSentBytes = null; + private int bufferLatencyReportCounter = 0; + StreamingOutputGate(final OutputGate wrappedOutputGate, final StreamListener streamListener) { super(wrappedOutputGate); @@ -55,7 +61,7 @@ public void writeRecord(final T record) throws IOException, InterruptedException final int numberOfOutputChannels = getNumberOfOutputChannels(); - if (this.lastTimestamp < 0) { + if (this.lastThroughputTimestamp < 0) { // Initialize array and fill it this.lastSentBytes = new long[numberOfOutputChannels]; for (int i = 0; i < numberOfOutputChannels; ++i) { @@ -67,13 +73,13 @@ public void writeRecord(final T record) throws IOException, InterruptedException final long amountOfDataTransmitted = outputChannel.getAmountOfDataTransmitted(); final long dataDiff = amountOfDataTransmitted - this.lastSentBytes[i]; this.lastSentBytes[i] = amountOfDataTransmitted; - final long timeDiff = timestamp - this.lastTimestamp; + final long timeDiff = timestamp - this.lastThroughputTimestamp; final double throughput = (double) (1000 * 8 * dataDiff) / (double) (1024 * 1024 * timeDiff); this.streamListener.reportChannelThroughput(outputChannel.getID(), throughput); } } - this.lastTimestamp = timestamp; + this.lastThroughputTimestamp = timestamp; } getWrappedOutputGate().writeRecord(record); @@ -85,6 +91,20 @@ public void writeRecord(final T record) throws IOException, InterruptedException @Override public void outputBufferSent(final ChannelID channelID) { + if (++this.bufferLatencyReportCounter == BUFFER_LATENCY_REPORT_INTERVAL) { + + final long timestamp = System.currentTimeMillis(); + + if (this.lastBufferLatencyTimestamp >= 0) { + + final int duration = (int) (timestamp - this.lastBufferLatencyTimestamp); + this.streamListener.reportBufferLatency(channelID, duration / BUFFER_LATENCY_REPORT_INTERVAL); + } + + this.lastBufferLatencyTimestamp = timestamp; + this.bufferLatencyReportCounter = 0; + } + getWrappedOutputGate().outputBufferSent(channelID); } } From 7b08f7f645b5546704b90184969be637fcf1f7b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Thu, 24 Nov 2011 23:17:19 +0100 Subject: [PATCH 097/310] * refactored most of the LatencyXYZ to ProfilingXYZ classes * implemented logging of profiling values * profiling paths (formerly known as LatencyPaths) can now start and end at edges --- .../streaming/latency/LatencyModel.java | 67 -------- .../latency/LatencyOptimizerThread.java | 14 +- .../streaming/latency/LatencyPath.java | 153 ----------------- .../streaming/latency/ProfilingLogger.java | 76 +++++++++ .../streaming/latency/ProfilingModel.java | 80 +++++++++ .../streaming/latency/ProfilingPath.java | 160 ++++++++++++++++++ .../latency/ProfilingPathSummary.java | 145 ++++++++++++++++ ...cySubgraph.java => ProfilingSubgraph.java} | 54 +++--- .../streaming/latency/ProfilingSummary.java | 87 ++++++++++ 9 files changed, 587 insertions(+), 249 deletions(-) delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingLogger.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingModel.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPath.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPathSummary.java rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/{LatencySubgraph.java => ProfilingSubgraph.java} (77%) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSummary.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java deleted file mode 100644 index 0284f26e4d874..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyModel.java +++ /dev/null @@ -1,67 +0,0 @@ -package eu.stratosphere.nephele.streaming.latency; - -import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; -import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; -import eu.stratosphere.nephele.streaming.types.ChannelLatency; -import eu.stratosphere.nephele.streaming.types.ChannelThroughput; -import eu.stratosphere.nephele.streaming.types.TaskLatency; - -public class LatencyModel { - - // private static Log LOG = LogFactory.getLog(LatencyModel.class); - - private ExecutionGraph executionGraph; - - private LatencySubgraph latencySubgraph; - - public LatencyModel(ExecutionGraph executionGraph) { - this.executionGraph = executionGraph; - - // FIXME naive implementation until we can annotate the job - // subgraphStart and subgraphEnd should be derived from the annotations - ExecutionGroupVertex subgraphStart = this.executionGraph.getInputVertex(0).getGroupVertex(); - ExecutionGroupVertex subgraphEnd = this.executionGraph.getOutputVertex(0).getGroupVertex(); - - this.latencySubgraph = new LatencySubgraph(executionGraph, subgraphStart, subgraphEnd); - } - - public void refreshEdgeLatency(ChannelLatency channelLatency) { - - ManagementEdgeID sourceEdgeID = latencySubgraph.getEdgeByReceiverVertexID(channelLatency.getSinkVertexID() - .toManagementVertexID()); - - EdgeCharacteristics edgeLatency = latencySubgraph.getEdgeCharacteristicsBySourceEdgeID(sourceEdgeID); - edgeLatency.addLatencyMeasurement(System.currentTimeMillis(), channelLatency.getChannelLatency()); - } - - public void refreshTaskLatency(TaskLatency taskLatency) { - VertexLatency vertexLatency = latencySubgraph - .getVertexLatency(taskLatency.getVertexID().toManagementVertexID()); - vertexLatency.addLatencyMeasurement(System.currentTimeMillis(), taskLatency.getTaskLatency()); - i++; - - if (i % 20 == 0) { - for (LatencyPath path : latencySubgraph.getLatencyPaths()) { - path.dumpLatencies(); - } - } - } - - // FIXME this should be removed later on - public int i = 0; - - public void refreshChannelThroughput(ChannelThroughput channelThroughput) { - ManagementEdgeID edgeID = new ManagementEdgeID(channelThroughput.getSourceChannelID()); - EdgeCharacteristics edgeCharaceristics = latencySubgraph.getEdgeCharacteristicsBySourceEdgeID(edgeID); - edgeCharaceristics.addThroughputMeasurement(System.currentTimeMillis(), channelThroughput.getThroughput()); - - // FIXME this should be removed later on - i++; - if (i % 20 == 0) { - for (LatencyPath path : latencySubgraph.getLatencyPaths()) { - path.dumpLatencies(); - } - } - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java index 9ff70e67d8c38..7f96931fb53f2 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java @@ -25,12 +25,12 @@ public class LatencyOptimizerThread extends Thread { private final ExecutionGraph executionGraph; - private final LatencyModel latencyModel; + private final ProfilingModel profilingModel; public LatencyOptimizerThread(StreamingJobManagerPlugin jobManagerPlugin, ExecutionGraph executionGraph) { this.jobManagerPlugin = jobManagerPlugin; this.executionGraph = executionGraph; - this.latencyModel = new LatencyModel(executionGraph); + this.profilingModel = new ProfilingModel(executionGraph); this.streamingDataQueue = new LinkedBlockingQueue(); } @@ -41,13 +41,16 @@ public void run() { while (!interrupted()) { AbstractStreamingData streamingData = streamingDataQueue.take(); + long now = System.currentTimeMillis(); if (streamingData instanceof ChannelLatency) { - latencyModel.refreshEdgeLatency((ChannelLatency) streamingData); + profilingModel.refreshEdgeLatency(now, (ChannelLatency) streamingData); } else if (streamingData instanceof TaskLatency) { - latencyModel.refreshTaskLatency((TaskLatency) streamingData); + profilingModel.refreshTaskLatency(now, (TaskLatency) streamingData); } else if (streamingData instanceof ChannelThroughput) { - latencyModel.refreshChannelThroughput((ChannelThroughput) streamingData); + profilingModel.refreshChannelThroughput(now, (ChannelThroughput) streamingData); } + + profilingModel.logProfilingSummaryIfNecessary(now); } } catch (InterruptedException e) { @@ -61,7 +64,6 @@ public void handOffStreamingData(AbstractStreamingData data) { } public void limitBufferSize(ManagementEdgeID sourceEdgeID, int bufferSize) { - final ChannelID sourceChannelID = sourceEdgeID.toChannelID(); final ExecutionVertex vertex = this.executionGraph.getVertexByChannelID(sourceChannelID); if (vertex == null) { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java deleted file mode 100644 index 45c9f0052fe11..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyPath.java +++ /dev/null @@ -1,153 +0,0 @@ -package eu.stratosphere.nephele.streaming.latency; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; - -import eu.stratosphere.nephele.managementgraph.ManagementEdge; -import eu.stratosphere.nephele.managementgraph.ManagementVertex; -import eu.stratosphere.nephele.managementgraph.ManagementVertexID; - -/** - * A latency path is a path through the ManagementGraph, defined by a sequence - * of {@link ManagementVertex} objects that are connected in the order in which they appear in - * the sequence. - * - * @author Bjoern Lohrmann - */ -public class LatencyPath implements Iterable { - - private LinkedList pathVertices; - - private HashMap ingoingEdges; - - private LatencySubgraph graph; - - private double pathLatencyInMillis; - - @SuppressWarnings("unchecked") - public LatencyPath(LatencyPath toClone) { - this.graph = toClone.graph; - this.pathVertices = (LinkedList) toClone.pathVertices.clone(); - this.ingoingEdges = (HashMap) toClone.ingoingEdges.clone(); - } - - public LatencyPath(LatencySubgraph graph, ManagementVertex firstVertex) { - this.graph = graph; - this.pathVertices = new LinkedList(); - this.ingoingEdges = new HashMap(); - this.pathVertices.add(firstVertex); - } - - public void appendVertex(ManagementVertex vertex, ManagementEdge ingoingEdge) { - pathVertices.add(vertex); - ingoingEdges.put(vertex.getID(), ingoingEdge); - } - - public ManagementVertex getBegin() { - return pathVertices.getFirst(); - } - - public ManagementVertex getEnd() { - return pathVertices.getLast(); - } - - public ManagementEdge getIngoingEdge(ManagementVertex vertex) { - return ingoingEdges.get(vertex.getID()); - } - - public void removeLastVertex() { - ManagementVertex removed = pathVertices.removeLast(); - ingoingEdges.remove(removed); - } - - /** - * Returns whether we have latency values for all parts (vertices and edges) of this - * path. - * - * @return Whether we have latency values for all parts of this path - */ - public boolean isActive() { - // FIXME inefficient, naive implementation. This may need to be precomputed. - - for (ManagementVertex vertex : pathVertices) { - if (((VertexLatency) vertex.getAttachment()).getLatencyInMillis() == -1) { - return false; - } - } - - for (ManagementEdge edge : ingoingEdges.values()) { - if (((EdgeCharacteristics) edge.getAttachment()).getLatencyInMillis() == -1) { - return false; - } - } - - return true; - } - - @Override - public Iterator iterator() { - return pathVertices.iterator(); - } - - public double refreshPathLatency() { - this.pathLatencyInMillis = 0; - for (ManagementVertex vertex : pathVertices) { - ManagementEdge ingoingEdge = ingoingEdges.get(vertex.getID()); - - if (ingoingEdge != null) { - double edgeLatency = ((EdgeCharacteristics) ingoingEdge.getAttachment()).getLatencyInMillis(); - if (edgeLatency == -1) { - throw new IllegalStateException("Edge latency has not yet been initialized: " + edgeLatency); - } - - this.pathLatencyInMillis += edgeLatency; - } - - double vertexLatency = ((VertexLatency) vertex.getAttachment()).getLatencyInMillis(); - if (vertexLatency == -1) { - throw new IllegalStateException("Vertex latency has not yet been initialized: " + vertexLatency); - } - this.pathLatencyInMillis += vertexLatency; - } - return this.pathLatencyInMillis; - } - - public double getPathLatencyInMillis() { - return this.pathLatencyInMillis; - } - - @Override - public String toString() { - StringBuilder builder = new StringBuilder(); - builder.append("LatencyPath["); - ManagementVertex previous = null; - for (ManagementVertex vertex : pathVertices) { - if (previous != null) { - builder.append("->"); - } - builder.append(vertex); - previous = vertex; - } - builder.append("]"); - - return builder.toString(); - } - - public void dumpLatencies() { - - for (ManagementVertex vertex : pathVertices) { - ManagementEdge ingoing = ingoingEdges.get(vertex.getID()); - - if (ingoing != null) { - System.out.printf("---edge(%.03f)---%s(%.03f)\n", - ((EdgeCharacteristics) ingoing.getAttachment()).getLatencyInMillis(), - vertex, - ((VertexLatency) vertex.getAttachment()).getLatencyInMillis()); - } else { - System.out.printf("%s(%.03f)\n", vertex, - ((VertexLatency) vertex.getAttachment()).getLatencyInMillis()); - } - } - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingLogger.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingLogger.java new file mode 100644 index 0000000000000..a0046e66ff06c --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingLogger.java @@ -0,0 +1,76 @@ +package eu.stratosphere.nephele.streaming.latency; + +import java.io.BufferedWriter; +import java.io.FileWriter; +import java.io.IOException; + +import eu.stratosphere.nephele.managementgraph.ManagementAttachment; +import eu.stratosphere.nephele.managementgraph.ManagementVertex; + +public class ProfilingLogger { + + private ProfilingSubgraph subgraph; + + private BufferedWriter writer; + + private boolean headersWritten; + + public ProfilingLogger(ProfilingSubgraph subgraph) + throws IOException { + + this.subgraph = subgraph; + this.writer = new BufferedWriter(new FileWriter("profiling.txt")); + this.headersWritten = false; + } + + public void logLatencies() throws IOException { + ProfilingSummary summary = new ProfilingSummary(subgraph); + if (!headersWritten) { + writeHeaders(summary); + } + + StringBuilder builder = new StringBuilder(); + builder.append(summary.noOfActivePaths); + builder.append(';'); + builder.append(summary.noOfInactivePaths); + builder.append(';'); + builder.append(summary.avgTotalPathLatency); + builder.append(';'); + builder.append(summary.medianPathLatency); + builder.append(';'); + builder.append(summary.minPathLatency); + builder.append(';'); + builder.append(summary.maxPathLatency); + + for (double avgElementLatency : summary.avgPathElementLatencies) { + builder.append(';'); + builder.append(avgElementLatency); + } + builder.append('\n'); + writer.write(builder.toString()); + writer.flush(); //FIXME + } + + private void writeHeaders(ProfilingSummary summary) throws IOException { + StringBuilder builder = new StringBuilder(); + builder.append("noOfActivePaths;"); + builder.append("noOfInactivePaths;"); + builder.append("avgTotalPathLatency;"); + builder.append("medianPathLatency;"); + builder.append("minPathLatency;"); + builder.append("maxPathLatency"); + + for (ManagementAttachment element : summary.pathElements) { + builder.append(';'); + if (element instanceof ManagementVertex) { + ManagementVertex vertex = (ManagementVertex) element; + builder.append(vertex.getGroupVertex().getName()); + } else { + builder.append("edge"); + } + } + builder.append('\n'); + writer.write(builder.toString()); + headersWritten = true; + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingModel.java new file mode 100644 index 0000000000000..3124330853a6c --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingModel.java @@ -0,0 +1,80 @@ +package eu.stratosphere.nephele.streaming.latency; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; +import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; +import eu.stratosphere.nephele.streaming.types.ChannelLatency; +import eu.stratosphere.nephele.streaming.types.ChannelThroughput; +import eu.stratosphere.nephele.streaming.types.TaskLatency; + +public class ProfilingModel { + + private static Log LOG = LogFactory.getLog(ProfilingModel.class); + + private final static long WAIT_INTERVAL_BEFORE_LOGGING = 10 * 1000; + + private final static long LOGGING_INTERVAL = 1000; + + private ExecutionGraph executionGraph; + + private ProfilingSubgraph profilingSubgraph; + + private ProfilingLogger logger; + + private long timeOfLastLogging; + + public ProfilingModel(ExecutionGraph executionGraph) { + this.executionGraph = executionGraph; + + // FIXME naive implementation until we can annotate the job + // subgraphStart and subgraphEnd should be derived from the annotations + ExecutionGroupVertex subgraphStart = this.executionGraph.getInputVertex(0).getGroupVertex(); + ExecutionGroupVertex subgraphEnd = this.executionGraph.getOutputVertex(0).getGroupVertex(); + + this.profilingSubgraph = new ProfilingSubgraph(executionGraph, subgraphStart, subgraphEnd, false, false); + + try { + this.logger = new ProfilingLogger(profilingSubgraph); + } catch (IOException e) { + LOG.error("Error when opening profiling logger file", e); + } + this.timeOfLastLogging = System.currentTimeMillis() + WAIT_INTERVAL_BEFORE_LOGGING; + } + + public void refreshEdgeLatency(long timestamp, ChannelLatency channelLatency) { + ManagementEdgeID sourceEdgeID = profilingSubgraph.getEdgeByReceiverVertexID(channelLatency.getSinkVertexID() + .toManagementVertexID()); + + EdgeCharacteristics edgeCharacteristics = profilingSubgraph.getEdgeCharacteristicsBySourceEdgeID(sourceEdgeID); + edgeCharacteristics.addLatencyMeasurement(timestamp, channelLatency.getChannelLatency()); + } + + public void refreshTaskLatency(long timestamp, TaskLatency taskLatency) { + VertexLatency vertexLatency = profilingSubgraph + .getVertexLatency(taskLatency.getVertexID().toManagementVertexID()); + vertexLatency.addLatencyMeasurement(timestamp, taskLatency.getTaskLatency()); + } + + public void refreshChannelThroughput(long timestamp, ChannelThroughput channelThroughput) { + ManagementEdgeID edgeID = new ManagementEdgeID(channelThroughput.getSourceChannelID()); + EdgeCharacteristics edgeCharaceristics = profilingSubgraph.getEdgeCharacteristicsBySourceEdgeID(edgeID); + edgeCharaceristics.addThroughputMeasurement(timestamp, channelThroughput.getThroughput()); + } + + public void logProfilingSummaryIfNecessary(long now) { + if ((now - timeOfLastLogging) >= LOGGING_INTERVAL) { + try { + logger.logLatencies(); + } catch (IOException e) { + LOG.error("Error when writing to profiling logger file", e); + } + timeOfLastLogging = now; + } + } + +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPath.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPath.java new file mode 100644 index 0000000000000..c2ff1c2db8a59 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPath.java @@ -0,0 +1,160 @@ +package eu.stratosphere.nephele.streaming.latency; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; + +import eu.stratosphere.nephele.managementgraph.ManagementAttachment; +import eu.stratosphere.nephele.managementgraph.ManagementEdge; +import eu.stratosphere.nephele.managementgraph.ManagementVertex; +import eu.stratosphere.nephele.managementgraph.ManagementVertexID; + +/** + * A profiling path is a path through the ManagementGraph, defined by a sequence + * of sequentially connected {@link ManagementVertex} and {@link ManagementEdge} objects. + * A profiling path may begin with a vertex or an edge an and may end with a vertex or an edge. + * By default profiling paths begin and end with vertices, but this can be changed with + * {{@link #setBeginVertexInProfilingPath(boolean)} and {{@link #setEndVertexInProfilingPath(boolean)}. + * + * @author Bjoern Lohrmann + */ +public class ProfilingPath implements Iterable { + + private LinkedList pathVertices; + + private HashMap ingoingEdges; + + private ProfilingSubgraph graph; + + private ProfilingPathSummary summary; + + private boolean beginVertexInProfilingPath; + + private boolean endVertexInProfilingPath; + + @SuppressWarnings("unchecked") + public ProfilingPath(ProfilingPath toClone) { + this.graph = toClone.graph; + this.pathVertices = (LinkedList) toClone.pathVertices.clone(); + this.ingoingEdges = (HashMap) toClone.ingoingEdges.clone(); + this.beginVertexInProfilingPath = toClone.beginVertexInProfilingPath; + this.endVertexInProfilingPath = toClone.endVertexInProfilingPath; + } + + public ProfilingPath(ProfilingSubgraph graph, ManagementVertex firstVertex) { + this.graph = graph; + this.pathVertices = new LinkedList(); + this.ingoingEdges = new HashMap(); + this.pathVertices.add(firstVertex); + this.beginVertexInProfilingPath = true; + this.endVertexInProfilingPath = true; + } + + public void appendVertex(ManagementVertex vertex, ManagementEdge ingoingEdge) { + pathVertices.add(vertex); + ingoingEdges.put(vertex.getID(), ingoingEdge); + } + + public ManagementVertex getBeginVertex() { + return pathVertices.getFirst(); + } + + public ManagementVertex getEndVertex() { + return pathVertices.getLast(); + } + + public void setBeginVertexInProfilingPath(boolean beginVertexInProfilingPath) { + this.beginVertexInProfilingPath = beginVertexInProfilingPath; + } + + public boolean isBeginVertexOnProfilingPath() { + return beginVertexInProfilingPath; + } + + public void setEndVertexInProfilingPath(boolean endVertexInProfilingPath) { + this.endVertexInProfilingPath = endVertexInProfilingPath; + } + + public boolean isEndVertexOnProfilingPath() { + return endVertexInProfilingPath; + } + + public ManagementEdge getIngoingEdge(ManagementVertex vertex) { + return ingoingEdges.get(vertex.getID()); + } + + public void removeLastVertex() { + ManagementVertex removed = pathVertices.removeLast(); + ingoingEdges.remove(removed); + } + + @Override + public Iterator iterator() { + return pathVertices.iterator(); + } + + public ProfilingPathSummary getSummary() { + if (this.summary == null) { + this.summary = new ProfilingPathSummary(walkProfilingPath()); + } + return this.summary; + } + + private ArrayList walkProfilingPath() { + ArrayList profilingPathElements = new ArrayList(); + + for (ManagementVertex vertex : pathVertices) { + + ManagementEdge ingoingEdge = ingoingEdges.get(vertex.getID()); + if (ingoingEdge != null) { + profilingPathElements.add(ingoingEdge); + } + profilingPathElements.add(vertex); + } + + if (!isBeginVertexOnProfilingPath()) { + profilingPathElements.remove(0); + } + + if (!isEndVertexOnProfilingPath()) { + profilingPathElements.remove(profilingPathElements.size() - 1); + } + + return profilingPathElements; + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("LatencyPath["); + ManagementVertex previous = null; + for (ManagementVertex vertex : pathVertices) { + if (previous != null) { + builder.append("->"); + } + builder.append(vertex); + previous = vertex; + } + builder.append("]"); + + return builder.toString(); + } + + // public void dumpLatencies() { + // + // for (ManagementVertex vertex : pathVertices) { + // ManagementEdge ingoing = ingoingEdges.get(vertex.getID()); + // + // if (ingoing != null) { + // System.out.printf("---edge(%.03f)---%s(%.03f)\n", + // ((EdgeCharacteristics) ingoing.getAttachment()).getLatencyInMillis(), + // vertex, + // ((VertexLatency) vertex.getAttachment()).getLatencyInMillis()); + // } else { + // System.out.printf("%s(%.03f)\n", vertex, + // ((VertexLatency) vertex.getAttachment()).getLatencyInMillis()); + // } + // } + // } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPathSummary.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPathSummary.java new file mode 100644 index 0000000000000..4646776af4433 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPathSummary.java @@ -0,0 +1,145 @@ +package eu.stratosphere.nephele.streaming.latency; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.Map.Entry; + +import eu.stratosphere.nephele.managementgraph.ManagementAttachment; +import eu.stratosphere.nephele.managementgraph.ManagementEdge; +import eu.stratosphere.nephele.managementgraph.ManagementVertex; + +public class ProfilingPathSummary { + + private double totalLatency; + + private double[] latencies; + + private boolean hasLatencies; + + private ArrayList pathElements; + + /** + * Initializes ProfilingPathLatency. + * + * @param pathElements + * Elements (vertices, edges, ..) of the path in the order in which they appear in the path. + */ + public ProfilingPathSummary(ArrayList pathElements) { + this.pathElements = pathElements; + this.latencies = new double[pathElements.size()]; + this.hasLatencies = false; + this.totalLatency = -1; + } + + private class LatencyPathEntry implements Entry { + ManagementAttachment key; + + double value; + + @Override + public ManagementAttachment getKey() { + return key; + } + + @Override + public Double getValue() { + return value; + } + + @Override + public Double setValue(Double value) { + throw new UnsupportedOperationException(); + } + } + + public Iterable> getLatencyIterable() { + return new Iterable>() { + @Override + public Iterator> iterator() { + return new Iterator>() { + int index = 0; + + LatencyPathEntry entry = new LatencyPathEntry(); + + @Override + public boolean hasNext() { + return index < latencies.length; + } + + @Override + public Entry next() { + entry.key = pathElements.get(index); + entry.value = latencies[index]; + return entry; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + }; + } + + /** + * Returns whether we have latency values for all elements (vertices and edges) of this + * path. + * + * @return Whether we have latency values for all parts of this path + */ + public boolean hasLatencies() { + if (!this.hasLatencies) { + + this.hasLatencies = true; + for (ManagementAttachment element : pathElements) { + + if ((element instanceof ManagementVertex && ((VertexLatency) element.getAttachment()) + .getLatencyInMillis() == -1) + || (element instanceof ManagementEdge && ((EdgeCharacteristics) element.getAttachment()) + .getLatencyInMillis() == -1)) { + this.hasLatencies = false; + break; + } + } + } + + return hasLatencies; + } + + public void refreshLatencies() { + if (!hasLatencies()) { + throw new UnsupportedOperationException( + "Elements of profiling path does not have all do not have the necessary latency values yet"); + } + + this.totalLatency = 0; + int index = 0; + for (ManagementAttachment managementAttachment : pathElements) { + double latency; + + if (managementAttachment instanceof ManagementVertex) { + latency = ((VertexLatency) managementAttachment.getAttachment()).getLatencyInMillis(); + } else { + latency = ((EdgeCharacteristics) managementAttachment.getAttachment()).getLatencyInMillis(); + } + + latencies[index] = latency; + this.totalLatency += latency; + index++; + } + } + + public double getTotalLatency() { + return this.totalLatency; + } + + public ArrayList getPathElements() { + return pathElements; + } + + public double[] getLatencies() { + return latencies; + } + +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSubgraph.java similarity index 77% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSubgraph.java index adab0de0bb68a..23edaad471295 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencySubgraph.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSubgraph.java @@ -25,7 +25,7 @@ * * @author Bjoern Lohrmann */ -public class LatencySubgraph { +public class ProfilingSubgraph { // private static Log LOG = LogFactory.getLog(LatencySubgraph.class); @@ -33,7 +33,7 @@ public class LatencySubgraph { private ManagementGroupVertex subgraphEnd; - private List latencyPaths; + private List profilingPaths; private HashMap vertexLatencies = new HashMap(); @@ -41,23 +41,24 @@ public class LatencySubgraph { private HashMap receiverVertexToSourceEdgeIDMap = new HashMap(); - public LatencySubgraph(ExecutionGraph executionGraph, ExecutionGroupVertex subgraphStart, - ExecutionGroupVertex subgraphEnd) { + public ProfilingSubgraph(ExecutionGraph executionGraph, ExecutionGroupVertex subgraphStart, + ExecutionGroupVertex subgraphEnd, boolean includeSubgraphStartInProfilingPaths, + boolean includeSubgraphEndInProfilingPaths) { ManagementGraph managementGraph = ManagementGraphFactory.fromExecutionGraph(executionGraph); determineAnchoringManagementGroupVertices(managementGraph, subgraphStart, subgraphEnd); - buildLatencyPaths(); - initLatenciesOnPaths(); + buildProfilingPaths(includeSubgraphStartInProfilingPaths, includeSubgraphEndInProfilingPaths); + initProfilingAttachmentsOnPaths(); initReceiverVertexToSourceEdgeIDMap(managementGraph); } - private void initLatenciesOnPaths() { - for (LatencyPath path : latencyPaths) { - initLatenciesOnPath(path); + private void initProfilingAttachmentsOnPaths() { + for (ProfilingPath path : profilingPaths) { + initProfilingAttachmentOnPath(path); } } - private void initLatenciesOnPath(LatencyPath path) { + private void initProfilingAttachmentOnPath(ProfilingPath path) { for (ManagementVertex vertex : path) { if (vertex.getAttachment() == null) { @@ -95,27 +96,34 @@ private void initReceiverVertexToSourceEdgeIDMap(final ManagementGraph managemen } } - private void buildLatencyPaths() { - this.latencyPaths = new LinkedList(); + private void buildProfilingPaths(boolean includeSubgraphStartInProfilingPaths, + boolean includeSubgraphEndInProfilingPaths) { + + this.profilingPaths = new LinkedList(); for (int i = 0; i < subgraphStart.getNumberOfGroupMembers(); i++) { ManagementVertex vertex = subgraphStart.getGroupMember(i); - LatencyPath initialPath = new LatencyPath(this, vertex); - depthFirstSearchLatencyPaths(initialPath, this.latencyPaths); + ProfilingPath initialPath = new ProfilingPath(this, vertex); + depthFirstSearchProfilingPaths(initialPath, this.profilingPaths); + } + + for (ProfilingPath profilingPath : profilingPaths) { + profilingPath.setBeginVertexInProfilingPath(includeSubgraphStartInProfilingPaths); + profilingPath.setEndVertexInProfilingPath(includeSubgraphEndInProfilingPaths); } } /** * Performs a recursive depth first search for {@link #subgraphEnd} starting at the end of the given path. - * All paths found to end in {@link #subgraphEnd} are added to foundLatencyPaths. + * All paths found to end in {@link #subgraphEnd} are added to foundProfilingPaths. * * @param path * Initial path with at least one element to start with (will be altered during recursive search). - * @param foundLatencyPaths + * @param foundProfilingPaths * Accumulates the paths found to end at {@link #subgraphEnd} */ - private void depthFirstSearchLatencyPaths(LatencyPath path, List foundLatencyPaths) { - ManagementVertex pathEnd = path.getEnd(); + private void depthFirstSearchProfilingPaths(ProfilingPath path, List foundProfilingPaths) { + ManagementVertex pathEnd = path.getEndVertex(); for (int i = 0; i < pathEnd.getNumberOfOutputGates(); i++) { ManagementGate outputGate = pathEnd.getOutputGate(i); @@ -128,9 +136,9 @@ private void depthFirstSearchLatencyPaths(LatencyPath path, List fo path.appendVertex(extension, edge); if (extension.getGroupVertex() == subgraphEnd) { - foundLatencyPaths.add(new LatencyPath(path)); + foundProfilingPaths.add(new ProfilingPath(path)); } else { - depthFirstSearchLatencyPaths(path, foundLatencyPaths); + depthFirstSearchProfilingPaths(path, foundProfilingPaths); } path.removeLastVertex(); @@ -149,7 +157,7 @@ private void determineAnchoringManagementGroupVertices(ManagementGraph managemen ManagementVertexID vertexInPathEndGroup = pathEndExecVertex.getGroupMember(0).getID().toManagementVertexID(); this.subgraphEnd = managementGraph.getVertexByID(vertexInPathEndGroup).getGroupVertex(); } - + public ManagementEdgeID getEdgeByReceiverVertexID(ManagementVertexID receiverVertexID) { return receiverVertexToSourceEdgeIDMap.get(receiverVertexID); } @@ -162,8 +170,8 @@ public VertexLatency getVertexLatency(ManagementVertexID managementVertexID) { return vertexLatencies.get(managementVertexID); } - public List getLatencyPaths() { - return latencyPaths; + public List getProfilingPaths() { + return profilingPaths; } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSummary.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSummary.java new file mode 100644 index 0000000000000..83aa211e01269 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSummary.java @@ -0,0 +1,87 @@ +package eu.stratosphere.nephele.streaming.latency; + +import java.util.ArrayList; +import java.util.Collections; + +import eu.stratosphere.nephele.managementgraph.ManagementAttachment; + +public class ProfilingSummary { + + ArrayList pathElements; + + int noOfActivePaths; + + int noOfInactivePaths; + + double avgTotalPathLatency; + + double medianPathLatency; + + double minPathLatency; + + double maxPathLatency; + + double[] avgPathElementLatencies; + + public ProfilingSummary(ProfilingSubgraph profilingSubgraph) { + noOfActivePaths = 0; + noOfInactivePaths = 0; + maxPathLatency = Long.MIN_VALUE; + minPathLatency = Long.MAX_VALUE; + pathElements = null; + avgPathElementLatencies = null; + avgTotalPathLatency = 0; + + // will be sorted later on to determine the median + ArrayList totalLatencies = new ArrayList(); + + for (ProfilingPath path : profilingSubgraph.getProfilingPaths()) { + ProfilingPathSummary pathSummary = path.getSummary(); + + if (pathElements == null) { + pathElements = pathSummary.getPathElements(); + avgPathElementLatencies = new double[pathElements.size()]; + } + + if (pathSummary.hasLatencies()) { + // refresh the latency values in the summary + pathSummary.refreshLatencies(); + + avgTotalPathLatency += pathSummary.getTotalLatency(); + totalLatencies.add(pathSummary.getTotalLatency()); + + // add the vertex/edge specific latency values to avgPathLatencies array + addValues(pathSummary.getLatencies(), avgPathElementLatencies); + + noOfActivePaths++; + maxPathLatency = Math.max(maxPathLatency, pathSummary.getTotalLatency()); + minPathLatency = Math.min(minPathLatency, pathSummary.getTotalLatency()); + } else { + noOfInactivePaths++; + } + } + + if (noOfActivePaths > 0) { + for (int i = 0; i < avgPathElementLatencies.length; i++) { + avgPathElementLatencies[i] = avgPathElementLatencies[i] / noOfActivePaths; + } + + avgTotalPathLatency = avgTotalPathLatency / noOfActivePaths; + + Collections.sort(totalLatencies); + medianPathLatency = totalLatencies.get(totalLatencies.size() / 2); + } else { + // set these back to zero because they have been set to very low/high values + // initially + maxPathLatency = 0; + minPathLatency = 0; + } + } + + private void addValues(double[] from, double[] to) { + for (int i = 0; i < from.length; i++) { + to[i] += from[i]; + } + } + +} From 7b1bc203ba8111f04291cdafe0e66a3fd8ee301e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Thu, 24 Nov 2011 23:19:10 +0100 Subject: [PATCH 098/310] renamed streaming.latency package to streaming.profiling --- .../nephele/streaming/StreamingJobManagerPlugin.java | 2 +- .../streaming/{latency => profiling}/EdgeCharacteristics.java | 2 +- .../{latency => profiling}/LatencyOptimizerThread.java | 2 +- .../streaming/{latency => profiling}/ProfilingLogger.java | 2 +- .../streaming/{latency => profiling}/ProfilingModel.java | 2 +- .../streaming/{latency => profiling}/ProfilingPath.java | 2 +- .../streaming/{latency => profiling}/ProfilingPathSummary.java | 2 +- .../streaming/{latency => profiling}/ProfilingSubgraph.java | 2 +- .../streaming/{latency => profiling}/ProfilingSummary.java | 2 +- .../streaming/{latency => profiling}/ProfilingValue.java | 2 +- .../{latency => profiling}/ProfilingValueStatistic.java | 2 +- .../streaming/{latency => profiling}/VertexLatency.java | 2 +- .../nephele/streaming/latency/ProfilingValueStatisticTest.java | 3 +++ 13 files changed, 15 insertions(+), 12 deletions(-) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{latency => profiling}/EdgeCharacteristics.java (96%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{latency => profiling}/LatencyOptimizerThread.java (98%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{latency => profiling}/ProfilingLogger.java (97%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{latency => profiling}/ProfilingModel.java (98%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{latency => profiling}/ProfilingPath.java (98%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{latency => profiling}/ProfilingPathSummary.java (98%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{latency => profiling}/ProfilingSubgraph.java (99%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{latency => profiling}/ProfilingSummary.java (97%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{latency => profiling}/ProfilingValue.java (94%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{latency => profiling}/ProfilingValueStatistic.java (98%) rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/{latency => profiling}/VertexLatency.java (94%) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index b12fa18fc47f2..616d386a29263 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -43,7 +43,7 @@ import eu.stratosphere.nephele.plugins.JobManagerPlugin; import eu.stratosphere.nephele.plugins.PluginID; import eu.stratosphere.nephele.streaming.actions.BufferSizeLimitAction; -import eu.stratosphere.nephele.streaming.latency.LatencyOptimizerThread; +import eu.stratosphere.nephele.streaming.profiling.LatencyOptimizerThread; import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; import eu.stratosphere.nephele.streaming.wrappers.StreamingFileInputWrapper; import eu.stratosphere.nephele.streaming.wrappers.StreamingFileOutputWrapper; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeCharacteristics.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java similarity index 96% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeCharacteristics.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java index 9f1481e6a9e0b..a4a902e869606 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/EdgeCharacteristics.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.streaming.latency; +package eu.stratosphere.nephele.streaming.profiling; import eu.stratosphere.nephele.managementgraph.ManagementEdge; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java similarity index 98% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java index 7f96931fb53f2..27b90891774ea 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/LatencyOptimizerThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.streaming.latency; +package eu.stratosphere.nephele.streaming.profiling; import java.util.concurrent.LinkedBlockingQueue; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingLogger.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java similarity index 97% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingLogger.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java index a0046e66ff06c..93f8d748bfaae 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingLogger.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.streaming.latency; +package eu.stratosphere.nephele.streaming.profiling; import java.io.BufferedWriter; import java.io.FileWriter; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java similarity index 98% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingModel.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java index 3124330853a6c..e65ddce20e8f5 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.streaming.latency; +package eu.stratosphere.nephele.streaming.profiling; import java.io.IOException; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPath.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPath.java similarity index 98% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPath.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPath.java index c2ff1c2db8a59..9f44d8dd3a12c 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPath.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPath.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.streaming.latency; +package eu.stratosphere.nephele.streaming.profiling; import java.util.ArrayList; import java.util.HashMap; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPathSummary.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java similarity index 98% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPathSummary.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java index 4646776af4433..5cb20380bf868 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingPathSummary.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.streaming.latency; +package eu.stratosphere.nephele.streaming.profiling; import java.util.ArrayList; import java.util.Iterator; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java similarity index 99% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSubgraph.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java index 23edaad471295..aafc634007632 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSubgraph.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.streaming.latency; +package eu.stratosphere.nephele.streaming.profiling; import java.util.HashMap; import java.util.Iterator; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSummary.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java similarity index 97% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSummary.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java index 83aa211e01269..8bb03807756f0 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingSummary.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.streaming.latency; +package eu.stratosphere.nephele.streaming.profiling; import java.util.ArrayList; import java.util.Collections; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValue.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValue.java similarity index 94% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValue.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValue.java index 70ea67affe267..c3d535a0190ac 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValue.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValue.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.streaming.latency; +package eu.stratosphere.nephele.streaming.profiling; public class ProfilingValue implements Comparable { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatistic.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java similarity index 98% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatistic.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java index fbee6d0163ff7..0d8c556bf8893 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatistic.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.streaming.latency; +package eu.stratosphere.nephele.streaming.profiling; import java.util.ArrayList; import java.util.Collections; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/VertexLatency.java similarity index 94% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/VertexLatency.java index 7967432171221..16d41d1c072bf 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/latency/VertexLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/VertexLatency.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.streaming.latency; +package eu.stratosphere.nephele.streaming.profiling; import eu.stratosphere.nephele.managementgraph.ManagementVertex; diff --git a/nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java b/nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java index b245e97cc3d27..edd05963c3a46 100644 --- a/nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java +++ b/nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java @@ -5,6 +5,9 @@ import org.junit.Before; import org.junit.Test; +import eu.stratosphere.nephele.streaming.profiling.ProfilingValue; +import eu.stratosphere.nephele.streaming.profiling.ProfilingValueStatistic; + public class ProfilingValueStatisticTest { private ProfilingValueStatistic valueStatistic; From 74de0c39acedf3f699c5388c6dee5ee032c0af5e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Thu, 24 Nov 2011 23:38:56 +0100 Subject: [PATCH 099/310] * log entries now contain proper edge numbering * log entries now contain timestamp --- .../nephele/streaming/profiling/ProfilingLogger.java | 9 +++++++-- .../nephele/streaming/profiling/ProfilingModel.java | 7 +++++-- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java index 93f8d748bfaae..0267a668d4ce2 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java @@ -23,13 +23,15 @@ public ProfilingLogger(ProfilingSubgraph subgraph) this.headersWritten = false; } - public void logLatencies() throws IOException { + public void logLatencies(long timestamp) throws IOException { ProfilingSummary summary = new ProfilingSummary(subgraph); if (!headersWritten) { writeHeaders(summary); } StringBuilder builder = new StringBuilder(); + builder.append(timestamp); + builder.append(';'); builder.append(summary.noOfActivePaths); builder.append(';'); builder.append(summary.noOfInactivePaths); @@ -53,6 +55,7 @@ public void logLatencies() throws IOException { private void writeHeaders(ProfilingSummary summary) throws IOException { StringBuilder builder = new StringBuilder(); + builder.append("timestamp;"); builder.append("noOfActivePaths;"); builder.append("noOfInactivePaths;"); builder.append("avgTotalPathLatency;"); @@ -60,13 +63,15 @@ private void writeHeaders(ProfilingSummary summary) throws IOException { builder.append("minPathLatency;"); builder.append("maxPathLatency"); + int nextEdgeIndex = 1; for (ManagementAttachment element : summary.pathElements) { builder.append(';'); if (element instanceof ManagementVertex) { ManagementVertex vertex = (ManagementVertex) element; builder.append(vertex.getGroupVertex().getName()); } else { - builder.append("edge"); + builder.append("edge" + nextEdgeIndex); + nextEdgeIndex++; } } builder.append('\n'); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java index e65ddce20e8f5..5a1e4bb52aafa 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java @@ -28,6 +28,8 @@ public class ProfilingModel { private long timeOfLastLogging; + private long timeBase; + public ProfilingModel(ExecutionGraph executionGraph) { this.executionGraph = executionGraph; @@ -35,7 +37,7 @@ public ProfilingModel(ExecutionGraph executionGraph) { // subgraphStart and subgraphEnd should be derived from the annotations ExecutionGroupVertex subgraphStart = this.executionGraph.getInputVertex(0).getGroupVertex(); ExecutionGroupVertex subgraphEnd = this.executionGraph.getOutputVertex(0).getGroupVertex(); - + this.profilingSubgraph = new ProfilingSubgraph(executionGraph, subgraphStart, subgraphEnd, false, false); try { @@ -44,6 +46,7 @@ public ProfilingModel(ExecutionGraph executionGraph) { LOG.error("Error when opening profiling logger file", e); } this.timeOfLastLogging = System.currentTimeMillis() + WAIT_INTERVAL_BEFORE_LOGGING; + this.timeBase = timeOfLastLogging; } public void refreshEdgeLatency(long timestamp, ChannelLatency channelLatency) { @@ -69,7 +72,7 @@ public void refreshChannelThroughput(long timestamp, ChannelThroughput channelTh public void logProfilingSummaryIfNecessary(long now) { if ((now - timeOfLastLogging) >= LOGGING_INTERVAL) { try { - logger.logLatencies(); + logger.logLatencies(now - timeBase); } catch (IOException e) { LOG.error("Error when writing to profiling logger file", e); } From 7e4583477b7a044973272367f72bbc31e165f9a6 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 26 Nov 2011 16:19:46 +0000 Subject: [PATCH 100/310] Removed .project files from repository --- nephele/nephele-clustermanager/.project | 23 --------------------- nephele/nephele-common/.project | 23 --------------------- nephele/nephele-compression-bzip2/.project | 23 --------------------- nephele/nephele-compression-lzma/.project | 23 --------------------- nephele/nephele-compression-snappy/.project | 23 --------------------- nephele/nephele-compression-zlib/.project | 23 --------------------- nephele/nephele-ec2cloudmanager/.project | 23 --------------------- nephele/nephele-examples/.project | 23 --------------------- nephele/nephele-hdfs/.project | 23 --------------------- nephele/nephele-management/.project | 23 --------------------- nephele/nephele-profiling/.project | 23 --------------------- nephele/nephele-queuescheduler/.project | 23 --------------------- nephele/nephele-s3/.project | 23 --------------------- nephele/nephele-server/.project | 23 --------------------- nephele/nephele-streaming/.project | 23 --------------------- nephele/nephele-visualization/.project | 23 --------------------- 16 files changed, 368 deletions(-) delete mode 100644 nephele/nephele-clustermanager/.project delete mode 100644 nephele/nephele-common/.project delete mode 100644 nephele/nephele-compression-bzip2/.project delete mode 100644 nephele/nephele-compression-lzma/.project delete mode 100644 nephele/nephele-compression-snappy/.project delete mode 100644 nephele/nephele-compression-zlib/.project delete mode 100644 nephele/nephele-ec2cloudmanager/.project delete mode 100644 nephele/nephele-examples/.project delete mode 100644 nephele/nephele-hdfs/.project delete mode 100644 nephele/nephele-management/.project delete mode 100644 nephele/nephele-profiling/.project delete mode 100644 nephele/nephele-queuescheduler/.project delete mode 100644 nephele/nephele-s3/.project delete mode 100644 nephele/nephele-server/.project delete mode 100644 nephele/nephele-streaming/.project delete mode 100644 nephele/nephele-visualization/.project diff --git a/nephele/nephele-clustermanager/.project b/nephele/nephele-clustermanager/.project deleted file mode 100644 index c0c14e44c9176..0000000000000 --- a/nephele/nephele-clustermanager/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-clustermanager - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-common/.project b/nephele/nephele-common/.project deleted file mode 100644 index cf5422b186565..0000000000000 --- a/nephele/nephele-common/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-common - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-compression-bzip2/.project b/nephele/nephele-compression-bzip2/.project deleted file mode 100644 index 7fbfff9d6c2d0..0000000000000 --- a/nephele/nephele-compression-bzip2/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-compression-bzip2 - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-compression-lzma/.project b/nephele/nephele-compression-lzma/.project deleted file mode 100644 index abe9bccf558b1..0000000000000 --- a/nephele/nephele-compression-lzma/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-compression-lzma - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-compression-snappy/.project b/nephele/nephele-compression-snappy/.project deleted file mode 100644 index 1415e8cb355ea..0000000000000 --- a/nephele/nephele-compression-snappy/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-compression-snappy - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-compression-zlib/.project b/nephele/nephele-compression-zlib/.project deleted file mode 100644 index 2a7c2b84ae02d..0000000000000 --- a/nephele/nephele-compression-zlib/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-compression-zlib - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-ec2cloudmanager/.project b/nephele/nephele-ec2cloudmanager/.project deleted file mode 100644 index 8527f72429d74..0000000000000 --- a/nephele/nephele-ec2cloudmanager/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-ec2cloudmanager - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-examples/.project b/nephele/nephele-examples/.project deleted file mode 100644 index ea9c3934906dd..0000000000000 --- a/nephele/nephele-examples/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-examples - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-hdfs/.project b/nephele/nephele-hdfs/.project deleted file mode 100644 index 9a40a6e4a022a..0000000000000 --- a/nephele/nephele-hdfs/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-hdfs - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-management/.project b/nephele/nephele-management/.project deleted file mode 100644 index a9d85cc576bbd..0000000000000 --- a/nephele/nephele-management/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-management - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-profiling/.project b/nephele/nephele-profiling/.project deleted file mode 100644 index 712a2fed53513..0000000000000 --- a/nephele/nephele-profiling/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-profiling - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-queuescheduler/.project b/nephele/nephele-queuescheduler/.project deleted file mode 100644 index 954d5faa8bae4..0000000000000 --- a/nephele/nephele-queuescheduler/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-queuescheduler - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-s3/.project b/nephele/nephele-s3/.project deleted file mode 100644 index 1f4962858eb7f..0000000000000 --- a/nephele/nephele-s3/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-s3 - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-server/.project b/nephele/nephele-server/.project deleted file mode 100644 index 3d87b6b42ec49..0000000000000 --- a/nephele/nephele-server/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-server - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-streaming/.project b/nephele/nephele-streaming/.project deleted file mode 100644 index 7b8c938718e6b..0000000000000 --- a/nephele/nephele-streaming/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-streaming - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - diff --git a/nephele/nephele-visualization/.project b/nephele/nephele-visualization/.project deleted file mode 100644 index c810a0d04b779..0000000000000 --- a/nephele/nephele-visualization/.project +++ /dev/null @@ -1,23 +0,0 @@ - - - nephele-visualization - - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - - From ba39af7b4c15f57a6fd33560828cca1f158303cf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Sun, 27 Nov 2011 01:39:29 +0100 Subject: [PATCH 101/310] fixed RuntimeException due to bug that caused violation of Comparable vs equals contract --- .../streaming/profiling/ProfilingValue.java | 24 ++++++++++++++++--- .../profiling/ProfilingValueStatistic.java | 18 +++++++------- 2 files changed, 30 insertions(+), 12 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValue.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValue.java index c3d535a0190ac..1c1fb183d7d81 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValue.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValue.java @@ -2,6 +2,10 @@ public class ProfilingValue implements Comparable { + private static long nextFreeId = 0; + + private long id; + private double value; private long timestamp; @@ -9,6 +13,7 @@ public class ProfilingValue implements Comparable { public ProfilingValue(double value, long timestamp) { this.value = value; this.timestamp = timestamp; + this.id = nextFreeId++; } public double getValue() { @@ -27,8 +32,12 @@ public void setTimestamp(long timestamp) { this.timestamp = timestamp; } + public long getId() { + return id; + } + /** - * Sorts first by value and then by timestamp. + * Sorts first by value and then by id. */ @Override public int compareTo(ProfilingValue other) { @@ -37,13 +46,22 @@ public int compareTo(ProfilingValue other) { } else if (this.value < other.value) { return -1; } else { - if (this.timestamp > other.timestamp) { + if (this.id > other.id) { return 1; - } else if (this.timestamp < other.timestamp) { + } else if (this.id < other.id) { return -1; } else { return 0; } } } + + public boolean equals(Object otherObj) { + if (otherObj instanceof ProfilingValue) { + ProfilingValue other = (ProfilingValue) otherObj; + return other.id == this.id; + } else { + return false; + } + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java index 0d8c556bf8893..9ff89a17c285c 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java @@ -8,18 +8,18 @@ public class ProfilingValueStatistic { private ArrayList sortedByValue; - private LinkedList sortedByTimestamp; + private LinkedList sortedById; - private int valueArraySize; + private int statisticWindowSize; private int noOfStoredValues; private double sumOfValues; - public ProfilingValueStatistic(int valueSetSize) { - this.sortedByTimestamp = new LinkedList(); + public ProfilingValueStatistic(int statisticWindowSize) { + this.sortedById = new LinkedList(); this.sortedByValue = new ArrayList(); - this.valueArraySize = valueSetSize; + this.statisticWindowSize = statisticWindowSize; this.noOfStoredValues = 0; this.sumOfValues = 0; } @@ -39,13 +39,13 @@ public void addValue(ProfilingValue value) { } private ProfilingValue insertIntoSortedByTimestamp(ProfilingValue value) { - if (!sortedByTimestamp.isEmpty() && sortedByTimestamp.getLast().getTimestamp() > value.getTimestamp()) { + if (!sortedById.isEmpty() && sortedById.getLast().getId() >= value.getId()) { throw new IllegalArgumentException("Trying to add stale profiling values. This should not happen."); } - sortedByTimestamp.add(value); + sortedById.add(value); - if (noOfStoredValues >= valueArraySize) { - return sortedByTimestamp.removeFirst(); + if (noOfStoredValues >= statisticWindowSize) { + return sortedById.removeFirst(); } else { return null; } From 010a0383f21a1a8f6c1d25c382b92c3101246300 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 27 Nov 2011 17:27:44 +0100 Subject: [PATCH 102/310] Added log message in task manager component of Nephele streaming plugin --- .../nephele/streaming/StreamingTaskManagerPlugin.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index dc8f4a0e54a9e..1a86d0ab561e8 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -94,6 +94,8 @@ public class StreamingTaskManagerPlugin implements TaskManagerPlugin { this.communicationThread = new StreamingCommunicationThread(jobManagerComponent); this.communicationThread.start(); + LOG.info("Configured tagging interval is " + this.taggingInterval); + INSTANCE = this; } From 6e372f8f2092113f034df4ab4ebf90c808ac2aef Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 27 Nov 2011 17:31:13 +0100 Subject: [PATCH 103/310] Added Nephele streaming plugin to distribution --- stratosphere-dist/pom.xml | 5 +++++ .../src/main/stratosphere-bin/bin/nephele-jobmanager.sh | 2 ++ .../src/main/stratosphere-bin/bin/nephele-taskmanager.sh | 2 ++ .../src/main/stratosphere-bin/conf/nephele-plugins.xml | 4 ++-- 4 files changed, 11 insertions(+), 2 deletions(-) diff --git a/stratosphere-dist/pom.xml b/stratosphere-dist/pom.xml index b20ee22147067..856ea9a97b200 100644 --- a/stratosphere-dist/pom.xml +++ b/stratosphere-dist/pom.xml @@ -72,6 +72,11 @@ nephele-visualization ${version} + + eu.stratosphere + nephele-streaming + ${version} + eu.stratosphere pact-common diff --git a/stratosphere-dist/src/main/stratosphere-bin/bin/nephele-jobmanager.sh b/stratosphere-dist/src/main/stratosphere-bin/bin/nephele-jobmanager.sh index 4f5fe1a5ae25d..a1d8ccb5343ea 100755 --- a/stratosphere-dist/src/main/stratosphere-bin/bin/nephele-jobmanager.sh +++ b/stratosphere-dist/src/main/stratosphere-bin/bin/nephele-jobmanager.sh @@ -63,6 +63,8 @@ constructJobManagerClassPath() { add=1 elif [[ "$jarfile" =~ 'nephele-ec2cloudmanager' ]]; then add=1 + elif [[ "$jarfile" =~ 'nephele-streaming' ]]; then + add=1 elif [[ "$jarfile" =~ 'commons-codec' ]]; then add=1 elif [[ "$jarfile" =~ 'commons-httpclient' ]]; then diff --git a/stratosphere-dist/src/main/stratosphere-bin/bin/nephele-taskmanager.sh b/stratosphere-dist/src/main/stratosphere-bin/bin/nephele-taskmanager.sh index 56dc5111f4442..b951b378f295e 100755 --- a/stratosphere-dist/src/main/stratosphere-bin/bin/nephele-taskmanager.sh +++ b/stratosphere-dist/src/main/stratosphere-bin/bin/nephele-taskmanager.sh @@ -50,6 +50,8 @@ constructTaskManagerClassPath() { add=1 elif [[ "$jarfile" =~ 'nephele-profiling' ]]; then add=1 + elif [[ "$jarfile" =~ 'nephele-streaming' ]]; then + add=1 elif [[ "$jarfile" =~ 'pact-common' ]]; then add=1 elif [[ "$jarfile" =~ 'pact-runtime' ]]; then diff --git a/stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml b/stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml index 726cb2708208d..f96612be8e284 100644 --- a/stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml +++ b/stratosphere-dist/src/main/stratosphere-bin/conf/nephele-plugins.xml @@ -21,8 +21,8 @@ eu.stratosphere.nephele.streaming.StreamingPluginLoader - testkey - testvalue + streaming.tagging.interval + 1000 From ae9914b1918ca0ed04734fdc6a26f44950454a09 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Sun, 27 Nov 2011 17:31:57 +0100 Subject: [PATCH 104/310] * fix for edge id lookup problem * workarounds to deal with broken profiling data (identical source and channel ids, NaNs) --- .../streaming/profiling/ProfilingModel.java | 53 +++++++++++++++++-- .../profiling/ProfilingPathSummary.java | 12 +++++ .../profiling/ProfilingSubgraph.java | 30 ++++++++--- .../streaming/profiling/XoredVertexID.java | 40 ++++++++++++++ 4 files changed, 125 insertions(+), 10 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/XoredVertexID.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java index 5a1e4bb52aafa..47aa2c95df625 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java @@ -7,6 +7,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; +import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; import eu.stratosphere.nephele.streaming.types.ChannelLatency; import eu.stratosphere.nephele.streaming.types.ChannelThroughput; @@ -50,20 +51,64 @@ public ProfilingModel(ExecutionGraph executionGraph) { } public void refreshEdgeLatency(long timestamp, ChannelLatency channelLatency) { - ManagementEdgeID sourceEdgeID = profilingSubgraph.getEdgeByReceiverVertexID(channelLatency.getSinkVertexID() - .toManagementVertexID()); + // FIXME workaround for bug that causes NaNs + if (Double.isInfinite(channelLatency.getChannelLatency()) || Double.isNaN(channelLatency.getChannelLatency())) { + return; + } + + // FIXME: workaround for bug caused by streaming plugin + if (!channelLatency.getSourceVertexID().equals(channelLatency.getSinkVertexID())) { + + XoredVertexID xored = new XoredVertexID(channelLatency.getSourceVertexID().toManagementVertexID(), + channelLatency.getSinkVertexID().toManagementVertexID()); + + ManagementEdgeID sourceEdgeID = profilingSubgraph.getSourceEdgeIDByXoredVertexID(xored); + + if (sourceEdgeID == null) { + ExecutionVertex source = executionGraph.getVertexByID(channelLatency.getSourceVertexID()); + ExecutionVertex sink = executionGraph.getVertexByID(channelLatency.getSinkVertexID()); + + throw new RuntimeException("No source edge ID for " + getName(source) + "->" + getName(sink) + " " + + xored.toString()); + } + + EdgeCharacteristics edgeCharacteristics = profilingSubgraph + .getEdgeCharacteristicsBySourceEdgeID(sourceEdgeID); + + edgeCharacteristics.addLatencyMeasurement(timestamp, channelLatency.getChannelLatency()); + } + } + + private String getName(ExecutionVertex source) { + String name = source.getName(); + for (int i = 0; i < source.getGroupVertex().getCurrentNumberOfGroupMembers(); i++) { + if (source.getGroupVertex().getGroupMember(i) == source) { + name += i; + break; + } + } - EdgeCharacteristics edgeCharacteristics = profilingSubgraph.getEdgeCharacteristicsBySourceEdgeID(sourceEdgeID); - edgeCharacteristics.addLatencyMeasurement(timestamp, channelLatency.getChannelLatency()); + return name; } public void refreshTaskLatency(long timestamp, TaskLatency taskLatency) { + // FIXME workaround for bug that causes NaNs + if (Double.isInfinite(taskLatency.getTaskLatency()) || Double.isNaN(taskLatency.getTaskLatency())) { + return; + } + VertexLatency vertexLatency = profilingSubgraph .getVertexLatency(taskLatency.getVertexID().toManagementVertexID()); vertexLatency.addLatencyMeasurement(timestamp, taskLatency.getTaskLatency()); } public void refreshChannelThroughput(long timestamp, ChannelThroughput channelThroughput) { + + // FIXME workaround for bug that causes NaNs + if (Double.isInfinite(channelThroughput.getThroughput()) || Double.isNaN(channelThroughput.getThroughput())) { + return; + } + ManagementEdgeID edgeID = new ManagementEdgeID(channelThroughput.getSourceChannelID()); EdgeCharacteristics edgeCharaceristics = profilingSubgraph.getEdgeCharacteristicsBySourceEdgeID(edgeID); edgeCharaceristics.addThroughputMeasurement(timestamp, channelThroughput.getThroughput()); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java index 5cb20380bf868..93fed348947a8 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java @@ -9,6 +9,8 @@ import eu.stratosphere.nephele.managementgraph.ManagementVertex; public class ProfilingPathSummary { + +// private static Log LOG = LogFactory.getLog(ProfilingPathSummary.class); private double totalLatency; @@ -98,6 +100,16 @@ public boolean hasLatencies() { .getLatencyInMillis() == -1) || (element instanceof ManagementEdge && ((EdgeCharacteristics) element.getAttachment()) .getLatencyInMillis() == -1)) { + +// if(element instanceof ManagementVertex) { +// ManagementVertex vertex = (ManagementVertex) element; +// LOG.info("no data for vertex " + vertex.getName() + vertex.getIndexInGroup()); +// } else { +// ManagementEdge edge = (ManagementEdge) element; +// String sourceName = edge.getSource().getVertex().getName() + edge.getSource().getVertex().getIndexInGroup(); +// String tgName = edge.getTarget().getVertex().getName() + edge.getTarget().getVertex().getIndexInGroup(); +// LOG.info("no data for edge " + sourceName + "-> " + tgName); +// } this.hasLatencies = false; break; } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java index aafc634007632..b5ef77d8adc59 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java @@ -39,7 +39,7 @@ public class ProfilingSubgraph { private HashMap edgeCharacteristics = new HashMap(); - private HashMap receiverVertexToSourceEdgeIDMap = new HashMap(); + private HashMap xoredVertexToSourceEdgeIDMap = new HashMap(); public ProfilingSubgraph(ExecutionGraph executionGraph, ExecutionGroupVertex subgraphStart, ExecutionGroupVertex subgraphEnd, boolean includeSubgraphStartInProfilingPaths, @@ -49,6 +49,8 @@ public ProfilingSubgraph(ExecutionGraph executionGraph, ExecutionGroupVertex sub determineAnchoringManagementGroupVertices(managementGraph, subgraphStart, subgraphEnd); buildProfilingPaths(includeSubgraphStartInProfilingPaths, includeSubgraphEndInProfilingPaths); initProfilingAttachmentsOnPaths(); + + // FIXME this is a workaround and not safe for multi-DAGs initReceiverVertexToSourceEdgeIDMap(managementGraph); } @@ -79,6 +81,7 @@ private void initProfilingAttachmentOnPath(ProfilingPath path) { private void initReceiverVertexToSourceEdgeIDMap(final ManagementGraph managementGraph) { + // FIXME this is a workaround and not safe for multi-DAGs final Iterator it = new ManagementGraphIterator(managementGraph, true); while (it.hasNext()) { @@ -90,15 +93,31 @@ private void initReceiverVertexToSourceEdgeIDMap(final ManagementGraph managemen for (int j = 0; j < numberOfOutgoingEdges; ++j) { final ManagementEdge edge = outputGate.getForwardEdge(j); final ManagementVertex receiver = edge.getTarget().getVertex(); - this.receiverVertexToSourceEdgeIDMap.put(receiver.getID(), edge.getSourceEdgeID()); + + XoredVertexID xored = new XoredVertexID(source.getID(), receiver.getID()); + System.out.println("putting edge " + getName(source) + "->" + getName(receiver) + " " + + xored.toString()); + this.xoredVertexToSourceEdgeIDMap.put(xored, edge.getSourceEdgeID()); } } } } + private String getName(ManagementVertex source) { + String name = source.getName(); + for (int i = 0; i < source.getGroupVertex().getNumberOfGroupMembers(); i++) { + if (source.getGroupVertex().getGroupMember(i) == source) { + name += i; + break; + } + } + + return name; + } + private void buildProfilingPaths(boolean includeSubgraphStartInProfilingPaths, boolean includeSubgraphEndInProfilingPaths) { - + this.profilingPaths = new LinkedList(); for (int i = 0; i < subgraphStart.getNumberOfGroupMembers(); i++) { @@ -158,8 +177,8 @@ private void determineAnchoringManagementGroupVertices(ManagementGraph managemen this.subgraphEnd = managementGraph.getVertexByID(vertexInPathEndGroup).getGroupVertex(); } - public ManagementEdgeID getEdgeByReceiverVertexID(ManagementVertexID receiverVertexID) { - return receiverVertexToSourceEdgeIDMap.get(receiverVertexID); + public ManagementEdgeID getSourceEdgeIDByXoredVertexID(XoredVertexID xored) { + return xoredVertexToSourceEdgeIDMap.get(xored); } public EdgeCharacteristics getEdgeCharacteristicsBySourceEdgeID(ManagementEdgeID sourceEdgeID) { @@ -173,5 +192,4 @@ public VertexLatency getVertexLatency(ManagementVertexID managementVertexID) { public List getProfilingPaths() { return profilingPaths; } - } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/XoredVertexID.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/XoredVertexID.java new file mode 100644 index 0000000000000..608c20b6f01e9 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/XoredVertexID.java @@ -0,0 +1,40 @@ +package eu.stratosphere.nephele.streaming.profiling; + +import java.io.IOException; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.AbstractID; +import eu.stratosphere.nephele.io.DataOutputBuffer; +import eu.stratosphere.nephele.managementgraph.ManagementVertexID; + +public class XoredVertexID extends AbstractID { + + public XoredVertexID(ExecutionVertexID one, ExecutionVertexID two) { + super(xorAbstractIDs(one, two)); + } + + public XoredVertexID(ManagementVertexID one, ManagementVertexID two) { + super(xorAbstractIDs(one, two)); + } + + private static byte[] xorAbstractIDs(AbstractID one, AbstractID two) { + DataOutputBuffer buffer = new DataOutputBuffer(16); + try { + one.write(buffer); + + byte[] data = new byte[16]; + System.arraycopy(buffer.getData().array(), 0, data, 0, 16); + buffer.reset(); + two.write(buffer); + + byte[] twoData = buffer.getData().array(); + for (int i = 0; i < 16; i++) { + data[i] = (byte) (data[i] ^ twoData[i]); + } + return data; + } catch (IOException e) { + e.printStackTrace(); + } + return null; + } +} From ba7d6fefaa862de7827fb3cc07f93b0ed3179876 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 27 Nov 2011 17:33:47 +0100 Subject: [PATCH 105/310] Implemented custom read logic in StreamingInputGate --- .../wrappers/StreamingInputGate.java | 98 ++++++++++++++++++- 1 file changed, 95 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java index c6aeb669b5c33..66a49f4a2264a 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java @@ -15,7 +15,9 @@ package eu.stratosphere.nephele.streaming.wrappers; +import java.io.EOFException; import java.io.IOException; +import java.util.ArrayDeque; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.plugins.wrapper.AbstractInputGateWrapper; @@ -26,6 +28,26 @@ public final class StreamingInputGate extends AbstractInputGat private final StreamListener streamListener; + /** + * Queue with indices of channels that store at least one available record. + */ + private final ArrayDeque availableChannels = new ArrayDeque(); + + /** + * The channel to read from next. + */ + private int channelToReadFrom = -1; + + /** + * The value returned by the last call of waitForAnyChannelToBecomeAvailable + */ + private int availableChannelRetVal = -1; + + /** + * The thread which executes the task connected to the input gate. + */ + private Thread executingThread = null; + StreamingInputGate(final InputGate wrappedInputGate, final StreamListener streamListener) { super(wrappedInputGate); @@ -42,10 +64,80 @@ public final class StreamingInputGate extends AbstractInputGat @Override public T readRecord(final T target) throws IOException, InterruptedException { - final T retVal = getWrappedInputGate().readRecord(target); + T record = null; + + if (this.executingThread == null) { + this.executingThread = Thread.currentThread(); + } + + if (this.executingThread.isInterrupted()) { + throw new InterruptedException(); + } + + while (true) { + + if (this.channelToReadFrom == -1) { + this.availableChannelRetVal = waitForAnyChannelToBecomeAvailable(); + this.channelToReadFrom = this.availableChannelRetVal; + } + try { + record = this.getInputChannel(this.channelToReadFrom).readRecord(target); + } catch (EOFException e) { + // System.out.println("### Caught EOF exception at channel " + channelToReadFrom + "(" + + // this.getInputChannel(channelToReadFrom).getType().toString() + ")"); + if (this.isClosed()) { + return null; + } + } - this.streamListener.recordReceived(retVal); + if (++this.channelToReadFrom == getNumberOfInputChannels()) { + this.channelToReadFrom = 0; + } + + if (record != null) { + break; + } else { + if (this.channelToReadFrom == this.availableChannelRetVal) { + this.channelToReadFrom = -1; + } + } + } - return retVal; + this.streamListener.recordReceived(record); + + return record; + } + + /** + * This method returns the index of a channel which has at least + * one record available. The method may block until at least one + * channel has become ready. + * + * @return the index of the channel which has at least one record available + */ + public int waitForAnyChannelToBecomeAvailable() throws InterruptedException { + + synchronized (this.availableChannels) { + + while (this.availableChannels.isEmpty()) { + + this.availableChannels.wait(); + } + + return this.availableChannels.removeFirst().intValue(); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void notifyRecordIsAvailable(final int channelIndex) { + + synchronized (this.availableChannels) { + + this.availableChannels.add(Integer.valueOf(channelIndex)); + this.availableChannels.notify(); + } } } From ace61a514839d70aa156e1eb8b07c8405daf4483 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 27 Nov 2011 19:47:13 +0100 Subject: [PATCH 106/310] Redesigned implementation to calculate buffer latencies --- .../wrappers/StreamingOutputGate.java | 64 ++++++++++++++----- 1 file changed, 48 insertions(+), 16 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java index 183f117bf9851..19eec99270e5f 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java @@ -16,6 +16,8 @@ package eu.stratosphere.nephele.streaming.wrappers; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; @@ -26,17 +28,48 @@ public final class StreamingOutputGate extends AbstractOutputGateWrapper { - private static final int BUFFER_LATENCY_REPORT_INTERVAL = 10; - private final StreamListener streamListener; private long lastThroughputTimestamp = -1L; - private long lastBufferLatencyTimestamp = -1L; - private long[] lastSentBytes = null; - private int bufferLatencyReportCounter = 0; + private Map bufferLatencyMap = new HashMap(); + + private static final class BufferLatency { + + private static final int BUFFER_LATENCY_REPORT_INTERVAL = 1000; + + private long lastBufferLatencyTimestamp = -1L; + + private int accumulatedLatency = 0; + + private int latencyCounter = 0; + + private BufferLatency(final long initialTimestamp) { + this.lastBufferLatencyTimestamp = initialTimestamp; + } + + private void addLatency(long timestamp) { + + this.accumulatedLatency += (int) (timestamp - this.lastBufferLatencyTimestamp); + this.lastBufferLatencyTimestamp = timestamp; + ++this.latencyCounter; + } + + private int getLatency() { + + if (this.accumulatedLatency < BUFFER_LATENCY_REPORT_INTERVAL) { + return -1; + } + + final int latency = this.accumulatedLatency / this.latencyCounter; + this.accumulatedLatency = 0; + this.latencyCounter = 0; + + return latency; + } + } StreamingOutputGate(final OutputGate wrappedOutputGate, final StreamListener streamListener) { super(wrappedOutputGate); @@ -91,18 +124,17 @@ public void writeRecord(final T record) throws IOException, InterruptedException @Override public void outputBufferSent(final ChannelID channelID) { - if (++this.bufferLatencyReportCounter == BUFFER_LATENCY_REPORT_INTERVAL) { - - final long timestamp = System.currentTimeMillis(); - - if (this.lastBufferLatencyTimestamp >= 0) { - - final int duration = (int) (timestamp - this.lastBufferLatencyTimestamp); - this.streamListener.reportBufferLatency(channelID, duration / BUFFER_LATENCY_REPORT_INTERVAL); - } + final long timestamp = System.currentTimeMillis(); + final BufferLatency bufferLatency = this.bufferLatencyMap.get(channelID); + if (bufferLatency == null) { + this.bufferLatencyMap.put(channelID, new BufferLatency(timestamp)); + return; + } - this.lastBufferLatencyTimestamp = timestamp; - this.bufferLatencyReportCounter = 0; + bufferLatency.addLatency(timestamp); + final int latency = bufferLatency.getLatency(); + if (latency >= 0) { + this.streamListener.reportBufferLatency(channelID, latency); } getWrappedOutputGate().outputBufferSent(channelID); From d9f472d9ad3466d4cedbf4a11ce981f03c10217b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 28 Nov 2011 16:17:45 +0100 Subject: [PATCH 107/310] Fixed bug in custom streaming read logic --- .../streaming/wrappers/StreamingInputGate.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java index 66a49f4a2264a..3e7c6c0301c3b 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java @@ -74,6 +74,8 @@ public T readRecord(final T target) throws IOException, InterruptedException { throw new InterruptedException(); } + final int numberOfInputChannels = getNumberOfInputChannels(); + while (true) { if (this.channelToReadFrom == -1) { @@ -90,16 +92,17 @@ record = this.getInputChannel(this.channelToReadFrom).readRecord(target); } } - if (++this.channelToReadFrom == getNumberOfInputChannels()) { + if (record == null && this.channelToReadFrom == this.availableChannelRetVal) { + this.channelToReadFrom = -1; + continue; + } + + if (++this.channelToReadFrom == numberOfInputChannels) { this.channelToReadFrom = 0; } if (record != null) { break; - } else { - if (this.channelToReadFrom == this.availableChannelRetVal) { - this.channelToReadFrom = -1; - } } } @@ -120,7 +123,6 @@ public int waitForAnyChannelToBecomeAvailable() throws InterruptedException { synchronized (this.availableChannels) { while (this.availableChannels.isEmpty()) { - this.availableChannels.wait(); } From c6daabd9c456158fe9653673e94acae88ff09750 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Mon, 28 Nov 2011 20:31:46 +0100 Subject: [PATCH 108/310] included output buffer latency in profiling data --- .../profiling/EdgeCharacteristics.java | 32 +++- .../profiling/LatencyOptimizerThread.java | 3 + .../streaming/profiling/ProfilingLogger.java | 2 + .../streaming/profiling/ProfilingModel.java | 21 +-- .../profiling/ProfilingPathSummary.java | 143 ++++++++---------- .../profiling/ProfilingSubgraph.java | 15 +- .../streaming/profiling/ProfilingSummary.java | 30 ++-- .../streaming/profiling/ProfilingUtils.java | 44 ++++++ 8 files changed, 160 insertions(+), 130 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java index a4a902e869606..c87795b90bf6d 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java @@ -10,17 +10,20 @@ public class EdgeCharacteristics { private ProfilingValueStatistic throughputInMbitStatistic; + private ProfilingValueStatistic outputBufferLatencyStatistic; + public EdgeCharacteristics(ManagementEdge edge) { this.edge = edge; this.latencyInMillisStatistic = new ProfilingValueStatistic(20); this.throughputInMbitStatistic = new ProfilingValueStatistic(20); + this.outputBufferLatencyStatistic = new ProfilingValueStatistic(20); } public ManagementEdge getEdge() { return edge; } - public double getLatencyInMillis() { + public double getChannelLatencyInMillis() { if (latencyInMillisStatistic.hasValues()) { return latencyInMillisStatistic.getMedianValue(); } else { @@ -28,9 +31,20 @@ public double getLatencyInMillis() { } } - public void addLatencyMeasurement(long timestamp, double latencyInMillis) { - ProfilingValue value = new ProfilingValue(latencyInMillis, timestamp); - this.latencyInMillisStatistic.addValue(value); + public double getChannelThroughputInMbit() { + if (throughputInMbitStatistic.hasValues()) { + return throughputInMbitStatistic.getMedianValue(); + } else { + return -1; + } + } + + public double getOutputBufferLatencyInMillis() { + if (outputBufferLatencyStatistic.hasValues()) { + return outputBufferLatencyStatistic.getMedianValue(); + } else { + return -1; + } } public double getThroughputInMbit() { @@ -41,8 +55,18 @@ public double getThroughputInMbit() { } } + public void addLatencyMeasurement(long timestamp, double latencyInMillis) { + ProfilingValue value = new ProfilingValue(latencyInMillis, timestamp); + this.latencyInMillisStatistic.addValue(value); + } + public void addThroughputMeasurement(long timestamp, double throughputInMbit) { ProfilingValue value = new ProfilingValue(throughputInMbit, timestamp); this.throughputInMbitStatistic.addValue(value); } + + public void addOutputBufferLatencyMeasurement(long timestamp, double latencyInMillis) { + ProfilingValue value = new ProfilingValue(latencyInMillis, timestamp); + this.outputBufferLatencyStatistic.addValue(value); + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java index 27b90891774ea..999b1eed3e414 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java @@ -13,6 +13,7 @@ import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; import eu.stratosphere.nephele.streaming.types.ChannelLatency; import eu.stratosphere.nephele.streaming.types.ChannelThroughput; +import eu.stratosphere.nephele.streaming.types.OutputBufferLatency; import eu.stratosphere.nephele.streaming.types.TaskLatency; public class LatencyOptimizerThread extends Thread { @@ -48,6 +49,8 @@ public void run() { profilingModel.refreshTaskLatency(now, (TaskLatency) streamingData); } else if (streamingData instanceof ChannelThroughput) { profilingModel.refreshChannelThroughput(now, (ChannelThroughput) streamingData); + } else if (streamingData instanceof OutputBufferLatency) { + profilingModel.refreshChannelOutputBufferLatency(now, (OutputBufferLatency) streamingData); } profilingModel.logProfilingSummaryIfNecessary(now); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java index 0267a668d4ce2..341cf2f870ab5 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java @@ -70,6 +70,8 @@ private void writeHeaders(ProfilingSummary summary) throws IOException { ManagementVertex vertex = (ManagementVertex) element; builder.append(vertex.getGroupVertex().getName()); } else { + builder.append("edge" + nextEdgeIndex + "obl"); + builder.append(';'); builder.append("edge" + nextEdgeIndex); nextEdgeIndex++; } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java index 47aa2c95df625..0d4832e2c848e 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java @@ -11,6 +11,7 @@ import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; import eu.stratosphere.nephele.streaming.types.ChannelLatency; import eu.stratosphere.nephele.streaming.types.ChannelThroughput; +import eu.stratosphere.nephele.streaming.types.OutputBufferLatency; import eu.stratosphere.nephele.streaming.types.TaskLatency; public class ProfilingModel { @@ -68,7 +69,7 @@ public void refreshEdgeLatency(long timestamp, ChannelLatency channelLatency) { ExecutionVertex source = executionGraph.getVertexByID(channelLatency.getSourceVertexID()); ExecutionVertex sink = executionGraph.getVertexByID(channelLatency.getSinkVertexID()); - throw new RuntimeException("No source edge ID for " + getName(source) + "->" + getName(sink) + " " + throw new RuntimeException("No source edge ID for " + ProfilingUtils.formatName(source) + "->" + ProfilingUtils.formatName(sink) + " " + xored.toString()); } @@ -79,18 +80,6 @@ public void refreshEdgeLatency(long timestamp, ChannelLatency channelLatency) { } } - private String getName(ExecutionVertex source) { - String name = source.getName(); - for (int i = 0; i < source.getGroupVertex().getCurrentNumberOfGroupMembers(); i++) { - if (source.getGroupVertex().getGroupMember(i) == source) { - name += i; - break; - } - } - - return name; - } - public void refreshTaskLatency(long timestamp, TaskLatency taskLatency) { // FIXME workaround for bug that causes NaNs if (Double.isInfinite(taskLatency.getTaskLatency()) || Double.isNaN(taskLatency.getTaskLatency())) { @@ -113,6 +102,12 @@ public void refreshChannelThroughput(long timestamp, ChannelThroughput channelTh EdgeCharacteristics edgeCharaceristics = profilingSubgraph.getEdgeCharacteristicsBySourceEdgeID(edgeID); edgeCharaceristics.addThroughputMeasurement(timestamp, channelThroughput.getThroughput()); } + + public void refreshChannelOutputBufferLatency(long timestamp, OutputBufferLatency latency) { + ManagementEdgeID sourceEdgeID = new ManagementEdgeID(latency.getSourceChannelID()); + EdgeCharacteristics edgeCharaceristics = profilingSubgraph.getEdgeCharacteristicsBySourceEdgeID(sourceEdgeID); + edgeCharaceristics.addOutputBufferLatencyMeasurement(timestamp, latency.getBufferLatency()); + } public void logProfilingSummaryIfNecessary(long now) { if ((now - timeOfLastLogging) >= LOGGING_INTERVAL) { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java index 93fed348947a8..739f123357225 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java @@ -1,16 +1,13 @@ package eu.stratosphere.nephele.streaming.profiling; import java.util.ArrayList; -import java.util.Iterator; -import java.util.Map.Entry; import eu.stratosphere.nephele.managementgraph.ManagementAttachment; -import eu.stratosphere.nephele.managementgraph.ManagementEdge; import eu.stratosphere.nephele.managementgraph.ManagementVertex; public class ProfilingPathSummary { - -// private static Log LOG = LogFactory.getLog(ProfilingPathSummary.class); + + // private static Log LOG = LogFactory.getLog(ProfilingPathSummary.class); private double totalLatency; @@ -20,6 +17,8 @@ public class ProfilingPathSummary { private ArrayList pathElements; + private int noOfPathElementLatencies; + /** * Initializes ProfilingPathLatency. * @@ -28,60 +27,22 @@ public class ProfilingPathSummary { */ public ProfilingPathSummary(ArrayList pathElements) { this.pathElements = pathElements; - this.latencies = new double[pathElements.size()]; + this.noOfPathElementLatencies = countLatencyValuesOnPath(); + this.latencies = new double[noOfPathElementLatencies]; this.hasLatencies = false; this.totalLatency = -1; } - private class LatencyPathEntry implements Entry { - ManagementAttachment key; - - double value; - - @Override - public ManagementAttachment getKey() { - return key; - } - - @Override - public Double getValue() { - return value; - } - - @Override - public Double setValue(Double value) { - throw new UnsupportedOperationException(); - } - } - - public Iterable> getLatencyIterable() { - return new Iterable>() { - @Override - public Iterator> iterator() { - return new Iterator>() { - int index = 0; - - LatencyPathEntry entry = new LatencyPathEntry(); - - @Override - public boolean hasNext() { - return index < latencies.length; - } - - @Override - public Entry next() { - entry.key = pathElements.get(index); - entry.value = latencies[index]; - return entry; - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; + private int countLatencyValuesOnPath() { + int valuesOnPath = 0; + for (ManagementAttachment element : pathElements) { + if (element instanceof ManagementVertex) { + valuesOnPath++; + } else { + valuesOnPath += 2; } - }; + } + return valuesOnPath; } /** @@ -96,52 +57,66 @@ public boolean hasLatencies() { this.hasLatencies = true; for (ManagementAttachment element : pathElements) { - if ((element instanceof ManagementVertex && ((VertexLatency) element.getAttachment()) - .getLatencyInMillis() == -1) - || (element instanceof ManagementEdge && ((EdgeCharacteristics) element.getAttachment()) - .getLatencyInMillis() == -1)) { - -// if(element instanceof ManagementVertex) { -// ManagementVertex vertex = (ManagementVertex) element; -// LOG.info("no data for vertex " + vertex.getName() + vertex.getIndexInGroup()); -// } else { -// ManagementEdge edge = (ManagementEdge) element; -// String sourceName = edge.getSource().getVertex().getName() + edge.getSource().getVertex().getIndexInGroup(); -// String tgName = edge.getTarget().getVertex().getName() + edge.getTarget().getVertex().getIndexInGroup(); -// LOG.info("no data for edge " + sourceName + "-> " + tgName); -// } - this.hasLatencies = false; - break; + if (element instanceof ManagementVertex) { + VertexLatency vertexLatency = (VertexLatency) element.getAttachment(); + if (vertexLatency.getLatencyInMillis() == -1) { + // ManagementEdge edge = (ManagementEdge) element; + // String sourceName = edge.getSource().getVertex().getName() + + // edge.getSource().getVertex().getIndexInGroup(); + // String tgName = edge.getTarget().getVertex().getName() + + // edge.getTarget().getVertex().getIndexInGroup(); + // LOG.info("no data for edge " + sourceName + "-> " + tgName); + this.hasLatencies = false; + break; + } + } else { + EdgeCharacteristics edgeChar = (EdgeCharacteristics) element.getAttachment(); + if (edgeChar.getChannelLatencyInMillis() == -1 || + edgeChar.getOutputBufferLatencyInMillis() == -1) { + // ManagementVertex vertex = (ManagementVertex) element; + // LOG.info("no data for vertex " + vertex.getName() + vertex.getIndexInGroup()); + this.hasLatencies = false; + break; + } } } } - return hasLatencies; } public void refreshLatencies() { if (!hasLatencies()) { throw new UnsupportedOperationException( - "Elements of profiling path does not have all do not have the necessary latency values yet"); + "Elements of profiling path do not have the necessary latency values yet"); } this.totalLatency = 0; int index = 0; - for (ManagementAttachment managementAttachment : pathElements) { - double latency; + for (ManagementAttachment element : pathElements) { - if (managementAttachment instanceof ManagementVertex) { - latency = ((VertexLatency) managementAttachment.getAttachment()).getLatencyInMillis(); + if (element instanceof ManagementVertex) { + latencies[index] = ((VertexLatency) element.getAttachment()).getLatencyInMillis(); + this.totalLatency += latencies[index]; } else { - latency = ((EdgeCharacteristics) managementAttachment.getAttachment()).getLatencyInMillis(); - } + EdgeCharacteristics edgeCharacteristics = (EdgeCharacteristics) element.getAttachment(); + latencies[index] = edgeCharacteristics.getOutputBufferLatencyInMillis() / 2; - latencies[index] = latency; - this.totalLatency += latency; + if (latencies[index] < 0) { + throw new RuntimeException(ProfilingUtils.formatName(element) + + " has invalid negative output buffer latency: " + latencies[index]); + } + + index++; + // channel latency includes output buffer latency, hence we subtract the output buffer latency + // in order not to count it twice + latencies[index] = Math.max(0, edgeCharacteristics.getChannelLatencyInMillis() - latencies[index - 1]); + + this.totalLatency += latencies[index] + latencies[index - 1]; + } index++; } } - + public double getTotalLatency() { return this.totalLatency; } @@ -150,7 +125,11 @@ public ArrayList getPathElements() { return pathElements; } - public double[] getLatencies() { + public int getNoOfPathElementLatencies() { + return noOfPathElementLatencies; + } + + public double[] getPathElementLatencies() { return latencies; } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java index b5ef77d8adc59..ae156ee766cd7 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java @@ -95,26 +95,13 @@ private void initReceiverVertexToSourceEdgeIDMap(final ManagementGraph managemen final ManagementVertex receiver = edge.getTarget().getVertex(); XoredVertexID xored = new XoredVertexID(source.getID(), receiver.getID()); - System.out.println("putting edge " + getName(source) + "->" + getName(receiver) + " " - + xored.toString()); +// System.out.println("putting edge " + ProfilingUtils.formatName(edge) + " " + xored.toString()); this.xoredVertexToSourceEdgeIDMap.put(xored, edge.getSourceEdgeID()); } } } } - private String getName(ManagementVertex source) { - String name = source.getName(); - for (int i = 0; i < source.getGroupVertex().getNumberOfGroupMembers(); i++) { - if (source.getGroupVertex().getGroupMember(i) == source) { - name += i; - break; - } - } - - return name; - } - private void buildProfilingPaths(boolean includeSubgraphStartInProfilingPaths, boolean includeSubgraphEndInProfilingPaths) { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java index 8bb03807756f0..ea1ab7147181a 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java @@ -10,7 +10,7 @@ public class ProfilingSummary { ArrayList pathElements; int noOfActivePaths; - + int noOfInactivePaths; double avgTotalPathLatency; @@ -26,13 +26,15 @@ public class ProfilingSummary { public ProfilingSummary(ProfilingSubgraph profilingSubgraph) { noOfActivePaths = 0; noOfInactivePaths = 0; - maxPathLatency = Long.MIN_VALUE; - minPathLatency = Long.MAX_VALUE; + avgTotalPathLatency = 0; + minPathLatency = 0; + maxPathLatency = 0; + medianPathLatency = 0; + pathElements = null; avgPathElementLatencies = null; - avgTotalPathLatency = 0; - - // will be sorted later on to determine the median + + // will be sorted later on to determine min, max and median ArrayList totalLatencies = new ArrayList(); for (ProfilingPath path : profilingSubgraph.getProfilingPaths()) { @@ -40,7 +42,7 @@ public ProfilingSummary(ProfilingSubgraph profilingSubgraph) { if (pathElements == null) { pathElements = pathSummary.getPathElements(); - avgPathElementLatencies = new double[pathElements.size()]; + avgPathElementLatencies = new double[pathSummary.getNoOfPathElementLatencies()]; } if (pathSummary.hasLatencies()) { @@ -51,17 +53,15 @@ public ProfilingSummary(ProfilingSubgraph profilingSubgraph) { totalLatencies.add(pathSummary.getTotalLatency()); // add the vertex/edge specific latency values to avgPathLatencies array - addValues(pathSummary.getLatencies(), avgPathElementLatencies); + addValues(pathSummary.getPathElementLatencies(), avgPathElementLatencies); noOfActivePaths++; - maxPathLatency = Math.max(maxPathLatency, pathSummary.getTotalLatency()); - minPathLatency = Math.min(minPathLatency, pathSummary.getTotalLatency()); } else { noOfInactivePaths++; } } - if (noOfActivePaths > 0) { + if (noOfActivePaths > 0) { for (int i = 0; i < avgPathElementLatencies.length; i++) { avgPathElementLatencies[i] = avgPathElementLatencies[i] / noOfActivePaths; } @@ -69,12 +69,9 @@ public ProfilingSummary(ProfilingSubgraph profilingSubgraph) { avgTotalPathLatency = avgTotalPathLatency / noOfActivePaths; Collections.sort(totalLatencies); + minPathLatency = totalLatencies.get(0); medianPathLatency = totalLatencies.get(totalLatencies.size() / 2); - } else { - // set these back to zero because they have been set to very low/high values - // initially - maxPathLatency = 0; - minPathLatency = 0; + maxPathLatency = totalLatencies.get(totalLatencies.size() -1); } } @@ -83,5 +80,4 @@ private void addValues(double[] from, double[] to) { to[i] += from[i]; } } - } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java new file mode 100644 index 0000000000000..f20e7ea28ffec --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java @@ -0,0 +1,44 @@ +package eu.stratosphere.nephele.streaming.profiling; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertex; +import eu.stratosphere.nephele.managementgraph.ManagementAttachment; +import eu.stratosphere.nephele.managementgraph.ManagementEdge; +import eu.stratosphere.nephele.managementgraph.ManagementVertex; + +public class ProfilingUtils { + + public static String formatName(ManagementAttachment managementAttachment) { + if (managementAttachment instanceof ManagementVertex) { + return formatName((ManagementVertex) managementAttachment); + } else { + return formatName((ManagementEdge) managementAttachment); + } + } + + public static String formatName(ManagementEdge edge) { + return formatName(edge.getSource().getVertex()) + "->" + formatName(edge.getTarget().getVertex()); + } + + public static String formatName(ManagementVertex vertex) { + String name = vertex.getName(); + for (int i = 0; i < vertex.getGroupVertex().getNumberOfGroupMembers(); i++) { + if (vertex.getGroupVertex().getGroupMember(i) == vertex) { + name += i; + break; + } + } + return name; + } + + public static String formatName(ExecutionVertex vertex) { + String name = vertex.getName(); + for (int i = 0; i < vertex.getGroupVertex().getCurrentNumberOfGroupMembers(); i++) { + if (vertex.getGroupVertex().getGroupMember(i) == vertex) { + name += i; + break; + } + } + return name; + } + +} From f8c28b8dee309a19c78e7f7433da6396be46268c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 29 Nov 2011 14:27:02 +0100 Subject: [PATCH 109/310] Decreased NIO sleep time from 500 to 50 ms --- .../taskmanager/bytebuffered/OutgoingConnectionThread.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java index 0d07715557375..e315745598ce7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java @@ -144,7 +144,7 @@ public void run() { } try { - this.selector.select(500); + this.selector.select(50); } catch (IOException e) { LOG.error(e); } From 9a0ffc4caf5f682a73632fe76f6e65946b0b7633 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 29 Nov 2011 14:27:30 +0100 Subject: [PATCH 110/310] Introduced mapper class to facilitate dynamic task chaining --- .../eu/stratosphere/nephele/execution/Mapper.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) create mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java new file mode 100644 index 0000000000000..bc1e33df4ec5d --- /dev/null +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java @@ -0,0 +1,12 @@ +package eu.stratosphere.nephele.execution; + +import java.util.Queue; + +import eu.stratosphere.nephele.types.Record; + +public interface Mapper { + + void map(I input, Queue output) throws Exception; + + void close(Queue output); +} From 47dc219a0d41e95e9dea693bdd890dfce83af107 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 29 Nov 2011 16:05:56 +0000 Subject: [PATCH 111/310] Introduced API to construct stream chains --- .../nephele/execution/Environment.java | 5 ++ .../nephele/execution/RuntimeEnvironment.java | 10 +++ .../streaming/StreamingJobManagerPlugin.java | 4 +- .../streaming/StreamingTaskManagerPlugin.java | 11 ++- .../streaming/actions/AbstractAction.java | 24 +---- .../actions/ConstructStreamChainAction.java | 90 +++++++++++++++++++ ...Action.java => LimitBufferSizeAction.java} | 32 ++++++- .../chaining/StreamChainCoordinator.java | 49 ++++++++++ .../streaming/chaining/StreamChainLink.java | 53 +++++++++++ .../streaming/listeners/StreamListener.java | 17 +++- .../listeners/StreamListenerContext.java | 40 ++++++--- .../wrappers/StreamingEnvironment.java | 13 +++ 12 files changed, 302 insertions(+), 46 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/ConstructStreamChainAction.java rename nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/{BufferSizeLimitAction.java => LimitBufferSizeAction.java} (80%) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java index efe45c34d876f..b8729f498283b 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java @@ -22,6 +22,8 @@ import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RecordDeserializer; +import eu.stratosphere.nephele.io.RecordReader; +import eu.stratosphere.nephele.io.RecordWriter; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; @@ -184,4 +186,7 @@ InputGate createInputGate(GateID gateID, RecordDeserializer inputGate); + + void registerMapper(Mapper mapper, RecordReader reader, + RecordWriter writer); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 04ef4a49da909..5795277cb1d14 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -35,6 +35,8 @@ import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RecordDeserializer; +import eu.stratosphere.nephele.io.RecordReader; +import eu.stratosphere.nephele.io.RecordWriter; import eu.stratosphere.nephele.io.RuntimeInputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; @@ -1003,4 +1005,12 @@ private void releaseAllChannelResources() { this.outputGates.get(i).releaseAllChannelResources(); } } + + @Override + public void registerMapper(final Mapper mapper, + final RecordReader reader, final RecordWriter writer) { + + // Nothing to do here + + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 616d386a29263..48304b465317e 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -42,7 +42,7 @@ import eu.stratosphere.nephele.jobgraph.JobOutputVertex; import eu.stratosphere.nephele.plugins.JobManagerPlugin; import eu.stratosphere.nephele.plugins.PluginID; -import eu.stratosphere.nephele.streaming.actions.BufferSizeLimitAction; +import eu.stratosphere.nephele.streaming.actions.LimitBufferSizeAction; import eu.stratosphere.nephele.streaming.profiling.LatencyOptimizerThread; import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; import eu.stratosphere.nephele.streaming.wrappers.StreamingFileInputWrapper; @@ -220,7 +220,7 @@ public void limitBufferSize(final ExecutionVertex vertex, final ChannelID source return; } - final BufferSizeLimitAction bsla = new BufferSizeLimitAction(jobID, vertexID, sourceChannelID, bufferSize); + final LimitBufferSizeAction bsla = new LimitBufferSizeAction(jobID, vertexID, sourceChannelID, bufferSize); try { instance.sendData(this.pluginID, bsla); } catch (IOException e) { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java index 1a86d0ab561e8..af286b8722fc4 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java @@ -30,6 +30,7 @@ import eu.stratosphere.nephele.plugins.PluginCommunication; import eu.stratosphere.nephele.plugins.TaskManagerPlugin; import eu.stratosphere.nephele.streaming.actions.AbstractAction; +import eu.stratosphere.nephele.streaming.chaining.StreamChainCoordinator; import eu.stratosphere.nephele.streaming.listeners.StreamListenerContext; public class StreamingTaskManagerPlugin implements TaskManagerPlugin { @@ -85,6 +86,8 @@ public class StreamingTaskManagerPlugin implements TaskManagerPlugin { */ private final StreamingCommunicationThread communicationThread; + private final StreamChainCoordinator chainCoordinator; + StreamingTaskManagerPlugin(final Configuration pluginConfiguration, final PluginCommunication jobManagerComponent) { this.taggingInterval = pluginConfiguration.getInteger(TAGGING_INTERVAL_KEY, DEFAULT_TAGGING_INTERVAL); @@ -94,6 +97,8 @@ public class StreamingTaskManagerPlugin implements TaskManagerPlugin { this.communicationThread = new StreamingCommunicationThread(jobManagerComponent); this.communicationThread.start(); + this.chainCoordinator = new StreamChainCoordinator(); + LOG.info("Configured tagging interval is " + this.taggingInterval); INSTANCE = this; @@ -138,13 +143,13 @@ public void registerTask(final ExecutionVertexID id, final Configuration jobConf StreamListenerContext listenerContext = null; if (environment.getNumberOfInputGates() == 0) { listenerContext = StreamListenerContext.createForInputTask(jobID, id, this.communicationThread, - aggregationInterval, taggingInterval); + this.chainCoordinator, aggregationInterval, taggingInterval); } else if (environment.getNumberOfOutputGates() == 0) { listenerContext = StreamListenerContext.createForOutputTask(jobID, id, this.communicationThread, - aggregationInterval); + this.chainCoordinator, aggregationInterval); } else { listenerContext = StreamListenerContext.createForRegularTask(jobID, id, this.communicationThread, - aggregationInterval); + this.chainCoordinator, aggregationInterval); } this.listenerContexts.putIfAbsent(idAsString, listenerContext); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java index 0b98841946afc..67b1f9bb4459d 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java @@ -36,31 +36,19 @@ public abstract class AbstractAction implements IOReadableWritable { */ private final JobID jobID; - /** - * The ID of the vertex the initiated action applies to. - */ - private final ExecutionVertexID vertexID; - /** * Constructs a new abstract action object. * * @param jobID * the ID of the job the initiated action applies to - * @param vertexID - * the ID of the vertex the initiated action applies to */ - AbstractAction(final JobID jobID, final ExecutionVertexID vertexID) { + AbstractAction(final JobID jobID) { if (jobID == null) { throw new IllegalArgumentException("Argument jobID must not be null"); } - if (vertexID == null) { - throw new IllegalArgumentException("Argument vertexID must not be null"); - } - this.jobID = jobID; - this.vertexID = vertexID; } /** @@ -68,7 +56,6 @@ public abstract class AbstractAction implements IOReadableWritable { */ AbstractAction() { this.jobID = new JobID(); - this.vertexID = new ExecutionVertexID(); } /** @@ -78,7 +65,6 @@ public abstract class AbstractAction implements IOReadableWritable { public void write(final DataOutput out) throws IOException { this.jobID.write(out); - this.vertexID.write(out); } /** @@ -88,7 +74,6 @@ public void write(final DataOutput out) throws IOException { public void read(final DataInput in) throws IOException { this.jobID.read(in); - this.vertexID.read(in); } /** @@ -100,14 +85,11 @@ public JobID getJobID() { return this.jobID; } - + /** * Returns the ID of the vertex the initiated action applies to. * * @return the ID of the vertex the initiated action applies to */ - public ExecutionVertexID getVertexID() { - - return this.vertexID; - } + public abstract ExecutionVertexID getVertexID(); } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/ConstructStreamChainAction.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/ConstructStreamChainAction.java new file mode 100644 index 0000000000000..13d6629fe511f --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/ConstructStreamChainAction.java @@ -0,0 +1,90 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.actions; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.util.SerializableArrayList; + +/** + * This class implements an action to construct a stream chain for a particular sub-path of the graph. + * + * @author warneke + */ +public final class ConstructStreamChainAction extends AbstractAction { + + private final SerializableArrayList vertexIDs = new SerializableArrayList(); + + public ConstructStreamChainAction(final JobID jobID, final List vertexIDs) { + super(jobID); + + if (vertexIDs == null) { + throw new IllegalArgumentException("Argument vertexIDs must not be null"); + } + + if (vertexIDs.size() < 2) { + throw new IllegalArgumentException("Argument vertexIDs must be a list with at least two elements"); + } + + this.vertexIDs.addAll(vertexIDs); + } + + public ConstructStreamChainAction() { + super(); + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + + super.write(out); + + this.vertexIDs.write(out); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + + super.read(in); + + this.vertexIDs.read(in); + } + + public List getVertexIDs() { + + return Collections.unmodifiableList(this.vertexIDs); + } + + /** + * {@inheritDoc} + */ + @Override + public ExecutionVertexID getVertexID() { + + return this.vertexIDs.get(0); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/LimitBufferSizeAction.java similarity index 80% rename from nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java rename to nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/LimitBufferSizeAction.java index 81ab86c3702c3..33633dd854f4f 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/BufferSizeLimitAction.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/LimitBufferSizeAction.java @@ -28,7 +28,12 @@ * * @author warneke */ -public final class BufferSizeLimitAction extends AbstractAction { +public final class LimitBufferSizeAction extends AbstractAction { + + /** + * The ID of the vertex the initiated action applies to. + */ + private final ExecutionVertexID vertexID; /** * The ID of the output channel whose buffer size shall be limited. @@ -52,9 +57,13 @@ public final class BufferSizeLimitAction extends AbstractAction { * @param bufferSize * the new buffer size in bytes */ - public BufferSizeLimitAction(final JobID jobID, final ExecutionVertexID vertexID, final ChannelID sourceChannelID, + public LimitBufferSizeAction(final JobID jobID, final ExecutionVertexID vertexID, final ChannelID sourceChannelID, final int bufferSize) { - super(jobID, vertexID); + super(jobID); + + if (vertexID == null) { + throw new IllegalArgumentException("Argument vertexID must not be null"); + } if (sourceChannelID == null) { throw new IllegalArgumentException("Argument sourceChannelID must not be null"); @@ -64,6 +73,7 @@ public BufferSizeLimitAction(final JobID jobID, final ExecutionVertexID vertexID throw new IllegalArgumentException("Argument bufferSize must be greather than zero"); } + this.vertexID = vertexID; this.sourceChannelID = sourceChannelID; this.bufferSize = bufferSize; } @@ -71,8 +81,9 @@ public BufferSizeLimitAction(final JobID jobID, final ExecutionVertexID vertexID /** * Default constructor for deserialization. */ - public BufferSizeLimitAction() { + public LimitBufferSizeAction() { super(); + this.vertexID = new ExecutionVertexID(); this.sourceChannelID = new ChannelID(); this.bufferSize = 0; } @@ -85,6 +96,7 @@ public void write(final DataOutput out) throws IOException { super.write(out); + this.vertexID.write(out); this.sourceChannelID.write(out); out.writeInt(this.bufferSize); } @@ -97,6 +109,7 @@ public void read(final DataInput in) throws IOException { super.read(in); + this.vertexID.read(in); this.sourceChannelID.read(in); this.bufferSize = in.readInt(); } @@ -120,4 +133,15 @@ public int getBufferSize() { return this.bufferSize; } + + /** + * Returns the ID of the vertex the initiated action applies to. + * + * @return the ID of the vertex the initiated action applies to + */ + @Override + public ExecutionVertexID getVertexID() { + + return this.vertexID; + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java new file mode 100644 index 0000000000000..e3c45c326cea5 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java @@ -0,0 +1,49 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.chaining; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.execution.Mapper; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.RecordReader; +import eu.stratosphere.nephele.io.RecordWriter; +import eu.stratosphere.nephele.types.Record; + +public final class StreamChainCoordinator { + + /** + * The log object. + */ + private static final Log LOG = LogFactory.getLog(StreamChainCoordinator.class); + + private final ConcurrentMap chainLinks = new ConcurrentHashMap(); + + public void registerMapper(final ExecutionVertexID vertexID, + final Mapper mapper, + final RecordReader reader, final RecordWriter writer) { + + final StreamChainLink chainLink = new StreamChainLink(mapper, reader, writer); + + if (this.chainLinks.putIfAbsent(vertexID, chainLink) == null) { + LOG.info("Registering stream chain link for vertex ID " + vertexID); + } + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java new file mode 100644 index 0000000000000..f96c522648d2f --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java @@ -0,0 +1,53 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.chaining; + +import eu.stratosphere.nephele.execution.Mapper; +import eu.stratosphere.nephele.io.RecordReader; +import eu.stratosphere.nephele.io.RecordWriter; +import eu.stratosphere.nephele.types.Record; + +public final class StreamChainLink { + + private final Mapper mapper; + + private final RecordReader reader; + + private final RecordWriter writer; + + StreamChainLink(final Mapper mapper, RecordReader reader, + RecordWriter writer) { + + this.mapper = mapper; + this.reader = reader; + this.writer = writer; + } + + Mapper getMapper() { + + return this.mapper; + } + + RecordReader getReader() { + + return this.reader; + } + + RecordWriter getWriter() { + + return this.writer; + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java index 211b84f55942a..0de93448c2ccf 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java @@ -25,8 +25,11 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.execution.Mapper; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.RecordReader; +import eu.stratosphere.nephele.io.RecordWriter; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; @@ -34,7 +37,7 @@ import eu.stratosphere.nephele.streaming.StreamingTag; import eu.stratosphere.nephele.streaming.StreamingTaskManagerPlugin; import eu.stratosphere.nephele.streaming.actions.AbstractAction; -import eu.stratosphere.nephele.streaming.actions.BufferSizeLimitAction; +import eu.stratosphere.nephele.streaming.actions.LimitBufferSizeAction; import eu.stratosphere.nephele.streaming.types.ChannelLatency; import eu.stratosphere.nephele.streaming.types.ChannelThroughput; import eu.stratosphere.nephele.streaming.types.OutputBufferLatency; @@ -230,8 +233,8 @@ private void checkForPendingActions() { final AbstractAction action = pendingActions.poll(); - if (action instanceof BufferSizeLimitAction) { - limitBufferSize((BufferSizeLimitAction) action); + if (action instanceof LimitBufferSizeAction) { + limitBufferSize((LimitBufferSizeAction) action); } else { LOG.error("Ignoring unknown action of type " + action.getClass()); } @@ -239,7 +242,7 @@ private void checkForPendingActions() { } } - private void limitBufferSize(final BufferSizeLimitAction bsla) { + private void limitBufferSize(final LimitBufferSizeAction bsla) { final ChannelID sourceChannelID = bsla.getSourceChannelID(); final int bufferSize = bsla.getBufferSize(); @@ -266,4 +269,10 @@ public void registerOutputGate(final StreamingOutputGate outpu this.outputGateMap.put(outputGate.getGateID(), outputGate); } + + public void registerMapper(final Mapper mapper, + final RecordReader reader, final RecordWriter writer) { + + this.listenerContext.registerMapper(mapper, reader, writer); + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java index 911c614b0475b..36bab2aadeefa 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java @@ -18,11 +18,16 @@ import java.util.ArrayDeque; import java.util.Queue; +import eu.stratosphere.nephele.execution.Mapper; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.RecordReader; +import eu.stratosphere.nephele.io.RecordWriter; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.streaming.StreamingCommunicationThread; import eu.stratosphere.nephele.streaming.actions.AbstractAction; +import eu.stratosphere.nephele.streaming.chaining.StreamChainCoordinator; import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; +import eu.stratosphere.nephele.types.Record; public final class StreamListenerContext { @@ -40,6 +45,8 @@ private static enum TaskType { private final StreamingCommunicationThread communicationThread; + private final StreamChainCoordinator chainCoordinator; + private final TaskType taskType; private final int aggregationInterval; @@ -47,8 +54,8 @@ private static enum TaskType { private final int taggingInterval; private StreamListenerContext(final JobID jobID, final ExecutionVertexID vertexID, - final StreamingCommunicationThread communicationThread, final TaskType taskType, - final int aggregationInterval, final int taggingInterval) { + final StreamingCommunicationThread communicationThread, final StreamChainCoordinator chainCoordinator, + final TaskType taskType, final int aggregationInterval, final int taggingInterval) { if (jobID == null) { throw new IllegalArgumentException("Parameter jobID must not be null"); @@ -77,31 +84,34 @@ private StreamListenerContext(final JobID jobID, final ExecutionVertexID vertexI this.jobID = jobID; this.vertexID = vertexID; this.communicationThread = communicationThread; + this.chainCoordinator = chainCoordinator; this.taskType = taskType; this.aggregationInterval = aggregationInterval; this.taggingInterval = taggingInterval; } public static StreamListenerContext createForInputTask(final JobID jobID, final ExecutionVertexID vertexID, - final StreamingCommunicationThread communicationThread, final int aggregationInterval, - final int taggingInterval) { + final StreamingCommunicationThread communicationThread, final StreamChainCoordinator chainCoordinator, + final int aggregationInterval, final int taggingInterval) { - return new StreamListenerContext(jobID, vertexID, communicationThread, TaskType.INPUT, aggregationInterval, - taggingInterval); + return new StreamListenerContext(jobID, vertexID, communicationThread, chainCoordinator, TaskType.INPUT, + aggregationInterval, taggingInterval); } public static StreamListenerContext createForRegularTask(final JobID jobID, final ExecutionVertexID vertexID, - final StreamingCommunicationThread communicationThread, final int aggregationInterval) { + final StreamingCommunicationThread communicationThread, final StreamChainCoordinator chainCoordinator, + final int aggregationInterval) { - return new StreamListenerContext(jobID, vertexID, communicationThread, TaskType.REGULAR, aggregationInterval, - -1); + return new StreamListenerContext(jobID, vertexID, communicationThread, chainCoordinator, TaskType.REGULAR, + aggregationInterval, -1); } public static StreamListenerContext createForOutputTask(final JobID jobID, final ExecutionVertexID vertexID, - final StreamingCommunicationThread communicationThread, final int aggregationInterval) { + final StreamingCommunicationThread communicationThread, final StreamChainCoordinator chainCoordinator, + final int aggregationInterval) { - return new StreamListenerContext(jobID, vertexID, communicationThread, TaskType.OUTPUT, aggregationInterval, - -1); + return new StreamListenerContext(jobID, vertexID, communicationThread, chainCoordinator, TaskType.OUTPUT, + aggregationInterval, -1); } boolean isInputVertex() { @@ -155,4 +165,10 @@ Queue getPendingActionsQueue() { return this.pendingActions; } + + void registerMapper(final Mapper mapper, + final RecordReader reader, final RecordWriter writer) { + + this.chainCoordinator.registerMapper(this.vertexID, mapper, reader, writer); + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java index baa2788ac0ab8..b7c958b953a3c 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java @@ -16,12 +16,15 @@ package eu.stratosphere.nephele.streaming.wrappers; import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.Mapper; import eu.stratosphere.nephele.io.ChannelSelector; import eu.stratosphere.nephele.io.DistributionPattern; import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RecordDeserializer; +import eu.stratosphere.nephele.io.RecordReader; +import eu.stratosphere.nephele.io.RecordWriter; import eu.stratosphere.nephele.plugins.wrapper.AbstractEnvironmentWrapper; import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.types.Record; @@ -80,4 +83,14 @@ public InputGate createInputGate(final GateID gateID, return new StreamingInputGate(inputGate, this.streamListener); } + + /** + * {@inheritDoc} + */ + @Override + public void registerMapper(final Mapper mapper, + final RecordReader reader, final RecordWriter writer) { + + this.streamListener.registerMapper(mapper, reader, writer); + } } From 8a55215739adcb17c95b371fd7cd193f5c075dac Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 29 Nov 2011 18:16:37 +0000 Subject: [PATCH 112/310] Finished implementation of stream chaining --- .../nephele/execution/Environment.java | 5 +- .../nephele/execution/RuntimeEnvironment.java | 5 +- .../streaming/chaining/StreamChain.java | 80 +++++++++++++++++++ .../chaining/StreamChainCoordinator.java | 32 +++++++- .../streaming/chaining/StreamChainLink.java | 31 +++---- .../streaming/listeners/StreamListener.java | 26 +++++- .../listeners/StreamListenerContext.java | 15 +++- .../wrappers/StreamingEnvironment.java | 32 ++++++-- .../wrappers/StreamingOutputGate.java | 16 +++- 9 files changed, 199 insertions(+), 43 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java index b8729f498283b..892b9cba1914a 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java @@ -22,8 +22,6 @@ import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RecordDeserializer; -import eu.stratosphere.nephele.io.RecordReader; -import eu.stratosphere.nephele.io.RecordWriter; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; @@ -187,6 +185,5 @@ InputGate createInputGate(GateID gateID, RecordDeserializer inputGate); - void registerMapper(Mapper mapper, RecordReader reader, - RecordWriter writer); + void registerMapper(Mapper mapper); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 5795277cb1d14..8582ab747dfb2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -35,8 +35,6 @@ import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RecordDeserializer; -import eu.stratosphere.nephele.io.RecordReader; -import eu.stratosphere.nephele.io.RecordWriter; import eu.stratosphere.nephele.io.RuntimeInputGate; import eu.stratosphere.nephele.io.RuntimeOutputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; @@ -1007,8 +1005,7 @@ private void releaseAllChannelResources() { } @Override - public void registerMapper(final Mapper mapper, - final RecordReader reader, final RecordWriter writer) { + public void registerMapper(final Mapper mapper) { // Nothing to do here diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java new file mode 100644 index 0000000000000..c052b5a7526a4 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java @@ -0,0 +1,80 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.chaining; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.List; +import java.util.Queue; + +import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; +import eu.stratosphere.nephele.types.Record; +import eu.stratosphere.nephele.util.StringUtils; + +public final class StreamChain { + + @SuppressWarnings("rawtypes") + private final List chainLinks; + + @SuppressWarnings("rawtypes") + StreamChain(final List chainLinks) { + + if (chainLinks.isEmpty()) { + throw new IllegalArgumentException("List chainLinks must not be empty"); + } + + this.chainLinks = chainLinks; + } + + @SuppressWarnings("unchecked") + public StreamingOutputGate getFirstOutputGate() { + + return this.chainLinks.get(0).getOutputGate(); + } + + public void writeRecord(final Record record) throws IOException, InterruptedException { + + try { + executeMapper(record, 1); + } catch (Exception e) { + throw new IOException(StringUtils.stringifyException(e)); + } + } + + @SuppressWarnings({ "rawtypes", "unchecked" }) + void executeMapper(final Record record, final int chainIndex) throws Exception { + + final StreamChainLink chainLink = this.chainLinks.get(chainIndex); + final Queue output = new ArrayDeque(); + + chainLink.getMapper().map(record, output); + + if (chainIndex == this.chainLinks.size() - 1) { + + final StreamingOutputGate outputGate = chainLink.getOutputGate(); + + while (!output.isEmpty()) { + outputGate.writeRecord((Record)output.poll()); + } + + } else { + + while (!output.isEmpty()) { + executeMapper((Record)output.poll(), chainIndex+1); + } + } + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java index e3c45c326cea5..b12e9727b472e 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java @@ -15,6 +15,10 @@ package eu.stratosphere.nephele.streaming.chaining; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -23,8 +27,8 @@ import eu.stratosphere.nephele.execution.Mapper; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.RecordReader; -import eu.stratosphere.nephele.io.RecordWriter; +import eu.stratosphere.nephele.streaming.wrappers.StreamingInputGate; +import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; import eu.stratosphere.nephele.types.Record; public final class StreamChainCoordinator { @@ -38,12 +42,32 @@ public final class StreamChainCoordinator { public void registerMapper(final ExecutionVertexID vertexID, final Mapper mapper, - final RecordReader reader, final RecordWriter writer) { + final StreamingInputGate inputGate, final StreamingOutputGate outputGate) { - final StreamChainLink chainLink = new StreamChainLink(mapper, reader, writer); + final StreamChainLink chainLink = new StreamChainLink(mapper, inputGate, outputGate); if (this.chainLinks.putIfAbsent(vertexID, chainLink) == null) { LOG.info("Registering stream chain link for vertex ID " + vertexID); } } + + public StreamChain constructStreamChain(final List vertexIDs) { + + final Iterator it = vertexIDs.iterator(); + final List chainLinkList = new ArrayList(); + while (it.hasNext()) { + + final ExecutionVertexID vertexID = it.next(); + final StreamChainLink chainLink = this.chainLinks.get(vertexID); + if (chainLink == null) { + LOG.error("Cannot construct stream chain from " + vertexIDs.get(0) + " to " + + vertexIDs.get(vertexIDs.size() - 1) + ": No chain link for vertex ID " + vertexID); + return null; + } + + chainLinkList.add(chainLink); + } + + return new StreamChain(Collections.unmodifiableList(chainLinkList)); + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java index f96c522648d2f..0003b77719372 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java @@ -16,38 +16,39 @@ package eu.stratosphere.nephele.streaming.chaining; import eu.stratosphere.nephele.execution.Mapper; -import eu.stratosphere.nephele.io.RecordReader; -import eu.stratosphere.nephele.io.RecordWriter; +import eu.stratosphere.nephele.streaming.wrappers.StreamingInputGate; +import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; import eu.stratosphere.nephele.types.Record; -public final class StreamChainLink { +public final class StreamChainLink { - private final Mapper mapper; + private final Mapper mapper; - private final RecordReader reader; + private final StreamingInputGate inputGate; - private final RecordWriter writer; + private final StreamingOutputGate outputGate; - StreamChainLink(final Mapper mapper, RecordReader reader, - RecordWriter writer) { + StreamChainLink(final Mapper mapper, + final StreamingInputGate inputGate, + final StreamingOutputGate outputGate) { this.mapper = mapper; - this.reader = reader; - this.writer = writer; + this.inputGate = inputGate; + this.outputGate = outputGate; } - Mapper getMapper() { + Mapper getMapper() { return this.mapper; } - RecordReader getReader() { + StreamingInputGate getInputGate() { - return this.reader; + return this.inputGate; } - RecordWriter getWriter() { + StreamingOutputGate getOutputGate() { - return this.writer; + return this.outputGate; } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java index 0de93448c2ccf..d2e129e86f42c 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java @@ -28,8 +28,6 @@ import eu.stratosphere.nephele.execution.Mapper; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.GateID; -import eu.stratosphere.nephele.io.RecordReader; -import eu.stratosphere.nephele.io.RecordWriter; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; @@ -37,11 +35,14 @@ import eu.stratosphere.nephele.streaming.StreamingTag; import eu.stratosphere.nephele.streaming.StreamingTaskManagerPlugin; import eu.stratosphere.nephele.streaming.actions.AbstractAction; +import eu.stratosphere.nephele.streaming.actions.ConstructStreamChainAction; import eu.stratosphere.nephele.streaming.actions.LimitBufferSizeAction; +import eu.stratosphere.nephele.streaming.chaining.StreamChain; import eu.stratosphere.nephele.streaming.types.ChannelLatency; import eu.stratosphere.nephele.streaming.types.ChannelThroughput; import eu.stratosphere.nephele.streaming.types.OutputBufferLatency; import eu.stratosphere.nephele.streaming.types.TaskLatency; +import eu.stratosphere.nephele.streaming.wrappers.StreamingInputGate; import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; import eu.stratosphere.nephele.types.AbstractTaggableRecord; import eu.stratosphere.nephele.types.Record; @@ -235,6 +236,8 @@ private void checkForPendingActions() { if (action instanceof LimitBufferSizeAction) { limitBufferSize((LimitBufferSizeAction) action); + } else if (action instanceof ConstructStreamChainAction) { + constructStreamChain((ConstructStreamChainAction) action); } else { LOG.error("Ignoring unknown action of type " + action.getClass()); } @@ -242,6 +245,21 @@ private void checkForPendingActions() { } } + private void constructStreamChain(final ConstructStreamChainAction csca) { + + final StreamChain streamChain = this.listenerContext.constructStreamChain(csca.getVertexIDs()); + if (streamChain == null) { + return; + } + + final StreamingOutputGate outputGate = streamChain.getFirstOutputGate(); + try { + outputGate.redirectToStreamChain(streamChain); + } catch(Exception e) { + LOG.error(StringUtils.stringifyException(e)); + } + } + private void limitBufferSize(final LimitBufferSizeAction bsla) { final ChannelID sourceChannelID = bsla.getSourceChannelID(); @@ -271,8 +289,8 @@ public void registerOutputGate(final StreamingOutputGate outpu } public void registerMapper(final Mapper mapper, - final RecordReader reader, final RecordWriter writer) { + final StreamingInputGate input, final StreamingOutputGate output) { - this.listenerContext.registerMapper(mapper, reader, writer); + this.listenerContext.registerMapper(mapper, input, output); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java index 36bab2aadeefa..010eafcbf2ff6 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java @@ -16,17 +16,19 @@ package eu.stratosphere.nephele.streaming.listeners; import java.util.ArrayDeque; +import java.util.List; import java.util.Queue; import eu.stratosphere.nephele.execution.Mapper; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.RecordReader; -import eu.stratosphere.nephele.io.RecordWriter; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.streaming.StreamingCommunicationThread; import eu.stratosphere.nephele.streaming.actions.AbstractAction; +import eu.stratosphere.nephele.streaming.chaining.StreamChain; import eu.stratosphere.nephele.streaming.chaining.StreamChainCoordinator; import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; +import eu.stratosphere.nephele.streaming.wrappers.StreamingInputGate; +import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; import eu.stratosphere.nephele.types.Record; public final class StreamListenerContext { @@ -167,8 +169,13 @@ Queue getPendingActionsQueue() { } void registerMapper(final Mapper mapper, - final RecordReader reader, final RecordWriter writer) { + final StreamingInputGate inputGate, final StreamingOutputGate outputGate) { - this.chainCoordinator.registerMapper(this.vertexID, mapper, reader, writer); + this.chainCoordinator.registerMapper(this.vertexID, mapper, inputGate, outputGate); + } + + StreamChain constructStreamChain(final List vertexIDs) { + + return this.chainCoordinator.constructStreamChain(vertexIDs); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java index b7c958b953a3c..cf694d6d3100f 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java @@ -15,6 +15,9 @@ package eu.stratosphere.nephele.streaming.wrappers; +import java.util.ArrayList; +import java.util.List; + import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.Mapper; import eu.stratosphere.nephele.io.ChannelSelector; @@ -23,8 +26,6 @@ import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RecordDeserializer; -import eu.stratosphere.nephele.io.RecordReader; -import eu.stratosphere.nephele.io.RecordWriter; import eu.stratosphere.nephele.plugins.wrapper.AbstractEnvironmentWrapper; import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.types.Record; @@ -41,6 +42,10 @@ public final class StreamingEnvironment extends AbstractEnvironmentWrapper { private final StreamListener streamListener; + private final List> streamingInputGates = new ArrayList>(); + + private final List> streamingOutputGates = new ArrayList>(); + /** * Constructs a new streaming environment * @@ -67,7 +72,10 @@ public OutputGate createOutputGate(final GateID gateID, final OutputGate outputGate = getWrappedEnvironment().createOutputGate(gateID, outputClass, selector, isBroadcast); - return new StreamingOutputGate(outputGate, this.streamListener); + final StreamingOutputGate sog = new StreamingOutputGate(outputGate, this.streamListener); + this.streamingOutputGates.add(sog); + + return sog; } /** @@ -81,16 +89,26 @@ public InputGate createInputGate(final GateID gateID, final InputGate inputGate = getWrappedEnvironment().createInputGate(gateID, deserializer, distributionPattern); - return new StreamingInputGate(inputGate, this.streamListener); + final StreamingInputGate sig = new StreamingInputGate(inputGate, this.streamListener); + this.streamingInputGates.add(sig); + + return sig; } /** * {@inheritDoc} */ @Override - public void registerMapper(final Mapper mapper, - final RecordReader reader, final RecordWriter writer) { + public void registerMapper(final Mapper mapper) { + + if (this.streamingInputGates.size() != 1) { + return; + } + + if (this.streamingOutputGates.size() != 1) { + return; + } - this.streamListener.registerMapper(mapper, reader, writer); + this.streamListener.registerMapper(mapper, this.streamingInputGates.get(0), this.streamingOutputGates.get(0)); } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java index 19eec99270e5f..8cb2033fee661 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java @@ -23,6 +23,7 @@ import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.plugins.wrapper.AbstractOutputGateWrapper; +import eu.stratosphere.nephele.streaming.chaining.StreamChain; import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.types.Record; @@ -34,6 +35,8 @@ public final class StreamingOutputGate extends AbstractOutputG private long[] lastSentBytes = null; + private StreamChain streamChain = null; + private Map bufferLatencyMap = new HashMap(); private static final class BufferLatency { @@ -115,7 +118,11 @@ public void writeRecord(final T record) throws IOException, InterruptedException this.lastThroughputTimestamp = timestamp; } - getWrappedOutputGate().writeRecord(record); + if (this.streamChain == null) { + getWrappedOutputGate().writeRecord(record); + } else { + this.streamChain.writeRecord(record); + } } /** @@ -139,4 +146,11 @@ public void outputBufferSent(final ChannelID channelID) { getWrappedOutputGate().outputBufferSent(channelID); } + + public void redirectToStreamChain(final StreamChain streamChain) throws IOException, InterruptedException { + + getWrappedOutputGate().flush(); + + this.streamChain = streamChain; + } } From b17995dbeddc8f6556ae2a1eb736c03f71751a76 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 29 Nov 2011 19:05:26 +0000 Subject: [PATCH 113/310] Temporarily disabled thread-safety check --- .../main/java/eu/stratosphere/nephele/taskmanager/Task.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index 596b6ab7a8598..bad746a237b29 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -15,7 +15,6 @@ package eu.stratosphere.nephele.taskmanager; -import java.util.ConcurrentModificationException; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -234,10 +233,11 @@ public boolean isCanceled() { */ public void initialExecutionResourcesExhausted() { - if (this.environment.getExecutingThread() != Thread.currentThread()) { + //TODO: Reactivate me + /*if (this.environment.getExecutingThread() != Thread.currentThread()) { throw new ConcurrentModificationException( "initialExecutionResourcesExhausted must be called from the task that executes the user code"); - } + }*/ // Construct a resource utilization snapshot final long timestamp = System.currentTimeMillis(); From f5527cf77556cd7d814107ad593c4694442baa0d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 29 Nov 2011 19:47:25 +0000 Subject: [PATCH 114/310] Integrated statistics collection in stream chaining --- .../streaming/chaining/StreamChain.java | 14 +++++++++----- .../streaming/chaining/StreamChainLink.java | 3 +-- .../streaming/listeners/StreamListener.java | 3 +++ .../wrappers/StreamingInputGate.java | 7 ++++++- .../wrappers/StreamingOutputGate.java | 19 +++++++++++-------- 5 files changed, 30 insertions(+), 16 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java index c052b5a7526a4..8484dd492d195 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java @@ -60,20 +60,24 @@ void executeMapper(final Record record, final int chainIndex) throws Exception { final StreamChainLink chainLink = this.chainLinks.get(chainIndex); final Queue output = new ArrayDeque(); + + chainLink.getInputGate().reportRecordReceived(record); chainLink.getMapper().map(record, output); + final StreamingOutputGate outputGate = chainLink.getOutputGate(); + if (chainIndex == this.chainLinks.size() - 1) { - final StreamingOutputGate outputGate = chainLink.getOutputGate(); - while (!output.isEmpty()) { - outputGate.writeRecord((Record)output.poll()); + outputGate.writeRecord((Record) output.poll()); } - + } else { while (!output.isEmpty()) { - executeMapper((Record)output.poll(), chainIndex+1); + final Record outputRecord = (Record) output.poll(); + outputGate.reportRecordEmitted(outputRecord); + executeMapper(outputRecord, chainIndex + 1); } } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java index 0003b77719372..da53a9c756010 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java @@ -28,8 +28,7 @@ public final class StreamChainLink { private final StreamingOutputGate outputGate; - StreamChainLink(final Mapper mapper, - final StreamingInputGate inputGate, + StreamChainLink(final Mapper mapper, final StreamingInputGate inputGate, final StreamingOutputGate outputGate) { this.mapper = mapper; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java index d2e129e86f42c..5129371524b0f 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java @@ -254,6 +254,9 @@ private void constructStreamChain(final ConstructStreamChainAction csca) { final StreamingOutputGate outputGate = streamChain.getFirstOutputGate(); try { + + outputGate.flush(); + outputGate.redirectToStreamChain(streamChain); } catch(Exception e) { LOG.error(StringUtils.stringifyException(e)); diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java index 3e7c6c0301c3b..d652480eebba3 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java @@ -106,11 +106,16 @@ record = this.getInputChannel(this.channelToReadFrom).readRecord(target); } } - this.streamListener.recordReceived(record); + reportRecordReceived(record); return record; } + public void reportRecordReceived(final Record record) { + + this.streamListener.recordReceived(record); + } + /** * This method returns the index of a channel which has at least * one record available. The method may block until at least one diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java index 8cb2033fee661..e8de321349377 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java @@ -92,6 +92,17 @@ private int getLatency() { @Override public void writeRecord(final T record) throws IOException, InterruptedException { + reportRecordEmitted(record); + + if (this.streamChain == null) { + getWrappedOutputGate().writeRecord(record); + } else { + this.streamChain.writeRecord(record); + } + } + + public void reportRecordEmitted(final Record record) { + final long timestamp = this.streamListener.recordEmitted(record); if (timestamp >= 0) { @@ -117,12 +128,6 @@ public void writeRecord(final T record) throws IOException, InterruptedException this.lastThroughputTimestamp = timestamp; } - - if (this.streamChain == null) { - getWrappedOutputGate().writeRecord(record); - } else { - this.streamChain.writeRecord(record); - } } /** @@ -149,8 +154,6 @@ public void outputBufferSent(final ChannelID channelID) { public void redirectToStreamChain(final StreamChain streamChain) throws IOException, InterruptedException { - getWrappedOutputGate().flush(); - this.streamChain = streamChain; } } From f5ca67039f2dfc768319b582d37118c6575dfb24 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 29 Nov 2011 19:48:27 +0000 Subject: [PATCH 115/310] Added code to trigger chaining event --- .../streaming/StreamingJobManagerPlugin.java | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 48304b465317e..2d4364df878e0 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -16,7 +16,9 @@ package eu.stratosphere.nephele.streaming; import java.io.IOException; +import java.util.ArrayList; import java.util.Iterator; +import java.util.List; import java.util.concurrent.ConcurrentHashMap; @@ -25,6 +27,7 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.executiongraph.InternalJobStatus; @@ -42,6 +45,7 @@ import eu.stratosphere.nephele.jobgraph.JobOutputVertex; import eu.stratosphere.nephele.plugins.JobManagerPlugin; import eu.stratosphere.nephele.plugins.PluginID; +import eu.stratosphere.nephele.streaming.actions.ConstructStreamChainAction; import eu.stratosphere.nephele.streaming.actions.LimitBufferSizeAction; import eu.stratosphere.nephele.streaming.profiling.LatencyOptimizerThread; import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; @@ -141,6 +145,44 @@ public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) LatencyOptimizerThread optimizerThread = new LatencyOptimizerThread(this, executionGraph); latencyOptimizerThreads.put(jobId, optimizerThread); optimizerThread.start(); + + // Temporary code start + final Runnable run = new Runnable() { + + @Override + public void run() { + + try { + Thread.sleep(30000); + } catch (InterruptedException e) { + e.printStackTrace(); + return; + } + + int count = 0; + final Iterator it = new ExecutionGraphIterator(executionGraph, true); + final List vertexIDs = new ArrayList(); + AbstractInstance instance = null; + while (it.hasNext()) { + ++count; + final ExecutionVertex vertex = it.next(); + + if (count > 1 && count < 6) { + if (instance == null) { + instance = vertex.getAllocatedResource().getInstance(); + } + vertexIDs.add(vertex.getID()); + } + } + + constructStreamChain(executionGraph.getJobID(), instance, vertexIDs); + } + + }; + + new Thread(run).start(); + // Temporary code end + return executionGraph; } @@ -227,4 +269,15 @@ public void limitBufferSize(final ExecutionVertex vertex, final ChannelID source LOG.error(StringUtils.stringifyException(e)); } } + + public void constructStreamChain(final JobID jobID, final AbstractInstance instance, + final List vertexIDs) { + + final ConstructStreamChainAction csca = new ConstructStreamChainAction(jobID, vertexIDs); + try { + instance.sendData(this.pluginID, csca); + } catch (IOException e) { + LOG.error(StringUtils.stringifyException(e)); + } + } } From 7fca4474bb1052bfb3372c1b5ff85d220a9c1188 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Tue, 29 Nov 2011 21:28:29 +0100 Subject: [PATCH 116/310] implemented and tested first sketch of buffer size optimization --- .../streaming/buffers/BufferSizeHistory.java | 55 +++++ .../buffers/BufferSizeHistoryEntry.java | 37 +++ .../streaming/buffers/BufferSizeManager.java | 230 ++++++++++++++++++ .../profiling/EdgeCharacteristics.java | 28 +-- .../profiling/LatencyOptimizerThread.java | 40 +-- .../streaming/profiling/ProfilingLogger.java | 50 ++-- .../streaming/profiling/ProfilingModel.java | 39 +-- .../streaming/profiling/ProfilingPath.java | 57 ++++- .../profiling/ProfilingSubgraph.java | 10 +- .../streaming/profiling/ProfilingSummary.java | 57 ++++- .../streaming/profiling/ProfilingUtils.java | 8 + .../profiling/ProfilingValueStatistic.java | 20 +- .../streaming/profiling/VertexLatency.java | 4 +- 13 files changed, 528 insertions(+), 107 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistory.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistoryEntry.java create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistory.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistory.java new file mode 100644 index 0000000000000..316f31c457d27 --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistory.java @@ -0,0 +1,55 @@ +package eu.stratosphere.nephele.streaming.buffers; + +import eu.stratosphere.nephele.managementgraph.ManagementEdge; + +public class BufferSizeHistory { + + private BufferSizeHistoryEntry[] entries; + + private int entriesInHistory; + + private ManagementEdge edge; + + public BufferSizeHistory(ManagementEdge edge, int noOfHistoryEntries) { + this.edge = edge; + this.entries = new BufferSizeHistoryEntry[noOfHistoryEntries]; + this.entriesInHistory = 0; + } + + public void addToHistory(long timestamp, int newBufferSize) { + BufferSizeHistoryEntry newEntry = new BufferSizeHistoryEntry(Math.min(entriesInHistory, entries.length - 1), + edge, timestamp, newBufferSize); + + if (entriesInHistory < entries.length) { + entries[entriesInHistory] = newEntry; + entriesInHistory++; + } else { + System.arraycopy(entries, 1, entries, 0, entriesInHistory - 1); + entries[entriesInHistory - 1] = newEntry; + } + } + + public BufferSizeHistoryEntry[] getEntries() { + return entries; + } + + public BufferSizeHistoryEntry getFirstEntry() { + return entries[0]; + } + + public BufferSizeHistoryEntry getLastEntry() { + if (entriesInHistory > 0) { + return entries[entriesInHistory - 1]; + } else { + return null; + } + } + + public boolean hasEntries() { + return entriesInHistory > 0; + } + + public int getNumberOfEntries() { + return entriesInHistory; + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistoryEntry.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistoryEntry.java new file mode 100644 index 0000000000000..0a876b3f6b70d --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistoryEntry.java @@ -0,0 +1,37 @@ +package eu.stratosphere.nephele.streaming.buffers; + +import eu.stratosphere.nephele.managementgraph.ManagementEdge; + +public class BufferSizeHistoryEntry { + private int entryIndex; + + private ManagementEdge edge; + + private long timestamp; + + private int bufferSize; + + public BufferSizeHistoryEntry(int entryIndex, ManagementEdge edge, long timestamp, int bufferSize) { + this.entryIndex = entryIndex; + this.edge = edge; + this.timestamp = timestamp; + this.bufferSize = bufferSize; + } + + public int getEntryIndex() { + return entryIndex; + } + + public ManagementEdge getEdge() { + return edge; + } + + public long getTimestamp() { + return timestamp; + } + + public int getBufferSize() { + return bufferSize; + } + +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java new file mode 100644 index 0000000000000..f8d9ebc60320e --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java @@ -0,0 +1,230 @@ +package eu.stratosphere.nephele.streaming.buffers; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.configuration.GlobalConfiguration; +import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.executiongraph.ExecutionVertex; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.managementgraph.ManagementAttachment; +import eu.stratosphere.nephele.managementgraph.ManagementEdge; +import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; +import eu.stratosphere.nephele.streaming.StreamingJobManagerPlugin; +import eu.stratosphere.nephele.streaming.profiling.EdgeCharacteristics; +import eu.stratosphere.nephele.streaming.profiling.ProfilingModel; +import eu.stratosphere.nephele.streaming.profiling.ProfilingPath; +import eu.stratosphere.nephele.streaming.profiling.ProfilingSummary; +import eu.stratosphere.nephele.streaming.profiling.ProfilingUtils; +import eu.stratosphere.nephele.taskmanager.bufferprovider.GlobalBufferPool; + +public class BufferSizeManager { + + private final static long WAIT_BEFORE_FIRST_ADJUSTMENT = 30 * 1000; + + public final static long ADJUSTMENT_INTERVAL = 10 * 1000; + + private Log LOG = LogFactory.getLog(BufferSizeManager.class); + + private long latencyGoal; + + private ProfilingModel profilingModel; + + private StreamingJobManagerPlugin jobManagerPlugin; + + private ExecutionGraph executionGraph; + + private HashMap bufferSizes; + + private long timeOfNextAdjustment; + + public BufferSizeManager(long latencyGoal, ProfilingModel profilingModel, + StreamingJobManagerPlugin jobManagerPlugin, + ExecutionGraph executionGraph) { + this.latencyGoal = latencyGoal; + this.profilingModel = profilingModel; + this.jobManagerPlugin = jobManagerPlugin; + this.executionGraph = executionGraph; + this.bufferSizes = new HashMap(); + this.timeOfNextAdjustment = ProfilingUtils.alignToNextFullSecond(System.currentTimeMillis() + + WAIT_BEFORE_FIRST_ADJUSTMENT); + initBufferSizes(); + } + + private void initBufferSizes() { + int bufferSize = GlobalConfiguration.getInteger("channel.network.bufferSizeInBytes", + GlobalBufferPool.DEFAULT_BUFFER_SIZE_IN_BYTES); + + long now = System.currentTimeMillis(); + for (ProfilingPath path : profilingModel.getProfilingSubgraph().getProfilingPaths()) { + for (ManagementAttachment pathElement : path.getPathElements()) { + if (pathElement instanceof ManagementEdge) { + ManagementEdge edge = (ManagementEdge) pathElement; + BufferSizeHistory bufferSizeHistory = new BufferSizeHistory(edge, 2); + bufferSizeHistory.addToHistory(now, bufferSize); + bufferSizes.put(edge, bufferSizeHistory); + } + } + } + } + + public void adjustBufferSizes(ProfilingSummary summary) { + HashMap edgesToAdjust = new HashMap(); + + for (ProfilingPath activePath : summary.getActivePaths()) { + if (activePath.getSummary().getTotalLatency() > latencyGoal) { + collectEdgesToAdjust(activePath, edgesToAdjust); + } + } + + doAdjust(edgesToAdjust); + + refreshTimeOfNextAdjustment(); + } + + private void doAdjust(HashMap edgesToAdjust) { + + for (ManagementEdge edge : edgesToAdjust.keySet()) { + int newBufferSize = edgesToAdjust.get(edge); + + BufferSizeHistory sizeHistory = bufferSizes.get(edge); + + LOG.info(String.format("New buffer size: %s new: %d (old: %d)", ProfilingUtils.formatName(edge), + newBufferSize, sizeHistory.getLastEntry().getBufferSize())); + + setBufferSize(edge.getSourceEdgeID(), newBufferSize); + + sizeHistory.addToHistory(timeOfNextAdjustment, newBufferSize); + } + } + + private void refreshTimeOfNextAdjustment() { + long now = System.currentTimeMillis(); + while (timeOfNextAdjustment <= now) { + timeOfNextAdjustment += ADJUSTMENT_INTERVAL; + } + } + + ArrayList edgesSortedByLatency = new ArrayList(); + + Comparator edgeComparator = new Comparator() { + @Override + public int compare(ManagementEdge first, ManagementEdge second) { + double firstLatency = ((EdgeCharacteristics) first.getAttachment()).getChannelLatencyInMillis(); + double secondLatency = ((EdgeCharacteristics) second.getAttachment()).getChannelLatencyInMillis(); + + if (firstLatency < secondLatency) { + return -1; + } else if (firstLatency > secondLatency) { + return 1; + } else { + return 0; + } + } + }; + + private void collectEdgesToAdjust(ProfilingPath path, HashMap edgesToAdjust) { + for (ManagementAttachment element : path.getPathElements()) { + if (element instanceof ManagementEdge) { + edgesSortedByLatency.add((ManagementEdge) element); + } + } + + Collections.sort(edgesSortedByLatency, edgeComparator); + + for (ManagementEdge edge : edgesSortedByLatency) { + + if (edgesToAdjust.containsKey(edge)) { + continue; + } + + EdgeCharacteristics edgeChar = (EdgeCharacteristics) edge.getAttachment(); + + if (!hasFreshValues(edge)) { +// LOG.info("Rejecting edge due to stale values: " + ProfilingUtils.formatName(edge)); + continue; + } + + double edgeLatency = edgeChar.getChannelLatencyInMillis(); + double avgOutputBufferLatency = edgeChar.getOutputBufferLatencyInMillis() / 2; + + if (avgOutputBufferLatency > 5 && avgOutputBufferLatency >= 0.05 * edgeLatency) { + reduceBufferSize(edge, edgesToAdjust); + } else if (avgOutputBufferLatency <= 1) { + increaseBufferSize(edge, edgesToAdjust); + } + } + + edgesSortedByLatency.clear(); + } + + private void increaseBufferSize(ManagementEdge edge, HashMap edgesToAdjust) { + int oldBufferSize = bufferSizes.get(edge).getLastEntry().getBufferSize(); + int newBufferSize = proposedIncreasedBufferSize(oldBufferSize); + + edgesToAdjust.put(edge, newBufferSize); + } + + private int proposedIncreasedBufferSize(int oldBufferSize) { + return (int) (oldBufferSize * 1.2); + } + + private void reduceBufferSize(ManagementEdge edge, HashMap edgesToAdjust) { + int oldBufferSize = bufferSizes.get(edge).getLastEntry().getBufferSize(); + int newBufferSize = proposedReducedBufferSize(edge, oldBufferSize); + + // filters pointless minor changes in buffer size + if (isRelevantReduction(newBufferSize, oldBufferSize)) { + edgesToAdjust.put(edge, newBufferSize); + } + +// else { +// LOG.info(String.format("Filtering reduction due to insignificance: %s (old:%d new:%d)", +// ProfilingUtils.formatName(edge), oldBufferSize, newBufferSize)); +// } + } + + private boolean isRelevantReduction(int newBufferSize, int oldBufferSize) { + return newBufferSize < oldBufferSize * 0.98; + } + + private int proposedReducedBufferSize(ManagementEdge edge, int oldBufferSize) { + EdgeCharacteristics edgeChar = (EdgeCharacteristics) edge.getAttachment(); + + double avgOutputBufferLatency = edgeChar.getOutputBufferLatencyInMillis() / 2; + + double reductionFactor = Math.pow(0.99, avgOutputBufferLatency); + reductionFactor = Math.max(0.1, reductionFactor); + + int newBufferSize = (int) Math.max(100, oldBufferSize * reductionFactor); + + return newBufferSize; + } + + private boolean hasFreshValues(ManagementEdge edge) { + EdgeCharacteristics edgeChar = (EdgeCharacteristics) edge.getAttachment(); + long freshnessThreshold = bufferSizes.get(edge).getLastEntry().getTimestamp(); + + return edgeChar.isChannelLatencyFresherThan(freshnessThreshold) + && edgeChar.isOutputBufferLatencyFresherThan(freshnessThreshold); + } + + public boolean isAdjustmentNecessary(long now) { + return now >= timeOfNextAdjustment; + } + + private void setBufferSize(ManagementEdgeID sourceEdgeID, int bufferSize) { + ChannelID sourceChannelID = sourceEdgeID.toChannelID(); + ExecutionVertex vertex = this.executionGraph.getVertexByChannelID(sourceChannelID); + if (vertex == null) { + LOG.error("Cannot find vertex to channel ID " + vertex); + return; + } + this.jobManagerPlugin.limitBufferSize(vertex, sourceChannelID, bufferSize); + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java index c87795b90bf6d..0fa988b28652d 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java @@ -14,9 +14,9 @@ public class EdgeCharacteristics { public EdgeCharacteristics(ManagementEdge edge) { this.edge = edge; - this.latencyInMillisStatistic = new ProfilingValueStatistic(20); - this.throughputInMbitStatistic = new ProfilingValueStatistic(20); - this.outputBufferLatencyStatistic = new ProfilingValueStatistic(20); + this.latencyInMillisStatistic = new ProfilingValueStatistic(10); + this.throughputInMbitStatistic = new ProfilingValueStatistic(10); + this.outputBufferLatencyStatistic = new ProfilingValueStatistic(10); } public ManagementEdge getEdge() { @@ -25,7 +25,7 @@ public ManagementEdge getEdge() { public double getChannelLatencyInMillis() { if (latencyInMillisStatistic.hasValues()) { - return latencyInMillisStatistic.getMedianValue(); + return latencyInMillisStatistic.getArithmeticMean(); } else { return -1; } @@ -33,7 +33,7 @@ public double getChannelLatencyInMillis() { public double getChannelThroughputInMbit() { if (throughputInMbitStatistic.hasValues()) { - return throughputInMbitStatistic.getMedianValue(); + return throughputInMbitStatistic.getArithmeticMean(); } else { return -1; } @@ -41,15 +41,7 @@ public double getChannelThroughputInMbit() { public double getOutputBufferLatencyInMillis() { if (outputBufferLatencyStatistic.hasValues()) { - return outputBufferLatencyStatistic.getMedianValue(); - } else { - return -1; - } - } - - public double getThroughputInMbit() { - if (throughputInMbitStatistic.hasValues()) { - return throughputInMbitStatistic.getMedianValue(); + return outputBufferLatencyStatistic.getArithmeticMean(); } else { return -1; } @@ -69,4 +61,12 @@ public void addOutputBufferLatencyMeasurement(long timestamp, double latencyInMi ProfilingValue value = new ProfilingValue(latencyInMillis, timestamp); this.outputBufferLatencyStatistic.addValue(value); } + + public boolean isChannelLatencyFresherThan(long freshnessThreshold) { + return latencyInMillisStatistic.getOldestValue().getTimestamp() >= freshnessThreshold; + } + + public boolean isOutputBufferLatencyFresherThan(long freshnessThreshold) { + return outputBufferLatencyStatistic.getOldestValue().getTimestamp() >= freshnessThreshold; + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java index 999b1eed3e414..9b3d1cf79f37d 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java @@ -1,15 +1,14 @@ package eu.stratosphere.nephele.streaming.profiling; +import java.io.IOException; import java.util.concurrent.LinkedBlockingQueue; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; import eu.stratosphere.nephele.streaming.StreamingJobManagerPlugin; +import eu.stratosphere.nephele.streaming.buffers.BufferSizeManager; import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; import eu.stratosphere.nephele.streaming.types.ChannelLatency; import eu.stratosphere.nephele.streaming.types.ChannelThroughput; @@ -28,11 +27,23 @@ public class LatencyOptimizerThread extends Thread { private final ProfilingModel profilingModel; + private ProfilingLogger logger; + + private BufferSizeManager bufferSizeManager; + public LatencyOptimizerThread(StreamingJobManagerPlugin jobManagerPlugin, ExecutionGraph executionGraph) { this.jobManagerPlugin = jobManagerPlugin; this.executionGraph = executionGraph; this.profilingModel = new ProfilingModel(executionGraph); this.streamingDataQueue = new LinkedBlockingQueue(); + try { + this.logger = new ProfilingLogger(); + } catch (IOException e) { + LOG.error("Error when opening profiling logger file", e); + } + + this.bufferSizeManager = new BufferSizeManager(200, this.profilingModel, this.jobManagerPlugin, + this.executionGraph); } public void run() { @@ -53,7 +64,18 @@ public void run() { profilingModel.refreshChannelOutputBufferLatency(now, (OutputBufferLatency) streamingData); } - profilingModel.logProfilingSummaryIfNecessary(now); + if (this.logger.isLoggingNecessary(now)) { + ProfilingSummary summary = profilingModel.computeProfilingSummary(); + try { + logger.logLatencies(summary); + } catch (IOException e) { + LOG.error("Error when writing to profiling logger file", e); + } + + if (bufferSizeManager.isAdjustmentNecessary(now)) { + bufferSizeManager.adjustBufferSizes(summary); + } + } } } catch (InterruptedException e) { @@ -66,14 +88,4 @@ public void handOffStreamingData(AbstractStreamingData data) { streamingDataQueue.add(data); } - public void limitBufferSize(ManagementEdgeID sourceEdgeID, int bufferSize) { - final ChannelID sourceChannelID = sourceEdgeID.toChannelID(); - final ExecutionVertex vertex = this.executionGraph.getVertexByChannelID(sourceChannelID); - if (vertex == null) { - LOG.error("Cannot find vertex to channel ID " + vertex); - return; - } - - this.jobManagerPlugin.limitBufferSize(vertex, sourceChannelID, bufferSize); - } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java index 341cf2f870ab5..d360ada7611a0 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java @@ -9,22 +9,35 @@ public class ProfilingLogger { - private ProfilingSubgraph subgraph; + private final static long WAIT_BEFORE_FIRST_LOGGING = 10 * 1000; + + private final static long LOGGING_INTERVAL = 1000; private BufferedWriter writer; private boolean headersWritten; - public ProfilingLogger(ProfilingSubgraph subgraph) + private long timeOfNextLogging; + + private long timeBase; + + public ProfilingLogger() throws IOException { - this.subgraph = subgraph; this.writer = new BufferedWriter(new FileWriter("profiling.txt")); this.headersWritten = false; + this.timeOfNextLogging = ProfilingUtils.alignToNextFullSecond(System.currentTimeMillis() + WAIT_BEFORE_FIRST_LOGGING); + this.timeBase = timeOfNextLogging; + } + + public boolean isLoggingNecessary(long now) { + return now >= timeOfNextLogging; } - public void logLatencies(long timestamp) throws IOException { - ProfilingSummary summary = new ProfilingSummary(subgraph); + public void logLatencies(ProfilingSummary summary) throws IOException { + long now = System.currentTimeMillis(); + long timestamp = now - timeBase; + if (!headersWritten) { writeHeaders(summary); } @@ -32,25 +45,34 @@ public void logLatencies(long timestamp) throws IOException { StringBuilder builder = new StringBuilder(); builder.append(timestamp); builder.append(';'); - builder.append(summary.noOfActivePaths); + builder.append(summary.getNoOfActivePaths()); builder.append(';'); - builder.append(summary.noOfInactivePaths); + builder.append(summary.getNoOfInactivePaths()); builder.append(';'); - builder.append(summary.avgTotalPathLatency); + builder.append(summary.getAvgTotalPathLatency()); builder.append(';'); - builder.append(summary.medianPathLatency); + builder.append(summary.getMedianPathLatency()); builder.append(';'); - builder.append(summary.minPathLatency); + builder.append(summary.getMinPathLatency()); builder.append(';'); - builder.append(summary.maxPathLatency); + builder.append(summary.getMaxPathLatency()); - for (double avgElementLatency : summary.avgPathElementLatencies) { + for (double avgElementLatency : summary.getAvgPathElementLatencies()) { builder.append(';'); builder.append(avgElementLatency); } builder.append('\n'); writer.write(builder.toString()); - writer.flush(); //FIXME + writer.flush(); // FIXME + + refreshTimeOfNextLogging(); + } + + private void refreshTimeOfNextLogging() { + long now = System.currentTimeMillis(); + while(timeOfNextLogging <= now) { + timeOfNextLogging += LOGGING_INTERVAL; + } } private void writeHeaders(ProfilingSummary summary) throws IOException { @@ -64,7 +86,7 @@ private void writeHeaders(ProfilingSummary summary) throws IOException { builder.append("maxPathLatency"); int nextEdgeIndex = 1; - for (ManagementAttachment element : summary.pathElements) { + for (ManagementAttachment element : summary.getPathElements()) { builder.append(';'); if (element instanceof ManagementVertex) { ManagementVertex vertex = (ManagementVertex) element; diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java index 0d4832e2c848e..01ce8627965ea 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java @@ -1,10 +1,5 @@ package eu.stratosphere.nephele.streaming.profiling; -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; @@ -16,22 +11,12 @@ public class ProfilingModel { - private static Log LOG = LogFactory.getLog(ProfilingModel.class); - - private final static long WAIT_INTERVAL_BEFORE_LOGGING = 10 * 1000; - - private final static long LOGGING_INTERVAL = 1000; +// private static Log LOG = LogFactory.getLog(ProfilingModel.class); private ExecutionGraph executionGraph; private ProfilingSubgraph profilingSubgraph; - private ProfilingLogger logger; - - private long timeOfLastLogging; - - private long timeBase; - public ProfilingModel(ExecutionGraph executionGraph) { this.executionGraph = executionGraph; @@ -41,14 +26,6 @@ public ProfilingModel(ExecutionGraph executionGraph) { ExecutionGroupVertex subgraphEnd = this.executionGraph.getOutputVertex(0).getGroupVertex(); this.profilingSubgraph = new ProfilingSubgraph(executionGraph, subgraphStart, subgraphEnd, false, false); - - try { - this.logger = new ProfilingLogger(profilingSubgraph); - } catch (IOException e) { - LOG.error("Error when opening profiling logger file", e); - } - this.timeOfLastLogging = System.currentTimeMillis() + WAIT_INTERVAL_BEFORE_LOGGING; - this.timeBase = timeOfLastLogging; } public void refreshEdgeLatency(long timestamp, ChannelLatency channelLatency) { @@ -109,15 +86,11 @@ public void refreshChannelOutputBufferLatency(long timestamp, OutputBufferLatenc edgeCharaceristics.addOutputBufferLatencyMeasurement(timestamp, latency.getBufferLatency()); } - public void logProfilingSummaryIfNecessary(long now) { - if ((now - timeOfLastLogging) >= LOGGING_INTERVAL) { - try { - logger.logLatencies(now - timeBase); - } catch (IOException e) { - LOG.error("Error when writing to profiling logger file", e); - } - timeOfLastLogging = now; - } + public ProfilingSummary computeProfilingSummary() { + return new ProfilingSummary(profilingSubgraph); } + public ProfilingSubgraph getProfilingSubgraph() { + return profilingSubgraph; + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPath.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPath.java index 9f44d8dd3a12c..d46ffab302cb0 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPath.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPath.java @@ -21,11 +21,13 @@ */ public class ProfilingPath implements Iterable { + private ProfilingSubgraph graph; + private LinkedList pathVertices; private HashMap ingoingEdges; - private ProfilingSubgraph graph; + private ArrayList pathElements; private ProfilingPathSummary summary; @@ -42,13 +44,14 @@ public ProfilingPath(ProfilingPath toClone) { this.endVertexInProfilingPath = toClone.endVertexInProfilingPath; } - public ProfilingPath(ProfilingSubgraph graph, ManagementVertex firstVertex) { + public ProfilingPath(ProfilingSubgraph graph, ManagementVertex firstVertex, boolean beginVertexInProfilingPath, + boolean endVertexInProfilingPath) { this.graph = graph; this.pathVertices = new LinkedList(); this.ingoingEdges = new HashMap(); this.pathVertices.add(firstVertex); - this.beginVertexInProfilingPath = true; - this.endVertexInProfilingPath = true; + this.beginVertexInProfilingPath = beginVertexInProfilingPath; + this.endVertexInProfilingPath = endVertexInProfilingPath; } public void appendVertex(ManagementVertex vertex, ManagementEdge ingoingEdge) { @@ -65,6 +68,13 @@ public ManagementVertex getEndVertex() { } public void setBeginVertexInProfilingPath(boolean beginVertexInProfilingPath) { + // changing this on the fly will invalidate any already computed list of path elements + // and the summary + if (beginVertexInProfilingPath != this.beginVertexInProfilingPath) { + this.pathElements = null; + this.summary = null; + } + this.beginVertexInProfilingPath = beginVertexInProfilingPath; } @@ -73,6 +83,13 @@ public boolean isBeginVertexOnProfilingPath() { } public void setEndVertexInProfilingPath(boolean endVertexInProfilingPath) { + // changing this on the fly will invalidate any already computed list of path elements + // and the summary + if (endVertexInProfilingPath != this.endVertexInProfilingPath) { + this.pathElements = null; + this.summary = null; + } + this.endVertexInProfilingPath = endVertexInProfilingPath; } @@ -95,33 +112,44 @@ public Iterator iterator() { } public ProfilingPathSummary getSummary() { + ensurePathSummaryInitialized(); + return this.summary; + } + + private void ensurePathSummaryInitialized() { if (this.summary == null) { - this.summary = new ProfilingPathSummary(walkProfilingPath()); + ensurePathElementsInitialized(); + this.summary = new ProfilingPathSummary(this.pathElements); + } + } + + private void ensurePathElementsInitialized() { + if (this.pathElements == null) { + this.pathElements = walkProfilingPath(); } - return this.summary; } private ArrayList walkProfilingPath() { - ArrayList profilingPathElements = new ArrayList(); + ArrayList pathElements = new ArrayList(); for (ManagementVertex vertex : pathVertices) { ManagementEdge ingoingEdge = ingoingEdges.get(vertex.getID()); if (ingoingEdge != null) { - profilingPathElements.add(ingoingEdge); + pathElements.add(ingoingEdge); } - profilingPathElements.add(vertex); + pathElements.add(vertex); } if (!isBeginVertexOnProfilingPath()) { - profilingPathElements.remove(0); + pathElements.remove(0); } if (!isEndVertexOnProfilingPath()) { - profilingPathElements.remove(profilingPathElements.size() - 1); + pathElements.remove(pathElements.size() - 1); } - return profilingPathElements; + return pathElements; } @Override @@ -141,6 +169,11 @@ public String toString() { return builder.toString(); } + public ArrayList getPathElements() { + ensurePathElementsInitialized(); + return this.pathElements; + } + // public void dumpLatencies() { // // for (ManagementVertex vertex : pathVertices) { diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java index ae156ee766cd7..698346cc4dae3 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java @@ -95,7 +95,7 @@ private void initReceiverVertexToSourceEdgeIDMap(final ManagementGraph managemen final ManagementVertex receiver = edge.getTarget().getVertex(); XoredVertexID xored = new XoredVertexID(source.getID(), receiver.getID()); -// System.out.println("putting edge " + ProfilingUtils.formatName(edge) + " " + xored.toString()); + // System.out.println("putting edge " + ProfilingUtils.formatName(edge) + " " + xored.toString()); this.xoredVertexToSourceEdgeIDMap.put(xored, edge.getSourceEdgeID()); } } @@ -109,14 +109,10 @@ private void buildProfilingPaths(boolean includeSubgraphStartInProfilingPaths, for (int i = 0; i < subgraphStart.getNumberOfGroupMembers(); i++) { ManagementVertex vertex = subgraphStart.getGroupMember(i); - ProfilingPath initialPath = new ProfilingPath(this, vertex); + ProfilingPath initialPath = new ProfilingPath(this, vertex, includeSubgraphStartInProfilingPaths, + includeSubgraphEndInProfilingPaths); depthFirstSearchProfilingPaths(initialPath, this.profilingPaths); } - - for (ProfilingPath profilingPath : profilingPaths) { - profilingPath.setBeginVertexInProfilingPath(includeSubgraphStartInProfilingPaths); - profilingPath.setEndVertexInProfilingPath(includeSubgraphEndInProfilingPaths); - } } /** diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java index ea1ab7147181a..91c664124e53c 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java @@ -7,21 +7,23 @@ public class ProfilingSummary { - ArrayList pathElements; + private ArrayList pathElements; - int noOfActivePaths; + private int noOfActivePaths; - int noOfInactivePaths; + private int noOfInactivePaths; - double avgTotalPathLatency; + private double avgTotalPathLatency; - double medianPathLatency; + private double medianPathLatency; - double minPathLatency; + private double minPathLatency; - double maxPathLatency; + private double maxPathLatency; - double[] avgPathElementLatencies; + private double[] avgPathElementLatencies; + + private ArrayList activePaths; public ProfilingSummary(ProfilingSubgraph profilingSubgraph) { noOfActivePaths = 0; @@ -30,6 +32,7 @@ public ProfilingSummary(ProfilingSubgraph profilingSubgraph) { minPathLatency = 0; maxPathLatency = 0; medianPathLatency = 0; + activePaths = new ArrayList(); pathElements = null; avgPathElementLatencies = null; @@ -46,6 +49,8 @@ public ProfilingSummary(ProfilingSubgraph profilingSubgraph) { } if (pathSummary.hasLatencies()) { + activePaths.add(path); + // refresh the latency values in the summary pathSummary.refreshLatencies(); @@ -80,4 +85,40 @@ private void addValues(double[] from, double[] to) { to[i] += from[i]; } } + + public ArrayList getPathElements() { + return pathElements; + } + + public int getNoOfActivePaths() { + return noOfActivePaths; + } + + public int getNoOfInactivePaths() { + return noOfInactivePaths; + } + + public double getAvgTotalPathLatency() { + return avgTotalPathLatency; + } + + public double getMedianPathLatency() { + return medianPathLatency; + } + + public double getMinPathLatency() { + return minPathLatency; + } + + public double getMaxPathLatency() { + return maxPathLatency; + } + + public double[] getAvgPathElementLatencies() { + return avgPathElementLatencies; + } + + public ArrayList getActivePaths() { + return activePaths; + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java index f20e7ea28ffec..7d93830decf75 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java @@ -41,4 +41,12 @@ public static String formatName(ExecutionVertex vertex) { return name; } + public static long alignToNextFullSecond(long timestampInMillis) { + long remainder = timestampInMillis % 1000; + + if (remainder > 0) { + return timestampInMillis - remainder + 1000; + } + return timestampInMillis; + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java index 9ff89a17c285c..dcd2a5fba4539 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java @@ -23,7 +23,7 @@ public ProfilingValueStatistic(int statisticWindowSize) { this.noOfStoredValues = 0; this.sumOfValues = 0; } - + public void addValue(ProfilingValue value) { ProfilingValue droppedValue = insertIntoSortedByTimestamp(value); @@ -56,7 +56,7 @@ protected void insertIntoSortedByValue(ProfilingValue value) { if (insertionIndex < 0) { insertionIndex = -(insertionIndex + 1); } - + sortedByValue.add(insertionIndex, value); } @@ -91,6 +91,20 @@ public double getMinValue() { return sortedByValue.get(0).getValue(); } + public ProfilingValue getOldestValue() { + if (noOfStoredValues == 0) { + throw new RuntimeException("Cannot get the oldest value of empty value set"); + } + return sortedById.getFirst(); + } + + public ProfilingValue getNewestValue() { + if (noOfStoredValues == 0) { + throw new RuntimeException("Cannot get the newest value of empty value set"); + } + return sortedById.getLast(); + } + public double getArithmeticMean() { if (noOfStoredValues == 0) { throw new RuntimeException("Cannot calculate the arithmetic mean of empty value set"); @@ -98,7 +112,7 @@ public double getArithmeticMean() { return sumOfValues / noOfStoredValues; } - + public boolean hasValues() { return noOfStoredValues > 0; } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/VertexLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/VertexLatency.java index 16d41d1c072bf..53b2e8b8941e6 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/VertexLatency.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/VertexLatency.java @@ -10,7 +10,7 @@ public class VertexLatency { public VertexLatency(ManagementVertex vertex) { this.vertex = vertex; - this.latencyStatistics = new ProfilingValueStatistic(20); + this.latencyStatistics = new ProfilingValueStatistic(10); } public ManagementVertex getVertex() { @@ -19,7 +19,7 @@ public ManagementVertex getVertex() { public double getLatencyInMillis() { if (latencyStatistics.hasValues()) { - return latencyStatistics.getMedianValue(); + return latencyStatistics.getArithmeticMean(); } else { return -1; } From 3aeafb570983c98edc1d3bf8f99c83d26bcb588a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Lohrmann?= Date: Tue, 29 Nov 2011 21:59:27 +0100 Subject: [PATCH 117/310] fix for excess buffer size increase bug --- .../streaming/buffers/BufferSizeManager.java | 26 +++++++++++++------ 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java index f8d9ebc60320e..93189ecb558ec 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java @@ -43,6 +43,8 @@ public class BufferSizeManager { private long timeOfNextAdjustment; + private int maximumBufferSize; + public BufferSizeManager(long latencyGoal, ProfilingModel profilingModel, StreamingJobManagerPlugin jobManagerPlugin, ExecutionGraph executionGraph) { @@ -60,6 +62,8 @@ private void initBufferSizes() { int bufferSize = GlobalConfiguration.getInteger("channel.network.bufferSizeInBytes", GlobalBufferPool.DEFAULT_BUFFER_SIZE_IN_BYTES); + this.maximumBufferSize = bufferSize; + long now = System.currentTimeMillis(); for (ProfilingPath path : profilingModel.getProfilingSubgraph().getProfilingPaths()) { for (ManagementAttachment pathElement : path.getPathElements()) { @@ -146,7 +150,7 @@ private void collectEdgesToAdjust(ProfilingPath path, HashMap edgesToAdjust) { int oldBufferSize = bufferSizes.get(edge).getLastEntry().getBufferSize(); - int newBufferSize = proposedIncreasedBufferSize(oldBufferSize); + int newBufferSize = Math.min(proposedIncreasedBufferSize(oldBufferSize), this.maximumBufferSize); - edgesToAdjust.put(edge, newBufferSize); + if (isRelevantIncrease(oldBufferSize, newBufferSize)) { + edgesToAdjust.put(edge, newBufferSize); + } + } + + private boolean isRelevantIncrease(int oldBufferSize, int newBufferSize) { + return newBufferSize >= oldBufferSize + 100; } private int proposedIncreasedBufferSize(int oldBufferSize) { @@ -182,11 +192,11 @@ private void reduceBufferSize(ManagementEdge edge, HashMap Date: Tue, 29 Nov 2011 22:33:41 +0100 Subject: [PATCH 118/310] Reduced NIO timed waiting interval to 10 ms --- .../taskmanager/bytebuffered/OutgoingConnectionThread.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java index e315745598ce7..3887fd2ba4298 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java @@ -144,7 +144,7 @@ public void run() { } try { - this.selector.select(50); + this.selector.select(10); } catch (IOException e) { LOG.error(e); } From f078fa3e95ea88556791f08907481f64ebd8a20a Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 30 Nov 2011 13:26:45 +0100 Subject: [PATCH 119/310] Extended job manager plugin interface to expose possible profiling dependency --- .../nephele/plugins/JobManagerPlugin.java | 7 ++++ .../nephele/profiling/JobManagerProfiler.java | 2 +- .../streaming/StreamingJobManagerPlugin.java | 36 +++++++++++-------- .../score/ScoreJobManagerPlugin.java | 9 +++++ 4 files changed, 39 insertions(+), 15 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java index a6e1c63876a9f..9fa865b34718a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java @@ -25,6 +25,13 @@ */ public interface JobManagerPlugin extends PluginCommunication { + /** + * Checks whether the plugin requires a job to be executed with profiling enabled in order to work properly. + * + * @return true if the job requires profiling to be enabled for a job, false otherwise + */ + boolean requiresProfiling(); + /** * This method is called upon the reception of a new job graph. It gives the plugin the possibility to to rewrite * the job graph before it is processed further. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/JobManagerProfiler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/JobManagerProfiler.java index e956a02416739..3fc6973424ae9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/JobManagerProfiler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/JobManagerProfiler.java @@ -34,7 +34,7 @@ public interface JobManagerProfiler { */ void registerProfilingJob(ExecutionGraph executionGraph); -/** + /** * Unregisters the given {@link ExecutionGraph} from profiling. Calling this * method will also unregister all of the job's registered listeners. * diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 2d4364df878e0..434a908f19786 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -147,7 +147,7 @@ public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) optimizerThread.start(); // Temporary code start - final Runnable run = new Runnable() { + /*final Runnable run = new Runnable() { @Override public void run() { @@ -158,29 +158,28 @@ public void run() { e.printStackTrace(); return; } - - int count = 0; final Iterator it = new ExecutionGraphIterator(executionGraph, true); - final List vertexIDs = new ArrayList(); - AbstractInstance instance = null; + while (it.hasNext()) { - ++count; + final ExecutionVertex vertex = it.next(); + if (vertex.getName().contains("Decoder")) { + + final List vertexIDs = new ArrayList(); + final AbstractInstance instance = vertex.getAllocatedResource().getInstance(); - if (count > 1 && count < 6) { - if (instance == null) { - instance = vertex.getAllocatedResource().getInstance(); - } vertexIDs.add(vertex.getID()); + vertexIDs.add(it.next().getID()); + vertexIDs.add(it.next().getID()); + vertexIDs.add(it.next().getID()); + + constructStreamChain(executionGraph.getJobID(), instance, vertexIDs); } } - - constructStreamChain(executionGraph.getJobID(), instance, vertexIDs); } - }; - new Thread(run).start(); + new Thread(run).start();*/ // Temporary code end return executionGraph; @@ -280,4 +279,13 @@ public void constructStreamChain(final JobID jobID, final AbstractInstance insta LOG.error(StringUtils.stringifyException(e)); } } + + /** + * {@inheritDoc} + */ + @Override + public boolean requiresProfiling() { + + return true; + } } diff --git a/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java b/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java index f6a65204f6b8d..9b11e9950c42c 100644 --- a/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java +++ b/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java @@ -84,4 +84,13 @@ public IOReadableWritable requestData(final IOReadableWritable data) throws IOEx // TODO Auto-generated method stub return null; } + + /** + * {@inheritDoc} + */ + @Override + public boolean requiresProfiling() { + + return false; + } } From 3079401372804d37f0957ab97f9e46c20bd8e667 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 30 Nov 2011 13:38:29 +0100 Subject: [PATCH 120/310] JobManager now considers profiling dependencies of plugins --- .../nephele/jobmanager/JobManager.java | 26 +++++++++++++------ 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 2e21fe4c3beaf..044280641ed70 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -465,11 +465,22 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { LOG.debug("The dependency chain for instance sharing is acyclic"); + // Check if the job will be executed with profiling enabled + boolean jobRunsWithProfiling = false; + if (this.profiler != null && job.getJobConfiguration().getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) { + jobRunsWithProfiling = true; + } + // Allow plugins to rewrite the job graph Iterator it = this.jobManagerPlugins.values().iterator(); while (it.hasNext()) { final JobManagerPlugin plugin = it.next(); + if (plugin.requiresProfiling() && !jobRunsWithProfiling) { + LOG.debug("Skipping job graph rewrite by plugin " + plugin + " because job " + job.getJobID() + + " will not be executed with profiling"); + continue; + } final JobGraph inputJob = job; job = plugin.rewriteJobGraph(inputJob); @@ -498,6 +509,11 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { while (it.hasNext()) { final JobManagerPlugin plugin = it.next(); + if (plugin.requiresProfiling() && !jobRunsWithProfiling) { + LOG.debug("Skipping execution graph rewrite by plugin " + plugin + " because job " + job.getJobID() + + " will not be executed with profiling"); + continue; + } final ExecutionGraph inputGraph = eg; eg = plugin.rewriteExecutionGraph(inputGraph); @@ -510,19 +526,13 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { } } - // Check if profiling should be enabled for this job - boolean profilingEnabled = false; - if (this.profiler != null && job.getJobConfiguration().getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) { - profilingEnabled = true; - } - // Register job with the progress collector if (this.eventCollector != null) { - this.eventCollector.registerJob(eg, profilingEnabled); + this.eventCollector.registerJob(eg, jobRunsWithProfiling); } // Check if profiling should be enabled for this job - if (profilingEnabled) { + if (jobRunsWithProfiling) { this.profiler.registerProfilingJob(eg); if (this.eventCollector != null) { From 99d9948a1c46ce8f1f57dec3f3e94ca7bac3289c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 30 Nov 2011 14:01:38 +0100 Subject: [PATCH 121/310] Plugins can now return custom profiling listeners --- .../nephele/jobmanager/JobManager.java | 11 +++ .../nephele/plugins/JobManagerPlugin.java | 14 ++++ .../streaming/StreamingJobManagerPlugin.java | 72 ++++++++++--------- .../score/ScoreJobManagerPlugin.java | 11 +++ 4 files changed, 75 insertions(+), 33 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 044280641ed70..c34e3e0e12bd7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -110,6 +110,7 @@ import eu.stratosphere.nephele.plugins.PluginID; import eu.stratosphere.nephele.plugins.PluginManager; import eu.stratosphere.nephele.profiling.JobManagerProfiler; +import eu.stratosphere.nephele.profiling.ProfilingListener; import eu.stratosphere.nephele.profiling.ProfilingUtils; import eu.stratosphere.nephele.protocols.ChannelLookupProtocol; import eu.stratosphere.nephele.protocols.ExtendedManagementProtocol; @@ -538,6 +539,16 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { if (this.eventCollector != null) { this.profiler.registerForProfilingData(eg.getJobID(), this.eventCollector); } + + // Allow plugins to register their own profiling listeners for the job + it = this.jobManagerPlugins.values().iterator(); + while (it.hasNext()) { + + final ProfilingListener listener = it.next().getProfilingListener(eg.getJobID()); + if (listener != null) { + this.profiler.registerForProfilingData(eg.getJobID(), listener); + } + } } // Register job with the dynamic input split assigner diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java index 9fa865b34718a..20c78f64d6dad 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/JobManagerPlugin.java @@ -17,6 +17,8 @@ import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.jobgraph.JobGraph; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.profiling.ProfilingListener; /** * This interface must be implemented by every plugin component which is supposed to run inside Nephele's job manager. @@ -53,6 +55,18 @@ public interface JobManagerPlugin extends PluginCommunication { */ ExecutionGraph rewriteExecutionGraph(ExecutionGraph executionGraph); + /** + * This method is called before the deployment of the execution graph. It provides the plugin with the possibility + * to return a custom {@link ProfilingListener} which is then registered with the profiling component. As a result, + * the plugin will receive profiling events during the job execution. + * + * @param jobID + * the ID of the job to return a profiling listener for + * @return the profiling listener for the job or null if the plugin does not want to receive profiling + * data for the job + */ + ProfilingListener getProfilingListener(JobID jobID); + /** * Called by the job manager to indicate that Nephele is about to shut down. */ diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 434a908f19786..881b6d1351715 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -45,6 +45,7 @@ import eu.stratosphere.nephele.jobgraph.JobOutputVertex; import eu.stratosphere.nephele.plugins.JobManagerPlugin; import eu.stratosphere.nephele.plugins.PluginID; +import eu.stratosphere.nephele.profiling.ProfilingListener; import eu.stratosphere.nephele.streaming.actions.ConstructStreamChainAction; import eu.stratosphere.nephele.streaming.actions.LimitBufferSizeAction; import eu.stratosphere.nephele.streaming.profiling.LatencyOptimizerThread; @@ -147,39 +148,33 @@ public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) optimizerThread.start(); // Temporary code start - /*final Runnable run = new Runnable() { - - @Override - public void run() { - - try { - Thread.sleep(30000); - } catch (InterruptedException e) { - e.printStackTrace(); - return; - } - final Iterator it = new ExecutionGraphIterator(executionGraph, true); - - while (it.hasNext()) { - - final ExecutionVertex vertex = it.next(); - if (vertex.getName().contains("Decoder")) { - - final List vertexIDs = new ArrayList(); - final AbstractInstance instance = vertex.getAllocatedResource().getInstance(); - - vertexIDs.add(vertex.getID()); - vertexIDs.add(it.next().getID()); - vertexIDs.add(it.next().getID()); - vertexIDs.add(it.next().getID()); - - constructStreamChain(executionGraph.getJobID(), instance, vertexIDs); - } - } - } - }; - - new Thread(run).start();*/ + /* + * final Runnable run = new Runnable() { + * @Override + * public void run() { + * try { + * Thread.sleep(30000); + * } catch (InterruptedException e) { + * e.printStackTrace(); + * return; + * } + * final Iterator it = new ExecutionGraphIterator(executionGraph, true); + * while (it.hasNext()) { + * final ExecutionVertex vertex = it.next(); + * if (vertex.getName().contains("Decoder")) { + * final List vertexIDs = new ArrayList(); + * final AbstractInstance instance = vertex.getAllocatedResource().getInstance(); + * vertexIDs.add(vertex.getID()); + * vertexIDs.add(it.next().getID()); + * vertexIDs.add(it.next().getID()); + * vertexIDs.add(it.next().getID()); + * constructStreamChain(executionGraph.getJobID(), instance, vertexIDs); + * } + * } + * } + * }; + * new Thread(run).start(); + */ // Temporary code end return executionGraph; @@ -288,4 +283,15 @@ public boolean requiresProfiling() { return true; } + + /** + * {@inheritDoc} + */ + @Override + public ProfilingListener getProfilingListener(final JobID jobID) { + + System.out.println("REGISTERED PROFILING LISTENER"); + + return null; + } } diff --git a/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java b/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java index 9b11e9950c42c..ab71db6e672ff 100644 --- a/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java +++ b/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java @@ -23,7 +23,9 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.jobgraph.JobGraph; +import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.plugins.JobManagerPlugin; +import eu.stratosphere.nephele.profiling.ProfilingListener; public final class ScoreJobManagerPlugin implements JobManagerPlugin { @@ -93,4 +95,13 @@ public boolean requiresProfiling() { return false; } + + /** + * {@inheritDoc} + */ + @Override + public ProfilingListener getProfilingListener(final JobID jobID) { + + return null; + } } From b272261a75438cccb1cecfc81a1d7a555f2b38c1 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 6 Dec 2011 13:54:32 +0100 Subject: [PATCH 122/310] Records are now copied among different mappers --- .../nephele/execution/Mapper.java | 6 +- .../streaming/StreamingJobManagerPlugin.java | 56 ++++++++-------- .../streaming/chaining/RecordUtils.java | 66 +++++++++++++++++++ .../streaming/chaining/StreamChain.java | 22 ++++--- 4 files changed, 109 insertions(+), 41 deletions(-) create mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/RecordUtils.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java index bc1e33df4ec5d..fa602362e0079 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java @@ -6,7 +6,9 @@ public interface Mapper { - void map(I input, Queue output) throws Exception; + void map(I input) throws Exception; + + Queue getOutputCollector(); - void close(Queue output); + void close(); } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java index 881b6d1351715..95ee71be8e0fc 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java @@ -148,33 +148,31 @@ public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) optimizerThread.start(); // Temporary code start - /* - * final Runnable run = new Runnable() { - * @Override - * public void run() { - * try { - * Thread.sleep(30000); - * } catch (InterruptedException e) { - * e.printStackTrace(); - * return; - * } - * final Iterator it = new ExecutionGraphIterator(executionGraph, true); - * while (it.hasNext()) { - * final ExecutionVertex vertex = it.next(); - * if (vertex.getName().contains("Decoder")) { - * final List vertexIDs = new ArrayList(); - * final AbstractInstance instance = vertex.getAllocatedResource().getInstance(); - * vertexIDs.add(vertex.getID()); - * vertexIDs.add(it.next().getID()); - * vertexIDs.add(it.next().getID()); - * vertexIDs.add(it.next().getID()); - * constructStreamChain(executionGraph.getJobID(), instance, vertexIDs); - * } - * } - * } - * }; - * new Thread(run).start(); - */ + final Runnable run = new Runnable() { + @Override + public void run() { + try { + Thread.sleep(2000); + } catch (InterruptedException e) { + e.printStackTrace(); + return; + } + final Iterator it = new ExecutionGraphIterator(executionGraph, true); + while (it.hasNext()) { + final ExecutionVertex vertex = it.next(); + if (vertex.getName().contains("Decoder")) { + final List vertexIDs = new ArrayList(); + final AbstractInstance instance = vertex.getAllocatedResource().getInstance(); + vertexIDs.add(vertex.getID()); + vertexIDs.add(it.next().getID()); + vertexIDs.add(it.next().getID()); + vertexIDs.add(it.next().getID()); + constructStreamChain(executionGraph.getJobID(), instance, vertexIDs); + } + } + } + }; + //new Thread(run).start(); // Temporary code end return executionGraph; @@ -289,9 +287,9 @@ public boolean requiresProfiling() { */ @Override public ProfilingListener getProfilingListener(final JobID jobID) { - + System.out.println("REGISTERED PROFILING LISTENER"); - + return null; } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/RecordUtils.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/RecordUtils.java new file mode 100644 index 0000000000000..af62d423b687f --- /dev/null +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/RecordUtils.java @@ -0,0 +1,66 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.streaming.chaining; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +import eu.stratosphere.nephele.types.Record; +import eu.stratosphere.nephele.util.StringUtils; + +public final class RecordUtils { + + /** + * Private constructor to prevent instantiation. + */ + private RecordUtils() { + } + + /** + * Creates a copy of the given {@link Record} object by an in-memory serialization and subsequent + * deserialization. + * + * @param original + * the original object to be copied + * @return the copy of original object created by the original object's serialization/deserialization methods + * @throws IOException + * thrown if an error occurs while copying the object + */ + public static Record createCopy(final Record original) throws IOException { + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputStream dos = new DataOutputStream(baos); + + original.write(dos); + + Record copy; + try { + copy = original.getClass().newInstance(); + } catch (Exception e) { + throw new IOException(StringUtils.stringifyException(e)); + } + + final ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); + final DataInputStream dis = new DataInputStream(bais); + + copy.read(dis); + + return copy; + } +} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java index 8484dd492d195..529f4b1467343 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java @@ -16,10 +16,10 @@ package eu.stratosphere.nephele.streaming.chaining; import java.io.IOException; -import java.util.ArrayDeque; import java.util.List; import java.util.Queue; +import eu.stratosphere.nephele.execution.Mapper; import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.util.StringUtils; @@ -58,26 +58,28 @@ public void writeRecord(final Record record) throws IOException, InterruptedExce void executeMapper(final Record record, final int chainIndex) throws Exception { final StreamChainLink chainLink = this.chainLinks.get(chainIndex); - final Queue output = new ArrayDeque(); + final Mapper mapper = chainLink.getMapper(); - chainLink.getInputGate().reportRecordReceived(record); - chainLink.getMapper().map(record, output); + mapper.map(record); final StreamingOutputGate outputGate = chainLink.getOutputGate(); - + + final Queue outputCollector = mapper.getOutputCollector(); + if (chainIndex == this.chainLinks.size() - 1) { - while (!output.isEmpty()) { - outputGate.writeRecord((Record) output.poll()); + while (!outputCollector.isEmpty()) { + + outputGate.writeRecord((Record)outputCollector.poll()); } } else { - while (!output.isEmpty()) { - final Record outputRecord = (Record) output.poll(); + while (!outputCollector.isEmpty()) { + final Record outputRecord = (Record) outputCollector.poll(); outputGate.reportRecordEmitted(outputRecord); - executeMapper(outputRecord, chainIndex + 1); + executeMapper(RecordUtils.createCopy(outputRecord), chainIndex + 1); } } } From f1a05b8f498444ef5cca676d955dbc09a3e79002 Mon Sep 17 00:00:00 2001 From: marrus Date: Wed, 7 Dec 2011 11:54:03 +0100 Subject: [PATCH 123/310] extracted decision method --- .../CheckpointDecisionCoordinator.java | 65 ++++++++++--------- 1 file changed, 33 insertions(+), 32 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index 10eac34e55157..d04365222cbec 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -91,40 +91,9 @@ public void registerJob(final ExecutionGraph executionGraph) { * the current resource utilization of the vertex */ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { - boolean checkpointDecision = false; LOG.info("Checkpoint decision for vertex " + vertex + " required"); - // This implementation always creates the checkpoint - - // TODO: Provide sensible implementation here - if(rus.getForced() == null){ - if(rus.getTotalInputAmount() != 0 && (rus.getTotalOutputAmount() * 1.0 / rus.getTotalInputAmount() > 2.0)){ - //estimated size of checkpoint - //TODO progress estimation would make sense here - checkpointDecision = false; - LOG.info("Chechpoint to large"); - } - else if (rus.getUserCPU() >= 90) { - LOG.info("CPU-Bottleneck"); - //CPU bottleneck - checkpointDecision = true; - } else { - - if ( vertex.getNumberOfSuccessors() != 0 - && vertex.getNumberOfPredecessors() * 1.0 / vertex.getNumberOfSuccessors() > 1.5) { - LOG.info("vertex.getNumberOfPredecessors()/ vertex.getNumberOfSuccessors() > 1.5"); - //less output-channels than input-channels - //checkpoint at this position probably saves network-traffic - checkpointDecision = true; - } else if (true) { - //always create checkpoint for testing - checkpointDecision = true; - } - } - }else{ - //checkpoint decision was forced by the user - checkpointDecision = rus.getForced(); - } + boolean checkpointDecision = getDecision(vertex, rus); final ExecutionGraph graph = vertex.getExecutionGraph(); final Map> checkpointDecisions = new HashMap>(); final List checkpointDecisionList = new SerializableArrayList(); @@ -137,6 +106,38 @@ else if (rus.getUserCPU() >= 90) { // Propagate checkpoint decisions this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); } + + private boolean getDecision(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { + // This implementation always creates the checkpoint + if(rus.getForced() == null){ + if(rus.getTotalInputAmount() != 0 && (rus.getTotalOutputAmount() * 1.0 / rus.getTotalInputAmount() > 2.0)){ + //estimated size of checkpoint + //TODO progress estimation would make sense here + LOG.info("Chechpoint to large selektivity " + (rus.getTotalOutputAmount() * 1.0 / rus.getTotalInputAmount() > 2.0)); + return false; + + } + if (rus.getUserCPU() >= 90) { + LOG.info("CPU-Bottleneck"); + //CPU bottleneck + return true; + } + + if ( vertex.getNumberOfSuccessors() != 0 + && vertex.getNumberOfPredecessors() * 1.0 / vertex.getNumberOfSuccessors() > 1.5) { + LOG.info("vertex.getNumberOfPredecessors()/ vertex.getNumberOfSuccessors() > 1.5"); + //less output-channels than input-channels + //checkpoint at this position probably saves network-traffic + return true; + } + }else{ + //checkpoint decision was forced by the user + return rus.getForced(); + } + //FIXME always create checkpoint for testing + return true; + } + } From 42c13bd355d18108ce59e4a3e5f4d4c27d63a69f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 13 Dec 2011 22:40:48 +0100 Subject: [PATCH 124/310] First steps towards implementation of scripted job failure patterns --- .../swt/AbstractFailureEvent.java | 66 ++++++++++ .../swt/InstanceFailureEvent.java | 60 +++++++++ .../visualization/swt/JobFailurePattern.java | 53 ++++++++ .../swt/SWTFailurePatternsManager.java | 124 ++++++++++++++++++ .../swt/SWTVisualizationGUI.java | 58 ++++++-- .../visualization/swt/VertexFailureEvent.java | 60 +++++++++ 6 files changed, 411 insertions(+), 10 deletions(-) create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AbstractFailureEvent.java create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/InstanceFailureEvent.java create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/VertexFailureEvent.java diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AbstractFailureEvent.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AbstractFailureEvent.java new file mode 100644 index 0000000000000..857dbd4d08be7 --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AbstractFailureEvent.java @@ -0,0 +1,66 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +/** + * This class implements an abstract failure event which can be used to trigger either a planned vertex or planned + * instance failure. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public abstract class AbstractFailureEvent implements Comparable { + + /** + * The interval in milliseconds until this event will occur. + */ + private final int interval; + + /** + * Constructs a new abstract failure event. + * + * @param interval + * the interval in milliseconds until this event will occur + */ + AbstractFailureEvent(final int interval) { + + if (interval < 0) { + throw new IllegalArgumentException("Argument interval must be larger than or equal to zero"); + } + + this.interval = interval; + } + + /** + * {@inheritDoc} + */ + @Override + public int compareTo(final AbstractFailureEvent o) { + + return (this.interval - o.getInterval()); + } + + /** + * Returns the interval in milliseconds until this event will occur. + * + * @return the interval in milliseconds until this event will occur + */ + public int getInterval() { + + return this.interval; + } +} diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/InstanceFailureEvent.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/InstanceFailureEvent.java new file mode 100644 index 0000000000000..b0b2135e75bf9 --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/InstanceFailureEvent.java @@ -0,0 +1,60 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +/** + * This class implements an instance failure event which can be used to deliberately kill a particular instance after + * the given period of time. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class InstanceFailureEvent extends AbstractFailureEvent { + + /** + * The name of the instance to be killed. + */ + private final String instanceName; + + /** + * Constructs a new instance failure event + * + * @param interval + * the interval in milliseconds until this event will occur + * @param instanceName + * the name of the instance to be killed + */ + InstanceFailureEvent(final int interval, final String instanceName) { + super(interval); + + if (instanceName == null) { + throw new IllegalArgumentException("Argument instanceName must not be null"); + } + + this.instanceName = instanceName; + } + + /** + * Returns the name of the instance to be killed. + * + * @return the name of the instance to be killed + */ + public String getInstanceName() { + + return this.instanceName; + } +} diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java new file mode 100644 index 0000000000000..21e3af81e6ed0 --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java @@ -0,0 +1,53 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +import java.util.SortedSet; +import java.util.TreeSet; + +/** + * A job failure pattern consists of a series of {@link AbstractFailureEvent} objects that occur over time. + *

+ * This class is not thread-safe. + * + * @author warneke + */ +public final class JobFailurePattern { + + /** + * The series of events belonging to this job failure pattern. + */ + private final SortedSet events = new TreeSet(); + + /** + * Adds a new failure event to this job failure pattern. A particular event cannot be added more than once. + * + * @param event + * the event to be added + */ + public void addEvent(final AbstractFailureEvent event) { + + if (event == null) { + throw new IllegalArgumentException("Argument event must not be null"); + } + + if (this.events.contains(event)) { + return; + } + + this.events.add(event); + } +} diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java new file mode 100644 index 0000000000000..371549260e563 --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java @@ -0,0 +1,124 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.eclipse.swt.SWT; +import org.eclipse.swt.custom.CTabFolder; +import org.eclipse.swt.custom.CTabItem; +import org.eclipse.swt.custom.SashForm; +import org.eclipse.swt.events.SelectionEvent; +import org.eclipse.swt.events.SelectionListener; +import org.eclipse.swt.layout.FillLayout; +import org.eclipse.swt.layout.GridData; +import org.eclipse.swt.layout.GridLayout; +import org.eclipse.swt.widgets.Group; +import org.eclipse.swt.widgets.Shell; +import org.eclipse.swt.widgets.Tree; + +public final class SWTFailurePatternsManager implements SelectionListener { + + private static final Log LOG = LogFactory.getLog(SWTFailurePatternsManager.class); + + private static final int WIDTH = 800; + + private static final int HEIGHT = 400; + + private final Shell shell; + + private final Tree jobTree; + + private final CTabFolder jobTabFolder; + + private final CTabItem taskFailurePatternsTab; + + private final CTabItem instanceFailurePatternsTab; + + private final Map failurePatterns = new HashMap(); + + SWTFailurePatternsManager(final Shell parent) { + + // Set size + this.shell = new Shell(parent); + this.shell.setSize(WIDTH, HEIGHT); + this.shell.setText("Manage Outage Patterns"); + + this.shell.setLayout(new GridLayout(2, false)); + + final SashForm horizontalSash = new SashForm(this.shell, SWT.HORIZONTAL); + horizontalSash.setLayoutData(new GridData(GridData.FILL_BOTH)); + + final Group jobGroup = new Group(horizontalSash, SWT.NONE); + jobGroup.setText("Job Failure Patterns"); + jobGroup.setLayout(new FillLayout()); + + this.jobTree = new Tree(jobGroup, SWT.SINGLE | SWT.BORDER); + this.jobTree.addSelectionListener(this); + + this.jobTabFolder = new CTabFolder(horizontalSash, SWT.TOP); + this.jobTabFolder.addSelectionListener(this); + this.jobTabFolder.setVisible(false); + + this.taskFailurePatternsTab = new CTabItem(this.jobTabFolder, SWT.NONE); + + this.instanceFailurePatternsTab = new CTabItem(this.jobTabFolder, SWT.NONE); + + this.jobTabFolder.setSelection(this.taskFailurePatternsTab); + + horizontalSash.setWeights(new int[] { 2, 8 }); + + this.taskFailurePatternsTab.setText("Task Failure Patterns"); + this.instanceFailurePatternsTab.setText("Instance Failure Patterns"); + + } + + public void open() { + + this.shell.open(); + } + + /** + * {@inheritDoc} + */ + @Override + public void widgetDefaultSelected(final SelectionEvent arg0) { + // TODO Auto-generated method stub + + } + + /** + * {@inheritDoc} + */ + @Override + public void widgetSelected(final SelectionEvent arg0) { + // TODO Auto-generated method stub + + } + + public void startFailurePattern(final String jobName, final long referenceTime) { + + final JobFailurePattern failurePattern = this.failurePatterns.get(jobName.toLowerCase()); + if (failurePattern == null) { + LOG.info("No failure pattern for job " + jobName); + } + + + } +} diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java index 4aadb7a1baf7e..bef3da0a97a1d 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java @@ -98,6 +98,8 @@ public class SWTVisualizationGUI implements SelectionListener, Runnable { private final boolean detectBottlenecks; + private volatile boolean applyFailurePatterns = true; + private final ExtendedManagementProtocol jobManager; private final CTabFolder jobTabFolder; @@ -106,6 +108,8 @@ public class SWTVisualizationGUI implements SelectionListener, Runnable { private Map recentJobs = new HashMap(); + private final SWTFailurePatternsManager failurePatternsManager; + /** * Set to filter duplicate events received from the job manager. */ @@ -189,23 +193,47 @@ public void widgetSelected(SelectionEvent arg0) { } }); - final MenuItem diagnosisMenuItem = new MenuItem(this.menuBar, SWT.CASCADE); - diagnosisMenuItem.setText("&Diagnosis"); + final MenuItem debuggingMenuItem = new MenuItem(this.menuBar, SWT.CASCADE); + debuggingMenuItem.setText("&Debugging"); - final Menu diagnosisMenu = new Menu(this.shell, SWT.DROP_DOWN); - diagnosisMenuItem.setMenu(diagnosisMenu); + final Menu debuggingMenu = new Menu(this.shell, SWT.DROP_DOWN); + debuggingMenuItem.setMenu(debuggingMenu); - final MenuItem diagnosisLBUItem = new MenuItem(diagnosisMenu, SWT.PUSH); - diagnosisLBUItem.setText("&Log buffer utilization"); - diagnosisLBUItem.addSelectionListener(new SelectionAdapter() { + final MenuItem debuggingLBUItem = new MenuItem(debuggingMenu, SWT.PUSH); + debuggingLBUItem.setText("&Log buffer utilization"); + debuggingLBUItem.addSelectionListener(new SelectionAdapter() { @Override - public void widgetSelected(SelectionEvent arg0) { + public void widgetSelected(final SelectionEvent arg0) { logBufferUtilization(); shell.setMenuBar(null); } }); + // Insert a separator before the last item in the help menu + new MenuItem(debuggingMenu, SWT.SEPARATOR); + + final MenuItem debuggingAFPItem = new MenuItem(debuggingMenu, SWT.CHECK); + debuggingAFPItem.setText("&Apply failure patterns"); + debuggingAFPItem.setSelection(this.applyFailurePatterns); + debuggingAFPItem.addSelectionListener(new SelectionAdapter() { + + public void widgetSelected(final SelectionEvent arg0) { + applyFailurePatterns = debuggingAFPItem.getSelection(); + shell.setMenuBar(null); + } + }); + + final MenuItem debuggingMFPItem = new MenuItem(debuggingMenu, SWT.PUSH); + debuggingMFPItem.setText("&Manage failure patterns..."); + debuggingMFPItem.addSelectionListener(new SelectionAdapter() { + + public void widgetSelected(final SelectionEvent arg0) { + manageFailurePatterns(); + shell.setMenuBar(null); + } + }); + final MenuItem helpMenuItem = new MenuItem(this.menuBar, SWT.CASCADE); helpMenuItem.setText("&Help"); @@ -217,7 +245,7 @@ public void widgetSelected(SelectionEvent arg0) { helpJavaDocItem.addSelectionListener(new SelectionAdapter() { @Override - public void widgetSelected(SelectionEvent arg0) { + public void widgetSelected(final SelectionEvent arg0) { viewJavaDoc(); shell.setMenuBar(null); } @@ -273,6 +301,9 @@ public void handleEvent(final Event arg0) { } }); + // Create outage patterns manager + this.failurePatternsManager = new SWTFailurePatternsManager(this.shell); + // Launch the timer that will query for events this.display.timerExec(QUERYINTERVAL * 1000, this); } @@ -421,6 +452,8 @@ public void run() { while (it.hasNext()) { final RecentJobEvent newJobEvent = it.next(); addJob(newJobEvent.getJobID(), newJobEvent.getJobName(), newJobEvent.isProfilingAvailable()); + // Find a matching failure pattern and start it + this.failurePatternsManager.startFailurePattern(newJobEvent.getJobName(), newJobEvent.getTimestamp()); } } @@ -545,7 +578,7 @@ private void addJob(JobID jobID, String jobName, boolean isProfilingAvailable) t final TreeItem jobItem = new TreeItem(jobTree, SWT.NONE); jobItem.setText(jobName + " (" + jobID.toString() + ")"); jobItem.setData(graphVisualizationData); - + this.recentJobs.put(jobID, graphVisualizationData); } } @@ -669,6 +702,11 @@ private void cleanUpOldEvents(long sleepTime) { } } + private void manageFailurePatterns() { + + this.failurePatternsManager.open(); + } + private void logBufferUtilization() { if (this.jobTree.getItemCount() == 0) { diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/VertexFailureEvent.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/VertexFailureEvent.java new file mode 100644 index 0000000000000..94842dd8978f8 --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/VertexFailureEvent.java @@ -0,0 +1,60 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +/** + * This class implements a vertex failure event which can be used to deliberately kill a particular task after the + * given period of time. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class VertexFailureEvent extends AbstractFailureEvent { + + /** + * The name of the vertex to be killed. + */ + private final String vertexName; + + /** + * Constructs a new vertex failure event + * + * @param interval + * the interval in milliseconds until this event will occur + * @param vertexName + * the name of the vertex to be killed + */ + VertexFailureEvent(final int interval, final String vertexName) { + super(interval); + + if (vertexName == null) { + throw new IllegalArgumentException("Argument vertexName must not be null"); + } + + this.vertexName = vertexName; + } + + /** + * Returns the name of the vertex to be killed. + * + * @return the name of the vertex to be killed + */ + public String getVertexName() { + + return this.vertexName; + } +} From 9fae000a4ec241064bfbca406c4601444904c28f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 14 Dec 2011 14:00:16 +0100 Subject: [PATCH 125/310] Worked on job failure pattern manager --- .../visualization/swt/JobFailurePattern.java | 6 +- .../swt/JobFailurePatternExecutor.java | 82 +++++++++++++++++++ .../swt/SWTFailurePatternsManager.java | 11 ++- .../swt/SWTVisualizationGUI.java | 29 +++---- 4 files changed, 108 insertions(+), 20 deletions(-) create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternExecutor.java diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java index 21e3af81e6ed0..8330c472d3bd5 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java @@ -33,12 +33,12 @@ public final class JobFailurePattern { private final SortedSet events = new TreeSet(); /** - * Adds a new failure event to this job failure pattern. A particular event cannot be added more than once. + * Adds or updates a failure event to this job failure pattern. * * @param event - * the event to be added + * the event to be added or updated */ - public void addEvent(final AbstractFailureEvent event) { + public void addOrUpdateEvent(final AbstractFailureEvent event) { if (event == null) { throw new IllegalArgumentException("Argument event must not be null"); diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternExecutor.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternExecutor.java new file mode 100644 index 0000000000000..28b74810a69c9 --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternExecutor.java @@ -0,0 +1,82 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +import org.eclipse.swt.widgets.Display; + +import eu.stratosphere.nephele.jobgraph.JobID; + +public final class JobFailurePatternExecutor implements Runnable { + + private final Display timer; + + private final JobID jobID; + + private final String jobName; + + private long offset = 0L; + + private boolean stopRequested = false; + + private boolean executorStarted = false; + + JobFailurePatternExecutor(final Display timer, final JobID jobID, final String jobName, + final JobFailurePattern failurePattern) { + + this.timer = timer; + this.jobID = jobID; + this.jobName = jobName; + } + + /** + * {@inheritDoc} + */ + @Override + public void run() { + + if (this.stopRequested) { + this.stopRequested = false; + return; + } + + scheduleNextEvent(); + } + + public void start(final long referenceTime) { + + if (this.executorStarted) { + throw new IllegalStateException("The executor has already been started"); + } + + final long now = System.currentTimeMillis(); + + this.executorStarted = true; + this.offset = now - referenceTime; + + scheduleNextEvent(); + } + + public void stop() { + + this.stopRequested = true; + this.executorStarted = false; + } + + private void scheduleNextEvent() { + + // TODO: Implement me + } +} diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java index 371549260e563..3182d6c0a13bd 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java @@ -33,6 +33,8 @@ import org.eclipse.swt.widgets.Shell; import org.eclipse.swt.widgets.Tree; +import eu.stratosphere.nephele.jobgraph.JobID; + public final class SWTFailurePatternsManager implements SelectionListener { private static final Log LOG = LogFactory.getLog(SWTFailurePatternsManager.class); @@ -112,13 +114,16 @@ public void widgetSelected(final SelectionEvent arg0) { } - public void startFailurePattern(final String jobName, final long referenceTime) { + public void startFailurePattern(final JobID jobID, final String jobName, final long referenceTime) { final JobFailurePattern failurePattern = this.failurePatterns.get(jobName.toLowerCase()); if (failurePattern == null) { LOG.info("No failure pattern for job " + jobName); } - - + + final JobFailurePatternExecutor executor = new JobFailurePatternExecutor(this.shell.getDisplay(), jobID, + jobName, failurePattern); + + executor.start(referenceTime); } } diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java index bef3da0a97a1d..c6c750089f24e 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java @@ -99,7 +99,7 @@ public class SWTVisualizationGUI implements SelectionListener, Runnable { private final boolean detectBottlenecks; private volatile boolean applyFailurePatterns = true; - + private final ExtendedManagementProtocol jobManager; private final CTabFolder jobTabFolder; @@ -109,7 +109,7 @@ public class SWTVisualizationGUI implements SelectionListener, Runnable { private Map recentJobs = new HashMap(); private final SWTFailurePatternsManager failurePatternsManager; - + /** * Set to filter duplicate events received from the job manager. */ @@ -212,28 +212,28 @@ public void widgetSelected(final SelectionEvent arg0) { // Insert a separator before the last item in the help menu new MenuItem(debuggingMenu, SWT.SEPARATOR); - + final MenuItem debuggingAFPItem = new MenuItem(debuggingMenu, SWT.CHECK); debuggingAFPItem.setText("&Apply failure patterns"); debuggingAFPItem.setSelection(this.applyFailurePatterns); debuggingAFPItem.addSelectionListener(new SelectionAdapter() { - + public void widgetSelected(final SelectionEvent arg0) { - applyFailurePatterns = debuggingAFPItem.getSelection(); + applyFailurePatterns = debuggingAFPItem.getSelection(); shell.setMenuBar(null); } }); - + final MenuItem debuggingMFPItem = new MenuItem(debuggingMenu, SWT.PUSH); debuggingMFPItem.setText("&Manage failure patterns..."); debuggingMFPItem.addSelectionListener(new SelectionAdapter() { - + public void widgetSelected(final SelectionEvent arg0) { - manageFailurePatterns(); + manageFailurePatterns(); shell.setMenuBar(null); } }); - + final MenuItem helpMenuItem = new MenuItem(this.menuBar, SWT.CASCADE); helpMenuItem.setText("&Help"); @@ -303,7 +303,7 @@ public void handleEvent(final Event arg0) { // Create outage patterns manager this.failurePatternsManager = new SWTFailurePatternsManager(this.shell); - + // Launch the timer that will query for events this.display.timerExec(QUERYINTERVAL * 1000, this); } @@ -453,7 +453,8 @@ public void run() { final RecentJobEvent newJobEvent = it.next(); addJob(newJobEvent.getJobID(), newJobEvent.getJobName(), newJobEvent.isProfilingAvailable()); // Find a matching failure pattern and start it - this.failurePatternsManager.startFailurePattern(newJobEvent.getJobName(), newJobEvent.getTimestamp()); + this.failurePatternsManager.startFailurePattern(newJobEvent.getJobID(), newJobEvent.getJobName(), + newJobEvent.getTimestamp()); } } @@ -578,7 +579,7 @@ private void addJob(JobID jobID, String jobName, boolean isProfilingAvailable) t final TreeItem jobItem = new TreeItem(jobTree, SWT.NONE); jobItem.setText(jobName + " (" + jobID.toString() + ")"); jobItem.setData(graphVisualizationData); - + this.recentJobs.put(jobID, graphVisualizationData); } } @@ -703,10 +704,10 @@ private void cleanUpOldEvents(long sleepTime) { } private void manageFailurePatterns() { - + this.failurePatternsManager.open(); } - + private void logBufferUtilization() { if (this.jobTree.getItemCount() == 0) { From 896111ed5905d211b8113bf328bbf4479fc30abd Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 5 Jan 2012 12:59:20 +0100 Subject: [PATCH 126/310] Worked on GUI extension to simulate execution failures --- .../swt/SWTFailurePatternsManager.java | 146 +++++++++++++++++- .../swt/SWTNewFailurePatternDialog.java | 5 + 2 files changed, 145 insertions(+), 6 deletions(-) create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java index 3182d6c0a13bd..7992716d11779 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java @@ -18,18 +18,27 @@ import java.util.HashMap; import java.util.Map; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.eclipse.swt.SWT; import org.eclipse.swt.custom.CTabFolder; import org.eclipse.swt.custom.CTabItem; import org.eclipse.swt.custom.SashForm; +import org.eclipse.swt.events.MenuEvent; +import org.eclipse.swt.events.MenuListener; +import org.eclipse.swt.events.SelectionAdapter; import org.eclipse.swt.events.SelectionEvent; import org.eclipse.swt.events.SelectionListener; import org.eclipse.swt.layout.FillLayout; import org.eclipse.swt.layout.GridData; import org.eclipse.swt.layout.GridLayout; +import org.eclipse.swt.widgets.Button; +import org.eclipse.swt.widgets.Composite; import org.eclipse.swt.widgets.Group; +import org.eclipse.swt.widgets.Label; +import org.eclipse.swt.widgets.Menu; +import org.eclipse.swt.widgets.MenuItem; import org.eclipse.swt.widgets.Shell; import org.eclipse.swt.widgets.Tree; @@ -61,10 +70,27 @@ public final class SWTFailurePatternsManager implements SelectionListener { this.shell = new Shell(parent); this.shell.setSize(WIDTH, HEIGHT); this.shell.setText("Manage Outage Patterns"); - - this.shell.setLayout(new GridLayout(2, false)); - - final SashForm horizontalSash = new SashForm(this.shell, SWT.HORIZONTAL); + GridLayout gl = new GridLayout(1, false); + gl.horizontalSpacing = 0; + gl.verticalSpacing = 0; + gl.marginRight = 0; + gl.marginLeft = 0; + gl.marginBottom = 0; + gl.marginTop = 0; + gl.marginHeight = 0; + gl.marginWidth = 0; + this.shell.setLayout(gl); + + final Composite mainComposite = new Composite(this.shell, SWT.NONE); + mainComposite.setLayout(new GridLayout(1, false)); + GridData gridData = new GridData(); + gridData.horizontalAlignment = GridData.FILL; + gridData.verticalAlignment = GridData.FILL; + gridData.grabExcessHorizontalSpace = true; + gridData.grabExcessVerticalSpace = true; + mainComposite.setLayoutData(gridData); + + final SashForm horizontalSash = new SashForm(mainComposite, SWT.HORIZONTAL); horizontalSash.setLayoutData(new GridData(GridData.FILL_BOTH)); final Group jobGroup = new Group(horizontalSash, SWT.NONE); @@ -73,10 +99,10 @@ public final class SWTFailurePatternsManager implements SelectionListener { this.jobTree = new Tree(jobGroup, SWT.SINGLE | SWT.BORDER); this.jobTree.addSelectionListener(this); - + this.jobTree.setMenu(createTreeContextMenu()); + this.jobTabFolder = new CTabFolder(horizontalSash, SWT.TOP); this.jobTabFolder.addSelectionListener(this); - this.jobTabFolder.setVisible(false); this.taskFailurePatternsTab = new CTabItem(this.jobTabFolder, SWT.NONE); @@ -89,6 +115,26 @@ public final class SWTFailurePatternsManager implements SelectionListener { this.taskFailurePatternsTab.setText("Task Failure Patterns"); this.instanceFailurePatternsTab.setText("Instance Failure Patterns"); + + final Composite buttonComposite = new Composite(this.shell, SWT.NONE); + buttonComposite.setLayout(new GridLayout(2, false)); + gridData = new GridData(); + gridData.horizontalAlignment = GridData.FILL; + buttonComposite.setLayoutData(gridData); + + final Label fillLabel = new Label(buttonComposite, SWT.NONE); + gridData = new GridData(); + gridData.horizontalAlignment = GridData.FILL; + gridData.grabExcessHorizontalSpace = true; + gridData.grabExcessVerticalSpace = false; + fillLabel.setLayoutData(gridData); + + final Button closeButton = new Button(buttonComposite, SWT.PUSH); + closeButton.setText("Close"); + gridData = new GridData(); + gridData.horizontalAlignment = SWT.RIGHT; + closeButton.setLayoutData(gridData); + } public void open() { @@ -96,6 +142,94 @@ public void open() { this.shell.open(); } + private Menu createTreeContextMenu() { + + final Menu treeContextMenu = new Menu(this.shell); + final MenuItem createItem = new MenuItem(treeContextMenu, SWT.PUSH); + createItem.setText("Create..."); + createItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + createNewFailurePattern(); + } + }); + new MenuItem(treeContextMenu, SWT.SEPARATOR); + final MenuItem deleteItem = new MenuItem(treeContextMenu, SWT.PUSH); + deleteItem.setText("Delete..."); + deleteItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + deleteFailurePattern(); + } + }); + new MenuItem(treeContextMenu, SWT.SEPARATOR); + final MenuItem saveItem = new MenuItem(treeContextMenu, SWT.PUSH); + saveItem.setText("Save..."); + saveItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + saveFailurePattern(); + } + }); + final MenuItem loadItem = new MenuItem(treeContextMenu, SWT.PUSH); + loadItem.setText("Load..."); + loadItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + loadFailurePattern(); + } + }); + + treeContextMenu.addMenuListener(new MenuListener() { + + @Override + public void menuShown(final MenuEvent arg0) { + + if(jobTree.getSelection().length == 0) { + createItem.setEnabled(true); + deleteItem.setEnabled(false); + saveItem.setEnabled(false); + loadItem.setEnabled(true); + } else { + createItem.setEnabled(false); + deleteItem.setEnabled(true); + saveItem.setEnabled(true); + loadItem.setEnabled(false); + } + } + + @Override + public void menuHidden(final MenuEvent arg0) { + // TODO Auto-generated method stub + + } + }); + + return treeContextMenu; + } + + private void createNewFailurePattern() { + //TODO: Implement me + + + } + + private void deleteFailurePattern() { + //TODO: Implement me + } + + private void saveFailurePattern() { + //TODO: Implement me + } + + private void loadFailurePattern() { + //TODO: Implement me + } + /** * {@inheritDoc} */ diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java new file mode 100644 index 0000000000000..1a9f080f578e6 --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java @@ -0,0 +1,5 @@ +package eu.stratosphere.nephele.visualization.swt; + +public class SWTNewFailurePatternDialog { + +} From ebb085bd6bec1ee1ba793238159d59bc735ffa4c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 5 Jan 2012 13:47:54 +0100 Subject: [PATCH 127/310] Reintegrated union code --- .../eu/stratosphere/nephele/io/InputGate.java | 17 +++++ .../nephele/io/RuntimeInputGate.java | 67 ++++++++++++++++++- .../wrappers/StreamingInputGate.java | 25 ++++++- 3 files changed, 105 insertions(+), 4 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java index aa8ba3345c9e3..71c225cd6ead4 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java @@ -158,4 +158,21 @@ InMemoryInputChannel createInMemoryInputChannel(InputGate inputGate, Chann * Removes all input channels from the input gate. */ void removeAllInputChannels(); + + /** + * Registers a {@link RecordAvailabilityListener} with this input gate. + * + * @param listener + * the listener object to be registered + */ + void registerRecordAvailabilityListener(RecordAvailabilityListener listener); + + /** + * Checks if the input gate has records available. + * + * @return true if the gate has records available, false otherwise + * @throws IOException + * @throws InterruptedException + */ + boolean hasRecordAvailable() throws IOException, InterruptedException; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java index 43a10ee8c0bdb..feacd67ab8400 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java @@ -83,6 +83,16 @@ public class RuntimeInputGate extends AbstractGate implemen */ private InputGateListener[] inputGateListeners = null; + /** + * The listener object to be notified when a channel has at least one record available. + */ + private RecordAvailabilityListener recordAvailabilityListener = null; + + /** + * If the value of this variable is set to true, the input gate is closed. + */ + private boolean isClosed = false; + /** * The channel to read from next. */ @@ -304,6 +314,11 @@ public T readRecord(final T target) throws IOException, InterruptedException { while (true) { if (this.channelToReadFrom == -1) { + + if (this.isClosed()) { + return null; + } + this.channelToReadFrom = waitForAnyChannelToBecomeAvailable(); } try { @@ -312,6 +327,7 @@ record = this.getInputChannel(this.channelToReadFrom).readRecord(target); // System.out.println("### Caught EOF exception at channel " + channelToReadFrom + "(" + // this.getInputChannel(channelToReadFrom).getType().toString() + ")"); if (this.isClosed()) { + this.channelToReadFrom = -1; return null; } } @@ -330,12 +346,15 @@ record = this.getInputChannel(this.channelToReadFrom).readRecord(target); * {@inheritDoc} */ @Override - public void notifyRecordIsAvailable(final int channelIndex) { - + public void notifyRecordIsAvailable(int channelIndex) { synchronized (this.availableChannels) { this.availableChannels.add(Integer.valueOf(channelIndex)); this.availableChannels.notify(); + + if (this.recordAvailabilityListener != null) { + this.recordAvailabilityListener.reportRecordAvailability(this); + } } } @@ -371,6 +390,10 @@ public int waitForAnyChannelToBecomeAvailable() throws InterruptedException { @Override public boolean isClosed() throws IOException, InterruptedException { + if (this.isClosed) { + return true; + } + for (int i = 0; i < this.getNumberOfInputChannels(); i++) { final AbstractInputChannel inputChannel = this.inputChannels.get(i); if (!inputChannel.isClosed()) { @@ -378,6 +401,8 @@ public boolean isClosed() throws IOException, InterruptedException { } } + this.isClosed = true; + return true; } @@ -471,4 +496,42 @@ public void activateInputChannels() throws IOException, InterruptedException { // TODO Auto-generated method stub } + + /** + * {@inheritDoc} + */ + @Override + public void registerRecordAvailabilityListener(final RecordAvailabilityListener listener) { + + synchronized (this.availableChannels) { + + if (this.recordAvailabilityListener != null) { + throw new IllegalStateException(this.recordAvailabilityListener + + " is already registered as a record availability listener"); + } + + this.recordAvailabilityListener = listener; + } + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasRecordAvailable() throws IOException, InterruptedException { + + if (this.channelToReadFrom == -1) { + + if (this.isClosed()) { + return true; + } + + synchronized (this.availableChannels) { + + return !(this.availableChannels.isEmpty()); + } + } + + return true; + } } diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java index d652480eebba3..c7f9fd80dfcbd 100644 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java +++ b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java @@ -20,6 +20,7 @@ import java.util.ArrayDeque; import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.RecordAvailabilityListener; import eu.stratosphere.nephele.plugins.wrapper.AbstractInputGateWrapper; import eu.stratosphere.nephele.streaming.listeners.StreamListener; import eu.stratosphere.nephele.types.Record; @@ -112,10 +113,10 @@ record = this.getInputChannel(this.channelToReadFrom).readRecord(target); } public void reportRecordReceived(final Record record) { - + this.streamListener.recordReceived(record); } - + /** * This method returns the index of a channel which has at least * one record available. The method may block until at least one @@ -147,4 +148,24 @@ public void notifyRecordIsAvailable(final int channelIndex) { this.availableChannels.notify(); } } + + /** + * {@inheritDoc} + */ + @Override + public void registerRecordAvailabilityListener(final RecordAvailabilityListener listener) { + + // TODO: Implement me + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasRecordAvailable() throws IOException, InterruptedException { + + // TODO: Implement me + + return false; + } } From b3b677c2d91b72766567de839ebe6a9fed2bc94d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 9 Jan 2012 17:06:35 +0100 Subject: [PATCH 128/310] Implemented a combo box with auto-completion features in SWT --- .../swt/AutoCompletionCombo.java | 160 ++++++++++++++++++ 1 file changed, 160 insertions(+) create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java new file mode 100644 index 0000000000000..17936062d345a --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java @@ -0,0 +1,160 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.eclipse.swt.SWT; +import org.eclipse.swt.events.KeyEvent; +import org.eclipse.swt.events.KeyListener; +import org.eclipse.swt.graphics.Point; +import org.eclipse.swt.layout.FillLayout; +import org.eclipse.swt.widgets.Combo; +import org.eclipse.swt.widgets.Composite; + +/** + * The class implements a combo box with auto-completion features. + * + * @author warneke + */ +public final class AutoCompletionCombo extends Composite implements KeyListener { + + /** + * The list of possible suggestions for the auto-completion. + */ + private final ArrayList suggestions; + + /** + * The internal SWT combo box. + */ + private final Combo combo; + + /** + * Constructs a new auto-completion combo box. + * + * @param parent + * the parent composite + * @param style + * the style of the combo box + * @param suggestions + * a list of suggestions for the auto-completion + */ + public AutoCompletionCombo(final Composite parent, final int style, final List suggestions) { + super(parent, style); + + // First, sort the suggestions + this.suggestions = new ArrayList(suggestions); + Collections.sort(this.suggestions); + + setLayout(new FillLayout()); + + this.combo = new Combo(this, style); + this.combo.addKeyListener(this); + + // Add the suggestions + for (final String suggestion : this.suggestions) { + this.combo.add(suggestion); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void keyPressed(final KeyEvent arg0) { + + // Nothing to do here + } + + /** + * {@inheritDoc} + */ + @Override + public void keyReleased(final KeyEvent arg0) { + + if (arg0.character == 0) { + return; + } + + final String text = this.combo.getText(); + if (text == null) { + return; + } + + final List matchingSuggestions = updateSuggestions(text); + + final int length = text.length(); + + if (arg0.character == SWT.BS) { + + this.combo.setText(text); + this.combo.setSelection(new Point(length, length)); + + } else { + + if (length != 0) { + Point selection; + if (matchingSuggestions.isEmpty()) { + this.combo.setText(text); + selection = new Point(length, length); + } else { + final String suggestion = matchingSuggestions.get(0); + this.combo.setText(suggestion); + selection = new Point(length, suggestion.length()); + } + + this.combo.setSelection(selection); + } + } + } + + /** + * Computes an update of the combo box's suggestions. All suggestions have the entered text as a prefix. + * + * @param text + * the text entered in the combo box's text field so far + * @return the list of suggestions, possibly empty + */ + private List updateSuggestions(final String text) { + + final ArrayList matchingSuggestions = new ArrayList(this.suggestions.size()); + for (final String suggestion : this.suggestions) { + + if (suggestion.startsWith(text)) { + matchingSuggestions.add(suggestion); + } + } + + this.combo.removeAll(); + for (final String suggestion : matchingSuggestions) { + this.combo.add(suggestion); + } + + return matchingSuggestions; + } + + /** + * Returns the text from the text field of the combo box. + * + * @return the text from the text field of the combo box + */ + public String getText() { + + return this.combo.getText(); + } +} From 3b5be468f6af72dc148d86e008337d5bfb93539d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 9 Jan 2012 17:14:19 +0100 Subject: [PATCH 129/310] Fixed problem with auto-completion combo box --- .../swt/AutoCompletionCombo.java | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java index 17936062d345a..3f0a0df7aa0ab 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java @@ -22,6 +22,8 @@ import org.eclipse.swt.SWT; import org.eclipse.swt.events.KeyEvent; import org.eclipse.swt.events.KeyListener; +import org.eclipse.swt.events.SelectionEvent; +import org.eclipse.swt.events.SelectionListener; import org.eclipse.swt.graphics.Point; import org.eclipse.swt.layout.FillLayout; import org.eclipse.swt.widgets.Combo; @@ -32,7 +34,7 @@ * * @author warneke */ -public final class AutoCompletionCombo extends Composite implements KeyListener { +public final class AutoCompletionCombo extends Composite implements KeyListener, SelectionListener { /** * The list of possible suggestions for the auto-completion. @@ -65,6 +67,7 @@ public AutoCompletionCombo(final Composite parent, final int style, final List Date: Mon, 9 Jan 2012 18:06:40 +0100 Subject: [PATCH 130/310] Added possibility to add key listener to combo box with auto-completion --- .../visualization/swt/AutoCompletionCombo.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java index 3f0a0df7aa0ab..27cb2c9910127 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java @@ -179,4 +179,15 @@ public void widgetSelected(final SelectionEvent arg0) { updateSuggestions(text); this.combo.setText(text); } + + /** + * Adds a {@link KeyListener} to the combo box. + * + * @param keyListener + * the key listener object to be added + */ + public void addKeyListener(final KeyListener keyListener) { + + this.combo.addKeyListener(keyListener); + } } From 0d01c4606ce5472a7fb417c254fb102ece8ea71a Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 9 Jan 2012 18:07:33 +0100 Subject: [PATCH 131/310] Implemented dialog to create new failure pattern --- .../swt/SWTNewFailurePatternDialog.java | 189 +++++++++++++++++- 1 file changed, 188 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java index 1a9f080f578e6..bc7e83509c26d 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java @@ -1,5 +1,192 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; -public class SWTNewFailurePatternDialog { +import java.util.List; + +import org.eclipse.swt.SWT; +import org.eclipse.swt.layout.GridData; +import org.eclipse.swt.layout.GridLayout; +import org.eclipse.swt.layout.RowLayout; +import org.eclipse.swt.widgets.Button; +import org.eclipse.swt.widgets.Composite; +import org.eclipse.swt.widgets.Display; +import org.eclipse.swt.widgets.Event; +import org.eclipse.swt.widgets.Listener; +import org.eclipse.swt.widgets.MessageBox; +import org.eclipse.swt.widgets.Shell; +import org.eclipse.swt.events.KeyAdapter; +import org.eclipse.swt.events.KeyEvent; + +/** + * This class implements a dialogue for creating a new failure pattern. + * + * @author warneke + */ +public final class SWTNewFailurePatternDialog { + + /** + * The width of the dialog. + */ + private static final int WIDTH = 300; + + /** + * The height of the dialog. + */ + private static final int HEIGHT = 100; + + /** + * The shell for this dialog. + */ + private final Shell shell; + + /** + * The auto-completion combo box. + */ + private final AutoCompletionCombo input; + + /** + * The return value of the showDialog method. + */ + private String returnValue = null; + + /** + * Constructs a new dialog for creating a new failure pattern. + * + * @param parent + * the parent of this dialog + * @param nameSuggestions + * name suggestions to be displayed inside auto-completion combo box + */ + public SWTNewFailurePatternDialog(final Shell parent, final List nameSuggestions) { + + this.shell = new Shell(parent); + this.shell.setSize(WIDTH, HEIGHT); + this.shell.setText("Create New Failure Pattern"); + this.shell.setLayout(new GridLayout(1, false)); + + GridData gridData = new GridData(); + gridData.horizontalAlignment = GridData.FILL; + gridData.grabExcessHorizontalSpace = true; + gridData.grabExcessVerticalSpace = false; + + this.input = new AutoCompletionCombo(this.shell, SWT.NONE, nameSuggestions); + this.input.setLayoutData(gridData); + this.input.addKeyListener(new KeyAdapter() { + + @Override + public void keyReleased(final KeyEvent arg0) { + + if (arg0.character != SWT.CR) { + return; + } + + if (isInputValid()) { + returnValue = input.getText(); + shell.dispose(); + } + } + }); + + gridData = new GridData(); + gridData.horizontalAlignment = SWT.RIGHT; + gridData.verticalAlignment = SWT.BOTTOM; + gridData.grabExcessHorizontalSpace = true; + gridData.grabExcessVerticalSpace = true; + + final Composite buttonComposite = new Composite(this.shell, SWT.RIGHT_TO_LEFT); + final RowLayout rowLayout = new RowLayout(SWT.HORIZONTAL); + rowLayout.marginBottom = 0; + rowLayout.marginHeight = 0; + rowLayout.marginLeft = 0; + rowLayout.marginRight = 0; + rowLayout.marginTop = 0; + rowLayout.marginWidth = 0; + rowLayout.pack = false; + + buttonComposite.setLayoutData(gridData); + buttonComposite.setLayout(rowLayout); + + final Button ok = new Button(buttonComposite, SWT.PUSH); + ok.setText("OK"); + ok.addListener(SWT.Selection, new Listener() { + + @Override + public void handleEvent(final Event arg0) { + + if (isInputValid()) { + returnValue = input.getText(); + shell.dispose(); + } + } + }); + + final Button cancel = new Button(buttonComposite, SWT.PUSH); + cancel.setText("Cancel"); + cancel.addListener(SWT.Selection, new Listener() { + + @Override + public void handleEvent(final Event arg0) { + + returnValue = null; + shell.dispose(); + } + }); + } + + /** + * Checks whether the input is valid and displays an error message box if not. + * + * @return true if the input is valid, false otherwise + */ + private boolean isInputValid() { + + final String text = this.input.getText(); + if (text.isEmpty()) { + final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); + + messageBox.setText("Invalid Input"); + messageBox.setMessage("Name for failure pattern must not be empty."); + messageBox.open(); + + this.input.setFocus(); + + return false; + } + + return true; + } + + /** + * Opens the dialog. + * + * @return the name for the new failure pattern or null if the user has canceled the dialog + */ + public String showDialog() { + + this.shell.open(); + + final Display display = this.shell.getDisplay(); + + while (!this.shell.isDisposed()) { + if (!display.readAndDispatch()) { + display.sleep(); + } + } + return this.returnValue; + } } From ad9a59307913f5eb1d2cc8d15e5653475b40a9f8 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 9 Jan 2012 20:27:23 +0100 Subject: [PATCH 132/310] Worked on failure patten manager --- .../visualization/swt/JobFailurePattern.java | 34 ++ .../swt/SWTFailurePatternsManager.java | 337 ++++++++++++++++-- 2 files changed, 340 insertions(+), 31 deletions(-) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java index 8330c472d3bd5..18ac5a16eae8a 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java @@ -27,11 +27,45 @@ */ public final class JobFailurePattern { + /** + * The name of the job failure pattern. + */ + private final String name; + /** * The series of events belonging to this job failure pattern. */ private final SortedSet events = new TreeSet(); + /** + * Constructs a new job failure pattern with a given name. + * + * @param name + * the name of the job failure pattern + */ + public JobFailurePattern(final String name) { + + if (name == null) { + throw new IllegalArgumentException("Argument name must not be null"); + } + + if (name.isEmpty()) { + throw new IllegalArgumentException("Argument name must not be empty"); + } + + this.name = name; + } + + /** + * Returns the name of the job failure pattern. + * + * @return the name of the job failure pattern + */ + public String getName() { + + return this.name; + } + /** * Adds or updates a failure event to this job failure pattern. * diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java index 7992716d11779..0cef86361ebd3 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java @@ -15,9 +15,15 @@ package eu.stratosphere.nephele.visualization.swt; +import java.io.FileInputStream; +import java.io.InputStream; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -35,12 +41,22 @@ import org.eclipse.swt.layout.GridLayout; import org.eclipse.swt.widgets.Button; import org.eclipse.swt.widgets.Composite; +import org.eclipse.swt.widgets.FileDialog; import org.eclipse.swt.widgets.Group; import org.eclipse.swt.widgets.Label; import org.eclipse.swt.widgets.Menu; import org.eclipse.swt.widgets.MenuItem; +import org.eclipse.swt.widgets.MessageBox; import org.eclipse.swt.widgets.Shell; +import org.eclipse.swt.widgets.Table; +import org.eclipse.swt.widgets.TableColumn; +import org.eclipse.swt.widgets.TableItem; import org.eclipse.swt.widgets.Tree; +import org.eclipse.swt.widgets.TreeItem; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; import eu.stratosphere.nephele.jobgraph.JobID; @@ -60,8 +76,12 @@ public final class SWTFailurePatternsManager implements SelectionListener { private final CTabItem taskFailurePatternsTab; + private final Table taskFailureTable; + private final CTabItem instanceFailurePatternsTab; + private final Table instanceFailureTable; + private final Map failurePatterns = new HashMap(); SWTFailurePatternsManager(final Shell parent) { @@ -69,7 +89,7 @@ public final class SWTFailurePatternsManager implements SelectionListener { // Set size this.shell = new Shell(parent); this.shell.setSize(WIDTH, HEIGHT); - this.shell.setText("Manage Outage Patterns"); + this.shell.setText("Manage Failure Patterns"); GridLayout gl = new GridLayout(1, false); gl.horizontalSpacing = 0; gl.verticalSpacing = 0; @@ -80,7 +100,7 @@ public final class SWTFailurePatternsManager implements SelectionListener { gl.marginHeight = 0; gl.marginWidth = 0; this.shell.setLayout(gl); - + final Composite mainComposite = new Composite(this.shell, SWT.NONE); mainComposite.setLayout(new GridLayout(1, false)); GridData gridData = new GridData(); @@ -89,7 +109,7 @@ public final class SWTFailurePatternsManager implements SelectionListener { gridData.grabExcessHorizontalSpace = true; gridData.grabExcessVerticalSpace = true; mainComposite.setLayoutData(gridData); - + final SashForm horizontalSash = new SashForm(mainComposite, SWT.HORIZONTAL); horizontalSash.setLayoutData(new GridData(GridData.FILL_BOTH)); @@ -100,8 +120,9 @@ public final class SWTFailurePatternsManager implements SelectionListener { this.jobTree = new Tree(jobGroup, SWT.SINGLE | SWT.BORDER); this.jobTree.addSelectionListener(this); this.jobTree.setMenu(createTreeContextMenu()); - + this.jobTabFolder = new CTabFolder(horizontalSash, SWT.TOP); + this.jobTabFolder.setLayout(new GridLayout()); this.jobTabFolder.addSelectionListener(this); this.taskFailurePatternsTab = new CTabItem(this.jobTabFolder, SWT.NONE); @@ -112,29 +133,50 @@ public final class SWTFailurePatternsManager implements SelectionListener { horizontalSash.setWeights(new int[] { 2, 8 }); + gridData = new GridData(SWT.FILL, SWT.FILL, true, true); + gridData.heightHint = 200; + + this.taskFailureTable = new Table(this.jobTabFolder, SWT.MULTI | SWT.BORDER | SWT.VIRTUAL); + this.taskFailureTable.setLayoutData(gridData); + this.taskFailureTable.setLinesVisible(true); + this.taskFailureTable.setHeaderVisible(true); + this.taskFailureTable.setSize(200, 200); + new TableColumn(this.taskFailureTable, SWT.LEFT).setText("Task name"); + new TableColumn(this.taskFailureTable, SWT.LEFT).setText("Failure time"); + + this.instanceFailureTable = new Table(this.jobTabFolder, SWT.MULTI | SWT.BORDER | SWT.VIRTUAL); + this.instanceFailureTable.setLayoutData(gridData); + this.instanceFailureTable.setVisible(true); + this.instanceFailureTable.setLinesVisible(true); + this.instanceFailureTable.setHeaderVisible(true); + new TableColumn(this.instanceFailureTable, SWT.LEFT).setText("Instance name"); + new TableColumn(this.instanceFailureTable, SWT.LEFT).setText("Failure time"); + this.taskFailurePatternsTab.setText("Task Failure Patterns"); + this.taskFailurePatternsTab.setControl(this.taskFailureTable); + this.instanceFailurePatternsTab.setText("Instance Failure Patterns"); + this.instanceFailurePatternsTab.setControl(this.instanceFailureTable); - final Composite buttonComposite = new Composite(this.shell, SWT.NONE); buttonComposite.setLayout(new GridLayout(2, false)); gridData = new GridData(); gridData.horizontalAlignment = GridData.FILL; buttonComposite.setLayoutData(gridData); - + final Label fillLabel = new Label(buttonComposite, SWT.NONE); gridData = new GridData(); gridData.horizontalAlignment = GridData.FILL; gridData.grabExcessHorizontalSpace = true; gridData.grabExcessVerticalSpace = false; fillLabel.setLayoutData(gridData); - + final Button closeButton = new Button(buttonComposite, SWT.PUSH); closeButton.setText("Close"); gridData = new GridData(); gridData.horizontalAlignment = SWT.RIGHT; closeButton.setLayoutData(gridData); - + } public void open() { @@ -143,12 +185,12 @@ public void open() { } private Menu createTreeContextMenu() { - + final Menu treeContextMenu = new Menu(this.shell); final MenuItem createItem = new MenuItem(treeContextMenu, SWT.PUSH); createItem.setText("Create..."); createItem.addSelectionListener(new SelectionAdapter() { - + @Override public void widgetSelected(final SelectionEvent arg0) { createNewFailurePattern(); @@ -158,7 +200,7 @@ public void widgetSelected(final SelectionEvent arg0) { final MenuItem deleteItem = new MenuItem(treeContextMenu, SWT.PUSH); deleteItem.setText("Delete..."); deleteItem.addSelectionListener(new SelectionAdapter() { - + @Override public void widgetSelected(final SelectionEvent arg0) { deleteFailurePattern(); @@ -168,7 +210,7 @@ public void widgetSelected(final SelectionEvent arg0) { final MenuItem saveItem = new MenuItem(treeContextMenu, SWT.PUSH); saveItem.setText("Save..."); saveItem.addSelectionListener(new SelectionAdapter() { - + @Override public void widgetSelected(final SelectionEvent arg0) { saveFailurePattern(); @@ -177,19 +219,19 @@ public void widgetSelected(final SelectionEvent arg0) { final MenuItem loadItem = new MenuItem(treeContextMenu, SWT.PUSH); loadItem.setText("Load..."); loadItem.addSelectionListener(new SelectionAdapter() { - + @Override public void widgetSelected(final SelectionEvent arg0) { loadFailurePattern(); } }); - + treeContextMenu.addMenuListener(new MenuListener() { - + @Override public void menuShown(final MenuEvent arg0) { - - if(jobTree.getSelection().length == 0) { + + if (jobTree.getSelection().length == 0) { createItem.setEnabled(true); deleteItem.setEnabled(false); saveItem.setEnabled(false); @@ -201,35 +243,268 @@ public void menuShown(final MenuEvent arg0) { loadItem.setEnabled(false); } } - + @Override public void menuHidden(final MenuEvent arg0) { // TODO Auto-generated method stub - + } }); - + return treeContextMenu; } - + private void createNewFailurePattern() { - //TODO: Implement me - - + + // TODO: Provide proper list of name suggestions + final List suggestions = new ArrayList(); + + final SWTNewFailurePatternDialog dialog = new SWTNewFailurePatternDialog(this.shell, suggestions); + + final String patternName = dialog.showDialog(); + if (patternName == null) { + return; + } + + final JobFailurePattern jobFailurePattern = new JobFailurePattern(patternName); + + addFailurePatternToTree(jobFailurePattern); + displayFailurePattern(jobFailurePattern); } - + private void deleteFailurePattern() { - //TODO: Implement me + // TODO: Implement me } - + private void saveFailurePattern() { - //TODO: Implement me + // TODO: Implement me } - + private void loadFailurePattern() { - //TODO: Implement me + + final FileDialog fileDialog = new FileDialog(this.shell, SWT.OPEN); + fileDialog.setText("Load Failure Pattern"); + final String[] filterExts = { "*.xml", "*.*" }; + fileDialog.setFilterExtensions(filterExts); + + final String selectedFile = fileDialog.open(); + if (selectedFile == null) { + return; + } + + final JobFailurePattern failurePattern = loadFailurePatternFromFile(selectedFile); + + addFailurePatternToTree(failurePattern); + displayFailurePattern(failurePattern); + } + + private void addFailurePatternToTree(final JobFailurePattern failurePattern) { + + final TreeItem jobFailureItem = new TreeItem(this.jobTree, SWT.NONE); + jobFailureItem.setText(failurePattern.getName()); + jobFailureItem.setData(failurePattern); } - + + private JobFailurePattern loadFailurePatternFromFile(final String filename) { + + final DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory.newInstance(); + // Ignore comments in the XML file + docBuilderFactory.setIgnoringComments(true); + docBuilderFactory.setNamespaceAware(true); + + JobFailurePattern jobFailurePattern = null; + InputStream inputStream = null; + + try { + + inputStream = new FileInputStream(filename); + + final DocumentBuilder builder = docBuilderFactory.newDocumentBuilder(); + Document doc = null; + Element root = null; + + doc = builder.parse(inputStream); + + if (doc == null) { + throw new Exception("Document is null"); + } + + root = doc.getDocumentElement(); + if (root == null) { + throw new Exception("Root element is null"); + } + + if (!"pattern".equals(root.getNodeName())) { + throw new Exception("Encountered unknown element " + root.getNodeName()); + } + + final NodeList patternChildren = root.getChildNodes(); + for (int i = 0; i < patternChildren.getLength(); ++i) { + + final Node patternChild = patternChildren.item(i); + + if (patternChild instanceof org.w3c.dom.Text) { + continue; + } + + if (patternChild instanceof Element) { + + final Element patternElement = (Element) patternChild; + if ("name".equals(patternElement.getNodeName())) { + final String name = extractValueFromElement(patternElement); + if (jobFailurePattern != null) { + throw new Exception("Element name detected more than once in the file"); + } + + jobFailurePattern = new JobFailurePattern(name); + continue; + } + + if ("failures".equals(patternElement.getNodeName())) { + + if (jobFailurePattern == null) { + throw new Exception("Expected pattern name to be stored before the failure events"); + } + + final NodeList failuresChildren = patternElement.getChildNodes(); + for (int j = 0; j < failuresChildren.getLength(); ++j) { + + final Node failuresChild = failuresChildren.item(j); + + if (failuresChild instanceof org.w3c.dom.Text) { + continue; + } + + if (!(failuresChild instanceof Element)) { + throw new Exception("Expected type element as child of element 'failures'"); + } + + final Element failuresElement = (Element) failuresChild; + + if (!"failure".equals(failuresElement.getNodeName())) { + throw new Exception("Expected element 'failure' as child of element 'failures'"); + } + + final String type = failuresElement.getAttribute("type"); + if (type == null) { + throw new Exception("Element 'failure' lacks the attribute 'type'"); + } + + final boolean taskFailure = ("task".equals(type)); + String name = null; + String interval = null; + + final NodeList failureChildren = failuresElement.getChildNodes(); + for (int k = 0; k < failureChildren.getLength(); ++k) { + + final Node failureChild = failureChildren.item(k); + + if (failureChild instanceof org.w3c.dom.Text) { + continue; + } + + if (!(failureChild instanceof Element)) { + throw new Exception("Expected type element as child of element 'failure'"); + } + + final Element failureElement = (Element) failureChild; + if ("name".equals(failureElement.getNodeName())) { + name = extractValueFromElement(failureElement); + } + + if ("interval".equals(failureElement.getNodeName())) { + interval = extractValueFromElement(failureElement); + } + } + + if (name == null) { + throw new Exception("Could not find name for failure event " + j); + } + + if (interval == null) { + throw new Exception("Could not find interval for failure event " + j); + } + + int iv = 0; + try { + iv = Integer.parseInt(interval); + + } catch (NumberFormatException e) { + throw new Exception("Interval " + interval + " for failure event " + j + + " is not an integer number"); + } + + if (iv <= 0) { + throw new Exception("Interval for failure event " + j + + " must be greather than zero, but is " + iv); + } + + AbstractFailureEvent failureEvent = null; + if (taskFailure) { + failureEvent = new VertexFailureEvent(iv, name); + } else { + failureEvent = new InstanceFailureEvent(iv, name); + } + + jobFailurePattern.addOrUpdateEvent(failureEvent); + } + + continue; + } + + throw new Exception("Uncountered unecpted element " + patternElement.getNodeName()); + + } else { + throw new Exception("Encountered unexpected child of type " + patternChild.getClass()); + } + } + + } catch (Exception e) { + + final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); + messageBox.setText("Cannot load failure pattern"); + messageBox.setMessage(e.getMessage()); + messageBox.open(); + return null; + } finally { + if (inputStream != null) { + try { + inputStream.close(); + } catch (Exception e) { + } + } + } + + return jobFailurePattern; + } + + private String extractValueFromElement(final Element element) throws Exception { + + final NodeList children = element.getChildNodes(); + if (children.getLength() != 1) { + throw new Exception("Element " + element.getNodeName() + " has an unexpected number of children"); + } + + final Node child = children.item(0); + + if (!(child instanceof org.w3c.dom.Text)) { + throw new Exception("Expected child of element " + element.getNodeName() + " to be of type text"); + } + + org.w3c.dom.Text childText = (org.w3c.dom.Text) child; + + return childText.getTextContent(); + } + + private void displayFailurePattern(final JobFailurePattern jobFailurePattern) { + + final TableItem ti = new TableItem(this.taskFailureTable, SWT.LEFT); + ti.setText(0, "Test"); + ti.setText(1, "Test2"); + + + } + /** * {@inheritDoc} */ From dd4f8f9fd7040528d8663808d8e17d998517bbe9 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 11 Jan 2012 18:04:03 +0100 Subject: [PATCH 133/310] Started to implement failure event editor --- .../visualization/swt/JobFailurePattern.java | 32 +- .../swt/SWTFailureEventEditor.java | 300 ++++++++++++++++++ .../swt/SWTFailurePatternsManager.java | 123 +++++-- 3 files changed, 427 insertions(+), 28 deletions(-) create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java index 18ac5a16eae8a..bfe3aef29782b 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePattern.java @@ -15,6 +15,7 @@ package eu.stratosphere.nephele.visualization.swt; +import java.util.Iterator; import java.util.SortedSet; import java.util.TreeSet; @@ -67,12 +68,12 @@ public String getName() { } /** - * Adds or updates a failure event to this job failure pattern. + * Adds a failure event to this job failure pattern. * * @param event - * the event to be added or updated + * the event to be added */ - public void addOrUpdateEvent(final AbstractFailureEvent event) { + public void addEvent(final AbstractFailureEvent event) { if (event == null) { throw new IllegalArgumentException("Argument event must not be null"); @@ -84,4 +85,29 @@ public void addOrUpdateEvent(final AbstractFailureEvent event) { this.events.add(event); } + + /** + * Removes a failure event from this job failure pattern. + * + * @param event + * the event to be removed + */ + public void removeEvent(final AbstractFailureEvent event) { + + if (event == null) { + throw new IllegalArgumentException("Argument event must not be null"); + } + + this.events.remove(event); + } + + /** + * Returns an iterator to access all the events stored in this job failure pattern. + * + * @return an iterator to access all the events stored in this job failure pattern + */ + public Iterator iterator() { + + return this.events.iterator(); + } } diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java new file mode 100644 index 0000000000000..7eef7bdcf6910 --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java @@ -0,0 +1,300 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +import java.util.List; + +import org.eclipse.swt.SWT; +import org.eclipse.swt.events.KeyAdapter; +import org.eclipse.swt.events.KeyEvent; +import org.eclipse.swt.layout.GridData; +import org.eclipse.swt.layout.GridLayout; +import org.eclipse.swt.layout.RowLayout; +import org.eclipse.swt.widgets.Button; +import org.eclipse.swt.widgets.Composite; +import org.eclipse.swt.widgets.Display; +import org.eclipse.swt.widgets.Event; +import org.eclipse.swt.widgets.Label; +import org.eclipse.swt.widgets.Listener; +import org.eclipse.swt.widgets.MessageBox; +import org.eclipse.swt.widgets.Shell; +import org.eclipse.swt.widgets.Text; + +public final class SWTFailureEventEditor { + + /** + * The width of the dialog. + */ + private static final int WIDTH = 300; + + /** + * The height of the dialog. + */ + private static final int HEIGHT = 120; + + /** + * The shell for this dialog. + */ + private final Shell shell; + + /** + * Stores whether this editor is used to create/edit a task or an instance failure event. + */ + private final boolean isTaskEvent; + + /** + * The auto-completion combo box for the name. + */ + private final AutoCompletionCombo name; + + /** + * The text field for the interval. + */ + private final Text interval; + + /** + * The return value of the editor. + */ + private AbstractFailureEvent returnValue = null; + + public SWTFailureEventEditor(final Shell parent, final List nameSuggestions, final boolean isTaskEvent, + final AbstractFailureEvent failureEvent) { + + this.isTaskEvent = isTaskEvent; + + this.shell = new Shell(parent); + this.shell.setSize(WIDTH, HEIGHT); + + // Determine the correct title for the window + String title = null; + if (isTaskEvent) { + if (failureEvent == null) { + title = "Create new task failure event"; + } else { + title = "Edit task failure event"; + } + } else { + if (failureEvent == null) { + title = "Create new instance failure event"; + } else { + title = "Edit instance failure event"; + } + } + + this.shell.setText(title); + this.shell.setLayout(new GridLayout(2, false)); + + final GridData labelGridData = new GridData(); + labelGridData.horizontalAlignment = GridData.BEGINNING; + labelGridData.grabExcessHorizontalSpace = false; + labelGridData.grabExcessVerticalSpace = false; + + final Label nameLabel = new Label(this.shell, SWT.NONE); + if (isTaskEvent) { + nameLabel.setText("Task name:"); + } else { + nameLabel.setText("Instance name:"); + } + nameLabel.setLayoutData(labelGridData); + + final GridData fieldGridData = new GridData(); + fieldGridData.horizontalAlignment = GridData.FILL; + fieldGridData.grabExcessHorizontalSpace = true; + fieldGridData.grabExcessVerticalSpace = false; + + this.name = new AutoCompletionCombo(this.shell, SWT.NONE, nameSuggestions); + this.name.setLayoutData(fieldGridData); + this.name.addKeyListener(new KeyAdapter() { + + @Override + public void keyReleased(final KeyEvent arg0) { + + if (arg0.character != SWT.CR) { + return; + } + + if (isInputValid()) { + returnValue = assembleReturnValue(); + shell.dispose(); + } + } + }); + + final Label intervalLabel = new Label(this.shell, SWT.NONE); + intervalLabel.setText("Interval:"); + intervalLabel.setLayoutData(labelGridData); + + this.interval = new Text(this.shell, SWT.SINGLE | SWT.BORDER); + this.interval.setLayoutData(fieldGridData); + this.interval.addKeyListener(new KeyAdapter() { + + @Override + public void keyReleased(final KeyEvent arg0) { + + if (arg0.character != SWT.CR) { + return; + } + + if (isInputValid()) { + returnValue = assembleReturnValue(); + shell.dispose(); + } + } + }); + + final GridData buttonGridData = new GridData(); + buttonGridData.horizontalAlignment = SWT.RIGHT; + buttonGridData.verticalAlignment = SWT.BOTTOM; + buttonGridData.grabExcessHorizontalSpace = true; + buttonGridData.grabExcessVerticalSpace = true; + buttonGridData.horizontalSpan = 2; + + final Composite buttonComposite = new Composite(this.shell, SWT.RIGHT_TO_LEFT); + final RowLayout rowLayout = new RowLayout(SWT.HORIZONTAL); + rowLayout.marginBottom = 0; + rowLayout.marginHeight = 0; + rowLayout.marginLeft = 0; + rowLayout.marginRight = 0; + rowLayout.marginTop = 0; + rowLayout.marginWidth = 0; + rowLayout.pack = false; + + buttonComposite.setLayoutData(buttonGridData); + buttonComposite.setLayout(rowLayout); + + final Button ok = new Button(buttonComposite, SWT.PUSH); + ok.setText("OK"); + ok.addListener(SWT.Selection, new Listener() { + + @Override + public void handleEvent(final Event arg0) { + + if (isInputValid()) { + + returnValue = assembleReturnValue(); + shell.dispose(); + } + } + }); + + final Button cancel = new Button(buttonComposite, SWT.PUSH); + cancel.setText("Cancel"); + cancel.addListener(SWT.Selection, new Listener() { + + @Override + public void handleEvent(final Event arg0) { + + returnValue = null; + shell.dispose(); + } + }); + } + + private AbstractFailureEvent assembleReturnValue() { + + final String n = this.name.getText(); + final int iv = Integer.parseInt(this.interval.getText()); + + if (this.isTaskEvent) { + return new VertexFailureEvent(iv, n); + } else { + return new InstanceFailureEvent(iv, n); + } + } + + /** + * Checks whether the input is valid and displays an error message box if not. + * + * @return true if the input is valid, false otherwise + */ + private boolean isInputValid() { + + if (this.name.getText().isEmpty()) { + + final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); + messageBox.setText("Invalid Input"); + if (this.isTaskEvent) { + messageBox.setMessage("Task name must not be empty."); + } else { + messageBox.setMessage("Instance name must not be empty."); + } + messageBox.open(); + this.name.setFocus(); + + return false; + } + + final String intervalString = this.interval.getText(); + if (intervalString.isEmpty()) { + + final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); + messageBox.setText("Invalid Input"); + messageBox.setMessage("Interval must not be empty"); + messageBox.open(); + this.interval.setFocus(); + + return false; + } + + // Try parsing the interval number + int interval = -1; + try { + interval = Integer.parseInt(intervalString); + } catch (NumberFormatException nfe) { + + final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); + messageBox.setText("Invalid Input"); + messageBox.setMessage("Given interval is not an integer number"); + messageBox.open(); + this.interval.setFocus(); + + return false; + } + + if (interval <= 0) { + + final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); + messageBox.setText("Invalid Input"); + messageBox.setMessage("Given interval is must be greater than 0"); + messageBox.open(); + this.interval.setFocus(); + + return false; + } + + return true; + } + + /** + * Opens the dialog. + * + * @return the new {@link AbstractFailureEvent} created with the help of the dialog. + */ + public AbstractFailureEvent showDialog() { + + this.shell.open(); + + final Display display = this.shell.getDisplay(); + + while (!this.shell.isDisposed()) { + if (!display.readAndDispatch()) { + display.sleep(); + } + } + + return this.returnValue; + } +} diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java index 0cef86361ebd3..997b259d9bbf0 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java @@ -19,6 +19,7 @@ import java.io.InputStream; import java.util.ArrayList; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -33,9 +34,12 @@ import org.eclipse.swt.custom.SashForm; import org.eclipse.swt.events.MenuEvent; import org.eclipse.swt.events.MenuListener; +import org.eclipse.swt.events.MouseAdapter; +import org.eclipse.swt.events.MouseEvent; import org.eclipse.swt.events.SelectionAdapter; import org.eclipse.swt.events.SelectionEvent; import org.eclipse.swt.events.SelectionListener; +import org.eclipse.swt.graphics.Point; import org.eclipse.swt.layout.FillLayout; import org.eclipse.swt.layout.GridData; import org.eclipse.swt.layout.GridLayout; @@ -68,6 +72,8 @@ public final class SWTFailurePatternsManager implements SelectionListener { private static final int HEIGHT = 400; + private static final int COLUMN_WIDTH = 200; + private final Shell shell; private final Tree jobTree; @@ -133,28 +139,13 @@ public final class SWTFailurePatternsManager implements SelectionListener { horizontalSash.setWeights(new int[] { 2, 8 }); - gridData = new GridData(SWT.FILL, SWT.FILL, true, true); - gridData.heightHint = 200; - - this.taskFailureTable = new Table(this.jobTabFolder, SWT.MULTI | SWT.BORDER | SWT.VIRTUAL); - this.taskFailureTable.setLayoutData(gridData); - this.taskFailureTable.setLinesVisible(true); - this.taskFailureTable.setHeaderVisible(true); - this.taskFailureTable.setSize(200, 200); - new TableColumn(this.taskFailureTable, SWT.LEFT).setText("Task name"); - new TableColumn(this.taskFailureTable, SWT.LEFT).setText("Failure time"); - - this.instanceFailureTable = new Table(this.jobTabFolder, SWT.MULTI | SWT.BORDER | SWT.VIRTUAL); - this.instanceFailureTable.setLayoutData(gridData); - this.instanceFailureTable.setVisible(true); - this.instanceFailureTable.setLinesVisible(true); - this.instanceFailureTable.setHeaderVisible(true); - new TableColumn(this.instanceFailureTable, SWT.LEFT).setText("Instance name"); - new TableColumn(this.instanceFailureTable, SWT.LEFT).setText("Failure time"); - + // Create task failure table + this.taskFailureTable = createTaskOrInstanceFailureTable(true); this.taskFailurePatternsTab.setText("Task Failure Patterns"); this.taskFailurePatternsTab.setControl(this.taskFailureTable); + // Create instance failure table + this.instanceFailureTable = createTaskOrInstanceFailureTable(false); this.instanceFailurePatternsTab.setText("Instance Failure Patterns"); this.instanceFailurePatternsTab.setControl(this.instanceFailureTable); @@ -177,11 +168,57 @@ public final class SWTFailurePatternsManager implements SelectionListener { gridData.horizontalAlignment = SWT.RIGHT; closeButton.setLayoutData(gridData); + // Initialize the tables + displayFailurePattern(null); + } + + private Table createTaskOrInstanceFailureTable(final boolean isTaskTable) { + + final Table table = new Table(this.jobTabFolder, SWT.BORDER | SWT.MULTI); + table.setHeaderVisible(true); + table.setLinesVisible(true); + + if (isTaskTable) { + new TableColumn(table, SWT.NONE).setText("Task name"); + } else { + new TableColumn(table, SWT.NONE).setText("Instance name"); + } + new TableColumn(table, SWT.NONE).setText("Interval"); + + for (int i = 0; i < table.getColumnCount(); ++i) { + table.getColumn(i).setWidth(COLUMN_WIDTH); + } + + table.addMouseListener(new MouseAdapter() { + + @Override + public void mouseDoubleClick(final MouseEvent arg0) { + + final TableItem ti = table.getItem(new Point(arg0.x, arg0.y)); + if (ti == null) { + return; + } + + final List suggestions = new ArrayList(); // TODO: Compute proper same suggestions here + + final AbstractFailureEvent oldEvent = (AbstractFailureEvent) ti.getData(); + + final SWTFailureEventEditor editor = new SWTFailureEventEditor(shell, suggestions, isTaskTable, + oldEvent); + + final AbstractFailureEvent newEvent = editor.showDialog(); + + + } + }); + + return table; } public void open() { this.shell.open(); + } private Menu createTreeContextMenu() { @@ -446,7 +483,7 @@ private JobFailurePattern loadFailurePatternFromFile(final String filename) { failureEvent = new InstanceFailureEvent(iv, name); } - jobFailurePattern.addOrUpdateEvent(failureEvent); + jobFailurePattern.addEvent(failureEvent); } continue; @@ -498,11 +535,47 @@ private String extractValueFromElement(final Element element) throws Exception { private void displayFailurePattern(final JobFailurePattern jobFailurePattern) { - final TableItem ti = new TableItem(this.taskFailureTable, SWT.LEFT); - ti.setText(0, "Test"); - ti.setText(1, "Test2"); - - + // Clear old content from tables + this.taskFailureTable.clearAll(); + this.instanceFailureTable.clearAll(); + + if (jobFailurePattern == null) { + this.taskFailureTable.setEnabled(false); + this.instanceFailureTable.setEnabled(false); + return; + } + + this.taskFailureTable.setEnabled(true); + this.instanceFailureTable.setEnabled(true); + + final Iterator it = jobFailurePattern.iterator(); + while (it.hasNext()) { + + final AbstractFailureEvent event = it.next(); + if (event instanceof VertexFailureEvent) { + + final VertexFailureEvent vfe = (VertexFailureEvent) event; + final TableItem ti = new TableItem(this.instanceFailureTable, SWT.NONE); + ti.setText(0, vfe.getVertexName()); + ti.setText(1, Integer.toString(vfe.getInterval())); + ti.setData(vfe); + + } else if (event instanceof InstanceFailureEvent) { + + final InstanceFailureEvent ife = (InstanceFailureEvent) event; + final TableItem ti = new TableItem(this.instanceFailureTable, SWT.NONE); + ti.setText(0, ife.getInstanceName()); + ti.setText(1, Integer.toString(ife.getInterval())); + ti.setData(ife); + + } else { + LOG.error("Encountered unknown failure event " + event.getClass()); + } + } + + // Finally, add item to create new entry in both tables + new TableItem(this.taskFailureTable, SWT.NONE); + new TableItem(this.instanceFailureTable, SWT.NONE); } /** From 0b24ebb32389dc35caa9e8856a9d2b38c4bd36de Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 11 Jan 2012 19:11:53 +0100 Subject: [PATCH 134/310] Finished implementation of failure event editor --- .../swt/AbstractFailureEvent.java | 24 ++- .../swt/AutoCompletionCombo.java | 13 ++ .../swt/InstanceFailureEvent.java | 30 +--- .../swt/SWTFailureEventEditor.java | 97 ++++++++---- .../swt/SWTFailurePatternsManager.java | 144 +++++++++--------- .../visualization/swt/VertexFailureEvent.java | 28 +--- 6 files changed, 177 insertions(+), 159 deletions(-) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AbstractFailureEvent.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AbstractFailureEvent.java index 857dbd4d08be7..08b50074d4e25 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AbstractFailureEvent.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AbstractFailureEvent.java @@ -25,6 +25,11 @@ */ public abstract class AbstractFailureEvent implements Comparable { + /** + * The name of the event. + */ + private final String name; + /** * The interval in milliseconds until this event will occur. */ @@ -33,15 +38,22 @@ public abstract class AbstractFailureEvent implements Comparable nameSuggestions, final boolean isTaskEvent, + public SWTFailureEventEditor(final Shell parent, final List nameSuggestions, final AbstractFailureEvent failureEvent) { - this.isTaskEvent = isTaskEvent; - this.shell = new Shell(parent); this.shell.setSize(WIDTH, HEIGHT); // Determine the correct title for the window String title = null; - if (isTaskEvent) { - if (failureEvent == null) { - title = "Create new task failure event"; - } else { - title = "Edit task failure event"; - } + if (failureEvent == null) { + title = "Create new failure event"; } else { - if (failureEvent == null) { - title = "Create new instance failure event"; - } else { - title = "Edit instance failure event"; - } + title = "Edit failure event"; } this.shell.setText(title); @@ -101,13 +97,11 @@ public SWTFailureEventEditor(final Shell parent, final List nameSuggesti labelGridData.horizontalAlignment = GridData.BEGINNING; labelGridData.grabExcessHorizontalSpace = false; labelGridData.grabExcessVerticalSpace = false; + labelGridData.widthHint = 50; + labelGridData.minimumWidth = 50; final Label nameLabel = new Label(this.shell, SWT.NONE); - if (isTaskEvent) { - nameLabel.setText("Task name:"); - } else { - nameLabel.setText("Instance name:"); - } + nameLabel.setText("Name:"); nameLabel.setLayoutData(labelGridData); final GridData fieldGridData = new GridData(); @@ -132,6 +126,9 @@ public void keyReleased(final KeyEvent arg0) { } } }); + if (failureEvent != null) { + this.name.setText(failureEvent.getName()); + } final Label intervalLabel = new Label(this.shell, SWT.NONE); intervalLabel.setText("Interval:"); @@ -154,6 +151,39 @@ public void keyReleased(final KeyEvent arg0) { } } }); + if (failureEvent != null) { + this.interval.setText(Integer.toString(failureEvent.getInterval())); + } + + final GridData groupGridData = new GridData(); + groupGridData.horizontalAlignment = GridData.FILL; + groupGridData.grabExcessHorizontalSpace = true; + groupGridData.grabExcessVerticalSpace = false; + groupGridData.horizontalSpan = 2; + + final Group typeGroup = new Group(this.shell, SWT.BORDER); + typeGroup.setText("Event type"); + typeGroup.setLayoutData(groupGridData); + typeGroup.setLayout(new GridLayout(1, true)); + + this.taskFailureButton = new Button(typeGroup, SWT.RADIO); + this.taskFailureButton.setText("Task failure"); + + this.instanceFailureButton = new Button(typeGroup, SWT.RADIO); + this.instanceFailureButton.setText("Instance failure"); + + if (failureEvent == null) { + this.taskFailureButton.setSelection(true); + this.instanceFailureButton.setSelection(false); + } else { + if (failureEvent instanceof VertexFailureEvent) { + this.taskFailureButton.setSelection(true); + this.instanceFailureButton.setSelection(false); + } else { + this.taskFailureButton.setSelection(false); + this.instanceFailureButton.setSelection(true); + } + } final GridData buttonGridData = new GridData(); buttonGridData.horizontalAlignment = SWT.RIGHT; @@ -203,15 +233,20 @@ public void handleEvent(final Event arg0) { }); } + /** + * Constructs the dialog's return value from the current dialog settings. + * + * @return the dialog's return value + */ private AbstractFailureEvent assembleReturnValue() { final String n = this.name.getText(); final int iv = Integer.parseInt(this.interval.getText()); - if (this.isTaskEvent) { - return new VertexFailureEvent(iv, n); + if (this.taskFailureButton.getSelection()) { + return new VertexFailureEvent(n, iv); } else { - return new InstanceFailureEvent(iv, n); + return new InstanceFailureEvent(n, iv); } } @@ -226,11 +261,7 @@ private boolean isInputValid() { final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); messageBox.setText("Invalid Input"); - if (this.isTaskEvent) { - messageBox.setMessage("Task name must not be empty."); - } else { - messageBox.setMessage("Instance name must not be empty."); - } + messageBox.setMessage("Name must not be empty."); messageBox.open(); this.name.setFocus(); diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java index 997b259d9bbf0..6baf46154d23e 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java @@ -29,8 +29,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.eclipse.swt.SWT; -import org.eclipse.swt.custom.CTabFolder; -import org.eclipse.swt.custom.CTabItem; import org.eclipse.swt.custom.SashForm; import org.eclipse.swt.events.MenuEvent; import org.eclipse.swt.events.MenuListener; @@ -72,24 +70,20 @@ public final class SWTFailurePatternsManager implements SelectionListener { private static final int HEIGHT = 400; - private static final int COLUMN_WIDTH = 200; + private static final int ICON_COLUMN_WIDTH = 20; + + private static final int TEXT_COLUMN_WIDTH = 200; private final Shell shell; private final Tree jobTree; - private final CTabFolder jobTabFolder; - - private final CTabItem taskFailurePatternsTab; - - private final Table taskFailureTable; - - private final CTabItem instanceFailurePatternsTab; - - private final Table instanceFailureTable; + private final Table failureEventTable; private final Map failurePatterns = new HashMap(); + private JobFailurePattern selectedFailurePattern = null; + SWTFailurePatternsManager(final Shell parent) { // Set size @@ -127,28 +121,9 @@ public final class SWTFailurePatternsManager implements SelectionListener { this.jobTree.addSelectionListener(this); this.jobTree.setMenu(createTreeContextMenu()); - this.jobTabFolder = new CTabFolder(horizontalSash, SWT.TOP); - this.jobTabFolder.setLayout(new GridLayout()); - this.jobTabFolder.addSelectionListener(this); - - this.taskFailurePatternsTab = new CTabItem(this.jobTabFolder, SWT.NONE); - - this.instanceFailurePatternsTab = new CTabItem(this.jobTabFolder, SWT.NONE); - - this.jobTabFolder.setSelection(this.taskFailurePatternsTab); - + this.failureEventTable = createFailureEventTable(horizontalSash); horizontalSash.setWeights(new int[] { 2, 8 }); - // Create task failure table - this.taskFailureTable = createTaskOrInstanceFailureTable(true); - this.taskFailurePatternsTab.setText("Task Failure Patterns"); - this.taskFailurePatternsTab.setControl(this.taskFailureTable); - - // Create instance failure table - this.instanceFailureTable = createTaskOrInstanceFailureTable(false); - this.instanceFailurePatternsTab.setText("Instance Failure Patterns"); - this.instanceFailurePatternsTab.setControl(this.instanceFailureTable); - final Composite buttonComposite = new Composite(this.shell, SWT.NONE); buttonComposite.setLayout(new GridLayout(2, false)); gridData = new GridData(); @@ -172,21 +147,22 @@ public final class SWTFailurePatternsManager implements SelectionListener { displayFailurePattern(null); } - private Table createTaskOrInstanceFailureTable(final boolean isTaskTable) { + private Table createFailureEventTable(final Composite parent) { - final Table table = new Table(this.jobTabFolder, SWT.BORDER | SWT.MULTI); + final Table table = new Table(parent, SWT.BORDER | SWT.MULTI); table.setHeaderVisible(true); table.setLinesVisible(true); - if (isTaskTable) { - new TableColumn(table, SWT.NONE).setText("Task name"); - } else { - new TableColumn(table, SWT.NONE).setText("Instance name"); - } + new TableColumn(table, SWT.NONE); + new TableColumn(table, SWT.NONE).setText("Task name"); new TableColumn(table, SWT.NONE).setText("Interval"); for (int i = 0; i < table.getColumnCount(); ++i) { - table.getColumn(i).setWidth(COLUMN_WIDTH); + if (i == 0) { + table.getColumn(i).setWidth(ICON_COLUMN_WIDTH); + } else { + table.getColumn(i).setWidth(TEXT_COLUMN_WIDTH); + } } table.addMouseListener(new MouseAdapter() { @@ -195,26 +171,63 @@ private Table createTaskOrInstanceFailureTable(final boolean isTaskTable) { public void mouseDoubleClick(final MouseEvent arg0) { final TableItem ti = table.getItem(new Point(arg0.x, arg0.y)); - if (ti == null) { + + if (selectedFailurePattern == null) { return; } final List suggestions = new ArrayList(); // TODO: Compute proper same suggestions here - final AbstractFailureEvent oldEvent = (AbstractFailureEvent) ti.getData(); + AbstractFailureEvent oldEvent = null; + if (ti != null) { + oldEvent = (AbstractFailureEvent) ti.getData(); + } - final SWTFailureEventEditor editor = new SWTFailureEventEditor(shell, suggestions, isTaskTable, - oldEvent); + final SWTFailureEventEditor editor = new SWTFailureEventEditor(shell, suggestions, oldEvent); final AbstractFailureEvent newEvent = editor.showDialog(); + if (newEvent == null) { + return; + } - + if (oldEvent != null) { + selectedFailurePattern.removeEvent(oldEvent); + } + selectedFailurePattern.addEvent(newEvent); + + updateTableItem(ti, newEvent); } }); return table; } + private void updateTableItem(TableItem ti, final AbstractFailureEvent event) { + + boolean newItemCreated = false; + + if (ti == null) { + ti = new TableItem(this.failureEventTable, SWT.NONE, this.failureEventTable.getItemCount() - 1); + newItemCreated = true; + } + + if (event instanceof VertexFailureEvent) { + ti.setText(0, "T"); + } else { + ti.setText(0, "I"); + } + + ti.setText(1, event.getName()); + ti.setText(2, Integer.toString(event.getInterval())); + + // Add new blank item if the old one has been used to create the new event + if (ti.getData() == null && !newItemCreated) { + new TableItem(this.failureEventTable, SWT.NONE); + } + + ti.setData(event); + } + public void open() { this.shell.open(); @@ -478,9 +491,9 @@ private JobFailurePattern loadFailurePatternFromFile(final String filename) { AbstractFailureEvent failureEvent = null; if (taskFailure) { - failureEvent = new VertexFailureEvent(iv, name); + failureEvent = new VertexFailureEvent(name, iv); } else { - failureEvent = new InstanceFailureEvent(iv, name); + failureEvent = new InstanceFailureEvent(name, iv); } jobFailurePattern.addEvent(failureEvent); @@ -535,47 +548,28 @@ private String extractValueFromElement(final Element element) throws Exception { private void displayFailurePattern(final JobFailurePattern jobFailurePattern) { - // Clear old content from tables - this.taskFailureTable.clearAll(); - this.instanceFailureTable.clearAll(); + // Clear old content from event table + this.failureEventTable.clearAll(); if (jobFailurePattern == null) { - this.taskFailureTable.setEnabled(false); - this.instanceFailureTable.setEnabled(false); + this.failureEventTable.setEnabled(false); return; } - this.taskFailureTable.setEnabled(true); - this.instanceFailureTable.setEnabled(true); + this.failureEventTable.setEnabled(true); final Iterator it = jobFailurePattern.iterator(); while (it.hasNext()) { final AbstractFailureEvent event = it.next(); - if (event instanceof VertexFailureEvent) { - - final VertexFailureEvent vfe = (VertexFailureEvent) event; - final TableItem ti = new TableItem(this.instanceFailureTable, SWT.NONE); - ti.setText(0, vfe.getVertexName()); - ti.setText(1, Integer.toString(vfe.getInterval())); - ti.setData(vfe); - - } else if (event instanceof InstanceFailureEvent) { - - final InstanceFailureEvent ife = (InstanceFailureEvent) event; - final TableItem ti = new TableItem(this.instanceFailureTable, SWT.NONE); - ti.setText(0, ife.getInstanceName()); - ti.setText(1, Integer.toString(ife.getInterval())); - ti.setData(ife); - - } else { - LOG.error("Encountered unknown failure event " + event.getClass()); - } + final TableItem ti = new TableItem(this.failureEventTable, SWT.NONE); + updateTableItem(ti, event); } // Finally, add item to create new entry in both tables - new TableItem(this.taskFailureTable, SWT.NONE); - new TableItem(this.instanceFailureTable, SWT.NONE); + new TableItem(this.failureEventTable, SWT.NONE); + + this.selectedFailurePattern = jobFailurePattern; } /** diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/VertexFailureEvent.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/VertexFailureEvent.java index 94842dd8978f8..5734035581be2 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/VertexFailureEvent.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/VertexFailureEvent.java @@ -25,36 +25,16 @@ */ public final class VertexFailureEvent extends AbstractFailureEvent { - /** - * The name of the vertex to be killed. - */ - private final String vertexName; - /** * Constructs a new vertex failure event * - * @param interval - * the interval in milliseconds until this event will occur * @param vertexName * the name of the vertex to be killed + * @param interval + * the interval in milliseconds until this event will occur */ - VertexFailureEvent(final int interval, final String vertexName) { - super(interval); - - if (vertexName == null) { - throw new IllegalArgumentException("Argument vertexName must not be null"); - } - - this.vertexName = vertexName; - } - - /** - * Returns the name of the vertex to be killed. - * - * @return the name of the vertex to be killed - */ - public String getVertexName() { + VertexFailureEvent(final String vertexName, final int interval) { + super(vertexName, interval); - return this.vertexName; } } From 8012a61610323515a7729faa8a31cd06020ec833 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 11 Jan 2012 20:18:56 +0100 Subject: [PATCH 135/310] Implemented sorting for failure event table --- .../swt/SWTFailurePatternsManager.java | 128 +++++++++++++----- 1 file changed, 97 insertions(+), 31 deletions(-) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java index 6baf46154d23e..e2da8dd4e0d87 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java @@ -18,6 +18,8 @@ import java.io.FileInputStream; import java.io.InputStream; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -36,16 +38,17 @@ import org.eclipse.swt.events.MouseEvent; import org.eclipse.swt.events.SelectionAdapter; import org.eclipse.swt.events.SelectionEvent; -import org.eclipse.swt.events.SelectionListener; import org.eclipse.swt.graphics.Point; import org.eclipse.swt.layout.FillLayout; import org.eclipse.swt.layout.GridData; import org.eclipse.swt.layout.GridLayout; import org.eclipse.swt.widgets.Button; import org.eclipse.swt.widgets.Composite; +import org.eclipse.swt.widgets.Event; import org.eclipse.swt.widgets.FileDialog; import org.eclipse.swt.widgets.Group; import org.eclipse.swt.widgets.Label; +import org.eclipse.swt.widgets.Listener; import org.eclipse.swt.widgets.Menu; import org.eclipse.swt.widgets.MenuItem; import org.eclipse.swt.widgets.MessageBox; @@ -62,7 +65,7 @@ import eu.stratosphere.nephele.jobgraph.JobID; -public final class SWTFailurePatternsManager implements SelectionListener { +public final class SWTFailurePatternsManager extends SelectionAdapter { private static final Log LOG = LogFactory.getLog(SWTFailurePatternsManager.class); @@ -153,9 +156,11 @@ private Table createFailureEventTable(final Composite parent) { table.setHeaderVisible(true); table.setLinesVisible(true); - new TableColumn(table, SWT.NONE); - new TableColumn(table, SWT.NONE).setText("Task name"); - new TableColumn(table, SWT.NONE).setText("Interval"); + final TableColumn iconColumn = new TableColumn(table, SWT.NONE); + final TableColumn nameColumn = new TableColumn(table, SWT.NONE); + nameColumn.setText("Name"); + final TableColumn intervalColumn = new TableColumn(table, SWT.NONE); + intervalColumn.setText("Interval"); for (int i = 0; i < table.getColumnCount(); ++i) { if (i == 0) { @@ -165,6 +170,7 @@ private Table createFailureEventTable(final Composite parent) { } } + // Implement listener to add and update events table.addMouseListener(new MouseAdapter() { @Override @@ -199,6 +205,79 @@ public void mouseDoubleClick(final MouseEvent arg0) { } }); + // Implement sorting of columns + final Listener sortListener = new Listener() { + + @Override + public void handleEvent(final Event arg0) { + + final TableColumn sortColumn = failureEventTable.getSortColumn(); + final TableColumn currentColumn = (TableColumn) arg0.widget; + int dir = failureEventTable.getSortDirection(); + if (sortColumn == currentColumn) { + dir = (dir == SWT.UP) ? SWT.DOWN : SWT.UP; + } else { + table.setSortColumn(currentColumn); + dir = SWT.UP; + } + + final int direction = dir; + final AbstractFailureEvent[] failureEvents = new AbstractFailureEvent[table.getItemCount()]; + for (int i = 0; i < table.getItemCount(); ++i) { + failureEvents[i] = (AbstractFailureEvent) failureEventTable.getItem(i).getData(); + } + Arrays.sort(failureEvents, new Comparator() { + + @Override + public int compare(final AbstractFailureEvent o1, AbstractFailureEvent o2) { + + if (o1 == null) { + return -1; + } + + if (o2 == null) { + return 1; + } + + if (currentColumn == iconColumn) { + + final int v1 = (o1 instanceof VertexFailureEvent) ? 0 : 1; + final int v2 = (o2 instanceof VertexFailureEvent) ? 0 : 1; + return (direction == SWT.UP) ? (v1 - v2) : (v2 - v1); + + } else if (currentColumn == nameColumn) { + + if (direction == SWT.UP) { + return String.CASE_INSENSITIVE_ORDER.compare(o1.getName(), o2.getName()); + } else { + return String.CASE_INSENSITIVE_ORDER.compare(o2.getName(), o1.getName()); + } + + } else { + + if (direction == SWT.UP) { + return (o1.getInterval() - o2.getInterval()); + } else { + return (o2.getInterval() - o1.getInterval()); + } + } + } + }); + + failureEventTable.removeAll(); + for (int i = 0; i < failureEvents.length; ++i) { + updateTableItem(null, failureEvents[i]); + } + + failureEventTable.setSortColumn(currentColumn); + failureEventTable.setSortDirection(direction); + } + }; + + iconColumn.addListener(SWT.Selection, sortListener); + nameColumn.addListener(SWT.Selection, sortListener); + intervalColumn.addListener(SWT.Selection, sortListener); + return table; } @@ -207,18 +286,23 @@ private void updateTableItem(TableItem ti, final AbstractFailureEvent event) { boolean newItemCreated = false; if (ti == null) { - ti = new TableItem(this.failureEventTable, SWT.NONE, this.failureEventTable.getItemCount() - 1); + + final int index = (failureEventTable.getItemCount() == 0) ? 0 : (this.failureEventTable.getItemCount() - 1); + + ti = new TableItem(this.failureEventTable, SWT.NONE, index); newItemCreated = true; } - if (event instanceof VertexFailureEvent) { - ti.setText(0, "T"); - } else { - ti.setText(0, "I"); - } + if (event != null) { + if (event instanceof VertexFailureEvent) { + ti.setText(0, "T"); + } else { + ti.setText(0, "I"); + } - ti.setText(1, event.getName()); - ti.setText(2, Integer.toString(event.getInterval())); + ti.setText(1, event.getName()); + ti.setText(2, Integer.toString(event.getInterval())); + } // Add new blank item if the old one has been used to create the new event if (ti.getData() == null && !newItemCreated) { @@ -572,24 +656,6 @@ private void displayFailurePattern(final JobFailurePattern jobFailurePattern) { this.selectedFailurePattern = jobFailurePattern; } - /** - * {@inheritDoc} - */ - @Override - public void widgetDefaultSelected(final SelectionEvent arg0) { - // TODO Auto-generated method stub - - } - - /** - * {@inheritDoc} - */ - @Override - public void widgetSelected(final SelectionEvent arg0) { - // TODO Auto-generated method stub - - } - public void startFailurePattern(final JobID jobID, final String jobName, final long referenceTime) { final JobFailurePattern failurePattern = this.failurePatterns.get(jobName.toLowerCase()); From 8df9557bf8fa9b159a0166d7918a00c1c1d00b70 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 11 Jan 2012 20:51:00 +0100 Subject: [PATCH 136/310] Refactored failure patterns editor --- .../swt/SWTFailurePatternsEditor.java | 655 ++++++++++++++++++ .../swt/SWTFailurePatternsManager.java | 637 +---------------- .../swt/SWTVisualizationGUI.java | 10 +- 3 files changed, 675 insertions(+), 627 deletions(-) create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java new file mode 100644 index 0000000000000..8f6aafb2374c5 --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java @@ -0,0 +1,655 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +import java.io.FileInputStream; +import java.io.InputStream; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; + +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; + +import org.eclipse.swt.SWT; +import org.eclipse.swt.custom.SashForm; +import org.eclipse.swt.events.MenuAdapter; +import org.eclipse.swt.events.MenuEvent; +import org.eclipse.swt.events.MouseAdapter; +import org.eclipse.swt.events.MouseEvent; +import org.eclipse.swt.events.SelectionAdapter; +import org.eclipse.swt.events.SelectionEvent; +import org.eclipse.swt.graphics.Point; +import org.eclipse.swt.layout.FillLayout; +import org.eclipse.swt.layout.GridData; +import org.eclipse.swt.layout.GridLayout; +import org.eclipse.swt.widgets.Button; +import org.eclipse.swt.widgets.Composite; +import org.eclipse.swt.widgets.Display; +import org.eclipse.swt.widgets.Event; +import org.eclipse.swt.widgets.FileDialog; +import org.eclipse.swt.widgets.Group; +import org.eclipse.swt.widgets.Label; +import org.eclipse.swt.widgets.Listener; +import org.eclipse.swt.widgets.Menu; +import org.eclipse.swt.widgets.MenuItem; +import org.eclipse.swt.widgets.MessageBox; +import org.eclipse.swt.widgets.Shell; +import org.eclipse.swt.widgets.Table; +import org.eclipse.swt.widgets.TableColumn; +import org.eclipse.swt.widgets.TableItem; +import org.eclipse.swt.widgets.Tree; +import org.eclipse.swt.widgets.TreeItem; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; + +public final class SWTFailurePatternsEditor extends SelectionAdapter { + + private static final int WIDTH = 800; + + private static final int HEIGHT = 400; + + private static final int ICON_COLUMN_WIDTH = 20; + + private static final int TEXT_COLUMN_WIDTH = 200; + + private final Shell shell; + + private final Tree jobTree; + + private final Table failureEventTable; + + private final List jobSuggestions; + + private final List nameSuggestions; + + private JobFailurePattern selectedFailurePattern = null; + + SWTFailurePatternsEditor(final Shell parent, final List jobSuggestions, final List nameSuggestions) { + + this.jobSuggestions = jobSuggestions; + this.nameSuggestions = nameSuggestions; + + // Set size + this.shell = new Shell(parent); + this.shell.setSize(WIDTH, HEIGHT); + this.shell.setText("Manage Failure Patterns"); + GridLayout gl = new GridLayout(1, false); + gl.horizontalSpacing = 0; + gl.verticalSpacing = 0; + gl.marginRight = 0; + gl.marginLeft = 0; + gl.marginBottom = 0; + gl.marginTop = 0; + gl.marginHeight = 0; + gl.marginWidth = 0; + this.shell.setLayout(gl); + + final Composite mainComposite = new Composite(this.shell, SWT.NONE); + mainComposite.setLayout(new GridLayout(1, false)); + GridData gridData = new GridData(); + gridData.horizontalAlignment = GridData.FILL; + gridData.verticalAlignment = GridData.FILL; + gridData.grabExcessHorizontalSpace = true; + gridData.grabExcessVerticalSpace = true; + mainComposite.setLayoutData(gridData); + + final SashForm horizontalSash = new SashForm(mainComposite, SWT.HORIZONTAL); + horizontalSash.setLayoutData(new GridData(GridData.FILL_BOTH)); + + final Group jobGroup = new Group(horizontalSash, SWT.NONE); + jobGroup.setText("Job Failure Patterns"); + jobGroup.setLayout(new FillLayout()); + + this.jobTree = new Tree(jobGroup, SWT.SINGLE | SWT.BORDER); + this.jobTree.addSelectionListener(this); + this.jobTree.setMenu(createTreeContextMenu()); + + this.failureEventTable = createFailureEventTable(horizontalSash); + horizontalSash.setWeights(new int[] { 2, 8 }); + + final Composite buttonComposite = new Composite(this.shell, SWT.NONE); + buttonComposite.setLayout(new GridLayout(2, false)); + gridData = new GridData(); + gridData.horizontalAlignment = GridData.FILL; + buttonComposite.setLayoutData(gridData); + + final Label fillLabel = new Label(buttonComposite, SWT.NONE); + gridData = new GridData(); + gridData.horizontalAlignment = GridData.FILL; + gridData.grabExcessHorizontalSpace = true; + gridData.grabExcessVerticalSpace = false; + fillLabel.setLayoutData(gridData); + + final Button closeButton = new Button(buttonComposite, SWT.PUSH); + closeButton.setText("Close"); + gridData = new GridData(); + gridData.horizontalAlignment = SWT.RIGHT; + closeButton.setLayoutData(gridData); + + // Initialize the tables + displayFailurePattern(null); + } + + private Table createFailureEventTable(final Composite parent) { + + final Table table = new Table(parent, SWT.BORDER | SWT.MULTI); + table.setHeaderVisible(true); + table.setLinesVisible(true); + + final TableColumn iconColumn = new TableColumn(table, SWT.NONE); + final TableColumn nameColumn = new TableColumn(table, SWT.NONE); + nameColumn.setText("Name"); + final TableColumn intervalColumn = new TableColumn(table, SWT.NONE); + intervalColumn.setText("Interval"); + + for (int i = 0; i < table.getColumnCount(); ++i) { + if (i == 0) { + table.getColumn(i).setWidth(ICON_COLUMN_WIDTH); + } else { + table.getColumn(i).setWidth(TEXT_COLUMN_WIDTH); + } + } + + // Implement listener to add and update events + table.addMouseListener(new MouseAdapter() { + + @Override + public void mouseDoubleClick(final MouseEvent arg0) { + + final TableItem ti = table.getItem(new Point(arg0.x, arg0.y)); + + if (selectedFailurePattern == null) { + return; + } + + AbstractFailureEvent oldEvent = null; + if (ti != null) { + oldEvent = (AbstractFailureEvent) ti.getData(); + } + + final SWTFailureEventEditor editor = new SWTFailureEventEditor(shell, nameSuggestions, oldEvent); + + final AbstractFailureEvent newEvent = editor.showDialog(); + if (newEvent == null) { + return; + } + + if (oldEvent != null) { + selectedFailurePattern.removeEvent(oldEvent); + } + selectedFailurePattern.addEvent(newEvent); + + updateTableItem(ti, newEvent); + } + }); + + // Implement sorting of columns + final Listener sortListener = new Listener() { + + @Override + public void handleEvent(final Event arg0) { + + final TableColumn sortColumn = failureEventTable.getSortColumn(); + final TableColumn currentColumn = (TableColumn) arg0.widget; + int dir = failureEventTable.getSortDirection(); + if (sortColumn == currentColumn) { + dir = (dir == SWT.UP) ? SWT.DOWN : SWT.UP; + } else { + table.setSortColumn(currentColumn); + dir = SWT.UP; + } + + final int direction = dir; + final AbstractFailureEvent[] failureEvents = new AbstractFailureEvent[table.getItemCount()]; + for (int i = 0; i < table.getItemCount(); ++i) { + failureEvents[i] = (AbstractFailureEvent) failureEventTable.getItem(i).getData(); + } + Arrays.sort(failureEvents, new Comparator() { + + @Override + public int compare(final AbstractFailureEvent o1, AbstractFailureEvent o2) { + + if (o1 == null) { + return -1; + } + + if (o2 == null) { + return 1; + } + + if (currentColumn == iconColumn) { + + final int v1 = (o1 instanceof VertexFailureEvent) ? 0 : 1; + final int v2 = (o2 instanceof VertexFailureEvent) ? 0 : 1; + return (direction == SWT.UP) ? (v1 - v2) : (v2 - v1); + + } else if (currentColumn == nameColumn) { + + if (direction == SWT.UP) { + return String.CASE_INSENSITIVE_ORDER.compare(o1.getName(), o2.getName()); + } else { + return String.CASE_INSENSITIVE_ORDER.compare(o2.getName(), o1.getName()); + } + + } else { + + if (direction == SWT.UP) { + return (o1.getInterval() - o2.getInterval()); + } else { + return (o2.getInterval() - o1.getInterval()); + } + } + } + }); + + failureEventTable.removeAll(); + for (int i = 0; i < failureEvents.length; ++i) { + updateTableItem(null, failureEvents[i]); + } + + failureEventTable.setSortColumn(currentColumn); + failureEventTable.setSortDirection(direction); + } + }; + + iconColumn.addListener(SWT.Selection, sortListener); + nameColumn.addListener(SWT.Selection, sortListener); + intervalColumn.addListener(SWT.Selection, sortListener); + + return table; + } + + private void updateTableItem(TableItem ti, final AbstractFailureEvent event) { + + boolean newItemCreated = false; + + if (ti == null) { + + final int index = (failureEventTable.getItemCount() == 0) ? 0 : (this.failureEventTable.getItemCount() - 1); + + ti = new TableItem(this.failureEventTable, SWT.NONE, index); + newItemCreated = true; + } + + if (event != null) { + if (event instanceof VertexFailureEvent) { + ti.setText(0, "T"); + } else { + ti.setText(0, "I"); + } + + ti.setText(1, event.getName()); + ti.setText(2, Integer.toString(event.getInterval())); + } + + // Add new blank item if the old one has been used to create the new event + if (ti.getData() == null && !newItemCreated) { + new TableItem(this.failureEventTable, SWT.NONE); + } + + ti.setData(event); + } + + public List show() { + + this.shell.open(); + + final Display display = this.shell.getDisplay(); + + while (!this.shell.isDisposed()) { + if (!display.readAndDispatch()) { + display.sleep(); + } + } + + // TODO: Fix me + return null; + } + + private Menu createTreeContextMenu() { + + final Menu treeContextMenu = new Menu(this.shell); + final MenuItem createItem = new MenuItem(treeContextMenu, SWT.PUSH); + createItem.setText("Create..."); + createItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + createNewFailurePattern(); + } + }); + new MenuItem(treeContextMenu, SWT.SEPARATOR); + final MenuItem deleteItem = new MenuItem(treeContextMenu, SWT.PUSH); + deleteItem.setText("Delete..."); + deleteItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + deleteFailurePattern(); + } + }); + new MenuItem(treeContextMenu, SWT.SEPARATOR); + final MenuItem saveItem = new MenuItem(treeContextMenu, SWT.PUSH); + saveItem.setText("Save..."); + saveItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + saveFailurePattern(); + } + }); + final MenuItem loadItem = new MenuItem(treeContextMenu, SWT.PUSH); + loadItem.setText("Load..."); + loadItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + loadFailurePattern(); + } + }); + + treeContextMenu.addMenuListener(new MenuAdapter() { + + @Override + public void menuShown(final MenuEvent arg0) { + + if (jobTree.getSelection().length == 0) { + createItem.setEnabled(true); + deleteItem.setEnabled(false); + saveItem.setEnabled(false); + loadItem.setEnabled(true); + } else { + createItem.setEnabled(false); + deleteItem.setEnabled(true); + saveItem.setEnabled(true); + loadItem.setEnabled(false); + } + } + }); + + return treeContextMenu; + } + + private void createNewFailurePattern() { + + final SWTNewFailurePatternDialog dialog = new SWTNewFailurePatternDialog(this.shell, this.jobSuggestions); + + final String patternName = dialog.showDialog(); + if (patternName == null) { + return; + } + + final JobFailurePattern jobFailurePattern = new JobFailurePattern(patternName); + + addFailurePatternToTree(jobFailurePattern); + displayFailurePattern(jobFailurePattern); + } + + private void deleteFailurePattern() { + // TODO: Implement me + } + + private void saveFailurePattern() { + // TODO: Implement me + } + + private void loadFailurePattern() { + + final FileDialog fileDialog = new FileDialog(this.shell, SWT.OPEN); + fileDialog.setText("Load Failure Pattern"); + final String[] filterExts = { "*.xml", "*.*" }; + fileDialog.setFilterExtensions(filterExts); + + final String selectedFile = fileDialog.open(); + if (selectedFile == null) { + return; + } + + final JobFailurePattern failurePattern = loadFailurePatternFromFile(selectedFile); + + addFailurePatternToTree(failurePattern); + displayFailurePattern(failurePattern); + } + + private void addFailurePatternToTree(final JobFailurePattern failurePattern) { + + final TreeItem jobFailureItem = new TreeItem(this.jobTree, SWT.NONE); + jobFailureItem.setText(failurePattern.getName()); + jobFailureItem.setData(failurePattern); + } + + private JobFailurePattern loadFailurePatternFromFile(final String filename) { + + final DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory.newInstance(); + // Ignore comments in the XML file + docBuilderFactory.setIgnoringComments(true); + docBuilderFactory.setNamespaceAware(true); + + JobFailurePattern jobFailurePattern = null; + InputStream inputStream = null; + + try { + + inputStream = new FileInputStream(filename); + + final DocumentBuilder builder = docBuilderFactory.newDocumentBuilder(); + Document doc = null; + Element root = null; + + doc = builder.parse(inputStream); + + if (doc == null) { + throw new Exception("Document is null"); + } + + root = doc.getDocumentElement(); + if (root == null) { + throw new Exception("Root element is null"); + } + + if (!"pattern".equals(root.getNodeName())) { + throw new Exception("Encountered unknown element " + root.getNodeName()); + } + + final NodeList patternChildren = root.getChildNodes(); + for (int i = 0; i < patternChildren.getLength(); ++i) { + + final Node patternChild = patternChildren.item(i); + + if (patternChild instanceof org.w3c.dom.Text) { + continue; + } + + if (patternChild instanceof Element) { + + final Element patternElement = (Element) patternChild; + if ("name".equals(patternElement.getNodeName())) { + final String name = extractValueFromElement(patternElement); + if (jobFailurePattern != null) { + throw new Exception("Element name detected more than once in the file"); + } + + jobFailurePattern = new JobFailurePattern(name); + continue; + } + + if ("failures".equals(patternElement.getNodeName())) { + + if (jobFailurePattern == null) { + throw new Exception("Expected pattern name to be stored before the failure events"); + } + + final NodeList failuresChildren = patternElement.getChildNodes(); + for (int j = 0; j < failuresChildren.getLength(); ++j) { + + final Node failuresChild = failuresChildren.item(j); + + if (failuresChild instanceof org.w3c.dom.Text) { + continue; + } + + if (!(failuresChild instanceof Element)) { + throw new Exception("Expected type element as child of element 'failures'"); + } + + final Element failuresElement = (Element) failuresChild; + + if (!"failure".equals(failuresElement.getNodeName())) { + throw new Exception("Expected element 'failure' as child of element 'failures'"); + } + + final String type = failuresElement.getAttribute("type"); + if (type == null) { + throw new Exception("Element 'failure' lacks the attribute 'type'"); + } + + final boolean taskFailure = ("task".equals(type)); + String name = null; + String interval = null; + + final NodeList failureChildren = failuresElement.getChildNodes(); + for (int k = 0; k < failureChildren.getLength(); ++k) { + + final Node failureChild = failureChildren.item(k); + + if (failureChild instanceof org.w3c.dom.Text) { + continue; + } + + if (!(failureChild instanceof Element)) { + throw new Exception("Expected type element as child of element 'failure'"); + } + + final Element failureElement = (Element) failureChild; + if ("name".equals(failureElement.getNodeName())) { + name = extractValueFromElement(failureElement); + } + + if ("interval".equals(failureElement.getNodeName())) { + interval = extractValueFromElement(failureElement); + } + } + + if (name == null) { + throw new Exception("Could not find name for failure event " + j); + } + + if (interval == null) { + throw new Exception("Could not find interval for failure event " + j); + } + + int iv = 0; + try { + iv = Integer.parseInt(interval); + + } catch (NumberFormatException e) { + throw new Exception("Interval " + interval + " for failure event " + j + + " is not an integer number"); + } + + if (iv <= 0) { + throw new Exception("Interval for failure event " + j + + " must be greather than zero, but is " + iv); + } + + AbstractFailureEvent failureEvent = null; + if (taskFailure) { + failureEvent = new VertexFailureEvent(name, iv); + } else { + failureEvent = new InstanceFailureEvent(name, iv); + } + + jobFailurePattern.addEvent(failureEvent); + } + + continue; + } + + throw new Exception("Uncountered unecpted element " + patternElement.getNodeName()); + + } else { + throw new Exception("Encountered unexpected child of type " + patternChild.getClass()); + } + } + + } catch (Exception e) { + + final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); + messageBox.setText("Cannot load failure pattern"); + messageBox.setMessage(e.getMessage()); + messageBox.open(); + return null; + } finally { + if (inputStream != null) { + try { + inputStream.close(); + } catch (Exception e) { + } + } + } + + return jobFailurePattern; + } + + private String extractValueFromElement(final Element element) throws Exception { + + final NodeList children = element.getChildNodes(); + if (children.getLength() != 1) { + throw new Exception("Element " + element.getNodeName() + " has an unexpected number of children"); + } + + final Node child = children.item(0); + + if (!(child instanceof org.w3c.dom.Text)) { + throw new Exception("Expected child of element " + element.getNodeName() + " to be of type text"); + } + + org.w3c.dom.Text childText = (org.w3c.dom.Text) child; + + return childText.getTextContent(); + } + + private void displayFailurePattern(final JobFailurePattern jobFailurePattern) { + + // Clear old content from event table + this.failureEventTable.clearAll(); + + if (jobFailurePattern == null) { + this.failureEventTable.setEnabled(false); + return; + } + + this.failureEventTable.setEnabled(true); + + final Iterator it = jobFailurePattern.iterator(); + while (it.hasNext()) { + + final AbstractFailureEvent event = it.next(); + final TableItem ti = new TableItem(this.failureEventTable, SWT.NONE); + updateTableItem(ti, event); + } + + // Finally, add item to create new entry in both tables + new TableItem(this.failureEventTable, SWT.NONE); + + this.selectedFailurePattern = jobFailurePattern; + } + +} diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java index e2da8dd4e0d87..199ae2dd742a0 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java @@ -15,645 +15,27 @@ package eu.stratosphere.nephele.visualization.swt; -import java.io.FileInputStream; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.eclipse.swt.SWT; -import org.eclipse.swt.custom.SashForm; -import org.eclipse.swt.events.MenuEvent; -import org.eclipse.swt.events.MenuListener; -import org.eclipse.swt.events.MouseAdapter; -import org.eclipse.swt.events.MouseEvent; -import org.eclipse.swt.events.SelectionAdapter; -import org.eclipse.swt.events.SelectionEvent; -import org.eclipse.swt.graphics.Point; -import org.eclipse.swt.layout.FillLayout; -import org.eclipse.swt.layout.GridData; -import org.eclipse.swt.layout.GridLayout; -import org.eclipse.swt.widgets.Button; -import org.eclipse.swt.widgets.Composite; -import org.eclipse.swt.widgets.Event; -import org.eclipse.swt.widgets.FileDialog; -import org.eclipse.swt.widgets.Group; -import org.eclipse.swt.widgets.Label; -import org.eclipse.swt.widgets.Listener; -import org.eclipse.swt.widgets.Menu; -import org.eclipse.swt.widgets.MenuItem; -import org.eclipse.swt.widgets.MessageBox; +import org.eclipse.swt.widgets.Display; import org.eclipse.swt.widgets.Shell; -import org.eclipse.swt.widgets.Table; -import org.eclipse.swt.widgets.TableColumn; -import org.eclipse.swt.widgets.TableItem; -import org.eclipse.swt.widgets.Tree; -import org.eclipse.swt.widgets.TreeItem; -import org.w3c.dom.Document; -import org.w3c.dom.Element; -import org.w3c.dom.Node; -import org.w3c.dom.NodeList; import eu.stratosphere.nephele.jobgraph.JobID; -public final class SWTFailurePatternsManager extends SelectionAdapter { +public final class SWTFailurePatternsManager { private static final Log LOG = LogFactory.getLog(SWTFailurePatternsManager.class); - private static final int WIDTH = 800; - - private static final int HEIGHT = 400; - - private static final int ICON_COLUMN_WIDTH = 20; - - private static final int TEXT_COLUMN_WIDTH = 200; - - private final Shell shell; - - private final Tree jobTree; - - private final Table failureEventTable; + private final Display display; private final Map failurePatterns = new HashMap(); - private JobFailurePattern selectedFailurePattern = null; - - SWTFailurePatternsManager(final Shell parent) { - - // Set size - this.shell = new Shell(parent); - this.shell.setSize(WIDTH, HEIGHT); - this.shell.setText("Manage Failure Patterns"); - GridLayout gl = new GridLayout(1, false); - gl.horizontalSpacing = 0; - gl.verticalSpacing = 0; - gl.marginRight = 0; - gl.marginLeft = 0; - gl.marginBottom = 0; - gl.marginTop = 0; - gl.marginHeight = 0; - gl.marginWidth = 0; - this.shell.setLayout(gl); - - final Composite mainComposite = new Composite(this.shell, SWT.NONE); - mainComposite.setLayout(new GridLayout(1, false)); - GridData gridData = new GridData(); - gridData.horizontalAlignment = GridData.FILL; - gridData.verticalAlignment = GridData.FILL; - gridData.grabExcessHorizontalSpace = true; - gridData.grabExcessVerticalSpace = true; - mainComposite.setLayoutData(gridData); - - final SashForm horizontalSash = new SashForm(mainComposite, SWT.HORIZONTAL); - horizontalSash.setLayoutData(new GridData(GridData.FILL_BOTH)); - - final Group jobGroup = new Group(horizontalSash, SWT.NONE); - jobGroup.setText("Job Failure Patterns"); - jobGroup.setLayout(new FillLayout()); - - this.jobTree = new Tree(jobGroup, SWT.SINGLE | SWT.BORDER); - this.jobTree.addSelectionListener(this); - this.jobTree.setMenu(createTreeContextMenu()); - - this.failureEventTable = createFailureEventTable(horizontalSash); - horizontalSash.setWeights(new int[] { 2, 8 }); - - final Composite buttonComposite = new Composite(this.shell, SWT.NONE); - buttonComposite.setLayout(new GridLayout(2, false)); - gridData = new GridData(); - gridData.horizontalAlignment = GridData.FILL; - buttonComposite.setLayoutData(gridData); - - final Label fillLabel = new Label(buttonComposite, SWT.NONE); - gridData = new GridData(); - gridData.horizontalAlignment = GridData.FILL; - gridData.grabExcessHorizontalSpace = true; - gridData.grabExcessVerticalSpace = false; - fillLabel.setLayoutData(gridData); - - final Button closeButton = new Button(buttonComposite, SWT.PUSH); - closeButton.setText("Close"); - gridData = new GridData(); - gridData.horizontalAlignment = SWT.RIGHT; - closeButton.setLayoutData(gridData); - - // Initialize the tables - displayFailurePattern(null); - } - - private Table createFailureEventTable(final Composite parent) { - - final Table table = new Table(parent, SWT.BORDER | SWT.MULTI); - table.setHeaderVisible(true); - table.setLinesVisible(true); - - final TableColumn iconColumn = new TableColumn(table, SWT.NONE); - final TableColumn nameColumn = new TableColumn(table, SWT.NONE); - nameColumn.setText("Name"); - final TableColumn intervalColumn = new TableColumn(table, SWT.NONE); - intervalColumn.setText("Interval"); - - for (int i = 0; i < table.getColumnCount(); ++i) { - if (i == 0) { - table.getColumn(i).setWidth(ICON_COLUMN_WIDTH); - } else { - table.getColumn(i).setWidth(TEXT_COLUMN_WIDTH); - } - } - - // Implement listener to add and update events - table.addMouseListener(new MouseAdapter() { - - @Override - public void mouseDoubleClick(final MouseEvent arg0) { - - final TableItem ti = table.getItem(new Point(arg0.x, arg0.y)); - - if (selectedFailurePattern == null) { - return; - } - - final List suggestions = new ArrayList(); // TODO: Compute proper same suggestions here - - AbstractFailureEvent oldEvent = null; - if (ti != null) { - oldEvent = (AbstractFailureEvent) ti.getData(); - } - - final SWTFailureEventEditor editor = new SWTFailureEventEditor(shell, suggestions, oldEvent); - - final AbstractFailureEvent newEvent = editor.showDialog(); - if (newEvent == null) { - return; - } - - if (oldEvent != null) { - selectedFailurePattern.removeEvent(oldEvent); - } - selectedFailurePattern.addEvent(newEvent); - - updateTableItem(ti, newEvent); - } - }); - - // Implement sorting of columns - final Listener sortListener = new Listener() { - - @Override - public void handleEvent(final Event arg0) { - - final TableColumn sortColumn = failureEventTable.getSortColumn(); - final TableColumn currentColumn = (TableColumn) arg0.widget; - int dir = failureEventTable.getSortDirection(); - if (sortColumn == currentColumn) { - dir = (dir == SWT.UP) ? SWT.DOWN : SWT.UP; - } else { - table.setSortColumn(currentColumn); - dir = SWT.UP; - } - - final int direction = dir; - final AbstractFailureEvent[] failureEvents = new AbstractFailureEvent[table.getItemCount()]; - for (int i = 0; i < table.getItemCount(); ++i) { - failureEvents[i] = (AbstractFailureEvent) failureEventTable.getItem(i).getData(); - } - Arrays.sort(failureEvents, new Comparator() { - - @Override - public int compare(final AbstractFailureEvent o1, AbstractFailureEvent o2) { - - if (o1 == null) { - return -1; - } - - if (o2 == null) { - return 1; - } - - if (currentColumn == iconColumn) { - - final int v1 = (o1 instanceof VertexFailureEvent) ? 0 : 1; - final int v2 = (o2 instanceof VertexFailureEvent) ? 0 : 1; - return (direction == SWT.UP) ? (v1 - v2) : (v2 - v1); - - } else if (currentColumn == nameColumn) { - - if (direction == SWT.UP) { - return String.CASE_INSENSITIVE_ORDER.compare(o1.getName(), o2.getName()); - } else { - return String.CASE_INSENSITIVE_ORDER.compare(o2.getName(), o1.getName()); - } - - } else { - - if (direction == SWT.UP) { - return (o1.getInterval() - o2.getInterval()); - } else { - return (o2.getInterval() - o1.getInterval()); - } - } - } - }); - - failureEventTable.removeAll(); - for (int i = 0; i < failureEvents.length; ++i) { - updateTableItem(null, failureEvents[i]); - } - - failureEventTable.setSortColumn(currentColumn); - failureEventTable.setSortDirection(direction); - } - }; - - iconColumn.addListener(SWT.Selection, sortListener); - nameColumn.addListener(SWT.Selection, sortListener); - intervalColumn.addListener(SWT.Selection, sortListener); - - return table; - } - - private void updateTableItem(TableItem ti, final AbstractFailureEvent event) { - - boolean newItemCreated = false; - - if (ti == null) { - - final int index = (failureEventTable.getItemCount() == 0) ? 0 : (this.failureEventTable.getItemCount() - 1); - - ti = new TableItem(this.failureEventTable, SWT.NONE, index); - newItemCreated = true; - } - - if (event != null) { - if (event instanceof VertexFailureEvent) { - ti.setText(0, "T"); - } else { - ti.setText(0, "I"); - } - - ti.setText(1, event.getName()); - ti.setText(2, Integer.toString(event.getInterval())); - } - - // Add new blank item if the old one has been used to create the new event - if (ti.getData() == null && !newItemCreated) { - new TableItem(this.failureEventTable, SWT.NONE); - } - - ti.setData(event); - } - - public void open() { - - this.shell.open(); - - } - - private Menu createTreeContextMenu() { - - final Menu treeContextMenu = new Menu(this.shell); - final MenuItem createItem = new MenuItem(treeContextMenu, SWT.PUSH); - createItem.setText("Create..."); - createItem.addSelectionListener(new SelectionAdapter() { - - @Override - public void widgetSelected(final SelectionEvent arg0) { - createNewFailurePattern(); - } - }); - new MenuItem(treeContextMenu, SWT.SEPARATOR); - final MenuItem deleteItem = new MenuItem(treeContextMenu, SWT.PUSH); - deleteItem.setText("Delete..."); - deleteItem.addSelectionListener(new SelectionAdapter() { - - @Override - public void widgetSelected(final SelectionEvent arg0) { - deleteFailurePattern(); - } - }); - new MenuItem(treeContextMenu, SWT.SEPARATOR); - final MenuItem saveItem = new MenuItem(treeContextMenu, SWT.PUSH); - saveItem.setText("Save..."); - saveItem.addSelectionListener(new SelectionAdapter() { - - @Override - public void widgetSelected(final SelectionEvent arg0) { - saveFailurePattern(); - } - }); - final MenuItem loadItem = new MenuItem(treeContextMenu, SWT.PUSH); - loadItem.setText("Load..."); - loadItem.addSelectionListener(new SelectionAdapter() { - - @Override - public void widgetSelected(final SelectionEvent arg0) { - loadFailurePattern(); - } - }); - - treeContextMenu.addMenuListener(new MenuListener() { - - @Override - public void menuShown(final MenuEvent arg0) { - - if (jobTree.getSelection().length == 0) { - createItem.setEnabled(true); - deleteItem.setEnabled(false); - saveItem.setEnabled(false); - loadItem.setEnabled(true); - } else { - createItem.setEnabled(false); - deleteItem.setEnabled(true); - saveItem.setEnabled(true); - loadItem.setEnabled(false); - } - } - - @Override - public void menuHidden(final MenuEvent arg0) { - // TODO Auto-generated method stub - - } - }); - - return treeContextMenu; - } - - private void createNewFailurePattern() { - - // TODO: Provide proper list of name suggestions - final List suggestions = new ArrayList(); - - final SWTNewFailurePatternDialog dialog = new SWTNewFailurePatternDialog(this.shell, suggestions); - - final String patternName = dialog.showDialog(); - if (patternName == null) { - return; - } - - final JobFailurePattern jobFailurePattern = new JobFailurePattern(patternName); - - addFailurePatternToTree(jobFailurePattern); - displayFailurePattern(jobFailurePattern); - } - - private void deleteFailurePattern() { - // TODO: Implement me - } - - private void saveFailurePattern() { - // TODO: Implement me - } - - private void loadFailurePattern() { - - final FileDialog fileDialog = new FileDialog(this.shell, SWT.OPEN); - fileDialog.setText("Load Failure Pattern"); - final String[] filterExts = { "*.xml", "*.*" }; - fileDialog.setFilterExtensions(filterExts); - - final String selectedFile = fileDialog.open(); - if (selectedFile == null) { - return; - } - - final JobFailurePattern failurePattern = loadFailurePatternFromFile(selectedFile); - - addFailurePatternToTree(failurePattern); - displayFailurePattern(failurePattern); - } - - private void addFailurePatternToTree(final JobFailurePattern failurePattern) { - - final TreeItem jobFailureItem = new TreeItem(this.jobTree, SWT.NONE); - jobFailureItem.setText(failurePattern.getName()); - jobFailureItem.setData(failurePattern); - } - - private JobFailurePattern loadFailurePatternFromFile(final String filename) { - - final DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory.newInstance(); - // Ignore comments in the XML file - docBuilderFactory.setIgnoringComments(true); - docBuilderFactory.setNamespaceAware(true); - - JobFailurePattern jobFailurePattern = null; - InputStream inputStream = null; - - try { - - inputStream = new FileInputStream(filename); - - final DocumentBuilder builder = docBuilderFactory.newDocumentBuilder(); - Document doc = null; - Element root = null; - - doc = builder.parse(inputStream); - - if (doc == null) { - throw new Exception("Document is null"); - } - - root = doc.getDocumentElement(); - if (root == null) { - throw new Exception("Root element is null"); - } - - if (!"pattern".equals(root.getNodeName())) { - throw new Exception("Encountered unknown element " + root.getNodeName()); - } - - final NodeList patternChildren = root.getChildNodes(); - for (int i = 0; i < patternChildren.getLength(); ++i) { - - final Node patternChild = patternChildren.item(i); - - if (patternChild instanceof org.w3c.dom.Text) { - continue; - } - - if (patternChild instanceof Element) { - - final Element patternElement = (Element) patternChild; - if ("name".equals(patternElement.getNodeName())) { - final String name = extractValueFromElement(patternElement); - if (jobFailurePattern != null) { - throw new Exception("Element name detected more than once in the file"); - } - - jobFailurePattern = new JobFailurePattern(name); - continue; - } - - if ("failures".equals(patternElement.getNodeName())) { - - if (jobFailurePattern == null) { - throw new Exception("Expected pattern name to be stored before the failure events"); - } - - final NodeList failuresChildren = patternElement.getChildNodes(); - for (int j = 0; j < failuresChildren.getLength(); ++j) { - - final Node failuresChild = failuresChildren.item(j); - - if (failuresChild instanceof org.w3c.dom.Text) { - continue; - } - - if (!(failuresChild instanceof Element)) { - throw new Exception("Expected type element as child of element 'failures'"); - } - - final Element failuresElement = (Element) failuresChild; - - if (!"failure".equals(failuresElement.getNodeName())) { - throw new Exception("Expected element 'failure' as child of element 'failures'"); - } - - final String type = failuresElement.getAttribute("type"); - if (type == null) { - throw new Exception("Element 'failure' lacks the attribute 'type'"); - } - - final boolean taskFailure = ("task".equals(type)); - String name = null; - String interval = null; - - final NodeList failureChildren = failuresElement.getChildNodes(); - for (int k = 0; k < failureChildren.getLength(); ++k) { - - final Node failureChild = failureChildren.item(k); - - if (failureChild instanceof org.w3c.dom.Text) { - continue; - } - - if (!(failureChild instanceof Element)) { - throw new Exception("Expected type element as child of element 'failure'"); - } - - final Element failureElement = (Element) failureChild; - if ("name".equals(failureElement.getNodeName())) { - name = extractValueFromElement(failureElement); - } - - if ("interval".equals(failureElement.getNodeName())) { - interval = extractValueFromElement(failureElement); - } - } - - if (name == null) { - throw new Exception("Could not find name for failure event " + j); - } - - if (interval == null) { - throw new Exception("Could not find interval for failure event " + j); - } - - int iv = 0; - try { - iv = Integer.parseInt(interval); - - } catch (NumberFormatException e) { - throw new Exception("Interval " + interval + " for failure event " + j - + " is not an integer number"); - } - - if (iv <= 0) { - throw new Exception("Interval for failure event " + j - + " must be greather than zero, but is " + iv); - } - - AbstractFailureEvent failureEvent = null; - if (taskFailure) { - failureEvent = new VertexFailureEvent(name, iv); - } else { - failureEvent = new InstanceFailureEvent(name, iv); - } - - jobFailurePattern.addEvent(failureEvent); - } - - continue; - } - - throw new Exception("Uncountered unecpted element " + patternElement.getNodeName()); - - } else { - throw new Exception("Encountered unexpected child of type " + patternChild.getClass()); - } - } - - } catch (Exception e) { - - final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); - messageBox.setText("Cannot load failure pattern"); - messageBox.setMessage(e.getMessage()); - messageBox.open(); - return null; - } finally { - if (inputStream != null) { - try { - inputStream.close(); - } catch (Exception e) { - } - } - } - - return jobFailurePattern; - } - - private String extractValueFromElement(final Element element) throws Exception { - - final NodeList children = element.getChildNodes(); - if (children.getLength() != 1) { - throw new Exception("Element " + element.getNodeName() + " has an unexpected number of children"); - } - - final Node child = children.item(0); - - if (!(child instanceof org.w3c.dom.Text)) { - throw new Exception("Expected child of element " + element.getNodeName() + " to be of type text"); - } - - org.w3c.dom.Text childText = (org.w3c.dom.Text) child; - - return childText.getTextContent(); - } - - private void displayFailurePattern(final JobFailurePattern jobFailurePattern) { - - // Clear old content from event table - this.failureEventTable.clearAll(); - - if (jobFailurePattern == null) { - this.failureEventTable.setEnabled(false); - return; - } - - this.failureEventTable.setEnabled(true); - - final Iterator it = jobFailurePattern.iterator(); - while (it.hasNext()) { - - final AbstractFailureEvent event = it.next(); - final TableItem ti = new TableItem(this.failureEventTable, SWT.NONE); - updateTableItem(ti, event); - } - - // Finally, add item to create new entry in both tables - new TableItem(this.failureEventTable, SWT.NONE); - - this.selectedFailurePattern = jobFailurePattern; + public SWTFailurePatternsManager(final Display display) { + this.display = display; } public void startFailurePattern(final JobID jobID, final String jobName, final long referenceTime) { @@ -663,9 +45,16 @@ public void startFailurePattern(final JobID jobID, final String jobName, final l LOG.info("No failure pattern for job " + jobName); } - final JobFailurePatternExecutor executor = new JobFailurePatternExecutor(this.shell.getDisplay(), jobID, + final JobFailurePatternExecutor executor = new JobFailurePatternExecutor(this.display, jobID, jobName, failurePattern); executor.start(referenceTime); } + + public void openEditor(final Shell parent, final List jobSuggestions, final List nameSuggestions) { + + final SWTFailurePatternsEditor editor = new SWTFailurePatternsEditor(parent, jobSuggestions, nameSuggestions); + + editor.show(); + } } diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java index c6c750089f24e..000aa8af42634 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java @@ -16,6 +16,7 @@ package eu.stratosphere.nephele.visualization.swt; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -301,8 +302,8 @@ public void handleEvent(final Event arg0) { } }); - // Create outage patterns manager - this.failurePatternsManager = new SWTFailurePatternsManager(this.shell); + // Create failure patterns manager + this.failurePatternsManager = new SWTFailurePatternsManager(this.shell.getDisplay()); // Launch the timer that will query for events this.display.timerExec(QUERYINTERVAL * 1000, this); @@ -705,7 +706,10 @@ private void cleanUpOldEvents(long sleepTime) { private void manageFailurePatterns() { - this.failurePatternsManager.open(); + // TODO: Replace empty list by sensible suggestions + final List emptyList = Collections.emptyList(); + + this.failurePatternsManager.openEditor(this.shell, emptyList, emptyList); } private void logBufferUtilization() { From f8b70146bcf75f20c80507b365f2a31e7d75a3e3 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 11 Jan 2012 21:42:31 +0100 Subject: [PATCH 137/310] Improved robustness of failure manager --- .../swt/AutoCompletionCombo.java | 3 +- .../swt/SWTFailureEventEditor.java | 4 +- .../swt/SWTFailurePatternsEditor.java | 65 ++++++++++++++++--- .../swt/SWTFailurePatternsManager.java | 17 ++--- .../swt/SWTNewFailurePatternDialog.java | 35 +++++++++- .../swt/SWTVisualizationGUI.java | 6 +- 6 files changed, 104 insertions(+), 26 deletions(-) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java index d014edf902bb6..7c36a2d9e0f76 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/AutoCompletionCombo.java @@ -18,6 +18,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Set; import org.eclipse.swt.SWT; import org.eclipse.swt.events.KeyEvent; @@ -56,7 +57,7 @@ public final class AutoCompletionCombo extends Composite implements KeyListener, * @param suggestions * a list of suggestions for the auto-completion */ - public AutoCompletionCombo(final Composite parent, final int style, final List suggestions) { + public AutoCompletionCombo(final Composite parent, final int style, final Set suggestions) { super(parent, style); // First, sort the suggestions diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java index e4c80956230bc..ac6388cb47843 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java @@ -15,7 +15,7 @@ package eu.stratosphere.nephele.visualization.swt; -import java.util.List; +import java.util.Set; import org.eclipse.swt.SWT; import org.eclipse.swt.events.KeyAdapter; @@ -76,7 +76,7 @@ public final class SWTFailureEventEditor { */ private AbstractFailureEvent returnValue = null; - public SWTFailureEventEditor(final Shell parent, final List nameSuggestions, + public SWTFailureEventEditor(final Shell parent, final Set nameSuggestions, final AbstractFailureEvent failureEvent) { this.shell = new Shell(parent); diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java index 8f6aafb2374c5..a296c3a50babc 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java @@ -20,7 +20,8 @@ import java.util.Arrays; import java.util.Comparator; import java.util.Iterator; -import java.util.List; +import java.util.Map; +import java.util.Set; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; @@ -75,16 +76,20 @@ public final class SWTFailurePatternsEditor extends SelectionAdapter { private final Table failureEventTable; - private final List jobSuggestions; + private final Set jobSuggestions; - private final List nameSuggestions; + private final Set nameSuggestions; + + private final Map loadedPatterns; private JobFailurePattern selectedFailurePattern = null; - SWTFailurePatternsEditor(final Shell parent, final List jobSuggestions, final List nameSuggestions) { + SWTFailurePatternsEditor(final Shell parent, final Set jobSuggestions, final Set nameSuggestions, + final Map loadedPatterns) { this.jobSuggestions = jobSuggestions; this.nameSuggestions = nameSuggestions; + this.loadedPatterns = loadedPatterns; // Set size this.shell = new Shell(parent); @@ -142,9 +147,31 @@ public final class SWTFailurePatternsEditor extends SelectionAdapter { gridData = new GridData(); gridData.horizontalAlignment = SWT.RIGHT; closeButton.setLayoutData(gridData); + closeButton.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(SelectionEvent arg0) { + + shell.dispose(); + } + }); + + final Iterator it = this.loadedPatterns.values().iterator(); + while (it.hasNext()) { + addFailurePatternToTree(it.next()); + } // Initialize the tables - displayFailurePattern(null); + if (this.jobTree.getItemCount() > 0) { + + final TreeItem ti = this.jobTree.getItem(0); + this.jobTree.setSelection(ti); + displayFailurePattern((JobFailurePattern) ti.getData()); + + } else { + displayFailurePattern(null); + } + } private Table createFailureEventTable(final Composite parent) { @@ -307,7 +334,7 @@ private void updateTableItem(TableItem ti, final AbstractFailureEvent event) { ti.setData(event); } - public List show() { + public Map show() { this.shell.open(); @@ -319,8 +346,7 @@ public List show() { } } - // TODO: Fix me - return null; + return this.loadedPatterns; } private Menu createTreeContextMenu() { @@ -389,7 +415,10 @@ public void menuShown(final MenuEvent arg0) { private void createNewFailurePattern() { - final SWTNewFailurePatternDialog dialog = new SWTNewFailurePatternDialog(this.shell, this.jobSuggestions); + final Set takenNames = this.loadedPatterns.keySet(); + + final SWTNewFailurePatternDialog dialog = new SWTNewFailurePatternDialog(this.shell, this.jobSuggestions, + takenNames); final String patternName = dialog.showDialog(); if (patternName == null) { @@ -398,6 +427,9 @@ private void createNewFailurePattern() { final JobFailurePattern jobFailurePattern = new JobFailurePattern(patternName); + // Add to loaded patterns + this.loadedPatterns.put(jobFailurePattern.getName(), jobFailurePattern); + addFailurePatternToTree(jobFailurePattern); displayFailurePattern(jobFailurePattern); } @@ -424,6 +456,19 @@ private void loadFailurePattern() { final JobFailurePattern failurePattern = loadFailurePatternFromFile(selectedFile); + if (this.loadedPatterns.containsKey(failurePattern.getName())) { + + final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); + messageBox.setText("Cannot load failure pattern"); + messageBox.setMessage("There is already a failure pattern loaded with the name '" + + failurePattern.getName() + "'. Please remove it first."); + messageBox.open(); + + return; + } + + this.loadedPatterns.put(failurePattern.getName(), failurePattern); + addFailurePatternToTree(failurePattern); displayFailurePattern(failurePattern); } @@ -643,9 +688,11 @@ private void displayFailurePattern(final JobFailurePattern jobFailurePattern) { final AbstractFailureEvent event = it.next(); final TableItem ti = new TableItem(this.failureEventTable, SWT.NONE); + ti.setData(event); updateTableItem(ti, event); } + System.out.println("Created empty item"); // Finally, add item to create new entry in both tables new TableItem(this.failureEventTable, SWT.NONE); diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java index 199ae2dd742a0..e1b67ba66e60d 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java @@ -16,8 +16,8 @@ package eu.stratosphere.nephele.visualization.swt; import java.util.HashMap; -import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -32,7 +32,7 @@ public final class SWTFailurePatternsManager { private final Display display; - private final Map failurePatterns = new HashMap(); + private Map failurePatterns = new HashMap(); public SWTFailurePatternsManager(final Display display) { this.display = display; @@ -40,7 +40,7 @@ public SWTFailurePatternsManager(final Display display) { public void startFailurePattern(final JobID jobID, final String jobName, final long referenceTime) { - final JobFailurePattern failurePattern = this.failurePatterns.get(jobName.toLowerCase()); + final JobFailurePattern failurePattern = this.failurePatterns.get(jobName); if (failurePattern == null) { LOG.info("No failure pattern for job " + jobName); } @@ -50,11 +50,12 @@ public void startFailurePattern(final JobID jobID, final String jobName, final l executor.start(referenceTime); } - - public void openEditor(final Shell parent, final List jobSuggestions, final List nameSuggestions) { - - final SWTFailurePatternsEditor editor = new SWTFailurePatternsEditor(parent, jobSuggestions, nameSuggestions); - + + public void openEditor(final Shell parent, final Set jobSuggestions, final Set nameSuggestions) { + + final SWTFailurePatternsEditor editor = new SWTFailurePatternsEditor(parent, jobSuggestions, nameSuggestions, + this.failurePatterns); + editor.show(); } } diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java index bc7e83509c26d..fc56c27cf229b 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java @@ -15,7 +15,8 @@ package eu.stratosphere.nephele.visualization.swt; -import java.util.List; +import java.util.HashSet; +import java.util.Set; import org.eclipse.swt.SWT; import org.eclipse.swt.layout.GridData; @@ -58,6 +59,11 @@ public final class SWTNewFailurePatternDialog { */ private final AutoCompletionCombo input; + /** + * The taken names which must not be accepted as names for the new pattern. + */ + private final Set takenNames; + /** * The return value of the showDialog method. */ @@ -70,8 +76,13 @@ public final class SWTNewFailurePatternDialog { * the parent of this dialog * @param nameSuggestions * name suggestions to be displayed inside auto-completion combo box + * @param takenNames + * names that are already taken and must not be accepted as input */ - public SWTNewFailurePatternDialog(final Shell parent, final List nameSuggestions) { + public SWTNewFailurePatternDialog(final Shell parent, final Set nameSuggestions, + final Set takenNames) { + + this.takenNames = takenNames; this.shell = new Shell(parent); this.shell.setSize(WIDTH, HEIGHT); @@ -83,7 +94,11 @@ public SWTNewFailurePatternDialog(final Shell parent, final List nameSug gridData.grabExcessHorizontalSpace = true; gridData.grabExcessVerticalSpace = false; - this.input = new AutoCompletionCombo(this.shell, SWT.NONE, nameSuggestions); + // Remove the taken names from + final Set nameSugg = new HashSet(nameSuggestions); + nameSugg.removeAll(takenNames); + + this.input = new AutoCompletionCombo(this.shell, SWT.NONE, nameSugg); this.input.setLayoutData(gridData); this.input.addKeyListener(new KeyAdapter() { @@ -167,6 +182,20 @@ private boolean isInputValid() { return false; } + if (this.takenNames.contains(text)) { + + final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); + + messageBox.setText("Invalid Input"); + messageBox.setMessage("The chosen name is already used by another loaded failure pattern."); + messageBox.open(); + + this.input.setFocus(); + + return false; + + } + return true; } diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java index 000aa8af42634..aa242b983b1c1 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java @@ -706,10 +706,10 @@ private void cleanUpOldEvents(long sleepTime) { private void manageFailurePatterns() { - // TODO: Replace empty list by sensible suggestions - final List emptyList = Collections.emptyList(); + // TODO: Replace empty set by sensible suggestions + final Set emptySet = Collections.emptySet(); - this.failurePatternsManager.openEditor(this.shell, emptyList, emptyList); + this.failurePatternsManager.openEditor(this.shell, emptySet, emptySet); } private void logBufferUtilization() { From d2a671fe39a722e1bca51bbdadd31044d18f8022 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 11 Jan 2012 22:29:56 +0100 Subject: [PATCH 138/310] Implemented keyboard control --- .../swt/SWTFailurePatternsEditor.java | 121 +++++++++++++++++- 1 file changed, 120 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java index a296c3a50babc..cf38e4b6983ae 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java @@ -28,6 +28,8 @@ import org.eclipse.swt.SWT; import org.eclipse.swt.custom.SashForm; +import org.eclipse.swt.events.KeyAdapter; +import org.eclipse.swt.events.KeyEvent; import org.eclipse.swt.events.MenuAdapter; import org.eclipse.swt.events.MenuEvent; import org.eclipse.swt.events.MouseAdapter; @@ -176,7 +178,7 @@ public void widgetSelected(SelectionEvent arg0) { private Table createFailureEventTable(final Composite parent) { - final Table table = new Table(parent, SWT.BORDER | SWT.MULTI); + final Table table = new Table(parent, SWT.BORDER | SWT.SINGLE); table.setHeaderVisible(true); table.setLinesVisible(true); @@ -300,9 +302,55 @@ public int compare(final AbstractFailureEvent o1, AbstractFailureEvent o2) { nameColumn.addListener(SWT.Selection, sortListener); intervalColumn.addListener(SWT.Selection, sortListener); + // Implement keyboard commands + table.addKeyListener(new KeyAdapter() { + + @Override + public void keyReleased(final KeyEvent arg0) { + + if (arg0.keyCode != SWT.DEL) { + return; + } + + removeSelectedTableItems(); + } + }); + + // Set the menu + table.setMenu(createTableContextMenu()); + return table; } + private void removeSelectedTableItems() { + + final TableItem[] selectedItems = this.failureEventTable.getSelection(); + if (selectedItems == null) { + return; + } + + for (final TableItem selectedItem : selectedItems) { + removeTableItem(selectedItem); + } + } + + private void removeTableItem(final TableItem ti) { + + final AbstractFailureEvent event = (AbstractFailureEvent) ti.getData(); + if (event == null) { + return; + } + + final MessageBox messageBox = new MessageBox(shell, SWT.ICON_QUESTION | SWT.YES | SWT.NO); + messageBox.setText("Confirm removal"); + messageBox.setMessage("Do you really want to remove the event '" + event.getName() + "'"); + if (messageBox.open() == SWT.YES) { + ti.dispose(); + } + + this.selectedFailurePattern.removeEvent(event); + } + private void updateTableItem(TableItem ti, final AbstractFailureEvent event) { boolean newItemCreated = false; @@ -349,6 +397,77 @@ public Map show() { return this.loadedPatterns; } + private Menu createTableContextMenu() { + + final Menu tableContextMenu = new Menu(this.shell); + + final MenuItem createItem = new MenuItem(tableContextMenu, SWT.PUSH); + createItem.setText("Create..."); + createItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + // TODO: Implement me + } + + }); + + final MenuItem editItem = new MenuItem(tableContextMenu, SWT.PUSH); + editItem.setText("Edit..."); + editItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + // TODO: Implement me + } + + }); + + final MenuItem removeItem = new MenuItem(tableContextMenu, SWT.PUSH); + removeItem.setText("Remove..."); + removeItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + removeSelectedTableItems(); + } + }); + + tableContextMenu.addMenuListener(new MenuAdapter() { + + @Override + public void menuShown(final MenuEvent arg0) { + + TableItem[] selectedItems = failureEventTable.getSelection(); + if (selectedItems == null) { + + return; + } + + if (selectedItems.length == 0) { + + editItem.setEnabled(false); + removeItem.setEnabled(false); + + return; + } + + if (selectedItems[0].getData() == null) { + + editItem.setEnabled(false); + removeItem.setEnabled(false); + + return; + } + + editItem.setEnabled(true); + removeItem.setEnabled(true); + } + }); + + return tableContextMenu; + } + private Menu createTreeContextMenu() { final Menu treeContextMenu = new Menu(this.shell); From bab2df1fd579f77deeff82485559f93494f5c161 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 13 Jan 2012 12:52:43 +0100 Subject: [PATCH 139/310] Refactored failure event table --- .../swt/SWTFailureEventTable.java | 380 ++++++++++++++++++ .../swt/SWTFailurePatternsEditor.java | 339 +--------------- 2 files changed, 386 insertions(+), 333 deletions(-) create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java new file mode 100644 index 0000000000000..bebc820fd9095 --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java @@ -0,0 +1,380 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Set; + +import org.eclipse.swt.SWT; +import org.eclipse.swt.events.KeyAdapter; +import org.eclipse.swt.events.KeyEvent; +import org.eclipse.swt.events.MenuAdapter; +import org.eclipse.swt.events.MenuEvent; +import org.eclipse.swt.events.MouseAdapter; +import org.eclipse.swt.events.MouseEvent; +import org.eclipse.swt.events.SelectionAdapter; +import org.eclipse.swt.events.SelectionEvent; +import org.eclipse.swt.graphics.Point; +import org.eclipse.swt.layout.FillLayout; +import org.eclipse.swt.widgets.Composite; +import org.eclipse.swt.widgets.Event; +import org.eclipse.swt.widgets.Listener; +import org.eclipse.swt.widgets.Menu; +import org.eclipse.swt.widgets.MenuItem; +import org.eclipse.swt.widgets.MessageBox; +import org.eclipse.swt.widgets.Table; +import org.eclipse.swt.widgets.TableColumn; +import org.eclipse.swt.widgets.TableItem; + +public final class SWTFailureEventTable extends Composite { + + private static final int ICON_COLUMN_WIDTH = 20; + + private static final int TEXT_COLUMN_WIDTH = 200; + + private final Table failureEventTable; + + private final Set nameSuggestions; + + private JobFailurePattern selectedFailurePattern = null; + + public SWTFailureEventTable(final Composite parent, final int style, final Set nameSuggestions) { + super(parent, SWT.NONE); + + this.nameSuggestions = nameSuggestions; + + setLayout(new FillLayout()); + + this.failureEventTable = new Table(this, style); + this.failureEventTable.setHeaderVisible(true); + this.failureEventTable.setLinesVisible(true); + + final TableColumn iconColumn = new TableColumn(this.failureEventTable, SWT.NONE); + final TableColumn nameColumn = new TableColumn(this.failureEventTable, SWT.NONE); + nameColumn.setText("Name"); + final TableColumn intervalColumn = new TableColumn(this.failureEventTable, SWT.NONE); + intervalColumn.setText("Interval"); + + for (int i = 0; i < this.failureEventTable.getColumnCount(); ++i) { + if (i == 0) { + this.failureEventTable.getColumn(i).setWidth(ICON_COLUMN_WIDTH); + } else { + this.failureEventTable.getColumn(i).setWidth(TEXT_COLUMN_WIDTH); + } + } + + // Implement listener to add and update events + this.failureEventTable.addMouseListener(new MouseAdapter() { + + @Override + public void mouseDoubleClick(final MouseEvent arg0) { + + final TableItem ti = failureEventTable.getItem(new Point(arg0.x, arg0.y)); + + if (selectedFailurePattern == null) { + return; + } + + addOrEditTableItem(ti); + } + }); + + // Implement sorting of columns + final Listener sortListener = new Listener() { + + @Override + public void handleEvent(final Event arg0) { + + final TableColumn sortColumn = failureEventTable.getSortColumn(); + final TableColumn currentColumn = (TableColumn) arg0.widget; + int dir = failureEventTable.getSortDirection(); + if (sortColumn == currentColumn) { + dir = (dir == SWT.UP) ? SWT.DOWN : SWT.UP; + } else { + failureEventTable.setSortColumn(currentColumn); + dir = SWT.UP; + } + + final int direction = dir; + final AbstractFailureEvent[] failureEvents = new AbstractFailureEvent[failureEventTable.getItemCount()]; + for (int i = 0; i < failureEventTable.getItemCount(); ++i) { + failureEvents[i] = (AbstractFailureEvent) failureEventTable.getItem(i).getData(); + } + Arrays.sort(failureEvents, new Comparator() { + + @Override + public int compare(final AbstractFailureEvent o1, AbstractFailureEvent o2) { + + if (o1 == null) { + return -1; + } + + if (o2 == null) { + return 1; + } + + if (currentColumn == iconColumn) { + + final int v1 = (o1 instanceof VertexFailureEvent) ? 0 : 1; + final int v2 = (o2 instanceof VertexFailureEvent) ? 0 : 1; + return (direction == SWT.UP) ? (v1 - v2) : (v2 - v1); + + } else if (currentColumn == nameColumn) { + + if (direction == SWT.UP) { + return String.CASE_INSENSITIVE_ORDER.compare(o1.getName(), o2.getName()); + } else { + return String.CASE_INSENSITIVE_ORDER.compare(o2.getName(), o1.getName()); + } + + } else { + + if (direction == SWT.UP) { + return (o1.getInterval() - o2.getInterval()); + } else { + return (o2.getInterval() - o1.getInterval()); + } + } + } + }); + + failureEventTable.removeAll(); + for (int i = 0; i < failureEvents.length; ++i) { + updateTableItem(null, failureEvents[i]); + } + + failureEventTable.setSortColumn(currentColumn); + failureEventTable.setSortDirection(direction); + } + }; + + iconColumn.addListener(SWT.Selection, sortListener); + nameColumn.addListener(SWT.Selection, sortListener); + intervalColumn.addListener(SWT.Selection, sortListener); + + // Implement keyboard commands + this.failureEventTable.addKeyListener(new KeyAdapter() { + + @Override + public void keyReleased(final KeyEvent arg0) { + + if (arg0.keyCode == SWT.DEL) { + removeSelectedTableItems(); + } else if (arg0.keyCode == SWT.CR) { + addOrEditSelectedTableItems(); + } + } + }); + + // Set the menu + this.failureEventTable.setMenu(createTableContextMenu()); + } + + private void updateTableItem(TableItem ti, final AbstractFailureEvent event) { + + boolean newItemCreated = false; + + if (ti == null) { + + final int index = (failureEventTable.getItemCount() == 0) ? 0 : (this.failureEventTable.getItemCount() - 1); + + ti = new TableItem(this.failureEventTable, SWT.NONE, index); + newItemCreated = true; + } + + if (event != null) { + if (event instanceof VertexFailureEvent) { + ti.setText(0, "T"); + } else { + ti.setText(0, "I"); + } + + ti.setText(1, event.getName()); + ti.setText(2, Integer.toString(event.getInterval())); + } + + // Add new blank item if the old one has been used to create the new event + if (ti.getData() == null && !newItemCreated) { + new TableItem(this.failureEventTable, SWT.NONE); + } + + ti.setData(event); + } + + private Menu createTableContextMenu() { + + final Menu tableContextMenu = new Menu(getShell()); + + final MenuItem createItem = new MenuItem(tableContextMenu, SWT.PUSH); + createItem.setText("Create..."); + createItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + addOrEditSelectedTableItems(); + } + + }); + + final MenuItem editItem = new MenuItem(tableContextMenu, SWT.PUSH); + editItem.setText("Edit..."); + editItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + addOrEditSelectedTableItems(); + } + + }); + + final MenuItem removeItem = new MenuItem(tableContextMenu, SWT.PUSH); + removeItem.setText("Remove..."); + removeItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + removeSelectedTableItems(); + } + }); + + tableContextMenu.addMenuListener(new MenuAdapter() { + + @Override + public void menuShown(final MenuEvent arg0) { + + TableItem[] selectedItems = failureEventTable.getSelection(); + if (selectedItems == null) { + + return; + } + + if (selectedItems.length == 0) { + + editItem.setEnabled(false); + removeItem.setEnabled(false); + + return; + } + + if (selectedItems[0].getData() == null) { + + editItem.setEnabled(false); + removeItem.setEnabled(false); + + return; + } + + editItem.setEnabled(true); + removeItem.setEnabled(true); + } + }); + + return tableContextMenu; + } + + private void addOrEditSelectedTableItems() { + + final TableItem[] selectedItems = this.failureEventTable.getSelection(); + if (selectedItems == null) { + return; + } + + for (final TableItem selectedItem : selectedItems) { + addOrEditTableItem(selectedItem); + } + } + + private void addOrEditTableItem(final TableItem ti) { + + AbstractFailureEvent oldEvent = null; + if (ti != null) { + oldEvent = (AbstractFailureEvent) ti.getData(); + } + + final SWTFailureEventEditor editor = new SWTFailureEventEditor(getShell(), nameSuggestions, oldEvent); + + final AbstractFailureEvent newEvent = editor.showDialog(); + if (newEvent == null) { + return; + } + + if (oldEvent != null) { + selectedFailurePattern.removeEvent(oldEvent); + } + selectedFailurePattern.addEvent(newEvent); + + updateTableItem(ti, newEvent); + + } + + private void removeSelectedTableItems() { + + final TableItem[] selectedItems = this.failureEventTable.getSelection(); + if (selectedItems == null) { + return; + } + + for (final TableItem selectedItem : selectedItems) { + removeTableItem(selectedItem); + } + } + + private void removeTableItem(final TableItem ti) { + + final AbstractFailureEvent event = (AbstractFailureEvent) ti.getData(); + if (event == null) { + return; + } + + final MessageBox messageBox = new MessageBox(getShell(), SWT.ICON_QUESTION | SWT.YES | SWT.NO); + messageBox.setText("Confirm removal"); + messageBox.setMessage("Do you really want to remove the event '" + event.getName() + "'"); + if (messageBox.open() == SWT.YES) { + ti.dispose(); + } + + this.selectedFailurePattern.removeEvent(event); + } + + public void showFailurePattern(final JobFailurePattern jobFailurePattern) { + + // Clear old content from event table + this.failureEventTable.clearAll(); + + if (jobFailurePattern == null) { + this.failureEventTable.setEnabled(false); + return; + } + + this.failureEventTable.setEnabled(true); + + final Iterator it = jobFailurePattern.iterator(); + while (it.hasNext()) { + + final AbstractFailureEvent event = it.next(); + final TableItem ti = new TableItem(this.failureEventTable, SWT.NONE); + ti.setData(event); + updateTableItem(ti, event); + } + + // Finally, add item to create new entry in both tables + new TableItem(this.failureEventTable, SWT.NONE); + + this.selectedFailurePattern = jobFailurePattern; + } +} diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java index cf38e4b6983ae..79c222a59ef6b 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java @@ -17,8 +17,6 @@ import java.io.FileInputStream; import java.io.InputStream; -import java.util.Arrays; -import java.util.Comparator; import java.util.Iterator; import java.util.Map; import java.util.Set; @@ -28,33 +26,23 @@ import org.eclipse.swt.SWT; import org.eclipse.swt.custom.SashForm; -import org.eclipse.swt.events.KeyAdapter; -import org.eclipse.swt.events.KeyEvent; import org.eclipse.swt.events.MenuAdapter; import org.eclipse.swt.events.MenuEvent; -import org.eclipse.swt.events.MouseAdapter; -import org.eclipse.swt.events.MouseEvent; import org.eclipse.swt.events.SelectionAdapter; import org.eclipse.swt.events.SelectionEvent; -import org.eclipse.swt.graphics.Point; import org.eclipse.swt.layout.FillLayout; import org.eclipse.swt.layout.GridData; import org.eclipse.swt.layout.GridLayout; import org.eclipse.swt.widgets.Button; import org.eclipse.swt.widgets.Composite; import org.eclipse.swt.widgets.Display; -import org.eclipse.swt.widgets.Event; import org.eclipse.swt.widgets.FileDialog; import org.eclipse.swt.widgets.Group; import org.eclipse.swt.widgets.Label; -import org.eclipse.swt.widgets.Listener; import org.eclipse.swt.widgets.Menu; import org.eclipse.swt.widgets.MenuItem; import org.eclipse.swt.widgets.MessageBox; import org.eclipse.swt.widgets.Shell; -import org.eclipse.swt.widgets.Table; -import org.eclipse.swt.widgets.TableColumn; -import org.eclipse.swt.widgets.TableItem; import org.eclipse.swt.widgets.Tree; import org.eclipse.swt.widgets.TreeItem; import org.w3c.dom.Document; @@ -68,29 +56,20 @@ public final class SWTFailurePatternsEditor extends SelectionAdapter { private static final int HEIGHT = 400; - private static final int ICON_COLUMN_WIDTH = 20; - - private static final int TEXT_COLUMN_WIDTH = 200; - private final Shell shell; private final Tree jobTree; - private final Table failureEventTable; + private final SWTFailureEventTable failureEventTable; private final Set jobSuggestions; - private final Set nameSuggestions; - private final Map loadedPatterns; - private JobFailurePattern selectedFailurePattern = null; - SWTFailurePatternsEditor(final Shell parent, final Set jobSuggestions, final Set nameSuggestions, final Map loadedPatterns) { this.jobSuggestions = jobSuggestions; - this.nameSuggestions = nameSuggestions; this.loadedPatterns = loadedPatterns; // Set size @@ -128,7 +107,7 @@ public final class SWTFailurePatternsEditor extends SelectionAdapter { this.jobTree.addSelectionListener(this); this.jobTree.setMenu(createTreeContextMenu()); - this.failureEventTable = createFailureEventTable(horizontalSash); + this.failureEventTable = new SWTFailureEventTable(horizontalSash, SWT.BORDER | SWT.SINGLE, nameSuggestions); horizontalSash.setWeights(new int[] { 2, 8 }); final Composite buttonComposite = new Composite(this.shell, SWT.NONE); @@ -168,218 +147,12 @@ public void widgetSelected(SelectionEvent arg0) { final TreeItem ti = this.jobTree.getItem(0); this.jobTree.setSelection(ti); - displayFailurePattern((JobFailurePattern) ti.getData()); + this.failureEventTable.showFailurePattern((JobFailurePattern) ti.getData()); } else { - displayFailurePattern(null); - } - - } - - private Table createFailureEventTable(final Composite parent) { - - final Table table = new Table(parent, SWT.BORDER | SWT.SINGLE); - table.setHeaderVisible(true); - table.setLinesVisible(true); - - final TableColumn iconColumn = new TableColumn(table, SWT.NONE); - final TableColumn nameColumn = new TableColumn(table, SWT.NONE); - nameColumn.setText("Name"); - final TableColumn intervalColumn = new TableColumn(table, SWT.NONE); - intervalColumn.setText("Interval"); - - for (int i = 0; i < table.getColumnCount(); ++i) { - if (i == 0) { - table.getColumn(i).setWidth(ICON_COLUMN_WIDTH); - } else { - table.getColumn(i).setWidth(TEXT_COLUMN_WIDTH); - } - } - - // Implement listener to add and update events - table.addMouseListener(new MouseAdapter() { - - @Override - public void mouseDoubleClick(final MouseEvent arg0) { - - final TableItem ti = table.getItem(new Point(arg0.x, arg0.y)); - - if (selectedFailurePattern == null) { - return; - } - - AbstractFailureEvent oldEvent = null; - if (ti != null) { - oldEvent = (AbstractFailureEvent) ti.getData(); - } - - final SWTFailureEventEditor editor = new SWTFailureEventEditor(shell, nameSuggestions, oldEvent); - - final AbstractFailureEvent newEvent = editor.showDialog(); - if (newEvent == null) { - return; - } - - if (oldEvent != null) { - selectedFailurePattern.removeEvent(oldEvent); - } - selectedFailurePattern.addEvent(newEvent); - - updateTableItem(ti, newEvent); - } - }); - - // Implement sorting of columns - final Listener sortListener = new Listener() { - - @Override - public void handleEvent(final Event arg0) { - - final TableColumn sortColumn = failureEventTable.getSortColumn(); - final TableColumn currentColumn = (TableColumn) arg0.widget; - int dir = failureEventTable.getSortDirection(); - if (sortColumn == currentColumn) { - dir = (dir == SWT.UP) ? SWT.DOWN : SWT.UP; - } else { - table.setSortColumn(currentColumn); - dir = SWT.UP; - } - - final int direction = dir; - final AbstractFailureEvent[] failureEvents = new AbstractFailureEvent[table.getItemCount()]; - for (int i = 0; i < table.getItemCount(); ++i) { - failureEvents[i] = (AbstractFailureEvent) failureEventTable.getItem(i).getData(); - } - Arrays.sort(failureEvents, new Comparator() { - - @Override - public int compare(final AbstractFailureEvent o1, AbstractFailureEvent o2) { - - if (o1 == null) { - return -1; - } - - if (o2 == null) { - return 1; - } - - if (currentColumn == iconColumn) { - - final int v1 = (o1 instanceof VertexFailureEvent) ? 0 : 1; - final int v2 = (o2 instanceof VertexFailureEvent) ? 0 : 1; - return (direction == SWT.UP) ? (v1 - v2) : (v2 - v1); - - } else if (currentColumn == nameColumn) { - - if (direction == SWT.UP) { - return String.CASE_INSENSITIVE_ORDER.compare(o1.getName(), o2.getName()); - } else { - return String.CASE_INSENSITIVE_ORDER.compare(o2.getName(), o1.getName()); - } - - } else { - - if (direction == SWT.UP) { - return (o1.getInterval() - o2.getInterval()); - } else { - return (o2.getInterval() - o1.getInterval()); - } - } - } - }); - - failureEventTable.removeAll(); - for (int i = 0; i < failureEvents.length; ++i) { - updateTableItem(null, failureEvents[i]); - } - - failureEventTable.setSortColumn(currentColumn); - failureEventTable.setSortDirection(direction); - } - }; - - iconColumn.addListener(SWT.Selection, sortListener); - nameColumn.addListener(SWT.Selection, sortListener); - intervalColumn.addListener(SWT.Selection, sortListener); - - // Implement keyboard commands - table.addKeyListener(new KeyAdapter() { - - @Override - public void keyReleased(final KeyEvent arg0) { - - if (arg0.keyCode != SWT.DEL) { - return; - } - - removeSelectedTableItems(); - } - }); - - // Set the menu - table.setMenu(createTableContextMenu()); - - return table; - } - - private void removeSelectedTableItems() { - - final TableItem[] selectedItems = this.failureEventTable.getSelection(); - if (selectedItems == null) { - return; - } - - for (final TableItem selectedItem : selectedItems) { - removeTableItem(selectedItem); - } - } - - private void removeTableItem(final TableItem ti) { - - final AbstractFailureEvent event = (AbstractFailureEvent) ti.getData(); - if (event == null) { - return; - } - - final MessageBox messageBox = new MessageBox(shell, SWT.ICON_QUESTION | SWT.YES | SWT.NO); - messageBox.setText("Confirm removal"); - messageBox.setMessage("Do you really want to remove the event '" + event.getName() + "'"); - if (messageBox.open() == SWT.YES) { - ti.dispose(); - } - - this.selectedFailurePattern.removeEvent(event); - } - - private void updateTableItem(TableItem ti, final AbstractFailureEvent event) { - - boolean newItemCreated = false; - - if (ti == null) { - - final int index = (failureEventTable.getItemCount() == 0) ? 0 : (this.failureEventTable.getItemCount() - 1); - - ti = new TableItem(this.failureEventTable, SWT.NONE, index); - newItemCreated = true; - } - - if (event != null) { - if (event instanceof VertexFailureEvent) { - ti.setText(0, "T"); - } else { - ti.setText(0, "I"); - } - - ti.setText(1, event.getName()); - ti.setText(2, Integer.toString(event.getInterval())); + this.failureEventTable.showFailurePattern(null); } - // Add new blank item if the old one has been used to create the new event - if (ti.getData() == null && !newItemCreated) { - new TableItem(this.failureEventTable, SWT.NONE); - } - - ti.setData(event); } public Map show() { @@ -397,77 +170,6 @@ public Map show() { return this.loadedPatterns; } - private Menu createTableContextMenu() { - - final Menu tableContextMenu = new Menu(this.shell); - - final MenuItem createItem = new MenuItem(tableContextMenu, SWT.PUSH); - createItem.setText("Create..."); - createItem.addSelectionListener(new SelectionAdapter() { - - @Override - public void widgetSelected(final SelectionEvent arg0) { - // TODO: Implement me - } - - }); - - final MenuItem editItem = new MenuItem(tableContextMenu, SWT.PUSH); - editItem.setText("Edit..."); - editItem.addSelectionListener(new SelectionAdapter() { - - @Override - public void widgetSelected(final SelectionEvent arg0) { - // TODO: Implement me - } - - }); - - final MenuItem removeItem = new MenuItem(tableContextMenu, SWT.PUSH); - removeItem.setText("Remove..."); - removeItem.addSelectionListener(new SelectionAdapter() { - - @Override - public void widgetSelected(final SelectionEvent arg0) { - removeSelectedTableItems(); - } - }); - - tableContextMenu.addMenuListener(new MenuAdapter() { - - @Override - public void menuShown(final MenuEvent arg0) { - - TableItem[] selectedItems = failureEventTable.getSelection(); - if (selectedItems == null) { - - return; - } - - if (selectedItems.length == 0) { - - editItem.setEnabled(false); - removeItem.setEnabled(false); - - return; - } - - if (selectedItems[0].getData() == null) { - - editItem.setEnabled(false); - removeItem.setEnabled(false); - - return; - } - - editItem.setEnabled(true); - removeItem.setEnabled(true); - } - }); - - return tableContextMenu; - } - private Menu createTreeContextMenu() { final Menu treeContextMenu = new Menu(this.shell); @@ -550,7 +252,7 @@ private void createNewFailurePattern() { this.loadedPatterns.put(jobFailurePattern.getName(), jobFailurePattern); addFailurePatternToTree(jobFailurePattern); - displayFailurePattern(jobFailurePattern); + this.failureEventTable.showFailurePattern(jobFailurePattern); } private void deleteFailurePattern() { @@ -589,7 +291,7 @@ private void loadFailurePattern() { this.loadedPatterns.put(failurePattern.getName(), failurePattern); addFailurePatternToTree(failurePattern); - displayFailurePattern(failurePattern); + this.failureEventTable.showFailurePattern(failurePattern); } private void addFailurePatternToTree(final JobFailurePattern failurePattern) { @@ -789,33 +491,4 @@ private String extractValueFromElement(final Element element) throws Exception { return childText.getTextContent(); } - - private void displayFailurePattern(final JobFailurePattern jobFailurePattern) { - - // Clear old content from event table - this.failureEventTable.clearAll(); - - if (jobFailurePattern == null) { - this.failureEventTable.setEnabled(false); - return; - } - - this.failureEventTable.setEnabled(true); - - final Iterator it = jobFailurePattern.iterator(); - while (it.hasNext()) { - - final AbstractFailureEvent event = it.next(); - final TableItem ti = new TableItem(this.failureEventTable, SWT.NONE); - ti.setData(event); - updateTableItem(ti, event); - } - - System.out.println("Created empty item"); - // Finally, add item to create new entry in both tables - new TableItem(this.failureEventTable, SWT.NONE); - - this.selectedFailurePattern = jobFailurePattern; - } - } From 5d454e4102d6893ec252d88d0a27450c94190f3f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 13 Jan 2012 17:33:15 +0100 Subject: [PATCH 140/310] Added icons to failure event table --- .../swt/SWTFailureEventTable.java | 51 +++++++++++++++++- .../nephele/visualization/swt/nodeicon.png | Bin 0 -> 761 bytes .../nephele/visualization/swt/taskicon.png | Bin 0 -> 514 bytes 3 files changed, 49 insertions(+), 2 deletions(-) create mode 100755 nephele/nephele-visualization/src/main/resources/eu/stratosphere/nephele/visualization/swt/nodeicon.png create mode 100755 nephele/nephele-visualization/src/main/resources/eu/stratosphere/nephele/visualization/swt/taskicon.png diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java index bebc820fd9095..e3b9260858166 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java @@ -15,11 +15,14 @@ package eu.stratosphere.nephele.visualization.swt; +import java.io.InputStream; import java.util.Arrays; import java.util.Comparator; import java.util.Iterator; import java.util.Set; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.eclipse.swt.SWT; import org.eclipse.swt.events.KeyAdapter; import org.eclipse.swt.events.KeyEvent; @@ -29,9 +32,12 @@ import org.eclipse.swt.events.MouseEvent; import org.eclipse.swt.events.SelectionAdapter; import org.eclipse.swt.events.SelectionEvent; +import org.eclipse.swt.graphics.GC; +import org.eclipse.swt.graphics.Image; import org.eclipse.swt.graphics.Point; import org.eclipse.swt.layout.FillLayout; import org.eclipse.swt.widgets.Composite; +import org.eclipse.swt.widgets.Display; import org.eclipse.swt.widgets.Event; import org.eclipse.swt.widgets.Listener; import org.eclipse.swt.widgets.Menu; @@ -41,12 +47,22 @@ import org.eclipse.swt.widgets.TableColumn; import org.eclipse.swt.widgets.TableItem; +import eu.stratosphere.nephele.util.StringUtils; + public final class SWTFailureEventTable extends Composite { + private static final Log LOG = LogFactory.getLog(SWTFailureEventTable.class); + + private static final int DEFAULT_ICON_SIZE = 16; + private static final int ICON_COLUMN_WIDTH = 20; private static final int TEXT_COLUMN_WIDTH = 200; + private static Image TASK_ICON = null; + + private static Image INSTANCE_ICON = null; + private final Table failureEventTable; private final Set nameSuggestions; @@ -56,6 +72,18 @@ public final class SWTFailureEventTable extends Composite { public SWTFailureEventTable(final Composite parent, final int style, final Set nameSuggestions) { super(parent, SWT.NONE); + // Load images + synchronized (SWTFailureEventTable.class) { + + if (TASK_ICON == null) { + TASK_ICON = loadIcon("/eu/stratosphere/nephele/visualization/swt/taskicon.png"); + } + + if (INSTANCE_ICON == null) { + INSTANCE_ICON = loadIcon("/eu/stratosphere/nephele/visualization/swt/nodeicon.png"); + } + } + this.nameSuggestions = nameSuggestions; setLayout(new FillLayout()); @@ -185,6 +213,25 @@ public void keyReleased(final KeyEvent arg0) { this.failureEventTable.setMenu(createTableContextMenu()); } + private Image loadIcon(final String path) { + + final Display display = getDisplay(); + final InputStream in = getClass().getResourceAsStream(path); + try { + return new Image(display, in); + } catch (Exception e) { + LOG.warn(StringUtils.stringifyException(e)); + } + + final Image image = new Image(display, DEFAULT_ICON_SIZE, DEFAULT_ICON_SIZE); + final GC gc = new GC(image); + gc.setBackground(display.getSystemColor(SWT.COLOR_WHITE)); + gc.fillRectangle(image.getBounds()); + gc.dispose(); + + return image; + } + private void updateTableItem(TableItem ti, final AbstractFailureEvent event) { boolean newItemCreated = false; @@ -199,9 +246,9 @@ private void updateTableItem(TableItem ti, final AbstractFailureEvent event) { if (event != null) { if (event instanceof VertexFailureEvent) { - ti.setText(0, "T"); + ti.setImage(TASK_ICON); } else { - ti.setText(0, "I"); + ti.setImage(INSTANCE_ICON); } ti.setText(1, event.getName()); diff --git a/nephele/nephele-visualization/src/main/resources/eu/stratosphere/nephele/visualization/swt/nodeicon.png b/nephele/nephele-visualization/src/main/resources/eu/stratosphere/nephele/visualization/swt/nodeicon.png new file mode 100755 index 0000000000000000000000000000000000000000..85e5aa4257d23f03012a4088f86046b8185bd6e4 GIT binary patch literal 761 zcmVPx#24YJ`L;(K){{a7>y{D4^000SaNLh0L01FcU01FcV0GgZ_00007bV*G`2iyS- z4?8&y$)}V60013nR9JLFZ*6U5Zgc_CX>@2HM@dakWG-a~0006| zNkl>v$>_K56M!)Ae=Cz)I`7uszIDS|jsz(@H$sU>kb0gyriL?}56i3T z9O;)*L9Dj@MA!XmeU~pbJKb&%ELn$NibKutT#J^Q@(ZeDuQSbwyp zs;Yt*;{|M4R*?7|#5Q%dG<}x=TyH-)c(t<)P&5#cq&c*lPzlEBBEqvHNX8ey%(W~u zy%EC1)YuU8L=>{>fa-8TakwDK3gY1q;*k(+i?8RAt12r`#?$9M%+3FVm0Ct@K8WPv zFC^m&kcyP5002ot<*o{ZqY1q1^`T_bR&Y~?5DYfi8zM3QvKh{|oknPS29alZ2G9il$RuXbvYVZi2HEhy)}{MqWRQrT~D5(B0n;4K{?uU@-$0 zTLU+Ba8_J}ns-oDTxmFtq_F};*1%02l=CzFMW~8gsDN>f;g4Stjl}@LAh-#}3@}sA zED|P`5=p%APfQgkux+wcRgxzse;_CbM-CqZHw_Tk>m$qs20x51_XT*@_;Bx|0>JxE zvn`&TWv}xez$ literal 0 HcmV?d00001 diff --git a/nephele/nephele-visualization/src/main/resources/eu/stratosphere/nephele/visualization/swt/taskicon.png b/nephele/nephele-visualization/src/main/resources/eu/stratosphere/nephele/visualization/swt/taskicon.png new file mode 100755 index 0000000000000000000000000000000000000000..a7ddb333585ce338eb701f268a6646b1c69c6d2b GIT binary patch literal 514 zcmeAS@N?(olHy`uVBq!ia0vp^0wB!61|;P_|4#%`jKx9jP7LeL$-D$|*pj^6U4S$Y z{B+)352QE?JR*x37`TN&n2}-D90{Nxdx@v7EBhTrUIAIjhs+O5fkLvGArU3c`MJ5N zc_lzD1A}u>YGO%hib8p2Nrr;Er*A-tUMf4#x=2qK#}JFt$q5n}2ReKWQh!}tv;2MC zu?+^sQU88@?r)qs^<^iA{_Ay>kya-wcr6OPuKn;NxBr}7?SJoioM&JB|1bHk`_o3F zU9UK;pI4oHe>lq_AeKntv_aZ+i0pfsa9F&&%iD@Av*V z^7;M!{A2b1>x>^9ir@eL$NtxcPFEV#>KGjR?Qd@+`Tu|7kN?vPcGy<@J1KFvp?v;+ z&3}JCcdwY>zKuEWzf(}W-YzhD3VzrN1!!6BWVdP(2VR%b(=1Rw~QEGRHV z@Y$d5&(E*_|NsBrM2DmM{}*pEGI*iIVIkfkHTTfWDM0b_Gp=pj`~2bQ{=|Iy+UY;P w{^oC20PU(9nW)5wRnd76Q7{KNeCVxbG$=7K`s)78&qol`;+0Q-mKz5oCK literal 0 HcmV?d00001 From 29e2e7a63995ebedf7ed96da6dc048441e055811 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 13 Jan 2012 19:49:40 +0100 Subject: [PATCH 141/310] Finished I/O operations of failure patterns manager --- .../swt/JobFailurePatternTreeListener.java | 31 +++ .../swt/SWTFailureEventTable.java | 2 +- .../swt/SWTFailurePatternsEditor.java | 213 ++++++++++-------- .../swt/SWTJobFailurePatternTree.java | 187 +++++++++++++++ 4 files changed, 343 insertions(+), 90 deletions(-) create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternTreeListener.java create mode 100644 nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTJobFailurePatternTree.java diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternTreeListener.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternTreeListener.java new file mode 100644 index 0000000000000..ac2e39698a361 --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternTreeListener.java @@ -0,0 +1,31 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +import org.eclipse.swt.widgets.TreeItem; + +public interface JobFailurePatternTreeListener { + + void addFailurePattern(); + + void removeFailurePattern(TreeItem selectedItem); + + void saveFailurePattern(TreeItem selectedItem); + + void loadFailurePattern(); + + void jobFailurePatternSelected(TreeItem selectedItem); +} diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java index e3b9260858166..44c658e665557 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java @@ -401,7 +401,7 @@ private void removeTableItem(final TableItem ti) { public void showFailurePattern(final JobFailurePattern jobFailurePattern) { // Clear old content from event table - this.failureEventTable.clearAll(); + this.failureEventTable.removeAll(); if (jobFailurePattern == null) { this.failureEventTable.setEnabled(false); diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java index 79c222a59ef6b..a48cc67bdf0c6 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java @@ -16,6 +16,7 @@ package eu.stratosphere.nephele.visualization.swt; import java.io.FileInputStream; +import java.io.FileOutputStream; import java.io.InputStream; import java.util.Iterator; import java.util.Map; @@ -23,11 +24,14 @@ import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.transform.OutputKeys; +import javax.xml.transform.Transformer; +import javax.xml.transform.TransformerFactory; +import javax.xml.transform.dom.DOMSource; +import javax.xml.transform.stream.StreamResult; import org.eclipse.swt.SWT; import org.eclipse.swt.custom.SashForm; -import org.eclipse.swt.events.MenuAdapter; -import org.eclipse.swt.events.MenuEvent; import org.eclipse.swt.events.SelectionAdapter; import org.eclipse.swt.events.SelectionEvent; import org.eclipse.swt.layout.FillLayout; @@ -39,18 +43,15 @@ import org.eclipse.swt.widgets.FileDialog; import org.eclipse.swt.widgets.Group; import org.eclipse.swt.widgets.Label; -import org.eclipse.swt.widgets.Menu; -import org.eclipse.swt.widgets.MenuItem; import org.eclipse.swt.widgets.MessageBox; import org.eclipse.swt.widgets.Shell; -import org.eclipse.swt.widgets.Tree; import org.eclipse.swt.widgets.TreeItem; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.Node; import org.w3c.dom.NodeList; -public final class SWTFailurePatternsEditor extends SelectionAdapter { +public final class SWTFailurePatternsEditor implements JobFailurePatternTreeListener { private static final int WIDTH = 800; @@ -58,7 +59,7 @@ public final class SWTFailurePatternsEditor extends SelectionAdapter { private final Shell shell; - private final Tree jobTree; + private final SWTJobFailurePatternTree jobTree; private final SWTFailureEventTable failureEventTable; @@ -103,9 +104,7 @@ public final class SWTFailurePatternsEditor extends SelectionAdapter { jobGroup.setText("Job Failure Patterns"); jobGroup.setLayout(new FillLayout()); - this.jobTree = new Tree(jobGroup, SWT.SINGLE | SWT.BORDER); - this.jobTree.addSelectionListener(this); - this.jobTree.setMenu(createTreeContextMenu()); + this.jobTree = new SWTJobFailurePatternTree(jobGroup, SWT.SINGLE | SWT.BORDER, this); this.failureEventTable = new SWTFailureEventTable(horizontalSash, SWT.BORDER | SWT.SINGLE, nameSuggestions); horizontalSash.setWeights(new int[] { 2, 8 }); @@ -139,7 +138,7 @@ public void widgetSelected(SelectionEvent arg0) { final Iterator it = this.loadedPatterns.values().iterator(); while (it.hasNext()) { - addFailurePatternToTree(it.next()); + this.jobTree.addFailurePatternToTree(it.next()); } // Initialize the tables @@ -170,100 +169,143 @@ public Map show() { return this.loadedPatterns; } - private Menu createTreeContextMenu() { + @Override + public void addFailurePattern() { - final Menu treeContextMenu = new Menu(this.shell); - final MenuItem createItem = new MenuItem(treeContextMenu, SWT.PUSH); - createItem.setText("Create..."); - createItem.addSelectionListener(new SelectionAdapter() { + final Set takenNames = this.loadedPatterns.keySet(); - @Override - public void widgetSelected(final SelectionEvent arg0) { - createNewFailurePattern(); - } - }); - new MenuItem(treeContextMenu, SWT.SEPARATOR); - final MenuItem deleteItem = new MenuItem(treeContextMenu, SWT.PUSH); - deleteItem.setText("Delete..."); - deleteItem.addSelectionListener(new SelectionAdapter() { + final SWTNewFailurePatternDialog dialog = new SWTNewFailurePatternDialog(this.shell, this.jobSuggestions, + takenNames); - @Override - public void widgetSelected(final SelectionEvent arg0) { - deleteFailurePattern(); - } - }); - new MenuItem(treeContextMenu, SWT.SEPARATOR); - final MenuItem saveItem = new MenuItem(treeContextMenu, SWT.PUSH); - saveItem.setText("Save..."); - saveItem.addSelectionListener(new SelectionAdapter() { + final String patternName = dialog.showDialog(); + if (patternName == null) { + return; + } - @Override - public void widgetSelected(final SelectionEvent arg0) { - saveFailurePattern(); - } - }); - final MenuItem loadItem = new MenuItem(treeContextMenu, SWT.PUSH); - loadItem.setText("Load..."); - loadItem.addSelectionListener(new SelectionAdapter() { + final JobFailurePattern jobFailurePattern = new JobFailurePattern(patternName); - @Override - public void widgetSelected(final SelectionEvent arg0) { - loadFailurePattern(); - } - }); + // Add to loaded patterns + this.loadedPatterns.put(jobFailurePattern.getName(), jobFailurePattern); - treeContextMenu.addMenuListener(new MenuAdapter() { + this.jobTree.addFailurePatternToTree(jobFailurePattern); + this.failureEventTable.showFailurePattern(jobFailurePattern); + } - @Override - public void menuShown(final MenuEvent arg0) { + @Override + public void removeFailurePattern(final TreeItem selectedItem) { - if (jobTree.getSelection().length == 0) { - createItem.setEnabled(true); - deleteItem.setEnabled(false); - saveItem.setEnabled(false); - loadItem.setEnabled(true); - } else { - createItem.setEnabled(false); - deleteItem.setEnabled(true); - saveItem.setEnabled(true); - loadItem.setEnabled(false); - } - } - }); + final JobFailurePattern failurePattern = (JobFailurePattern) selectedItem.getData(); + if (failurePattern == null) { + return; + } - return treeContextMenu; + final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_QUESTION | SWT.YES | SWT.NO); + messageBox.setText("Confirm Removal"); + messageBox + .setMessage("Do you really want to remove the job failure pattern '" + failurePattern.getName() + "'?"); + + if (messageBox.open() != SWT.YES) { + return; + } + + selectedItem.dispose(); + + this.loadedPatterns.remove(failurePattern.getName()); + + if (this.jobTree.getItemCount() == 0) { + jobFailurePatternSelected(null); + } else { + jobFailurePatternSelected(this.jobTree.getItem(0)); + } } - private void createNewFailurePattern() { + @Override + public void saveFailurePattern(final TreeItem selectedItem) { - final Set takenNames = this.loadedPatterns.keySet(); + final JobFailurePattern failurePattern = (JobFailurePattern) selectedItem.getData(); + if (failurePattern == null) { + return; + } - final SWTNewFailurePatternDialog dialog = new SWTNewFailurePatternDialog(this.shell, this.jobSuggestions, - takenNames); + final FileDialog fileDialog = new FileDialog(this.shell, SWT.SAVE); + fileDialog.setText("Save Failure Pattern"); + final String[] filterExts = { "*.xml", "*.*" }; + fileDialog.setFilterExtensions(filterExts); - final String patternName = dialog.showDialog(); - if (patternName == null) { + final String selectedFile = fileDialog.open(); + if (selectedFile == null) { return; } - final JobFailurePattern jobFailurePattern = new JobFailurePattern(patternName); + final DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory.newInstance(); - // Add to loaded patterns - this.loadedPatterns.put(jobFailurePattern.getName(), jobFailurePattern); + try { + final DocumentBuilder builder = docBuilderFactory.newDocumentBuilder(); - addFailurePatternToTree(jobFailurePattern); - this.failureEventTable.showFailurePattern(jobFailurePattern); - } + final Document doc = builder.newDocument(); + + // Construct the DOM tree + final Element root = doc.createElement("pattern"); + doc.appendChild(root); + + final Element name = doc.createElement("name"); + root.appendChild(name); + name.appendChild(doc.createTextNode(failurePattern.getName())); + + final Element failures = doc.createElement("failures"); + root.appendChild(failures); + + final Iterator it = failurePattern.iterator(); + while (it.hasNext()) { + + final AbstractFailureEvent event = it.next(); + + final Element failure = doc.createElement("failure"); + failure.setAttribute("type", (event instanceof VertexFailureEvent) ? "task" : "instance"); + + final Element failureName = doc.createElement("name"); + failureName.appendChild(doc.createTextNode(event.getName())); + failure.appendChild(failureName); + + final Element interval = doc.createElement("interval"); + interval.appendChild(doc.createTextNode(Integer.toString(event.getInterval()))); + failure.appendChild(interval); + + failures.appendChild(failure); + } + + // Write the DOM tree to the chosen file + final DOMSource domSource = new DOMSource(doc); + final TransformerFactory transformerFactory = TransformerFactory.newInstance(); + + final FileOutputStream fos = new FileOutputStream(selectedFile); + final Transformer transformer = transformerFactory.newTransformer(); + transformer.setOutputProperty(OutputKeys.INDENT, "yes"); + transformer.setOutputProperty(OutputKeys.ENCODING, "UTF-8"); + transformer.transform(domSource, new StreamResult(fos)); + + } catch (Exception e) { + + final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); + messageBox.setText("Cannot load failure pattern"); + messageBox.setMessage(e.getMessage()); + messageBox.open(); + } - private void deleteFailurePattern() { - // TODO: Implement me } - private void saveFailurePattern() { - // TODO: Implement me + @Override + public void jobFailurePatternSelected(final TreeItem selectedItem) { + + if (selectedItem == null) { + this.failureEventTable.showFailurePattern(null); + } else { + this.failureEventTable.showFailurePattern((JobFailurePattern) selectedItem.getData()); + } } - private void loadFailurePattern() { + @Override + public void loadFailurePattern() { final FileDialog fileDialog = new FileDialog(this.shell, SWT.OPEN); fileDialog.setText("Load Failure Pattern"); @@ -290,17 +332,10 @@ private void loadFailurePattern() { this.loadedPatterns.put(failurePattern.getName(), failurePattern); - addFailurePatternToTree(failurePattern); + this.jobTree.addFailurePatternToTree(failurePattern); this.failureEventTable.showFailurePattern(failurePattern); } - private void addFailurePatternToTree(final JobFailurePattern failurePattern) { - - final TreeItem jobFailureItem = new TreeItem(this.jobTree, SWT.NONE); - jobFailureItem.setText(failurePattern.getName()); - jobFailureItem.setData(failurePattern); - } - private JobFailurePattern loadFailurePatternFromFile(final String filename) { final DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory.newInstance(); diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTJobFailurePatternTree.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTJobFailurePatternTree.java new file mode 100644 index 0000000000000..b36906b6a8a5c --- /dev/null +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTJobFailurePatternTree.java @@ -0,0 +1,187 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.visualization.swt; + +import org.eclipse.swt.SWT; +import org.eclipse.swt.events.KeyAdapter; +import org.eclipse.swt.events.KeyEvent; +import org.eclipse.swt.events.MouseAdapter; +import org.eclipse.swt.events.MouseEvent; +import org.eclipse.swt.events.SelectionAdapter; +import org.eclipse.swt.events.SelectionEvent; +import org.eclipse.swt.graphics.Point; +import org.eclipse.swt.layout.FillLayout; +import org.eclipse.swt.widgets.Composite; +import org.eclipse.swt.widgets.Menu; +import org.eclipse.swt.widgets.MenuItem; +import org.eclipse.swt.widgets.Tree; +import org.eclipse.swt.widgets.TreeItem; + +public final class SWTJobFailurePatternTree extends Composite { + + private final Tree jobTree; + + private final MenuItem addItem; + + private final MenuItem removeItem; + + private final MenuItem saveItem; + + private final MenuItem loadItem; + + SWTJobFailurePatternTree(final Composite parent, final int style, final JobFailurePatternTreeListener treeListener) { + super(parent, SWT.NONE); + + setLayout(new FillLayout()); + + final Menu treeContextMenu = new Menu(getShell()); + this.addItem = new MenuItem(treeContextMenu, SWT.PUSH); + this.addItem.setText("Add..."); + this.addItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + treeListener.addFailurePattern(); + } + }); + + new MenuItem(treeContextMenu, SWT.SEPARATOR); + + this.removeItem = new MenuItem(treeContextMenu, SWT.PUSH); + this.removeItem.setText("Remove..."); + this.removeItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + + final TreeItem[] selectedItems = jobTree.getSelection(); + if (selectedItems == null) { + return; + } + + for (final TreeItem selectedItem : selectedItems) { + treeListener.removeFailurePattern(selectedItem); + } + } + }); + new MenuItem(treeContextMenu, SWT.SEPARATOR); + this.saveItem = new MenuItem(treeContextMenu, SWT.PUSH); + this.saveItem.setText("Save..."); + this.saveItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + + final TreeItem[] selectedItems = jobTree.getSelection(); + if (selectedItems == null) { + return; + } + + for (final TreeItem selectedItem : selectedItems) { + treeListener.saveFailurePattern(selectedItem); + } + } + }); + this.loadItem = new MenuItem(treeContextMenu, SWT.PUSH); + this.loadItem.setText("Load..."); + this.loadItem.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + treeListener.loadFailurePattern(); + } + }); + + this.jobTree = new Tree(this, style); + this.jobTree.setMenu(treeContextMenu); + this.jobTree.addMouseListener(new MouseAdapter() { + + @Override + public void mouseDown(final MouseEvent arg0) { + + final boolean itemSelected = (jobTree.getItem(new Point(arg0.x, arg0.y)) != null); + addItem.setEnabled(!itemSelected); + removeItem.setEnabled(itemSelected); + saveItem.setEnabled(itemSelected); + loadItem.setEnabled(!itemSelected); + } + }); + + this.jobTree.addKeyListener(new KeyAdapter() { + + @Override + public void keyReleased(final KeyEvent arg0) { + + if (arg0.keyCode == SWT.DEL) { + + final TreeItem[] selectedItems = jobTree.getSelection(); + if (selectedItems == null) { + return; + } + + for (final TreeItem selectedItem : selectedItems) { + treeListener.removeFailurePattern(selectedItem); + } + } + } + + }); + + this.jobTree.addSelectionListener(new SelectionAdapter() { + + @Override + public void widgetSelected(final SelectionEvent arg0) { + + final TreeItem[] selectedItems = jobTree.getSelection(); + + if (selectedItems == null) { + return; + } + + for (final TreeItem selectedItem : selectedItems) { + treeListener.jobFailurePatternSelected(selectedItem); + } + } + }); + } + + public int getItemCount() { + + return this.jobTree.getItemCount(); + } + + public TreeItem getItem(final int index) { + + return this.jobTree.getItem(index); + } + + public void setSelection(final TreeItem treeItem) { + + this.jobTree.setSelection(treeItem); + } + + public TreeItem[] getSelection() { + + return this.jobTree.getSelection(); + } + + public void addFailurePatternToTree(final JobFailurePattern failurePattern) { + + final TreeItem jobFailureItem = new TreeItem(this.jobTree, SWT.NONE); + jobFailureItem.setText(failurePattern.getName()); + jobFailureItem.setData(failurePattern); + } +} From 23c732905b0edb85619b031b19a32f1a44df243b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 13 Jan 2012 20:02:18 +0100 Subject: [PATCH 142/310] Minor corrections and bug fixes --- .../visualization/swt/SWTFailureEventEditor.java | 4 ++-- .../visualization/swt/SWTFailurePatternsEditor.java | 10 ++++++---- .../visualization/swt/SWTJobFailurePatternTree.java | 2 ++ .../visualization/swt/SWTNewFailurePatternDialog.java | 2 +- 4 files changed, 11 insertions(+), 7 deletions(-) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java index ac6388cb47843..506ea6efdb896 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventEditor.java @@ -85,9 +85,9 @@ public SWTFailureEventEditor(final Shell parent, final Set nameSuggestio // Determine the correct title for the window String title = null; if (failureEvent == null) { - title = "Create new failure event"; + title = "Add Failure Event"; } else { - title = "Edit failure event"; + title = "Edit Failure Event"; } this.shell.setText(title); diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java index a48cc67bdf0c6..2ec04e5abef43 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsEditor.java @@ -215,7 +215,9 @@ public void removeFailurePattern(final TreeItem selectedItem) { if (this.jobTree.getItemCount() == 0) { jobFailurePatternSelected(null); } else { - jobFailurePatternSelected(this.jobTree.getItem(0)); + final TreeItem ti = this.jobTree.getItem(0); + this.jobTree.setSelection(ti); + jobFailurePatternSelected(ti); } } @@ -287,7 +289,7 @@ public void saveFailurePattern(final TreeItem selectedItem) { } catch (Exception e) { final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); - messageBox.setText("Cannot load failure pattern"); + messageBox.setText("Cannot Save Failure Pattern"); messageBox.setMessage(e.getMessage()); messageBox.open(); } @@ -322,7 +324,7 @@ public void loadFailurePattern() { if (this.loadedPatterns.containsKey(failurePattern.getName())) { final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); - messageBox.setText("Cannot load failure pattern"); + messageBox.setText("Cannot Load Failure Pattern"); messageBox.setMessage("There is already a failure pattern loaded with the name '" + failurePattern.getName() + "'. Please remove it first."); messageBox.open(); @@ -493,7 +495,7 @@ private JobFailurePattern loadFailurePatternFromFile(final String filename) { } catch (Exception e) { final MessageBox messageBox = new MessageBox(this.shell, SWT.ICON_ERROR); - messageBox.setText("Cannot load failure pattern"); + messageBox.setText("Cannot Load Failure Pattern"); messageBox.setMessage(e.getMessage()); messageBox.open(); return null; diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTJobFailurePatternTree.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTJobFailurePatternTree.java index b36906b6a8a5c..52c3ecd60234d 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTJobFailurePatternTree.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTJobFailurePatternTree.java @@ -183,5 +183,7 @@ public void addFailurePatternToTree(final JobFailurePattern failurePattern) { final TreeItem jobFailureItem = new TreeItem(this.jobTree, SWT.NONE); jobFailureItem.setText(failurePattern.getName()); jobFailureItem.setData(failurePattern); + + this.jobTree.setSelection(jobFailureItem); } } diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java index fc56c27cf229b..3d0ec472fef46 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTNewFailurePatternDialog.java @@ -86,7 +86,7 @@ public SWTNewFailurePatternDialog(final Shell parent, final Set nameSugg this.shell = new Shell(parent); this.shell.setSize(WIDTH, HEIGHT); - this.shell.setText("Create New Failure Pattern"); + this.shell.setText("Add New Failure Pattern"); this.shell.setLayout(new GridLayout(1, false)); GridData gridData = new GridData(); From c9aee78e0eaf9b3c9fbdfb785fc2cef853cddc4f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 13 Jan 2012 20:28:51 +0100 Subject: [PATCH 143/310] Implemented name suggestions --- .../swt/SWTFailureEventTable.java | 6 ++--- .../swt/SWTVisualizationGUI.java | 27 ++++++++++++++++--- 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java index 44c658e665557..7046e04f3360f 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailureEventTable.java @@ -267,9 +267,9 @@ private Menu createTableContextMenu() { final Menu tableContextMenu = new Menu(getShell()); - final MenuItem createItem = new MenuItem(tableContextMenu, SWT.PUSH); - createItem.setText("Create..."); - createItem.addSelectionListener(new SelectionAdapter() { + final MenuItem addItem = new MenuItem(tableContextMenu, SWT.PUSH); + addItem.setText("Add..."); + addItem.addSelectionListener(new SelectionAdapter() { @Override public void widgetSelected(final SelectionEvent arg0) { diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java index aa242b983b1c1..607132978a3df 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java @@ -16,7 +16,6 @@ package eu.stratosphere.nephele.visualization.swt; import java.io.IOException; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -706,10 +705,30 @@ private void cleanUpOldEvents(long sleepTime) { private void manageFailurePatterns() { - // TODO: Replace empty set by sensible suggestions - final Set emptySet = Collections.emptySet(); + final Set jobSuggestions = new HashSet(); + final Set nameSuggestions = new HashSet(); - this.failurePatternsManager.openEditor(this.shell, emptySet, emptySet); + final Iterator it = this.recentJobs.values().iterator(); + while (it.hasNext()) { + + final GraphVisualizationData gvd = it.next(); + + jobSuggestions.add(gvd.getJobName()); + + final ManagementGraphIterator mgi = new ManagementGraphIterator(gvd.getManagementGraph(), true); + while (mgi.hasNext()) { + + final ManagementVertex vertex = mgi.next(); + final String vertexName = (vertex.getName() != null) ? vertex.getName() : "null"; + final String vertexNameWithIndex = vertexName + " " + (vertex.getIndexInGroup() + 1); + nameSuggestions.add(vertexNameWithIndex); + if (vertex.getInstanceName() != null) { + nameSuggestions.add(vertex.getInstanceName()); + } + } + } + + this.failurePatternsManager.openEditor(this.shell, jobSuggestions, nameSuggestions); } private void logBufferUtilization() { From f3ff1f8a0cb379a8b981bd53e41381a8e86e989b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 13 Jan 2012 21:45:20 +0100 Subject: [PATCH 144/310] Implemented scheduling of failure events --- .../swt/JobFailurePatternExecutor.java | 102 ++++++++++++++---- .../swt/SWTFailurePatternsManager.java | 30 ++++-- .../swt/SWTVisualizationGUI.java | 17 +-- 3 files changed, 112 insertions(+), 37 deletions(-) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternExecutor.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternExecutor.java index 28b74810a69c9..a1bbcc8df320f 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternExecutor.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/JobFailurePatternExecutor.java @@ -15,30 +15,69 @@ package eu.stratosphere.nephele.visualization.swt; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.eclipse.swt.widgets.Display; import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.managementgraph.ManagementGraph; +import eu.stratosphere.nephele.managementgraph.ManagementGraphIterator; +import eu.stratosphere.nephele.managementgraph.ManagementVertex; +import eu.stratosphere.nephele.managementgraph.ManagementVertexID; +import eu.stratosphere.nephele.protocols.ExtendedManagementProtocol; +import eu.stratosphere.nephele.types.StringRecord; +import eu.stratosphere.nephele.util.StringUtils; public final class JobFailurePatternExecutor implements Runnable { + private static final Log LOG = LogFactory.getLog(JobFailurePatternExecutor.class); + private final Display timer; private final JobID jobID; - private final String jobName; + private final ExtendedManagementProtocol jobManager; - private long offset = 0L; + private final Map nameToIDMap; - private boolean stopRequested = false; + private final Iterator eventIterator; - private boolean executorStarted = false; + private final int offset; - JobFailurePatternExecutor(final Display timer, final JobID jobID, final String jobName, - final JobFailurePattern failurePattern) { + private AbstractFailureEvent nextEvent = null; + + private boolean stopRequested = false; + + JobFailurePatternExecutor(final Display timer, final ExtendedManagementProtocol jobManager, + final ManagementGraph managementGraph, final JobFailurePattern failurePattern, final long referenceTime) { this.timer = timer; - this.jobID = jobID; - this.jobName = jobName; + this.jobManager = jobManager; + this.jobID = managementGraph.getJobID(); + + final long now = System.currentTimeMillis(); + + this.offset = (int) (now - referenceTime); + + final Map tmpMap = new HashMap(); + final Iterator it = new ManagementGraphIterator(managementGraph, true); + while (it.hasNext()) { + + final ManagementVertex vertex = it.next(); + tmpMap.put(SWTFailurePatternsManager.getSuggestedName(vertex), vertex.getID()); + } + + this.nameToIDMap = Collections.unmodifiableMap(tmpMap); + + this.eventIterator = failurePattern.iterator(); + + scheduleNextEvent(); } /** @@ -48,35 +87,54 @@ public final class JobFailurePatternExecutor implements Runnable { public void run() { if (this.stopRequested) { + LOG.info("Stopping job failure pattern executor for job " + this.jobID); this.stopRequested = false; return; } - scheduleNextEvent(); - } - - public void start(final long referenceTime) { - - if (this.executorStarted) { - throw new IllegalStateException("The executor has already been started"); + LOG.info("Triggering event " + this.nextEvent.getName() + " for job " + this.jobID); + if (this.nextEvent instanceof VertexFailureEvent) { + + // Find out the ID of the vertex to be killed + final ManagementVertexID vertexID = this.nameToIDMap.get(this.nextEvent.getName()); + if (vertexID == null) { + LOG.error("Cannot determine ID for vertex " + this.nextEvent.getName()); + } else { + try { + this.jobManager.cancelTask(this.jobID, vertexID); + } catch (IOException ioe) { + LOG.error(StringUtils.stringifyException(ioe)); + } + } + + } else { + try { + this.jobManager.killInstance(new StringRecord(this.nextEvent.getName())); + } catch (IOException ioe) { + LOG.error(StringUtils.stringifyException(ioe)); + } } - final long now = System.currentTimeMillis(); - - this.executorStarted = true; - this.offset = now - referenceTime; - + // Schedule next event scheduleNextEvent(); } public void stop() { this.stopRequested = true; - this.executorStarted = false; } private void scheduleNextEvent() { - // TODO: Implement me + if (this.eventIterator.hasNext()) { + this.nextEvent = this.eventIterator.next(); + } else { + this.nextEvent = null; + return; + } + + final int interval = this.nextEvent.getInterval() - this.offset; + + this.timer.timerExec(interval, this); } } diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java index e1b67ba66e60d..8aad9a087595b 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTFailurePatternsManager.java @@ -24,7 +24,9 @@ import org.eclipse.swt.widgets.Display; import org.eclipse.swt.widgets.Shell; -import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.managementgraph.ManagementGraph; +import eu.stratosphere.nephele.managementgraph.ManagementVertex; +import eu.stratosphere.nephele.protocols.ExtendedManagementProtocol; public final class SWTFailurePatternsManager { @@ -32,23 +34,30 @@ public final class SWTFailurePatternsManager { private final Display display; + private final ExtendedManagementProtocol jobManager; + private Map failurePatterns = new HashMap(); - public SWTFailurePatternsManager(final Display display) { + public SWTFailurePatternsManager(final Display display, final ExtendedManagementProtocol jobManager) { + this.display = display; + this.jobManager = jobManager; } - public void startFailurePattern(final JobID jobID, final String jobName, final long referenceTime) { + public void startFailurePattern(final String jobName, final ManagementGraph managementGraph, + final long referenceTime) { final JobFailurePattern failurePattern = this.failurePatterns.get(jobName); if (failurePattern == null) { - LOG.info("No failure pattern for job " + jobName); + if (LOG.isDebugEnabled()) { + LOG.debug("No failure pattern for job " + jobName); + } + return; } - final JobFailurePatternExecutor executor = new JobFailurePatternExecutor(this.display, jobID, - jobName, failurePattern); + LOG.info("Starting failure pattern for job " + jobName); - executor.start(referenceTime); + new JobFailurePatternExecutor(this.display, this.jobManager, managementGraph, failurePattern, referenceTime); } public void openEditor(final Shell parent, final Set jobSuggestions, final Set nameSuggestions) { @@ -58,4 +67,11 @@ public void openEditor(final Shell parent, final Set jobSuggestions, fin editor.show(); } + + public static String getSuggestedName(final ManagementVertex vertex) { + + final String vertexName = (vertex.getName() != null) ? vertex.getName() : "null"; + + return vertexName + " " + (vertex.getIndexInGroup() + 1); + } } diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java index 607132978a3df..b576ebdd2b132 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVisualizationGUI.java @@ -302,7 +302,7 @@ public void handleEvent(final Event arg0) { }); // Create failure patterns manager - this.failurePatternsManager = new SWTFailurePatternsManager(this.shell.getDisplay()); + this.failurePatternsManager = new SWTFailurePatternsManager(this.shell.getDisplay(), jobManager); // Launch the timer that will query for events this.display.timerExec(QUERYINTERVAL * 1000, this); @@ -451,9 +451,7 @@ public void run() { final Iterator it = newJobs.iterator(); while (it.hasNext()) { final RecentJobEvent newJobEvent = it.next(); - addJob(newJobEvent.getJobID(), newJobEvent.getJobName(), newJobEvent.isProfilingAvailable()); - // Find a matching failure pattern and start it - this.failurePatternsManager.startFailurePattern(newJobEvent.getJobID(), newJobEvent.getJobName(), + addJob(newJobEvent.getJobID(), newJobEvent.getJobName(), newJobEvent.isProfilingAvailable(), newJobEvent.getTimestamp()); } } @@ -527,7 +525,8 @@ private void updateView() { ((SWTJobTabItem) control).updateView(); } - private void addJob(JobID jobID, String jobName, boolean isProfilingAvailable) throws IOException { + private void addJob(JobID jobID, String jobName, boolean isProfilingAvailable, final long referenceTime) + throws IOException { synchronized (this.recentJobs) { @@ -580,6 +579,9 @@ private void addJob(JobID jobID, String jobName, boolean isProfilingAvailable) t jobItem.setText(jobName + " (" + jobID.toString() + ")"); jobItem.setData(graphVisualizationData); + // Find a matching failure pattern and start it + this.failurePatternsManager.startFailurePattern(jobName, managementGraph, referenceTime); + this.recentJobs.put(jobID, graphVisualizationData); } } @@ -719,9 +721,8 @@ private void manageFailurePatterns() { while (mgi.hasNext()) { final ManagementVertex vertex = mgi.next(); - final String vertexName = (vertex.getName() != null) ? vertex.getName() : "null"; - final String vertexNameWithIndex = vertexName + " " + (vertex.getIndexInGroup() + 1); - nameSuggestions.add(vertexNameWithIndex); + final String vertexName = SWTFailurePatternsManager.getSuggestedName(vertex); + nameSuggestions.add(vertexName); if (vertex.getInstanceName() != null) { nameSuggestions.add(vertex.getInstanceName()); } From f91e1f6e5ce43cd15cc6b41c4a3ba40201550eb5 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 14 Jan 2012 19:24:42 +0100 Subject: [PATCH 145/310] Finished implementation to support task failures programmatically --- .../executiongraph/ExecutionVertex.java | 34 +++++++++ .../executiongraph/ExecutionVertexID.java | 23 +++++- .../nephele/instance/AbstractInstance.java | 17 ++++- .../nephele/jobmanager/JobManager.java | 32 +++++++- .../protocols/TaskOperationProtocol.java | 28 +++++-- .../nephele/taskmanager/Task.java | 74 +++++++++++++------ .../nephele/taskmanager/TaskCancelResult.java | 4 +- .../nephele/taskmanager/TaskKillResult.java | 47 ++++++++++++ .../nephele/taskmanager/TaskManager.java | 51 +++++++++++-- 9 files changed, 266 insertions(+), 44 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index c0708dddfa2b5..8ed09729ba25f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -40,6 +40,7 @@ import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.taskmanager.AbstractTaskResult; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; +import eu.stratosphere.nephele.taskmanager.TaskKillResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult; import eu.stratosphere.nephele.template.AbstractInvokable; import eu.stratosphere.nephele.types.Record; @@ -589,6 +590,39 @@ public TaskSubmissionResult startTask() { } } + /** + * Kills and removes the task represented by this vertex from the instance it is currently running on. If the + * corresponding task is not in the state RUNNING, this call will be ignored. If the call has been + * executed + * successfully, the task will change the state FAILED. + * + * @return the result of the task kill attempt + */ + public TaskKillResult killTask() { + + final ExecutionState state = this.executionState.get(); + + if (state != ExecutionState.RUNNING) { + final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.ERROR); + result.setDescription("Vertex " + this.toString() + " is in state " + state); + return result; + } + + if (this.allocatedResource == null) { + final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.ERROR); + result.setDescription("Assigned instance of vertex " + this.toString() + " is null!"); + return result; + } + + try { + return this.allocatedResource.getInstance().killTask(this.vertexID); + } catch (IOException e) { + final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.ERROR); + result.setDescription(StringUtils.stringifyException(e)); + return result; + } + } + /** * Cancels and removes the task represented by this vertex * from the instance it is currently running on. If the task diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java index 62c70396ca215..86f79ba46027d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java @@ -36,9 +36,26 @@ public class ExecutionVertexID extends AbstractID { */ public ManagementVertexID toManagementVertexID() { - final ManagementVertexID newId = new ManagementVertexID(); - newId.setID(this); + final ManagementVertexID newID = new ManagementVertexID(); + newID.setID(this); - return newId; + return newID; + } + + /** + * Converts the given management vertex ID into the corresponding execution vertex ID. The new execution vertex ID + * will be equals to the management vertex ID in the sense that the equals method will return + * true when both IDs are compared. + * + * @param vertexID + * the management vertex ID to be converted + * @return the resulting execution vertex ID + */ + public static ExecutionVertexID fromManagementVertexID(final ManagementVertexID vertexID) { + + final ExecutionVertexID newID = new ExecutionVertexID(); + newID.setID(vertexID); + + return newID; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index 2113e7fe02556..f2046b4a8c279 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -35,11 +35,11 @@ import eu.stratosphere.nephele.net.NetUtils; import eu.stratosphere.nephele.protocols.TaskOperationProtocol; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; +import eu.stratosphere.nephele.taskmanager.TaskKillResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionWrapper; import eu.stratosphere.nephele.topology.NetworkNode; import eu.stratosphere.nephele.topology.NetworkTopology; -import eu.stratosphere.nephele.util.SerializableHashSet; /** * An abstract instance represents a resource a {@link eu.stratosphere.nephele.taskmanager.TaskManager} runs on. @@ -235,6 +235,21 @@ public synchronized TaskCancelResult cancelTask(final ExecutionVertexID id) thro return getTaskManager().cancelTask(id); } + + /** + * Kills the task identified by the given ID at the instance's + * {@link eu.stratosphere.nephele.taskmanager.TaskManager}. + * + * @param id + * the ID identifying the task to be killed + * @throws IOException + * thrown if an error occurs while transmitting the request or receiving the response + * @return the result of the kill attempt + */ + public synchronized TaskKillResult killTask(final ExecutionVertexID id) throws IOException { + + return getTaskManager().killTask(id); + } /** * Removes the checkpoints identified by the given list of vertex IDs at the instance's diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index b715e032cea4a..d373f91e6cc4e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -116,6 +116,7 @@ import eu.stratosphere.nephele.taskmanager.TaskCancelResult; import eu.stratosphere.nephele.taskmanager.TaskCheckpointState; import eu.stratosphere.nephele.taskmanager.TaskExecutionState; +import eu.stratosphere.nephele.taskmanager.TaskKillResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionWrapper; import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse; @@ -889,8 +890,35 @@ public List getEvents(final JobID jobID) throws IOException { */ @Override public void killTask(final JobID jobID, final ManagementVertexID id) throws IOException { - // TODO Auto-generated method stub - LOG.debug("Cancelling job " + jobID); + + final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID); + if (eg == null) { + LOG.error("Cannot find execution graph for job " + jobID); + return; + } + + final ExecutionVertex vertex = eg.getVertexByID(ExecutionVertexID.fromManagementVertexID(id)); + if (vertex == null) { + LOG.error("Cannot find execution vertex with ID " + id); + return; + } + + LOG.info("Killing task " + vertex + " of job " + jobID); + + final Runnable runnable = new Runnable() { + + @Override + public void run() { + + final TaskKillResult result = vertex.killTask(); + if (result.getReturnCode() == AbstractTaskResult.ReturnCode.ERROR) { + LOG.error(result.getDescription()); + } + } + }; + + // Hand it over to the executor service + this.executorService.execute(runnable); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index f67b443a98b83..722f9ce4edd25 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -30,6 +30,7 @@ import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.protocols.VersionedProtocol; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; +import eu.stratosphere.nephele.taskmanager.TaskKillResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionWrapper; @@ -74,16 +75,27 @@ List submitTasks(List tasks) throws IOException; /** - * Advises the task manager to cancel the task with the given ID + * Advises the task manager to cancel the task with the given ID. * * @param id * the ID of the task to cancel - * @return the result of the task cancel + * @return the result of the task cancel attempt * @throws IOException * thrown if an error occurs during this remote procedure call */ TaskCancelResult cancelTask(ExecutionVertexID id) throws IOException; + /** + * Advises the task manager to kill the task with the given ID. + * + * @param id + * the ID of the task to kill + * @return the result of the task kill attempt + * @throws IOException + * thrown if an error occurs during this remote procedure call + */ + TaskKillResult killTask(ExecutionVertexID id) throws IOException; + /** * Queries the task manager about the cache status of the libraries stated in the {@link LibraryCacheProfileRequest} * object. @@ -140,11 +152,13 @@ List submitTasks(List tasks) /** * Restarts a given Task by unregistering an submitting it + * * @param executionVertexID - * @param activeOutputChannels - * @param environment - * @param jobConfiguration - * @throws IOException + * @param activeOutputChannels + * @param environment + * @param jobConfiguration + * @throws IOException */ - void restartTask(ExecutionVertexID executionVertexID, Configuration jobConfiguration, Environment environment, Set activeOutputChannels) throws IOException; + void restartTask(ExecutionVertexID executionVertexID, Configuration jobConfiguration, Environment environment, + Set activeOutputChannels) throws IOException; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index a3674f8e2495a..3e62a86c05ea0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -17,7 +17,6 @@ import java.lang.management.ManagementFactory; import java.lang.management.ThreadMXBean; -import java.util.ConcurrentModificationException; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -56,7 +55,7 @@ public class Task implements ExecutionObserver { private static final Log LOG = LogFactory.getLog(Task.class); private static final long NANO_TO_MILLISECONDS = 1000 * 1000; - + private final ExecutionVertexID vertexID; private final Environment environment; @@ -184,16 +183,36 @@ public void markAsFailed() { */ public void cancelExecution() { + cancelOrKillExecution(true); + } + + /** + * Kills the task (i.e. interrupts the execution thread). + */ + public void killExecution() { + + cancelOrKillExecution(false); + } + + /** + * Cancels or kills the task. + * + * @param cancel + * true/code> if the task shall be cancelled, false if it shall be killed + */ + private void cancelOrKillExecution(final boolean cancel) { + final Thread executingThread = this.environment.getExecutingThread(); if (executingThread == null) { return; } - this.isCanceled = true; - - // Change state - executionStateChanged(ExecutionState.CANCELING, null); + if (cancel) { + this.isCanceled = true; + // Change state + executionStateChanged(ExecutionState.CANCELING, null); + } // Request user code to shut down try { @@ -210,8 +229,14 @@ public void cancelExecution() { executingThread.interrupt(); - if (this.executionState == ExecutionState.CANCELED) { - break; + if (cancel) { + if (this.executionState == ExecutionState.CANCELED) { + break; + } + } else { + if (this.executionState == ExecutionState.FAILED) { + break; + } } try { @@ -246,18 +271,19 @@ public boolean isCanceled() { */ public void initialExecutionResourcesExhausted() { -// if (this.environment.getExecutingThread() != Thread.currentThread()) { -// throw new ConcurrentModificationException( -// "initialExecutionResourcesExhausted must be called from the task that executes the user code"); -// } + // if (this.environment.getExecutingThread() != Thread.currentThread()) { + // throw new ConcurrentModificationException( + // "initialExecutionResourcesExhausted must be called from the task that executes the user code"); + // } // Construct a resource utilization snapshot final long timestamp = System.currentTimeMillis(); - //Get CPU-Usertime in percent + // Get CPU-Usertime in percent ThreadMXBean threadBean = ManagementFactory.getThreadMXBean(); - long userCPU = (threadBean.getCurrentThreadUserTime()/NANO_TO_MILLISECONDS) * 100 / (timestamp - this.startTime); - - //collect outputChannelUtilization + long userCPU = (threadBean.getCurrentThreadUserTime() / NANO_TO_MILLISECONDS) * 100 + / (timestamp - this.startTime); + + // collect outputChannelUtilization final Map channelUtilization = new HashMap(); long totalOutputAmount = 0; for (int i = 0; i < this.environment.getNumberOfOutputGates(); ++i) { @@ -281,18 +307,20 @@ public void initialExecutionResourcesExhausted() { } } Boolean force = null; - - if(this.environment.getInvokable().getClass().isAnnotationPresent(Statefull.class) && !this.environment.getInvokable().getClass().isAnnotationPresent(Stateless.class) ){ - //Don't checkpoint statefull tasks + + if (this.environment.getInvokable().getClass().isAnnotationPresent(Statefull.class) + && !this.environment.getInvokable().getClass().isAnnotationPresent(Stateless.class)) { + // Don't checkpoint statefull tasks force = false; - }else{ - //look for a forced decision from the user + } else { + // look for a forced decision from the user ForceCheckpoint forced = this.environment.getInvokable().getClass().getAnnotation(ForceCheckpoint.class); - if(forced != null){ + if (forced != null) { force = forced.checkpoint(); } } - final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, channelUtilization, userCPU, force, totalInputAmount, totalOutputAmount); + final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, channelUtilization, userCPU, + force, totalInputAmount, totalOutputAmount); // Notify the listener objects final Iterator it = this.registeredListeners.iterator(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskCancelResult.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskCancelResult.java index c5e393cef68f7..29ac8c8d216e3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskCancelResult.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskCancelResult.java @@ -19,7 +19,7 @@ /** * A TaskCancelResult is used to report the results - * of a task cancel. It contains the ID of the task to be canceled, a return code and + * of a task cancel attempt. It contains the ID of the task to be canceled, a return code and * a description. In case of an error during the cancel operation the description includes an error message. * * @author warneke @@ -34,7 +34,7 @@ public class TaskCancelResult extends AbstractTaskResult { * @param returnCode * the return code of the cancel */ - public TaskCancelResult(ExecutionVertexID vertexID, ReturnCode returnCode) { + public TaskCancelResult(final ExecutionVertexID vertexID, final ReturnCode returnCode) { super(vertexID, returnCode); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java new file mode 100644 index 0000000000000..35f8ea60b52a4 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java @@ -0,0 +1,47 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; + +/** + * A TaskKillResult is used to report the results + * of a task kill attempt. It contains the ID of the task to be killed, a return code and + * a description. In case of an error during the kill operation the description includes an error message. + * + * @author warneke + */ +public class TaskKillResult extends AbstractTaskResult { + + /** + * Constructs a new task kill result. + * + * @param vertexID + * the task ID this result belongs to + * @param returnCode + * the return code of the kill + */ + public TaskKillResult(final ExecutionVertexID vertexID, final ReturnCode returnCode) { + super(vertexID, returnCode); + } + + /** + * Constructs an empty task kill result. + */ + public TaskKillResult() { + super(); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 2aaac25f1a820..5a3cd643997f6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -417,6 +417,42 @@ public void run() { return new TaskCancelResult(id, AbstractTaskResult.ReturnCode.SUCCESS); } + /** + * {@inheritDoc} + */ + @Override + public TaskKillResult killTask(final ExecutionVertexID id) throws IOException { + + // Check if the task is registered with our task manager + Task tmpTask; + + synchronized (this.runningTasks) { + + tmpTask = this.runningTasks.get(id); + + if (tmpTask == null) { + final TaskKillResult taskKillResult = new TaskKillResult(id, AbstractTaskResult.ReturnCode.ERROR); + taskKillResult.setDescription("No task with ID + " + id + " is currently running"); + return taskKillResult; + } + } + + final Task task = tmpTask; + // Execute call in a new thread so IPC thread can return immediately + final Thread tmpThread = new Thread(new Runnable() { + + @Override + public void run() { + + // Finally, request user code to cancel + task.killExecution(); + } + }); + tmpThread.start(); + + return new TaskKillResult(id, AbstractTaskResult.ReturnCode.SUCCESS); + } + /** * {@inheritDoc} */ @@ -574,7 +610,8 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf * {@inheritDoc} */ @Override - public SerializableArrayList replayCheckpoints(final List vertexIDs) throws IOException { + public SerializableArrayList replayCheckpoints(final List vertexIDs) + throws IOException { final SerializableArrayList checkpointResultList = new SerializableArrayList(); @@ -694,7 +731,8 @@ void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final } } - if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED ||newExecutionState == ExecutionState.FAILED) { + if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED + || newExecutionState == ExecutionState.FAILED) { // In any of these states the task's thread will be terminated, so we remove the task from the running tasks // map @@ -881,16 +919,17 @@ public void run() { /** * {@inheritDoc} - * @throws IOException + * + * @throws IOException */ @Override public void restartTask(ExecutionVertexID id, Configuration jobConfiguration, Environment environment, Set activeOutputChannels) throws IOException { Task torestart = this.runningTasks.get(id); - //torestart.markAsRestarting(); + // torestart.markAsRestarting(); torestart.cancelExecution(); - //unregisterTask(id,torestart); + // unregisterTask(id,torestart); submitTask(id, jobConfiguration, environment, activeOutputChannels); - + } } From 6b61a24ba01c12c8642e023d77b12875f7d5749e Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 14 Jan 2012 21:52:44 +0100 Subject: [PATCH 146/310] Reintroduced set for recently removed channel ID in order to make cancelling of tasks more robust --- .../nephele/taskmanager/TaskManager.java | 3 + .../ByteBufferedChannelManager.java | 19 ++- .../RecentlyRemovedChannelIDSet.java | 121 ++++++++++++++++++ 3 files changed, 139 insertions(+), 4 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RecentlyRemovedChannelIDSet.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 5a3cd643997f6..34fe1a1bc0257 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -375,6 +375,9 @@ public void runIOLoop() { // Check the status of the task threads to detect unexpected thread terminations checkTaskExecution(); + + // Clean up set of recently unregistered channels + this.byteBufferedChannelManager.cleanUpRecentlyRemovedChannelIDSet(); } // Shutdown the individual components of the task manager diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 069da6aca65e3..dd6da14de5a8d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -76,6 +76,8 @@ public final class ByteBufferedChannelManager implements TransferEnvelopeDispatc private final NetworkConnectionManager networkConnectionManager; + private final RecentlyRemovedChannelIDSet recentlyRemovedChannelIDSet = new RecentlyRemovedChannelIDSet(); + private final ChannelLookupProtocol channelLookupService; private final InstanceConnectionInfo localConnectionInfo; @@ -226,6 +228,9 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { final Environment environment = task.getEnvironment(); + // Mark all channel IDs to be recently removed + this.recentlyRemovedChannelIDSet.add(environment); + for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { final OutputGate outputGate = environment.getOutputGate(i); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { @@ -359,10 +364,8 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, for (final ChannelID localReceiver : localReceivers) { final ChannelContext cc = this.registeredChannels.get(localReceiver); - if (cc == null) { - //TODO: Make this more robust, for example by introducing a separate map for failed channel IDs. - LOG.warn("Cannot find channel context for local receiver " + localReceiver); - continue; + if (cc == null && !this.recentlyRemovedChannelIDSet.contains(localReceiver)) { + throw new IOException("Cannot find channel context for local receiver " + localReceiver); } if (!cc.isInputChannel()) { @@ -676,4 +679,12 @@ public boolean registerSpillingQueueWithNetworkConnection(final JobID jobID, fin return true; } + + /** + * Triggers the clean-up method of the canceled channel ID set. + */ + public void cleanUpRecentlyRemovedChannelIDSet() { + + this.recentlyRemovedChannelIDSet.cleanup(); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RecentlyRemovedChannelIDSet.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RecentlyRemovedChannelIDSet.java new file mode 100644 index 0000000000000..d6358c6ff76c7 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RecentlyRemovedChannelIDSet.java @@ -0,0 +1,121 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.bytebuffered; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.types.Record; + +/** + * This channel set stores the IDs of all channels that have been recently removed. The set can be cleaned up by + * periodically calling the method cleanup. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class RecentlyRemovedChannelIDSet { + + /** + * The period of time the entries must at least remain in the map. + */ + private final static long CLEANUP_INTERVAL = 30000; // 30 sec. + + /** + * The map which stores the ID's of the channels whose tasks have been canceled. + */ + private final Map recentlyRemovedChannels = new HashMap(); + + /** + * Checks whether the given channel ID is stored within this set. + * + * @param channelID + * the channel ID to check for + * @return true if the given channel ID was found in the set, false otherwise + */ + public boolean contains(final ChannelID channelID) { + + synchronized (this.recentlyRemovedChannels) { + + return this.recentlyRemovedChannels.containsKey(channelID); + } + } + + /** + * Removes all entries from the set which have been added longer than CLEANUP_INTERVAL milliseconds + * ago. + */ + public void cleanup() { + + final long now = System.currentTimeMillis(); + + synchronized (this.recentlyRemovedChannels) { + + final Iterator> it = this.recentlyRemovedChannels.entrySet().iterator(); + while (it.hasNext()) { + + final Map.Entry entry = it.next(); + if ((entry.getValue().longValue() + CLEANUP_INTERVAL) < now) { + it.remove(); + } + } + } + } + + /** + * Adds the IDs of all the channels that are attached to the given environment to this set. + * + * @param environment + * the environment whose IDs shall be added to this set + */ + public void add(final Environment environment) { + + final Long now = Long.valueOf(System.currentTimeMillis()); + + synchronized (this.recentlyRemovedChannels) { + + final int numberOfOutputGates = environment.getNumberOfOutputGates(); + + for (int i = 0; i < numberOfOutputGates; ++i) { + + final OutputGate outputGate = environment.getOutputGate(i); + final int numberOfOutputChannels = outputGate.getNumberOfOutputChannels(); + for (int j = 0; j < numberOfOutputChannels; ++j) { + + this.recentlyRemovedChannels.put(outputGate.getOutputChannel(j).getID(), now); + } + } + + final int numberOfInputGates = environment.getNumberOfInputGates(); + + for (int i = 0; i < numberOfInputGates; ++i) { + + final InputGate inputGate = environment.getInputGate(i); + final int numberOfInputChannels = inputGate.getNumberOfInputChannels(); + for (int j = 0; j < numberOfInputChannels; ++j) { + + this.recentlyRemovedChannels.put(inputGate.getInputChannel(j).getID(), now); + } + } + } + } +} From 10af8394244bdd3e8d0fc75cd76a8d26b84f2cd3 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 15 Jan 2012 11:45:45 +0100 Subject: [PATCH 147/310] Minor fix to improve robustness of byte-buffered channel manager --- .../bytebuffered/ByteBufferedChannelManager.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index dd6da14de5a8d..b0c4e015ff01a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -364,8 +364,13 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, for (final ChannelID localReceiver : localReceivers) { final ChannelContext cc = this.registeredChannels.get(localReceiver); - if (cc == null && !this.recentlyRemovedChannelIDSet.contains(localReceiver)) { - throw new IOException("Cannot find channel context for local receiver " + localReceiver); + if (cc == null) { + + if (this.recentlyRemovedChannelIDSet.contains(localReceiver)) { + continue; + } else { + throw new IOException("Cannot find channel context for local receiver " + localReceiver); + } } if (!cc.isInputChannel()) { From 447149a272a76b70a828991d3d205c4b361d4d4e Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 15 Jan 2012 15:27:47 +0100 Subject: [PATCH 148/310] Fixed problem with job lock-up as a result of user cancel request --- .../executiongraph/ExecutionGraph.java | 141 ++++++++------- .../nephele/jobmanager/JobManager.java | 8 +- .../nephele/jobmanager/JobManagerITCase.java | 166 +++++++++--------- 3 files changed, 159 insertions(+), 156 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index 76baf49f2681b..74f74da29cad9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -117,7 +118,8 @@ public class ExecutionGraph implements ExecutionListener { /** * The current status of the job which is represented by this execution graph. */ - private volatile InternalJobStatus jobStatus = InternalJobStatus.CREATED; + private final AtomicReference jobStatus = new AtomicReference( + InternalJobStatus.CREATED); /** * The error description of the first task which causes this job to fail. @@ -1204,88 +1206,79 @@ private boolean jobHasFailedOrCanceledStatus() { return true; } - /** - * Checks and updates the current execution status of the - * job which is represented by this execution graph. - * - * @param latestStateChange - * the latest execution state change which occurred - */ - public void checkAndUpdateJobStatus(final ExecutionState latestStateChange) { + // TODO: Make this static + private InternalJobStatus determineNewJobStatus(final ExecutionGraph eg, + final ExecutionState latestStateChange) { - switch (this.jobStatus) { + final InternalJobStatus currentJobStatus = eg.getJobStatus(); + + switch (currentJobStatus) { case CREATED: - if (jobHasScheduledStatus()) { - this.jobStatus = InternalJobStatus.SCHEDULED; + if (eg.jobHasScheduledStatus()) { + return InternalJobStatus.SCHEDULED; } else if (latestStateChange == ExecutionState.CANCELED) { - if (jobHasFailedOrCanceledStatus()) { - this.jobStatus = InternalJobStatus.CANCELED; + if (eg.jobHasFailedOrCanceledStatus()) { + return InternalJobStatus.CANCELED; } } break; case SCHEDULED: if (latestStateChange == ExecutionState.RUNNING) { - this.jobStatus = InternalJobStatus.RUNNING; - return; + return InternalJobStatus.RUNNING; } else if (latestStateChange == ExecutionState.CANCELED) { - if (jobHasFailedOrCanceledStatus()) { - this.jobStatus = InternalJobStatus.CANCELED; + if (eg.jobHasFailedOrCanceledStatus()) { + return InternalJobStatus.CANCELED; } } break; case RUNNING: if (latestStateChange == ExecutionState.CANCELING || latestStateChange == ExecutionState.CANCELED) { - this.jobStatus = InternalJobStatus.CANCELING; - return; + return InternalJobStatus.CANCELING; } if (latestStateChange == ExecutionState.FAILED) { - final Iterator it = new ExecutionGraphIterator(this, true); + final Iterator it = new ExecutionGraphIterator(eg, true); while (it.hasNext()) { final ExecutionVertex vertex = it.next(); - if (vertex.getExecutionState() == ExecutionState.FAILED ) { - if (!vertex.hasRetriesLeft()){ - System.out.println(" Vertex failed finally" ); - this.jobStatus = InternalJobStatus.FAILING; - return; - }else{ - this.jobStatus = InternalJobStatus.RECOVERING; - return; + if (vertex.getExecutionState() == ExecutionState.FAILED) { + if (!vertex.hasRetriesLeft()) { + System.out.println(" Vertex failed finally"); + return InternalJobStatus.FAILING; + } else { + return InternalJobStatus.RECOVERING; } } } } if (latestStateChange == ExecutionState.RECOVERING) { - this.jobStatus = InternalJobStatus.RECOVERING; - return; + return InternalJobStatus.RECOVERING; } - if (jobHasFinishedStatus()) { - this.jobStatus = InternalJobStatus.FINISHED; + if (eg.jobHasFinishedStatus()) { + return InternalJobStatus.FINISHED; } break; case RECOVERING: if (latestStateChange == ExecutionState.RERUNNING) { - if(this.recovering.isEmpty()){ - this.jobStatus = InternalJobStatus.RUNNING; - break; + if (this.recovering.isEmpty()) { + return InternalJobStatus.RUNNING; } } - if (latestStateChange == ExecutionState.FAILED){ + if (latestStateChange == ExecutionState.FAILED) { LOG.info("Another Failed Vertex while recovering"); } break; case FAILING: - if (jobHasFailedOrCanceledStatus()) { - this.jobStatus = InternalJobStatus.FAILED; + if (eg.jobHasFailedOrCanceledStatus()) { + return InternalJobStatus.FAILED; } break; case FAILED: LOG.error("Received update of execute state in job status FAILED"); break; case CANCELING: - if (jobHasFailedOrCanceledStatus()) { - this.jobStatus = InternalJobStatus.CANCELED; + if (eg.jobHasFailedOrCanceledStatus()) { + return InternalJobStatus.CANCELED; } break; case CANCELED: @@ -1295,6 +1288,8 @@ public void checkAndUpdateJobStatus(final ExecutionState latestStateChange) { LOG.error("Received update of execute state in job status FINISHED"); break; } + + return currentJobStatus; } /** @@ -1304,7 +1299,8 @@ public void checkAndUpdateJobStatus(final ExecutionState latestStateChange) { * @return the current status of the job */ public InternalJobStatus getJobStatus() { - return this.jobStatus; + + return this.jobStatus.get(); } /** @@ -1314,13 +1310,11 @@ public InternalJobStatus getJobStatus() { public void executionStateChanged(final JobID jobID, final ExecutionVertexID vertexID, final ExecutionState newExecutionState, String optionalMessage) { - final InternalJobStatus oldStatus = this.jobStatus; - if (newExecutionState == ExecutionState.RERUNNING) { this.recovering.remove(getVertexByID(vertexID)); } - checkAndUpdateJobStatus(newExecutionState); + final InternalJobStatus newJobStatus = determineNewJobStatus(this, newExecutionState); if (newExecutionState == ExecutionState.FINISHED) { // It is worth checking if the current stage has complete @@ -1337,30 +1331,45 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver } } } - if (newExecutionState == ExecutionState.FAILED && this.jobStatus == InternalJobStatus.RECOVERING){ - LOG.info("RECOVERING"); - //FIXME (marrus) see if we even need that - if(!this.recovering.contains(vertexID)){ - this.recovering.add(this.getVertexByID(vertexID)); - } - } - - if (this.jobStatus != oldStatus) { - - // The task caused the entire job to fail, save the error description - if (this.jobStatus == InternalJobStatus.FAILING) { - this.errorDescription = optionalMessage; + if (newExecutionState == ExecutionState.FAILED && newJobStatus == InternalJobStatus.RECOVERING) { + LOG.info("RECOVERING"); + // FIXME (marrus) see if we even need that + if (!this.recovering.contains(vertexID)) { + this.recovering.add(this.getVertexByID(vertexID)); } + } - // If this is the final failure state change, reuse the saved error description - if (this.jobStatus == InternalJobStatus.FAILED) { - optionalMessage = this.errorDescription; - } + updateJobStatus(newJobStatus, optionalMessage); + } - final Iterator it = this.jobStatusListeners.iterator(); - while (it.hasNext()) { - it.next().jobStatusHasChanged(this, this.jobStatus, optionalMessage); - } + /** + * Updates the job status to given status and triggers the execution of the {@link JobStatusListener} objects. + * + * @param newJobStatus + * the new job status + * @param optionalMessage + * an optional message providing details on the reasons for the state change + */ + public void updateJobStatus(final InternalJobStatus newJobStatus, String optionalMessage) { + + // Check if the new job status equals the old one + if (this.jobStatus.getAndSet(newJobStatus) == newJobStatus) { + return; + } + + // The task caused the entire job to fail, save the error description + if (newJobStatus == InternalJobStatus.FAILING) { + this.errorDescription = optionalMessage; + } + + // If this is the final failure state change, reuse the saved error description + if (newJobStatus == InternalJobStatus.FAILED) { + optionalMessage = this.errorDescription; + } + + final Iterator it = this.jobStatusListeners.iterator(); + while (it.hasNext()) { + it.next().jobStatusHasChanged(this, newJobStatus, optionalMessage); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index d373f91e6cc4e..8e07240991c64 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -663,10 +663,7 @@ public JobCancelResult cancelJob(final JobID jobID) throws IOException { @Override public void run() { - final TaskCancelResult errorResult = cancelJob(eg); - if (errorResult != null) { - LOG.error("Cannot cancel job " + jobID + ": " + errorResult); - } + eg.updateJobStatus(InternalJobStatus.CANCELING, "Job canceled by user"); } }; this.executorService.execute(cancelJobRunnable); @@ -1136,9 +1133,6 @@ public void deploy(final JobID jobID, final AbstractInstance instance, return; } - // Method executionGraph field of vertex is immutable, so no need to synchronized access - final ExecutionGraph eg = verticesToBeDeployed.get(0).getExecutionGraph(); - for (final ExecutionVertex vertex : verticesToBeDeployed) { // Check vertex state diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java index 8ac92e9acde8e..e2820ec1f17eb 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java @@ -320,89 +320,89 @@ public void testExecutionWithDirectoryInput() { } } -// /** -// * Tests the Nephele execution when an exception occurs. In particular, it is tested if the information that is -// * wrapped by the exception is correctly passed on to the client. -// */ -// @Test -// public void testExecutionWithException() { -// -// final String exceptionClassName = ExceptionTask.class.getSimpleName(); -// File inputFile = null; -// File outputFile = null; -// File jarFile = null; -// -// try { -// -// inputFile = ServerTestUtils.createInputFile(0); -// outputFile = new File(ServerTestUtils.getTempDir() + File.separator -// + ServerTestUtils.getRandomFilename()); -// jarFile = ServerTestUtils.createJarFile(exceptionClassName); -// -// // Create job graph -// final JobGraph jg = new JobGraph("Job Graph for Exception Test"); -// -// // input vertex -// final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg); -// i1.setFileInputClass(FileLineReader.class); -// i1.setFilePath(new Path("file://" + inputFile.getAbsolutePath().toString())); -// -// // task vertex 1 -// final JobTaskVertex t1 = new JobTaskVertex("Task with Exception", jg); -// t1.setTaskClass(ExceptionTask.class); -// -// // output vertex -// JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg); -// o1.setFileOutputClass(FileLineWriter.class); -// o1.setFilePath(new Path("file://" + outputFile.getAbsolutePath().toString())); -// -// t1.setVertexToShareInstancesWith(i1); -// o1.setVertexToShareInstancesWith(i1); -// -// // connect vertices -// i1.connectTo(t1, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); -// t1.connectTo(o1, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); -// -// // add jar -// jg.addJar(new Path("file://" + ServerTestUtils.getTempDir() + File.separator + exceptionClassName + ".jar")); -// -// // Create job client and launch job -// final JobClient jobClient = new JobClient(jg, configuration); -// -// try { -// jobClient.submitJobAndWait(); -// } catch (JobExecutionException e) { -// // Check if the correct error message is encapsulated in the exception -// if (e.getMessage() == null) { -// fail("JobExecutionException does not contain an error message"); -// } -// if (!e.getMessage().contains(ExceptionTask.ERROR_MESSAGE)) { -// fail("JobExecutionException does not contain the expected error message"); -// } -// -// return; -// } -// -// fail("Expected exception but did not receive it"); -// -// } catch (JobGraphDefinitionException jgde) { -// fail(jgde.getMessage()); -// } catch (IOException ioe) { -// fail(ioe.getMessage()); -// } finally { -// -// // Remove temporary files -// if (inputFile != null) { -// inputFile.delete(); -// } -// if (outputFile != null) { -// outputFile.delete(); -// } -// if (jarFile != null) { -// jarFile.delete(); -// } -// } -// } + /** + * Tests the Nephele execution when an exception occurs. In particular, it is tested if the information that is + * wrapped by the exception is correctly passed on to the client. + */ + @Test + public void testExecutionWithException() { + + final String exceptionClassName = ExceptionTask.class.getSimpleName(); + File inputFile = null; + File outputFile = null; + File jarFile = null; + + try { + + inputFile = ServerTestUtils.createInputFile(0); + outputFile = new File(ServerTestUtils.getTempDir() + File.separator + + ServerTestUtils.getRandomFilename()); + jarFile = ServerTestUtils.createJarFile(exceptionClassName); + + // Create job graph + final JobGraph jg = new JobGraph("Job Graph for Exception Test"); + + // input vertex + final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg); + i1.setFileInputClass(FileLineReader.class); + i1.setFilePath(new Path("file://" + inputFile.getAbsolutePath().toString())); + + // task vertex 1 + final JobTaskVertex t1 = new JobTaskVertex("Task with Exception", jg); + t1.setTaskClass(ExceptionTask.class); + + // output vertex + JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg); + o1.setFileOutputClass(FileLineWriter.class); + o1.setFilePath(new Path("file://" + outputFile.getAbsolutePath().toString())); + + t1.setVertexToShareInstancesWith(i1); + o1.setVertexToShareInstancesWith(i1); + + // connect vertices + i1.connectTo(t1, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + t1.connectTo(o1, ChannelType.INMEMORY, CompressionLevel.NO_COMPRESSION); + + // add jar + jg.addJar(new Path("file://" + ServerTestUtils.getTempDir() + File.separator + exceptionClassName + ".jar")); + + // Create job client and launch job + final JobClient jobClient = new JobClient(jg, configuration); + + try { + jobClient.submitJobAndWait(); + } catch (JobExecutionException e) { + // Check if the correct error message is encapsulated in the exception + if (e.getMessage() == null) { + fail("JobExecutionException does not contain an error message"); + } + if (!e.getMessage().contains(ExceptionTask.ERROR_MESSAGE)) { + fail("JobExecutionException does not contain the expected error message"); + } + + return; + } + + fail("Expected exception but did not receive it"); + + } catch (JobGraphDefinitionException jgde) { + fail(jgde.getMessage()); + } catch (IOException ioe) { + fail(ioe.getMessage()); + } finally { + + // Remove temporary files + if (inputFile != null) { + inputFile.delete(); + } + if (outputFile != null) { + outputFile.delete(); + } + if (jarFile != null) { + jarFile.delete(); + } + } + } /** * Tests the Nephele execution when a runtime exception during the registration of the input/output gates occurs. From d81ad1f2223ab5b10b165ba22c3da614f27e1499 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 15 Jan 2012 18:27:36 +0100 Subject: [PATCH 149/310] Removed cancel job from callback to ensure proper interaction with recovery mechanisms --- .../nephele/execution/ResourceUtilizationSnapshot.java | 1 - .../java/eu/stratosphere/nephele/jobmanager/JobManager.java | 6 +++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java index 7fd9bbf25f2b3..1038e68c0e4a9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java @@ -22,7 +22,6 @@ import java.util.Iterator; import java.util.Map; -import eu.stratosphere.nephele.annotations.ForceCheckpoint; import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.io.channels.ChannelID; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 8e07240991c64..4abe8c60a5ad6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -664,6 +664,10 @@ public JobCancelResult cancelJob(final JobID jobID) throws IOException { @Override public void run() { eg.updateJobStatus(InternalJobStatus.CANCELING, "Job canceled by user"); + final TaskCancelResult cancelResult = cancelJob(eg); + if(cancelResult.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS) { + LOG.error(cancelResult.getDescription()); + } } }; this.executorService.execute(cancelJobRunnable); @@ -1039,7 +1043,7 @@ public void jobStatusHasChanged(final ExecutionGraph executionGraph, final Inter LOG.info("Status of job " + executionGraph.getJobName() + "(" + executionGraph.getJobID() + ")" + " changed to " + newJobStatus); - if (newJobStatus == InternalJobStatus.CANCELING || newJobStatus == InternalJobStatus.FAILING) { + if (newJobStatus == InternalJobStatus.FAILING) { // Cancel all remaining tasks cancelJob(executionGraph); From 265280f0f17f3235138036c68b7655c06ab18b81 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 15 Jan 2012 22:00:59 +0100 Subject: [PATCH 150/310] Improved robustness of checkpoint replay manager --- .../CheckpointReplayManager.java | 53 ++++++++---- .../checkpointing/CheckpointReplayTask.java | 83 ++++++++++++++----- .../checkpointing/ReplayFinishedNotifier.java | 36 ++++++++ .../bytebuffered/InputChannelContext.java | 8 +- 4 files changed, 138 insertions(+), 42 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayFinishedNotifier.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java index c76dcfcec7e9b..8d3978de22c9d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java @@ -16,6 +16,8 @@ package eu.stratosphere.nephele.checkpointing; import java.io.File; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -24,7 +26,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; -public class CheckpointReplayManager { +public class CheckpointReplayManager implements ReplayFinishedNotifier { private static final Log LOG = LogFactory.getLog(CheckpointReplayManager.class); @@ -41,12 +43,16 @@ public class CheckpointReplayManager { private final String checkpointDirectory; + private final ConcurrentMap runningReplayTasks; + public CheckpointReplayManager(final TransferEnvelopeDispatcher transferEnvelopeDispatcher) { this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.checkpointDirectory = GlobalConfiguration .getString(CHECKPOINT_DIRECTORY_KEY, DEFAULT_CHECKPOINT_DIRECTORY); + + this.runningReplayTasks = new ConcurrentHashMap(); } public boolean hasCompleteCheckpointAvailable(final ExecutionVertexID vertexID) { @@ -77,12 +83,17 @@ public boolean hasPartialCheckpointAvailable(final ExecutionVertexID vertexID) { public void replayCheckpoint(final ExecutionVertexID vertexID) { - final CheckpointReplayTask replayTask = new CheckpointReplayTask(vertexID, this.checkpointDirectory, + final CheckpointReplayTask newReplayTask = new CheckpointReplayTask(this, vertexID, this.checkpointDirectory, this.transferEnvelopeDispatcher, hasCompleteCheckpointAvailable(vertexID)); - replayTask.start(); + final CheckpointReplayTask runningReplayTask = this.runningReplayTasks.put(vertexID, newReplayTask); + if (runningReplayTask != null) { + LOG.info("There is already a replay task running for task " + vertexID + ", cancelling it first..."); + runningReplayTask.cancelAndWait(); + } LOG.info("Replaying checkpoint for vertex " + vertexID); + newReplayTask.start(); } /** @@ -92,21 +103,29 @@ public void replayCheckpoint(final ExecutionVertexID vertexID) { * the vertex whose checkpoint shall be removed */ public void removeCheckpoint(final ExecutionVertexID vertexID) { - File file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + File file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_final"); - if (file.exists()) { - file.delete(); - return; - } - file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_0"); - if (file.exists()) { - file.delete(); - } - - file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_part"); - if (file.exists()) { - file.delete(); - } + if (file.exists()) { + file.delete(); + return; + } + file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_0"); + if (file.exists()) { + file.delete(); + } + + file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_part"); + if (file.exists()) { + file.delete(); + } } + /** + * {@inheritDoc} + */ + @Override + public void replayFinished(final ExecutionVertexID vertexID) { + + this.runningReplayTasks.remove(vertexID); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java index cf1fe02feb682..aae1c84788ad2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java @@ -29,6 +29,8 @@ class CheckpointReplayTask extends Thread { + private final ReplayFinishedNotifier replayFinishedNotifier; + private final ExecutionVertexID vertexID; private final String checkpointDirectory; @@ -37,9 +39,13 @@ class CheckpointReplayTask extends Thread { private final boolean isCheckpointComplete; - CheckpointReplayTask(final ExecutionVertexID vertexID, final String checkpointDirectory, - final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final boolean isCheckpointComplete) { + private volatile boolean isCanceled = false; + + CheckpointReplayTask(final ReplayFinishedNotifier replayFinishedNotifier, final ExecutionVertexID vertexID, + final String checkpointDirectory, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + final boolean isCheckpointComplete) { + this.replayFinishedNotifier = replayFinishedNotifier; this.vertexID = vertexID; this.checkpointDirectory = checkpointDirectory; this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; @@ -57,28 +63,41 @@ public void run() { } catch (IOException ioe) { // TODO: Handle this correctly ioe.printStackTrace(); + } + + // Notify the checkpoint replay manager that the replay has been finished + this.replayFinishedNotifier.replayFinished(this.vertexID); + } + + void cancelAndWait() { + + this.isCanceled = true; + interrupt(); + + try { + join(); } catch (InterruptedException ie) { - // TODO: Handle this correctly ie.printStackTrace(); } } - private void replayCheckpoint() throws IOException, InterruptedException { + private void replayCheckpoint() throws IOException { final CheckpointDeserializer deserializer = new CheckpointDeserializer(this.vertexID); int metaDataIndex = 0; + while (true) { // Try to locate the meta data file final File metaDataFile = new File(this.checkpointDirectory + File.separator - + CheckpointReplayManager.METADATA_PREFIX + "_" + this.vertexID + "_" + metaDataIndex); + + CheckpointReplayManager.METADATA_PREFIX + "_" + this.vertexID + "_" + metaDataIndex); while (!metaDataFile.exists()) { // Try to locate the final meta data file final File finalMetaDataFile = new File(this.checkpointDirectory + File.separator - + CheckpointReplayManager.METADATA_PREFIX + "_" + this.vertexID + "_final"); + + CheckpointReplayManager.METADATA_PREFIX + "_" + this.vertexID + "_final"); if (finalMetaDataFile.exists()) { return; @@ -86,33 +105,51 @@ private void replayCheckpoint() throws IOException, InterruptedException { if (this.isCheckpointComplete) { throw new FileNotFoundException("Cannot find meta data file " + metaDataIndex - + " for checkpoint of vertex " + this.vertexID); + + " for checkpoint of vertex " + this.vertexID); } // Wait for the file to be created - Thread.sleep(100); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + // Ignore exception + } + + if (this.isCanceled) { + return; + } } - final FileInputStream fis = new FileInputStream(metaDataFile); - final FileChannel fileChannel = fis.getChannel(); + FileInputStream fis = null; - while (true) { - try { - deserializer.read(fileChannel); + try { + + fis = new FileInputStream(metaDataFile); + final FileChannel fileChannel = fis.getChannel(); + + while (!this.isCanceled) { + try { + deserializer.read(fileChannel); - final TransferEnvelope transferEnvelope = deserializer.getFullyDeserializedTransferEnvelope(); - if (transferEnvelope != null) { - this.transferEnvelopeDispatcher.processEnvelopeFromOutputChannel(transferEnvelope); + final TransferEnvelope transferEnvelope = deserializer.getFullyDeserializedTransferEnvelope(); + if (transferEnvelope != null) { + this.transferEnvelopeDispatcher.processEnvelopeFromOutputChannel(transferEnvelope); + } + } catch (EOFException eof) { + // Close the file channel + fileChannel.close(); + // Increase the index of the meta data file + ++metaDataIndex; + break; } - } catch (EOFException eof) { - // Close the file channel - fileChannel.close(); - // Increase the index of the meta data file - ++metaDataIndex; - break; + } + } catch (InterruptedException e) { + // Ignore exception + } finally { + if (fis != null) { + fis.close(); } } } - } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayFinishedNotifier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayFinishedNotifier.java new file mode 100644 index 0000000000000..b7ef56da97034 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayFinishedNotifier.java @@ -0,0 +1,36 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; + +/** + * This notifier will be called by a {@link CheckpointReplayTask} after the replay of a checkpoint has been finished, + * either because all data has been replayed, the same checkpoint shall be replayed by another + * {@link CheckpointReplayTask} object, or an error occurred. + * + * @author warneke + */ +public interface ReplayFinishedNotifier { + + /** + * Indicates the {@link CheckpointReplayTask} for the task represented by the given vertex ID has finished. + * + * @param vertexID + * the ID identifying the {@link CheckpointReplayTask} that has finished + */ + public void replayFinished(ExecutionVertexID vertexID); +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java index 591d90a2bbcb1..9efe5fcd7d89c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java @@ -98,7 +98,7 @@ public BufferPairResponse getReadBufferToConsume() { // TODO: Fix implementation breaks compression, fix it later on final BufferPairResponse response = new BufferPairResponse(null, transferEnvelope.getBuffer()); // No need to - // Moved event processing to releaseConsumedReadBuffer method // copy anything + // Moved event processing to releaseConsumedReadBuffer method // copy anything return response; } @@ -165,7 +165,11 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { synchronized (this.queuedEnvelopes) { - if (sequenceNumber != (this.lastReceivedEnvelope + 1)) { + final int expectedSequenceNumber = this.lastReceivedEnvelope + 1; + if (sequenceNumber != expectedSequenceNumber) { + + // LOG.info("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber + // + " but received " + sequenceNumber); final Buffer buffer = transferEnvelope.getBuffer(); if (buffer != null) { From 43bef07b0b188e6546be6dfb5feeb6f6862d0fbe Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 15 Jan 2012 22:15:22 +0100 Subject: [PATCH 151/310] Added names for recovery threads --- .../nephele/checkpointing/CheckpointReplayTask.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java index aae1c84788ad2..0a219f020564c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java @@ -45,6 +45,8 @@ class CheckpointReplayTask extends Thread { final String checkpointDirectory, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final boolean isCheckpointComplete) { + super("Recovery task for " + vertexID); + this.replayFinishedNotifier = replayFinishedNotifier; this.vertexID = vertexID; this.checkpointDirectory = checkpointDirectory; From 7b9a0c36d21d434eb3e8f1c754c477a571bb603f Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 17 Jan 2012 13:15:06 +0100 Subject: [PATCH 152/310] added forced group vertex checkpoint --- .../CheckpointDecisionCoordinator.java | 66 +++++++++++++++---- .../nephele/instance/AbstractInstance.java | 2 +- .../nephele/jobmanager/RecoveryThread.java | 9 +-- .../ByteBufferedChannelManager.java | 2 +- 4 files changed, 60 insertions(+), 19 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index 3c76273b8f8ae..596599bfba5f6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -15,6 +15,7 @@ package eu.stratosphere.nephele.checkpointing; +import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -26,7 +27,9 @@ import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; +import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.util.SerializableArrayList; @@ -51,6 +54,10 @@ public final class CheckpointDecisionCoordinator { * The object in charge of propagating checkpoint decisions to the respective task managers. */ private final CheckpointDecisionPropagator decisionPropagator; + + + + private List decidedVertices = new ArrayList(); /** * Constructs a new checkpoint decision coordinator. @@ -71,7 +78,6 @@ public CheckpointDecisionCoordinator(final CheckpointDecisionPropagator decision * the job to register */ public void registerJob(final ExecutionGraph executionGraph) { - final Iterator it = new ExecutionGraphIterator(executionGraph, true); while (it.hasNext()) { final ExecutionVertex vertex = it.next(); @@ -89,19 +95,53 @@ public void registerJob(final ExecutionGraph executionGraph) { */ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { LOG.info("Checkpoint decision for vertex " + vertex + " required"); - - boolean checkpointDecision = getDecision(vertex, rus); - final ExecutionGraph graph = vertex.getExecutionGraph(); - final Map> checkpointDecisions = new HashMap>(); - final List checkpointDecisionList = new SerializableArrayList(); - - synchronized (graph) { - checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), checkpointDecision)); - checkpointDecisions.put(vertex.getAllocatedResource().getInstance(), checkpointDecisionList); + synchronized (decidedVertices) { + if (!decidedVertices.contains(vertex.getID())) { + boolean checkpointDecision = getDecision(vertex, rus); + final ExecutionGraph graph = vertex.getExecutionGraph(); + final Map> checkpointDecisions = new HashMap>(); + List checkpointDecisionList = null; + + + synchronized (graph) { + ExecutionGroupVertex groupVertex = vertex.getGroupVertex(); + //force decision to all groupVertex members + for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); i++) { + ExecutionVertex member = groupVertex.getGroupMember(i); + AbstractInstance instance = member.getAllocatedResource().getInstance(); + if(checkpointDecisions.containsKey(instance)){ + //if instance already in list append new decision + checkpointDecisionList = checkpointDecisions.get(instance); + }else{ + //make an new list for each instance + checkpointDecisionList = new SerializableArrayList(); + } + checkpointDecisionList.add(new CheckpointDecision(member.getID(), checkpointDecision)); + checkpointDecisions.put(instance, checkpointDecisionList); + + this.decidedVertices.add(member.getID()); + } + } + + // Propagate checkpoint decisions + this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); + } } - - // Propagate checkpoint decisions - this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); +// LOG.info("Checkpoint decision for vertex " + vertex + " required"); +// +// // TODO: Provide sensible implementation here +// boolean checkpointDecision = getDecision(vertex, rus); +// final ExecutionGraph graph = vertex.getExecutionGraph(); +// final Map> checkpointDecisions = new HashMap>(); +// final List checkpointDecisionList = new SerializableArrayList(); +// +// synchronized (graph) { +// checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), checkpointDecision)); +// checkpointDecisions.put(vertex.getAllocatedResource().getInstance(), checkpointDecisionList); +// } +// +// // Propagate checkpoint decisions +// this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); } private boolean getDecision(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index 2113e7fe02556..df7cb8887e066 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -162,7 +162,7 @@ public synchronized void checkLibraryAvailability(final JobID jobID) throws IOEx LibraryCacheProfileResponse response = null; response = getTaskManager().getLibraryCacheProfile(request); - // Check response and transfer libaries if necesarry + // Check response and transfer libraries if necessary for (int k = 0; k < requiredLibraries.length; k++) { if (!response.isCached(k)) { LibraryCacheUpdate update = new LibraryCacheUpdate(requiredLibraries[k]); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java index 235c75ad7dd98..58a40209ed629 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java @@ -119,7 +119,7 @@ private boolean recover(final ExecutionVertex failed) { return false; } - //get list of instances of consistencheckpoints + //get list of instances of consistent checkpoints List instances = new SerializableArrayList(); for (ExecutionVertexID id : this.globalConsistentCheckpoint) { AbstractInstance instance = this.job.getVertexByID(id).getAllocatedResource().getInstance(); @@ -131,15 +131,16 @@ private boolean recover(final ExecutionVertex failed) { while (instanceIterator.hasNext()) { //replay all necessary checkpoints try { - - replayCheckpoints.addAll(instanceIterator.next().replayCheckpoints(this.globalConsistentCheckpoint)); + AbstractInstance instance = instanceIterator.next(); + + replayCheckpoints.addAll(instance.replayCheckpoints(this.globalConsistentCheckpoint)); } catch (IOException e) { e.printStackTrace(); this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); return false; } } - + //State job to Failed if a checkpoint-replay failed for(CheckpointReplayResult replayResult : replayCheckpoints ){ if (replayResult.getReturnCode() == ReturnCode.ERROR) { LOG.info("Replay of Checkpoints return Error " + replayResult.getDescription() ); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index e9d3d07b23e64..e1051a8611057 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -651,7 +651,7 @@ public void reportCheckpointDecisions(final List checkpointD LOG.error("Cannot report checkpoint decision for vertex " + cd.getVertexID()); continue; } - + LOG.info("reporting checkpoint decision for " + cd.getVertexID()); taskContext.setCheckpointDecisionAsynchronously(cd.getCheckpointDecision()); taskContext.reportAsynchronousEvent(); } From 127d15f6fa68dadd67ca7cb31f957f61a41aba66 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 24 Jan 2012 17:51:06 +0100 Subject: [PATCH 153/310] Corrected typo in checkpointing annotations --- .../nephele/annotations/{Statefull.java => Stateful.java} | 6 +++--- .../java/eu/stratosphere/nephele/annotations/Stateless.java | 4 ++-- .../main/java/eu/stratosphere/nephele/taskmanager/Task.java | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) rename nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/{Statefull.java => Stateful.java} (92%) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Statefull.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateful.java similarity index 92% rename from nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Statefull.java rename to nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateful.java index 40a37189f9f0d..83edba1bd6c7d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Statefull.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateful.java @@ -20,12 +20,12 @@ import java.lang.annotation.Target; /** - * Annotation indicate a state-full Task + * This annotation can be used to indicate a stateful task. + * * @author marrus - * */ @Retention(RetentionPolicy.RUNTIME) @Target(ElementType.TYPE) -public @interface Statefull { +public @interface Stateful { } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java index 3b1800640a474..0f30086decf0b 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java @@ -20,9 +20,9 @@ import java.lang.annotation.Target; /** - * Annotation indicate a state-full Task + * This annotation can be used to indicate a stateless task. + * * @author marrus - * */ @Retention(RetentionPolicy.RUNTIME) @Target(ElementType.TYPE) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index 3e62a86c05ea0..173c16a074b9f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -27,7 +27,7 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.annotations.ForceCheckpoint; -import eu.stratosphere.nephele.annotations.Statefull; +import eu.stratosphere.nephele.annotations.Stateful; import eu.stratosphere.nephele.annotations.Stateless; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionListener; @@ -308,7 +308,7 @@ public void initialExecutionResourcesExhausted() { } Boolean force = null; - if (this.environment.getInvokable().getClass().isAnnotationPresent(Statefull.class) + if (this.environment.getInvokable().getClass().isAnnotationPresent(Stateful.class) && !this.environment.getInvokable().getClass().isAnnotationPresent(Stateless.class)) { // Don't checkpoint statefull tasks force = false; From b643f04329847e9bb61bbeed8b97d77542d40d79 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 24 Jan 2012 18:24:52 +0100 Subject: [PATCH 154/310] Removed superfluous execution state RESTARTING --- .../eu/stratosphere/nephele/execution/ExecutionState.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java index 001a55c21ef70..ff9562242b957 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java @@ -83,12 +83,6 @@ public enum ExecutionState { RECOVERING, - /** - * The task is restarting during recovery. - */ - - RESTARTING, - /** * Recovery is finished the job is running normal again. */ From fce75a11fa0f8be6b306296338bb6c960adede84 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 25 Jan 2012 15:18:03 +0100 Subject: [PATCH 155/310] Introduced priorities for execution listeners --- .../impl/EnvironmentListenerImpl.java | 9 ++++++ .../CheckpointExecutionListener.java | 9 ++++++ .../nephele/execution/ExecutionListener.java | 9 ++++++ .../executiongraph/ExecutionGraph.java | 9 ++++++ .../executiongraph/ExecutionVertex.java | 29 ++++++++++++++----- .../nephele/jobmanager/EventCollector.java | 9 ++++++ .../scheduler/AbstractExecutionListener.java | 9 ++++++ 7 files changed, 75 insertions(+), 8 deletions(-) diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java index d9803c1a7a99a..f592eb0b440e3 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java @@ -101,4 +101,13 @@ public void initialExecutionResourcesExhausted(final JobID jobID, final Executio final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { // Nothing to do here } + + /** + * {@inheritDoc} + */ + @Override + public int getPriority() { + + return 1; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointExecutionListener.java index ca4dca80c7b50..f8f707f159b5e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointExecutionListener.java @@ -102,4 +102,13 @@ public void initialExecutionResourcesExhausted(final JobID jobID, final Executio this.decisionCoordinator.checkpointDecisionRequired(this.executionVertex, resourceUtilizationSnapshot); } + /** + * {@inheritDoc} + */ + @Override + public int getPriority() { + + return 2; + } + } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java index fde9547b3efa7..04f6e0faa710b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java @@ -26,6 +26,15 @@ */ public interface ExecutionListener { + /** + * Returns the priority of the execution listener object. If multiple execution listener objects are registered for + * a given vertex, the priority determines in which order they will be called. Priorities are expressed as + * non-negative integer values. The lower the integer value, the higher the call priority. + * + * @return the priority of this execution listener + */ + int getPriority(); + /** * Called when the execution state of the associated task has changed. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index 74f74da29cad9..eaf10066eb4ea 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -1534,4 +1534,13 @@ public Iterator iterator() { return this.stages.iterator(); } + + /** + * {@inheritDoc} + */ + @Override + public int getPriority() { + + return 1; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 8ed09729ba25f..59c0f7ed07ba9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -17,6 +17,8 @@ import java.io.IOException; import java.util.Iterator; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicReference; @@ -112,10 +114,9 @@ public final class ExecutionVertex { private final CopyOnWriteArrayList checkpointStateListeners = new CopyOnWriteArrayList(); /** - * A list of {@link ExecutionListener} objects to be notified about state changes of the vertex's - * checkpoint. + * A map of {@link ExecutionListener} objects to be notified about the state changes of a vertex. */ - private final CopyOnWriteArrayList executionListeners = new CopyOnWriteArrayList(); + private final ConcurrentMap executionListeners = new ConcurrentSkipListMap(); /** * The current execution state of the task represented by this vertex @@ -310,7 +311,7 @@ public void updateExecutionState(final ExecutionState newExecutionState, final S } // Notify the listener objects - final Iterator it = this.executionListeners.iterator(); + final Iterator it = this.executionListeners.values().iterator(); while (it.hasNext()) { it.next().executionStateChanged(this.executionGraph.getJobID(), this.vertexID, newExecutionState, optionalMessage); @@ -335,7 +336,7 @@ public boolean compareAndUpdateExecutionState(final ExecutionState expected, fin this.retries--; } // Notify the listener objects - final Iterator it = this.executionListeners.iterator(); + final Iterator it = this.executionListeners.values().iterator(); while (it.hasNext()) { it.next().executionStateChanged(this.executionGraph.getJobID(), this.vertexID, update, null); @@ -366,7 +367,7 @@ public void initialExecutionResourcesExhausted( final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { // Notify the listener objects - final Iterator it = this.executionListeners.iterator(); + final Iterator it = this.executionListeners.values().iterator(); while (it.hasNext()) { it.next().initialExecutionResourcesExhausted(this.environment.getJobID(), this.vertexID, resourceUtilizationSnapshot); @@ -613,7 +614,7 @@ public TaskKillResult killTask() { result.setDescription("Assigned instance of vertex " + this.toString() + " is null!"); return result; } - + try { return this.allocatedResource.getInstance().killTask(this.vertexID); } catch (IOException e) { @@ -759,7 +760,19 @@ public void unregisterCheckpointStateListener(final CheckpointStateListener chec */ public void registerExecutionListener(final ExecutionListener executionListener) { - this.executionListeners.addIfAbsent(executionListener); + final Integer priority = Integer.valueOf(executionListener.getPriority()); + + if (priority.intValue() < 0) { + LOG.error("Priority for execution listener " + executionListener.getClass() + " must be non-negative."); + return; + } + + final ExecutionListener previousValue = this.executionListeners.putIfAbsent(priority, executionListener); + + if (previousValue != null) { + LOG.error("Cannot register " + executionListener.getClass() + " as an execution listener. Priority " + + priority.intValue() + " is already taken."); + } } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java index 66309d2c8ca3b..ff9daa525f175 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java @@ -170,6 +170,15 @@ public void initialExecutionResourcesExhausted(final JobID jobID, final Executio // Nothing to do here } + /** + * {@inheritDoc} + */ + @Override + public int getPriority() { + + return 20; + } + } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java index 40ecf24ecbe2e..9f48c62406d18 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java @@ -215,4 +215,13 @@ public void initialExecutionResourcesExhausted(final JobID jobID, final Executio * } */ } + + /** + * {@inheritDoc} + */ + @Override + public int getPriority() { + + return 0; + } } From 6f345525c84428d272e7330df89b08115f64fcab Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 25 Jan 2012 16:06:15 +0100 Subject: [PATCH 156/310] Improved javadoc of execution listener interface --- .../stratosphere/nephele/execution/ExecutionListener.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java index 04f6e0faa710b..fda61432d5eee 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java @@ -36,7 +36,11 @@ public interface ExecutionListener { int getPriority(); /** - * Called when the execution state of the associated task has changed. + * Called when the execution state of the associated task has changed. It is important to point out that multiple + * execution listeners can be invoked as a reaction to a state change, according to their priority. As a result, the + * value of newExecutionState may be out-dated by the time a particular execution listener is called. + * To determine the most recent state of the respective task, it is recommended to store a reference on the + * execution that represents it and then call getExecutionState() on the vertex within this method. * * @param jobID * the ID of the job the task belongs to From 409d1a084577755d73850c98b0265607f6290634 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 25 Jan 2012 18:15:00 +0100 Subject: [PATCH 157/310] Introduced new method to keep track if execution retries in a thread-safe manner --- .../scheduler/queue/QueueScheduler.java | 2 ++ .../executiongraph/ExecutionVertex.java | 29 +++++++++++++------ .../scheduler/AbstractScheduler.java | 20 +++++++++++-- .../scheduler/local/LocalScheduler.java | 4 ++- 4 files changed, 43 insertions(+), 12 deletions(-) diff --git a/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java b/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java index 5be52a8cde6c7..3c2fcef72adf3 100644 --- a/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java +++ b/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java @@ -95,6 +95,8 @@ void removeJobFromSchedule(final ExecutionGraph executionGraphToRemove) { LOG.error("Cannot find job " + executionGraphToRemove.getJobName() + " (" + executionGraphToRemove.getJobID() + ") to remove"); } + + //TODO: Remove vertices from restart map } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 59c0f7ed07ba9..ae34712bf0f3f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -20,6 +20,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.logging.Log; @@ -124,14 +125,14 @@ public final class ExecutionVertex { private final AtomicEnum executionState = new AtomicEnum(ExecutionState.CREATED); /** - * The current checkpoint state of this vertex. + * Stores the number of times the vertex may be still be started before the corresponding task is considered to be + * failed. */ + private final AtomicInteger retriesLeft; /** - * Number of times this vertex may be restarted + * The current checkpoint state of this vertex. */ - private int retries = 3; // TODO make this configurable - private final AtomicEnum checkpointState = new AtomicEnum(CheckpointState.NONE); /** @@ -192,6 +193,8 @@ private ExecutionVertex(final ExecutionVertexID vertexID, final Class it = this.executionListeners.values().iterator(); while (it.hasNext()) { @@ -696,13 +695,25 @@ public String toString() { * * @return true if the task has a retry attempt left, false otherwise */ + @Deprecated public boolean hasRetriesLeft() { - if (this.retries < 0) { + if (this.retriesLeft.get() <= 0) { return false; } return true; } + /** + * Decrements the number of retries left and checks whether another attempt to run the task is possible. + * + * @return trueif the task represented by this vertex can be started at least once more, + * false/ otherwise + */ + public boolean decrementRetriesLeftAndCheck() { + + return (this.retriesLeft.decrementAndGet() > 0); + } + /** * Registers the {@link VertexAssignmentListener} object for this vertex. This object * will be notified about reassignments of this vertex to another instance. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index 4821b8490ecb5..95ab0e12136e2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -20,6 +20,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -91,6 +92,11 @@ public abstract class AbstractScheduler implements InstanceListener { */ private final boolean allowTaskMerging; + /** + * Stores the vertices to be restarted once they have switched to the CANCELED state. + */ + private final Map verticesToBeRestarted = new ConcurrentHashMap(); + /** * Constructs a new abstract scheduler. * @@ -472,12 +478,22 @@ protected void replayCheckpointsFromPreviousStage(final ExecutionGraph execution } } + /** + * Returns a map of vertices to be restarted once they have switched to their CANCELED state. + * + * @return the map of vertices to be restarted + */ + Map getVerticesToBeRestarted() { + + return this.verticesToBeRestarted; + } + /** * {@inheritDoc} */ @Override public void allocatedResourcesDied(final JobID jobID, final List allocatedResource) { - - //TODO: Don't forget to synchronize on stage here + + // TODO: Don't forget to synchronize on stage here } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java index b1151d837c448..95dba03d526f6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java @@ -62,7 +62,7 @@ public LocalScheduler(final DeploymentManager deploymentManager, final InstanceM void removeJobFromSchedule(final ExecutionGraph executionGraphToRemove) { boolean removedFromQueue = false; - + synchronized (this.jobQueue) { final Iterator it = this.jobQueue.iterator(); @@ -83,6 +83,8 @@ void removeJobFromSchedule(final ExecutionGraph executionGraphToRemove) { LOG.error("Cannot find job " + executionGraphToRemove.getJobName() + " (" + executionGraphToRemove.getJobID() + ") to remove"); } + + //TODO: Remove vertices from restart map } /** From f2a508a04acac91b3450327b054cc607e7ea415d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 25 Jan 2012 19:13:31 +0100 Subject: [PATCH 158/310] Moved content of class RecoveryThread to class RecoveryLogic --- .../nephele/instance/AbstractInstance.java | 17 +- .../scheduler/AbstractExecutionListener.java | 12 +- .../scheduler/AbstractScheduler.java | 26 ++- .../jobmanager/scheduler/RecoveryLogic.java | 164 ++++++++++++++++++ 4 files changed, 199 insertions(+), 20 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index f2046b4a8c279..bbb553ff1293a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -209,7 +209,7 @@ public synchronized List submitTasks(final List replayCheckpoints(List vertexIDs) + public synchronized List replayCheckpoints(final List vertexIDs) throws IOException { return getTaskManager().replayCheckpoints(vertexIDs); @@ -235,7 +235,7 @@ public synchronized TaskCancelResult cancelTask(final ExecutionVertexID id) thro return getTaskManager().cancelTask(id); } - + /** * Kills the task identified by the given ID at the instance's * {@link eu.stratosphere.nephele.taskmanager.TaskManager}. @@ -323,14 +323,13 @@ public synchronized void killTaskManager() throws IOException { } /** - * @param executionVertexID - * @param serializableHashSet - * @param environment - * @param configuration - * @throws IOException - * + * @param executionVertexID + * @param serializableHashSet + * @param environment + * @param configuration + * @throws IOException */ - public void restartTask(final ExecutionVertexID id, final Configuration jobConfiguration, + public void restartTask(final ExecutionVertexID id, final Configuration jobConfiguration, final Environment environment, final Set activeOutputChannels) throws IOException { getTaskManager().restartTask(id, jobConfiguration, environment, activeOutputChannels); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java index 9f48c62406d18..24eee565138e9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java @@ -91,11 +91,11 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver this.scheduler.checkAndReleaseAllocatedResource(eg, this.executionVertex.getAllocatedResource()); } - // In case of an error, check if vertex can be rescheduled + // In case of an error, check if the vertex shall be recovered if (newExecutionState == ExecutionState.FAILED) { - if (this.executionVertex.hasRetriesLeft()) { - // Reschedule vertex - this.executionVertex.updateExecutionState(ExecutionState.SCHEDULED); + if (this.executionVertex.decrementRetriesLeftAndCheck()) { + + RecoveryLogic.recover(this.executionVertex, this.scheduler.getVerticesToBeRestarted()); } } @@ -215,13 +215,13 @@ public void initialExecutionResourcesExhausted(final JobID jobID, final Executio * } */ } - + /** * {@inheritDoc} */ @Override public int getPriority() { - + return 0; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index 95ab0e12136e2..1e8c70523ad60 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -38,6 +38,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.instance.AllocatedResource; +import eu.stratosphere.nephele.instance.AllocationID; import eu.stratosphere.nephele.instance.DummyInstance; import eu.stratosphere.nephele.instance.InstanceException; import eu.stratosphere.nephele.instance.InstanceListener; @@ -402,7 +403,8 @@ public void resourcesAllocated(final JobID jobID, final List * @param allocatedResource * the allocated resource to check the assignment for */ - public void checkAndReleaseAllocatedResource(ExecutionGraph executionGraph, AllocatedResource allocatedResource) { + public void checkAndReleaseAllocatedResource(final ExecutionGraph executionGraph, + final AllocatedResource allocatedResource) { if (allocatedResource == null) { LOG.error("Resource to lock is null!"); @@ -420,8 +422,8 @@ public void checkAndReleaseAllocatedResource(ExecutionGraph executionGraph, Allo return; } - boolean instanceCanBeReleased = true; - final Iterator it = assignedVertices.iterator(); + boolean resourceCanBeReleased = true; + Iterator it = assignedVertices.iterator(); while (it.hasNext()) { final ExecutionVertex vertex = it.next(); final ExecutionState state = vertex.getExecutionState(); @@ -429,12 +431,26 @@ public void checkAndReleaseAllocatedResource(ExecutionGraph executionGraph, Allo if (state != ExecutionState.CREATED && state != ExecutionState.FINISHED && state != ExecutionState.FAILED && state != ExecutionState.CANCELED) { - instanceCanBeReleased = false; + resourceCanBeReleased = false; break; } } - if (instanceCanBeReleased) { + if (resourceCanBeReleased) { + + final DummyInstance dummyInstance = DummyInstance.createDummyInstance(allocatedResource.getInstance() + .getType()); + final AllocatedResource dummyResource = new AllocatedResource(dummyInstance, + allocatedResource.getInstanceType(), new AllocationID()); + + // Assign vertices back to a dummy resource in case we need the resource information once more for another + // execution. + it = assignedVertices.iterator(); + while (it.hasNext()) { + final ExecutionVertex vertex = it.next(); + vertex.setAllocatedResource(dummyResource); + } + LOG.info("Releasing instance " + allocatedResource.getInstance()); try { getInstanceManager().releaseAllocatedResource(executionGraph.getJobID(), executionGraph diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java new file mode 100644 index 0000000000000..35b70e1b15bad --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -0,0 +1,164 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.jobmanager.scheduler; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; +import eu.stratosphere.nephele.execution.ExecutionState; +import eu.stratosphere.nephele.executiongraph.CheckpointState; +import eu.stratosphere.nephele.executiongraph.ExecutionVertex; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.instance.AbstractInstance; +import eu.stratosphere.nephele.taskmanager.TaskCancelResult; +import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; +import eu.stratosphere.nephele.util.SerializableArrayList; +import eu.stratosphere.nephele.util.StringUtils; + +/** + * @author marrus + */ +public final class RecoveryLogic { + + /** + * The logger to report information and problems. + */ + private static final Log LOG = LogFactory.getLog(RecoveryLogic.class); + + /** + * Private constructor so class cannot be instantiated. + */ + private RecoveryLogic() { + } + + public static boolean recover(final ExecutionVertex failedVertex, + final Map verticesToBeRestarted) { + + // Perform initial sanity check + if (failedVertex.getExecutionState() != ExecutionState.FAILED) { + LOG.error("Vertex " + failedVertex + " is requested to be recovered, but is not failed"); + return false; + } + + LOG.info("Starting recovery for failed vertex " + failedVertex); + + final Set verticesToBeCanceled = new HashSet(); + + final Map> checkpointsToBeReplayed = new HashMap>(); + + findVerticesToRestart(failedVertex, verticesToBeCanceled, checkpointsToBeReplayed); + + // Restart all predecessors without checkpoint + final Iterator cancelIterator = verticesToBeCanceled.iterator(); + while (cancelIterator.hasNext()) { + + final ExecutionVertex vertex = cancelIterator.next(); + LOG.info(vertex + " is canceled by recovery logic"); + final TaskCancelResult cancelResult = vertex.cancelTask(); + verticesToBeRestarted.put(vertex.getID(), vertex); + if (cancelResult.getReturnCode() != ReturnCode.SUCCESS) { + verticesToBeRestarted.remove(vertex.getID()); + LOG.error(cancelResult.getDescription()); + return false; + } + + } + + // Replay all necessary checkpoints + final Iterator>> checkpointIterator = checkpointsToBeReplayed + .entrySet().iterator(); + + while (checkpointIterator.hasNext()) { + + final Map.Entry> entry = checkpointIterator.next(); + final AbstractInstance instance = entry.getKey(); + + try { + final List results = instance.replayCheckpoints(entry.getValue()); + for (final CheckpointReplayResult result : results) { + if (result.getReturnCode() != ReturnCode.SUCCESS) { + LOG.error(result.getDescription()); + return false; + } + } + + } catch (IOException ioe) { + LOG.error(StringUtils.stringifyException(ioe)); + return false; + } + } + + // Restart failed vertex + failedVertex.updateExecutionState(ExecutionState.SCHEDULED); + + return true; + } + + private static void findVerticesToRestart(final ExecutionVertex failedVertex, + final Set verticesToBeCanceled, + final Map> checkpointsToBeReplayed) { + + final Queue verticesToTest = new ArrayDeque(); + final Set visited = new HashSet(); + verticesToTest.add(failedVertex); + + while (!verticesToTest.isEmpty()) { + + final ExecutionVertex vertex = verticesToTest.poll(); + + if (!vertex.getID().equals(failedVertex.getID())) { + verticesToBeCanceled.add(vertex); + } + + // Predecessors must be either checkpoints or need to be restarted, too + for (int j = 0; j < vertex.getNumberOfPredecessors(); j++) { + final ExecutionVertex predecessor = vertex.getPredecessor(j); + if (predecessor.getCheckpointState() != CheckpointState.PARTIAL + && predecessor.getCheckpointState() != CheckpointState.COMPLETE) { + + verticesToBeCanceled.add(predecessor); + if (!visited.contains(predecessor)) { + verticesToTest.add(predecessor); + } + } else { + + // Group IDs by instance + final AbstractInstance instance = predecessor.getAllocatedResource().getInstance(); + List checkpointIDs = checkpointsToBeReplayed.get(instance); + if (checkpointIDs == null) { + checkpointIDs = new SerializableArrayList(); + checkpointsToBeReplayed.put(instance, checkpointIDs); + } + + if (!checkpointIDs.contains(predecessor.getID())) { + checkpointIDs.add(predecessor.getID()); + } + } + } + visited.add(vertex); + } + } +} From c873fbc65d5c4e42bc4db6b95abb04c2a4cd8c4f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 25 Jan 2012 22:03:06 +0100 Subject: [PATCH 159/310] Moved code to handle dead hosts from queue scheduler to abstract base class --- .../scheduler/queue/QueueScheduler.java | 66 +------------------ .../scheduler/AbstractScheduler.java | 42 +++++++++++- 2 files changed, 43 insertions(+), 65 deletions(-) diff --git a/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java b/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java index 3c2fcef72adf3..db0fff7dfa02b 100644 --- a/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java +++ b/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java @@ -18,7 +18,6 @@ import java.util.ArrayDeque; import java.util.Deque; import java.util.Iterator; -import java.util.List; import java.util.Map; import org.apache.hadoop.util.StringUtils; @@ -31,8 +30,6 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.InternalJobStatus; import eu.stratosphere.nephele.executiongraph.JobStatusListener; -import eu.stratosphere.nephele.instance.AllocatedResource; -import eu.stratosphere.nephele.instance.DummyInstance; import eu.stratosphere.nephele.instance.InstanceException; import eu.stratosphere.nephele.instance.InstanceManager; import eu.stratosphere.nephele.instance.InstanceRequestMap; @@ -95,8 +92,8 @@ void removeJobFromSchedule(final ExecutionGraph executionGraphToRemove) { LOG.error("Cannot find job " + executionGraphToRemove.getJobName() + " (" + executionGraphToRemove.getJobID() + ") to remove"); } - - //TODO: Remove vertices from restart map + + // TODO: Remove vertices from restart map } /** @@ -190,65 +187,6 @@ public ExecutionGraph getExecutionGraphByID(final JobID jobID) { return null; } - /** - * {@inheritDoc} - */ - @Override - public void allocatedResourcesDied(final JobID jobID, final List allocatedResources) { - - for (final AllocatedResource allocatedResource : allocatedResources) { - - LOG.info("Resource on " + allocatedResource.getInstance().getName() + " for Job " + jobID + " died."); - // TODO (marrus) - - ExecutionGraph job = this.jobQueue.getFirst(); - synchronized (job) { - - Iterator iterator = this.jobQueue.descendingIterator(); - while (job.getJobID() != jobID) { - if (iterator.hasNext()) { - job = iterator.next(); - } else { - LOG.error("No Job with ID " + jobID + " in Queue"); - return; - } - } - List vertices = job.getVerticesAssignedToResource(allocatedResource); - Iterator vertexIter = vertices.iterator(); - while (vertexIter.hasNext()) { - ExecutionVertex vertex = vertexIter.next(); - vertex.updateExecutionState(ExecutionState.FAILED, "The Resource " - + allocatedResource.getInstance().getName() + " the Vertex " - + vertex.getEnvironment().getTaskName() + " was assigned to, died"); - if (vertex.getExecutionState() == ExecutionState.FAILED) { - job.executionStateChanged(jobID, vertex.getID(), ExecutionState.FAILED, "The Resource " - + allocatedResource.getInstance().getName() + " the Vertex " - + vertex.getEnvironment().getTaskName() + " was assigned to, died"); - return; - } - - vertex.setAllocatedResource(new AllocatedResource(DummyInstance - .createDummyInstance(allocatedResource - .getInstanceType()), allocatedResource.getInstanceType(), - null)); - } - - try { - LOG.info("Trying to allocate instance of type " - + allocatedResource.getInstanceType().getIdentifier()); - final InstanceRequestMap instanceMap = new InstanceRequestMap(); - instanceMap.setMaximumNumberOfInstances(allocatedResource.getInstanceType(), 1); - instanceMap.setMinimumNumberOfInstances(allocatedResource.getInstanceType(), 1); - this.getInstanceManager().requestInstance(jobID, job.getJobConfiguration(), instanceMap, null); - - } catch (InstanceException e) { - e.printStackTrace(); - } - job.executionStateChanged(jobID, vertices.get(0).getID(), ExecutionState.RECOVERING, null); - } - } - } - /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index 1e8c70523ad60..9fd9384fbfeb9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -29,6 +29,7 @@ import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; +import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; @@ -508,8 +509,47 @@ Map getVerticesToBeRestarted() { * {@inheritDoc} */ @Override - public void allocatedResourcesDied(final JobID jobID, final List allocatedResource) { + public void allocatedResourcesDied(final JobID jobID, final List allocatedResources) { // TODO: Don't forget to synchronize on stage here + + for (final AllocatedResource allocatedResource : allocatedResources) { + + LOG.info("Resource on " + allocatedResource.getInstance().getName() + " for Job " + jobID + " died."); + // TODO (marrus) + + final ExecutionGraph executionGraph = getExecutionGraphByID(jobID); + + if (executionGraph == null) { + LOG.error("Cannot find execution graph for job " + jobID); + return; + } + + final List vertices = executionGraph.getVerticesAssignedToResource(allocatedResource); + final Iterator vertexIter = vertices.iterator(); + while (vertexIter.hasNext()) { + final ExecutionVertex vertex = vertexIter.next(); + + // Even if the vertex had a checkpoint before, it is now gone + vertex.updateCheckpointState(CheckpointState.NONE); + + final ExecutionState state = vertex.getExecutionState(); + + switch (state) { + case ASSIGNED: + case READY: + case STARTING: + case RUNNING: + case FINISHING: + + vertex.updateExecutionState(ExecutionState.FAILED, "The resource " + + allocatedResource.getInstance().getName() + " the vertex " + + vertex.getEnvironment().getTaskName() + " was assigned to died"); + + break; + default: + } + } + } } } From ca39de3c17c6540f988bfc450770302bb3dee529 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 25 Jan 2012 22:12:49 +0100 Subject: [PATCH 160/310] Fixed minor bug in method to cancel job --- .../java/eu/stratosphere/nephele/jobmanager/JobManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 4abe8c60a5ad6..71b6076644394 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -665,7 +665,7 @@ public JobCancelResult cancelJob(final JobID jobID) throws IOException { public void run() { eg.updateJobStatus(InternalJobStatus.CANCELING, "Job canceled by user"); final TaskCancelResult cancelResult = cancelJob(eg); - if(cancelResult.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS) { + if (cancelResult != null) { LOG.error(cancelResult.getDescription()); } } @@ -683,7 +683,7 @@ public void run() { * * @param eg * the execution graph representing the job to cancel. - * @return null no error occurred during the cancel attempt, + * @return null if no error occurred during the cancel attempt, * otherwise the returned object will describe the error */ private TaskCancelResult cancelJob(final ExecutionGraph eg) { From 57490442b114154af65516c46d747c04ad3a2279 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 25 Jan 2012 22:19:49 +0100 Subject: [PATCH 161/310] Fixed flaw in execution state listener of the execution graph --- .../nephele/executiongraph/ExecutionGraph.java | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index eaf10066eb4ea..51998e2e164c6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -1310,13 +1310,22 @@ public InternalJobStatus getJobStatus() { public void executionStateChanged(final JobID jobID, final ExecutionVertexID vertexID, final ExecutionState newExecutionState, String optionalMessage) { - if (newExecutionState == ExecutionState.RERUNNING) { + // Do not use the parameter newExecutionState here as it may already be out-dated + + final ExecutionVertex vertex = getVertexByID(vertexID); + if (vertex == null) { + LOG.error("Cannot find execution vertex with the ID " + vertexID); + } + + final ExecutionState actualExecutionState = vertex.getExecutionState(); + + if (actualExecutionState == ExecutionState.RERUNNING) { this.recovering.remove(getVertexByID(vertexID)); } - final InternalJobStatus newJobStatus = determineNewJobStatus(this, newExecutionState); + final InternalJobStatus newJobStatus = determineNewJobStatus(this, actualExecutionState); - if (newExecutionState == ExecutionState.FINISHED) { + if (actualExecutionState == ExecutionState.FINISHED) { // It is worth checking if the current stage has complete if (this.isCurrentStageCompleted()) { // Increase current execution stage @@ -1331,7 +1340,7 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver } } } - if (newExecutionState == ExecutionState.FAILED && newJobStatus == InternalJobStatus.RECOVERING) { + if (actualExecutionState == ExecutionState.FAILED && newJobStatus == InternalJobStatus.RECOVERING) { LOG.info("RECOVERING"); // FIXME (marrus) see if we even need that if (!this.recovering.contains(vertexID)) { From 177478af4061b82e111664198f500ad274526063 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 25 Jan 2012 22:24:46 +0100 Subject: [PATCH 162/310] Marked restartTask method as deprecated --- .../eu/stratosphere/nephele/protocols/TaskOperationProtocol.java | 1 + 1 file changed, 1 insertion(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index 722f9ce4edd25..9cf74d4574ed9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -159,6 +159,7 @@ List submitTasks(List tasks) * @param jobConfiguration * @throws IOException */ + @Deprecated void restartTask(ExecutionVertexID executionVertexID, Configuration jobConfiguration, Environment environment, Set activeOutputChannels) throws IOException; } From be11aa20d7956d854008d2e07615e13c97b14f82 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 25 Jan 2012 23:11:30 +0100 Subject: [PATCH 163/310] Minor fix to recovery logic --- .../scheduler/AbstractExecutionListener.java | 16 +++++++++++++++- .../jobmanager/scheduler/RecoveryLogic.java | 7 ++++++- 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java index 24eee565138e9..f76484e061594 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java @@ -23,6 +23,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionPipeline; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.instance.InstanceException; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobmanager.scheduler.local.LocalScheduler; @@ -95,7 +96,20 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver if (newExecutionState == ExecutionState.FAILED) { if (this.executionVertex.decrementRetriesLeftAndCheck()) { - RecoveryLogic.recover(this.executionVertex, this.scheduler.getVerticesToBeRestarted()); + if (RecoveryLogic.recover(this.executionVertex, this.scheduler.getVerticesToBeRestarted())) { + + // Run through the deployment procedure + this.scheduler.deployAssignedVertices(eg); + + try { + this.scheduler.requestInstances(this.executionVertex.getGroupVertex().getExecutionStage()); + } catch (InstanceException e) { + e.printStackTrace(); + // TODO: Cancel the entire job in this case + } + } else { + // TODO: Cancel the entire job in this case + } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index 35b70e1b15bad..b879374d9e36e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -33,6 +33,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.AbstractInstance; +import eu.stratosphere.nephele.instance.DummyInstance; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; import eu.stratosphere.nephele.util.SerializableArrayList; @@ -112,7 +113,11 @@ public static boolean recover(final ExecutionVertex failedVertex, } // Restart failed vertex - failedVertex.updateExecutionState(ExecutionState.SCHEDULED); + if (failedVertex.getAllocatedResource().getInstance() instanceof DummyInstance) { + failedVertex.updateExecutionState(ExecutionState.CREATED); + } else { + failedVertex.updateExecutionState(ExecutionState.ASSIGNED); + } return true; } From c0a4f756df3d81a2069e39cb7e87131556a396cc Mon Sep 17 00:00:00 2001 From: marrus Date: Thu, 26 Jan 2012 10:57:06 +0100 Subject: [PATCH 164/310] uncommented sequenznumber error log --- .../nephele/taskmanager/bytebuffered/InputChannelContext.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java index 9efe5fcd7d89c..ee0eca03695f6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java @@ -168,8 +168,8 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { final int expectedSequenceNumber = this.lastReceivedEnvelope + 1; if (sequenceNumber != expectedSequenceNumber) { - // LOG.info("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber - // + " but received " + sequenceNumber); + LOG.info("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber + + " but received " + sequenceNumber); final Buffer buffer = transferEnvelope.getBuffer(); if (buffer != null) { From 9cf89ddf4c09559156252ae2ba510945bc8df4f3 Mon Sep 17 00:00:00 2001 From: marrus Date: Thu, 26 Jan 2012 14:01:34 +0100 Subject: [PATCH 165/310] fixed PactProgram constructor --- .../eu/stratosphere/pact/client/nephele/api/PactProgram.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pact/pact-clients/src/main/java/eu/stratosphere/pact/client/nephele/api/PactProgram.java b/pact/pact-clients/src/main/java/eu/stratosphere/pact/client/nephele/api/PactProgram.java index 8ad4df22c2b75..ba21d3c05d429 100644 --- a/pact/pact-clients/src/main/java/eu/stratosphere/pact/client/nephele/api/PactProgram.java +++ b/pact/pact-clients/src/main/java/eu/stratosphere/pact/client/nephele/api/PactProgram.java @@ -106,9 +106,9 @@ public PactProgram(File jarFile, String... args) */ public PactProgram(File jarFile, String className, String... args) throws ProgramInvocationException { - this.assemblerClass = getPactAssemblerFromJar(jarFile, className); this.jarFile = jarFile; this.args = args; + this.assemblerClass = getPactAssemblerFromJar(jarFile, className); } /** From 66d27b7811ea616a0dc7b2d28063930f1e397c8d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 26 Jan 2012 19:09:30 +0100 Subject: [PATCH 166/310] Fixed duplicate registration of execution vertex listener --- .../eu/stratosphere/nephele/executiongraph/ExecutionVertex.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index ae34712bf0f3f..e8b770a0878d4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -166,8 +166,6 @@ public ExecutionVertex(final JobID jobID, final Class Date: Thu, 26 Jan 2012 20:29:29 +0100 Subject: [PATCH 167/310] Fixed problem with instance assignment update event in event collector --- .../nephele/jobmanager/EventCollector.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java index ff9daa525f175..e86a85d863e02 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java @@ -296,7 +296,15 @@ public void vertexAssignmentChanged(final ExecutionVertexID id, final AllocatedR if (instance == null) { event = new VertexAssignmentEvent(timestamp, managementVertexID, "null", "null"); } else { - event = new VertexAssignmentEvent(timestamp, managementVertexID, instance.getName(), instance.getType() + + String instanceName = null; + if (instance.getInstanceConnectionInfo() != null) { + instanceName = instance.getInstanceConnectionInfo().toString(); + } else { + instanceName = instance.toString(); + } + + event = new VertexAssignmentEvent(timestamp, managementVertexID, instanceName, instance.getType() .getIdentifier()); } From 01676eb0a13642eda134e79e990eae80e561a909 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 26 Jan 2012 21:00:08 +0100 Subject: [PATCH 168/310] Added Stratosphere header to DefaultDeserializer.java --- .../transferenvelope/DefaultDeserializer.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java index 5b7d101b038bf..2bd269c3dedea 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.transferenvelope; import java.io.IOException; From d30864576ab08c68b42927e3ef6ce378331d5e0a Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 26 Jan 2012 22:15:16 +0100 Subject: [PATCH 169/310] Improved robustness of network transfers when receiver has already died --- .../eu/stratosphere/nephele/taskmanager/TaskManager.java | 2 ++ .../bytebuffered/ByteBufferedChannelManager.java | 8 +++++++- .../taskmanager/bytebuffered/IncomingConnection.java | 2 +- 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 34fe1a1bc0257..2b3004c425456 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -616,6 +616,8 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf public SerializableArrayList replayCheckpoints(final List vertexIDs) throws IOException { + //TODO: Invalidate lookup caches + final SerializableArrayList checkpointResultList = new SerializableArrayList(); for (final ExecutionVertexID vertexID : vertexIDs) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index b0c4e015ff01a..32e4e17155f2f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -593,7 +593,13 @@ public BufferProvider getBufferProvider(final JobID jobID, final ChannelID sourc final ChannelID localReceiver = localReceivers.get(0); final ChannelContext cc = this.registeredChannels.get(localReceiver); if (cc == null) { - throw new IOException("Cannot find channel context for local receiver " + localReceiver); + + if (this.recentlyRemovedChannelIDSet.contains(localReceiver)) { + // Use the transit buffer for this purpose, data will be discarded in most cases anyway. + return this.transitBufferPool; + } else { + throw new IOException("Cannot find channel context for local receiver " + localReceiver); + } } if (!cc.isInputChannel()) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java index 1985ba8d87ddb..74256696e2869 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java @@ -80,7 +80,7 @@ public void reportTransmissionProblem(SelectionKey key, IOException ioe) { LOG.error("Connection from " + socketChannel.socket().getRemoteSocketAddress() + " encountered an IOException"); } - LOG.error(ioe); + LOG.error(StringUtils.stringifyException(ioe)); try { this.readableByteChannel.close(); From c5e1404f422eccc9bff012a55f15804b4b262e50 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 27 Jan 2012 17:10:36 +0100 Subject: [PATCH 170/310] Improved robustness of buffer implementation --- .../nephele/io/channels/FileBuffer.java | 25 ++++--- .../nephele/io/channels/MemoryBuffer.java | 65 +++++++++---------- 2 files changed, 43 insertions(+), 47 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java index f8bba5153ddf1..925ace96d570c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java @@ -20,6 +20,7 @@ import java.nio.channels.FileChannel; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; +import java.util.concurrent.atomic.AtomicBoolean; import eu.stratosphere.nephele.io.AbstractID; import eu.stratosphere.nephele.io.channels.InternalBuffer; @@ -36,7 +37,7 @@ public class FileBuffer implements InternalBuffer { private FileChannel fileChannel; - private volatile boolean writeMode; + private final AtomicBoolean writeMode; private long totalBytesWritten = 0; @@ -50,7 +51,7 @@ public class FileBuffer implements InternalBuffer { this.offset = offset; this.ownerID = ownerID; this.fileBufferManager = fileBufferManager; - this.writeMode = false; + this.writeMode = new AtomicBoolean(false); } FileBuffer(final int bufferSize, final AbstractID ownerID, final FileBufferManager fileBufferManager) { @@ -60,13 +61,13 @@ public class FileBuffer implements InternalBuffer { this.offset = 0L; this.ownerID = ownerID; this.fileBufferManager = fileBufferManager; - this.writeMode = true; + this.writeMode = new AtomicBoolean(true); } @Override public int read(WritableByteChannel writableByteChannel) throws IOException { - if (this.writeMode) { + if (this.writeMode.get()) { throw new IOException("FileBuffer is still in write mode!"); } @@ -98,7 +99,7 @@ public int read(WritableByteChannel writableByteChannel) throws IOException { @Override public int read(ByteBuffer dst) throws IOException { - if (this.writeMode) { + if (this.writeMode.get()) { throw new IOException("FileBuffer is still in write mode!"); } @@ -143,7 +144,7 @@ public int read(ByteBuffer dst) throws IOException { @Override public int write(final ReadableByteChannel readableByteChannel) throws IOException { - if (!this.writeMode) { + if (!this.writeMode.get()) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } @@ -169,7 +170,7 @@ public int write(final ReadableByteChannel readableByteChannel) throws IOExcepti @Override public int write(final ByteBuffer src) throws IOException { - if (!this.writeMode) { + if (!this.writeMode.get()) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } @@ -217,7 +218,7 @@ public boolean isOpen() { @Override public int remaining() { - if (this.writeMode) { + if (this.writeMode.get()) { return (int) (this.bufferSize - this.totalBytesWritten); } else { return (int) (this.bufferSize - this.totalBytesRead); @@ -247,7 +248,7 @@ public void recycleBuffer() { @Override public void finishWritePhase() throws IOException { - if (this.writeMode) { + if (this.writeMode.compareAndSet(true, false)) { final long currentFileSize = this.offset + this.totalBytesWritten; // If the input channel this buffer belongs to is already canceled, fileChannel may be null @@ -257,8 +258,6 @@ public void finishWritePhase() throws IOException { this.fileChannel = null; this.bufferSize = this.totalBytesWritten; // System.out.println("Buffer size: " + this.bufferSize); - // TODO: Check synchronization - this.writeMode = false; this.fileID = this.fileBufferManager.reportEndOfWritePhase(this.ownerID, currentFileSize); } @@ -276,7 +275,7 @@ public InternalBuffer duplicate() throws IOException, InterruptedException { this.fileBufferManager.increaseBufferCounter(this.ownerID, this.fileID); final FileBuffer dup = new FileBuffer((int) this.bufferSize, this.ownerID, this.fileBufferManager); - dup.writeMode = this.writeMode; + dup.writeMode.set(this.writeMode.get()); dup.fileID = this.fileID; dup.offset = this.offset; @@ -315,7 +314,7 @@ public void copyToBuffer(final Buffer destinationBuffer) throws IOException { @Override public boolean isInWriteMode() { - return this.writeMode; + return this.writeMode.get(); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java index 2395ea3c49173..14743da11d736 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java @@ -20,48 +20,46 @@ import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; import java.util.Queue; +import java.util.concurrent.atomic.AtomicBoolean; import eu.stratosphere.nephele.io.channels.InternalBuffer; public class MemoryBuffer implements InternalBuffer { private final MemoryBufferRecycler bufferRecycler; - + private final ByteBuffer byteBuffer; - - private volatile boolean writeMode = true; - MemoryBuffer(int bufferSize, ByteBuffer byteBuffer, Queue queueForRecycledBuffers) { + private final AtomicBoolean writeMode = new AtomicBoolean(true); + + MemoryBuffer(final int bufferSize, final ByteBuffer byteBuffer, final Queue queueForRecycledBuffers) { if (bufferSize > byteBuffer.capacity()) { throw new IllegalArgumentException("Requested buffer size is " + bufferSize + ", but provided byte buffer only has a capacity of " + byteBuffer.capacity()); } - + this.bufferRecycler = new MemoryBufferRecycler(byteBuffer, queueForRecycledBuffers); - + this.byteBuffer = byteBuffer; this.byteBuffer.position(0); this.byteBuffer.limit(bufferSize); } - - private MemoryBuffer(int bufferSize, ByteBuffer byteBuffer, MemoryBufferRecycler bufferRecycler) { - + + private MemoryBuffer(final int bufferSize, final ByteBuffer byteBuffer, final MemoryBufferRecycler bufferRecycler) { + this.bufferRecycler = bufferRecycler; - this.byteBuffer = byteBuffer; this.byteBuffer.position(0); this.byteBuffer.limit(bufferSize); } @Override - public int read(ByteBuffer dst) throws IOException { + public int read(final ByteBuffer dst) throws IOException { - if (this.writeMode) { - this.writeMode = false; - this.byteBuffer.flip(); - // System.out.println("Switching to read mode: " + this.byteBuffer); + if (this.writeMode.get()) { + throw new IOException("Buffer is still in write mode!"); } if (!this.byteBuffer.hasRemaining()) { @@ -88,9 +86,9 @@ public int read(ByteBuffer dst) throws IOException { } @Override - public int read(WritableByteChannel writableByteChannel) throws IOException { + public int read(final WritableByteChannel writableByteChannel) throws IOException { - if (this.writeMode) { + if (this.writeMode.get()) { throw new IOException("Buffer is still in write mode!"); } @@ -114,9 +112,9 @@ public boolean isOpen() { } @Override - public int write(ByteBuffer src) throws IOException { + public int write(final ByteBuffer src) throws IOException { - if (!this.writeMode) { + if (!this.writeMode.get()) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } @@ -134,9 +132,9 @@ public int write(ByteBuffer src) throws IOException { } @Override - public int write(ReadableByteChannel readableByteChannel) throws IOException { + public int write(final ReadableByteChannel readableByteChannel) throws IOException { - if (!this.writeMode) { + if (!this.writeMode.get()) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } @@ -171,12 +169,11 @@ public void recycleBuffer() { @Override public void finishWritePhase() { - if (!this.writeMode) { - throw new IllegalStateException("MemoryBuffer is already in write mode!"); + if (!this.writeMode.compareAndSet(true, false)) { + throw new IllegalStateException("MemoryBuffer is already in read mode!"); } this.byteBuffer.flip(); - this.writeMode = false; } @Override @@ -190,28 +187,28 @@ public InternalBuffer duplicate() { final MemoryBuffer duplicatedMemoryBuffer = new MemoryBuffer(this.byteBuffer.limit(), this.byteBuffer .duplicate(), this.bufferRecycler); - + this.bufferRecycler.increaseReferenceCounter(); duplicatedMemoryBuffer.byteBuffer.position(this.byteBuffer.position()); duplicatedMemoryBuffer.byteBuffer.limit(this.byteBuffer.limit()); - duplicatedMemoryBuffer.writeMode = this.writeMode; + duplicatedMemoryBuffer.writeMode.set(this.writeMode.get()); return duplicatedMemoryBuffer; } @Override - public void copyToBuffer(Buffer destinationBuffer) throws IOException { - + public void copyToBuffer(final Buffer destinationBuffer) throws IOException { + final int oldPos = this.byteBuffer.position(); this.byteBuffer.position(0); - - while(remaining() > 0) { + + while (remaining() > 0) { destinationBuffer.write(this); } - + this.byteBuffer.position(oldPos); - + destinationBuffer.finishWritePhase(); } @@ -220,7 +217,7 @@ public void copyToBuffer(Buffer destinationBuffer) throws IOException { */ @Override public boolean isInWriteMode() { - - return this.writeMode; + + return this.writeMode.get(); } } From 5cc68c612a5eea82d5d345bb5eb0e18eed0cc586 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 27 Jan 2012 17:55:25 +0100 Subject: [PATCH 171/310] Refactored code for recovery --- .../nephele/execution/ExecutionState.java | 13 +- .../executiongraph/ExecutionGraph.java | 41 +--- .../executiongraph/InternalJobStatus.java | 7 +- .../nephele/jobmanager/JobManager.java | 10 - .../nephele/jobmanager/RecoveryThread.java | 222 ------------------ 5 files changed, 4 insertions(+), 289 deletions(-) delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java index ff9562242b957..a6bcd3cb47482 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java @@ -75,16 +75,5 @@ public enum ExecutionState { /** * The task has been aborted due to a failure during execution. */ - FAILED, - - /** - * The task has been failed and will be restarted. - */ - - RECOVERING, - - /** - * Recovery is finished the job is running normal again. - */ - RERUNNING; + FAILED; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index 51998e2e164c6..a1952e6701f08 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -136,8 +136,6 @@ public class ExecutionGraph implements ExecutionListener { */ private final CopyOnWriteArrayList executionStageListeners = new CopyOnWriteArrayList(); - private final CopyOnWriteArrayList recovering = new CopyOnWriteArrayList(); - /** * Private constructor used for duplicating execution vertices. * @@ -1206,8 +1204,7 @@ private boolean jobHasFailedOrCanceledStatus() { return true; } - // TODO: Make this static - private InternalJobStatus determineNewJobStatus(final ExecutionGraph eg, + private static InternalJobStatus determineNewJobStatus(final ExecutionGraph eg, final ExecutionState latestStateChange) { final InternalJobStatus currentJobStatus = eg.getJobStatus(); @@ -1242,32 +1239,14 @@ private InternalJobStatus determineNewJobStatus(final ExecutionGraph eg, final ExecutionVertex vertex = it.next(); if (vertex.getExecutionState() == ExecutionState.FAILED) { - if (!vertex.hasRetriesLeft()) { - System.out.println(" Vertex failed finally"); - return InternalJobStatus.FAILING; - } else { - return InternalJobStatus.RECOVERING; - } + return InternalJobStatus.FAILING; } } } - if (latestStateChange == ExecutionState.RECOVERING) { - return InternalJobStatus.RECOVERING; - } if (eg.jobHasFinishedStatus()) { return InternalJobStatus.FINISHED; } break; - case RECOVERING: - if (latestStateChange == ExecutionState.RERUNNING) { - if (this.recovering.isEmpty()) { - return InternalJobStatus.RUNNING; - } - } - if (latestStateChange == ExecutionState.FAILED) { - LOG.info("Another Failed Vertex while recovering"); - } - break; case FAILING: if (eg.jobHasFailedOrCanceledStatus()) { return InternalJobStatus.FAILED; @@ -1319,10 +1298,6 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver final ExecutionState actualExecutionState = vertex.getExecutionState(); - if (actualExecutionState == ExecutionState.RERUNNING) { - this.recovering.remove(getVertexByID(vertexID)); - } - final InternalJobStatus newJobStatus = determineNewJobStatus(this, actualExecutionState); if (actualExecutionState == ExecutionState.FINISHED) { @@ -1340,13 +1315,6 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver } } } - if (actualExecutionState == ExecutionState.FAILED && newJobStatus == InternalJobStatus.RECOVERING) { - LOG.info("RECOVERING"); - // FIXME (marrus) see if we even need that - if (!this.recovering.contains(vertexID)) { - this.recovering.add(this.getVertexByID(vertexID)); - } - } updateJobStatus(newJobStatus, optionalMessage); } @@ -1517,11 +1485,6 @@ public void initialExecutionResourcesExhausted(final JobID jobID, final Executio // Nothing to do here } - public List getFailedVertices() { - - return this.recovering; - } - /** * Reconstructs the execution pipelines for the entire execution graph. */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/InternalJobStatus.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/InternalJobStatus.java index 534b87348630d..ef332167b9ec1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/InternalJobStatus.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/InternalJobStatus.java @@ -68,12 +68,7 @@ public enum InternalJobStatus { /** * All of the job's tasks have successfully finished. */ - FINISHED, - - /** - * At least one task of the job has failed. The recovery-process is in progress. - */ - RECOVERING; + FINISHED; /** * Converts an internal job status in a {@link JobStatus} state. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 71b6076644394..6fca22abff136 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -1059,16 +1059,6 @@ public void jobStatusHasChanged(final ExecutionGraph executionGraph, final Inter // Unregister job for Nephele's monitoring, optimization components, and dynamic input split assignment unregisterJob(executionGraph); } - - if (newJobStatus == InternalJobStatus.RECOVERING) { - try { - RecoveryThread recoverythread = new RecoveryThread(executionGraph); - recoverythread.start(); - } catch (Exception e) { - e.printStackTrace(); - } - - } } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java deleted file mode 100644 index 235c75ad7dd98..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/RecoveryThread.java +++ /dev/null @@ -1,222 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.jobmanager; - -import java.io.IOException; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Queue; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; -import eu.stratosphere.nephele.execution.ExecutionState; -import eu.stratosphere.nephele.executiongraph.CheckpointState; -import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.instance.AbstractInstance; -import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; -import eu.stratosphere.nephele.util.SerializableArrayList; - -/** - * @author marrus - */ -public class RecoveryThread extends Thread { - - private static final Log LOG = LogFactory.getLog(RecoveryThread.class); - - private final ExecutionGraph job; - - List failedVertices; - - List recovered = new ArrayList(); - private List globalConsistentCheckpoint = new SerializableArrayList(); - - /** - * Initializes RecoveryThread. - * - * @param job The Job with - * @throws Exception - */ - public RecoveryThread(final ExecutionGraph job) throws Exception { - super("Recovery Thread"); - this.job = job; - this.failedVertices = new ArrayList(); - this.failedVertices.addAll(job.getFailedVertices()); - } - - /** - * {@inheritDoc} - */ - @Override - public void run() { - if (this.failedVertices.isEmpty()) { - LOG.error("No failed vertices to recover"); - } - //FIXME (marrus) dirty fix - while (!this.failedVertices.isEmpty()) { - ExecutionVertex torecover = this.failedVertices.remove(0); - if(!this.recovered.contains(torecover)){ - recover(torecover); - this.recovered.add(torecover); - } - if (this.failedVertices.isEmpty()) { - this.failedVertices = this.job.getFailedVertices(); - } - } - LOG.info("Recovery Finished"); - } - - /** - * - */ - private boolean recover(final ExecutionVertex failed) { - List replayCheckpoints = new ArrayList(); - - - LOG.info("Staring Recovery for " + failed); - //findRestarts(failed); - - final List restart = findRestarts(failed); - - //restart all predecessors without checkpoint - Iterator restartIterator = restart.iterator(); - while (restartIterator.hasNext()) { - ExecutionVertex vertex = restartIterator.next(); - if (vertex.getID() != failed.getID()) { - try { - vertex.getAllocatedResource().getInstance().restartTask(vertex.getID(),this.job.getJobConfiguration(), vertex.getEnvironment(), vertex.constructInitialActiveOutputChannelsSet() ); - } catch (IOException e) { - e.printStackTrace(); - this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); - return false; - } - } - - } - //restart failed vertex - try { - failed.getAllocatedResource().getInstance().submitTask(failed.getID(), this.job.getJobConfiguration(), failed.getEnvironment(), failed.constructInitialActiveOutputChannelsSet()); - } catch (IOException e1) { - e1.printStackTrace(); - this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); - return false; - } - - //get list of instances of consistencheckpoints - List instances = new SerializableArrayList(); - for (ExecutionVertexID id : this.globalConsistentCheckpoint) { - AbstractInstance instance = this.job.getVertexByID(id).getAllocatedResource().getInstance(); - if (!instances.contains(instance)) { - instances.add(instance); - } - } - Iterator instanceIterator = instances.iterator(); - while (instanceIterator.hasNext()) { - //replay all necessary checkpoints - try { - - replayCheckpoints.addAll(instanceIterator.next().replayCheckpoints(this.globalConsistentCheckpoint)); - } catch (IOException e) { - e.printStackTrace(); - this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); - return false; - } - } - - for(CheckpointReplayResult replayResult : replayCheckpoints ){ - if (replayResult.getReturnCode() == ReturnCode.ERROR) { - LOG.info("Replay of Checkpoints return Error " + replayResult.getDescription() ); - this.job.executionStateChanged(this.job.getJobID(), null, ExecutionState.FAILED, null); - return false; - } - } - LOG.info("FINISHED RECOVERY for " + failed.getName()); - this.job.executionStateChanged(this.job.getJobID(), failed.getID(), ExecutionState.RERUNNING, null); - - return true; - } - - /** - * @param failed - * @return - */ - private List findRestarts(ExecutionVertex failed) { - ArrayList restart = new ArrayList(); - Queue totest = new ArrayDeque(); - ArrayList visited = new ArrayList(); - totest.add(failed); - ExecutionVertex vertex = failed; - while (!totest.isEmpty()) { - - vertex = totest.peek(); - totest.remove(vertex); - if (!restart.contains(vertex)) { - restart.add(vertex); - } - //predecessors must be either checkpoints or need to be restarted too - for (int j = 0; j < vertex.getNumberOfPredecessors(); j++) { - ExecutionVertex predecessor = vertex.getPredecessor(j); - if (predecessor.getCheckpointState() != CheckpointState.PARTIAL - && predecessor.getCheckpointState() != CheckpointState.COMPLETE) { - - restart.add(predecessor); - if (!visited.contains(predecessor)) { - totest.add(predecessor); - } - } else { - if (!this.globalConsistentCheckpoint.contains(predecessor.getID())) { - this.globalConsistentCheckpoint.add(predecessor.getID()); - } - - - } - } - visited.add(vertex); - } - - return restart; - } - -// private List findFollowers(ExecutionVertex vertex, ArrayList restart) { -// ArrayList follower = new ArrayList(); -// -// for (int i = 0; i < vertex.getNumberOfSuccessors(); i++) { -// ExecutionVertex successor = vertex.getSuccessor(i); -// if (!restart.contains(successor)) { -// follower.add(successor); -// if (successor.getCheckpointState() == CheckpointState.PARTIAL) { -// this.checkpoints.remove(successor); -// -// this.globalConsistentCheckpoint.remove(successor.getID()); -// final SerializableArrayList checkpointsToRemove = new SerializableArrayList(); -// checkpointsToRemove.add(successor.getID()); -// try { -// successor.getAllocatedResource().getInstance().removeCheckpoints(checkpointsToRemove); -// } catch (IOException e) { -// e.printStackTrace(); -// } -// } -// } -// } -// -// return follower; -// } - -} From 8422d8888012e0fc44b7597a13159ed2424ee324 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 27 Jan 2012 18:28:36 +0100 Subject: [PATCH 172/310] Decoupled task execution from network failures --- .../ByteBufferedChannelManager.java | 27 ------------------- .../bytebuffered/ChannelContext.java | 4 --- .../bytebuffered/InputChannelContext.java | 20 +++++++------- .../bytebuffered/OutputChannelContext.java | 6 +---- 4 files changed, 12 insertions(+), 45 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 32e4e17155f2f..106258601549d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -274,33 +274,6 @@ public void shutdown() { this.networkConnectionManager.shutDown(); } - public void reportIOExceptionForAllInputChannels(IOException ioe) { - - final Iterator it = this.registeredChannels.values().iterator(); - - while (it.hasNext()) { - - final ChannelContext channelContext = it.next(); - if (channelContext.isInputChannel()) { - channelContext.reportIOException(ioe); - } - } - } - - public void reportIOExceptionForOutputChannel(ChannelID sourceChannelID, IOException ioe) { - - final ChannelContext channelContext = this.registeredChannels.get(sourceChannelID); - - if (channelContext == null) { - LOG.error("Cannot find network output channel with ID " + sourceChannelID); - return; - } - - if (channelContext.isInputChannel()) { - channelContext.reportIOException(ioe); - } - } - public NetworkConnectionManager getNetworkConnectionManager() { return this.networkConnectionManager; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java index e1a3b54cca48e..009631ccd3619 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java @@ -15,8 +15,6 @@ package eu.stratosphere.nephele.taskmanager.bytebuffered; -import java.io.IOException; - import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; @@ -30,8 +28,6 @@ interface ChannelContext { public ChannelID getChannelID(); public ChannelID getConnectedChannelID(); - - public void reportIOException(IOException ioe); public void queueTransferEnvelope(TransferEnvelope transferEnvelope); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java index 9efe5fcd7d89c..a67e02932f1f7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java @@ -168,8 +168,17 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { final int expectedSequenceNumber = this.lastReceivedEnvelope + 1; if (sequenceNumber != expectedSequenceNumber) { - // LOG.info("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber - // + " but received " + sequenceNumber); + if (sequenceNumber > expectedSequenceNumber) { + // This is a problem, now we are actually missing some data + this.byteBufferedInputChannel.reportIOException(new IOException("Missing data packet " + + expectedSequenceNumber + " from sender")); + this.byteBufferedInputChannel.checkForNetworkEvents(); + } + + if (LOG.isDebugEnabled()) { + LOG.info("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber + + " but received " + sequenceNumber); + } final Buffer buffer = transferEnvelope.getBuffer(); if (buffer != null) { @@ -188,13 +197,6 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { this.byteBufferedInputChannel.checkForNetworkEvents(); } - @Override - public void reportIOException(IOException ioe) { - - this.byteBufferedInputChannel.reportIOException(ioe); - this.byteBufferedInputChannel.checkForNetworkEvents(); - } - @Override public ChannelID getChannelID() { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java index c1774287dcfc0..cc047748d1256 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java @@ -250,11 +250,7 @@ void processEvent(AbstractEvent abstractEvent) { this.byteBufferedOutputChannel.processEvent(abstractEvent); } - /** - * {@inheritDoc} - */ - @Override - public void reportIOException(IOException ioe) { + void reportIOException(final IOException ioe) { this.byteBufferedOutputChannel.reportIOException(ioe); } From ccda7445d7ca6c363885ba6bb2eaa8114e39d1fd Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 27 Jan 2012 20:18:50 +0100 Subject: [PATCH 173/310] Improved robustness and logging of byte-buffered channel manager --- .../bytebuffered/ByteBufferedChannelManager.java | 8 +++++++- .../taskmanager/bytebuffered/InputChannelContext.java | 4 ++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 106258601549d..60c244b7dd595 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -314,7 +314,13 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, final ChannelContext cc = this.registeredChannels.get(localReceiver); if (cc == null) { - throw new IOException("Cannot find channel context for local receiver " + localReceiver); + + if (this.recentlyRemovedChannelIDSet.contains(localReceiver)) { + //TODO: Reconsider buffer recycling here + return; + } else { + throw new IOException("Cannot find channel context for local receiver " + localReceiver); + } } if (!cc.isInputChannel()) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java index a67e02932f1f7..1a0c65a185fdd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java @@ -170,8 +170,8 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { if (sequenceNumber > expectedSequenceNumber) { // This is a problem, now we are actually missing some data - this.byteBufferedInputChannel.reportIOException(new IOException("Missing data packet " - + expectedSequenceNumber + " from sender")); + this.byteBufferedInputChannel.reportIOException(new IOException("Expected data packet " + + expectedSequenceNumber + " but received " + sequenceNumber)); this.byteBufferedInputChannel.checkForNetworkEvents(); } From a4a84ac353e53fbc342e721019b6642088e1890f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 27 Jan 2012 22:01:48 +0100 Subject: [PATCH 174/310] Removed deprecated restartTask method --- .../nephele/instance/AbstractInstance.java | 12 ------------ .../nephele/protocols/TaskOperationProtocol.java | 13 ------------- .../nephele/taskmanager/TaskManager.java | 16 ---------------- 3 files changed, 41 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index bbb553ff1293a..ec93d94f74023 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -321,16 +321,4 @@ public synchronized void killTaskManager() throws IOException { getTaskManager().killTaskManager(); } - - /** - * @param executionVertexID - * @param serializableHashSet - * @param environment - * @param configuration - * @throws IOException - */ - public void restartTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final Environment environment, final Set activeOutputChannels) throws IOException { - getTaskManager().restartTask(id, jobConfiguration, environment, activeOutputChannels); - } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index 9cf74d4574ed9..89a55e9c6d788 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -149,17 +149,4 @@ List submitTasks(List tasks) * throws if an error occurs during this remote procedure call */ void killTaskManager() throws IOException; - - /** - * Restarts a given Task by unregistering an submitting it - * - * @param executionVertexID - * @param activeOutputChannels - * @param environment - * @param jobConfiguration - * @throws IOException - */ - @Deprecated - void restartTask(ExecutionVertexID executionVertexID, Configuration jobConfiguration, Environment environment, - Set activeOutputChannels) throws IOException; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 2b3004c425456..b732b4c09e4ae 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -921,20 +921,4 @@ public void run() { timer.schedule(timerTask, 10L); } - - /** - * {@inheritDoc} - * - * @throws IOException - */ - @Override - public void restartTask(ExecutionVertexID id, Configuration jobConfiguration, - Environment environment, Set activeOutputChannels) throws IOException { - Task torestart = this.runningTasks.get(id); - // torestart.markAsRestarting(); - torestart.cancelExecution(); - // unregisterTask(id,torestart); - submitTask(id, jobConfiguration, environment, activeOutputChannels); - - } } From 9f34e38d1dd267cc48079c20018304ead2be3573 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 27 Jan 2012 22:15:22 +0100 Subject: [PATCH 175/310] Introduced task manager RPC method to invalidate entries from the receiver lookup cache --- .../nephele/instance/AbstractInstance.java | 13 +++++++++++++ .../protocols/TaskOperationProtocol.java | 18 ++++++++++++++---- .../nephele/taskmanager/TaskManager.java | 17 +++++++++++++---- .../ByteBufferedChannelManager.java | 19 +++++++++++++++++-- 4 files changed, 57 insertions(+), 10 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index ec93d94f74023..34bb97d29d667 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -321,4 +321,17 @@ public synchronized void killTaskManager() throws IOException { getTaskManager().killTaskManager(); } + + /** + * Invalidates the entries identified by the given channel IDs from the remote task manager's receiver lookup cache. + * + * @param channelIDs + * the channel IDs identifying the cache entries to invalidate + * @throws IOException + * thrown if an error occurs during this remote procedure call + */ + public synchronized void invalidateLookupCacheEntries(final Set channelIDs) throws IOException { + + getTaskManager().invalidateLookupCacheEntries(channelIDs); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index 89a55e9c6d788..1d670b0d9a73c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -115,7 +115,7 @@ List submitTasks(List tasks) * @param update * a {@link LibraryCacheUpdate} object used to transmit the library data * @throws IOException - * if an error occurs during this remote procedure call + * thrown if an error occurs during this remote procedure call */ void updateLibraryCache(LibraryCacheUpdate update) throws IOException; @@ -129,16 +129,26 @@ List submitTasks(List tasks) * @param listOfVertexIDs * the list of vertex IDs which identify the checkpoints to be removed * @throws IOException - * if an error occurs during this remote procedure call + * thrown if an error occurs during this remote procedure call */ void removeCheckpoints(List listOfVertexIDs) throws IOException; + /** + * Invalidates the entries identified by the given channel IDs from the task manager's receiver lookup cache. + * + * @param channelIDs + * the channel IDs identifying the cache entries to invalidate + * @throws IOException + * thrown if an error occurs during this remote procedure call + */ + void invalidateLookupCacheEntries(Set channelIDs) throws IOException; + /** * Triggers the task manager write the current utilization of its read and write buffers to its logs. * This method is primarily for debugging purposes. * * @throws IOException - * throws if an error occurs while transmitting the request + * thrown if an error occurs while transmitting the request */ void logBufferUtilization() throws IOException; @@ -146,7 +156,7 @@ List submitTasks(List tasks) * Kills the task manager. This method is mainly intended to test and debug Nephele's fault tolerance mechanisms. * * @throws IOException - * throws if an error occurs during this remote procedure call + * thrown if an error occurs during this remote procedure call */ void killTaskManager() throws IOException; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index b732b4c09e4ae..42bc7577223db 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -375,9 +375,9 @@ public void runIOLoop() { // Check the status of the task threads to detect unexpected thread terminations checkTaskExecution(); - + // Clean up set of recently unregistered channels - this.byteBufferedChannelManager.cleanUpRecentlyRemovedChannelIDSet(); + this.byteBufferedChannelManager.cleanUpRecentlyRemovedChannelIDSet(); } // Shutdown the individual components of the task manager @@ -616,8 +616,8 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf public SerializableArrayList replayCheckpoints(final List vertexIDs) throws IOException { - //TODO: Invalidate lookup caches - + // TODO: Invalidate lookup caches + final SerializableArrayList checkpointResultList = new SerializableArrayList(); for (final ExecutionVertexID vertexID : vertexIDs) { @@ -921,4 +921,13 @@ public void run() { timer.schedule(timerTask, 10L); } + + /** + * {@inheritDoc} + */ + @Override + public void invalidateLookupCacheEntries(final Set channelIDs) throws IOException { + + this.byteBufferedChannelManager.invalidateLookupCacheEntries(channelIDs); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 60c244b7dd595..3081eaaf6d2bc 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -314,9 +314,9 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, final ChannelContext cc = this.registeredChannels.get(localReceiver); if (cc == null) { - + if (this.recentlyRemovedChannelIDSet.contains(localReceiver)) { - //TODO: Reconsider buffer recycling here + // TODO: Reconsider buffer recycling here return; } else { throw new IOException("Cannot find channel context for local receiver " + localReceiver); @@ -677,4 +677,19 @@ public void cleanUpRecentlyRemovedChannelIDSet() { this.recentlyRemovedChannelIDSet.cleanup(); } + + /** + * Invalidates the entries identified by the given channel IDs from the receiver lookup cache. + * + * @param channelIDs + * the channel IDs identifying the cache entries to invalidate + */ + public void invalidateLookupCacheEntries(final Set channelIDs) { + + final Iterator it = channelIDs.iterator(); + while (it.hasNext()) { + + this.receiverCache.remove(it.next()); + } + } } From be4886233a76cc531537d7840da74f737ac64257 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 28 Jan 2012 14:56:21 +0100 Subject: [PATCH 176/310] Added method to invalidate lookup cache entries to recovery logic --- .../jobmanager/scheduler/RecoveryLogic.java | 107 ++++++++++++++++++ 1 file changed, 107 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index b879374d9e36e..905453660171e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -28,15 +28,25 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.executiongraph.CheckpointState; +import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.instance.DummyInstance; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.channels.AbstractInputChannel; +import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; +import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.util.SerializableArrayList; +import eu.stratosphere.nephele.util.SerializableHashSet; import eu.stratosphere.nephele.util.StringUtils; /** @@ -88,6 +98,11 @@ public static boolean recover(final ExecutionVertex failedVertex, } + // Invalidate the lookup caches + if (!invalidateReceiverLookupCaches(failedVertex, verticesToBeCanceled)) { + return false; + } + // Replay all necessary checkpoints final Iterator>> checkpointIterator = checkpointsToBeReplayed .entrySet().iterator(); @@ -166,4 +181,96 @@ private static void findVerticesToRestart(final ExecutionVertex failedVertex, visited.add(vertex); } } + + private static final boolean invalidateReceiverLookupCaches(final ExecutionVertex failedVertex, + final Set verticesToBeCanceled) { + + final Map> entriesToInvalidate = new HashMap>(); + + final ExecutionGraph eg = failedVertex.getExecutionGraph(); + + final Environment env = failedVertex.getEnvironment(); + for (int i = 0; i < env.getNumberOfOutputGates(); ++i) { + + final OutputGate outputGate = env.getOutputGate(i); + for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { + + final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); + if (outputChannel.getType() == ChannelType.FILE) { + // Connected vertex is not yet running + continue; + } + + final ExecutionVertex connectedVertex = eg.getVertexByChannelID(outputChannel.getConnectedChannelID()); + if (connectedVertex == null) { + LOG.error("Connected vertex is null"); + continue; + } + + if (verticesToBeCanceled.contains(connectedVertex)) { + // Vertex will be canceled anyways + continue; + } + + final AbstractInstance instance = connectedVertex.getAllocatedResource().getInstance(); + Set channelIDs = entriesToInvalidate.get(instance); + if (channelIDs == null) { + channelIDs = new SerializableHashSet(); + entriesToInvalidate.put(instance, channelIDs); + } + + channelIDs.add(outputChannel.getID()); + } + } + + for (int i = 0; i < env.getNumberOfInputGates(); ++i) { + + final InputGate inputGate = env.getInputGate(i); + for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { + + final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); + if (inputChannel.getType() == ChannelType.FILE) { + // Connected vertex is not running anymore + continue; + } + + final ExecutionVertex connectedVertex = eg.getVertexByChannelID(inputChannel.getConnectedChannelID()); + if (connectedVertex == null) { + LOG.error("Connected vertex is null"); + continue; + } + + if (verticesToBeCanceled.contains(connectedVertex)) { + // Vertex will be canceled anyways + continue; + } + + final AbstractInstance instance = connectedVertex.getAllocatedResource().getInstance(); + Set channelIDs = entriesToInvalidate.get(instance); + if (channelIDs == null) { + channelIDs = new SerializableHashSet(); + entriesToInvalidate.put(instance, channelIDs); + } + + channelIDs.add(inputChannel.getID()); + } + } + + final Iterator>> it = entriesToInvalidate.entrySet().iterator(); + + while (it.hasNext()) { + + final Map.Entry> entry = it.next(); + final AbstractInstance instance = entry.getKey(); + + try { + instance.invalidateLookupCacheEntries(entry.getValue()); + } catch (IOException ioe) { + LOG.error(StringUtils.stringifyException(ioe)); + return false; + } + } + + return true; + } } From 63bb42dcd205507ad10bb8586b204c8b74b1f841 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 28 Jan 2012 16:15:37 +0100 Subject: [PATCH 177/310] Improved logging of errors at the task manager --- .../main/java/eu/stratosphere/nephele/taskmanager/Task.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index 173c16a074b9f..08c4e98295986 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -92,6 +92,11 @@ public void executionStateChanged(final ExecutionState newExecutionState, final // Check the state transition ExecutionStateTransition.checkTransition(getTaskName(), this.executionState, newExecutionState); + // Make sure the reason for a transition to FAILED appears in the log files + if (newExecutionState == ExecutionState.FAILED) { + LOG.error(optionalMessage); + } + // Notify all listener objects final Iterator it = this.registeredListeners.iterator(); while (it.hasNext()) { From 6a39c855cf27d36c18ee7ce2f672b1abfa639fa0 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 28 Jan 2012 21:01:50 +0100 Subject: [PATCH 178/310] Unified event delivery model across input and output channels --- .../AbstractByteBufferedOutputChannel.java | 84 +------------------ .../bytebuffered/OutputChannelContext.java | 64 +++++++++----- .../bytebuffered/OutputGateContext.java | 8 +- 3 files changed, 48 insertions(+), 108 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java index 86d575859117f..dabce96f7ef7d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java @@ -26,7 +26,6 @@ import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.io.channels.SerializationBuffer; import eu.stratosphere.nephele.io.compression.CompressionEvent; import eu.stratosphere.nephele.io.compression.CompressionLevel; @@ -51,37 +50,16 @@ public abstract class AbstractByteBufferedOutputChannel extend */ private boolean closeRequested = false; - /** - * Stores whether the channel has received the acknowledgment - * for the close request from its connected input channel. - */ - private boolean closeAcknowledgementReceived = false; - /** * The output channel broker the channel should contact to request and release write buffers. */ private ByteBufferedOutputChannelBroker outputChannelBroker = null; - /** - * Synchronization object to protect variables that are accessed by the task and the framework. - */ - private final Object synchronisationObject = new Object(); - - /** - * Temporarily stores a possible IOException that may be reported by the framework. - */ - private IOException ioException = null; - /** * The compressor used to compress the outgoing data. */ private Compressor compressor = null; - /** - * Indicates the period of time this output channel shall throttle down so that the consumer can catch up. - */ - private long throttelingDuration = 0L; - /** * Buffer for the uncompressed data. */ @@ -122,18 +100,8 @@ public boolean isClosed() throws IOException, InterruptedException { if (this.closeRequested && this.uncompressedDataBuffer == null && !this.serializationBuffer.dataLeftFromPreviousSerialization()) { - if (this.ioException != null) { - throw this.ioException; - } - - if (this.outputChannelBroker.hasDataLeftToTransmit()) { - return false; - } - - synchronized (this.synchronisationObject) { - if (this.closeAcknowledgementReceived) { - return true; - } + if (!this.outputChannelBroker.hasDataLeftToTransmit()) { + return true; } } @@ -216,20 +184,6 @@ public void writeRecord(T record) throws IOException, InterruptedException { // Get a write buffer from the broker if (this.uncompressedDataBuffer == null) { - synchronized (this.synchronisationObject) { - - if (this.ioException != null) { - throw this.ioException; - } - - if (this.throttelingDuration > 0L) { - // Temporarily, stop producing data - this.synchronisationObject.wait(this.throttelingDuration); - // Reset throttling duration - this.throttelingDuration = 0L; - } - } - requestWriteBuffersFromBroker(); } @@ -303,19 +257,6 @@ public void processEvent(AbstractEvent event) { if (event instanceof AbstractTaskEvent) { getOutputGate().deliverEvent((AbstractTaskEvent) event); - } else if (event instanceof NetworkThrottleEvent) { - if (getType() == ChannelType.FILE) { - LOG.error("FileChannel " + getID() + " received NetworkThrottleEvent"); - } else { - synchronized (this.synchronisationObject) { - final NetworkThrottleEvent nte = (NetworkThrottleEvent) event; - this.throttelingDuration = nte.getDuration(); - } - } - } else if (event instanceof ByteBufferedChannelCloseEvent) { - synchronized (this.synchronisationObject) { - this.closeAcknowledgementReceived = true; - } } else { LOG.error("Channel " + getID() + " received unknown event " + event); } @@ -378,23 +319,6 @@ public void flush() throws IOException, InterruptedException { } } - /** - * This method is called by the framework to report - * an {@link IOException} that occurred while trying to process - * one of the buffers issued by this channel. - * - * @param ioe - * the {@link IOException} which occurred - */ - public void reportIOException(IOException ioe) { - - synchronized (this.synchronisationObject) { - this.ioException = ioe; - // Wake up thread if it has been throttled down - this.synchronisationObject.notify(); - } - } - /** * {@inheritDoc} */ @@ -403,10 +327,6 @@ public void releaseResources() { this.closeRequested = true; - synchronized (this.synchronisationObject) { - this.closeAcknowledgementReceived = true; - } - this.serializationBuffer.clear(); if (this.compressedDataBuffer != null) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java index cc047748d1256..bd158acb8c465 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java @@ -17,6 +17,8 @@ import java.io.IOException; import java.util.Iterator; +import java.util.Queue; +import java.util.concurrent.LinkedBlockingDeque; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -28,6 +30,7 @@ import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; import eu.stratosphere.nephele.io.channels.bytebuffered.BufferPairResponse; import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelActivateEvent; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedOutputChannelBroker; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; @@ -59,12 +62,22 @@ final class OutputChannelContext implements ByteBufferedOutputChannelBroker, Cha /** * Indicates whether the receiver of an envelope is currently running. */ - private volatile boolean isReceiverRunning = false; + private boolean isReceiverRunning = false; + + /** + * Stores whether the receiver has acknowledged the close request from this channel. + */ + private boolean closeAcknowledgementReceived = false; /** * Queue to store outgoing transfer envelope in case the receiver of the envelopes is not yet running. */ - private SpillingQueue queuedOutgoingEnvelopes; + private final SpillingQueue queuedOutgoingEnvelopes; + + /** + * Stores incoming events for this output channel. + */ + private final Queue incomingEventQueue = new LinkedBlockingDeque(); /** * The sequence number for the next {@link TransferEnvelope} to be created. @@ -145,6 +158,8 @@ private TransferEnvelope createNewOutgoingTransferEnvelope() { @Override public void releaseWriteBuffers() throws IOException, InterruptedException { + processIncomingEvents(); + if (this.outgoingTransferEnvelope == null) { LOG.error("Cannot find transfer envelope for channel with ID " + this.byteBufferedOutputChannel.getID()); return; @@ -245,14 +260,9 @@ boolean isChannelActive() { * @param abstractEvent * the event to be reported */ - void processEvent(AbstractEvent abstractEvent) { + void processEvent(final AbstractEvent abstractEvent) { - this.byteBufferedOutputChannel.processEvent(abstractEvent); - } - - void reportIOException(final IOException ioe) { - - this.byteBufferedOutputChannel.reportIOException(ioe); + this.incomingEventQueue.offer(abstractEvent); } /** @@ -286,7 +296,7 @@ public JobID getJobID() { * {@inheritDoc} */ @Override - public void queueTransferEnvelope(TransferEnvelope transferEnvelope) { + public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { if (transferEnvelope.getBuffer() != null) { LOG.error("Transfer envelope for output channel has buffer attached"); @@ -294,15 +304,7 @@ public void queueTransferEnvelope(TransferEnvelope transferEnvelope) { final Iterator it = transferEnvelope.getEventList().iterator(); while (it.hasNext()) { - - final AbstractEvent event = it.next(); - - if (event instanceof ByteBufferedChannelActivateEvent) { - this.isReceiverRunning = true; - this.outputGateContext.reportAsynchronousEvent(); - } else { - this.byteBufferedOutputChannel.processEvent(event); - } + processEvent(it.next()); } } @@ -339,13 +341,15 @@ void flushQueuedOutgoingEnvelopes() throws IOException, InterruptedException { @Override public boolean hasDataLeftToTransmit() throws IOException, InterruptedException { + processIncomingEvents(); + if (!this.isReceiverRunning) { return true; } flushQueuedOutgoingEnvelopes(); - return (!this.queuedOutgoingEnvelopes.isEmpty()); + return (!this.closeAcknowledgementReceived); } long getAmountOfMainMemoryInQueue() { @@ -370,4 +374,24 @@ long spillQueueWithOutgoingEnvelopes() throws IOException { return this.queuedOutgoingEnvelopes.spillSynchronouslyIncludingHead(); } + + /** + * Processes all queues incoming events. + */ + private void processIncomingEvents() { + + AbstractEvent event = this.incomingEventQueue.poll(); + while (event != null) { + + if (event instanceof ByteBufferedChannelCloseEvent) { + this.closeAcknowledgementReceived = true; + } else if (event instanceof ByteBufferedChannelActivateEvent) { + this.isReceiverRunning = true; + } else { + this.byteBufferedOutputChannel.processEvent(event); + } + + event = this.incomingEventQueue.poll(); + } + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java index 36b22e69f9822..374e382a8be75 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java @@ -60,7 +60,7 @@ AbstractID getFileOwnerID() { return this.taskContext.getFileOwnerID(); } - private long spillQueueWithLargestAmountOfMainMemory() { + private long spillQueueWithLargestAmountOfMainMemory() throws IOException { if (this.inactiveOutputChannels.isEmpty()) { return 0L; @@ -83,11 +83,7 @@ private long spillQueueWithLargestAmountOfMainMemory() { } if (maxContext != null) { - try { - return maxContext.spillQueueWithOutgoingEnvelopes(); - } catch (IOException ioe) { - maxContext.reportIOException(ioe); - } + return maxContext.spillQueueWithOutgoingEnvelopes(); } return 0L; From c28df98fb80969c507fc1eeb0495e5ee545ee5b5 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 28 Jan 2012 22:28:16 +0100 Subject: [PATCH 179/310] Added event type to report transfer envelopes with unexpected sequence numbers --- .../bytebuffered/InputChannelContext.java | 14 ++- .../bytebuffered/UnexpectedEnvelopeEvent.java | 89 +++++++++++++++++++ 2 files changed, 100 insertions(+), 3 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java index 1a0c65a185fdd..15ef4d56ca717 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java @@ -34,6 +34,7 @@ import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; +import eu.stratosphere.nephele.util.StringUtils; final class InputChannelContext implements ChannelContext, ByteBufferedInputChannelBroker, BufferProvider { @@ -169,16 +170,23 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { if (sequenceNumber != expectedSequenceNumber) { if (sequenceNumber > expectedSequenceNumber) { + // This is a problem, now we are actually missing some data this.byteBufferedInputChannel.reportIOException(new IOException("Expected data packet " + expectedSequenceNumber + " but received " + sequenceNumber)); this.byteBufferedInputChannel.checkForNetworkEvents(); + } else { + + // Tell the sender that we are expecting an envelope with a higher sequence number + try { + transferEventToOutputChannel(new UnexpectedEnvelopeEvent(expectedSequenceNumber)); + } catch (Exception e) { + LOG.error(StringUtils.stringifyException(e)); + } } - if (LOG.isDebugEnabled()) { - LOG.info("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber + LOG.warn("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber + " but received " + sequenceNumber); - } final Buffer buffer = transferEnvelope.getBuffer(); if (buffer != null) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.java new file mode 100644 index 0000000000000..6954f68327853 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.java @@ -0,0 +1,89 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.bytebuffered; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +/** + * This event is sent by an {@link InputChannelContext}. It indicates that the input channel context has received a + * {@link TransferEnvelope} with a lower sequence number than expected. The typical reason for this is that data is + * being replayed from a checkpoint. With the help of this event it is possible to request the sender to skip sending + * transfer envelopes up to the given expected sequence number. + * + * @author warneke + */ +public final class UnexpectedEnvelopeEvent extends AbstractEvent { + + /** + * The expected sequence number. + */ + private int expectedSequenceNumber; + + /** + * Constructs a new unexpected envelope event. + * + * @param expectedSequenceNumber + * the expected sequence number + */ + public UnexpectedEnvelopeEvent(final int expectedSequenceNumber) { + + if (expectedSequenceNumber < 0) { + throw new IllegalArgumentException("Argument expectedSequenceNumber must be non-negative."); + } + + this.expectedSequenceNumber = expectedSequenceNumber; + } + + /** + * Default constructor for serialization/deserialization. + */ + public UnexpectedEnvelopeEvent() { + } + + /** + * Returns the expected sequence number. + * + * @return the expected sequence number + */ + public int getExpectedSequenceNumber() { + + return this.expectedSequenceNumber; + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + + out.writeInt(this.expectedSequenceNumber); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + + this.expectedSequenceNumber = in.readInt(); + } + +} From 6c1c231547329558e92d9a9000ca938767732d07 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 29 Jan 2012 15:34:21 +0100 Subject: [PATCH 180/310] Simplified hand-over of transfer envelopes from tasks to the byte-buffered channel manager --- .../checkpointing/CheckpointReplayTask.java | 2 - .../ByteBufferedChannelManager.java | 171 +++++++++++------- .../TransferEnvelopeDispatcher.java | 10 +- 3 files changed, 104 insertions(+), 79 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java index 0a219f020564c..738da7558954e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java @@ -145,8 +145,6 @@ private void replayCheckpoint() throws IOException { break; } } - } catch (InterruptedException e) { - // Ignore exception } finally { if (fis != null) { fis.close(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 3081eaaf6d2bc..b74f38101661c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -279,15 +279,26 @@ public NetworkConnectionManager getNetworkConnectionManager() { return this.networkConnectionManager; } - private void processEnvelope(final TransferEnvelope transferEnvelope, final boolean freeSourceBuffer) - throws IOException, InterruptedException { + private void recycleBuffer(final TransferEnvelope envelope) { + + final Buffer buffer = envelope.getBuffer(); + if (buffer != null) { + buffer.recycleBuffer(); + } + } + + private void sendReceiverNotFoundEvent(final ChannelID originalSender, final ChannelID unknownReceiver) { + + } + + private void processEnvelope(final TransferEnvelope transferEnvelope, final boolean freeSourceBuffer) { final TransferEnvelopeReceiverList receiverList = getReceiverList(transferEnvelope.getJobID(), transferEnvelope.getSource()); if (receiverList == null) { - throw new IOException("Transfer envelope " + transferEnvelope.getSequenceNumber() + " from source channel " - + transferEnvelope.getSource() + " has not have a receiver list"); + recycleBuffer(transferEnvelope); + return; } // This envelope is known to have either no buffer or an memory-based input buffer @@ -299,15 +310,14 @@ private void processEnvelope(final TransferEnvelope transferEnvelope, final bool } private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, - final TransferEnvelopeReceiverList receiverList, final boolean freeSourceBuffer) - throws IOException, InterruptedException { + final TransferEnvelopeReceiverList receiverList, final boolean freeSourceBuffer) { // Handle the most common (unicast) case first if (!freeSourceBuffer) { final List localReceivers = receiverList.getLocalReceivers(); if (localReceivers.size() != 1) { - throw new IOException("Expected receiver list to have exactly one element"); + LOG.error("Expected receiver list to have exactly one element"); } final ChannelID localReceiver = localReceivers.get(0); @@ -315,17 +325,17 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, final ChannelContext cc = this.registeredChannels.get(localReceiver); if (cc == null) { - if (this.recentlyRemovedChannelIDSet.contains(localReceiver)) { - // TODO: Reconsider buffer recycling here - return; - } else { - throw new IOException("Cannot find channel context for local receiver " + localReceiver); + if (!this.recentlyRemovedChannelIDSet.contains(localReceiver)) { + sendReceiverNotFoundEvent(transferEnvelope.getSource(), localReceiver); } + + recycleBuffer(transferEnvelope); + return; } if (!cc.isInputChannel()) { - throw new IOException("Local receiver " + localReceiver - + " is not an input channel, but is supposed to accept a buffer"); + LOG.error("Local receiver " + localReceiver + + " is not an input channel, but is supposed to accept a buffer"); } cc.queueTransferEnvelope(transferEnvelope); @@ -345,21 +355,32 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, final ChannelContext cc = this.registeredChannels.get(localReceiver); if (cc == null) { - if (this.recentlyRemovedChannelIDSet.contains(localReceiver)) { - continue; - } else { - throw new IOException("Cannot find channel context for local receiver " + localReceiver); + if (!this.recentlyRemovedChannelIDSet.contains(localReceiver)) { + sendReceiverNotFoundEvent(transferEnvelope.getSource(), localReceiver); } + + continue; } if (!cc.isInputChannel()) { - throw new IOException("Local receiver " + localReceiver - + " is not an input channel, but is supposed to accept a buffer"); + LOG.error("Local receiver " + localReceiver + + " is not an input channel, but is supposed to accept a buffer"); + continue; } final InputChannelContext inputChannelContext = (InputChannelContext) cc; - final Buffer destBuffer = inputChannelContext.requestEmptyBufferBlocking(srcBuffer.size()); - srcBuffer.copyToBuffer(destBuffer); + + Buffer destBuffer = null; + try { + destBuffer = inputChannelContext.requestEmptyBufferBlocking(srcBuffer.size()); + srcBuffer.copyToBuffer(destBuffer); + } catch (Exception e) { + LOG.error(StringUtils.stringifyException(e)); + if (destBuffer != null) { + destBuffer.recycleBuffer(); + } + continue; + } // TODO: See if we can save one duplicate step here final TransferEnvelope dup = transferEnvelope.duplicateWithoutBuffer(); dup.setBuffer(destBuffer); @@ -372,7 +393,18 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, final List remoteReceivers = receiverList.getRemoteReceivers(); for (final InetSocketAddress remoteReceiver : remoteReceivers) { - this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, transferEnvelope.duplicate()); + TransferEnvelope dup = null; + try { + dup = transferEnvelope.duplicate(); + } catch (Exception e) { + LOG.error(StringUtils.stringifyException(e)); + if (dup != null) { + recycleBuffer(dup); + continue; + } + } + + this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, dup); } } @@ -421,62 +453,68 @@ private void addReceiverListHint(final AbstractChannel channel) { } } - private TransferEnvelopeReceiverList getReceiverList(final JobID jobID, final ChannelID sourceChannelID) - throws IOException, InterruptedException { + private TransferEnvelopeReceiverList getReceiverList(final JobID jobID, final ChannelID sourceChannelID) { TransferEnvelopeReceiverList receiverList = this.receiverCache.get(sourceChannelID); + if (receiverList == null) { - while (true) { + try { + while (true) { - final ConnectionInfoLookupResponse lookupResponse = this.channelLookupService.lookupConnectionInfo( + final ConnectionInfoLookupResponse lookupResponse = this.channelLookupService.lookupConnectionInfo( this.localConnectionInfo, jobID, sourceChannelID); - if (lookupResponse.receiverNotFound()) { - throw new IOException("Cannot find task(s) waiting for data from source channel with ID " + if (lookupResponse.receiverNotFound()) { + throw new IOException("Cannot find task(s) waiting for data from source channel with ID " + sourceChannelID); - } + } - if (lookupResponse.receiverNotReady()) { - Thread.sleep(500); - continue; - } + if (lookupResponse.receiverNotReady()) { + Thread.sleep(500); + continue; + } - if (lookupResponse.receiverReady()) { - receiverList = new TransferEnvelopeReceiverList(lookupResponse); - break; + if (lookupResponse.receiverReady()) { + receiverList = new TransferEnvelopeReceiverList(lookupResponse); + break; + } } - } - if (receiverList == null) { - LOG.error("Receiver list is null for source channel ID " + sourceChannelID); - } else { - this.receiverCache.put(sourceChannelID, receiverList); + if (receiverList == null) { + LOG.error("Receiver list is null for source channel ID " + sourceChannelID); + } else { + this.receiverCache.put(sourceChannelID, receiverList); - if (LOG.isDebugEnabled()) { + if (LOG.isDebugEnabled()) { - final StringBuilder sb = new StringBuilder(); - sb.append("Receiver list for source channel ID " + sourceChannelID + " at task manager " - + this.localConnectionInfo + "\n"); + final StringBuilder sb = new StringBuilder(); + sb.append("Receiver list for source channel ID " + sourceChannelID + " at task manager " + + this.localConnectionInfo + "\n"); - if (receiverList.hasLocalReceivers()) { - sb.append("\tLocal receivers:\n"); - final Iterator it = receiverList.getLocalReceivers().iterator(); - while (it.hasNext()) { - sb.append("\t\t" + it.next() + "\n"); + if (receiverList.hasLocalReceivers()) { + sb.append("\tLocal receivers:\n"); + final Iterator it = receiverList.getLocalReceivers().iterator(); + while (it.hasNext()) { + sb.append("\t\t" + it.next() + "\n"); + } } - } - if (receiverList.hasRemoteReceivers()) { - sb.append("Remote receivers:\n"); - final Iterator it = receiverList.getRemoteReceivers().iterator(); - while (it.hasNext()) { - sb.append("\t\t" + it.next() + "\n"); + if (receiverList.hasRemoteReceivers()) { + sb.append("Remote receivers:\n"); + final Iterator it = receiverList.getRemoteReceivers().iterator(); + while (it.hasNext()) { + sb.append("\t\t" + it.next() + "\n"); + } } - } - LOG.debug(sb.toString()); + LOG.debug(sb.toString()); + } } + } catch (InterruptedException ie) { + // TODO: Send appropriate notifications here + } catch (IOException ioe) { + // TODO: Send appropriate notifications here } } @@ -487,8 +525,7 @@ private TransferEnvelopeReceiverList getReceiverList(final JobID jobID, final Ch * {@inheritDoc} */ @Override - public void processEnvelopeFromOutputChannel(final TransferEnvelope transferEnvelope) throws IOException, - InterruptedException { + public void processEnvelopeFromOutputChannel(final TransferEnvelope transferEnvelope) { processEnvelope(transferEnvelope, true); } @@ -497,8 +534,7 @@ public void processEnvelopeFromOutputChannel(final TransferEnvelope transferEnve * {@inheritDoc} */ @Override - public void processEnvelopeFromInputChannel(final TransferEnvelope transferEnvelope) throws IOException, - InterruptedException { + public void processEnvelopeFromInputChannel(final TransferEnvelope transferEnvelope) { processEnvelope(transferEnvelope, false); } @@ -507,14 +543,9 @@ public void processEnvelopeFromInputChannel(final TransferEnvelope transferEnvel * {@inheritDoc} */ @Override - public void processEnvelopeFromNetwork(final TransferEnvelope transferEnvelope, boolean freeSourceBuffer) - throws IOException { + public void processEnvelopeFromNetwork(final TransferEnvelope transferEnvelope, boolean freeSourceBuffer) { - try { - processEnvelope(transferEnvelope, freeSourceBuffer); - } catch (InterruptedException e) { - LOG.error("Caught unexpected interrupted exception: " + StringUtils.stringifyException(e)); - } + processEnvelope(transferEnvelope, freeSourceBuffer); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.java index 19e1953d3c449..495872b2d1fd7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.java @@ -33,16 +33,12 @@ public interface TransferEnvelopeDispatcher { * * @param transferEnvelope * the transfer envelope to be processed - * @throws IOException - * thrown if an I/O error occurs while allocating the resources to further process the envelope - * @throws InterruptedException - * thrown if the thread is interrupted while waiting for the processing to complete */ - void processEnvelopeFromOutputChannel(TransferEnvelope transferEnvelope) throws IOException, InterruptedException; + void processEnvelopeFromOutputChannel(TransferEnvelope transferEnvelope); - void processEnvelopeFromInputChannel(TransferEnvelope transferEnvelope) throws IOException, InterruptedException; + void processEnvelopeFromInputChannel(TransferEnvelope transferEnvelope); - void processEnvelopeFromNetwork(TransferEnvelope transferEnvelope, boolean freeSourceBuffer) throws IOException; + void processEnvelopeFromNetwork(TransferEnvelope transferEnvelope, boolean freeSourceBuffer); /** * Registers the given spilling queue with a network connection. The network connection is in charge of polling the From d498b59e9171fbdd6c916d392f03d5e56b17b19a Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 29 Jan 2012 17:00:08 +0100 Subject: [PATCH 181/310] Introduced unknown receiver events to signalize delivery failures to the sender --- .../stratosphere/nephele/io/AbstractID.java | 6 +- .../nephele/io/channels/ChannelID.java | 22 +++++ .../ByteBufferedChannelManager.java | 24 ++++- .../bytebuffered/UnknownReceiverEvent.java | 89 +++++++++++++++++++ 4 files changed, 135 insertions(+), 6 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnknownReceiverEvent.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java index 5e7438b94d97c..1eb6fb7cafddb 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/AbstractID.java @@ -44,9 +44,11 @@ public abstract class AbstractID implements IOReadableWritable { */ public AbstractID(final byte[] bytes) { - if (bytes.length == SIZE) { - System.arraycopy(bytes, 0, this.bytes, 0, SIZE); + if (bytes.length != SIZE) { + throw new IllegalArgumentException("Argument bytes must by an array of " + SIZE + " bytes"); } + + System.arraycopy(bytes, 0, this.bytes, 0, SIZE); } /** diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java index f83fd3dea6f67..699d8065c4ae8 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java @@ -24,4 +24,26 @@ */ public class ChannelID extends AbstractID { + /** + * Constructs a new channel ID from the provided byte array. + * + * @param bytes + * the byte array to construct the channel ID from + */ + private ChannelID(final byte[] bytes) { + super(bytes); + } + + /** + * Constructs a new, random channel ID. + */ + public ChannelID() { + super(); + } + + /** + * The channel ID used by the framework to send notifications about problems with the data delivery. + */ + public static final ChannelID SYSTEM_ID = new ChannelID(new byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 }); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index b74f38101661c..0d72ecf408bdb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -287,8 +287,24 @@ private void recycleBuffer(final TransferEnvelope envelope) { } } - private void sendReceiverNotFoundEvent(final ChannelID originalSender, final ChannelID unknownReceiver) { + private void sendReceiverNotFoundEvent(final JobID jobID, final ChannelID unknownReceiver) { + if (ChannelID.SYSTEM_ID.equals(unknownReceiver)) { + LOG.error("Requested to send unknown receiver event from the system, dropping request..."); + return; + } + + final TransferEnvelope transferEnvelope = new TransferEnvelope(0, jobID, ChannelID.SYSTEM_ID); + final UnknownReceiverEvent unknownReceiverEvent = new UnknownReceiverEvent(unknownReceiver); + transferEnvelope.addEvent(unknownReceiverEvent); + + final TransferEnvelopeReceiverList receiverList = getReceiverList(jobID, unknownReceiver); + if (receiverList == null) { + LOG.error("Cannot determine receiver list for source channel ID " + unknownReceiver); + return; + } + + processEnvelopeEnvelopeWithoutBuffer(transferEnvelope, receiverList); } private void processEnvelope(final TransferEnvelope transferEnvelope, final boolean freeSourceBuffer) { @@ -326,7 +342,7 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, if (cc == null) { if (!this.recentlyRemovedChannelIDSet.contains(localReceiver)) { - sendReceiverNotFoundEvent(transferEnvelope.getSource(), localReceiver); + sendReceiverNotFoundEvent(transferEnvelope.getJobID(), localReceiver); } recycleBuffer(transferEnvelope); @@ -356,7 +372,7 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, if (cc == null) { if (!this.recentlyRemovedChannelIDSet.contains(localReceiver)) { - sendReceiverNotFoundEvent(transferEnvelope.getSource(), localReceiver); + sendReceiverNotFoundEvent(transferEnvelope.getJobID(), localReceiver); } continue; @@ -456,7 +472,7 @@ private void addReceiverListHint(final AbstractChannel channel) { private TransferEnvelopeReceiverList getReceiverList(final JobID jobID, final ChannelID sourceChannelID) { TransferEnvelopeReceiverList receiverList = this.receiverCache.get(sourceChannelID); - + if (receiverList == null) { try { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnknownReceiverEvent.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnknownReceiverEvent.java new file mode 100644 index 0000000000000..ec5893fb9a510 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnknownReceiverEvent.java @@ -0,0 +1,89 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.bytebuffered; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.io.channels.ChannelID; + +/** + * An unknown receiver event can be used by the framework to inform a sender task that the delivery of a + * {@link TransferEnvelope} has failed since the receiver could not be found. + * + * @author warneke + */ +public final class UnknownReceiverEvent extends AbstractEvent { + + /** + * The ID of the unknown receiver. + */ + private ChannelID unknownReceiverID; + + /** + * Constructs a new unknown receiver event. + * + * @param unknownReceiverID + * the ID of the unknown receiver + */ + public UnknownReceiverEvent(final ChannelID unknownReceiverID) { + + if (unknownReceiverID == null) { + throw new IllegalArgumentException("Argument unknownReceiverID must not be null"); + } + + this.unknownReceiverID = unknownReceiverID; + } + + /** + * Default constructor for serialization/deserialization. + */ + public UnknownReceiverEvent() { + + this.unknownReceiverID = new ChannelID(); + } + + /** + * Returns the ID of the unknown receiver. + * + * @return the ID of the unknown receiver + */ + public ChannelID getUnknownReceiverID() { + + return this.unknownReceiverID; + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + + this.unknownReceiverID.write(out); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + + this.unknownReceiverID.read(in); + } + +} From a41aa99109014d6b24d23fdd7e42dbd6eb9edf37 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 29 Jan 2012 17:13:01 +0100 Subject: [PATCH 182/310] Fixed problems with cache invalidation in the recovery logic --- .../nephele/jobmanager/scheduler/RecoveryLogic.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index 905453660171e..1f6c39a98e217 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -201,7 +201,8 @@ private static final boolean invalidateReceiverLookupCaches(final ExecutionVerte continue; } - final ExecutionVertex connectedVertex = eg.getVertexByChannelID(outputChannel.getConnectedChannelID()); + final ChannelID connectedChannelID = outputChannel.getConnectedChannelID(); + final ExecutionVertex connectedVertex = eg.getVertexByChannelID(connectedChannelID); if (connectedVertex == null) { LOG.error("Connected vertex is null"); continue; @@ -219,7 +220,7 @@ private static final boolean invalidateReceiverLookupCaches(final ExecutionVerte entriesToInvalidate.put(instance, channelIDs); } - channelIDs.add(outputChannel.getID()); + channelIDs.add(connectedChannelID); } } @@ -234,7 +235,8 @@ private static final boolean invalidateReceiverLookupCaches(final ExecutionVerte continue; } - final ExecutionVertex connectedVertex = eg.getVertexByChannelID(inputChannel.getConnectedChannelID()); + final ChannelID connectedChannelID = inputChannel.getConnectedChannelID(); + final ExecutionVertex connectedVertex = eg.getVertexByChannelID(connectedChannelID); if (connectedVertex == null) { LOG.error("Connected vertex is null"); continue; @@ -252,7 +254,7 @@ private static final boolean invalidateReceiverLookupCaches(final ExecutionVerte entriesToInvalidate.put(instance, channelIDs); } - channelIDs.add(inputChannel.getID()); + channelIDs.add(connectedChannelID); } } From 663bde8c52e3cbab67bcc71b1ddb74868ad6fd25 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 29 Jan 2012 17:45:32 +0100 Subject: [PATCH 183/310] Clean-up after merge --- .../nephele/execution/Environment.java | 2 - .../nephele/types/AbstractTaggableRecord.java | 76 ----- .../eu/stratosphere/nephele/types/Tag.java | 28 -- .../nephele/execution/RuntimeEnvironment.java | 7 - nephele/nephele-streaming/pom.xml | 27 -- .../StreamingCommunicationThread.java | 118 ------- .../streaming/StreamingJobManagerPlugin.java | 295 ----------------- .../streaming/StreamingPluginLoader.java | 102 ------ .../nephele/streaming/StreamingTag.java | 81 ----- .../streaming/StreamingTaskManagerPlugin.java | 201 ------------ .../streaming/actions/AbstractAction.java | 95 ------ .../actions/ConstructStreamChainAction.java | 90 ------ .../actions/LimitBufferSizeAction.java | 147 --------- .../streaming/buffers/BufferSizeHistory.java | 55 ---- .../buffers/BufferSizeHistoryEntry.java | 37 --- .../streaming/buffers/BufferSizeManager.java | 240 -------------- .../streaming/chaining/RecordUtils.java | 66 ---- .../streaming/chaining/StreamChain.java | 86 ----- .../chaining/StreamChainCoordinator.java | 73 ----- .../streaming/chaining/StreamChainLink.java | 53 ---- .../streaming/listeners/StreamListener.java | 299 ------------------ .../listeners/StreamListenerContext.java | 181 ----------- .../profiling/EdgeCharacteristics.java | 72 ----- .../profiling/LatencyOptimizerThread.java | 91 ------ .../streaming/profiling/ProfilingLogger.java | 105 ------ .../streaming/profiling/ProfilingModel.java | 96 ------ .../streaming/profiling/ProfilingPath.java | 193 ----------- .../profiling/ProfilingPathSummary.java | 136 -------- .../profiling/ProfilingSubgraph.java | 178 ----------- .../streaming/profiling/ProfilingSummary.java | 124 -------- .../streaming/profiling/ProfilingUtils.java | 52 --- .../streaming/profiling/ProfilingValue.java | 67 ---- .../profiling/ProfilingValueStatistic.java | 119 ------- .../streaming/profiling/VertexLatency.java | 37 --- .../streaming/profiling/XoredVertexID.java | 40 --- .../types/AbstractStreamingData.java | 79 ----- .../streaming/types/ChannelLatency.java | 154 --------- .../streaming/types/ChannelThroughput.java | 144 --------- .../streaming/types/OutputBufferLatency.java | 102 ------ .../nephele/streaming/types/TaskLatency.java | 112 ------- .../wrappers/StreamingEnvironment.java | 114 ------- .../wrappers/StreamingFileInputWrapper.java | 61 ---- .../wrappers/StreamingFileOutputWrapper.java | 61 ---- .../wrappers/StreamingInputGate.java | 171 ---------- .../wrappers/StreamingInputWrapper.java | 61 ---- .../wrappers/StreamingOutputGate.java | 159 ---------- .../wrappers/StreamingOutputWrapper.java | 61 ---- .../wrappers/StreamingTaskWrapper.java | 61 ---- .../streaming/wrappers/WrapperUtils.java | 93 ------ .../latency/ProfilingValueStatisticTest.java | 142 --------- nephele/pom.xml | 1 - pom.xml | 1 - score/pom.xml | 66 ---- .../score/ScoreExecutionListener.java | 72 ----- .../score/ScoreJobManagerPlugin.java | 107 ------- .../stratosphere/score/ScorePluginLoader.java | 83 ----- .../score/ScoreTaskManagerPlugin.java | 75 ----- 57 files changed, 5649 deletions(-) delete mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java delete mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/Tag.java delete mode 100644 nephele/nephele-streaming/pom.xml delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/ConstructStreamChainAction.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/LimitBufferSizeAction.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistory.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistoryEntry.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/RecordUtils.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPath.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValue.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/VertexLatency.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/XoredVertexID.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/AbstractStreamingData.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelLatency.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelThroughput.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/TaskLatency.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileInputWrapper.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileOutputWrapper.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputWrapper.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputWrapper.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingTaskWrapper.java delete mode 100644 nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/WrapperUtils.java delete mode 100644 nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java delete mode 100644 score/pom.xml delete mode 100644 score/src/main/java/eu/stratosphere/score/ScoreExecutionListener.java delete mode 100644 score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java delete mode 100644 score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java delete mode 100644 score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java index 892b9cba1914a..efe45c34d876f 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java @@ -184,6 +184,4 @@ InputGate createInputGate(GateID gateID, RecordDeserializer inputGate); - - void registerMapper(Mapper mapper); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java deleted file mode 100644 index b68600bbd64ca..0000000000000 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/AbstractTaggableRecord.java +++ /dev/null @@ -1,76 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.types; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.util.StringUtils; - -public abstract class AbstractTaggableRecord implements Record { - - private Tag tag = null; - - public void setTag(final Tag tag) { - this.tag = tag; - } - - public Tag getTag() { - - return this.tag; - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - - if (this.tag == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - StringRecord.writeString(out, this.tag.getClass().getName()); - this.tag.write(out); - } - } - - /** - * {@inheritDoc} - */ - @SuppressWarnings("unchecked") - @Override - public void read(final DataInput in) throws IOException { - - if (in.readBoolean()) { - final String tagType = StringRecord.readString(in); - Class clazz = null; - try { - clazz = (Class) Class.forName(tagType); - } catch (ClassNotFoundException e) { - throw new IOException(StringUtils.stringifyException(e)); - } - - try { - this.tag = clazz.newInstance(); - } catch (Exception e) { - throw new IOException(StringUtils.stringifyException(e)); - } - this.tag.read(in); - } - } -} diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/Tag.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/Tag.java deleted file mode 100644 index ce74b6af3406b..0000000000000 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/types/Tag.java +++ /dev/null @@ -1,28 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.types; - -import eu.stratosphere.nephele.io.IOReadableWritable; - -/** - * This interface specifies a tag. A tag is a piece of metadata that can be attached to {@link AbstractTaggableRecord} - * objects. - * - * @author warneke - */ -public interface Tag extends IOReadableWritable { - -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 8582ab747dfb2..04ef4a49da909 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -1003,11 +1003,4 @@ private void releaseAllChannelResources() { this.outputGates.get(i).releaseAllChannelResources(); } } - - @Override - public void registerMapper(final Mapper mapper) { - - // Nothing to do here - - } } diff --git a/nephele/nephele-streaming/pom.xml b/nephele/nephele-streaming/pom.xml deleted file mode 100644 index 431b139fb11b7..0000000000000 --- a/nephele/nephele-streaming/pom.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - - 4.0.0 - - - nephele - eu.stratosphere - 0.2 - - - nephele-streaming - nephele-streaming - jar - - - - - eu.stratosphere - nephele-server - ${version} - - - - - diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java deleted file mode 100644 index 55a17c4eb97a2..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingCommunicationThread.java +++ /dev/null @@ -1,118 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming; - -import java.io.IOException; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.plugins.PluginCommunication; -import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; -import eu.stratosphere.nephele.util.StringUtils; - -/** - * This class implements a communication thread to handle communication from the task manager plugin component to the - * job manager plugin component in an asynchronous fashion. The main reason for asynchronous communication is not - * influence the processing delay by the RPC call latency. - *

- * This class is thread-safe. - * - * @author warneke - */ -public final class StreamingCommunicationThread extends Thread { - - /** - * The log object. - */ - private static final Log LOG = LogFactory.getLog(StreamingCommunicationThread.class); - - /** - * The capacity of the data queue. - */ - private static final int QUEUE_CAPACITY = 128; - - /** - * Stub object representing the job manager component of this plugin. - */ - private final PluginCommunication jobManagerComponent; - - /** - * The blocking queue which is used to asynchronously exchange data with the job manager component of this plugin. - */ - private final BlockingQueue dataQueue = new ArrayBlockingQueue( - QUEUE_CAPACITY); - - /** - * Stores whether the communication thread has been requested to stop. - */ - private volatile boolean interrupted = false; - - /** - * Constructs a new streaming communication thread. - * - * @param jobManagerComponent - * the stub object for the plugin's job manager component. - */ - StreamingCommunicationThread(final PluginCommunication jobManagerComponent) { - this.jobManagerComponent = jobManagerComponent; - } - - /** - * {@inheritDoc} - */ - @Override - public void run() { - - while (!this.interrupted) { - - if (Thread.currentThread().isInterrupted()) { - break; - } - - try { - this.jobManagerComponent.sendData(this.dataQueue.take()); - } catch (InterruptedException e) { - break; - } catch (IOException ioe) { - LOG.error(StringUtils.stringifyException(ioe)); - } - } - } - - /** - * Stops the communication thread. - */ - void stopCommunicationThread() { - this.interrupted = true; - interrupt(); - } - - /** - * Sends the given data item asynchronously to the plugin's job manager component. - * - * @param data - * the data item to send to the plugin's job manager component - * @throws InterruptedException - * thrown if the thread is interrupted while waiting for the communication thread to accept the data - */ - public void sendDataAsynchronously(final AbstractStreamingData data) throws InterruptedException { - - this.dataQueue.put(data); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java deleted file mode 100644 index 95ee71be8e0fc..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingJobManagerPlugin.java +++ /dev/null @@ -1,295 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.executiongraph.InternalJobStatus; -import eu.stratosphere.nephele.executiongraph.JobStatusListener; -import eu.stratosphere.nephele.instance.AbstractInstance; -import eu.stratosphere.nephele.io.IOReadableWritable; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.jobgraph.AbstractJobInputVertex; -import eu.stratosphere.nephele.jobgraph.AbstractJobOutputVertex; -import eu.stratosphere.nephele.jobgraph.JobFileInputVertex; -import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex; -import eu.stratosphere.nephele.jobgraph.JobGraph; -import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.jobgraph.JobInputVertex; -import eu.stratosphere.nephele.jobgraph.JobOutputVertex; -import eu.stratosphere.nephele.plugins.JobManagerPlugin; -import eu.stratosphere.nephele.plugins.PluginID; -import eu.stratosphere.nephele.profiling.ProfilingListener; -import eu.stratosphere.nephele.streaming.actions.ConstructStreamChainAction; -import eu.stratosphere.nephele.streaming.actions.LimitBufferSizeAction; -import eu.stratosphere.nephele.streaming.profiling.LatencyOptimizerThread; -import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; -import eu.stratosphere.nephele.streaming.wrappers.StreamingFileInputWrapper; -import eu.stratosphere.nephele.streaming.wrappers.StreamingFileOutputWrapper; -import eu.stratosphere.nephele.streaming.wrappers.StreamingInputWrapper; -import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputWrapper; -import eu.stratosphere.nephele.streaming.wrappers.StreamingTaskWrapper; -import eu.stratosphere.nephele.streaming.wrappers.WrapperUtils; -import eu.stratosphere.nephele.jobgraph.JobTaskVertex; -import eu.stratosphere.nephele.template.AbstractInvokable; -import eu.stratosphere.nephele.util.StringUtils; - -public class StreamingJobManagerPlugin implements JobManagerPlugin, JobStatusListener { - - /** - * The log object. - */ - private static final Log LOG = LogFactory.getLog(StreamingJobManagerPlugin.class); - - private final PluginID pluginID; - - private ConcurrentHashMap latencyOptimizerThreads = new ConcurrentHashMap(); - - StreamingJobManagerPlugin(final PluginID pluginID, final Configuration pluginConfiguration) { - this.pluginID = pluginID; - } - - /** - * {@inheritDoc} - */ - @Override - public JobGraph rewriteJobGraph(final JobGraph jobGraph) { - - // Rewrite input vertices - final Iterator inputIt = jobGraph.getInputVertices(); - while (inputIt.hasNext()) { - - final AbstractJobInputVertex abstractInputVertex = inputIt.next(); - final Class originalClass = abstractInputVertex.getInvokableClass(); - - if (abstractInputVertex instanceof JobFileInputVertex) { - final JobFileInputVertex fileInputVertex = (JobFileInputVertex) abstractInputVertex; - fileInputVertex.setFileInputClass(StreamingFileInputWrapper.class); - } else if (abstractInputVertex instanceof JobInputVertex) { - final JobInputVertex inputVertex = (JobInputVertex) abstractInputVertex; - inputVertex.setInputClass(StreamingInputWrapper.class); - } else { - LOG.warn("Cannot wrap input task of type " + originalClass + ", skipping..."); - continue; - } - - abstractInputVertex.getConfiguration().setString(WrapperUtils.WRAPPED_CLASS_KEY, originalClass.getName()); - } - - // Rewrite the task vertices - final Iterator taskIt = jobGraph.getTaskVertices(); - while (taskIt.hasNext()) { - - final JobTaskVertex taskVertex = taskIt.next(); - final Class originalClass = taskVertex.getInvokableClass(); - taskVertex.setTaskClass(StreamingTaskWrapper.class); - taskVertex.getConfiguration().setString(WrapperUtils.WRAPPED_CLASS_KEY, originalClass.getName()); - } - - // Rewrite the output vertices - final Iterator outputIt = jobGraph.getOutputVertices(); - while (outputIt.hasNext()) { - - final AbstractJobOutputVertex abstractOutputVertex = outputIt.next(); - final Class originalClass = abstractOutputVertex.getInvokableClass(); - - if (abstractOutputVertex instanceof JobFileOutputVertex) { - final JobFileOutputVertex fileOutputVertex = (JobFileOutputVertex) abstractOutputVertex; - fileOutputVertex.setFileOutputClass(StreamingFileOutputWrapper.class); - } else if (abstractOutputVertex instanceof JobOutputVertex) { - final JobOutputVertex outputVertex = (JobOutputVertex) abstractOutputVertex; - outputVertex.setOutputClass(StreamingOutputWrapper.class); - } else { - LOG.warn("Cannot wrap output task of type " + originalClass + ", skipping..."); - continue; - } - - abstractOutputVertex.getConfiguration().setString(WrapperUtils.WRAPPED_CLASS_KEY, originalClass.getName()); - } - - return jobGraph; - } - - /** - * {@inheritDoc} - */ - @Override - public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) { - - JobID jobId = executionGraph.getJobID(); - LatencyOptimizerThread optimizerThread = new LatencyOptimizerThread(this, executionGraph); - latencyOptimizerThreads.put(jobId, optimizerThread); - optimizerThread.start(); - - // Temporary code start - final Runnable run = new Runnable() { - @Override - public void run() { - try { - Thread.sleep(2000); - } catch (InterruptedException e) { - e.printStackTrace(); - return; - } - final Iterator it = new ExecutionGraphIterator(executionGraph, true); - while (it.hasNext()) { - final ExecutionVertex vertex = it.next(); - if (vertex.getName().contains("Decoder")) { - final List vertexIDs = new ArrayList(); - final AbstractInstance instance = vertex.getAllocatedResource().getInstance(); - vertexIDs.add(vertex.getID()); - vertexIDs.add(it.next().getID()); - vertexIDs.add(it.next().getID()); - vertexIDs.add(it.next().getID()); - constructStreamChain(executionGraph.getJobID(), instance, vertexIDs); - } - } - } - }; - //new Thread(run).start(); - // Temporary code end - - return executionGraph; - } - - /** - * {@inheritDoc} - */ - @Override - public void shutdown() { - shutdownLatencyOptimizerThreads(); - } - - private void shutdownLatencyOptimizerThreads() { - Iterator iter = latencyOptimizerThreads.values().iterator(); - while (iter.hasNext()) { - LatencyOptimizerThread thread = iter.next(); - thread.interrupt(); - - // also removes jobID mappings from underlying map - iter.remove(); - } - } - - /** - * {@inheritDoc} - */ - @Override - public void sendData(final IOReadableWritable data) throws IOException { - - if (!(data instanceof AbstractStreamingData)) { - LOG.error("Received unexpected data of type " + data); - return; - } - - AbstractStreamingData streamingData = (AbstractStreamingData) data; - LatencyOptimizerThread optimizerThread = latencyOptimizerThreads.get(streamingData.getJobID()); - optimizerThread.handOffStreamingData(streamingData); - } - - /** - * {@inheritDoc} - */ - @Override - public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { - - if (!(data instanceof AbstractStreamingData)) { - LOG.error("Received unexpected data of type " + data); - return null; - } - - return null; - } - - @Override - public void jobStatusHasChanged(ExecutionGraph executionGraph, - InternalJobStatus newJobStatus, - String optionalMessage) { - - if (newJobStatus == InternalJobStatus.FAILED - || newJobStatus == InternalJobStatus.CANCELED - || newJobStatus == InternalJobStatus.FINISHED) { - - LatencyOptimizerThread optimizerThread = latencyOptimizerThreads.remove(executionGraph.getJobID()); - if (optimizerThread != null) { - optimizerThread.interrupt(); - } - } - } - - public void limitBufferSize(final ExecutionVertex vertex, final ChannelID sourceChannelID, final int bufferSize) { - - final JobID jobID = vertex.getExecutionGraph().getJobID(); - final ExecutionVertexID vertexID = vertex.getID(); - - final AbstractInstance instance = vertex.getAllocatedResource().getInstance(); - if (instance == null) { - LOG.error(vertex + " has no instance assigned"); - return; - } - - final LimitBufferSizeAction bsla = new LimitBufferSizeAction(jobID, vertexID, sourceChannelID, bufferSize); - try { - instance.sendData(this.pluginID, bsla); - } catch (IOException e) { - LOG.error(StringUtils.stringifyException(e)); - } - } - - public void constructStreamChain(final JobID jobID, final AbstractInstance instance, - final List vertexIDs) { - - final ConstructStreamChainAction csca = new ConstructStreamChainAction(jobID, vertexIDs); - try { - instance.sendData(this.pluginID, csca); - } catch (IOException e) { - LOG.error(StringUtils.stringifyException(e)); - } - } - - /** - * {@inheritDoc} - */ - @Override - public boolean requiresProfiling() { - - return true; - } - - /** - * {@inheritDoc} - */ - @Override - public ProfilingListener getProfilingListener(final JobID jobID) { - - System.out.println("REGISTERED PROFILING LISTENER"); - - return null; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java deleted file mode 100644 index 77a3e1ef140e1..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingPluginLoader.java +++ /dev/null @@ -1,102 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming; - -import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.plugins.AbstractPluginLoader; -import eu.stratosphere.nephele.plugins.JobManagerPlugin; -import eu.stratosphere.nephele.plugins.PluginID; -import eu.stratosphere.nephele.plugins.PluginLookupService; -import eu.stratosphere.nephele.plugins.TaskManagerPlugin; - -/** - * This class implements the loader functionality for the Nephele streaming plugin. - *

- * This class is thread-safe. - * - * @author warneke - */ -public final class StreamingPluginLoader extends AbstractPluginLoader { - - /** - * The job manager component of this plugin. - */ - private StreamingJobManagerPlugin jobManagerPlugin = null; - - /** - * The task manager component of this plugin. - */ - private StreamingTaskManagerPlugin taskManagerPlugin = null; - - /** - * The ID of this plugin. - */ - private final PluginID pluginID; - - /** - * Constructs a loader for the Nephele streaming plugin. - * - * @param pluginName - * the name of the plugin as specified in the plugin configuration file - * @param pluginConfiguration - * the configuration of this plugin - * @param pluginLookupService - * the lookup service to locate the remote components of this plugin - */ - public StreamingPluginLoader(final String pluginName, final Configuration pluginConfiguration, - final PluginLookupService pluginLookupService) { - super(pluginName, pluginConfiguration, pluginLookupService); - - this.pluginID = PluginID.fromByteArray(new byte[] { 0x3c, 0x00, 0x00, -0x1b, 0x38, 0x4a, 0x60, -0x61, -0x25, - 0x00, 0x00, 0x16, 0x00, 0x18, 0x7f, 0x01 }); - } - - /** - * {@inheritDoc} - */ - @Override - public synchronized JobManagerPlugin getJobManagerPlugin() { - - if (this.jobManagerPlugin == null) { - this.jobManagerPlugin = new StreamingJobManagerPlugin(this.pluginID, getPluginConfiguration()); - } - - return this.jobManagerPlugin; - } - - /** - * {@inheritDoc} - */ - @Override - public synchronized TaskManagerPlugin getTaskManagerPlugin() { - - if (this.taskManagerPlugin == null) { - this.taskManagerPlugin = new StreamingTaskManagerPlugin(getPluginConfiguration(), getPluginLookupService() - .getJobManagerComponent(getPluginID())); - } - - return this.taskManagerPlugin; - } - - /** - * {@inheritDoc} - */ - @Override - public PluginID getPluginID() { - - return this.pluginID; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java deleted file mode 100644 index a3f91eb915269..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTag.java +++ /dev/null @@ -1,81 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.types.Tag; - -public final class StreamingTag implements Tag { - - private final ExecutionVertexID sourceID; - - private long timestamp = 0L; - - public StreamingTag(final ExecutionVertexID sourceID) { - - if (sourceID == null) { - throw new IllegalArgumentException("sourceID must not be null"); - } - - this.sourceID = sourceID; - } - - /** - * Default constructor for deserialization. - */ - public StreamingTag() { - this.sourceID = new ExecutionVertexID(); - } - - public void setTimestamp(final long timestamp) { - this.timestamp = timestamp; - } - - public ExecutionVertexID getSourceID() { - - return this.sourceID; - } - - public long getTimestamp() { - - return this.timestamp; - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - // TODO Auto-generated method stub - - this.sourceID.write(out); - out.writeLong(this.timestamp); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - - this.sourceID.read(in); - this.timestamp = in.readLong(); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java deleted file mode 100644 index af286b8722fc4..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/StreamingTaskManagerPlugin.java +++ /dev/null @@ -1,201 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming; - -import java.io.IOException; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.RuntimeEnvironment; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.IOReadableWritable; -import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.plugins.PluginCommunication; -import eu.stratosphere.nephele.plugins.TaskManagerPlugin; -import eu.stratosphere.nephele.streaming.actions.AbstractAction; -import eu.stratosphere.nephele.streaming.chaining.StreamChainCoordinator; -import eu.stratosphere.nephele.streaming.listeners.StreamListenerContext; - -public class StreamingTaskManagerPlugin implements TaskManagerPlugin { - - /** - * The log object. - */ - private static final Log LOG = LogFactory.getLog(StreamingTaskManagerPlugin.class); - - /** - * Provides access to the configuration entry which defines the interval in which records shall be tagged. - */ - private static final String TAGGING_INTERVAL_KEY = "streaming.tagging.interval"; - - /** - * The default tagging interval. - */ - private static final int DEFAULT_TAGGING_INTERVAL = 10; - - /** - * Provides access to the configuration entry which defines the interval in which received tags shall be aggregated - * and sent to the job manager plugin component. - */ - private static final String AGGREGATION_INTERVAL_KEY = "streaming.aggregation.interval"; - - /** - * The default aggregation interval. - */ - private static final int DEFAULT_AGGREGATION_INTERVAL = 10; - - /** - * Stores the instance of the streaming task manager plugin. - */ - private static volatile StreamingTaskManagerPlugin INSTANCE = null; - - /** - * Map storing the listener context objects for the individual stream listners. - */ - private final ConcurrentMap listenerContexts = new ConcurrentHashMap(); - - /** - * The tagging interval as specified in the plugin configuration. - */ - private final int taggingInterval; - - /** - * The aggregation interval as specified in the plugin configuration. - */ - private final int aggregationInterval; - - /** - * A special thread to asynchronously send data to the job manager component without suffering from the RPC latency. - */ - private final StreamingCommunicationThread communicationThread; - - private final StreamChainCoordinator chainCoordinator; - - StreamingTaskManagerPlugin(final Configuration pluginConfiguration, final PluginCommunication jobManagerComponent) { - - this.taggingInterval = pluginConfiguration.getInteger(TAGGING_INTERVAL_KEY, DEFAULT_TAGGING_INTERVAL); - this.aggregationInterval = pluginConfiguration.getInteger(AGGREGATION_INTERVAL_KEY, - DEFAULT_AGGREGATION_INTERVAL); - - this.communicationThread = new StreamingCommunicationThread(jobManagerComponent); - this.communicationThread.start(); - - this.chainCoordinator = new StreamChainCoordinator(); - - LOG.info("Configured tagging interval is " + this.taggingInterval); - - INSTANCE = this; - } - - public static StreamListenerContext getStreamingListenerContext(final String listenerKey) { - - if (INSTANCE == null) { - throw new IllegalStateException("StreamingTaskManagerPlugin has not been initialized"); - } - - return INSTANCE.listenerContexts.get(listenerKey); - } - - /** - * {@inheritDoc} - */ - @Override - public void shutdown() { - - this.communicationThread.stopCommunicationThread(); - } - - /** - * {@inheritDoc} - */ - @Override - public void registerTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final RuntimeEnvironment environment) { - - // Check if user has provided a job-specific aggregation interval - final int aggregationInterval = jobConfiguration.getInteger(AGGREGATION_INTERVAL_KEY, - this.aggregationInterval); - - final int taggingInterval = jobConfiguration.getInteger(TAGGING_INTERVAL_KEY, this.taggingInterval); - - final String idAsString = id.toString(); - - environment.getTaskConfiguration().setString(StreamListenerContext.CONTEXT_CONFIGURATION_KEY, idAsString); - - final JobID jobID = environment.getJobID(); - StreamListenerContext listenerContext = null; - if (environment.getNumberOfInputGates() == 0) { - listenerContext = StreamListenerContext.createForInputTask(jobID, id, this.communicationThread, - this.chainCoordinator, aggregationInterval, taggingInterval); - } else if (environment.getNumberOfOutputGates() == 0) { - listenerContext = StreamListenerContext.createForOutputTask(jobID, id, this.communicationThread, - this.chainCoordinator, aggregationInterval); - } else { - listenerContext = StreamListenerContext.createForRegularTask(jobID, id, this.communicationThread, - this.chainCoordinator, aggregationInterval); - } - - this.listenerContexts.putIfAbsent(idAsString, listenerContext); - } - - /** - * {@inheritDoc} - */ - @Override - public void unregisterTask(final ExecutionVertexID id, final RuntimeEnvironment environment) { - - this.listenerContexts.remove(id.toString()); - } - - /** - * {@inheritDoc} - */ - @Override - public void sendData(final IOReadableWritable data) throws IOException { - - if (!(data instanceof AbstractAction)) { - LOG.error("Received data is of unknown type " + data.getClass()); - return; - } - - final AbstractAction action = (AbstractAction) data; - final StreamListenerContext listenerContext = this.listenerContexts.get(action.getVertexID().toString()); - - if (listenerContext == null) { - LOG.error("Cannot find listener context for vertex with ID " + action.getVertexID()); - return; - } - - // Queue the action and return - listenerContext.queuePendingAction(action); - } - - /** - * {@inheritDoc} - */ - @Override - public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { - - // TODO Implement me - - return null; - } - -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java deleted file mode 100644 index 67b1f9bb4459d..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/AbstractAction.java +++ /dev/null @@ -1,95 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.actions; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.IOReadableWritable; -import eu.stratosphere.nephele.jobgraph.JobID; - -/** - * This class implements an abstract base class for actions the job manager component of the Nephele streaming plugin - * can initiate to achieve particular latency or throughput goals. - * - * @author warneke - */ -public abstract class AbstractAction implements IOReadableWritable { - - /** - * The ID of the job the initiated action applies to. - */ - private final JobID jobID; - - /** - * Constructs a new abstract action object. - * - * @param jobID - * the ID of the job the initiated action applies to - */ - AbstractAction(final JobID jobID) { - - if (jobID == null) { - throw new IllegalArgumentException("Argument jobID must not be null"); - } - - this.jobID = jobID; - } - - /** - * Default constructor required for deserialization. - */ - AbstractAction() { - this.jobID = new JobID(); - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - - this.jobID.write(out); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - - this.jobID.read(in); - } - - /** - * Returns the ID of the job the initiated action applies to. - * - * @return the ID of the job the initiated action applies to - */ - public JobID getJobID() { - - return this.jobID; - } - - /** - * Returns the ID of the vertex the initiated action applies to. - * - * @return the ID of the vertex the initiated action applies to - */ - public abstract ExecutionVertexID getVertexID(); -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/ConstructStreamChainAction.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/ConstructStreamChainAction.java deleted file mode 100644 index 13d6629fe511f..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/ConstructStreamChainAction.java +++ /dev/null @@ -1,90 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.actions; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.Collections; -import java.util.List; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.util.SerializableArrayList; - -/** - * This class implements an action to construct a stream chain for a particular sub-path of the graph. - * - * @author warneke - */ -public final class ConstructStreamChainAction extends AbstractAction { - - private final SerializableArrayList vertexIDs = new SerializableArrayList(); - - public ConstructStreamChainAction(final JobID jobID, final List vertexIDs) { - super(jobID); - - if (vertexIDs == null) { - throw new IllegalArgumentException("Argument vertexIDs must not be null"); - } - - if (vertexIDs.size() < 2) { - throw new IllegalArgumentException("Argument vertexIDs must be a list with at least two elements"); - } - - this.vertexIDs.addAll(vertexIDs); - } - - public ConstructStreamChainAction() { - super(); - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - - super.write(out); - - this.vertexIDs.write(out); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - - super.read(in); - - this.vertexIDs.read(in); - } - - public List getVertexIDs() { - - return Collections.unmodifiableList(this.vertexIDs); - } - - /** - * {@inheritDoc} - */ - @Override - public ExecutionVertexID getVertexID() { - - return this.vertexIDs.get(0); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/LimitBufferSizeAction.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/LimitBufferSizeAction.java deleted file mode 100644 index 33633dd854f4f..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/actions/LimitBufferSizeAction.java +++ /dev/null @@ -1,147 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.actions; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.jobgraph.JobID; - -/** - * This class implements an action to limit the buffer size of a particular output channel. - * - * @author warneke - */ -public final class LimitBufferSizeAction extends AbstractAction { - - /** - * The ID of the vertex the initiated action applies to. - */ - private final ExecutionVertexID vertexID; - - /** - * The ID of the output channel whose buffer size shall be limited. - */ - private final ChannelID sourceChannelID; - - /** - * The new buffer size in bytes. - */ - private int bufferSize; - - /** - * Constructs a new buffer size limit action object. - * - * @param jobID - * the ID of the job the action applies to - * @param vertexID - * the ID of the vertex the action applies to - * @param sourceChannelID - * the ID of the output channel whose buffer size shall be limited - * @param bufferSize - * the new buffer size in bytes - */ - public LimitBufferSizeAction(final JobID jobID, final ExecutionVertexID vertexID, final ChannelID sourceChannelID, - final int bufferSize) { - super(jobID); - - if (vertexID == null) { - throw new IllegalArgumentException("Argument vertexID must not be null"); - } - - if (sourceChannelID == null) { - throw new IllegalArgumentException("Argument sourceChannelID must not be null"); - } - - if (bufferSize <= 0) { - throw new IllegalArgumentException("Argument bufferSize must be greather than zero"); - } - - this.vertexID = vertexID; - this.sourceChannelID = sourceChannelID; - this.bufferSize = bufferSize; - } - - /** - * Default constructor for deserialization. - */ - public LimitBufferSizeAction() { - super(); - this.vertexID = new ExecutionVertexID(); - this.sourceChannelID = new ChannelID(); - this.bufferSize = 0; - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - - super.write(out); - - this.vertexID.write(out); - this.sourceChannelID.write(out); - out.writeInt(this.bufferSize); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - - super.read(in); - - this.vertexID.read(in); - this.sourceChannelID.read(in); - this.bufferSize = in.readInt(); - } - - /** - * Returns the ID of the output channel whose buffer size shall be limited. - * - * @return the ID of the output channel whose buffer size shall be limited - */ - public ChannelID getSourceChannelID() { - - return this.sourceChannelID; - } - - /** - * Returns the new buffer size in bytes. - * - * @return the new buffer size in bytes - */ - public int getBufferSize() { - - return this.bufferSize; - } - - /** - * Returns the ID of the vertex the initiated action applies to. - * - * @return the ID of the vertex the initiated action applies to - */ - @Override - public ExecutionVertexID getVertexID() { - - return this.vertexID; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistory.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistory.java deleted file mode 100644 index 316f31c457d27..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistory.java +++ /dev/null @@ -1,55 +0,0 @@ -package eu.stratosphere.nephele.streaming.buffers; - -import eu.stratosphere.nephele.managementgraph.ManagementEdge; - -public class BufferSizeHistory { - - private BufferSizeHistoryEntry[] entries; - - private int entriesInHistory; - - private ManagementEdge edge; - - public BufferSizeHistory(ManagementEdge edge, int noOfHistoryEntries) { - this.edge = edge; - this.entries = new BufferSizeHistoryEntry[noOfHistoryEntries]; - this.entriesInHistory = 0; - } - - public void addToHistory(long timestamp, int newBufferSize) { - BufferSizeHistoryEntry newEntry = new BufferSizeHistoryEntry(Math.min(entriesInHistory, entries.length - 1), - edge, timestamp, newBufferSize); - - if (entriesInHistory < entries.length) { - entries[entriesInHistory] = newEntry; - entriesInHistory++; - } else { - System.arraycopy(entries, 1, entries, 0, entriesInHistory - 1); - entries[entriesInHistory - 1] = newEntry; - } - } - - public BufferSizeHistoryEntry[] getEntries() { - return entries; - } - - public BufferSizeHistoryEntry getFirstEntry() { - return entries[0]; - } - - public BufferSizeHistoryEntry getLastEntry() { - if (entriesInHistory > 0) { - return entries[entriesInHistory - 1]; - } else { - return null; - } - } - - public boolean hasEntries() { - return entriesInHistory > 0; - } - - public int getNumberOfEntries() { - return entriesInHistory; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistoryEntry.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistoryEntry.java deleted file mode 100644 index 0a876b3f6b70d..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeHistoryEntry.java +++ /dev/null @@ -1,37 +0,0 @@ -package eu.stratosphere.nephele.streaming.buffers; - -import eu.stratosphere.nephele.managementgraph.ManagementEdge; - -public class BufferSizeHistoryEntry { - private int entryIndex; - - private ManagementEdge edge; - - private long timestamp; - - private int bufferSize; - - public BufferSizeHistoryEntry(int entryIndex, ManagementEdge edge, long timestamp, int bufferSize) { - this.entryIndex = entryIndex; - this.edge = edge; - this.timestamp = timestamp; - this.bufferSize = bufferSize; - } - - public int getEntryIndex() { - return entryIndex; - } - - public ManagementEdge getEdge() { - return edge; - } - - public long getTimestamp() { - return timestamp; - } - - public int getBufferSize() { - return bufferSize; - } - -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java deleted file mode 100644 index 93189ecb558ec..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/buffers/BufferSizeManager.java +++ /dev/null @@ -1,240 +0,0 @@ -package eu.stratosphere.nephele.streaming.buffers; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.configuration.GlobalConfiguration; -import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.managementgraph.ManagementAttachment; -import eu.stratosphere.nephele.managementgraph.ManagementEdge; -import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; -import eu.stratosphere.nephele.streaming.StreamingJobManagerPlugin; -import eu.stratosphere.nephele.streaming.profiling.EdgeCharacteristics; -import eu.stratosphere.nephele.streaming.profiling.ProfilingModel; -import eu.stratosphere.nephele.streaming.profiling.ProfilingPath; -import eu.stratosphere.nephele.streaming.profiling.ProfilingSummary; -import eu.stratosphere.nephele.streaming.profiling.ProfilingUtils; -import eu.stratosphere.nephele.taskmanager.bufferprovider.GlobalBufferPool; - -public class BufferSizeManager { - - private final static long WAIT_BEFORE_FIRST_ADJUSTMENT = 30 * 1000; - - public final static long ADJUSTMENT_INTERVAL = 10 * 1000; - - private Log LOG = LogFactory.getLog(BufferSizeManager.class); - - private long latencyGoal; - - private ProfilingModel profilingModel; - - private StreamingJobManagerPlugin jobManagerPlugin; - - private ExecutionGraph executionGraph; - - private HashMap bufferSizes; - - private long timeOfNextAdjustment; - - private int maximumBufferSize; - - public BufferSizeManager(long latencyGoal, ProfilingModel profilingModel, - StreamingJobManagerPlugin jobManagerPlugin, - ExecutionGraph executionGraph) { - this.latencyGoal = latencyGoal; - this.profilingModel = profilingModel; - this.jobManagerPlugin = jobManagerPlugin; - this.executionGraph = executionGraph; - this.bufferSizes = new HashMap(); - this.timeOfNextAdjustment = ProfilingUtils.alignToNextFullSecond(System.currentTimeMillis() - + WAIT_BEFORE_FIRST_ADJUSTMENT); - initBufferSizes(); - } - - private void initBufferSizes() { - int bufferSize = GlobalConfiguration.getInteger("channel.network.bufferSizeInBytes", - GlobalBufferPool.DEFAULT_BUFFER_SIZE_IN_BYTES); - - this.maximumBufferSize = bufferSize; - - long now = System.currentTimeMillis(); - for (ProfilingPath path : profilingModel.getProfilingSubgraph().getProfilingPaths()) { - for (ManagementAttachment pathElement : path.getPathElements()) { - if (pathElement instanceof ManagementEdge) { - ManagementEdge edge = (ManagementEdge) pathElement; - BufferSizeHistory bufferSizeHistory = new BufferSizeHistory(edge, 2); - bufferSizeHistory.addToHistory(now, bufferSize); - bufferSizes.put(edge, bufferSizeHistory); - } - } - } - } - - public void adjustBufferSizes(ProfilingSummary summary) { - HashMap edgesToAdjust = new HashMap(); - - for (ProfilingPath activePath : summary.getActivePaths()) { - if (activePath.getSummary().getTotalLatency() > latencyGoal) { - collectEdgesToAdjust(activePath, edgesToAdjust); - } - } - - doAdjust(edgesToAdjust); - - refreshTimeOfNextAdjustment(); - } - - private void doAdjust(HashMap edgesToAdjust) { - - for (ManagementEdge edge : edgesToAdjust.keySet()) { - int newBufferSize = edgesToAdjust.get(edge); - - BufferSizeHistory sizeHistory = bufferSizes.get(edge); - - LOG.info(String.format("New buffer size: %s new: %d (old: %d)", ProfilingUtils.formatName(edge), - newBufferSize, sizeHistory.getLastEntry().getBufferSize())); - - setBufferSize(edge.getSourceEdgeID(), newBufferSize); - - sizeHistory.addToHistory(timeOfNextAdjustment, newBufferSize); - } - } - - private void refreshTimeOfNextAdjustment() { - long now = System.currentTimeMillis(); - while (timeOfNextAdjustment <= now) { - timeOfNextAdjustment += ADJUSTMENT_INTERVAL; - } - } - - ArrayList edgesSortedByLatency = new ArrayList(); - - Comparator edgeComparator = new Comparator() { - @Override - public int compare(ManagementEdge first, ManagementEdge second) { - double firstLatency = ((EdgeCharacteristics) first.getAttachment()).getChannelLatencyInMillis(); - double secondLatency = ((EdgeCharacteristics) second.getAttachment()).getChannelLatencyInMillis(); - - if (firstLatency < secondLatency) { - return -1; - } else if (firstLatency > secondLatency) { - return 1; - } else { - return 0; - } - } - }; - - private void collectEdgesToAdjust(ProfilingPath path, HashMap edgesToAdjust) { - for (ManagementAttachment element : path.getPathElements()) { - if (element instanceof ManagementEdge) { - edgesSortedByLatency.add((ManagementEdge) element); - } - } - - Collections.sort(edgesSortedByLatency, edgeComparator); - - for (ManagementEdge edge : edgesSortedByLatency) { - - if (edgesToAdjust.containsKey(edge)) { - continue; - } - - EdgeCharacteristics edgeChar = (EdgeCharacteristics) edge.getAttachment(); - - if (!hasFreshValues(edge)) { - // LOG.info("Rejecting edge due to stale values: " + ProfilingUtils.formatName(edge)); - continue; - } - - double edgeLatency = edgeChar.getChannelLatencyInMillis(); - double avgOutputBufferLatency = edgeChar.getOutputBufferLatencyInMillis() / 2; - - if (avgOutputBufferLatency > 5 && avgOutputBufferLatency >= 0.05 * edgeLatency) { - reduceBufferSize(edge, edgesToAdjust); - } else if (avgOutputBufferLatency <= 1) { - increaseBufferSize(edge, edgesToAdjust); - } - } - - edgesSortedByLatency.clear(); - } - - private void increaseBufferSize(ManagementEdge edge, HashMap edgesToAdjust) { - int oldBufferSize = bufferSizes.get(edge).getLastEntry().getBufferSize(); - int newBufferSize = Math.min(proposedIncreasedBufferSize(oldBufferSize), this.maximumBufferSize); - - if (isRelevantIncrease(oldBufferSize, newBufferSize)) { - edgesToAdjust.put(edge, newBufferSize); - } - } - - private boolean isRelevantIncrease(int oldBufferSize, int newBufferSize) { - return newBufferSize >= oldBufferSize + 100; - } - - private int proposedIncreasedBufferSize(int oldBufferSize) { - return (int) (oldBufferSize * 1.2); - } - - private void reduceBufferSize(ManagementEdge edge, HashMap edgesToAdjust) { - int oldBufferSize = bufferSizes.get(edge).getLastEntry().getBufferSize(); - int newBufferSize = proposedReducedBufferSize(edge, oldBufferSize); - - // filters pointless minor changes in buffer size - if (isRelevantReduction(newBufferSize, oldBufferSize)) { - edgesToAdjust.put(edge, newBufferSize); - } - - // else { - // LOG.info(String.format("Filtering reduction due to insignificance: %s (old:%d new:%d)", - // ProfilingUtils.formatName(edge), oldBufferSize, newBufferSize)); - // } - } - - private boolean isRelevantReduction(int newBufferSize, int oldBufferSize) { - return newBufferSize < oldBufferSize * 0.98; - } - - private int proposedReducedBufferSize(ManagementEdge edge, int oldBufferSize) { - EdgeCharacteristics edgeChar = (EdgeCharacteristics) edge.getAttachment(); - - double avgOutputBufferLatency = edgeChar.getOutputBufferLatencyInMillis() / 2; - - double reductionFactor = Math.pow(0.99, avgOutputBufferLatency); - reductionFactor = Math.max(0.1, reductionFactor); - - int newBufferSize = (int) Math.max(100, oldBufferSize * reductionFactor); - - return newBufferSize; - } - - private boolean hasFreshValues(ManagementEdge edge) { - EdgeCharacteristics edgeChar = (EdgeCharacteristics) edge.getAttachment(); - long freshnessThreshold = bufferSizes.get(edge).getLastEntry().getTimestamp(); - - return edgeChar.isChannelLatencyFresherThan(freshnessThreshold) - && edgeChar.isOutputBufferLatencyFresherThan(freshnessThreshold); - } - - public boolean isAdjustmentNecessary(long now) { - return now >= timeOfNextAdjustment; - } - - private void setBufferSize(ManagementEdgeID sourceEdgeID, int bufferSize) { - ChannelID sourceChannelID = sourceEdgeID.toChannelID(); - ExecutionVertex vertex = this.executionGraph.getVertexByChannelID(sourceChannelID); - if (vertex == null) { - LOG.error("Cannot find vertex to channel ID " + vertex); - return; - } - this.jobManagerPlugin.limitBufferSize(vertex, sourceChannelID, bufferSize); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/RecordUtils.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/RecordUtils.java deleted file mode 100644 index af62d423b687f..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/RecordUtils.java +++ /dev/null @@ -1,66 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.chaining; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; - -import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.util.StringUtils; - -public final class RecordUtils { - - /** - * Private constructor to prevent instantiation. - */ - private RecordUtils() { - } - - /** - * Creates a copy of the given {@link Record} object by an in-memory serialization and subsequent - * deserialization. - * - * @param original - * the original object to be copied - * @return the copy of original object created by the original object's serialization/deserialization methods - * @throws IOException - * thrown if an error occurs while copying the object - */ - public static Record createCopy(final Record original) throws IOException { - - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - final DataOutputStream dos = new DataOutputStream(baos); - - original.write(dos); - - Record copy; - try { - copy = original.getClass().newInstance(); - } catch (Exception e) { - throw new IOException(StringUtils.stringifyException(e)); - } - - final ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); - final DataInputStream dis = new DataInputStream(bais); - - copy.read(dis); - - return copy; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java deleted file mode 100644 index 529f4b1467343..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChain.java +++ /dev/null @@ -1,86 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.chaining; - -import java.io.IOException; -import java.util.List; -import java.util.Queue; - -import eu.stratosphere.nephele.execution.Mapper; -import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; -import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.util.StringUtils; - -public final class StreamChain { - - @SuppressWarnings("rawtypes") - private final List chainLinks; - - @SuppressWarnings("rawtypes") - StreamChain(final List chainLinks) { - - if (chainLinks.isEmpty()) { - throw new IllegalArgumentException("List chainLinks must not be empty"); - } - - this.chainLinks = chainLinks; - } - - @SuppressWarnings("unchecked") - public StreamingOutputGate getFirstOutputGate() { - - return this.chainLinks.get(0).getOutputGate(); - } - - public void writeRecord(final Record record) throws IOException, InterruptedException { - - try { - executeMapper(record, 1); - } catch (Exception e) { - throw new IOException(StringUtils.stringifyException(e)); - } - } - - @SuppressWarnings({ "rawtypes", "unchecked" }) - void executeMapper(final Record record, final int chainIndex) throws Exception { - - final StreamChainLink chainLink = this.chainLinks.get(chainIndex); - final Mapper mapper = chainLink.getMapper(); - - chainLink.getInputGate().reportRecordReceived(record); - mapper.map(record); - - final StreamingOutputGate outputGate = chainLink.getOutputGate(); - - final Queue outputCollector = mapper.getOutputCollector(); - - if (chainIndex == this.chainLinks.size() - 1) { - - while (!outputCollector.isEmpty()) { - - outputGate.writeRecord((Record)outputCollector.poll()); - } - - } else { - - while (!outputCollector.isEmpty()) { - final Record outputRecord = (Record) outputCollector.poll(); - outputGate.reportRecordEmitted(outputRecord); - executeMapper(RecordUtils.createCopy(outputRecord), chainIndex + 1); - } - } - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java deleted file mode 100644 index b12e9727b472e..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainCoordinator.java +++ /dev/null @@ -1,73 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.chaining; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.execution.Mapper; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.streaming.wrappers.StreamingInputGate; -import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; -import eu.stratosphere.nephele.types.Record; - -public final class StreamChainCoordinator { - - /** - * The log object. - */ - private static final Log LOG = LogFactory.getLog(StreamChainCoordinator.class); - - private final ConcurrentMap chainLinks = new ConcurrentHashMap(); - - public void registerMapper(final ExecutionVertexID vertexID, - final Mapper mapper, - final StreamingInputGate inputGate, final StreamingOutputGate outputGate) { - - final StreamChainLink chainLink = new StreamChainLink(mapper, inputGate, outputGate); - - if (this.chainLinks.putIfAbsent(vertexID, chainLink) == null) { - LOG.info("Registering stream chain link for vertex ID " + vertexID); - } - } - - public StreamChain constructStreamChain(final List vertexIDs) { - - final Iterator it = vertexIDs.iterator(); - final List chainLinkList = new ArrayList(); - while (it.hasNext()) { - - final ExecutionVertexID vertexID = it.next(); - final StreamChainLink chainLink = this.chainLinks.get(vertexID); - if (chainLink == null) { - LOG.error("Cannot construct stream chain from " + vertexIDs.get(0) + " to " - + vertexIDs.get(vertexIDs.size() - 1) + ": No chain link for vertex ID " + vertexID); - return null; - } - - chainLinkList.add(chainLink); - } - - return new StreamChain(Collections.unmodifiableList(chainLinkList)); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java deleted file mode 100644 index da53a9c756010..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/chaining/StreamChainLink.java +++ /dev/null @@ -1,53 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.chaining; - -import eu.stratosphere.nephele.execution.Mapper; -import eu.stratosphere.nephele.streaming.wrappers.StreamingInputGate; -import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; -import eu.stratosphere.nephele.types.Record; - -public final class StreamChainLink { - - private final Mapper mapper; - - private final StreamingInputGate inputGate; - - private final StreamingOutputGate outputGate; - - StreamChainLink(final Mapper mapper, final StreamingInputGate inputGate, - final StreamingOutputGate outputGate) { - - this.mapper = mapper; - this.inputGate = inputGate; - this.outputGate = outputGate; - } - - Mapper getMapper() { - - return this.mapper; - } - - StreamingInputGate getInputGate() { - - return this.inputGate; - } - - StreamingOutputGate getOutputGate() { - - return this.outputGate; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java deleted file mode 100644 index 5129371524b0f..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListener.java +++ /dev/null @@ -1,299 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.listeners; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Queue; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.Mapper; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.GateID; -import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; -import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.streaming.StreamingTag; -import eu.stratosphere.nephele.streaming.StreamingTaskManagerPlugin; -import eu.stratosphere.nephele.streaming.actions.AbstractAction; -import eu.stratosphere.nephele.streaming.actions.ConstructStreamChainAction; -import eu.stratosphere.nephele.streaming.actions.LimitBufferSizeAction; -import eu.stratosphere.nephele.streaming.chaining.StreamChain; -import eu.stratosphere.nephele.streaming.types.ChannelLatency; -import eu.stratosphere.nephele.streaming.types.ChannelThroughput; -import eu.stratosphere.nephele.streaming.types.OutputBufferLatency; -import eu.stratosphere.nephele.streaming.types.TaskLatency; -import eu.stratosphere.nephele.streaming.wrappers.StreamingInputGate; -import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; -import eu.stratosphere.nephele.types.AbstractTaggableRecord; -import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.util.StringUtils; - -public final class StreamListener { - - /** - * The log object. - */ - private static final Log LOG = LogFactory.getLog(StreamListener.class); - - private final Configuration taskConfiguration; - - private StreamListenerContext listenerContext = null; - - private int tagCounter = 0; - - private Map> outputGateMap = new HashMap>(); - - private Map> outputChannelMap; - - /** - * Indicates the time of the last received tagged incoming record - */ - private long lastTimestamp = -1L; - - public StreamListener(final Configuration taskConfiguration) { - - if (taskConfiguration == null) { - throw new IllegalArgumentException("Argument taskConfiguration must not be null"); - } - - this.taskConfiguration = taskConfiguration; - } - - /** - * Initializes the stream listener by retrieving the listener context from the task manager plugin. - */ - public void init() { - - final String listenerKey = this.taskConfiguration.getString(StreamListenerContext.CONTEXT_CONFIGURATION_KEY, - null); - - if (listenerKey == null) { - throw new RuntimeException("Stream listener is unable to retrieve context key"); - } - - this.listenerContext = StreamingTaskManagerPlugin.getStreamingListenerContext(listenerKey); - - final Map> tmpMap = new HashMap>(); - - final Iterator> it = this.outputGateMap.values().iterator(); - while (it.hasNext()) { - final StreamingOutputGate outputGate = it.next(); - final int numberOfOutputChannels = outputGate.getNumberOfOutputChannels(); - for (int i = 0; i < numberOfOutputChannels; ++i) { - final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(i); - tmpMap.put(outputChannel.getID(), outputChannel); - } - } - - this.outputChannelMap = Collections.unmodifiableMap(tmpMap); - } - - public long recordEmitted(final Record record) { - - long timestamp = -1L; - - // Input vertex - if (this.listenerContext.isInputVertex()) { - - final int taggingInterval = this.listenerContext.getTaggingInterval(); - final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - - // Tag every record and calculate task latency - if (this.tagCounter++ == taggingInterval) { - timestamp = System.currentTimeMillis(); - taggableRecord.setTag(createTag(timestamp)); - if (this.lastTimestamp > 0L) { - final long taskLatency = (timestamp - this.lastTimestamp) / taggingInterval; - try { - this.listenerContext.sendDataAsynchronously(new TaskLatency(this.listenerContext.getJobID(), - this.listenerContext.getVertexID(), taskLatency)); - } catch (InterruptedException e) { - LOG.error(StringUtils.stringifyException(e)); - } - } - this.lastTimestamp = timestamp; - this.tagCounter = 0; - - // Finally, check for pending actions - checkForPendingActions(); - } else { - taggableRecord.setTag(null); - } - - } else { - - final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - - if (this.lastTimestamp >= 0L) { - - timestamp = System.currentTimeMillis(); - taggableRecord.setTag(createTag(timestamp)); - final JobID jobID = this.listenerContext.getJobID(); - final ExecutionVertexID vertexID = this.listenerContext.getVertexID(); - - // Calculate task latency - final TaskLatency tl = new TaskLatency(jobID, vertexID, timestamp - this.lastTimestamp); - try { - this.listenerContext.sendDataAsynchronously(tl); - } catch (InterruptedException e) { - LOG.error(StringUtils.stringifyException(e)); - } - - this.lastTimestamp = -1L; - - // Finally, check for pending actions - checkForPendingActions(); - } else { - taggableRecord.setTag(null); - } - } - - return timestamp; - } - - /** - * {@inheritDoc} - */ - public void recordReceived(final Record record) { - - final AbstractTaggableRecord taggableRecord = (AbstractTaggableRecord) record; - final StreamingTag tag = (StreamingTag) taggableRecord.getTag(); - if (tag != null) { - - final long timestamp = System.currentTimeMillis(); - final JobID jobID = this.listenerContext.getJobID(); - - final ExecutionVertexID vertexID = this.listenerContext.getVertexID(); - - // Calculate channel latency - final ChannelLatency cl = new ChannelLatency(jobID, tag.getSourceID(), vertexID, timestamp - - tag.getTimestamp()); - try { - this.listenerContext.sendDataAsynchronously(cl); - } catch (InterruptedException e) { - LOG.warn(StringUtils.stringifyException(e)); - } - - this.lastTimestamp = timestamp; - } - } - - public void reportChannelThroughput(final ChannelID sourceChannelID, final double throughput) { - - try { - this.listenerContext.sendDataAsynchronously(new ChannelThroughput(this.listenerContext.getJobID(), - this.listenerContext.getVertexID(), sourceChannelID, throughput)); - } catch (InterruptedException e) { - LOG.error(StringUtils.stringifyException(e)); - } - } - - public void reportBufferLatency(final ChannelID sourceChannelID, final int bufferLatency) { - - try { - this.listenerContext.sendDataAsynchronously(new OutputBufferLatency(this.listenerContext.getJobID(), - this.listenerContext.getVertexID(), sourceChannelID, bufferLatency)); - } catch (InterruptedException e) { - LOG.error(StringUtils.stringifyException(e)); - } - } - - private StreamingTag createTag(final long timestamp) { - StreamingTag tag = new StreamingTag(this.listenerContext.getVertexID()); - tag.setTimestamp(timestamp); - return tag; - } - - private void checkForPendingActions() { - - final Queue pendingActions = this.listenerContext.getPendingActionsQueue(); - - synchronized (pendingActions) { - - while (!pendingActions.isEmpty()) { - - final AbstractAction action = pendingActions.poll(); - - if (action instanceof LimitBufferSizeAction) { - limitBufferSize((LimitBufferSizeAction) action); - } else if (action instanceof ConstructStreamChainAction) { - constructStreamChain((ConstructStreamChainAction) action); - } else { - LOG.error("Ignoring unknown action of type " + action.getClass()); - } - } - } - } - - private void constructStreamChain(final ConstructStreamChainAction csca) { - - final StreamChain streamChain = this.listenerContext.constructStreamChain(csca.getVertexIDs()); - if (streamChain == null) { - return; - } - - final StreamingOutputGate outputGate = streamChain.getFirstOutputGate(); - try { - - outputGate.flush(); - - outputGate.redirectToStreamChain(streamChain); - } catch(Exception e) { - LOG.error(StringUtils.stringifyException(e)); - } - } - - private void limitBufferSize(final LimitBufferSizeAction bsla) { - - final ChannelID sourceChannelID = bsla.getSourceChannelID(); - final int bufferSize = bsla.getBufferSize(); - - final AbstractOutputChannel outputChannel = this.outputChannelMap.get(sourceChannelID); - if (outputChannel == null) { - LOG.error("Cannot find output channel with ID " + sourceChannelID); - return; - } - - if (!(outputChannel instanceof AbstractByteBufferedOutputChannel)) { - LOG.error("Output channel with ID " + sourceChannelID + " is not a byte-buffered channel"); - return; - } - - final AbstractByteBufferedOutputChannel byteBufferedOutputChannel = - (AbstractByteBufferedOutputChannel) outputChannel; - - LOG.info("Setting buffer size limit of output channel " + sourceChannelID + " to " + bufferSize + " bytes"); - byteBufferedOutputChannel.limitBufferSize(bufferSize); - } - - public void registerOutputGate(final StreamingOutputGate outputGate) { - - this.outputGateMap.put(outputGate.getGateID(), outputGate); - } - - public void registerMapper(final Mapper mapper, - final StreamingInputGate input, final StreamingOutputGate output) { - - this.listenerContext.registerMapper(mapper, input, output); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java deleted file mode 100644 index 010eafcbf2ff6..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/listeners/StreamListenerContext.java +++ /dev/null @@ -1,181 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.listeners; - -import java.util.ArrayDeque; -import java.util.List; -import java.util.Queue; - -import eu.stratosphere.nephele.execution.Mapper; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.streaming.StreamingCommunicationThread; -import eu.stratosphere.nephele.streaming.actions.AbstractAction; -import eu.stratosphere.nephele.streaming.chaining.StreamChain; -import eu.stratosphere.nephele.streaming.chaining.StreamChainCoordinator; -import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; -import eu.stratosphere.nephele.streaming.wrappers.StreamingInputGate; -import eu.stratosphere.nephele.streaming.wrappers.StreamingOutputGate; -import eu.stratosphere.nephele.types.Record; - -public final class StreamListenerContext { - - public static final String CONTEXT_CONFIGURATION_KEY = "streaming.listener.context"; - - private static enum TaskType { - INPUT, REGULAR, OUTPUT - }; - - private final Queue pendingActions = new ArrayDeque(); - - private final JobID jobID; - - private final ExecutionVertexID vertexID; - - private final StreamingCommunicationThread communicationThread; - - private final StreamChainCoordinator chainCoordinator; - - private final TaskType taskType; - - private final int aggregationInterval; - - private final int taggingInterval; - - private StreamListenerContext(final JobID jobID, final ExecutionVertexID vertexID, - final StreamingCommunicationThread communicationThread, final StreamChainCoordinator chainCoordinator, - final TaskType taskType, final int aggregationInterval, final int taggingInterval) { - - if (jobID == null) { - throw new IllegalArgumentException("Parameter jobID must not be null"); - } - - if (vertexID == null) { - throw new IllegalArgumentException("Parameter vertexID must not be null"); - } - - if (communicationThread == null) { - throw new IllegalArgumentException("Parameter communicationThread must not be null"); - } - - if (taskType == null) { - throw new IllegalArgumentException("Parameter taskType must not be null"); - } - - if (aggregationInterval <= 0) { - throw new IllegalArgumentException("Parameter aggregationInterval must be greater than zero"); - } - - if (taggingInterval <= 0 && taskType == TaskType.INPUT) { - throw new IllegalArgumentException("Parameter taggingInterval must be greater than zero"); - } - - this.jobID = jobID; - this.vertexID = vertexID; - this.communicationThread = communicationThread; - this.chainCoordinator = chainCoordinator; - this.taskType = taskType; - this.aggregationInterval = aggregationInterval; - this.taggingInterval = taggingInterval; - } - - public static StreamListenerContext createForInputTask(final JobID jobID, final ExecutionVertexID vertexID, - final StreamingCommunicationThread communicationThread, final StreamChainCoordinator chainCoordinator, - final int aggregationInterval, final int taggingInterval) { - - return new StreamListenerContext(jobID, vertexID, communicationThread, chainCoordinator, TaskType.INPUT, - aggregationInterval, taggingInterval); - } - - public static StreamListenerContext createForRegularTask(final JobID jobID, final ExecutionVertexID vertexID, - final StreamingCommunicationThread communicationThread, final StreamChainCoordinator chainCoordinator, - final int aggregationInterval) { - - return new StreamListenerContext(jobID, vertexID, communicationThread, chainCoordinator, TaskType.REGULAR, - aggregationInterval, -1); - } - - public static StreamListenerContext createForOutputTask(final JobID jobID, final ExecutionVertexID vertexID, - final StreamingCommunicationThread communicationThread, final StreamChainCoordinator chainCoordinator, - final int aggregationInterval) { - - return new StreamListenerContext(jobID, vertexID, communicationThread, chainCoordinator, TaskType.OUTPUT, - aggregationInterval, -1); - } - - boolean isInputVertex() { - - return (this.taskType == TaskType.INPUT); - } - - boolean isOutputVertex() { - - return (this.taskType == TaskType.OUTPUT); - } - - boolean isRegularVertex() { - - return (this.taskType == TaskType.REGULAR); - } - - JobID getJobID() { - - return this.jobID; - } - - ExecutionVertexID getVertexID() { - - return this.vertexID; - } - - int getTaggingInterval() { - - return this.taggingInterval; - } - - int getAggregationInterval() { - - return this.aggregationInterval; - } - - void sendDataAsynchronously(final AbstractStreamingData data) throws InterruptedException { - - this.communicationThread.sendDataAsynchronously(data); - } - - public void queuePendingAction(final AbstractAction action) { - - synchronized (this.pendingActions) { - this.pendingActions.add(action); - } - } - - Queue getPendingActionsQueue() { - - return this.pendingActions; - } - - void registerMapper(final Mapper mapper, - final StreamingInputGate inputGate, final StreamingOutputGate outputGate) { - - this.chainCoordinator.registerMapper(this.vertexID, mapper, inputGate, outputGate); - } - - StreamChain constructStreamChain(final List vertexIDs) { - - return this.chainCoordinator.constructStreamChain(vertexIDs); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java deleted file mode 100644 index 0fa988b28652d..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/EdgeCharacteristics.java +++ /dev/null @@ -1,72 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import eu.stratosphere.nephele.managementgraph.ManagementEdge; - -public class EdgeCharacteristics { - - private ManagementEdge edge; - - private ProfilingValueStatistic latencyInMillisStatistic; - - private ProfilingValueStatistic throughputInMbitStatistic; - - private ProfilingValueStatistic outputBufferLatencyStatistic; - - public EdgeCharacteristics(ManagementEdge edge) { - this.edge = edge; - this.latencyInMillisStatistic = new ProfilingValueStatistic(10); - this.throughputInMbitStatistic = new ProfilingValueStatistic(10); - this.outputBufferLatencyStatistic = new ProfilingValueStatistic(10); - } - - public ManagementEdge getEdge() { - return edge; - } - - public double getChannelLatencyInMillis() { - if (latencyInMillisStatistic.hasValues()) { - return latencyInMillisStatistic.getArithmeticMean(); - } else { - return -1; - } - } - - public double getChannelThroughputInMbit() { - if (throughputInMbitStatistic.hasValues()) { - return throughputInMbitStatistic.getArithmeticMean(); - } else { - return -1; - } - } - - public double getOutputBufferLatencyInMillis() { - if (outputBufferLatencyStatistic.hasValues()) { - return outputBufferLatencyStatistic.getArithmeticMean(); - } else { - return -1; - } - } - - public void addLatencyMeasurement(long timestamp, double latencyInMillis) { - ProfilingValue value = new ProfilingValue(latencyInMillis, timestamp); - this.latencyInMillisStatistic.addValue(value); - } - - public void addThroughputMeasurement(long timestamp, double throughputInMbit) { - ProfilingValue value = new ProfilingValue(throughputInMbit, timestamp); - this.throughputInMbitStatistic.addValue(value); - } - - public void addOutputBufferLatencyMeasurement(long timestamp, double latencyInMillis) { - ProfilingValue value = new ProfilingValue(latencyInMillis, timestamp); - this.outputBufferLatencyStatistic.addValue(value); - } - - public boolean isChannelLatencyFresherThan(long freshnessThreshold) { - return latencyInMillisStatistic.getOldestValue().getTimestamp() >= freshnessThreshold; - } - - public boolean isOutputBufferLatencyFresherThan(long freshnessThreshold) { - return outputBufferLatencyStatistic.getOldestValue().getTimestamp() >= freshnessThreshold; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java deleted file mode 100644 index 9b3d1cf79f37d..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/LatencyOptimizerThread.java +++ /dev/null @@ -1,91 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import java.io.IOException; -import java.util.concurrent.LinkedBlockingQueue; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.streaming.StreamingJobManagerPlugin; -import eu.stratosphere.nephele.streaming.buffers.BufferSizeManager; -import eu.stratosphere.nephele.streaming.types.AbstractStreamingData; -import eu.stratosphere.nephele.streaming.types.ChannelLatency; -import eu.stratosphere.nephele.streaming.types.ChannelThroughput; -import eu.stratosphere.nephele.streaming.types.OutputBufferLatency; -import eu.stratosphere.nephele.streaming.types.TaskLatency; - -public class LatencyOptimizerThread extends Thread { - - private Log LOG = LogFactory.getLog(LatencyOptimizerThread.class); - - private final LinkedBlockingQueue streamingDataQueue; - - private final StreamingJobManagerPlugin jobManagerPlugin; - - private final ExecutionGraph executionGraph; - - private final ProfilingModel profilingModel; - - private ProfilingLogger logger; - - private BufferSizeManager bufferSizeManager; - - public LatencyOptimizerThread(StreamingJobManagerPlugin jobManagerPlugin, ExecutionGraph executionGraph) { - this.jobManagerPlugin = jobManagerPlugin; - this.executionGraph = executionGraph; - this.profilingModel = new ProfilingModel(executionGraph); - this.streamingDataQueue = new LinkedBlockingQueue(); - try { - this.logger = new ProfilingLogger(); - } catch (IOException e) { - LOG.error("Error when opening profiling logger file", e); - } - - this.bufferSizeManager = new BufferSizeManager(200, this.profilingModel, this.jobManagerPlugin, - this.executionGraph); - } - - public void run() { - LOG.info("Started optimizer thread for job " + executionGraph.getJobName()); - - try { - while (!interrupted()) { - AbstractStreamingData streamingData = streamingDataQueue.take(); - - long now = System.currentTimeMillis(); - if (streamingData instanceof ChannelLatency) { - profilingModel.refreshEdgeLatency(now, (ChannelLatency) streamingData); - } else if (streamingData instanceof TaskLatency) { - profilingModel.refreshTaskLatency(now, (TaskLatency) streamingData); - } else if (streamingData instanceof ChannelThroughput) { - profilingModel.refreshChannelThroughput(now, (ChannelThroughput) streamingData); - } else if (streamingData instanceof OutputBufferLatency) { - profilingModel.refreshChannelOutputBufferLatency(now, (OutputBufferLatency) streamingData); - } - - if (this.logger.isLoggingNecessary(now)) { - ProfilingSummary summary = profilingModel.computeProfilingSummary(); - try { - logger.logLatencies(summary); - } catch (IOException e) { - LOG.error("Error when writing to profiling logger file", e); - } - - if (bufferSizeManager.isAdjustmentNecessary(now)) { - bufferSizeManager.adjustBufferSizes(summary); - } - } - } - - } catch (InterruptedException e) { - } - - LOG.info("Stopped optimizer thread for job " + executionGraph.getJobName()); - } - - public void handOffStreamingData(AbstractStreamingData data) { - streamingDataQueue.add(data); - } - -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java deleted file mode 100644 index d360ada7611a0..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingLogger.java +++ /dev/null @@ -1,105 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import java.io.BufferedWriter; -import java.io.FileWriter; -import java.io.IOException; - -import eu.stratosphere.nephele.managementgraph.ManagementAttachment; -import eu.stratosphere.nephele.managementgraph.ManagementVertex; - -public class ProfilingLogger { - - private final static long WAIT_BEFORE_FIRST_LOGGING = 10 * 1000; - - private final static long LOGGING_INTERVAL = 1000; - - private BufferedWriter writer; - - private boolean headersWritten; - - private long timeOfNextLogging; - - private long timeBase; - - public ProfilingLogger() - throws IOException { - - this.writer = new BufferedWriter(new FileWriter("profiling.txt")); - this.headersWritten = false; - this.timeOfNextLogging = ProfilingUtils.alignToNextFullSecond(System.currentTimeMillis() + WAIT_BEFORE_FIRST_LOGGING); - this.timeBase = timeOfNextLogging; - } - - public boolean isLoggingNecessary(long now) { - return now >= timeOfNextLogging; - } - - public void logLatencies(ProfilingSummary summary) throws IOException { - long now = System.currentTimeMillis(); - long timestamp = now - timeBase; - - if (!headersWritten) { - writeHeaders(summary); - } - - StringBuilder builder = new StringBuilder(); - builder.append(timestamp); - builder.append(';'); - builder.append(summary.getNoOfActivePaths()); - builder.append(';'); - builder.append(summary.getNoOfInactivePaths()); - builder.append(';'); - builder.append(summary.getAvgTotalPathLatency()); - builder.append(';'); - builder.append(summary.getMedianPathLatency()); - builder.append(';'); - builder.append(summary.getMinPathLatency()); - builder.append(';'); - builder.append(summary.getMaxPathLatency()); - - for (double avgElementLatency : summary.getAvgPathElementLatencies()) { - builder.append(';'); - builder.append(avgElementLatency); - } - builder.append('\n'); - writer.write(builder.toString()); - writer.flush(); // FIXME - - refreshTimeOfNextLogging(); - } - - private void refreshTimeOfNextLogging() { - long now = System.currentTimeMillis(); - while(timeOfNextLogging <= now) { - timeOfNextLogging += LOGGING_INTERVAL; - } - } - - private void writeHeaders(ProfilingSummary summary) throws IOException { - StringBuilder builder = new StringBuilder(); - builder.append("timestamp;"); - builder.append("noOfActivePaths;"); - builder.append("noOfInactivePaths;"); - builder.append("avgTotalPathLatency;"); - builder.append("medianPathLatency;"); - builder.append("minPathLatency;"); - builder.append("maxPathLatency"); - - int nextEdgeIndex = 1; - for (ManagementAttachment element : summary.getPathElements()) { - builder.append(';'); - if (element instanceof ManagementVertex) { - ManagementVertex vertex = (ManagementVertex) element; - builder.append(vertex.getGroupVertex().getName()); - } else { - builder.append("edge" + nextEdgeIndex + "obl"); - builder.append(';'); - builder.append("edge" + nextEdgeIndex); - nextEdgeIndex++; - } - } - builder.append('\n'); - writer.write(builder.toString()); - headersWritten = true; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java deleted file mode 100644 index 01ce8627965ea..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingModel.java +++ /dev/null @@ -1,96 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; -import eu.stratosphere.nephele.streaming.types.ChannelLatency; -import eu.stratosphere.nephele.streaming.types.ChannelThroughput; -import eu.stratosphere.nephele.streaming.types.OutputBufferLatency; -import eu.stratosphere.nephele.streaming.types.TaskLatency; - -public class ProfilingModel { - -// private static Log LOG = LogFactory.getLog(ProfilingModel.class); - - private ExecutionGraph executionGraph; - - private ProfilingSubgraph profilingSubgraph; - - public ProfilingModel(ExecutionGraph executionGraph) { - this.executionGraph = executionGraph; - - // FIXME naive implementation until we can annotate the job - // subgraphStart and subgraphEnd should be derived from the annotations - ExecutionGroupVertex subgraphStart = this.executionGraph.getInputVertex(0).getGroupVertex(); - ExecutionGroupVertex subgraphEnd = this.executionGraph.getOutputVertex(0).getGroupVertex(); - - this.profilingSubgraph = new ProfilingSubgraph(executionGraph, subgraphStart, subgraphEnd, false, false); - } - - public void refreshEdgeLatency(long timestamp, ChannelLatency channelLatency) { - // FIXME workaround for bug that causes NaNs - if (Double.isInfinite(channelLatency.getChannelLatency()) || Double.isNaN(channelLatency.getChannelLatency())) { - return; - } - - // FIXME: workaround for bug caused by streaming plugin - if (!channelLatency.getSourceVertexID().equals(channelLatency.getSinkVertexID())) { - - XoredVertexID xored = new XoredVertexID(channelLatency.getSourceVertexID().toManagementVertexID(), - channelLatency.getSinkVertexID().toManagementVertexID()); - - ManagementEdgeID sourceEdgeID = profilingSubgraph.getSourceEdgeIDByXoredVertexID(xored); - - if (sourceEdgeID == null) { - ExecutionVertex source = executionGraph.getVertexByID(channelLatency.getSourceVertexID()); - ExecutionVertex sink = executionGraph.getVertexByID(channelLatency.getSinkVertexID()); - - throw new RuntimeException("No source edge ID for " + ProfilingUtils.formatName(source) + "->" + ProfilingUtils.formatName(sink) + " " - + xored.toString()); - } - - EdgeCharacteristics edgeCharacteristics = profilingSubgraph - .getEdgeCharacteristicsBySourceEdgeID(sourceEdgeID); - - edgeCharacteristics.addLatencyMeasurement(timestamp, channelLatency.getChannelLatency()); - } - } - - public void refreshTaskLatency(long timestamp, TaskLatency taskLatency) { - // FIXME workaround for bug that causes NaNs - if (Double.isInfinite(taskLatency.getTaskLatency()) || Double.isNaN(taskLatency.getTaskLatency())) { - return; - } - - VertexLatency vertexLatency = profilingSubgraph - .getVertexLatency(taskLatency.getVertexID().toManagementVertexID()); - vertexLatency.addLatencyMeasurement(timestamp, taskLatency.getTaskLatency()); - } - - public void refreshChannelThroughput(long timestamp, ChannelThroughput channelThroughput) { - - // FIXME workaround for bug that causes NaNs - if (Double.isInfinite(channelThroughput.getThroughput()) || Double.isNaN(channelThroughput.getThroughput())) { - return; - } - - ManagementEdgeID edgeID = new ManagementEdgeID(channelThroughput.getSourceChannelID()); - EdgeCharacteristics edgeCharaceristics = profilingSubgraph.getEdgeCharacteristicsBySourceEdgeID(edgeID); - edgeCharaceristics.addThroughputMeasurement(timestamp, channelThroughput.getThroughput()); - } - - public void refreshChannelOutputBufferLatency(long timestamp, OutputBufferLatency latency) { - ManagementEdgeID sourceEdgeID = new ManagementEdgeID(latency.getSourceChannelID()); - EdgeCharacteristics edgeCharaceristics = profilingSubgraph.getEdgeCharacteristicsBySourceEdgeID(sourceEdgeID); - edgeCharaceristics.addOutputBufferLatencyMeasurement(timestamp, latency.getBufferLatency()); - } - - public ProfilingSummary computeProfilingSummary() { - return new ProfilingSummary(profilingSubgraph); - } - - public ProfilingSubgraph getProfilingSubgraph() { - return profilingSubgraph; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPath.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPath.java deleted file mode 100644 index d46ffab302cb0..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPath.java +++ /dev/null @@ -1,193 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; - -import eu.stratosphere.nephele.managementgraph.ManagementAttachment; -import eu.stratosphere.nephele.managementgraph.ManagementEdge; -import eu.stratosphere.nephele.managementgraph.ManagementVertex; -import eu.stratosphere.nephele.managementgraph.ManagementVertexID; - -/** - * A profiling path is a path through the ManagementGraph, defined by a sequence - * of sequentially connected {@link ManagementVertex} and {@link ManagementEdge} objects. - * A profiling path may begin with a vertex or an edge an and may end with a vertex or an edge. - * By default profiling paths begin and end with vertices, but this can be changed with - * {{@link #setBeginVertexInProfilingPath(boolean)} and {{@link #setEndVertexInProfilingPath(boolean)}. - * - * @author Bjoern Lohrmann - */ -public class ProfilingPath implements Iterable { - - private ProfilingSubgraph graph; - - private LinkedList pathVertices; - - private HashMap ingoingEdges; - - private ArrayList pathElements; - - private ProfilingPathSummary summary; - - private boolean beginVertexInProfilingPath; - - private boolean endVertexInProfilingPath; - - @SuppressWarnings("unchecked") - public ProfilingPath(ProfilingPath toClone) { - this.graph = toClone.graph; - this.pathVertices = (LinkedList) toClone.pathVertices.clone(); - this.ingoingEdges = (HashMap) toClone.ingoingEdges.clone(); - this.beginVertexInProfilingPath = toClone.beginVertexInProfilingPath; - this.endVertexInProfilingPath = toClone.endVertexInProfilingPath; - } - - public ProfilingPath(ProfilingSubgraph graph, ManagementVertex firstVertex, boolean beginVertexInProfilingPath, - boolean endVertexInProfilingPath) { - this.graph = graph; - this.pathVertices = new LinkedList(); - this.ingoingEdges = new HashMap(); - this.pathVertices.add(firstVertex); - this.beginVertexInProfilingPath = beginVertexInProfilingPath; - this.endVertexInProfilingPath = endVertexInProfilingPath; - } - - public void appendVertex(ManagementVertex vertex, ManagementEdge ingoingEdge) { - pathVertices.add(vertex); - ingoingEdges.put(vertex.getID(), ingoingEdge); - } - - public ManagementVertex getBeginVertex() { - return pathVertices.getFirst(); - } - - public ManagementVertex getEndVertex() { - return pathVertices.getLast(); - } - - public void setBeginVertexInProfilingPath(boolean beginVertexInProfilingPath) { - // changing this on the fly will invalidate any already computed list of path elements - // and the summary - if (beginVertexInProfilingPath != this.beginVertexInProfilingPath) { - this.pathElements = null; - this.summary = null; - } - - this.beginVertexInProfilingPath = beginVertexInProfilingPath; - } - - public boolean isBeginVertexOnProfilingPath() { - return beginVertexInProfilingPath; - } - - public void setEndVertexInProfilingPath(boolean endVertexInProfilingPath) { - // changing this on the fly will invalidate any already computed list of path elements - // and the summary - if (endVertexInProfilingPath != this.endVertexInProfilingPath) { - this.pathElements = null; - this.summary = null; - } - - this.endVertexInProfilingPath = endVertexInProfilingPath; - } - - public boolean isEndVertexOnProfilingPath() { - return endVertexInProfilingPath; - } - - public ManagementEdge getIngoingEdge(ManagementVertex vertex) { - return ingoingEdges.get(vertex.getID()); - } - - public void removeLastVertex() { - ManagementVertex removed = pathVertices.removeLast(); - ingoingEdges.remove(removed); - } - - @Override - public Iterator iterator() { - return pathVertices.iterator(); - } - - public ProfilingPathSummary getSummary() { - ensurePathSummaryInitialized(); - return this.summary; - } - - private void ensurePathSummaryInitialized() { - if (this.summary == null) { - ensurePathElementsInitialized(); - this.summary = new ProfilingPathSummary(this.pathElements); - } - } - - private void ensurePathElementsInitialized() { - if (this.pathElements == null) { - this.pathElements = walkProfilingPath(); - } - } - - private ArrayList walkProfilingPath() { - ArrayList pathElements = new ArrayList(); - - for (ManagementVertex vertex : pathVertices) { - - ManagementEdge ingoingEdge = ingoingEdges.get(vertex.getID()); - if (ingoingEdge != null) { - pathElements.add(ingoingEdge); - } - pathElements.add(vertex); - } - - if (!isBeginVertexOnProfilingPath()) { - pathElements.remove(0); - } - - if (!isEndVertexOnProfilingPath()) { - pathElements.remove(pathElements.size() - 1); - } - - return pathElements; - } - - @Override - public String toString() { - StringBuilder builder = new StringBuilder(); - builder.append("LatencyPath["); - ManagementVertex previous = null; - for (ManagementVertex vertex : pathVertices) { - if (previous != null) { - builder.append("->"); - } - builder.append(vertex); - previous = vertex; - } - builder.append("]"); - - return builder.toString(); - } - - public ArrayList getPathElements() { - ensurePathElementsInitialized(); - return this.pathElements; - } - - // public void dumpLatencies() { - // - // for (ManagementVertex vertex : pathVertices) { - // ManagementEdge ingoing = ingoingEdges.get(vertex.getID()); - // - // if (ingoing != null) { - // System.out.printf("---edge(%.03f)---%s(%.03f)\n", - // ((EdgeCharacteristics) ingoing.getAttachment()).getLatencyInMillis(), - // vertex, - // ((VertexLatency) vertex.getAttachment()).getLatencyInMillis()); - // } else { - // System.out.printf("%s(%.03f)\n", vertex, - // ((VertexLatency) vertex.getAttachment()).getLatencyInMillis()); - // } - // } - // } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java deleted file mode 100644 index 739f123357225..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingPathSummary.java +++ /dev/null @@ -1,136 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import java.util.ArrayList; - -import eu.stratosphere.nephele.managementgraph.ManagementAttachment; -import eu.stratosphere.nephele.managementgraph.ManagementVertex; - -public class ProfilingPathSummary { - - // private static Log LOG = LogFactory.getLog(ProfilingPathSummary.class); - - private double totalLatency; - - private double[] latencies; - - private boolean hasLatencies; - - private ArrayList pathElements; - - private int noOfPathElementLatencies; - - /** - * Initializes ProfilingPathLatency. - * - * @param pathElements - * Elements (vertices, edges, ..) of the path in the order in which they appear in the path. - */ - public ProfilingPathSummary(ArrayList pathElements) { - this.pathElements = pathElements; - this.noOfPathElementLatencies = countLatencyValuesOnPath(); - this.latencies = new double[noOfPathElementLatencies]; - this.hasLatencies = false; - this.totalLatency = -1; - } - - private int countLatencyValuesOnPath() { - int valuesOnPath = 0; - for (ManagementAttachment element : pathElements) { - if (element instanceof ManagementVertex) { - valuesOnPath++; - } else { - valuesOnPath += 2; - } - } - return valuesOnPath; - } - - /** - * Returns whether we have latency values for all elements (vertices and edges) of this - * path. - * - * @return Whether we have latency values for all parts of this path - */ - public boolean hasLatencies() { - if (!this.hasLatencies) { - - this.hasLatencies = true; - for (ManagementAttachment element : pathElements) { - - if (element instanceof ManagementVertex) { - VertexLatency vertexLatency = (VertexLatency) element.getAttachment(); - if (vertexLatency.getLatencyInMillis() == -1) { - // ManagementEdge edge = (ManagementEdge) element; - // String sourceName = edge.getSource().getVertex().getName() + - // edge.getSource().getVertex().getIndexInGroup(); - // String tgName = edge.getTarget().getVertex().getName() + - // edge.getTarget().getVertex().getIndexInGroup(); - // LOG.info("no data for edge " + sourceName + "-> " + tgName); - this.hasLatencies = false; - break; - } - } else { - EdgeCharacteristics edgeChar = (EdgeCharacteristics) element.getAttachment(); - if (edgeChar.getChannelLatencyInMillis() == -1 || - edgeChar.getOutputBufferLatencyInMillis() == -1) { - // ManagementVertex vertex = (ManagementVertex) element; - // LOG.info("no data for vertex " + vertex.getName() + vertex.getIndexInGroup()); - this.hasLatencies = false; - break; - } - } - } - } - return hasLatencies; - } - - public void refreshLatencies() { - if (!hasLatencies()) { - throw new UnsupportedOperationException( - "Elements of profiling path do not have the necessary latency values yet"); - } - - this.totalLatency = 0; - int index = 0; - for (ManagementAttachment element : pathElements) { - - if (element instanceof ManagementVertex) { - latencies[index] = ((VertexLatency) element.getAttachment()).getLatencyInMillis(); - this.totalLatency += latencies[index]; - } else { - EdgeCharacteristics edgeCharacteristics = (EdgeCharacteristics) element.getAttachment(); - latencies[index] = edgeCharacteristics.getOutputBufferLatencyInMillis() / 2; - - if (latencies[index] < 0) { - throw new RuntimeException(ProfilingUtils.formatName(element) - + " has invalid negative output buffer latency: " + latencies[index]); - } - - index++; - // channel latency includes output buffer latency, hence we subtract the output buffer latency - // in order not to count it twice - latencies[index] = Math.max(0, edgeCharacteristics.getChannelLatencyInMillis() - latencies[index - 1]); - - this.totalLatency += latencies[index] + latencies[index - 1]; - } - index++; - } - } - - public double getTotalLatency() { - return this.totalLatency; - } - - public ArrayList getPathElements() { - return pathElements; - } - - public int getNoOfPathElementLatencies() { - return noOfPathElementLatencies; - } - - public double[] getPathElementLatencies() { - return latencies; - } - -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java deleted file mode 100644 index 698346cc4dae3..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSubgraph.java +++ /dev/null @@ -1,178 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; - -import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.executiongraph.ManagementGraphFactory; -import eu.stratosphere.nephele.managementgraph.ManagementEdge; -import eu.stratosphere.nephele.managementgraph.ManagementEdgeID; -import eu.stratosphere.nephele.managementgraph.ManagementGate; -import eu.stratosphere.nephele.managementgraph.ManagementGraph; -import eu.stratosphere.nephele.managementgraph.ManagementGraphIterator; -import eu.stratosphere.nephele.managementgraph.ManagementGroupVertex; -import eu.stratosphere.nephele.managementgraph.ManagementVertex; -import eu.stratosphere.nephele.managementgraph.ManagementVertexID; - -/** - * This class offers a way to find, store and compute the latencies of all possible paths between to - * {@link ExecutionGroupVertex} objects. Paths are computed on the {@link ExecutionVertex} level, not the - * {@link ExecutionGroupVertex} level, hence there may be many paths for high degrees of parallelization. - * - * @author Bjoern Lohrmann - */ -public class ProfilingSubgraph { - - // private static Log LOG = LogFactory.getLog(LatencySubgraph.class); - - private ManagementGroupVertex subgraphStart; - - private ManagementGroupVertex subgraphEnd; - - private List profilingPaths; - - private HashMap vertexLatencies = new HashMap(); - - private HashMap edgeCharacteristics = new HashMap(); - - private HashMap xoredVertexToSourceEdgeIDMap = new HashMap(); - - public ProfilingSubgraph(ExecutionGraph executionGraph, ExecutionGroupVertex subgraphStart, - ExecutionGroupVertex subgraphEnd, boolean includeSubgraphStartInProfilingPaths, - boolean includeSubgraphEndInProfilingPaths) { - - ManagementGraph managementGraph = ManagementGraphFactory.fromExecutionGraph(executionGraph); - determineAnchoringManagementGroupVertices(managementGraph, subgraphStart, subgraphEnd); - buildProfilingPaths(includeSubgraphStartInProfilingPaths, includeSubgraphEndInProfilingPaths); - initProfilingAttachmentsOnPaths(); - - // FIXME this is a workaround and not safe for multi-DAGs - initReceiverVertexToSourceEdgeIDMap(managementGraph); - } - - private void initProfilingAttachmentsOnPaths() { - for (ProfilingPath path : profilingPaths) { - initProfilingAttachmentOnPath(path); - } - } - - private void initProfilingAttachmentOnPath(ProfilingPath path) { - - for (ManagementVertex vertex : path) { - if (vertex.getAttachment() == null) { - VertexLatency vertexLatency = new VertexLatency(vertex); - vertex.setAttachment(vertexLatency); - vertexLatencies.put(vertex.getID(), vertexLatency); - } - - ManagementEdge ingoingEdge = path.getIngoingEdge(vertex); - if (ingoingEdge != null && ingoingEdge.getAttachment() == null) { - EdgeCharacteristics characteristics = new EdgeCharacteristics(ingoingEdge); - ingoingEdge.setAttachment(characteristics); - edgeCharacteristics.put(ingoingEdge.getSourceEdgeID(), characteristics); - edgeCharacteristics.put(ingoingEdge.getTargetEdgeID(), characteristics); - } - } - } - - private void initReceiverVertexToSourceEdgeIDMap(final ManagementGraph managementGraph) { - - // FIXME this is a workaround and not safe for multi-DAGs - final Iterator it = new ManagementGraphIterator(managementGraph, true); - while (it.hasNext()) { - - final ManagementVertex source = it.next(); - final int numberOfOutputGates = source.getNumberOfOutputGates(); - for (int i = 0; i < numberOfOutputGates; ++i) { - final ManagementGate outputGate = source.getOutputGate(i); - final int numberOfOutgoingEdges = outputGate.getNumberOfForwardEdges(); - for (int j = 0; j < numberOfOutgoingEdges; ++j) { - final ManagementEdge edge = outputGate.getForwardEdge(j); - final ManagementVertex receiver = edge.getTarget().getVertex(); - - XoredVertexID xored = new XoredVertexID(source.getID(), receiver.getID()); - // System.out.println("putting edge " + ProfilingUtils.formatName(edge) + " " + xored.toString()); - this.xoredVertexToSourceEdgeIDMap.put(xored, edge.getSourceEdgeID()); - } - } - } - } - - private void buildProfilingPaths(boolean includeSubgraphStartInProfilingPaths, - boolean includeSubgraphEndInProfilingPaths) { - - this.profilingPaths = new LinkedList(); - - for (int i = 0; i < subgraphStart.getNumberOfGroupMembers(); i++) { - ManagementVertex vertex = subgraphStart.getGroupMember(i); - ProfilingPath initialPath = new ProfilingPath(this, vertex, includeSubgraphStartInProfilingPaths, - includeSubgraphEndInProfilingPaths); - depthFirstSearchProfilingPaths(initialPath, this.profilingPaths); - } - } - - /** - * Performs a recursive depth first search for {@link #subgraphEnd} starting at the end of the given path. - * All paths found to end in {@link #subgraphEnd} are added to foundProfilingPaths. - * - * @param path - * Initial path with at least one element to start with (will be altered during recursive search). - * @param foundProfilingPaths - * Accumulates the paths found to end at {@link #subgraphEnd} - */ - private void depthFirstSearchProfilingPaths(ProfilingPath path, List foundProfilingPaths) { - ManagementVertex pathEnd = path.getEndVertex(); - - for (int i = 0; i < pathEnd.getNumberOfOutputGates(); i++) { - ManagementGate outputGate = pathEnd.getOutputGate(i); - - for (int j = 0; j < outputGate.getNumberOfForwardEdges(); j++) { - ManagementEdge edge = outputGate.getForwardEdge(j); - - ManagementVertex extension = edge.getTarget().getVertex(); - - path.appendVertex(extension, edge); - - if (extension.getGroupVertex() == subgraphEnd) { - foundProfilingPaths.add(new ProfilingPath(path)); - } else { - depthFirstSearchProfilingPaths(path, foundProfilingPaths); - } - - path.removeLastVertex(); - } - } - } - - private void determineAnchoringManagementGroupVertices(ManagementGraph managementGraph, - ExecutionGroupVertex pathBeginExecVertex, - ExecutionGroupVertex pathEndExecVertex) { - - ManagementVertexID vertexInPathBeginGroup = pathBeginExecVertex.getGroupMember(0).getID() - .toManagementVertexID(); - this.subgraphStart = managementGraph.getVertexByID(vertexInPathBeginGroup).getGroupVertex(); - - ManagementVertexID vertexInPathEndGroup = pathEndExecVertex.getGroupMember(0).getID().toManagementVertexID(); - this.subgraphEnd = managementGraph.getVertexByID(vertexInPathEndGroup).getGroupVertex(); - } - - public ManagementEdgeID getSourceEdgeIDByXoredVertexID(XoredVertexID xored) { - return xoredVertexToSourceEdgeIDMap.get(xored); - } - - public EdgeCharacteristics getEdgeCharacteristicsBySourceEdgeID(ManagementEdgeID sourceEdgeID) { - return edgeCharacteristics.get(sourceEdgeID); - } - - public VertexLatency getVertexLatency(ManagementVertexID managementVertexID) { - return vertexLatencies.get(managementVertexID); - } - - public List getProfilingPaths() { - return profilingPaths; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java deleted file mode 100644 index 91c664124e53c..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingSummary.java +++ /dev/null @@ -1,124 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import java.util.ArrayList; -import java.util.Collections; - -import eu.stratosphere.nephele.managementgraph.ManagementAttachment; - -public class ProfilingSummary { - - private ArrayList pathElements; - - private int noOfActivePaths; - - private int noOfInactivePaths; - - private double avgTotalPathLatency; - - private double medianPathLatency; - - private double minPathLatency; - - private double maxPathLatency; - - private double[] avgPathElementLatencies; - - private ArrayList activePaths; - - public ProfilingSummary(ProfilingSubgraph profilingSubgraph) { - noOfActivePaths = 0; - noOfInactivePaths = 0; - avgTotalPathLatency = 0; - minPathLatency = 0; - maxPathLatency = 0; - medianPathLatency = 0; - activePaths = new ArrayList(); - - pathElements = null; - avgPathElementLatencies = null; - - // will be sorted later on to determine min, max and median - ArrayList totalLatencies = new ArrayList(); - - for (ProfilingPath path : profilingSubgraph.getProfilingPaths()) { - ProfilingPathSummary pathSummary = path.getSummary(); - - if (pathElements == null) { - pathElements = pathSummary.getPathElements(); - avgPathElementLatencies = new double[pathSummary.getNoOfPathElementLatencies()]; - } - - if (pathSummary.hasLatencies()) { - activePaths.add(path); - - // refresh the latency values in the summary - pathSummary.refreshLatencies(); - - avgTotalPathLatency += pathSummary.getTotalLatency(); - totalLatencies.add(pathSummary.getTotalLatency()); - - // add the vertex/edge specific latency values to avgPathLatencies array - addValues(pathSummary.getPathElementLatencies(), avgPathElementLatencies); - - noOfActivePaths++; - } else { - noOfInactivePaths++; - } - } - - if (noOfActivePaths > 0) { - for (int i = 0; i < avgPathElementLatencies.length; i++) { - avgPathElementLatencies[i] = avgPathElementLatencies[i] / noOfActivePaths; - } - - avgTotalPathLatency = avgTotalPathLatency / noOfActivePaths; - - Collections.sort(totalLatencies); - minPathLatency = totalLatencies.get(0); - medianPathLatency = totalLatencies.get(totalLatencies.size() / 2); - maxPathLatency = totalLatencies.get(totalLatencies.size() -1); - } - } - - private void addValues(double[] from, double[] to) { - for (int i = 0; i < from.length; i++) { - to[i] += from[i]; - } - } - - public ArrayList getPathElements() { - return pathElements; - } - - public int getNoOfActivePaths() { - return noOfActivePaths; - } - - public int getNoOfInactivePaths() { - return noOfInactivePaths; - } - - public double getAvgTotalPathLatency() { - return avgTotalPathLatency; - } - - public double getMedianPathLatency() { - return medianPathLatency; - } - - public double getMinPathLatency() { - return minPathLatency; - } - - public double getMaxPathLatency() { - return maxPathLatency; - } - - public double[] getAvgPathElementLatencies() { - return avgPathElementLatencies; - } - - public ArrayList getActivePaths() { - return activePaths; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java deleted file mode 100644 index 7d93830decf75..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingUtils.java +++ /dev/null @@ -1,52 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.managementgraph.ManagementAttachment; -import eu.stratosphere.nephele.managementgraph.ManagementEdge; -import eu.stratosphere.nephele.managementgraph.ManagementVertex; - -public class ProfilingUtils { - - public static String formatName(ManagementAttachment managementAttachment) { - if (managementAttachment instanceof ManagementVertex) { - return formatName((ManagementVertex) managementAttachment); - } else { - return formatName((ManagementEdge) managementAttachment); - } - } - - public static String formatName(ManagementEdge edge) { - return formatName(edge.getSource().getVertex()) + "->" + formatName(edge.getTarget().getVertex()); - } - - public static String formatName(ManagementVertex vertex) { - String name = vertex.getName(); - for (int i = 0; i < vertex.getGroupVertex().getNumberOfGroupMembers(); i++) { - if (vertex.getGroupVertex().getGroupMember(i) == vertex) { - name += i; - break; - } - } - return name; - } - - public static String formatName(ExecutionVertex vertex) { - String name = vertex.getName(); - for (int i = 0; i < vertex.getGroupVertex().getCurrentNumberOfGroupMembers(); i++) { - if (vertex.getGroupVertex().getGroupMember(i) == vertex) { - name += i; - break; - } - } - return name; - } - - public static long alignToNextFullSecond(long timestampInMillis) { - long remainder = timestampInMillis % 1000; - - if (remainder > 0) { - return timestampInMillis - remainder + 1000; - } - return timestampInMillis; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValue.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValue.java deleted file mode 100644 index 1c1fb183d7d81..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValue.java +++ /dev/null @@ -1,67 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -public class ProfilingValue implements Comparable { - - private static long nextFreeId = 0; - - private long id; - - private double value; - - private long timestamp; - - public ProfilingValue(double value, long timestamp) { - this.value = value; - this.timestamp = timestamp; - this.id = nextFreeId++; - } - - public double getValue() { - return value; - } - - public void setValue(double value) { - this.value = value; - } - - public long getTimestamp() { - return timestamp; - } - - public void setTimestamp(long timestamp) { - this.timestamp = timestamp; - } - - public long getId() { - return id; - } - - /** - * Sorts first by value and then by id. - */ - @Override - public int compareTo(ProfilingValue other) { - if (this.value > other.value) { - return 1; - } else if (this.value < other.value) { - return -1; - } else { - if (this.id > other.id) { - return 1; - } else if (this.id < other.id) { - return -1; - } else { - return 0; - } - } - } - - public boolean equals(Object otherObj) { - if (otherObj instanceof ProfilingValue) { - ProfilingValue other = (ProfilingValue) otherObj; - return other.id == this.id; - } else { - return false; - } - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java deleted file mode 100644 index dcd2a5fba4539..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/ProfilingValueStatistic.java +++ /dev/null @@ -1,119 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.LinkedList; - -public class ProfilingValueStatistic { - - private ArrayList sortedByValue; - - private LinkedList sortedById; - - private int statisticWindowSize; - - private int noOfStoredValues; - - private double sumOfValues; - - public ProfilingValueStatistic(int statisticWindowSize) { - this.sortedById = new LinkedList(); - this.sortedByValue = new ArrayList(); - this.statisticWindowSize = statisticWindowSize; - this.noOfStoredValues = 0; - this.sumOfValues = 0; - } - - public void addValue(ProfilingValue value) { - ProfilingValue droppedValue = insertIntoSortedByTimestamp(value); - - if (droppedValue != null) { - removeFromSortedByValue(droppedValue); - noOfStoredValues--; - sumOfValues -= droppedValue.getValue(); - } - - insertIntoSortedByValue(value); - noOfStoredValues++; - sumOfValues += value.getValue(); - } - - private ProfilingValue insertIntoSortedByTimestamp(ProfilingValue value) { - if (!sortedById.isEmpty() && sortedById.getLast().getId() >= value.getId()) { - throw new IllegalArgumentException("Trying to add stale profiling values. This should not happen."); - } - sortedById.add(value); - - if (noOfStoredValues >= statisticWindowSize) { - return sortedById.removeFirst(); - } else { - return null; - } - } - - protected void insertIntoSortedByValue(ProfilingValue value) { - int insertionIndex = Collections.binarySearch(sortedByValue, value); - if (insertionIndex < 0) { - insertionIndex = -(insertionIndex + 1); - } - - sortedByValue.add(insertionIndex, value); - } - - protected void removeFromSortedByValue(ProfilingValue toRemove) { - int removeIndex = Collections.binarySearch(sortedByValue, toRemove); - if (removeIndex < 0) { - throw new IllegalArgumentException("Trying to drop inexistant profiling value. This should not happen."); - } - sortedByValue.remove(removeIndex); - } - - public double getMedianValue() { - if (noOfStoredValues == 0) { - throw new RuntimeException("Cannot calculate median of empty value set"); - } - - int medianIndex = noOfStoredValues / 2; - return sortedByValue.get(medianIndex).getValue(); - } - - public double getMaxValue() { - if (noOfStoredValues == 0) { - throw new RuntimeException("Cannot calculate the max value of empty value set"); - } - return sortedByValue.get(noOfStoredValues - 1).getValue(); - } - - public double getMinValue() { - if (noOfStoredValues == 0) { - throw new RuntimeException("Cannot calculate the min value of empty value set"); - } - return sortedByValue.get(0).getValue(); - } - - public ProfilingValue getOldestValue() { - if (noOfStoredValues == 0) { - throw new RuntimeException("Cannot get the oldest value of empty value set"); - } - return sortedById.getFirst(); - } - - public ProfilingValue getNewestValue() { - if (noOfStoredValues == 0) { - throw new RuntimeException("Cannot get the newest value of empty value set"); - } - return sortedById.getLast(); - } - - public double getArithmeticMean() { - if (noOfStoredValues == 0) { - throw new RuntimeException("Cannot calculate the arithmetic mean of empty value set"); - } - - return sumOfValues / noOfStoredValues; - } - - public boolean hasValues() { - return noOfStoredValues > 0; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/VertexLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/VertexLatency.java deleted file mode 100644 index 53b2e8b8941e6..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/VertexLatency.java +++ /dev/null @@ -1,37 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import eu.stratosphere.nephele.managementgraph.ManagementVertex; - -public class VertexLatency { - - private ManagementVertex vertex; - - private ProfilingValueStatistic latencyStatistics; - - public VertexLatency(ManagementVertex vertex) { - this.vertex = vertex; - this.latencyStatistics = new ProfilingValueStatistic(10); - } - - public ManagementVertex getVertex() { - return vertex; - } - - public double getLatencyInMillis() { - if (latencyStatistics.hasValues()) { - return latencyStatistics.getArithmeticMean(); - } else { - return -1; - } - } - - public void addLatencyMeasurement(long timestamp, double latencyInMillis) { - ProfilingValue value = new ProfilingValue(latencyInMillis, timestamp); - latencyStatistics.addValue(value); - } - - @Override - public String toString() { - return String.format("VertexLatency[%s|%.03f]", vertex.toString(), getLatencyInMillis()); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/XoredVertexID.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/XoredVertexID.java deleted file mode 100644 index 608c20b6f01e9..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/profiling/XoredVertexID.java +++ /dev/null @@ -1,40 +0,0 @@ -package eu.stratosphere.nephele.streaming.profiling; - -import java.io.IOException; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.AbstractID; -import eu.stratosphere.nephele.io.DataOutputBuffer; -import eu.stratosphere.nephele.managementgraph.ManagementVertexID; - -public class XoredVertexID extends AbstractID { - - public XoredVertexID(ExecutionVertexID one, ExecutionVertexID two) { - super(xorAbstractIDs(one, two)); - } - - public XoredVertexID(ManagementVertexID one, ManagementVertexID two) { - super(xorAbstractIDs(one, two)); - } - - private static byte[] xorAbstractIDs(AbstractID one, AbstractID two) { - DataOutputBuffer buffer = new DataOutputBuffer(16); - try { - one.write(buffer); - - byte[] data = new byte[16]; - System.arraycopy(buffer.getData().array(), 0, data, 0, 16); - buffer.reset(); - two.write(buffer); - - byte[] twoData = buffer.getData().array(); - for (int i = 0; i < 16; i++) { - data[i] = (byte) (data[i] ^ twoData[i]); - } - return data; - } catch (IOException e) { - e.printStackTrace(); - } - return null; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/AbstractStreamingData.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/AbstractStreamingData.java deleted file mode 100644 index 6919ffa4abe3d..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/AbstractStreamingData.java +++ /dev/null @@ -1,79 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.types; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.io.IOReadableWritable; -import eu.stratosphere.nephele.jobgraph.JobID; - -/** - * Abstract base class to be used to for exchanging data between the different - * components of the streaming plugin. - * - * @author warneke - */ -public abstract class AbstractStreamingData implements IOReadableWritable { - - /** - * The ID of the job this piece of streaming data refers to - */ - private final JobID jobID; - - public AbstractStreamingData(JobID jobID) { - if (jobID == null) { - throw new IllegalArgumentException("jobID must not be null"); - } - - this.jobID = jobID; - } - - /** - * Default constructor required for deserilization. - */ - public AbstractStreamingData() { - - this.jobID = new JobID(); - } - - /** - * Returns the ID of the job this path latency information refers to. - * - * @return the ID of the job this path latency information refers to - */ - public JobID getJobID() { - - return this.jobID; - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - this.jobID.write(out); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - this.jobID.read(in); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelLatency.java deleted file mode 100644 index 32efd82811cda..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelLatency.java +++ /dev/null @@ -1,154 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.types; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.jobgraph.JobID; - -/** - * This class stores information about the latency of a specific channel from a source to a sink vertex. - * - * @author warneke - */ -public final class ChannelLatency extends AbstractStreamingData { - - /** - * The ID of the vertex representing the source of the channel. - */ - private final ExecutionVertexID sourceVertexID; - - /** - * The ID of the vertex representing the sink of the channel. - */ - private final ExecutionVertexID sinkVertexID; - - /** - * The channel latency in milliseconds - */ - private double channelLatency; - - /** - * Constructs a new path latency object. - * - * @param jobID - * the ID of the job this channel latency information refers to - * @param sourceVertexID - * the ID of the vertex representing the source of the channel - * @param sinkVertexID - * the ID of the vertex representing the sink of the channel - * @param pathLatency - * the path latency in milliseconds - */ - public ChannelLatency(final JobID jobID, final ExecutionVertexID sourceVertexID, - final ExecutionVertexID sinkVertexID, final double channelLatency) { - - super(jobID); - - if (sourceVertexID == null) { - throw new IllegalArgumentException("sourceVertexID must not be null"); - } - - if (sinkVertexID == null) { - throw new IllegalArgumentException("sinkVertexID must not be null"); - } - - this.sourceVertexID = sourceVertexID; - this.sinkVertexID = sinkVertexID; - this.channelLatency = channelLatency; - } - - /** - * Default constructor for the deserialization of the object. - */ - public ChannelLatency() { - super(new JobID()); - this.sourceVertexID = new ExecutionVertexID(); - this.sinkVertexID = new ExecutionVertexID(); - this.channelLatency = 0.0; - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - super.write(out); - this.sourceVertexID.write(out); - this.sinkVertexID.write(out); - out.writeDouble(this.channelLatency); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - super.read(in); - this.sourceVertexID.read(in); - this.sinkVertexID.read(in); - this.channelLatency = in.readDouble(); - } - - /** - * Returns the ID of the vertex representing the source of the channel. - * - * @return the ID of the vertex representing the source of the channel - */ - public ExecutionVertexID getSourceVertexID() { - - return this.sourceVertexID; - } - - /** - * Returns the ID of the vertex representing the sink of the channel. - * - * @return the ID of the vertex representing the sink of the channel - */ - public ExecutionVertexID getSinkVertexID() { - - return this.sinkVertexID; - } - - /** - * Returns the channel latency in milliseconds. - * - * @return the channel latency in milliseconds - */ - public double getChannelLatency() { - - return this.channelLatency; - } - - /** - * {@inheritDoc} - */ - @Override - public String toString() { - - final StringBuilder str = new StringBuilder(); - str.append(this.sourceVertexID.toString()); - str.append(" -> "); - str.append(this.sinkVertexID.toString()); - str.append(": "); - str.append(this.channelLatency); - - return str.toString(); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelThroughput.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelThroughput.java deleted file mode 100644 index de6450373637d..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/ChannelThroughput.java +++ /dev/null @@ -1,144 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.types; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.jobgraph.JobID; - -/** - * This class stores information about the throughput of a specific output channel. - * - * @author warneke - */ -public final class ChannelThroughput extends AbstractStreamingData { - - /** - * The ID of the vertex which is connected to this output channel. - */ - private final ExecutionVertexID vertexID; - - /** - * The ID of the output channel. - */ - private final ChannelID sourceChannelID; - - /** - * The throughput in MBit/s. - */ - private double throughput; - - /** - * Constructs a new channel throughput object. - * - * @param jobID - * the ID of the job this channel throughput object belongs to - * @param vertexID - * the ID of the vertex which is connected to this output channel - * @param sourceChannelID - * the ID of the output channel - * @param throughput - * the throughput in MBit/s - */ - public ChannelThroughput(final JobID jobID, final ExecutionVertexID vertexID, final ChannelID sourceChannelID, - final double throughput) { - super(jobID); - - if (vertexID == null) { - throw new IllegalArgumentException("Argument vertexID must not be null"); - } - - if (sourceChannelID == null) { - throw new IllegalArgumentException("Argument sourceChannelID must not be null"); - } - - if (throughput < 0.0) { - throw new IllegalArgumentException("Argument throughput must not be positive"); - } - - this.vertexID = vertexID; - this.sourceChannelID = sourceChannelID; - this.throughput = throughput; - } - - /** - * Default constructor for deserialization. - */ - public ChannelThroughput() { - super(new JobID()); - this.vertexID = new ExecutionVertexID(); - this.sourceChannelID = new ChannelID(); - this.throughput = 0.0; - } - - /** - * The ID of the vertex which is connected to the output channel. - * - * @return the ID of the vertex which is connected to the output channel - */ - public ExecutionVertexID getVertexID() { - - return this.vertexID; - } - - /** - * The ID of the output channel. - * - * @return the ID of the output channel. - */ - public ChannelID getSourceChannelID() { - - return this.sourceChannelID; - } - - /** - * Returns the measured throughput for the channel in MBit/s. - * - * @return the measured throughput in MBit/s. - */ - public double getThroughput() { - - return this.throughput; - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - super.write(out); - - this.vertexID.write(out); - this.sourceChannelID.write(out); - out.writeDouble(this.throughput); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - super.read(in); - - this.vertexID.read(in); - this.sourceChannelID.read(in); - this.throughput = in.readDouble(); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java deleted file mode 100644 index 4ff9e9ac2bfd2..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/OutputBufferLatency.java +++ /dev/null @@ -1,102 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.types; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.jobgraph.JobID; - -public final class OutputBufferLatency extends AbstractStreamingData { - - private final ExecutionVertexID vertexID; - - private final ChannelID sourceChannelID; - - private int bufferLatency; - - public OutputBufferLatency(final JobID jobID, final ExecutionVertexID vertexID, final ChannelID sourceChannelID, - final int bufferLatency) { - super(jobID); - - if (vertexID == null) { - throw new IllegalArgumentException("Argument vertexID must not be null"); - } - - if (sourceChannelID == null) { - throw new IllegalArgumentException("Argument sourceChannelID must not be null"); - } - - if (bufferLatency < 0) { - throw new IllegalArgumentException("Argument bufferLatency must be greater than or equal to zero but is " - + bufferLatency); - } - - this.vertexID = vertexID; - this.sourceChannelID = sourceChannelID; - this.bufferLatency = bufferLatency; - } - - public OutputBufferLatency() { - super(); - - this.vertexID = new ExecutionVertexID(); - this.sourceChannelID = new ChannelID(); - this.bufferLatency = 0; - } - - public ExecutionVertexID getVertexID() { - - return this.vertexID; - } - - public ChannelID getSourceChannelID() { - - return this.sourceChannelID; - } - - public int getBufferLatency() { - - return this.bufferLatency; - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - super.write(out); - - this.vertexID.write(out); - this.sourceChannelID.write(out); - out.writeInt(this.bufferLatency); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - super.read(in); - - this.vertexID.read(in); - this.sourceChannelID.read(in); - this.bufferLatency = in.readInt(); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/TaskLatency.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/TaskLatency.java deleted file mode 100644 index 18afb543c6011..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/types/TaskLatency.java +++ /dev/null @@ -1,112 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.types; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.jobgraph.JobID; - -/** - * This class stores information about the latency of a specific (sub) path from a start to an end vertex. - * - * @author warneke - */ -public final class TaskLatency extends AbstractStreamingData { - - /** - * The ID of the vertex this task latency information refers to - */ - private final ExecutionVertexID vertexID; - - /** - * The task latency in milliseconds - */ - private double taskLatency; - - /** - * Constructs a new task latency object. - * - * @param jobID - * the ID of the job this path latency information refers to - * @param vertexID - * the ID of the vertex this task latency information refers to - * @param taskLatency - * the task latency in milliseconds - */ - public TaskLatency(final JobID jobID, final ExecutionVertexID vertexID, final double taskLatency) { - - super(jobID); - - if (vertexID == null) { - throw new IllegalArgumentException("vertexID must not be null"); - } - - this.vertexID = vertexID; - this.taskLatency = taskLatency; - } - - /** - * Default constructor for the deserialization of the object. - */ - public TaskLatency() { - super(new JobID()); - this.vertexID = new ExecutionVertexID(); - this.taskLatency = 0.0; - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - super.write(out); - this.vertexID.write(out); - out.writeDouble(this.taskLatency); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - super.read(in); - this.vertexID.read(in); - this.taskLatency = in.readDouble(); - } - - /** - * Returns the ID of the vertex this task latency information refers to. - * - * @return the ID of the vertex this task latency information refers to - */ - public ExecutionVertexID getVertexID() { - - return this.vertexID; - } - - /** - * Returns the task latency in milliseconds. - * - * @return the task latency in milliseconds - */ - public double getTaskLatency() { - - return this.taskLatency; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java deleted file mode 100644 index cf694d6d3100f..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingEnvironment.java +++ /dev/null @@ -1,114 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.wrappers; - -import java.util.ArrayList; -import java.util.List; - -import eu.stratosphere.nephele.execution.Environment; -import eu.stratosphere.nephele.execution.Mapper; -import eu.stratosphere.nephele.io.ChannelSelector; -import eu.stratosphere.nephele.io.DistributionPattern; -import eu.stratosphere.nephele.io.GateID; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.RecordDeserializer; -import eu.stratosphere.nephele.plugins.wrapper.AbstractEnvironmentWrapper; -import eu.stratosphere.nephele.streaming.listeners.StreamListener; -import eu.stratosphere.nephele.types.Record; - -/** - * A streaming environment wraps the created input and output gates in special {@link StreamingInputGate} and - * {@link StreamingOutputGate} objects to intercept particular methods calls necessary for the statistics collection. - *

- * This class is thread-safe. - * - * @author warneke - */ -public final class StreamingEnvironment extends AbstractEnvironmentWrapper { - - private final StreamListener streamListener; - - private final List> streamingInputGates = new ArrayList>(); - - private final List> streamingOutputGates = new ArrayList>(); - - /** - * Constructs a new streaming environment - * - * @param wrappedEnvironment - * the environment to be encapsulated by this streaming environment - * @param streamListener - * the stream listener - */ - StreamingEnvironment(final Environment wrappedEnvironment, final StreamListener streamListener) { - super(wrappedEnvironment); - - this.streamListener = streamListener; - } - - /** - * {@inheritDoc} - */ - @SuppressWarnings({ "rawtypes", "unchecked" }) - @Override - public OutputGate createOutputGate(final GateID gateID, - final Class outputClass, final ChannelSelector selector, - final boolean isBroadcast) { - - final OutputGate outputGate = getWrappedEnvironment().createOutputGate(gateID, outputClass, - selector, isBroadcast); - - final StreamingOutputGate sog = new StreamingOutputGate(outputGate, this.streamListener); - this.streamingOutputGates.add(sog); - - return sog; - } - - /** - * {@inheritDoc} - */ - @SuppressWarnings({ "rawtypes", "unchecked" }) - @Override - public InputGate createInputGate(final GateID gateID, - final RecordDeserializer deserializer, final DistributionPattern distributionPattern) { - - final InputGate inputGate = getWrappedEnvironment().createInputGate(gateID, deserializer, - distributionPattern); - - final StreamingInputGate sig = new StreamingInputGate(inputGate, this.streamListener); - this.streamingInputGates.add(sig); - - return sig; - } - - /** - * {@inheritDoc} - */ - @Override - public void registerMapper(final Mapper mapper) { - - if (this.streamingInputGates.size() != 1) { - return; - } - - if (this.streamingOutputGates.size() != 1) { - return; - } - - this.streamListener.registerMapper(mapper, this.streamingInputGates.get(0), this.streamingOutputGates.get(0)); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileInputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileInputWrapper.java deleted file mode 100644 index f4c3d4a2ce734..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileInputWrapper.java +++ /dev/null @@ -1,61 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.wrappers; - -import eu.stratosphere.nephele.streaming.listeners.StreamListener; -import eu.stratosphere.nephele.template.AbstractFileInputTask; -import eu.stratosphere.nephele.template.AbstractInvokable; - -/** - * This class provides a wrapper for Nephele tasks of the type {@link AbstractFileInputTask}. - *

- * This class is thread-safe. - * - * @author warneke - */ -public final class StreamingFileInputWrapper extends AbstractFileInputTask { - - /** - * The wrapped task. - */ - private volatile AbstractInvokable wrappedInvokable = null; - - /** - * The stream listener object. - */ - private volatile StreamListener streamListener = null; - - /** - * {@inheritDoc} - */ - @Override - public void registerInputOutput() { - - this.streamListener = WrapperUtils.createStreamListener(getEnvironment()); - this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment(), this.streamListener); - this.wrappedInvokable.registerInputOutput(); - } - - /** - * {@inheritDoc} - */ - @Override - public void invoke() throws Exception { - - this.streamListener.init(); - this.wrappedInvokable.invoke(); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileOutputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileOutputWrapper.java deleted file mode 100644 index 4db0e2b64ea0d..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingFileOutputWrapper.java +++ /dev/null @@ -1,61 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.wrappers; - -import eu.stratosphere.nephele.streaming.listeners.StreamListener; -import eu.stratosphere.nephele.template.AbstractFileOutputTask; -import eu.stratosphere.nephele.template.AbstractInvokable; - -/** - * This class provides a wrapper for Nephele tasks of the type {@link AbstractFileOutputTask}. - *

- * This class is thread-safe. - * - * @author warneke - */ -public final class StreamingFileOutputWrapper extends AbstractFileOutputTask { - - /** - * The wrapped task. - */ - private volatile AbstractInvokable wrappedInvokable = null; - - /** - * The stream listener object. - */ - private volatile StreamListener streamListener = null; - - /** - * {@inheritDoc} - */ - @Override - public void registerInputOutput() { - - this.streamListener = WrapperUtils.createStreamListener(getEnvironment()); - this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment(), this.streamListener); - this.wrappedInvokable.registerInputOutput(); - } - - /** - * {@inheritDoc} - */ - @Override - public void invoke() throws Exception { - - this.streamListener.init(); - this.wrappedInvokable.invoke(); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java deleted file mode 100644 index c7f9fd80dfcbd..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputGate.java +++ /dev/null @@ -1,171 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.wrappers; - -import java.io.EOFException; -import java.io.IOException; -import java.util.ArrayDeque; - -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.RecordAvailabilityListener; -import eu.stratosphere.nephele.plugins.wrapper.AbstractInputGateWrapper; -import eu.stratosphere.nephele.streaming.listeners.StreamListener; -import eu.stratosphere.nephele.types.Record; - -public final class StreamingInputGate extends AbstractInputGateWrapper { - - private final StreamListener streamListener; - - /** - * Queue with indices of channels that store at least one available record. - */ - private final ArrayDeque availableChannels = new ArrayDeque(); - - /** - * The channel to read from next. - */ - private int channelToReadFrom = -1; - - /** - * The value returned by the last call of waitForAnyChannelToBecomeAvailable - */ - private int availableChannelRetVal = -1; - - /** - * The thread which executes the task connected to the input gate. - */ - private Thread executingThread = null; - - StreamingInputGate(final InputGate wrappedInputGate, final StreamListener streamListener) { - super(wrappedInputGate); - - if (streamListener == null) { - throw new IllegalArgumentException("Argument streamListener must not be null"); - } - - this.streamListener = streamListener; - } - - /** - * {@inheritDoc} - */ - @Override - public T readRecord(final T target) throws IOException, InterruptedException { - - T record = null; - - if (this.executingThread == null) { - this.executingThread = Thread.currentThread(); - } - - if (this.executingThread.isInterrupted()) { - throw new InterruptedException(); - } - - final int numberOfInputChannels = getNumberOfInputChannels(); - - while (true) { - - if (this.channelToReadFrom == -1) { - this.availableChannelRetVal = waitForAnyChannelToBecomeAvailable(); - this.channelToReadFrom = this.availableChannelRetVal; - } - try { - record = this.getInputChannel(this.channelToReadFrom).readRecord(target); - } catch (EOFException e) { - // System.out.println("### Caught EOF exception at channel " + channelToReadFrom + "(" + - // this.getInputChannel(channelToReadFrom).getType().toString() + ")"); - if (this.isClosed()) { - return null; - } - } - - if (record == null && this.channelToReadFrom == this.availableChannelRetVal) { - this.channelToReadFrom = -1; - continue; - } - - if (++this.channelToReadFrom == numberOfInputChannels) { - this.channelToReadFrom = 0; - } - - if (record != null) { - break; - } - } - - reportRecordReceived(record); - - return record; - } - - public void reportRecordReceived(final Record record) { - - this.streamListener.recordReceived(record); - } - - /** - * This method returns the index of a channel which has at least - * one record available. The method may block until at least one - * channel has become ready. - * - * @return the index of the channel which has at least one record available - */ - public int waitForAnyChannelToBecomeAvailable() throws InterruptedException { - - synchronized (this.availableChannels) { - - while (this.availableChannels.isEmpty()) { - this.availableChannels.wait(); - } - - return this.availableChannels.removeFirst().intValue(); - } - } - - /** - * {@inheritDoc} - */ - @Override - public void notifyRecordIsAvailable(final int channelIndex) { - - synchronized (this.availableChannels) { - - this.availableChannels.add(Integer.valueOf(channelIndex)); - this.availableChannels.notify(); - } - } - - /** - * {@inheritDoc} - */ - @Override - public void registerRecordAvailabilityListener(final RecordAvailabilityListener listener) { - - // TODO: Implement me - } - - /** - * {@inheritDoc} - */ - @Override - public boolean hasRecordAvailable() throws IOException, InterruptedException { - - // TODO: Implement me - - return false; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputWrapper.java deleted file mode 100644 index 308320762996f..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingInputWrapper.java +++ /dev/null @@ -1,61 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.wrappers; - -import eu.stratosphere.nephele.streaming.listeners.StreamListener; -import eu.stratosphere.nephele.template.AbstractGenericInputTask; -import eu.stratosphere.nephele.template.AbstractInvokable; - -/** - * This class provides a wrapper for Nephele tasks of the type {@link AbstractGenericInputTask}. - *

- * This class is thread-safe. - * - * @author warneke - */ -public final class StreamingInputWrapper extends AbstractGenericInputTask { - - /** - * The wrapped task. - */ - private volatile AbstractInvokable wrappedInvokable = null; - - /** - * The stream listener object. - */ - private volatile StreamListener streamListener = null; - - /** - * {@inheritDoc} - */ - @Override - public void registerInputOutput() { - - this.streamListener = WrapperUtils.createStreamListener(getEnvironment()); - this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment(), this.streamListener); - this.wrappedInvokable.registerInputOutput(); - } - - /** - * {@inheritDoc} - */ - @Override - public void invoke() throws Exception { - - this.streamListener.init(); - this.wrappedInvokable.invoke(); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java deleted file mode 100644 index e8de321349377..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputGate.java +++ /dev/null @@ -1,159 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.wrappers; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.plugins.wrapper.AbstractOutputGateWrapper; -import eu.stratosphere.nephele.streaming.chaining.StreamChain; -import eu.stratosphere.nephele.streaming.listeners.StreamListener; -import eu.stratosphere.nephele.types.Record; - -public final class StreamingOutputGate extends AbstractOutputGateWrapper { - - private final StreamListener streamListener; - - private long lastThroughputTimestamp = -1L; - - private long[] lastSentBytes = null; - - private StreamChain streamChain = null; - - private Map bufferLatencyMap = new HashMap(); - - private static final class BufferLatency { - - private static final int BUFFER_LATENCY_REPORT_INTERVAL = 1000; - - private long lastBufferLatencyTimestamp = -1L; - - private int accumulatedLatency = 0; - - private int latencyCounter = 0; - - private BufferLatency(final long initialTimestamp) { - this.lastBufferLatencyTimestamp = initialTimestamp; - } - - private void addLatency(long timestamp) { - - this.accumulatedLatency += (int) (timestamp - this.lastBufferLatencyTimestamp); - this.lastBufferLatencyTimestamp = timestamp; - ++this.latencyCounter; - } - - private int getLatency() { - - if (this.accumulatedLatency < BUFFER_LATENCY_REPORT_INTERVAL) { - return -1; - } - - final int latency = this.accumulatedLatency / this.latencyCounter; - this.accumulatedLatency = 0; - this.latencyCounter = 0; - - return latency; - } - } - - StreamingOutputGate(final OutputGate wrappedOutputGate, final StreamListener streamListener) { - super(wrappedOutputGate); - - if (streamListener == null) { - throw new IllegalArgumentException("Argument streamListener must not be null"); - } - - streamListener.registerOutputGate(this); - - this.streamListener = streamListener; - } - - /** - * {@inheritDoc} - */ - @Override - public void writeRecord(final T record) throws IOException, InterruptedException { - - reportRecordEmitted(record); - - if (this.streamChain == null) { - getWrappedOutputGate().writeRecord(record); - } else { - this.streamChain.writeRecord(record); - } - } - - public void reportRecordEmitted(final Record record) { - - final long timestamp = this.streamListener.recordEmitted(record); - if (timestamp >= 0) { - - final int numberOfOutputChannels = getNumberOfOutputChannels(); - - if (this.lastThroughputTimestamp < 0) { - // Initialize array and fill it - this.lastSentBytes = new long[numberOfOutputChannels]; - for (int i = 0; i < numberOfOutputChannels; ++i) { - this.lastSentBytes[i] = getOutputChannel(i).getAmountOfDataTransmitted(); - } - } else { - for (int i = 0; i < numberOfOutputChannels; ++i) { - final AbstractOutputChannel outputChannel = getOutputChannel(i); - final long amountOfDataTransmitted = outputChannel.getAmountOfDataTransmitted(); - final long dataDiff = amountOfDataTransmitted - this.lastSentBytes[i]; - this.lastSentBytes[i] = amountOfDataTransmitted; - final long timeDiff = timestamp - this.lastThroughputTimestamp; - final double throughput = (double) (1000 * 8 * dataDiff) / (double) (1024 * 1024 * timeDiff); - this.streamListener.reportChannelThroughput(outputChannel.getID(), throughput); - } - } - - this.lastThroughputTimestamp = timestamp; - } - } - - /** - * {@inheritDoc} - */ - @Override - public void outputBufferSent(final ChannelID channelID) { - - final long timestamp = System.currentTimeMillis(); - final BufferLatency bufferLatency = this.bufferLatencyMap.get(channelID); - if (bufferLatency == null) { - this.bufferLatencyMap.put(channelID, new BufferLatency(timestamp)); - return; - } - - bufferLatency.addLatency(timestamp); - final int latency = bufferLatency.getLatency(); - if (latency >= 0) { - this.streamListener.reportBufferLatency(channelID, latency); - } - - getWrappedOutputGate().outputBufferSent(channelID); - } - - public void redirectToStreamChain(final StreamChain streamChain) throws IOException, InterruptedException { - - this.streamChain = streamChain; - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputWrapper.java deleted file mode 100644 index e73c8b9b36a85..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingOutputWrapper.java +++ /dev/null @@ -1,61 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.wrappers; - -import eu.stratosphere.nephele.streaming.listeners.StreamListener; -import eu.stratosphere.nephele.template.AbstractInvokable; -import eu.stratosphere.nephele.template.AbstractOutputTask; - -/** - * This class provides a wrapper for Nephele tasks of the type {@link AbstractOutputTask}. - *

- * This class is thread-safe. - * - * @author warneke - */ -public final class StreamingOutputWrapper extends AbstractOutputTask { - - /** - * The wrapped task. - */ - private volatile AbstractInvokable wrappedInvokable = null; - - /** - * The stream listener object. - */ - private volatile StreamListener streamListener = null; - - /** - * {@inheritDoc} - */ - @Override - public void registerInputOutput() { - - this.streamListener = WrapperUtils.createStreamListener(getEnvironment()); - this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment(), this.streamListener); - this.wrappedInvokable.registerInputOutput(); - } - - /** - * {@inheritDoc} - */ - @Override - public void invoke() throws Exception { - - this.streamListener.init(); - this.wrappedInvokable.invoke(); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingTaskWrapper.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingTaskWrapper.java deleted file mode 100644 index dba4d69adfd24..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/StreamingTaskWrapper.java +++ /dev/null @@ -1,61 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.wrappers; - -import eu.stratosphere.nephele.streaming.listeners.StreamListener; -import eu.stratosphere.nephele.template.AbstractInvokable; -import eu.stratosphere.nephele.template.AbstractTask; - -/** - * This class provides a wrapper for Nephele tasks of the type {@link AbstractTask}. - *

- * This class is thread-safe. - * - * @author warneke - */ -public final class StreamingTaskWrapper extends AbstractTask { - - /** - * The wrapped task. - */ - private volatile AbstractInvokable wrappedInvokable = null; - - /** - * The stream listener object. - */ - private volatile StreamListener streamListener = null; - - /** - * {@inheritDoc} - */ - @Override - public void registerInputOutput() { - - this.streamListener = WrapperUtils.createStreamListener(getEnvironment()); - this.wrappedInvokable = WrapperUtils.getWrappedInvokable(getEnvironment(), this.streamListener); - this.wrappedInvokable.registerInputOutput(); - } - - /** - * {@inheritDoc} - */ - @Override - public void invoke() throws Exception { - - this.streamListener.init(); - this.wrappedInvokable.invoke(); - } -} diff --git a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/WrapperUtils.java b/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/WrapperUtils.java deleted file mode 100644 index eb43b4dbc1496..0000000000000 --- a/nephele/nephele-streaming/src/main/java/eu/stratosphere/nephele/streaming/wrappers/WrapperUtils.java +++ /dev/null @@ -1,93 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.streaming.wrappers; - -import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.Environment; -import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; -import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.streaming.listeners.StreamListener; -import eu.stratosphere.nephele.template.AbstractInvokable; -import eu.stratosphere.nephele.util.StringUtils; - -/** - * This class contains convenience methods to access wrapped Nephele task classes. - * - * @author warneke - */ -public final class WrapperUtils { - - /** - * The configuration key to access the name of the wrapped class from the task configuration. - */ - public static final String WRAPPED_CLASS_KEY = "streaming.class.name"; - - /** - * Private constructor so class cannot be instantiated. - */ - private WrapperUtils() { - } - - /** - * Retrieves the name of the original class from the task configuration, loads the class, creates an instances of - * it, and finally wraps the given environment in an {@link StreamingEnvironment} object. - * - * @param environment - * the original environment - * @param streamListener - * the stream listener object - * @return an instance of the wrapped invokable class - */ - static AbstractInvokable getWrappedInvokable(final Environment environment, final StreamListener streamListener) { - - AbstractInvokable wrappedInvokable = null; - - final Configuration taskConfiguration = environment.getTaskConfiguration(); - final JobID jobID = environment.getJobID(); - final String className = taskConfiguration.getString(WRAPPED_CLASS_KEY, null); - if (className == null) { - throw new IllegalStateException("Cannot find name of wrapped class"); - } - - try { - final ClassLoader cl = LibraryCacheManager.getClassLoader(jobID); - - @SuppressWarnings("unchecked") - final Class invokableClass = (Class) Class - .forName(className, true, cl); - - wrappedInvokable = invokableClass.newInstance(); - } catch (Exception e) { - throw new RuntimeException(StringUtils.stringifyException(e)); - } - - wrappedInvokable.setEnvironment(new StreamingEnvironment(environment, streamListener)); - - return wrappedInvokable; - } - - /** - * Creates and configures a new stream listener. - * - * @param environment - * the environment for the newly created stream listener - * @return the configured stream listener - */ - static StreamListener createStreamListener(final Environment environment) { - - return new StreamListener(environment.getTaskConfiguration()); - } -} diff --git a/nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java b/nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java deleted file mode 100644 index edd05963c3a46..0000000000000 --- a/nephele/nephele-streaming/src/test/java/eu/stratosphere/nephele/streaming/latency/ProfilingValueStatisticTest.java +++ /dev/null @@ -1,142 +0,0 @@ -package eu.stratosphere.nephele.streaming.latency; - -import static org.junit.Assert.assertTrue; - -import org.junit.Before; -import org.junit.Test; - -import eu.stratosphere.nephele.streaming.profiling.ProfilingValue; -import eu.stratosphere.nephele.streaming.profiling.ProfilingValueStatistic; - -public class ProfilingValueStatisticTest { - - private ProfilingValueStatistic valueStatistic; - - @Before - public void setup() { - valueStatistic = new ProfilingValueStatistic(7); - } - - @Test - public void testValueSorted() { - valueStatistic.addValue(createProfilingValue(1, 1)); - valueStatistic.addValue(createProfilingValue(2, 3)); - valueStatistic.addValue(createProfilingValue(3, 7)); - valueStatistic.addValue(createProfilingValue(4, 8)); - valueStatistic.addValue(createProfilingValue(5, 21)); - valueStatistic.addValue(createProfilingValue(6, 35)); - valueStatistic.addValue(createProfilingValue(7, 41)); - - assertTrue(valueStatistic.getMedianValue() == 8); - assertTrue(valueStatistic.getMinValue() == 1); - assertTrue(valueStatistic.getMaxValue() == 41); - assertTrue((valueStatistic.getArithmeticMean() - 16.5714) < 0.0001); - } - - private ProfilingValue createProfilingValue(long timestamp, double value) { - return new ProfilingValue(value, timestamp); - } - - @Test - public void testAddValueUnsorted() { - valueStatistic.addValue(createProfilingValue(1, 7)); - valueStatistic.addValue(createProfilingValue(2, 15)); - valueStatistic.addValue(createProfilingValue(3, 13)); - valueStatistic.addValue(createProfilingValue(4, 1)); - valueStatistic.addValue(createProfilingValue(5, 5)); - valueStatistic.addValue(createProfilingValue(6, 7.5)); - valueStatistic.addValue(createProfilingValue(7, 8)); - - assertTrue(valueStatistic.getMedianValue() == 7.5); - assertTrue(valueStatistic.getMinValue() == 1); - assertTrue(valueStatistic.getMaxValue() == 15); - assertTrue((valueStatistic.getArithmeticMean() - 8.0714) < 0.0001); - } - - @Test - public void testAddValueReverseSorted() { - valueStatistic.addValue(createProfilingValue(1, 18)); - valueStatistic.addValue(createProfilingValue(2, 15)); - valueStatistic.addValue(createProfilingValue(3, 13)); - valueStatistic.addValue(createProfilingValue(4, 10)); - valueStatistic.addValue(createProfilingValue(5, 9)); - valueStatistic.addValue(createProfilingValue(6, 8)); - valueStatistic.addValue(createProfilingValue(7, 7)); - - assertTrue(valueStatistic.getMedianValue() == 10); - assertTrue(valueStatistic.getMinValue() == 7); - assertTrue(valueStatistic.getMaxValue() == 18); - assertTrue((valueStatistic.getArithmeticMean() - 11.4285) < 0.0001); - } - - @Test - public void testAddValueOverfullUnsorted() { - valueStatistic.addValue(createProfilingValue(1, 7)); - valueStatistic.addValue(createProfilingValue(2, 15)); - valueStatistic.addValue(createProfilingValue(3, 13)); - valueStatistic.addValue(createProfilingValue(4, 1)); - valueStatistic.addValue(createProfilingValue(5, 7.5)); - valueStatistic.addValue(createProfilingValue(6, 5)); - valueStatistic.addValue(createProfilingValue(7, 18)); - valueStatistic.addValue(createProfilingValue(8, 13)); - valueStatistic.addValue(createProfilingValue(9, 10)); - valueStatistic.addValue(createProfilingValue(10, 8)); - - assertTrue(valueStatistic.getMedianValue() == 8); - assertTrue(valueStatistic.getMinValue() == 1); - assertTrue(valueStatistic.getMaxValue() == 18); - assertTrue((valueStatistic.getArithmeticMean() - 8.9285) < 0.0001); - } - - @Test - public void testAddValueOverfullSorted() { - valueStatistic.addValue(createProfilingValue(1, 1)); - valueStatistic.addValue(createProfilingValue(2, 2)); - valueStatistic.addValue(createProfilingValue(3, 3)); - valueStatistic.addValue(createProfilingValue(4, 4)); - valueStatistic.addValue(createProfilingValue(5, 5)); - valueStatistic.addValue(createProfilingValue(6, 6)); - valueStatistic.addValue(createProfilingValue(7, 7)); - valueStatistic.addValue(createProfilingValue(8, 8)); - valueStatistic.addValue(createProfilingValue(9, 9)); - valueStatistic.addValue(createProfilingValue(10, 10)); - - assertTrue(valueStatistic.getMedianValue() == 7); - assertTrue(valueStatistic.getMinValue() == 4); - assertTrue(valueStatistic.getMaxValue() == 10); - assertTrue(valueStatistic.getArithmeticMean() == 7); - } - - @Test - public void testGetMedianUnderfull() { - valueStatistic.addValue(createProfilingValue(1, 18)); - valueStatistic.addValue(createProfilingValue(2, 15)); - assertTrue(valueStatistic.getMedianValue() == 18); - - valueStatistic.addValue(createProfilingValue(3, 17)); - assertTrue(valueStatistic.getMedianValue() == 17); - } - - @Test - public void testGetMinMaxUnderfull() { - valueStatistic.addValue(createProfilingValue(1, 18)); - valueStatistic.addValue(createProfilingValue(2, 15)); - assertTrue(valueStatistic.getMinValue() == 15); - assertTrue(valueStatistic.getMaxValue() == 18); - - valueStatistic.addValue(createProfilingValue(3, 17)); - assertTrue(valueStatistic.getMinValue() == 15); - assertTrue(valueStatistic.getMaxValue() == 18); - - } - - @Test - public void testGetArithmeticMeanUnderfull() { - valueStatistic.addValue(createProfilingValue(1, 18)); - valueStatistic.addValue(createProfilingValue(2, 15)); - assertTrue(valueStatistic.getArithmeticMean() == 16.5); - - valueStatistic.addValue(createProfilingValue(3, 17)); - assertTrue((valueStatistic.getArithmeticMean() - 16.6666) < 0.0001); - } -} diff --git a/nephele/pom.xml b/nephele/pom.xml index 84ec69a862671..e15588ac88c25 100644 --- a/nephele/pom.xml +++ b/nephele/pom.xml @@ -101,6 +101,5 @@ nephele-visualization nephele-server nephele-examples - nephele-streaming diff --git a/pom.xml b/pom.xml index 2450d8e573807..c6445b8710ed4 100644 --- a/pom.xml +++ b/pom.xml @@ -480,7 +480,6 @@ nephele pact build-tools - score stratosphere-dist diff --git a/score/pom.xml b/score/pom.xml deleted file mode 100644 index c55ba9a9322f4..0000000000000 --- a/score/pom.xml +++ /dev/null @@ -1,66 +0,0 @@ - - - - 4.0.0 - - - stratosphere - eu.stratosphere - 0.2 - - - score - score - - jar - - - - - eu.stratosphere - nephele-server - ${version} - - - - - - - - - - - - - - org.apache.maven.plugins - maven-surefire-plugin - 2.7 - - - true - WARN - - - **/*TestBase*.class - - once - - - - - maven-failsafe-plugin - 2.7 - - - WARN - - always - 1 - false - - - - - - diff --git a/score/src/main/java/eu/stratosphere/score/ScoreExecutionListener.java b/score/src/main/java/eu/stratosphere/score/ScoreExecutionListener.java deleted file mode 100644 index efeb0eda2e241..0000000000000 --- a/score/src/main/java/eu/stratosphere/score/ScoreExecutionListener.java +++ /dev/null @@ -1,72 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.score; - -import eu.stratosphere.nephele.execution.ExecutionListener; -import eu.stratosphere.nephele.execution.ExecutionState; -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.jobgraph.JobID; - -public class ScoreExecutionListener implements ExecutionListener { - - private final ExecutionVertex executionVertex; - - ScoreExecutionListener(final ExecutionVertex executionVertex) { - this.executionVertex = executionVertex; - } - - /** - * {@inheritDoc} - */ - @Override - public void executionStateChanged(final JobID jobID, final ExecutionVertexID vertexID, - final ExecutionState newExecutionState, final String optionalMessage) { - - System.out.println("SCORE received execution state update for vertex " + this.executionVertex + ": " - + newExecutionState); - } - - /** - * {@inheritDoc} - */ - @Override - public void userThreadStarted(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) { - // TODO Auto-generated method stub - - } - - /** - * {@inheritDoc} - */ - @Override - public void userThreadFinished(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) { - // TODO Auto-generated method stub - - } - - /** - * {@inheritDoc} - */ - @Override - public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID vertexID, - final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { - // TODO Auto-generated method stub - - } - -} diff --git a/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java b/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java deleted file mode 100644 index ab71db6e672ff..0000000000000 --- a/score/src/main/java/eu/stratosphere/score/ScoreJobManagerPlugin.java +++ /dev/null @@ -1,107 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.score; - -import java.io.IOException; -import java.util.Iterator; - -import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.io.IOReadableWritable; -import eu.stratosphere.nephele.jobgraph.JobGraph; -import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.plugins.JobManagerPlugin; -import eu.stratosphere.nephele.profiling.ProfilingListener; - -public final class ScoreJobManagerPlugin implements JobManagerPlugin { - - /** - * {@inheritDoc} - */ - @Override - public JobGraph rewriteJobGraph(final JobGraph jobGraph) { - - // Nothing to do here - - return jobGraph; - } - - /** - * {@inheritDoc} - */ - @Override - public ExecutionGraph rewriteExecutionGraph(final ExecutionGraph executionGraph) { - - synchronized (executionGraph) { - - // Register for events - final Iterator it = new ExecutionGraphIterator(executionGraph, true); - - while (it.hasNext()) { - final ExecutionVertex vertex = it.next(); - vertex.registerExecutionListener(new ScoreExecutionListener(vertex)); - } - } - - return executionGraph; - } - - /** - * {@inheritDoc} - */ - @Override - public void shutdown() { - // TODO Auto-generated method stub - - } - - /** - * {@inheritDoc} - */ - @Override - public void sendData(final IOReadableWritable data) throws IOException { - // TODO Auto-generated method stub - - } - - /** - * {@inheritDoc} - */ - @Override - public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { - // TODO Auto-generated method stub - return null; - } - - /** - * {@inheritDoc} - */ - @Override - public boolean requiresProfiling() { - - return false; - } - - /** - * {@inheritDoc} - */ - @Override - public ProfilingListener getProfilingListener(final JobID jobID) { - - return null; - } -} diff --git a/score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java b/score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java deleted file mode 100644 index 2ea1cbf3427d8..0000000000000 --- a/score/src/main/java/eu/stratosphere/score/ScorePluginLoader.java +++ /dev/null @@ -1,83 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.score; - -import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.plugins.AbstractPluginLoader; -import eu.stratosphere.nephele.plugins.JobManagerPlugin; -import eu.stratosphere.nephele.plugins.PluginID; -import eu.stratosphere.nephele.plugins.PluginLookupService; -import eu.stratosphere.nephele.plugins.TaskManagerPlugin; - -/** - * A plugin loader for the SCORE (Stratosphere Continuous Re-optimization) module. - *

- * This class is thread-safe. - * - * @author warneke - */ -public final class ScorePluginLoader extends AbstractPluginLoader { - - private ScoreJobManagerPlugin jobManagerPlugin = null; - - private ScoreTaskManagerPlugin taskManagerPlugin = null; - - private final PluginID pluginID; - - public ScorePluginLoader(final String pluginName, final Configuration pluginConfiguration, - final PluginLookupService pluginLookupService) { - - super(pluginName, pluginConfiguration, pluginLookupService); - - this.pluginID = PluginID.fromByteArray(new byte[] { 0x44, -0x6f, 0x18, -0x1d, 0x00, 0x4a, 0x60, -0x61, -0x25, - 0x00, 0x00, 0x16, 0x00, 0x18, -0x01, 0x62 }); - } - - /** - * {@inheritDoc} - */ - @Override - public synchronized JobManagerPlugin getJobManagerPlugin() { - - if (this.jobManagerPlugin == null) { - this.jobManagerPlugin = new ScoreJobManagerPlugin(); - } - - return this.jobManagerPlugin; - } - - /** - * {@inheritDoc} - */ - @Override - public synchronized TaskManagerPlugin getTaskManagerPlugin() { - - if (this.taskManagerPlugin == null) { - this.taskManagerPlugin = new ScoreTaskManagerPlugin(getPluginConfiguration()); - } - - return this.taskManagerPlugin; - } - - /** - * {@inheritDoc} - */ - @Override - public PluginID getPluginID() { - - return this.pluginID; - } -} diff --git a/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java b/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java deleted file mode 100644 index d3cba66193a43..0000000000000 --- a/score/src/main/java/eu/stratosphere/score/ScoreTaskManagerPlugin.java +++ /dev/null @@ -1,75 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.score; - -import java.io.IOException; - -import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.RuntimeEnvironment; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.IOReadableWritable; -import eu.stratosphere.nephele.plugins.TaskManagerPlugin; - -public final class ScoreTaskManagerPlugin implements TaskManagerPlugin { - - ScoreTaskManagerPlugin(final Configuration pluginConfiguration) { - } - - /** - * {@inheritDoc} - */ - @Override - public void shutdown() { - // TODO Auto-generated method stub - } - - /** - * {@inheritDoc} - */ - @Override - public void registerTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final RuntimeEnvironment environment) { - // TODO Auto-generated method stub - - } - - /** - * {@inheritDoc} - */ - @Override - public void unregisterTask(final ExecutionVertexID id, final RuntimeEnvironment environment) { - // TODO Auto-generated method stub - - } - - /** - * {@inheritDoc} - */ - @Override - public void sendData(final IOReadableWritable data) throws IOException { - // TODO Auto-generated method stub - - } - - /** - * {@inheritDoc} - */ - @Override - public IOReadableWritable requestData(final IOReadableWritable data) throws IOException { - // TODO Auto-generated method stub - return null; - } -} From 3e9d44b38f4c0109a52cdf19c78787487ccd5d53 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 29 Jan 2012 19:24:28 +0100 Subject: [PATCH 184/310] Continued to work on integration of streaming branch into checkpointing branch --- .../nephele/execution/Environment.java | 24 ++ .../AbstractByteBufferedOutputChannel.java | 12 - .../ByteBufferedOutputChannelBroker.java | 8 - .../impl/TaskManagerProfilerImpl.java | 6 +- .../checkpointing/EphemeralCheckpoint.java | 8 +- .../nephele/execution/RuntimeEnvironment.java | 60 +++ .../scheduler/AbstractScheduler.java | 4 +- .../jobmanager/scheduler/RecoveryLogic.java | 4 +- .../profiling/TaskManagerProfiler.java | 4 +- .../taskmanager/CheckpointReplayTask.java | 5 + .../nephele/taskmanager/RuntimeTask.java | 399 ++++++++++++++++++ .../nephele/taskmanager/Task.java | 363 +--------------- .../nephele/taskmanager/TaskManager.java | 12 +- .../ByteBufferedChannelManager.java | 50 ++- .../RecentlyRemovedChannelIDSet.java | 33 +- .../taskmanager/bytebuffered/TaskContext.java | 10 +- .../checkpointing/FailingJobITCase.java | 66 +-- .../pact/runtime/task/AbstractPactTask.java | 2 +- .../pact/runtime/task/util/OutputEmitter.java | 92 +--- .../runtime/task/util/OutputEmitterTest.java | 86 +--- .../pact/runtime/test/util/TaskTestBase.java | 14 +- 21 files changed, 617 insertions(+), 645 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/CheckpointReplayTask.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/RuntimeTask.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java index efe45c34d876f..645ecca0b76fd 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java @@ -15,6 +15,8 @@ package eu.stratosphere.nephele.execution; +import java.util.Set; + import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.io.ChannelSelector; import eu.stratosphere.nephele.io.DistributionPattern; @@ -22,6 +24,7 @@ import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.RecordDeserializer; +import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; @@ -184,4 +187,25 @@ InputGate createInputGate(GateID gateID, RecordDeserializer inputGate); + + /** + * Returns the IDs of all output channels connected to this environment. + * + * @return the IDs of all output channels connected to this environment + */ + Set getOutputChannelIDs(); + + /** + * Returns the IDs of all input channels connected to this environment. + * + * @return the IDs of all input channels connected to this environment + */ + Set getInputChannelIDs(); + + /** + * Returns the IDs of all input gates connected to this environment. + * + * @return the IDs of all input gates connected to this environment + */ + Set getInputGateIDs(); } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java index 712b97edbfb3d..064f329b9de00 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java @@ -355,16 +355,4 @@ public long getAmountOfDataTransmitted() { return this.amountOfDataTransmitted; } - - /** - * Limits the size of the buffer this channel will write its records to before passing them on to the framework. - * - * @param bufferSize - * the new limit for the by - */ - public void limitBufferSize(final int bufferSize) { - - // Delegate call to the assigned output channel broker - this.outputChannelBroker.limitBufferSize(bufferSize); - } } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java index a4d7dead0fd4e..a565dc58f9ee0 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java @@ -67,12 +67,4 @@ public interface ByteBufferedOutputChannelBroker { * thrown if an I/O error occurs while transfering the event */ void transferEventToInputChannel(AbstractEvent event) throws IOException, InterruptedException; - - /** - * Limits the size of the buffer the output channel is allowed to write into. - * - * @param bufferSize - * the new buffer size in bytes - */ - void limitBufferSize(int bufferSize); } diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java index b13a5f4628de3..f3eebc2f940c5 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java @@ -46,7 +46,7 @@ import eu.stratosphere.nephele.profiling.impl.types.InternalInstanceProfilingData; import eu.stratosphere.nephele.profiling.impl.types.InternalOutputGateProfilingData; import eu.stratosphere.nephele.profiling.impl.types.ProfilingDataContainer; -import eu.stratosphere.nephele.taskmanager.Task; +import eu.stratosphere.nephele.taskmanager.RuntimeTask; import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.util.StringUtils; @@ -112,10 +112,10 @@ public TaskManagerProfilerImpl(InetAddress jobManagerAddress, InstanceConnection * {@inheritDoc} */ @Override - public void registerExecutionListener(final Task task, final Configuration jobConfiguration) { + public void registerExecutionListener(final RuntimeTask task, final Configuration jobConfiguration) { // Register profiling hook for the environment - task.registerExecutionListener(new EnvironmentListenerImpl(this, task.getEnvironment())); + task.registerExecutionListener(new EnvironmentListenerImpl(this, task.getRuntimeEnvironment())); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 0f4a4de7c26ac..1bcdfbcf3d7e3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -26,7 +26,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.taskmanager.Task; +import eu.stratosphere.nephele.taskmanager.RuntimeTask; import eu.stratosphere.nephele.taskmanager.transferenvelope.CheckpointSerializer; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.configuration.GlobalConfiguration; @@ -77,7 +77,7 @@ public class EphemeralCheckpoint { /** * The task this checkpoint is created for. */ - private final Task task; + private final RuntimeTask task; /** * The number of channels connected to this checkpoint. @@ -126,13 +126,13 @@ private enum CheckpointingDecisionState { private volatile CheckpointingDecisionState asynchronousCheckpointingDecision; - public EphemeralCheckpoint(final Task task, final boolean ephemeral) { + public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { this.task = task; // Determine number of output channel int nooc = 0; - final RuntimeEnvironment environment = task.getEnvironment(); + final RuntimeEnvironment environment = task.getRuntimeEnvironment(); for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { nooc += environment.getOutputGate(i).getNumberOfOutputChannels(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 04ef4a49da909..9b55fedea91f5 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -19,8 +19,12 @@ import java.io.DataOutput; import java.io.IOException; import java.util.ArrayDeque; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Queue; +import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import org.apache.commons.logging.Log; @@ -1003,4 +1007,60 @@ private void releaseAllChannelResources() { this.outputGates.get(i).releaseAllChannelResources(); } } + + /** + * {@inheritDoc} + */ + @Override + public Set getOutputChannelIDs() { + + final Set outputChannelIDs = new HashSet(); + + final Iterator> gateIterator = this.outputGates.iterator(); + while (gateIterator.hasNext()) { + + final OutputGate outputGate = gateIterator.next(); + for (int i = 0; i < outputGate.getNumberOfOutputChannels(); ++i) { + outputChannelIDs.add(outputGate.getOutputChannel(i).getID()); + } + } + + return Collections.unmodifiableSet(outputChannelIDs); + } + + /** + * {@inheritDoc} + */ + @Override + public Set getInputChannelIDs() { + + final Set inputChannelIDs = new HashSet(); + + final Iterator> gateIterator = this.inputGates.iterator(); + while (gateIterator.hasNext()) { + + final InputGate outputGate = gateIterator.next(); + for (int i = 0; i < outputGate.getNumberOfInputChannels(); ++i) { + inputChannelIDs.add(outputGate.getInputChannel(i).getID()); + } + } + + return Collections.unmodifiableSet(inputChannelIDs); + } + + /** + * {@inheritDoc} + */ + @Override + public Set getInputGateIDs() { + + final Set inputGateIDs = new HashSet(); + + final Iterator> gateIterator = this.inputGates.iterator(); + while (gateIterator.hasNext()) { + inputGateIDs.add(gateIterator.next().getGateID()); + } + + return Collections.unmodifiableSet(inputGateIDs); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index 9fd9384fbfeb9..c9e010840c462 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -27,8 +27,8 @@ import org.apache.hadoop.util.StringUtils; import eu.stratosphere.nephele.configuration.GlobalConfiguration; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; @@ -222,7 +222,7 @@ void findVerticesToBeDeployed(final ExecutionVertex vertex, verticesForInstance.add(vertex); } - final Environment env = vertex.getEnvironment(); + final RuntimeEnvironment env = vertex.getEnvironment(); final int numberOfOutputGates = env.getNumberOfOutputGates(); for (int i = 0; i < numberOfOutputGates; ++i) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index 1f6c39a98e217..09f057dd1359f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -28,8 +28,8 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; -import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; @@ -189,7 +189,7 @@ private static final boolean invalidateReceiverLookupCaches(final ExecutionVerte final ExecutionGraph eg = failedVertex.getExecutionGraph(); - final Environment env = failedVertex.getEnvironment(); + final RuntimeEnvironment env = failedVertex.getEnvironment(); for (int i = 0; i < env.getNumberOfOutputGates(); ++i) { final OutputGate outputGate = env.getOutputGate(i); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java index 54b857801ad0d..24cd3bad0390f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java @@ -22,7 +22,7 @@ import eu.stratosphere.nephele.io.InputGateListener; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.OutputGateListener; -import eu.stratosphere.nephele.taskmanager.Task; +import eu.stratosphere.nephele.taskmanager.RuntimeTask; import eu.stratosphere.nephele.types.Record; /** @@ -41,7 +41,7 @@ public interface TaskManagerProfiler { * @param jobConfiguration * the job configuration sent with the task */ - void registerExecutionListener(Task task, Configuration jobConfiguration); + void registerExecutionListener(RuntimeTask task, Configuration jobConfiguration); /** * Registers a {@link InputGateListener} object for the given input gate. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/CheckpointReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/CheckpointReplayTask.java new file mode 100644 index 0000000000000..97d967962bf9f --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/CheckpointReplayTask.java @@ -0,0 +1,5 @@ +package eu.stratosphere.nephele.taskmanager; + +public class CheckpointReplayTask { + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/RuntimeTask.java new file mode 100644 index 0000000000000..c4d9a94211f9d --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/RuntimeTask.java @@ -0,0 +1,399 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager; + +import java.lang.management.ManagementFactory; +import java.lang.management.ThreadMXBean; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.annotations.ForceCheckpoint; +import eu.stratosphere.nephele.annotations.Stateful; +import eu.stratosphere.nephele.annotations.Stateless; +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.ExecutionListener; +import eu.stratosphere.nephele.execution.ExecutionObserver; +import eu.stratosphere.nephele.execution.ExecutionState; +import eu.stratosphere.nephele.execution.ExecutionStateTransition; +import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; +import eu.stratosphere.nephele.executiongraph.CheckpointState; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.channels.AbstractInputChannel; +import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.template.AbstractInvokable; +import eu.stratosphere.nephele.types.Record; +import eu.stratosphere.nephele.util.StringUtils; + +public final class RuntimeTask implements Task, ExecutionObserver { + + /** + * The log object used for debugging. + */ + private static final Log LOG = LogFactory.getLog(Task.class); + + private static final long NANO_TO_MILLISECONDS = 1000 * 1000; + + private final ExecutionVertexID vertexID; + + private final RuntimeEnvironment environment; + + private final TaskManager taskManager; + + /** + * Stores whether the task has been canceled. + */ + private volatile boolean isCanceled = false; + + /** + * The current execution state of the task + */ + private volatile ExecutionState executionState = ExecutionState.STARTING; + + private Queue registeredListeners = new ConcurrentLinkedQueue(); + + private long startTime; + + RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, final TaskManager taskManager) { + + this.vertexID = vertexID; + this.environment = environment; + this.taskManager = taskManager; + } + + /** + * {@inheritDoc} + */ + @Override + public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) { + + // Check the state transition + ExecutionStateTransition.checkTransition(getTaskName(), this.executionState, newExecutionState); + + // Make sure the reason for a transition to FAILED appears in the log files + if (newExecutionState == ExecutionState.FAILED) { + LOG.error(optionalMessage); + } + + // Notify all listener objects + final Iterator it = this.registeredListeners.iterator(); + while (it.hasNext()) { + it.next().executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState, + optionalMessage); + } + + // Store the new execution state + this.executionState = newExecutionState; + + // Finally propagate the state change to the job manager + this.taskManager.executionStateChanged(this.environment.getJobID(), this.vertexID, this, + newExecutionState, optionalMessage); + } + + /** + * Returns the name of the task associated with this observer object. + * + * @return the name of the task associated with this observer object + */ + private String getTaskName() { + + return this.environment.getTaskName() + " (" + (this.environment.getIndexInSubtaskGroup() + 1) + "/" + + this.environment.getCurrentNumberOfSubtasks() + ")"; + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadStarted(final Thread userThread) { + + // Notify the listeners + final Iterator it = this.registeredListeners.iterator(); + while (it.hasNext()) { + it.next().userThreadStarted(this.environment.getJobID(), this.vertexID, userThread); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadFinished(final Thread userThread) { + + // Notify the listeners + final Iterator it = this.registeredListeners.iterator(); + while (it.hasNext()) { + it.next().userThreadFinished(this.environment.getJobID(), this.vertexID, userThread); + } + } + + /** + * Registers the {@link ExecutionListener} object for this task. This object + * will be notified about important events during the task execution. + * + * @param executionListener + * the object to be notified for important events during the task execution + */ + + public void registerExecutionListener(final ExecutionListener executionListener) { + + this.registeredListeners.add(executionListener); + } + + /** + * Unregisters the {@link ExecutionListener} object for this environment. This object + * will no longer be notified about important events during the task execution. + * + * @param executionListener + * the lister object to be unregistered + */ + + public void unregisterExecutionListener(final ExecutionListener executionListener) { + + this.registeredListeners.remove(executionListener); + } + + /** + * Marks the task as failed and triggers the appropriate state changes. + */ + public void markAsFailed() { + + executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly"); + } + + /** + * Cancels the execution of the task (i.e. interrupts the execution thread). + */ + public void cancelExecution() { + + cancelOrKillExecution(true); + } + + /** + * Kills the task (i.e. interrupts the execution thread). + */ + public void killExecution() { + + cancelOrKillExecution(false); + } + + /** + * Cancels or kills the task. + * + * @param cancel + * true/code> if the task shall be cancelled, false if it shall be killed + */ + private void cancelOrKillExecution(final boolean cancel) { + + final Thread executingThread = this.environment.getExecutingThread(); + + if (executingThread == null) { + return; + } + + if (cancel) { + this.isCanceled = true; + // Change state + executionStateChanged(ExecutionState.CANCELING, null); + } + + // Request user code to shut down + try { + final AbstractInvokable invokable = this.environment.getInvokable(); + if (invokable != null) { + invokable.cancel(); + } + } catch (Throwable e) { + LOG.error(StringUtils.stringifyException(e)); + } + + // Continuously interrupt the user thread until it changed to state CANCELED + while (true) { + + executingThread.interrupt(); + + if (cancel) { + if (this.executionState == ExecutionState.CANCELED) { + break; + } + } else { + if (this.executionState == ExecutionState.FAILED) { + break; + } + } + + try { + Thread.sleep(100); + } catch (InterruptedException e) { + break; + } + } + } + + /** + * Starts the execution of this Nephele task. + */ + public void startExecution() { + + final Thread thread = this.environment.getExecutingThread(); + thread.start(); + this.startTime = System.currentTimeMillis(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isCanceled() { + + return this.isCanceled; + } + + /** + * Triggers the notification that the task has run out of its initial execution resources. + */ + public void initialExecutionResourcesExhausted() { + + // if (this.environment.getExecutingThread() != Thread.currentThread()) { + // throw new ConcurrentModificationException( + // "initialExecutionResourcesExhausted must be called from the task that executes the user code"); + // } + + // Construct a resource utilization snapshot + final long timestamp = System.currentTimeMillis(); + // Get CPU-Usertime in percent + ThreadMXBean threadBean = ManagementFactory.getThreadMXBean(); + long userCPU = (threadBean.getCurrentThreadUserTime() / NANO_TO_MILLISECONDS) * 100 + / (timestamp - this.startTime); + + // collect outputChannelUtilization + final Map channelUtilization = new HashMap(); + long totalOutputAmount = 0; + for (int i = 0; i < this.environment.getNumberOfOutputGates(); ++i) { + final OutputGate outputGate = this.environment.getOutputGate(i); + for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { + final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); + channelUtilization.put(outputChannel.getID(), + Long.valueOf(outputChannel.getAmountOfDataTransmitted())); + totalOutputAmount += outputChannel.getAmountOfDataTransmitted(); + } + } + long totalInputAmount = 0; + for (int i = 0; i < this.environment.getNumberOfInputGates(); ++i) { + final InputGate inputGate = this.environment.getInputGate(i); + for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { + final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); + channelUtilization.put(inputChannel.getID(), + Long.valueOf(inputChannel.getAmountOfDataTransmitted())); + totalInputAmount += inputChannel.getAmountOfDataTransmitted(); + + } + } + Boolean force = null; + + if (this.environment.getInvokable().getClass().isAnnotationPresent(Stateful.class) + && !this.environment.getInvokable().getClass().isAnnotationPresent(Stateless.class)) { + // Don't checkpoint statefull tasks + force = false; + } else { + // look for a forced decision from the user + ForceCheckpoint forced = this.environment.getInvokable().getClass().getAnnotation(ForceCheckpoint.class); + if (forced != null) { + force = forced.checkpoint(); + } + } + final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, channelUtilization, userCPU, + force, totalInputAmount, totalOutputAmount); + + // Notify the listener objects + final Iterator it = this.registeredListeners.iterator(); + while (it.hasNext()) { + it.next().initialExecutionResourcesExhausted(this.environment.getJobID(), this.vertexID, rus); + } + + // Finally, propagate event to the job manager + this.taskManager.initialExecutionResourcesExhausted(this.environment.getJobID(), this.vertexID, rus); + } + + public void checkpointStateChanged(final CheckpointState newCheckpointState) { + + // Propagate event to the job manager + this.taskManager.checkpointStateChanged(this.environment.getJobID(), this.vertexID, newCheckpointState); + } + + /** + * Checks if the state of the thread which is associated with this task is TERMINATED. + * + * @return true if the state of this thread which is associated with this task is + * TERMINATED, false otherwise + */ + public boolean isTerminated() { + + final Thread executingThread = this.environment.getExecutingThread(); + if (executingThread.getState() == Thread.State.TERMINATED) { + return true; + } + + return false; + } + + /** + * {@inheritDoc} + */ + @Override + public Environment getEnvironment() { + + return this.environment; + } + + /** + * Returns the runtime environment associated with this task. + * + * @return the runtime environment associated with this task + */ + public RuntimeEnvironment getRuntimeEnvironment() { + + return this.environment; + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + + return this.environment.getJobID(); + } + + /** + * {@inheritDoc} + */ + @Override + public ExecutionVertexID getVertexID() { + + return this.vertexID; + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index 08c4e98295986..86d613a29f687 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -15,377 +15,30 @@ package eu.stratosphere.nephele.taskmanager; -import java.lang.management.ManagementFactory; -import java.lang.management.ThreadMXBean; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.annotations.ForceCheckpoint; -import eu.stratosphere.nephele.annotations.Stateful; -import eu.stratosphere.nephele.annotations.Stateless; import eu.stratosphere.nephele.execution.Environment; -import eu.stratosphere.nephele.execution.ExecutionListener; -import eu.stratosphere.nephele.execution.ExecutionObserver; -import eu.stratosphere.nephele.execution.ExecutionState; -import eu.stratosphere.nephele.execution.ExecutionStateTransition; -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; -import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.channels.AbstractInputChannel; -import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; -import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.template.AbstractInvokable; -import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.util.StringUtils; -public class Task implements ExecutionObserver { +public interface Task { /** - * The log object used for debugging. - */ - private static final Log LOG = LogFactory.getLog(Task.class); - - private static final long NANO_TO_MILLISECONDS = 1000 * 1000; - - private final ExecutionVertexID vertexID; - - private final Environment environment; - - private final TaskManager taskManager; - - /** - * Stores whether the task has been canceled. - */ - private volatile boolean isCanceled = false; - - /** - * The current execution state of the task - */ - private volatile ExecutionState executionState = ExecutionState.STARTING; - - private Queue registeredListeners = new ConcurrentLinkedQueue(); - - private long startTime; - - Task(final ExecutionVertexID vertexID, final Environment environment, final TaskManager taskManager) { - - this.vertexID = vertexID; - this.environment = environment; - this.taskManager = taskManager; - } - - /** - * {@inheritDoc} - */ - @Override - public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) { - - // Check the state transition - ExecutionStateTransition.checkTransition(getTaskName(), this.executionState, newExecutionState); - - // Make sure the reason for a transition to FAILED appears in the log files - if (newExecutionState == ExecutionState.FAILED) { - LOG.error(optionalMessage); - } - - // Notify all listener objects - final Iterator it = this.registeredListeners.iterator(); - while (it.hasNext()) { - it.next().executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState, - optionalMessage); - } - - // Store the new execution state - this.executionState = newExecutionState; - - // Finally propagate the state change to the job manager - this.taskManager.executionStateChanged(this.environment.getJobID(), this.vertexID, this, - newExecutionState, optionalMessage); - } - - /** - * Returns the name of the task associated with this observer object. - * - * @return the name of the task associated with this observer object - */ - private String getTaskName() { - - return this.environment.getTaskName() + " (" + (this.environment.getIndexInSubtaskGroup() + 1) + "/" - + this.environment.getCurrentNumberOfSubtasks() + ")"; - } - - /** - * {@inheritDoc} - */ - @Override - public void userThreadStarted(final Thread userThread) { - - // Notify the listeners - final Iterator it = this.registeredListeners.iterator(); - while (it.hasNext()) { - it.next().userThreadStarted(this.environment.getJobID(), this.vertexID, userThread); - } - } - - /** - * {@inheritDoc} - */ - @Override - public void userThreadFinished(final Thread userThread) { - - // Notify the listeners - final Iterator it = this.registeredListeners.iterator(); - while (it.hasNext()) { - it.next().userThreadFinished(this.environment.getJobID(), this.vertexID, userThread); - } - } - - /** - * Registers the {@link ExecutionListener} object for this task. This object - * will be notified about important events during the task execution. - * - * @param executionListener - * the object to be notified for important events during the task execution - */ - - public void registerExecutionListener(final ExecutionListener executionListener) { - - this.registeredListeners.add(executionListener); - } - - /** - * Unregisters the {@link ExecutionListener} object for this environment. This object - * will no longer be notified about important events during the task execution. - * - * @param executionListener - * the lister object to be unregistered - */ - - public void unregisterExecutionListener(final ExecutionListener executionListener) { - - this.registeredListeners.remove(executionListener); - } - - /** - * Marks the task as failed and triggers the appropriate state changes. - */ - public void markAsFailed() { - - executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly"); - } - - /** - * Cancels the execution of the task (i.e. interrupts the execution thread). - */ - public void cancelExecution() { - - cancelOrKillExecution(true); - } - - /** - * Kills the task (i.e. interrupts the execution thread). - */ - public void killExecution() { - - cancelOrKillExecution(false); - } - - /** - * Cancels or kills the task. + * Returns the ID of the job this task belongs to. * - * @param cancel - * true/code> if the task shall be cancelled, false if it shall be killed - */ - private void cancelOrKillExecution(final boolean cancel) { - - final Thread executingThread = this.environment.getExecutingThread(); - - if (executingThread == null) { - return; - } - - if (cancel) { - this.isCanceled = true; - // Change state - executionStateChanged(ExecutionState.CANCELING, null); - } - - // Request user code to shut down - try { - final AbstractInvokable invokable = this.environment.getInvokable(); - if (invokable != null) { - invokable.cancel(); - } - } catch (Throwable e) { - LOG.error(StringUtils.stringifyException(e)); - } - - // Continuously interrupt the user thread until it changed to state CANCELED - while (true) { - - executingThread.interrupt(); - - if (cancel) { - if (this.executionState == ExecutionState.CANCELED) { - break; - } - } else { - if (this.executionState == ExecutionState.FAILED) { - break; - } - } - - try { - Thread.sleep(100); - } catch (InterruptedException e) { - break; - } - } - } - - /** - * Starts the execution of this Nephele task. - */ - public void startExecution() { - - final Thread thread = this.environment.getExecutingThread(); - thread.start(); - this.startTime = System.currentTimeMillis(); - } - - /** - * {@inheritDoc} - */ - @Override - public boolean isCanceled() { - - return this.isCanceled; - } - - /** - * Triggers the notification that the task has run out of its initial execution resources. + * @return the ID of the job this task belongs to */ - public void initialExecutionResourcesExhausted() { - - // if (this.environment.getExecutingThread() != Thread.currentThread()) { - // throw new ConcurrentModificationException( - // "initialExecutionResourcesExhausted must be called from the task that executes the user code"); - // } - - // Construct a resource utilization snapshot - final long timestamp = System.currentTimeMillis(); - // Get CPU-Usertime in percent - ThreadMXBean threadBean = ManagementFactory.getThreadMXBean(); - long userCPU = (threadBean.getCurrentThreadUserTime() / NANO_TO_MILLISECONDS) * 100 - / (timestamp - this.startTime); - - // collect outputChannelUtilization - final Map channelUtilization = new HashMap(); - long totalOutputAmount = 0; - for (int i = 0; i < this.environment.getNumberOfOutputGates(); ++i) { - final OutputGate outputGate = this.environment.getOutputGate(i); - for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { - final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); - channelUtilization.put(outputChannel.getID(), - Long.valueOf(outputChannel.getAmountOfDataTransmitted())); - totalOutputAmount += outputChannel.getAmountOfDataTransmitted(); - } - } - long totalInputAmount = 0; - for (int i = 0; i < this.environment.getNumberOfInputGates(); ++i) { - final InputGate inputGate = this.environment.getInputGate(i); - for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { - final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); - channelUtilization.put(inputChannel.getID(), - Long.valueOf(inputChannel.getAmountOfDataTransmitted())); - totalInputAmount += inputChannel.getAmountOfDataTransmitted(); - - } - } - Boolean force = null; - - if (this.environment.getInvokable().getClass().isAnnotationPresent(Stateful.class) - && !this.environment.getInvokable().getClass().isAnnotationPresent(Stateless.class)) { - // Don't checkpoint statefull tasks - force = false; - } else { - // look for a forced decision from the user - ForceCheckpoint forced = this.environment.getInvokable().getClass().getAnnotation(ForceCheckpoint.class); - if (forced != null) { - force = forced.checkpoint(); - } - } - final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, channelUtilization, userCPU, - force, totalInputAmount, totalOutputAmount); - - // Notify the listener objects - final Iterator it = this.registeredListeners.iterator(); - while (it.hasNext()) { - it.next().initialExecutionResourcesExhausted(this.environment.getJobID(), this.vertexID, rus); - } - - // Finally, propagate event to the job manager - this.taskManager.initialExecutionResourcesExhausted(this.environment.getJobID(), this.vertexID, rus); - } - - public void checkpointStateChanged(final CheckpointState newCheckpointState) { - - // Propagate event to the job manager - this.taskManager.checkpointStateChanged(this.environment.getJobID(), this.vertexID, newCheckpointState); - } + JobID getJobID(); /** - * Checks if the state of the thread which is associated with this task is TERMINATED. + * Returns the ID of this task. * - * @return true if the state of this thread which is associated with this task is - * TERMINATED, false otherwise + * @return the ID of this task */ - public boolean isTerminated() { - - final Thread executingThread = this.environment.getExecutingThread(); - if (executingThread.getState() == Thread.State.TERMINATED) { - return true; - } - - return false; - } + ExecutionVertexID getVertexID(); /** * Returns the environment associated with this task. * * @return the environment associated with this task */ - public Environment getEnvironment() { - - return this.environment; - } - - /** - * Returns the ID of the job this task belongs to. - * - * @return the ID of the job this task belongs to - */ - public JobID getJobID() { - - return this.environment.getJobID(); - } - - /** - * Returns the ID of this task. - * - * @return the ID of this task - */ - public ExecutionVertexID getVertexID() { - - return this.vertexID; - } + Environment getEnvironment(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 6738c2406f1c7..70a6e8849d8f1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -495,7 +495,7 @@ public TaskSubmissionResult submitTask(final ExecutionVertexID id, final Configu re.setInputSplitProvider(new TaskInputSplitProvider(re.getJobID(), id, this.globalInputSplitProvider)); // Create task object and register it with the environment - final Task task = new Task(id, re, this); + final RuntimeTask task = new RuntimeTask(id, re, this); re.setExecutionObserver(task); // Register the task @@ -538,7 +538,7 @@ public List submitTasks(final List re.setInputSplitProvider(new TaskInputSplitProvider(re.getJobID(), id, this.globalInputSplitProvider)); // Create task object and register it with the environment - final Task task = new Task(id, re, this); + final RuntimeTask task = new RuntimeTask(id, re, this); re.setExecutionObserver(task); // Register the task @@ -561,7 +561,7 @@ public List submitTasks(final List } /** - * Registers an newly incoming task with the task manager. + * Registers an newly incoming runtime task with the task manager. * * @param id * the ID of the task to register @@ -574,8 +574,8 @@ public List submitTasks(final List * @return null if the registration has been successful or a {@link TaskSubmissionResult} containing * the error that occurred */ - private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final Task task, final Set activeOutputChannels) { + private TaskSubmissionResult registerRuntimeTask(final ExecutionVertexID id, final Configuration jobConfiguration, + final RuntimeTask task, final Set activeOutputChannels) { if (id == null) { throw new IllegalArgumentException("Argument id is null"); @@ -595,7 +595,7 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf } } - final RuntimeEnvironment ee = task.getEnvironment(); + final RuntimeEnvironment ee = task.getRuntimeEnvironment(); // Register the task with the byte buffered channel manager this.byteBufferedChannelManager.register(task, activeOutputChannels); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 0d72ecf408bdb..960ff2a378a92 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -29,9 +29,11 @@ import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; import eu.stratosphere.nephele.io.AbstractID; +import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.channels.AbstractChannel; @@ -45,6 +47,7 @@ import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.protocols.ChannelLookupProtocol; +import eu.stratosphere.nephele.taskmanager.RuntimeTask; import eu.stratosphere.nephele.taskmanager.Task; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProviderBroker; @@ -134,9 +137,9 @@ public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupServi * @param the * set of output channels which are initially active */ - public void register(final Task task, final Set activeOutputChannels) { + public void register(final RuntimeTask task, final Set activeOutputChannels) { - final Environment environment = task.getEnvironment(); + final RuntimeEnvironment environment = task.getRuntimeEnvironment(); final TaskContext taskContext = new TaskContext(task, this, this.tasksWithUndecidedCheckpoints); @@ -231,35 +234,40 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { // Mark all channel IDs to be recently removed this.recentlyRemovedChannelIDSet.add(environment); - for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - final OutputGate outputGate = environment.getOutputGate(i); - for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { - final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); - this.registeredChannels.remove(outputChannel.getID()); - this.receiverCache.remove(outputChannel.getID()); - } + Iterator channelIterator = environment.getOutputChannelIDs().iterator(); + + while (channelIterator.hasNext()) { + + final ChannelID outputChannelID = channelIterator.next(); + this.registeredChannels.remove(outputChannelID); + this.receiverCache.remove(outputChannelID); } - for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { - final InputGate inputGate = environment.getInputGate(i); - for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { - final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); - this.registeredChannels.remove(inputChannel.getID()); - this.receiverCache.remove(inputChannel.getID()); - } + channelIterator = environment.getInputChannelIDs().iterator(); + + while (channelIterator.hasNext()) { + + final ChannelID outputChannelID = channelIterator.next(); + this.registeredChannels.remove(outputChannelID); + this.receiverCache.remove(outputChannelID); + } + + final Iterator inputGateIterator = environment.getInputGateIDs().iterator(); + + while (inputGateIterator.hasNext()) { - final LocalBufferPoolOwner owner = this.localBufferPoolOwner.remove(inputGate.getGateID()); + final GateID inputGateID = inputGateIterator.next(); + + final LocalBufferPoolOwner owner = this.localBufferPoolOwner.remove(inputGateID); if (owner == null) { - LOG.error("Cannot find local buffer pool owner for input gate " + inputGate.getGateID()); + LOG.error("Cannot find local buffer pool owner for input gate " + inputGateID); } else { owner.clearLocalBufferPool(); } } final LocalBufferPoolOwner owner = this.localBufferPoolOwner.remove(vertexID); - if (owner == null) { - LOG.error("Cannot find local buffer pool owner for vertex ID" + vertexID); - } else { + if (owner != null) { owner.clearLocalBufferPool(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RecentlyRemovedChannelIDSet.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RecentlyRemovedChannelIDSet.java index d6358c6ff76c7..0b65e95864219 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RecentlyRemovedChannelIDSet.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RecentlyRemovedChannelIDSet.java @@ -18,12 +18,10 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.Set; import eu.stratosphere.nephele.execution.Environment; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.types.Record; /** * This channel set stores the IDs of all channels that have been recently removed. The set can be cleaned up by @@ -91,30 +89,19 @@ public void add(final Environment environment) { final Long now = Long.valueOf(System.currentTimeMillis()); - synchronized (this.recentlyRemovedChannels) { - - final int numberOfOutputGates = environment.getNumberOfOutputGates(); - - for (int i = 0; i < numberOfOutputGates; ++i) { + final Set outputChannelIDs = environment.getOutputChannelIDs(); + final Set inputChannelIDs = environment.getInputChannelIDs(); - final OutputGate outputGate = environment.getOutputGate(i); - final int numberOfOutputChannels = outputGate.getNumberOfOutputChannels(); - for (int j = 0; j < numberOfOutputChannels; ++j) { + synchronized (this.recentlyRemovedChannels) { - this.recentlyRemovedChannels.put(outputGate.getOutputChannel(j).getID(), now); - } + Iterator it = outputChannelIDs.iterator(); + while (it.hasNext()) { + this.recentlyRemovedChannels.put(it.next(), now); } - final int numberOfInputGates = environment.getNumberOfInputGates(); - - for (int i = 0; i < numberOfInputGates; ++i) { - - final InputGate inputGate = environment.getInputGate(i); - final int numberOfInputChannels = inputGate.getNumberOfInputChannels(); - for (int j = 0; j < numberOfInputChannels; ++j) { - - this.recentlyRemovedChannels.put(inputGate.getInputChannel(j).getID(), now); - } + it = inputChannelIDs.iterator(); + while (it.hasNext()) { + this.recentlyRemovedChannels.put(it.next(), now); } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java index e94acb19d3d09..15c466753686c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java @@ -26,7 +26,7 @@ import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.io.channels.ChannelType; -import eu.stratosphere.nephele.taskmanager.Task; +import eu.stratosphere.nephele.taskmanager.RuntimeTask; import eu.stratosphere.nephele.taskmanager.bufferprovider.AsynchronousEventListener; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; @@ -40,7 +40,7 @@ final class TaskContext implements BufferProvider, LocalBufferPoolOwner, Asynchr private final LocalBufferPool localBufferPool; - private final Task task; + private final RuntimeTask task; private final AsynchronousEventListener[] subEventListener; @@ -57,13 +57,13 @@ final class TaskContext implements BufferProvider, LocalBufferPoolOwner, Asynchr */ private boolean initialExhaustionOfMemoryBuffersReported = false; - TaskContext(final Task task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + TaskContext(final RuntimeTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final Map tasksWithUndecidedCheckpoints) { this.localBufferPool = new LocalBufferPool(1, false, this); this.task = task; - final RuntimeEnvironment environment = task.getEnvironment(); + final RuntimeEnvironment environment = task.getRuntimeEnvironment(); // Compute number of output input channels int nooc = 0; @@ -163,7 +163,7 @@ public void logBufferUtilization() { final int req = this.localBufferPool.getRequestedNumberOfBuffers(); final int des = this.localBufferPool.getDesignatedNumberOfBuffers(); - final RuntimeEnvironment environment = this.task.getEnvironment(); + final RuntimeEnvironment environment = this.task.getRuntimeEnvironment(); System.out.println("\t\t" + environment.getTaskName() + ": " + ava + " available, " + req + " requested, " + des + " designated"); diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java index 7b45fc8ad8203..34b81bae080d6 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/checkpointing/FailingJobITCase.java @@ -174,7 +174,7 @@ public static void startNephele() { if (!new File(configDir).exists()) { configDir = userDir + "/src/test/resources/" + CONFIGURATION_DIRECTORY; } - + final Constructor c = JobManager.class.getDeclaredConstructor(new Class[] { String.class, String.class }); c.setAccessible(true); @@ -283,9 +283,9 @@ public void invoke() throws Exception { boolean failing = false; - final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + final int failAfterRecord = getTaskConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); synchronized (FAILED_ONCE) { - failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( + failing = (getIndexInSubtaskGroup() == getTaskConfiguration().getInteger( FAILURE_INDEX_KEY, -1)) && FAILED_ONCE.add(getEnvironment().getTaskName()); } @@ -300,13 +300,14 @@ public void invoke() throws Exception { } } } + @ForceCheckpoint(checkpoint = true) public final static class InnerTask extends AbstractTask { private MutableRecordReader recordReader; private RecordWriter recordWriter; - + /** * {@inheritDoc} */ @@ -328,9 +329,9 @@ public void invoke() throws Exception { boolean failing = false; - final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + final int failAfterRecord = getTaskConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); synchronized (FAILED_ONCE) { - failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( + failing = (getIndexInSubtaskGroup() == getTaskConfiguration().getInteger( FAILURE_INDEX_KEY, -1)) && FAILED_ONCE.add(getEnvironment().getTaskName()); } @@ -346,13 +347,14 @@ public void invoke() throws Exception { } } } + @ForceCheckpoint(checkpoint = false) public final static class NoCheckpointInnerTask extends AbstractTask { private MutableRecordReader recordReader; private RecordWriter recordWriter; - + /** * {@inheritDoc} */ @@ -374,9 +376,9 @@ public void invoke() throws Exception { boolean failing = false; - final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + final int failAfterRecord = getTaskConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); synchronized (FAILED_ONCE) { - failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( + failing = (getIndexInSubtaskGroup() == getTaskConfiguration().getInteger( FAILURE_INDEX_KEY, -1)) && FAILED_ONCE.add(getEnvironment().getTaskName()); } @@ -392,6 +394,7 @@ public void invoke() throws Exception { } } } + public final static class RefailingInnerTask extends AbstractTask { private MutableRecordReader recordReader; @@ -419,9 +422,8 @@ public void invoke() throws Exception { boolean failing = false; - final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); - failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger(FAILURE_INDEX_KEY, -1)); - + final int failAfterRecord = getTaskConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + failing = (getIndexInSubtaskGroup() == getTaskConfiguration().getInteger(FAILURE_INDEX_KEY, -1)); int count = 0; @@ -435,6 +437,7 @@ public void invoke() throws Exception { } } } + public static final class OutputTask extends AbstractOutputTask { private MutableRecordReader recordReader; @@ -457,9 +460,9 @@ public void invoke() throws Exception { boolean failing = false; - final int failAfterRecord = getRuntimeConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); + final int failAfterRecord = getTaskConfiguration().getInteger(FAILED_AFTER_RECORD_KEY, -1); synchronized (FAILED_ONCE) { - failing = (getIndexInSubtaskGroup() == getRuntimeConfiguration().getInteger( + failing = (getIndexInSubtaskGroup() == getTaskConfiguration().getInteger( FAILURE_INDEX_KEY, -1)) && FAILED_ONCE.add(getEnvironment().getTaskName()); } @@ -496,7 +499,6 @@ public void testFailingInternalVertex() { innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); - final JobTaskVertex innerVertex2 = new JobTaskVertex("Inner vertex 2", jobGraph); innerVertex2.setTaskClass(InnerTask.class); innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); @@ -600,6 +602,7 @@ public void testRecoveryFromFileChannels() { fail(StringUtils.stringifyException(e)); } } + /** * This test checks Nephele's capabilities to recover from network channel checkpoints. */ @@ -617,14 +620,14 @@ public void testRecoveryFromNetworkChannels() { innerVertex1.setTaskClass(InnerTask.class); innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); - + final JobTaskVertex innerVertex2 = new JobTaskVertex("Inner vertex 2", jobGraph); innerVertex2.setTaskClass(InnerTask.class); innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); innerVertex2.getConfiguration().setInteger(FAILED_AFTER_RECORD_KEY, 95490); innerVertex2.getConfiguration().setInteger(FAILURE_INDEX_KEY, 0); - + final JobGenericOutputVertex output = new JobGenericOutputVertex("Output", jobGraph); output.setOutputClass(OutputTask.class); output.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); @@ -660,6 +663,7 @@ public void testRecoveryFromNetworkChannels() { fail(StringUtils.stringifyException(e)); } } + /** * This test checks Nephele's fault tolerance capabilities by simulating a failing input vertex. */ @@ -684,7 +688,6 @@ public void testFailingInputVertex() { innerVertex2.setTaskClass(InnerTask.class); innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); - final JobTaskVertex innerVertex3 = new JobTaskVertex("Inner vertex 3", jobGraph); innerVertex3.setTaskClass(InnerTask.class); @@ -801,6 +804,7 @@ public void testSuccessivelyFailingInnerVertices() { } } + /** * This test checks Nephele's fault tolerance capabilities by simulating a successively failing one inner vertices. */ @@ -815,7 +819,7 @@ public void testRepeatedlyFailingSameInnerVertex() { input.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); final JobTaskVertex innerVertex1 = new JobTaskVertex("Inner vertex 1", jobGraph); - //Using re-failing inner task + // Using re-failing inner task innerVertex1.setTaskClass(RefailingInnerTask.class); innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); @@ -866,13 +870,13 @@ public void testRepeatedlyFailingSameInnerVertex() { } catch (IOException ioe) { fail(StringUtils.stringifyException(ioe)); } catch (JobExecutionException e) { - //This is expected here - assert(e.isJobCanceledByUser() == false); + // This is expected here + assert (e.isJobCanceledByUser() == false); return; } fail("Job expected to be cancled"); } - + /** * This test checks Nephele's fault tolerance capabilities by simulating a successively failing one inner vertices. */ @@ -945,10 +949,11 @@ public void testSuccessivelyFailingSeveralInnerVertices() { } } - - @After public void cleanUp(){ + + @After + public void cleanUp() { File file = new File("/tmp/"); - File[] files= file.listFiles(); + File[] files = file.listFiles(); for (int i = 0; i < files.length; i++) { String name = files[i].getName(); @@ -956,13 +961,14 @@ public void testSuccessivelyFailingSeveralInnerVertices() { files[i].delete(); } } - + System.out.println("deleted"); - - + } + /** - * This test checks Nephele's fault tolerance capabilities by simulating a failing inner vertex without all tasks checkpointing. + * This test checks Nephele's fault tolerance capabilities by simulating a failing inner vertex without all tasks + * checkpointing. */ @Test public void testFailingInternalVertexSomeCheckpoints() { @@ -979,12 +985,10 @@ public void testFailingInternalVertexSomeCheckpoints() { innerVertex1.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex1.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); - final JobTaskVertex innerVertex2 = new JobTaskVertex("Inner vertex 2", jobGraph); innerVertex2.setTaskClass(NoCheckpointInnerTask.class); innerVertex2.setNumberOfSubtasks(DEGREE_OF_PARALLELISM); innerVertex2.setNumberOfSubtasksPerInstance(DEGREE_OF_PARALLELISM); - final JobTaskVertex innerVertex3 = new JobTaskVertex("Inner vertex 3", jobGraph); innerVertex3.setTaskClass(InnerTask.class); diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java index e2b50dcf97595..9cfd7937b539a 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/AbstractPactTask.java @@ -443,7 +443,7 @@ public static OutputCollector getOutputCollector(AbstractInvokable task, TaskCon final OutputEmitter oe = (keyPositions == null || keyClasses == null) ? new OutputEmitter(strategy) : - new OutputEmitter(strategy, task.getEnvironment().getJobID(), keyPositions, keyClasses); + new OutputEmitter(strategy, keyPositions, keyClasses); if (task instanceof AbstractTask) { output.addWriter(new RecordWriter((AbstractTask) task, PactRecord.class, oe)); diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/OutputEmitter.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/OutputEmitter.java index 93af63c76d7ef..09e019a143fb5 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/OutputEmitter.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/OutputEmitter.java @@ -15,13 +15,7 @@ package eu.stratosphere.pact.runtime.task.util; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.io.ChannelSelector; -import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.pact.common.type.DeserializationException; import eu.stratosphere.pact.common.type.Key; import eu.stratosphere.pact.common.type.PactRecord; @@ -65,8 +59,6 @@ public enum ShipStrategy { private int[] keyPositions; private final byte[] salt; // the salt used to randomize the hash values - - private JobID jobId; // the job ID is necessary to obtain the class loader private PartitionFunction partitionFunction; @@ -94,21 +86,20 @@ public OutputEmitter(ShipStrategy strategy) this.salt = DEFAULT_SALT; } - public OutputEmitter(ShipStrategy strategy, JobID jobId, int[] keyPositions, Class[] keyTypes) + public OutputEmitter(ShipStrategy strategy, int[] keyPositions, Class[] keyTypes) { - this(strategy, jobId, DEFAULT_SALT, keyPositions, keyTypes); + this(strategy, DEFAULT_SALT, keyPositions, keyTypes); } - public OutputEmitter(ShipStrategy strategy, JobID jobId, byte[] salt , int[] keyPositions, Class[] keyTypes) + public OutputEmitter(ShipStrategy strategy, byte[] salt , int[] keyPositions, Class[] keyTypes) { - if (strategy == null | jobId == null | salt == null | keyPositions == null | keyTypes == null) { + if (strategy == null | salt == null | keyPositions == null | keyTypes == null) { throw new NullPointerException(); } this.strategy = strategy; this.salt = salt; this.keyPositions = keyPositions; this.keyClasses = keyTypes; - this.jobId = jobId; } // ------------------------------------------------------------------------ @@ -208,79 +199,4 @@ private final int[] hashPartitionDefault(PactRecord record, int numberOfChannels this.channels[0] = (hash < 0) ? -hash % numberOfChannels : hash % numberOfChannels; return this.channels; } - - // ------------------------------------------------------------------------ - // Serialization - // ------------------------------------------------------------------------ - - /* - * (non-Javadoc) - * @see eu.stratosphere.nephele.io.IOReadableWritable#read(java.io.DataInput) - */ - @Override - public void read(DataInput in) throws IOException - { - // strategy - this.strategy = ShipStrategy.valueOf(in.readUTF()); - - // check whether further parameters come - final boolean keyParameterized = in.readBoolean(); - - if (keyParameterized) { - // read the jobID to find the classloader - this.jobId = new JobID(); - this.jobId.read(in); - final ClassLoader loader = LibraryCacheManager.getClassLoader(this.jobId); - - // read the number of keys and key positions - int numKeys = in.readInt(); - this.keyPositions = new int[numKeys]; - for (int i = 0; i < numKeys; i++) { - this.keyPositions[i] = in.readInt(); - } - - // read the key types - @SuppressWarnings("unchecked") - Class[] classes = (Class[]) new Class[numKeys]; - try { - for (int i = 0; i < numKeys; i++) { - String className = in.readUTF(); - classes[i] = Class.forName(className, true, loader).asSubclass(Key.class); - } - } - catch (Exception e) { - throw new RuntimeException("Output Emmitter is unable to load the classes that describe the key types: " - + e.getMessage(), e); - } - this.keyClasses = classes; - } - } - - /* - * (non-Javadoc) - * @see eu.stratosphere.nephele.io.IOReadableWritable#write(java.io.DataOutput) - */ - @Override - public void write(DataOutput out) throws IOException - { - out.writeUTF(strategy.name()); - - if (this.keyClasses != null) { - // write additional info - out.writeBoolean(true); - this.jobId.write(out); - - // write number of keys, key positions and key types - out.writeInt(this.keyClasses.length); - for (int i = 0; i < this.keyPositions.length; i++) { - out.writeInt(this.keyPositions[i]); - } - for (int i = 0; i < this.keyClasses.length; i++) { - out.writeUTF(this.keyClasses[i].getName()); - } - } - else { - out.writeBoolean(false); - } - } } diff --git a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java index 747a3a65aff89..bd1e5efcdbedb 100644 --- a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java +++ b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java @@ -25,16 +25,13 @@ import org.junit.Test; import org.junit.runner.RunWith; -import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; -import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.pact.common.type.PactRecord; import eu.stratosphere.pact.common.type.base.PactDouble; import eu.stratosphere.pact.common.type.base.PactInteger; -import eu.stratosphere.pact.common.type.base.PactLong; import eu.stratosphere.pact.common.type.base.PactString; import eu.stratosphere.pact.runtime.task.util.OutputEmitter.ShipStrategy; @@ -47,7 +44,7 @@ public static void testPartitionHash() { // Test for PactInteger @SuppressWarnings("unchecked") - OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new JobID(), new int[] {0}, new Class[] {PactInteger.class}); + OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new int[] {0}, new Class[] {PactInteger.class}); int numChans = 100; int numRecs = 50000; @@ -72,7 +69,7 @@ public static void testPartitionHash() { // Test for PactString @SuppressWarnings("unchecked") - OutputEmitter oe2 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new JobID(), new int[] {0}, new Class[] {PactString.class}); + OutputEmitter oe2 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new int[] {0}, new Class[] {PactString.class}); numChans = 100; numRecs = 10000; @@ -103,7 +100,7 @@ public static void testForward() { // Test for PactInteger @SuppressWarnings("unchecked") - OutputEmitter oe1 = new OutputEmitter(ShipStrategy.FORWARD, new JobID(), new int[] {0}, new Class[] {PactInteger.class}); + OutputEmitter oe1 = new OutputEmitter(ShipStrategy.FORWARD, new int[] {0}, new Class[] {PactInteger.class}); int numChannels = 100; int numRecords = 50000; @@ -129,7 +126,7 @@ public static void testForward() { // Test for PactString @SuppressWarnings("unchecked") - OutputEmitter oe2 = new OutputEmitter(ShipStrategy.FORWARD, new JobID(), new int[] {0}, new Class[] {PactString.class}); + OutputEmitter oe2 = new OutputEmitter(ShipStrategy.FORWARD, new int[] {0}, new Class[] {PactString.class}); numChannels = 100; numRecords = 10000; @@ -160,7 +157,7 @@ public static void testBroadcast() { // Test for PactInteger @SuppressWarnings("unchecked") - OutputEmitter oe1 = new OutputEmitter(ShipStrategy.BROADCAST, new JobID(), new int[] {0}, new Class[] {PactInteger.class}); + OutputEmitter oe1 = new OutputEmitter(ShipStrategy.BROADCAST, new int[] {0}, new Class[] {PactInteger.class}); int numChannels = 100; int numRecords = 50000; @@ -183,7 +180,7 @@ public static void testBroadcast() { // Test for PactString @SuppressWarnings("unchecked") - OutputEmitter oe2 = new OutputEmitter(ShipStrategy.BROADCAST, new JobID(), new int[] {0}, new Class[] {PactString.class}); + OutputEmitter oe2 = new OutputEmitter(ShipStrategy.BROADCAST, new int[] {0}, new Class[] {PactString.class}); numChannels = 100; numRecords = 5000; @@ -209,7 +206,7 @@ public static void testBroadcast() { public static void testPartitionRange() { @SuppressWarnings("unchecked") - OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_RANGE, new JobID(), new int[] {0}, new Class[] {PactInteger.class}); + OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_RANGE, new int[] {0}, new Class[] {PactInteger.class}); boolean correctException = false; try { @@ -227,7 +224,7 @@ public static void testPartitionRange() { public static void testMultiKeys() { @SuppressWarnings("unchecked") - OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new JobID(), new int[] {0,1,3}, new Class[] {PactInteger.class, PactString.class, PactDouble.class}); + OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new int[] {0,1,3}, new Class[] {PactInteger.class, PactString.class, PactDouble.class}); int numChannels = 100; int numRecords = 5000; @@ -259,7 +256,7 @@ public static void testMultiKeys() { public static void testMissingKey() { @SuppressWarnings("unchecked") - OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new JobID(), new int[] {1}, new Class[] {PactInteger.class}); + OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new int[] {1}, new Class[] {PactInteger.class}); PactRecord rec = new PactRecord(0); rec.setField(0, new PactInteger(1)); @@ -279,7 +276,7 @@ public static void testMissingKey() { public static void testNullKey() { @SuppressWarnings("unchecked") - OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new JobID(), new int[] {0}, new Class[] {PactInteger.class}); + OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new int[] {0}, new Class[] {PactInteger.class}); PactRecord rec = new PactRecord(2); rec.setField(1, new PactInteger(1)); @@ -299,7 +296,7 @@ public static void testNullKey() { public static void testWrongKeyClass() { @SuppressWarnings("unchecked") - OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new JobID(), new int[] {0}, new Class[] {PactDouble.class}); + OutputEmitter oe1 = new OutputEmitter(ShipStrategy.PARTITION_HASH, new int[] {0}, new Class[] {PactDouble.class}); PipedInputStream pipedInput = new PipedInputStream(1024*1024); DataInputStream in = new DataInputStream(pipedInput); @@ -329,65 +326,4 @@ public static void testWrongKeyClass() { } assertTrue(correctException); } - - @Test - public static void testSerialization() throws IOException { - - final JobID jobId = new JobID(); - - PowerMockito.mockStatic( LibraryCacheManager.class ); - PowerMockito.when( LibraryCacheManager.getClassLoader(jobId) ).thenReturn(PactInteger.class.getClassLoader()); - - PipedInputStream pipedInput = new PipedInputStream(1024*1024); - DataInputStream in = new DataInputStream(pipedInput); - DataOutputStream out = null; - try { - out = new DataOutputStream(new PipedOutputStream(pipedInput)); - } catch (IOException e1) { - fail("Test erroneous"); - } - - OutputEmitter oe; - - oe = new OutputEmitter(); - try { - oe.write(out); - oe.read(in); - } catch (IOException e) { - fail("Error serializing output emitter."); - } - - oe = new OutputEmitter(ShipStrategy.PARTITION_HASH); - try { - oe.write(out); - oe.read(in); - } catch (IOException e) { - fail("Error serializing output emitter."); - } - - oe = new OutputEmitter(ShipStrategy.PARTITION_HASH, jobId, new int[] {0}, new Class[] {PactInteger.class}); - try { - oe.write(out); - oe.read(in); - } catch (IOException e) { - fail("Error serializing output emitter."); - } - - oe = new OutputEmitter(ShipStrategy.PARTITION_HASH, jobId, new int[] {0,3,5,7}, new Class[] {PactInteger.class, PactString.class, PactDouble.class, PactLong.class}); - try { - oe.write(out); - oe.read(in); - } catch (IOException e) { - fail("Error serializing output emitter."); - } - - oe = new OutputEmitter(ShipStrategy.BROADCAST, jobId, new byte[] {1,2,3,4,5,6,7,8}, new int[] {0}, new Class[] {PactInteger.class}); - try { - oe.write(out); - oe.read(in); - } catch (IOException e) { - fail("Error serializing output emitter."); - } - - } } diff --git a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java index a40cc32e520a5..2056f07b71d70 100644 --- a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java +++ b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java @@ -71,24 +71,24 @@ public void initEnvironment(long memorySize) { public void addInput(MutableObjectIterator input, int groupId) { this.mockEnv.addInput(input); - new TaskConfig(this.mockEnv.getRuntimeConfiguration()).addInputShipStrategy(ShipStrategy.FORWARD, groupId); + new TaskConfig(this.mockEnv.getTaskConfiguration()).addInputShipStrategy(ShipStrategy.FORWARD, groupId); } public void addOutput(List output) { this.mockEnv.addOutput(output); - new TaskConfig(this.mockEnv.getRuntimeConfiguration()).addOutputShipStrategy(ShipStrategy.FORWARD); + new TaskConfig(this.mockEnv.getTaskConfiguration()).addOutputShipStrategy(ShipStrategy.FORWARD); } public TaskConfig getTaskConfig() { - return new TaskConfig(this.mockEnv.getRuntimeConfiguration()); + return new TaskConfig(this.mockEnv.getTaskConfiguration()); } public Configuration getConfiguration() { - return this.mockEnv.getRuntimeConfiguration(); + return this.mockEnv.getTaskConfiguration(); } public void registerTask(AbstractTask task, Class stubClass) { - new TaskConfig(this.mockEnv.getRuntimeConfiguration()).setStubClass(stubClass); + new TaskConfig(this.mockEnv.getTaskConfiguration()).setStubClass(stubClass); task.setEnvironment(this.mockEnv); task.registerInputOutput(); } @@ -101,7 +101,7 @@ public void registerTask(AbstractTask task) { public void registerFileOutputTask(AbstractOutputTask outTask, Class stubClass, String outPath) { - TaskConfig dsConfig = new TaskConfig(this.mockEnv.getRuntimeConfiguration()); + TaskConfig dsConfig = new TaskConfig(this.mockEnv.getTaskConfiguration()); dsConfig.setStubClass(stubClass); dsConfig.setStubParameter(FileOutputFormat.FILE_PARAMETER_KEY, outPath); @@ -113,7 +113,7 @@ public void registerFileOutputTask(AbstractOutputTask outTask, public void registerFileInputTask(AbstractInputTask inTask, Class stubClass, String inPath, String delimiter) { - TaskConfig dsConfig = new TaskConfig(this.mockEnv.getRuntimeConfiguration()); + TaskConfig dsConfig = new TaskConfig(this.mockEnv.getTaskConfiguration()); dsConfig.setStubClass(stubClass); dsConfig.setStubParameter(FileInputFormat.FILE_PARAMETER_KEY, inPath); dsConfig.setStubParameter(DelimitedInputFormat.RECORD_DELIMITER, delimiter); From b20a3aa6c9fd2597c8d264a0a30a7d27aa30f709 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 30 Jan 2012 17:39:53 +0100 Subject: [PATCH 185/310] Continued to work on integration of streaming branch into checkpointing branch --- .../CheckpointReplayRequest.java | 121 ++++++++++++++++++ .../nephele/instance/AbstractInstance.java | 5 +- .../nephele/jobmanager/DeploymentManager.java | 8 +- .../nephele/jobmanager/JobManager.java | 11 +- .../scheduler/AbstractScheduler.java | 20 +-- .../jobmanager/scheduler/RecoveryLogic.java | 43 +++++-- .../nephele/plugins/TaskManagerPlugin.java | 6 +- .../protocols/TaskOperationProtocol.java | 3 +- .../nephele/taskmanager/RuntimeTask.java | 93 ++++++++++++-- .../nephele/taskmanager/Task.java | 83 ++++++++++++ .../nephele/taskmanager/TaskManager.java | 49 +++---- .../ByteBufferedChannelManager.java | 6 +- 12 files changed, 371 insertions(+), 77 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayRequest.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayRequest.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayRequest.java new file mode 100644 index 0000000000000..1992ae5c82012 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayRequest.java @@ -0,0 +1,121 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Collections; +import java.util.Set; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.IOReadableWritable; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.util.SerializableHashSet; + +/** + * A checkpoint replay request can be used to trigger the replay of a checkpoint at a remote task manager. + * + * @author warneke + */ +public final class CheckpointReplayRequest implements IOReadableWritable { + + /** + * The vertex ID which identifies the checkpoint to be replayed. + */ + private ExecutionVertexID vertexID; + + /** + * The IDs of the output channels the replay task will be in charge of. + */ + private SerializableHashSet outputChannelIDs = new SerializableHashSet(); + + /** + * Constructs a new checkpoint replay request. + * + * @param vertexID + * the vertex ID identifying the checkpoint to be replayed + */ + public CheckpointReplayRequest(final ExecutionVertexID vertexID) { + + if (vertexID == null) { + throw new IllegalArgumentException("Argument vertexID must no be null"); + } + + this.vertexID = vertexID; + } + + /** + * Default constructor for serialization/deserialization. + */ + public CheckpointReplayRequest() { + + this.vertexID = new ExecutionVertexID(); + } + + /** + * Adds a set of channel IDs which identify the output channels the replay task will be in charge of. + * + * @param outputChannelIDs + * the IDs of the output channels the replay task will be in charge of + */ + public void addOutputChannelIDs(final Set outputChannelIDs) { + + this.outputChannelIDs.addAll(outputChannelIDs); + } + + /** + * Returns the set of channel IDs which identify the output channels the replay task will be in charge of. + * + * @return the IDs of the output channels the replay task will be in charge of + */ + public Set getOutputChannelIDs() { + + return Collections.unmodifiableSet(this.outputChannelIDs); + } + + /** + * Returns the vertex ID identifying the checkpoint to be replayed. + * + * @return the vertex ID identifying the checkpoint to be replayed + */ + public ExecutionVertexID getVertexID() { + + return this.vertexID; + } + + /** + * {@inheritDoc} + */ + @Override + public void write(final DataOutput out) throws IOException { + + this.vertexID.write(out); + + this.outputChannelIDs.write(out); + } + + /** + * {@inheritDoc} + */ + @Override + public void read(final DataInput in) throws IOException { + + this.vertexID.read(in); + + this.outputChannelIDs.read(in); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index cc11e5cf6ee1c..55cc2d222c9d5 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -21,6 +21,7 @@ import java.util.Set; import eu.stratosphere.nephele.checkpointing.CheckpointDecision; +import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; @@ -238,10 +239,10 @@ public synchronized List submitTasks(final List replayCheckpoints(final List vertexIDs) + public synchronized List replayCheckpoints(final List replayRequests) throws IOException { - return getTaskManager().replayCheckpoints(vertexIDs); + return getTaskManager().replayCheckpoints(replayRequests); } public synchronized void propagateCheckpointDecisions(final List checkpointDecisions) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java index 994cc4fb44cf2..0222356b641bc 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java @@ -17,8 +17,8 @@ import java.util.List; +import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobmanager.scheduler.AbstractScheduler; @@ -51,8 +51,8 @@ public interface DeploymentManager { * the ID of the job the checkpoints to be replayed belongs to * @param instance * the instance on which the checkpoints shall be replayed - * @param checkpointsToBeReplayed - * the list of vertexIDs referencing the checkpoints to be replayed + * @param replayRequests + * the list of replay requests */ - void replayCheckpoints(JobID jobID, AbstractInstance instance, List vertexIDs); + void replayCheckpoints(JobID jobID, AbstractInstance instance, List replayRequests); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 9b19268a8d488..697943c6d1336 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -59,6 +59,7 @@ import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.checkpointing.CheckpointDecisionCoordinator; import eu.stratosphere.nephele.checkpointing.CheckpointDecisionPropagator; +import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; import eu.stratosphere.nephele.client.AbstractJobResult; import eu.stratosphere.nephele.client.JobCancelResult; @@ -1238,9 +1239,9 @@ public void run() { */ @Override public void replayCheckpoints(final JobID jobID, final AbstractInstance instance, - final List vertexIDs) { + final List replayRequests) { - if (vertexIDs.isEmpty()) { + if (replayRequests.isEmpty()) { LOG.error("Method 'replayCheckpoints' called but list of checkpoints to be replayed is empty"); return; } @@ -1257,15 +1258,15 @@ public void run() { List checkpointResultList = null; try { - checkpointResultList = instance.replayCheckpoints(vertexIDs); + checkpointResultList = instance.replayCheckpoints(replayRequests); } catch (final IOException ioe) { final String errorMsg = StringUtils.stringifyException(ioe); // TODO: Handle this correctly LOG.error(errorMsg); } - if (vertexIDs.size() != checkpointResultList.size()) { - LOG.error("size of submission result list does not match size of list with vertices to be deployed"); + if (replayRequests.size() != checkpointResultList.size()) { + LOG.error("size of submission result list does not match size of list with checkpoints to be deployed"); } for (final CheckpointReplayResult ccr : checkpointResultList) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index c9e010840c462..7fc485f4bd022 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.util.StringUtils; +import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.execution.RuntimeEnvironment; @@ -471,26 +472,29 @@ protected void replayCheckpointsFromPreviousStage(final ExecutionGraph execution final int currentStageIndex = executionGraph.getIndexOfCurrentExecutionStage(); final ExecutionStage previousStage = executionGraph.getStage(currentStageIndex - 1); - final Map> checkpointsToReplay = new HashMap>(); + final Map> checkpointsToReplay = new HashMap>(); for (int i = 0; i < previousStage.getNumberOfOutputExecutionVertices(); ++i) { final ExecutionVertex vertex = previousStage.getOutputExecutionVertex(i); final AbstractInstance instance = vertex.getAllocatedResource().getInstance(); - List vertexIDs = checkpointsToReplay.get(instance); - if (vertexIDs == null) { - vertexIDs = new SerializableArrayList(); - checkpointsToReplay.put(instance, vertexIDs); + List replayRequests = checkpointsToReplay.get(instance); + if (replayRequests == null) { + replayRequests = new SerializableArrayList(); + checkpointsToReplay.put(instance, replayRequests); } - vertexIDs.add(vertex.getID()); + final CheckpointReplayRequest replayRequest = new CheckpointReplayRequest(vertex.getID()); + replayRequest.addOutputChannelIDs(vertex.getEnvironment().getOutputChannelIDs()); + + replayRequests.add(replayRequest); } - final Iterator>> it = checkpointsToReplay.entrySet() + final Iterator>> it = checkpointsToReplay.entrySet() .iterator(); while (it.hasNext()) { - final Map.Entry> entry = it.next(); + final Map.Entry> entry = it.next(); this.deploymentManager.replayCheckpoints(executionGraph.getJobID(), entry.getKey(), entry.getValue()); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index 09f057dd1359f..6d36bf760d778 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -27,6 +27,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.execution.RuntimeEnvironment; @@ -78,7 +79,7 @@ public static boolean recover(final ExecutionVertex failedVertex, final Set verticesToBeCanceled = new HashSet(); - final Map> checkpointsToBeReplayed = new HashMap>(); + final Map> checkpointsToBeReplayed = new HashMap>(); findVerticesToRestart(failedVertex, verticesToBeCanceled, checkpointsToBeReplayed); @@ -104,16 +105,17 @@ public static boolean recover(final ExecutionVertex failedVertex, } // Replay all necessary checkpoints - final Iterator>> checkpointIterator = checkpointsToBeReplayed + final Iterator>> checkpointIterator = checkpointsToBeReplayed .entrySet().iterator(); while (checkpointIterator.hasNext()) { - final Map.Entry> entry = checkpointIterator.next(); + final Map.Entry> entry = checkpointIterator.next(); final AbstractInstance instance = entry.getKey(); try { - final List results = instance.replayCheckpoints(entry.getValue()); + final List results = instance.replayCheckpoints(toListOfReplayRequests( + failedVertex.getExecutionGraph(), entry.getValue())); for (final CheckpointReplayResult result : results) { if (result.getReturnCode() != ReturnCode.SUCCESS) { LOG.error(result.getDescription()); @@ -137,9 +139,32 @@ public static boolean recover(final ExecutionVertex failedVertex, return true; } + private static List toListOfReplayRequests(final ExecutionGraph executionGraph, + final Set vertexIDs) { + + final List replayRequests = new SerializableArrayList(); + final Iterator it = vertexIDs.iterator(); + while (it.hasNext()) { + + final ExecutionVertexID vertexID = it.next(); + final ExecutionVertex vertex = executionGraph.getVertexByID(vertexID); + + if (vertex == null) { + LOG.error("Cannot find execution vertex with ID " + vertexID); + continue; + } + + final CheckpointReplayRequest replayRequest = new CheckpointReplayRequest(vertexID); + replayRequest.addOutputChannelIDs(vertex.getEnvironment().getOutputChannelIDs()); + replayRequests.add(replayRequest); + } + + return replayRequests; + } + private static void findVerticesToRestart(final ExecutionVertex failedVertex, final Set verticesToBeCanceled, - final Map> checkpointsToBeReplayed) { + final Map> checkpointsToBeReplayed) { final Queue verticesToTest = new ArrayDeque(); final Set visited = new HashSet(); @@ -167,15 +192,13 @@ private static void findVerticesToRestart(final ExecutionVertex failedVertex, // Group IDs by instance final AbstractInstance instance = predecessor.getAllocatedResource().getInstance(); - List checkpointIDs = checkpointsToBeReplayed.get(instance); + Set checkpointIDs = checkpointsToBeReplayed.get(instance); if (checkpointIDs == null) { - checkpointIDs = new SerializableArrayList(); + checkpointIDs = new HashSet(); checkpointsToBeReplayed.put(instance, checkpointIDs); } - if (!checkpointIDs.contains(predecessor.getID())) { - checkpointIDs.add(predecessor.getID()); - } + checkpointIDs.add(predecessor.getID()); } } visited.add(vertex); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java index 1e05b4399ad60..d49ce2b8c6fd3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/plugins/TaskManagerPlugin.java @@ -16,7 +16,7 @@ package eu.stratosphere.nephele.plugins; import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.RuntimeEnvironment; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; public interface TaskManagerPlugin extends PluginCommunication { @@ -31,7 +31,7 @@ public interface TaskManagerPlugin extends PluginCommunication { * @param environment * the environment of the task */ - void registerTask(ExecutionVertexID id, Configuration jobConfiguration, RuntimeEnvironment environment); + void registerTask(ExecutionVertexID id, Configuration jobConfiguration, Environment environment); /** * Unregisters a finished, canceled, or failed task from this task manager plugin. @@ -41,7 +41,7 @@ public interface TaskManagerPlugin extends PluginCommunication { * @param environment * the environment of the task */ - void unregisterTask(ExecutionVertexID id, RuntimeEnvironment environment); + void unregisterTask(ExecutionVertexID id, Environment environment); /** * Called by the task manager to indicate that Nephele is about to shut down. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index 78c85967d1f98..82b9f7588058f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -20,6 +20,7 @@ import java.util.Set; import eu.stratosphere.nephele.checkpointing.CheckpointDecision; +import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.RuntimeEnvironment; @@ -119,7 +120,7 @@ List submitTasks(List tasks) */ void updateLibraryCache(LibraryCacheUpdate update) throws IOException; - List replayCheckpoints(List vertexIDs) throws IOException; + List replayCheckpoints(List replayRequests) throws IOException; void propagateCheckpointDecisions(List checkpointDecisions) throws IOException; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/RuntimeTask.java index c4d9a94211f9d..6098342f3a8b9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/RuntimeTask.java @@ -29,6 +29,7 @@ import eu.stratosphere.nephele.annotations.ForceCheckpoint; import eu.stratosphere.nephele.annotations.Stateful; import eu.stratosphere.nephele.annotations.Stateless; +import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionObserver; @@ -44,7 +45,11 @@ import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.profiling.TaskManagerProfiler; +import eu.stratosphere.nephele.services.iomanager.IOManager; +import eu.stratosphere.nephele.services.memorymanager.MemoryManager; import eu.stratosphere.nephele.template.AbstractInvokable; +import eu.stratosphere.nephele.template.InputSplitProvider; import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.util.StringUtils; @@ -177,24 +182,27 @@ public void unregisterExecutionListener(final ExecutionListener executionListene } /** - * Marks the task as failed and triggers the appropriate state changes. + * {@inheritDoc} */ + @Override public void markAsFailed() { executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly"); } /** - * Cancels the execution of the task (i.e. interrupts the execution thread). + * {@inheritDoc} */ + @Override public void cancelExecution() { cancelOrKillExecution(true); } /** - * Kills the task (i.e. interrupts the execution thread). + * {@inheritDoc} */ + @Override public void killExecution() { cancelOrKillExecution(false); @@ -254,8 +262,9 @@ private void cancelOrKillExecution(final boolean cancel) { } /** - * Starts the execution of this Nephele task. + * {@inheritDoc} */ + @Override public void startExecution() { final Thread thread = this.environment.getExecutingThread(); @@ -345,11 +354,9 @@ public void checkpointStateChanged(final CheckpointState newCheckpointState) { } /** - * Checks if the state of the thread which is associated with this task is TERMINATED. - * - * @return true if the state of this thread which is associated with this task is - * TERMINATED, false otherwise + * {@inheritDoc} */ + @Override public boolean isTerminated() { final Thread executingThread = this.environment.getExecutingThread(); @@ -396,4 +403,74 @@ public ExecutionVertexID getVertexID() { return this.vertexID; } + + /** + * {@inheritDoc} + */ + @Override + public void registerMemoryManager(final MemoryManager memoryManager) { + + this.environment.setMemoryManager(memoryManager); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerIOManager(final IOManager ioManager) { + + this.environment.setIOManager(ioManager); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerInputSplitProvider(final InputSplitProvider inputSplitProvider) { + + this.environment.setInputSplitProvider(inputSplitProvider); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerProfiler(final TaskManagerProfiler taskManagerProfiler, final Configuration jobConfiguration) { + + taskManagerProfiler.registerExecutionListener(this, jobConfiguration); + + for (int i = 0; i < this.environment.getNumberOfInputGates(); i++) { + taskManagerProfiler.registerInputGateListener(this.vertexID, jobConfiguration, + this.environment.getInputGate(i)); + } + + for (int i = 0; i < this.environment.getNumberOfOutputGates(); i++) { + taskManagerProfiler.registerOutputGateListener(this.vertexID, jobConfiguration, + this.environment.getOutputGate(i)); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void unregisterMemoryManager(final MemoryManager memoryManager) { + + if (memoryManager != null) { + memoryManager.releaseAll(this.environment.getInvokable()); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) { + + if (taskManagerProfiler != null) { + taskManagerProfiler.unregisterOutputGateListeners(this.vertexID); + taskManagerProfiler.unregisterInputGateListeners(this.vertexID); + taskManagerProfiler.unregisterExecutionListener(this.vertexID); + } + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index 86d613a29f687..e4be10e0a4ff2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -15,9 +15,14 @@ package eu.stratosphere.nephele.taskmanager; +import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.profiling.TaskManagerProfiler; +import eu.stratosphere.nephele.services.iomanager.IOManager; +import eu.stratosphere.nephele.services.memorymanager.MemoryManager; +import eu.stratosphere.nephele.template.InputSplitProvider; public interface Task { @@ -41,4 +46,82 @@ public interface Task { * @return the environment associated with this task */ Environment getEnvironment(); + + /** + * Marks the task as failed and triggers the appropriate state changes. + */ + void markAsFailed(); + + /** + * Checks if the state of the thread which is associated with this task is TERMINATED. + * + * @return true if the state of this thread which is associated with this task is + * TERMINATED, false otherwise + */ + boolean isTerminated(); + + /** + * Starts the execution of this task. + */ + void startExecution(); + + /** + * Cancels the execution of the task (i.e. interrupts the execution thread). + */ + void cancelExecution(); + + /** + * Kills the task (i.e. interrupts the execution thread). + */ + void killExecution(); + + /** + * Registers the central memory manager with the task. + * + * @param memoryManager + * the central memory manager + */ + void registerMemoryManager(MemoryManager memoryManager); + + /** + * Registers the central IO manager with the task. + * + * @param ioManager + * the central IO manager + */ + void registerIOManager(IOManager ioManager); + + /** + * Registers the input splits provider with the task. + * + * @param inputSplitProvider + * the input split provider + */ + void registerInputSplitProvider(InputSplitProvider inputSplitProvider); + + /** + * Registers the task manager profiler with the task. + * + * @param taskManagerProfiler + * the task manager profiler + * @param jobConfiguration + * the configuration attached to the job + */ + void registerProfiler(TaskManagerProfiler taskManagerProfiler, Configuration jobConfiguration); + + /** + * Unregisters the task from the central memory manager. + * + * @param memoryManager + * the central memory manager + */ + void unregisterMemoryManager(MemoryManager memoryManager); + + /** + * Unregisters the task from the task manager profiler. + * + * @param taskManagerProfiler + * the task manager profiler + */ + void unregisterProfiler(TaskManagerProfiler taskManagerProfiler); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 70a6e8849d8f1..b1528439f2eb9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -40,12 +40,14 @@ import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.checkpointing.CheckpointReplayManager; +import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; import eu.stratosphere.nephele.configuration.ConfigConstants; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.discovery.DiscoveryException; import eu.stratosphere.nephele.discovery.DiscoveryService; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.execution.RuntimeEnvironment; @@ -530,13 +532,6 @@ public List submitTasks(final List final Configuration jobConfiguration = tsw.getConfiguration(); final Set activeOutputChannels = tsw.getActiveOutputChannels(); - // Register task manager components in environment - re.setMemoryManager(this.memoryManager); - re.setIOManager(this.ioManager); - - // Register a new task input split provider - re.setInputSplitProvider(new TaskInputSplitProvider(re.getJobID(), id, this.globalInputSplitProvider)); - // Create task object and register it with the environment final RuntimeTask task = new RuntimeTask(id, re, this); re.setExecutionObserver(task); @@ -574,8 +569,8 @@ public List submitTasks(final List * @return null if the registration has been successful or a {@link TaskSubmissionResult} containing * the error that occurred */ - private TaskSubmissionResult registerRuntimeTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final RuntimeTask task, final Set activeOutputChannels) { + private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Configuration jobConfiguration, + final Task task, final Set activeOutputChannels) { if (id == null) { throw new IllegalArgumentException("Argument id is null"); @@ -595,7 +590,12 @@ private TaskSubmissionResult registerRuntimeTask(final ExecutionVertexID id, fin } } - final RuntimeEnvironment ee = task.getRuntimeEnvironment(); + final Environment ee = task.getEnvironment(); + + // Register task manager components with the task + task.registerMemoryManager(this.memoryManager); + task.registerIOManager(this.ioManager); + task.registerInputSplitProvider(new TaskInputSplitProvider(ee.getJobID(), id, this.globalInputSplitProvider)); // Register the task with the byte buffered channel manager this.byteBufferedChannelManager.register(task, activeOutputChannels); @@ -607,16 +607,7 @@ private TaskSubmissionResult registerRuntimeTask(final ExecutionVertexID id, fin // Register environment, input, and output gates for profiling if (enableProfiling) { - - this.profiler.registerExecutionListener(task, jobConfiguration); - - for (int i = 0; i < ee.getNumberOfInputGates(); i++) { - this.profiler.registerInputGateListener(id, jobConfiguration, ee.getInputGate(i)); - } - - for (int i = 0; i < ee.getNumberOfOutputGates(); i++) { - this.profiler.registerOutputGateListener(id, jobConfiguration, ee.getOutputGate(i)); - } + task.registerProfiler(this.profiler, jobConfiguration); } // Allow plugins to register their listeners for this task @@ -636,14 +627,12 @@ private TaskSubmissionResult registerRuntimeTask(final ExecutionVertexID id, fin * {@inheritDoc} */ @Override - public SerializableArrayList replayCheckpoints(final List vertexIDs) - throws IOException { - - // TODO: Invalidate lookup caches + public SerializableArrayList replayCheckpoints( + final List replayRequests) throws IOException { final SerializableArrayList checkpointResultList = new SerializableArrayList(); - for (final ExecutionVertexID vertexID : vertexIDs) { + for (final CheckpointReplayRequest replayRequest : replayRequests) { if (!this.checkpointManager.hasCompleteCheckpointAvailable(vertexID)) { @@ -688,16 +677,10 @@ private void unregisterTask(final ExecutionVertexID id, final Task task) { this.byteBufferedChannelManager.unregister(id, task); // Unregister task from profiling - if (this.profiler != null) { - this.profiler.unregisterOutputGateListeners(id); - this.profiler.unregisterInputGateListeners(id); - this.profiler.unregisterExecutionListener(id); - } + task.unregisterProfiler(this.profiler); // Unregister task from memory manager - if (this.memoryManager != null) { - this.memoryManager.releaseAll(task.getEnvironment().getInvokable()); - } + task.unregisterMemoryManager(this.memoryManager); // Allow plugins to unregister their listeners for this task if (!this.taskManagerPlugins.isEmpty()) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 960ff2a378a92..2b1ef6fa01f9a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -128,7 +128,7 @@ public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupServi + (this.allowSenderSideSpilling ? "enabled" : "disabled") + (this.mergeSpilledBuffers ? " and spilled buffer merging enabled" : "")); } - + /** * Registers the given task with the byte buffered channel manager. * @@ -137,9 +137,9 @@ public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupServi * @param the * set of output channels which are initially active */ - public void register(final RuntimeTask task, final Set activeOutputChannels) { + public void register(final Task task, final Set activeOutputChannels) { - final RuntimeEnvironment environment = task.getRuntimeEnvironment(); + final Environment environment = task.getEnvironment(); final TaskContext taskContext = new TaskContext(task, this, this.tasksWithUndecidedCheckpoints); From 248949ecd6ec06e8101ae73879d3d2e36090ad5b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 30 Jan 2012 22:35:34 +0100 Subject: [PATCH 186/310] Refactored context class hierarchie used by the byte-buffered channel manager --- .../nephele/execution/Environment.java | 25 ++ .../impl/TaskManagerProfilerImpl.java | 2 +- .../queue/TestDeploymentManager.java | 4 +- .../checkpointing/EphemeralCheckpoint.java | 2 +- .../nephele/execution/RuntimeEnvironment.java | 74 ++++ .../profiling/TaskManagerProfiler.java | 2 +- .../nephele/taskmanager/Task.java | 8 + .../nephele/taskmanager/TaskManager.java | 7 +- .../ByteBufferedChannelManager.java | 129 +++--- .../bytebuffered/ChannelContext.java | 5 +- .../taskmanager/bytebuffered/GateContext.java | 8 + .../bytebuffered/InputChannelContext.java | 305 +------------ .../bytebuffered/InputGateContext.java | 104 +---- .../bytebuffered/OutputChannelContext.java | 380 +--------------- .../bytebuffered/OutputGateContext.java | 212 +-------- .../taskmanager/bytebuffered/TaskContext.java | 277 +----------- .../runtime/RuntimeInputChannelContext.java | 337 +++++++++++++++ .../runtime/RuntimeInputGateContext.java | 180 ++++++++ .../runtime/RuntimeOutputChannelContext.java | 408 ++++++++++++++++++ .../runtime/RuntimeOutputGateContext.java | 273 ++++++++++++ .../{ => runtime}/RuntimeTask.java | 18 +- .../runtime/RuntimeTaskContext.java | 362 ++++++++++++++++ .../TransferEnvelopeReceiverList.java | 8 +- 23 files changed, 1785 insertions(+), 1345 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java rename nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/{ => runtime}/RuntimeTask.java (94%) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java index 645ecca0b76fd..1218c69f93c6d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java @@ -202,10 +202,35 @@ InputGate createInputGate(GateID gateID, RecordDeserializer getInputChannelIDs(); + /** + * Returns the IDs of all output gates connected to this environment. + * + * @return the IDs of all output gates connected to this environment + */ + Set getOutputGateIDs(); + /** * Returns the IDs of all input gates connected to this environment. * * @return the IDs of all input gates connected to this environment */ Set getInputGateIDs(); + + /** + * Returns the IDs of all the output channels connected to the gate with the given ID. + * + * @param gateID + * the gate ID + * @return the IDs of all the output channels connected to the gate with the given ID + */ + Set getOutputChannelIDsOfGate(GateID gateID); + + /** + * Returns the IDs of all the input channels connected to the gate with the given ID. + * + * @param gateID + * the gate ID + * @return the IDs of all the input channels connected to the gate with the given ID + */ + Set getInputChannelIDsOfGate(GateID gateID); } diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java index f3eebc2f940c5..9cc2803b386a5 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java @@ -46,7 +46,7 @@ import eu.stratosphere.nephele.profiling.impl.types.InternalInstanceProfilingData; import eu.stratosphere.nephele.profiling.impl.types.InternalOutputGateProfilingData; import eu.stratosphere.nephele.profiling.impl.types.ProfilingDataContainer; -import eu.stratosphere.nephele.taskmanager.RuntimeTask; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.util.StringUtils; diff --git a/nephele/nephele-queuescheduler/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java b/nephele/nephele-queuescheduler/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java index f0df096d49446..e6d598ed62934 100644 --- a/nephele/nephele-queuescheduler/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java +++ b/nephele/nephele-queuescheduler/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java @@ -17,8 +17,8 @@ import java.util.List; +import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobmanager.DeploymentManager; @@ -68,7 +68,7 @@ public void deploy(final JobID jobID, final AbstractInstance instance, */ @Override public void replayCheckpoints(final JobID jobID, final AbstractInstance instance, - final List vertexIDs) { + final List replayRequests) { throw new IllegalStateException("replayCheckpoints called on TestDeploymentManager"); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 1bcdfbcf3d7e3..30ea85f0ebf1d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -26,7 +26,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.taskmanager.RuntimeTask; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.taskmanager.transferenvelope.CheckpointSerializer; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.configuration.GlobalConfiguration; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 9b55fedea91f5..e6322b3f9e250 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -1063,4 +1063,78 @@ public Set getInputGateIDs() { return Collections.unmodifiableSet(inputGateIDs); } + + /** + * {@inheritDoc} + */ + @Override + public Set getOutputGateIDs() { + + final Set outputGateIDs = new HashSet(); + + final Iterator> gateIterator = this.outputGates.iterator(); + while (gateIterator.hasNext()) { + outputGateIDs.add(gateIterator.next().getGateID()); + } + + return Collections.unmodifiableSet(outputGateIDs); + } + + /** + * {@inheritDoc} + */ + @Override + public Set getOutputChannelIDsOfGate(final GateID gateID) { + + OutputGate outputGate = null; + final Iterator> gateIterator = this.outputGates.iterator(); + while (gateIterator.hasNext()) { + final OutputGate candidateGate = gateIterator.next(); + if (candidateGate.getGateID().equals(gateID)) { + outputGate = candidateGate; + break; + } + } + + if (outputGate == null) { + throw new IllegalArgumentException("Cannot find output gate with ID " + gateID); + } + + final Set outputChannelIDs = new HashSet(); + + for (int i = 0; i < outputGate.getNumberOfOutputChannels(); ++i) { + outputChannelIDs.add(outputGate.getOutputChannel(i).getID()); + } + + return Collections.unmodifiableSet(outputChannelIDs); + } + + /** + * {@inheritDoc} + */ + @Override + public Set getInputChannelIDsOfGate(final GateID gateID) { + + InputGate inputGate = null; + final Iterator> gateIterator = this.inputGates.iterator(); + while (gateIterator.hasNext()) { + final InputGate candidateGate = gateIterator.next(); + if (candidateGate.getGateID().equals(gateID)) { + inputGate = candidateGate; + break; + } + } + + if (inputGate == null) { + throw new IllegalArgumentException("Cannot find input gate with ID " + gateID); + } + + final Set inputChannelIDs = new HashSet(); + + for (int i = 0; i < inputGate.getNumberOfInputChannels(); ++i) { + inputChannelIDs.add(inputGate.getInputChannel(i).getID()); + } + + return Collections.unmodifiableSet(inputChannelIDs); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java index 24cd3bad0390f..8544dba3a1781 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java @@ -22,7 +22,7 @@ import eu.stratosphere.nephele.io.InputGateListener; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.OutputGateListener; -import eu.stratosphere.nephele.taskmanager.RuntimeTask; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.types.Record; /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index e4be10e0a4ff2..70b3e7a06e786 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -15,6 +15,8 @@ package eu.stratosphere.nephele.taskmanager; +import java.util.Map; + import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; @@ -22,6 +24,9 @@ import eu.stratosphere.nephele.profiling.TaskManagerProfiler; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; +import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; import eu.stratosphere.nephele.template.InputSplitProvider; public interface Task { @@ -124,4 +129,7 @@ public interface Task { * the task manager profiler */ void unregisterProfiler(TaskManagerProfiler taskManagerProfiler); + + TaskContext createTaskContext(TransferEnvelopeDispatcher transferEnvelopeDispatcher, + Map tasksWithUndecidedCheckpoints); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index b1528439f2eb9..ceddaac43c8ff 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -81,6 +81,7 @@ import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager; import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; import eu.stratosphere.nephele.taskmanager.bytebuffered.ByteBufferedChannelManager; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.util.SerializableArrayList; import eu.stratosphere.nephele.util.StringUtils; @@ -740,7 +741,7 @@ public void updateLibraryCache(LibraryCacheUpdate update) throws IOException { // Nothing to to here } - void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final Task task, + public void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final Task task, final ExecutionState newExecutionState, final String optionalDescription) { if (newExecutionState == ExecutionState.RUNNING || newExecutionState == ExecutionState.RUNNING) { @@ -784,7 +785,7 @@ void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final * @param resourceUtilizationSnapshot * snapshot of the task's resource utilization taken at the point in time when the exhaustion occurred */ - void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID id, + public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID id, final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { synchronized (this.jobManager) { @@ -796,7 +797,7 @@ void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertex } } - void checkpointStateChanged(final JobID jobID, final ExecutionVertexID id, final CheckpointState newCheckpointState) { + public void checkpointStateChanged(final JobID jobID, final ExecutionVertexID id, final CheckpointState newCheckpointState) { synchronized (this.jobManager) { try { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 2b1ef6fa01f9a..4053b696ad854 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -29,31 +29,23 @@ import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.execution.Environment; -import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; import eu.stratosphere.nephele.io.AbstractID; import eu.stratosphere.nephele.io.GateID; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.channels.AbstractChannel; -import eu.stratosphere.nephele.io.channels.AbstractInputChannel; -import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.io.channels.FileBufferManager; -import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.protocols.ChannelLookupProtocol; -import eu.stratosphere.nephele.taskmanager.RuntimeTask; import eu.stratosphere.nephele.taskmanager.Task; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProviderBroker; import eu.stratosphere.nephele.taskmanager.bufferprovider.GlobalBufferPool; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; @@ -75,7 +67,7 @@ public final class ByteBufferedChannelManager implements TransferEnvelopeDispatc private final Map localBufferPoolOwner = new ConcurrentHashMap(); - private final Map tasksWithUndecidedCheckpoints = new ConcurrentHashMap(); + private final Map tasksWithUndecidedCheckpoints = new ConcurrentHashMap(); private final NetworkConnectionManager networkConnectionManager; @@ -128,7 +120,7 @@ public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupServi + (this.allowSenderSideSpilling ? "enabled" : "disabled") + (this.mergeSpilledBuffers ? " and spilled buffer merging enabled" : "")); } - + /** * Registers the given task with the byte buffered channel manager. * @@ -137,84 +129,83 @@ public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupServi * @param the * set of output channels which are initially active */ - public void register(final Task task, final Set activeOutputChannels) { + public synchronized void register(final Task task, final Set activeOutputChannels) { final Environment environment = task.getEnvironment(); - final TaskContext taskContext = new TaskContext(task, this, this.tasksWithUndecidedCheckpoints); - - for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - final OutputGate outputGate = environment.getOutputGate(i); - final OutputGateContext outputGateContext = new OutputGateContext(taskContext, outputGate.getChannelType(), - outputGate.getIndex()); - for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { - final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); - if (!(outputChannel instanceof AbstractByteBufferedOutputChannel)) { - LOG.error("Output channel " + outputChannel.getID() + "of job " + environment.getJobID() - + " is not a byte buffered output channel, skipping..."); - continue; - } + final TaskContext taskContext = task.createTaskContext(this, this.tasksWithUndecidedCheckpoints); - final AbstractByteBufferedOutputChannel bboc = (AbstractByteBufferedOutputChannel) outputChannel; + final Set outputGateIDs = environment.getOutputGateIDs(); + for (final Iterator gateIt = outputGateIDs.iterator(); gateIt.hasNext();) { - if (this.registeredChannels.containsKey(bboc.getID())) { - LOG.error("Byte buffered output channel " + bboc.getID() + " is already registered"); - continue; - } + final GateID gateID = gateIt.next(); + final OutputGateContext outputGateContext = taskContext.createOutputGateContext(gateID); + final Set outputChannelIDs = environment.getOutputChannelIDsOfGate(gateID); + for (final Iterator channelIt = outputChannelIDs.iterator(); channelIt.hasNext();) { + + final ChannelID channelID = channelIt.next(); + final OutputChannelContext previousContext = (OutputChannelContext) this.registeredChannels + .get(channelID); + + final boolean isActive = activeOutputChannels.contains(channelID); + + final OutputChannelContext outputChannelContext = outputGateContext.createOutputChannelContext( + channelID, previousContext, isActive, this.mergeSpilledBuffers); // Add routing entry to receiver cache to reduce latency - if (bboc.getType() == ChannelType.INMEMORY) { - addReceiverListHint(bboc); + if (outputChannelContext.getType() == ChannelType.INMEMORY) { + addReceiverListHint(outputChannelContext); } - final boolean isActive = activeOutputChannels.contains(bboc.getID()); - if (LOG.isDebugEnabled()) - LOG.debug("Registering byte buffered output channel " + bboc.getID() + " (" + LOG.debug("Registering byte buffered output channel " + outputChannelContext.getChannelID() + " (" + (isActive ? "active" : "inactive") + ")"); - final OutputChannelContext outputChannelContext = new OutputChannelContext(outputGateContext, bboc, - isActive, this.mergeSpilledBuffers); - this.registeredChannels.put(bboc.getID(), outputChannelContext); + this.registeredChannels.put(outputChannelContext.getChannelID(), outputChannelContext); } } - for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { - final InputGate inputGate = environment.getInputGate(i); - final InputGateContext inputGateContext = new InputGateContext(inputGate.getNumberOfInputChannels()); - for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { - final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); - if (!(inputChannel instanceof AbstractByteBufferedInputChannel)) { - LOG.error("Input channel " + inputChannel.getID() + "of job " + environment.getJobID() - + " is not a byte buffered input channel, skipping..."); - continue; - } + final Set inputGateIDs = environment.getInputGateIDs(); + for (final Iterator gateIt = inputGateIDs.iterator(); gateIt.hasNext();) { - final AbstractByteBufferedInputChannel bbic = (AbstractByteBufferedInputChannel) inputChannel; + final GateID gateID = gateIt.next(); + final InputGateContext inputGateContext = taskContext.createInputGateContext(gateID); + final Set inputChannelIDs = environment.getInputChannelIDsOfGate(gateID); + for (final Iterator channelIt = inputChannelIDs.iterator(); channelIt.hasNext();) { - if (this.registeredChannels.containsKey(bbic.getID())) { - LOG.error("Byte buffered input channel " + bbic.getID() + " is already registered"); - continue; - } + final ChannelID channelID = channelIt.next(); + final InputChannelContext previousContext = (InputChannelContext) this.registeredChannels + .get(channelID); + + final InputChannelContext inputChannelContext = inputGateContext.createInputChannelContext( + channelID, previousContext); // Add routing entry to receiver cache to reduce latency - if (bbic.getType() == ChannelType.INMEMORY) { - addReceiverListHint(bbic); + if (inputChannelContext.getType() == ChannelType.INMEMORY) { + addReceiverListHint(inputChannelContext); } + final boolean isActive = activeOutputChannels.contains(inputChannelContext.getChannelID()); + if (LOG.isDebugEnabled()) - LOG.debug("Registering byte buffered input channel " + bbic.getID()); + LOG.debug("Registering byte buffered input channel " + inputChannelContext.getChannelID() + " (" + + (isActive ? "active" : "inactive") + ")"); - final InputChannelContext inputChannelContext = new InputChannelContext(inputGateContext, this, - bbic); - this.registeredChannels.put(bbic.getID(), inputChannelContext); + this.registeredChannels.put(inputChannelContext.getChannelID(), inputChannelContext); } // Add input gate context to set of local buffer pool owner - this.localBufferPoolOwner.put(inputGate.getGateID(), inputGateContext); + final LocalBufferPoolOwner bufferPoolOwner = inputGateContext.getLocalBufferPoolOwner(); + if (bufferPoolOwner != null) { + this.localBufferPoolOwner.put(inputGateContext.getGateID(), bufferPoolOwner); + } + } - this.localBufferPoolOwner.put(task.getVertexID(), taskContext); + final LocalBufferPoolOwner bufferPoolOwner = taskContext.getLocalBufferPoolOwner(); + if (bufferPoolOwner != null) { + this.localBufferPoolOwner.put(task.getVertexID(), bufferPoolOwner); + } redistributeGlobalBuffers(); } @@ -227,7 +218,7 @@ public void register(final Task task, final Set activeOutputChannels) * @param task * the task to be unregistered */ - public void unregister(final ExecutionVertexID vertexID, final Task task) { + public synchronized void unregister(final ExecutionVertexID vertexID, final Task task) { final Environment environment = task.getEnvironment(); @@ -259,9 +250,7 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { final GateID inputGateID = inputGateIterator.next(); final LocalBufferPoolOwner owner = this.localBufferPoolOwner.remove(inputGateID); - if (owner == null) { - LOG.error("Cannot find local buffer pool owner for input gate " + inputGateID); - } else { + if (owner != null) { owner.clearLocalBufferPool(); } } @@ -468,12 +457,12 @@ private boolean processEnvelopeEnvelopeWithoutBuffer(final TransferEnvelope tran return true; } - private void addReceiverListHint(final AbstractChannel channel) { + private void addReceiverListHint(final ChannelContext channelContext) { - TransferEnvelopeReceiverList receiverList = new TransferEnvelopeReceiverList(channel); + TransferEnvelopeReceiverList receiverList = new TransferEnvelopeReceiverList(channelContext); - if (this.receiverCache.put(channel.getID(), receiverList) != null) { - LOG.warn("Receiver cache already contained entry for " + channel.getID()); + if (this.receiverCache.put(channelContext.getChannelID(), receiverList) != null) { + LOG.warn("Receiver cache already contained entry for " + channelContext.getChannelID()); } } @@ -689,7 +678,7 @@ public void reportCheckpointDecisions(final List checkpointD for (final CheckpointDecision cd : checkpointDecisions) { - final TaskContext taskContext = this.tasksWithUndecidedCheckpoints.remove(cd.getVertexID()); + final RuntimeTaskContext taskContext = this.tasksWithUndecidedCheckpoints.remove(cd.getVertexID()); if (taskContext == null) { LOG.error("Cannot report checkpoint decision for vertex " + cd.getVertexID()); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java index 009631ccd3619..203e2bf142561 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java @@ -16,10 +16,11 @@ package eu.stratosphere.nephele.taskmanager.bytebuffered; import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -interface ChannelContext { +public interface ChannelContext { boolean isInputChannel(); @@ -29,5 +30,7 @@ interface ChannelContext { public ChannelID getConnectedChannelID(); + public ChannelType getType(); + public void queueTransferEnvelope(TransferEnvelope transferEnvelope); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java new file mode 100644 index 0000000000000..63ba4e7620629 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java @@ -0,0 +1,8 @@ +package eu.stratosphere.nephele.taskmanager.bytebuffered; + +import eu.stratosphere.nephele.io.GateID; + +public interface GateContext { + + GateID getGateID(); +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java index 15ef4d56ca717..2b1a584b41531 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java @@ -15,310 +15,11 @@ package eu.stratosphere.nephele.taskmanager.bytebuffered; -import java.io.IOException; -import java.util.ArrayDeque; -import java.util.Iterator; -import java.util.Queue; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.event.task.AbstractEvent; -import eu.stratosphere.nephele.event.task.EventList; -import eu.stratosphere.nephele.io.channels.Buffer; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.BufferPairResponse; -import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedInputChannelBroker; -import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; -import eu.stratosphere.nephele.util.StringUtils; - -final class InputChannelContext implements ChannelContext, ByteBufferedInputChannelBroker, BufferProvider { - - private static final Log LOG = LogFactory.getLog(InputChannelContext.class); - - private final InputGateContext inputGateContext; - - private final AbstractByteBufferedInputChannel byteBufferedInputChannel; - - private final TransferEnvelopeDispatcher transferEnvelopeDispatcher; - - private final Queue queuedEnvelopes = new ArrayDeque(); - - private int lastReceivedEnvelope = -1; - - InputChannelContext(final InputGateContext inputGateContext, - final TransferEnvelopeDispatcher transferEnvelopeDispatcher, - final AbstractByteBufferedInputChannel byteBufferedInputChannel) { - - this.inputGateContext = inputGateContext; - this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; - this.byteBufferedInputChannel = byteBufferedInputChannel; - this.byteBufferedInputChannel.setInputChannelBroker(this); - } - - @Override - public BufferPairResponse getReadBufferToConsume() { - - TransferEnvelope transferEnvelope = null; - - synchronized (this.queuedEnvelopes) { - - if (this.queuedEnvelopes.isEmpty()) { - return null; - } - - transferEnvelope = this.queuedEnvelopes.peek(); - - // If envelope does not have a buffer, remove it immediately - if (transferEnvelope.getBuffer() == null) { - this.queuedEnvelopes.poll(); - } - } - - // Make sure we have all necessary buffers before we go on - if (transferEnvelope.getBuffer() == null) { - - // No buffers necessary - final EventList eventList = transferEnvelope.getEventList(); - if (eventList != null) { - if (!eventList.isEmpty()) { - final Iterator it = eventList.iterator(); - while (it.hasNext()) { - this.byteBufferedInputChannel.processEvent(it.next()); - } - } - } - - return null; - } - - // TODO: Fix implementation breaks compression, fix it later on - final BufferPairResponse response = new BufferPairResponse(null, transferEnvelope.getBuffer()); // No need to - - // Moved event processing to releaseConsumedReadBuffer method // copy anything - - return response; - } - - @Override - public void releaseConsumedReadBuffer() { - - TransferEnvelope transferEnvelope = null; - synchronized (this.queuedEnvelopes) { - - if (this.queuedEnvelopes.isEmpty()) { - LOG.error("Inconsistency: releaseConsumedReadBuffer called on empty queue!"); - return; - } - - transferEnvelope = this.queuedEnvelopes.poll(); - } - - // Process events - final EventList eventList = transferEnvelope.getEventList(); - if (eventList != null) { - if (!eventList.isEmpty()) { - final Iterator it = eventList.iterator(); - while (it.hasNext()) { - this.byteBufferedInputChannel.processEvent(it.next()); - } - } - } - - final Buffer consumedBuffer = transferEnvelope.getBuffer(); - if (consumedBuffer == null) { - LOG.error("Inconsistency: consumed read buffer is null!"); - return; - } - - if (consumedBuffer.remaining() > 0) { - LOG.error("consumedReadBuffer has " + consumedBuffer.remaining() + " unconsumed bytes left!!"); - } - - // Recycle consumed read buffer - consumedBuffer.recycleBuffer(); - } - - /** - * {@inheritDoc} - */ - @Override - public void transferEventToOutputChannel(AbstractEvent event) throws IOException, InterruptedException { - - final TransferEnvelope ephemeralTransferEnvelope = new TransferEnvelope(0, getJobID(), getChannelID()); - - ephemeralTransferEnvelope.addEvent(event); - this.transferEnvelopeDispatcher.processEnvelopeFromInputChannel(ephemeralTransferEnvelope); - } - - /** - * {@inheritDoc} - */ - @Override - public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { - - // The sequence number of the envelope to be queued - final int sequenceNumber = transferEnvelope.getSequenceNumber(); - - synchronized (this.queuedEnvelopes) { - - final int expectedSequenceNumber = this.lastReceivedEnvelope + 1; - if (sequenceNumber != expectedSequenceNumber) { - - if (sequenceNumber > expectedSequenceNumber) { - - // This is a problem, now we are actually missing some data - this.byteBufferedInputChannel.reportIOException(new IOException("Expected data packet " - + expectedSequenceNumber + " but received " + sequenceNumber)); - this.byteBufferedInputChannel.checkForNetworkEvents(); - } else { - - // Tell the sender that we are expecting an envelope with a higher sequence number - try { - transferEventToOutputChannel(new UnexpectedEnvelopeEvent(expectedSequenceNumber)); - } catch (Exception e) { - LOG.error(StringUtils.stringifyException(e)); - } - } - - LOG.warn("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber - + " but received " + sequenceNumber); - - final Buffer buffer = transferEnvelope.getBuffer(); - if (buffer != null) { - buffer.recycleBuffer(); - } - - return; - } - - this.queuedEnvelopes.add(transferEnvelope); - - this.lastReceivedEnvelope = sequenceNumber; - } - - // Notify the channel about the new data - this.byteBufferedInputChannel.checkForNetworkEvents(); - } - - @Override - public ChannelID getChannelID() { - - return this.byteBufferedInputChannel.getID(); - } - - @Override - public ChannelID getConnectedChannelID() { - - return this.byteBufferedInputChannel.getConnectedChannelID(); - } - - @Override - public JobID getJobID() { - - return this.byteBufferedInputChannel.getJobID(); - } - - @Override - public boolean isInputChannel() { - - return this.byteBufferedInputChannel.isInputChannel(); - } - - public void releaseAllResources() { - - final Queue buffersToRecycle = new ArrayDeque(); - - synchronized (this.queuedEnvelopes) { - - while (!this.queuedEnvelopes.isEmpty()) { - final TransferEnvelope envelope = this.queuedEnvelopes.poll(); - if (envelope.getBuffer() != null) { - buffersToRecycle.add(envelope.getBuffer()); - } - } - } - - while (!buffersToRecycle.isEmpty()) { - buffersToRecycle.poll().recycleBuffer(); - } - } - - public int getNumberOfQueuedEnvelopes() { - - synchronized (this.queuedEnvelopes) { - - return this.queuedEnvelopes.size(); - } - } - - public int getNumberOfQueuedMemoryBuffers() { - - synchronized (this.queuedEnvelopes) { - - int count = 0; - - final Iterator it = this.queuedEnvelopes.iterator(); - while (it.hasNext()) { - - final TransferEnvelope envelope = it.next(); - if (envelope.getBuffer() != null) { - if (envelope.getBuffer().isBackedByMemory()) { - ++count; - } - } - } - - return count; - } - } - - /** - * {@inheritDoc} - */ - @Override - public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException { - - throw new IllegalStateException("requestEmptyBuffer called on InputChannelContext"); - } - - /** - * {@inheritDoc} - */ - @Override - public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException { - - return this.inputGateContext.requestEmptyBufferBlocking(minimumSizeOfBuffer); - } - - /** - * {@inheritDoc} - */ - @Override - public int getMaximumBufferSize() { - - return this.inputGateContext.getMaximumBufferSize(); - } - - /** - * {@inheritDoc} - */ - @Override - public boolean isShared() { - return this.inputGateContext.isShared(); - } +public interface InputChannelContext extends ChannelContext, BufferProvider { - /** - * {@inheritDoc} - */ - @Override - public void reportAsynchronousEvent() { + int getNumberOfQueuedEnvelopes(); - this.inputGateContext.reportAsynchronousEvent(); - } + int getNumberOfQueuedMemoryBuffers(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java index 88df130b20748..e7bd4a8a3bb21 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java @@ -15,106 +15,12 @@ package eu.stratosphere.nephele.taskmanager.bytebuffered; -import java.io.IOException; - -import eu.stratosphere.nephele.io.channels.Buffer; -import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; -import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; +import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; -final class InputGateContext implements BufferProvider, LocalBufferPoolOwner { - - private final LocalBufferPool localBufferPool; - - private final int numberOfInputChannels; - - InputGateContext(final int numberOfInputChannels) { - - this.localBufferPool = new LocalBufferPool(1, false); - - this.numberOfInputChannels = numberOfInputChannels; - } - - /** - * {@inheritDoc} - */ - @Override - public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException { - - return this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer); - } - - /** - * {@inheritDoc} - */ - @Override - public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException { - - return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer); - } - - /** - * {@inheritDoc} - */ - @Override - public int getMaximumBufferSize() { - - return this.localBufferPool.getMaximumBufferSize(); - } - - /** - * {@inheritDoc} - */ - @Override - public boolean isShared() { - - return this.localBufferPool.isShared(); - } - - /** - * {@inheritDoc} - */ - @Override - public void reportAsynchronousEvent() { - - this.localBufferPool.reportAsynchronousEvent(); - } - - @Override - public int getNumberOfChannels() { - - return this.numberOfInputChannels; - } - - /** - * {@inheritDoc} - */ - @Override - public void setDesignatedNumberOfBuffers(int numberOfBuffers) { - - this.localBufferPool.setDesignatedNumberOfBuffers(numberOfBuffers); - } - - /** - * {@inheritDoc} - */ - @Override - public void clearLocalBufferPool() { - - this.localBufferPool.clear(); - } - - /** - * {@inheritDoc} - */ - @Override - public void logBufferUtilization() { - - final int ava = this.localBufferPool.getNumberOfAvailableBuffers(); - final int req = this.localBufferPool.getRequestedNumberOfBuffers(); - final int des = this.localBufferPool.getDesignatedNumberOfBuffers(); +public interface InputGateContext extends GateContext { - System.out - .println("\t\tInputGateContext: " + ava + " available, " + req + " requested, " + des + " designated"); - } + InputChannelContext createInputChannelContext(ChannelID channelID, InputChannelContext previousContext); + + LocalBufferPoolOwner getLocalBufferPoolOwner(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java index bd158acb8c465..186eb8190a541 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java @@ -15,383 +15,5 @@ package eu.stratosphere.nephele.taskmanager.bytebuffered; -import java.io.IOException; -import java.util.Iterator; -import java.util.Queue; -import java.util.concurrent.LinkedBlockingDeque; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.event.task.AbstractEvent; -import eu.stratosphere.nephele.io.AbstractID; -import eu.stratosphere.nephele.io.channels.Buffer; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.BufferPairResponse; -import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelActivateEvent; -import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; -import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedOutputChannelBroker; -import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; - -final class OutputChannelContext implements ByteBufferedOutputChannelBroker, ChannelContext { - - /** - * The static object used for logging. - */ - private static final Log LOG = LogFactory.getLog(OutputChannelContext.class); - - /** - * The byte buffered output channel this context belongs to. - */ - private final AbstractByteBufferedOutputChannel byteBufferedOutputChannel; - - /** - * The output gate context associated with this context. - */ - private final OutputGateContext outputGateContext; - - /** - * Points to the {@link TransferEnvelope} object that will be passed to the framework upon - * the next releaseWriteBuffers call. - */ - private TransferEnvelope outgoingTransferEnvelope = null; - - /** - * Indicates whether the receiver of an envelope is currently running. - */ - private boolean isReceiverRunning = false; - - /** - * Stores whether the receiver has acknowledged the close request from this channel. - */ - private boolean closeAcknowledgementReceived = false; - - /** - * Queue to store outgoing transfer envelope in case the receiver of the envelopes is not yet running. - */ - private final SpillingQueue queuedOutgoingEnvelopes; - - /** - * Stores incoming events for this output channel. - */ - private final Queue incomingEventQueue = new LinkedBlockingDeque(); - - /** - * The sequence number for the next {@link TransferEnvelope} to be created. - */ - private int sequenceNumber = 0; - - /** - * Stores if the flushing the of spilling queue has already been triggered. - */ - private boolean spillingQueueAlreadyFlushed = false; - - OutputChannelContext(final OutputGateContext outputGateContext, - final AbstractByteBufferedOutputChannel byteBufferedOutputChannel, final boolean isReceiverRunning, - final boolean mergeSpilledBuffers) { - - this.outputGateContext = outputGateContext; - this.byteBufferedOutputChannel = byteBufferedOutputChannel; - this.byteBufferedOutputChannel.setByteBufferedOutputChannelBroker(this); - this.isReceiverRunning = isReceiverRunning; - - final AbstractID fileOwnerID = mergeSpilledBuffers ? outputGateContext.getFileOwnerID() - : byteBufferedOutputChannel.getID(); - this.queuedOutgoingEnvelopes = new SpillingQueue(fileOwnerID, this.outputGateContext); - - // Register as inactive channel so queue can be spilled to disk when we run out of memory buffers - if (!isReceiverRunning) { - this.outputGateContext.registerInactiveOutputChannel(this); - } - } - - /** - * {@inheritDoc} - */ - @Override - public BufferPairResponse requestEmptyWriteBuffers() throws InterruptedException, IOException { - - if (this.outgoingTransferEnvelope == null) { - this.outgoingTransferEnvelope = createNewOutgoingTransferEnvelope(); - } else { - if (this.outgoingTransferEnvelope.getBuffer() != null) { - LOG.error("Channel " + this.byteBufferedOutputChannel.getID() - + "'s transfer envelope already has a buffer attached"); - return null; - } - } - - final int uncompressedBufferSize = calculateBufferSize(); - - // TODO: This implementation breaks compression, we have to fix it later - final Buffer buffer = this.outputGateContext.requestEmptyBufferBlocking(uncompressedBufferSize); - final BufferPairResponse bufferResponse = new BufferPairResponse(null, buffer); - - // Put the buffer into the transfer envelope - this.outgoingTransferEnvelope.setBuffer(bufferResponse.getUncompressedDataBuffer()); - - return bufferResponse; - } - - /** - * Creates a new {@link TransferEnvelope} object. The method assigns - * and increases the sequence number. Moreover, it will look up the list of receivers for this transfer envelope. - * This method will block until the lookup is completed. - * - * @return a new {@link TransferEnvelope} object containing the correct sequence number and receiver list - */ - private TransferEnvelope createNewOutgoingTransferEnvelope() { - - final TransferEnvelope transferEnvelope = new TransferEnvelope(this.sequenceNumber++, - this.byteBufferedOutputChannel.getJobID(), - this.byteBufferedOutputChannel.getID()); - - return transferEnvelope; - } - - /** - * {@inheritDoc} - */ - @Override - public void releaseWriteBuffers() throws IOException, InterruptedException { - - processIncomingEvents(); - - if (this.outgoingTransferEnvelope == null) { - LOG.error("Cannot find transfer envelope for channel with ID " + this.byteBufferedOutputChannel.getID()); - return; - } - - // Consistency check - if (this.outgoingTransferEnvelope.getBuffer() == null) { - LOG.error("Channel " + this.byteBufferedOutputChannel.getID() + " has no buffer attached"); - return; - } - - // Finish the write phase of the buffer - final Buffer buffer = this.outgoingTransferEnvelope.getBuffer(); - buffer.finishWritePhase(); - - // TODO: Add to checkpoint - - if (!this.isReceiverRunning) { - this.queuedOutgoingEnvelopes.add(this.outgoingTransferEnvelope); - } else { - - if (this.queuedOutgoingEnvelopes.isEmpty()) { - this.outputGateContext.processEnvelope(this, this.outgoingTransferEnvelope); - } else { - this.queuedOutgoingEnvelopes.add(this.outgoingTransferEnvelope); - flushQueuedOutgoingEnvelopes(); - } - } - - this.outgoingTransferEnvelope = null; - } - - /** - * {@inheritDoc} - */ - @Override - public void transferEventToInputChannel(final AbstractEvent event) throws IOException, InterruptedException { - - if (this.outgoingTransferEnvelope != null) { - this.outgoingTransferEnvelope.addEvent(event); - } else { - - final TransferEnvelope ephemeralTransferEnvelope = createNewOutgoingTransferEnvelope(); - ephemeralTransferEnvelope.addEvent(event); - - // TODO: Add to checkpoint - - if (!this.isReceiverRunning) { - this.queuedOutgoingEnvelopes.add(ephemeralTransferEnvelope); - } else { - - if (this.queuedOutgoingEnvelopes.isEmpty()) { - this.outputGateContext.processEnvelope(this, ephemeralTransferEnvelope); - } else { - this.queuedOutgoingEnvelopes.add(ephemeralTransferEnvelope); - flushQueuedOutgoingEnvelopes(); - } - } - } - } - - /** - * Calculates the recommended size of the next buffer to be - * handed to the attached channel object in bytes. - * - * @return the recommended size of the next buffer in bytes - */ - private int calculateBufferSize() { - - // TODO: Include latency considerations - return this.outputGateContext.getMaximumBufferSize(); - } - - /** - * {@inheritDoc} - */ - @Override - public boolean isInputChannel() { - - return false; - } - - /** - * Checks if this channel is active, that means the receiver of the channel's data is able to able to accept the - * data. - * - * @return true if the channel is active, false otherwise - */ - boolean isChannelActive() { - - return this.isReceiverRunning; - } - - /** - * Called by the framework to report events to - * the attached channel object. - * - * @param abstractEvent - * the event to be reported - */ - void processEvent(final AbstractEvent abstractEvent) { - - this.incomingEventQueue.offer(abstractEvent); - } - - /** - * {@inheritDoc} - */ - @Override - public ChannelID getChannelID() { - - return this.byteBufferedOutputChannel.getID(); - } - - /** - * {@inheritDoc} - */ - @Override - public ChannelID getConnectedChannelID() { - - return this.byteBufferedOutputChannel.getConnectedChannelID(); - } - - /** - * {@inheritDoc} - */ - @Override - public JobID getJobID() { - - return this.byteBufferedOutputChannel.getJobID(); - } - - /** - * {@inheritDoc} - */ - @Override - public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { - - if (transferEnvelope.getBuffer() != null) { - LOG.error("Transfer envelope for output channel has buffer attached"); - } - - final Iterator it = transferEnvelope.getEventList().iterator(); - while (it.hasNext()) { - processEvent(it.next()); - } - } - - void flushQueuedOutgoingEnvelopes() throws IOException, InterruptedException { - - if (this.spillingQueueAlreadyFlushed) { - return; - } - - if (!this.queuedOutgoingEnvelopes.isEmpty()) { - - // TODO: Make this mechanisms smarter - this.queuedOutgoingEnvelopes.spillSynchronouslyIncludingHead(); - this.queuedOutgoingEnvelopes.printSpillingState(); - - if (!this.outputGateContext.registerSpillingQueueWithNetworkConnection( - this.byteBufferedOutputChannel.getID(), this.queuedOutgoingEnvelopes)) { - - // Direct connection, spill the queue but make sure we do not copy data back to main memory - this.queuedOutgoingEnvelopes.disableAsynchronousUnspilling(); - - while (!this.queuedOutgoingEnvelopes.isEmpty()) { - this.outputGateContext.processEnvelope(this, this.queuedOutgoingEnvelopes.poll()); - } - } - } - - this.spillingQueueAlreadyFlushed = true; - } - - /** - * {@inheritDoc} - */ - @Override - public boolean hasDataLeftToTransmit() throws IOException, InterruptedException { - - processIncomingEvents(); - - if (!this.isReceiverRunning) { - return true; - } - - flushQueuedOutgoingEnvelopes(); - - return (!this.closeAcknowledgementReceived); - } - - long getAmountOfMainMemoryInQueue() { - - return this.queuedOutgoingEnvelopes.getAmountOfMainMemoryInQueue(); - } - - /** - * Triggers the encapsulated output channel to flush and release its internal working buffers. - * - * @throws IOException - * thrown if an I/O error occurs while flushing the buffers - * @throws InterruptedException - * thrown if the thread is interrupted while waiting for the channel to flush - */ - void flush() throws IOException, InterruptedException { - - this.byteBufferedOutputChannel.flush(); - } - - long spillQueueWithOutgoingEnvelopes() throws IOException { - - return this.queuedOutgoingEnvelopes.spillSynchronouslyIncludingHead(); - } - - /** - * Processes all queues incoming events. - */ - private void processIncomingEvents() { - - AbstractEvent event = this.incomingEventQueue.poll(); - while (event != null) { - - if (event instanceof ByteBufferedChannelCloseEvent) { - this.closeAcknowledgementReceived = true; - } else if (event instanceof ByteBufferedChannelActivateEvent) { - this.isReceiverRunning = true; - } else { - this.byteBufferedOutputChannel.processEvent(event); - } - - event = this.incomingEventQueue.poll(); - } - } +public interface OutputChannelContext extends ChannelContext { } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java index 374e382a8be75..d50e1eb48c98e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java @@ -15,214 +15,10 @@ package eu.stratosphere.nephele.taskmanager.bytebuffered; -import java.io.IOException; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Set; - -import eu.stratosphere.nephele.event.task.AbstractEvent; -import eu.stratosphere.nephele.event.task.EventList; -import eu.stratosphere.nephele.io.AbstractID; -import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; -import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; -import eu.stratosphere.nephele.taskmanager.bufferprovider.AsynchronousEventListener; -import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; -import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; - -final class OutputGateContext implements BufferProvider, AsynchronousEventListener { - - private final TaskContext taskContext; - - private final ChannelType channelType; - - private final Set inactiveOutputChannels; - - OutputGateContext(final TaskContext taskContext, final ChannelType channelType, final int outputGateIndex) { - - this.taskContext = taskContext; - this.channelType = channelType; - - this.inactiveOutputChannels = new HashSet(); - - this.taskContext.registerAsynchronousEventListener(outputGateIndex, this); - } - - void registerInactiveOutputChannel(final OutputChannelContext outputChannelContext) { - - this.inactiveOutputChannels.add(outputChannelContext); - } - - AbstractID getFileOwnerID() { - - return this.taskContext.getFileOwnerID(); - } - - private long spillQueueWithLargestAmountOfMainMemory() throws IOException { - - if (this.inactiveOutputChannels.isEmpty()) { - return 0L; - } - - final Iterator it = this.inactiveOutputChannels.iterator(); - - long maxMainMemory = 0; - OutputChannelContext maxContext = null; - - while (it.hasNext()) { - - final OutputChannelContext context = it.next(); - final long mm = context.getAmountOfMainMemoryInQueue(); - - if (mm > maxMainMemory) { - maxMainMemory = mm; - maxContext = context; - } - } - - if (maxContext != null) { - return maxContext.spillQueueWithOutgoingEnvelopes(); - } - - return 0L; - } - - private void checkForActiveOutputChannels() throws IOException, InterruptedException { - - final Iterator it = this.inactiveOutputChannels.iterator(); - while (it.hasNext()) { - final OutputChannelContext channelContext = it.next(); - if (channelContext.isChannelActive()) { - channelContext.flushQueuedOutgoingEnvelopes(); - it.remove(); - } - } - } - - /** - * {@inheritDoc} - */ - @Override - public int getMaximumBufferSize() { - - return this.taskContext.getMaximumBufferSize(); - } - - /** - * Called by the attached output channel wrapper to forward a {@link TransferEnvelope} object - * to its final destination. Within this method the provided transfer envelope is possibly also - * forwarded to the assigned ephemeral checkpoint. - * - * @param caller - * the output channel context calling this method - * @param outgoingTransferEnvelope - * the transfer envelope to be forwarded - * @throws IOException - * thrown if an I/O error occurs while processing the envelope - * @throws InterruptedException - * thrown if the thread is interrupted while waiting for the envelope to be processed - */ - void processEnvelope(final OutputChannelContext caller, final TransferEnvelope outgoingTransferEnvelope) - throws IOException, InterruptedException { - - this.taskContext.processEnvelope(outgoingTransferEnvelope); - - if (this.channelType == ChannelType.FILE) { - // Check if the event list of the envelope contains a close event and acknowledge it - final EventList eventList = outgoingTransferEnvelope.getEventList(); - if (eventList != null) { - final Iterator it = eventList.iterator(); - while (it.hasNext()) { - final AbstractEvent event = it.next(); - if (event instanceof ByteBufferedChannelCloseEvent) { - caller.processEvent(event); - } - } - } - } - } - - /** - * {@inheritDoc} - */ - @Override - public void asynchronousEventOccurred() throws IOException, InterruptedException { - - checkForActiveOutputChannels(); - } - - /** - * {@inheritDoc} - */ - @Override - public Buffer requestEmptyBuffer(int minimumSizeOfBuffer) throws IOException { - - return this.taskContext.requestEmptyBuffer(minimumSizeOfBuffer); - } - - /** - * {@inheritDoc} - */ - @Override - public Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException, InterruptedException { - - Buffer buffer = this.taskContext.requestEmptyBuffer(minimumSizeOfBuffer); - - // No memory-based buffer available - if (buffer == null) { - - // Report exhaustion of memory buffers to the task context - this.taskContext.reportExhaustionOfMemoryBuffers(); - - // Spill queue that contains the largest amount of main memory, encapsulated in the queued buffers, to disk - spillQueueWithLargestAmountOfMainMemory(); - - // Wait until a memory-based buffer is available - buffer = this.taskContext.requestEmptyBufferBlocking(minimumSizeOfBuffer); - } - - return buffer; - } - - /** - * {@inheritDoc} - */ - @Override - public boolean isShared() { - - return this.taskContext.isShared(); - } - - /** - * {@inheritDoc} - */ - @Override - public void reportAsynchronousEvent() { - - this.taskContext.reportAsynchronousEvent(); - } - /** - * Registers the given spilling queue with a network connection. The network connection is in charge of polling the - * remaining elements from the queue. - * - * @param sourceChannelID - * the ID of the source channel which is associated with the spilling queue - * @param spillingQueue - * the spilling queue to be registered - * @return true if the has been successfully registered with the network connection, false - * if the receiver runs within the same task manager and there is no network operation required to transfer - * the queued data - * @throws IOException - * thrown if an I/O error occurs while looking up the destination of the queued envelopes - * @throws InterruptedException - * thrown if the thread is interrupted while looking up the destination of the queued envelopes - */ - boolean registerSpillingQueueWithNetworkConnection(final ChannelID sourceChannelID, - final SpillingQueue spillingQueue) throws IOException, InterruptedException { +public interface OutputGateContext extends GateContext { - return this.taskContext.registerSpillingQueueWithNetworkConnection(sourceChannelID, spillingQueue); - } -} + OutputChannelContext createOutputChannelContext(ChannelID channelID, OutputChannelContext previousContext, + boolean isReceiverRunning, boolean mergeSpillBuffers); +} \ No newline at end of file diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java index 15c466753686c..d679e7d3eecab 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java @@ -15,281 +15,14 @@ package eu.stratosphere.nephele.taskmanager.bytebuffered; -import java.io.IOException; -import java.util.Map; - -import eu.stratosphere.nephele.checkpointing.EphemeralCheckpoint; -import eu.stratosphere.nephele.execution.RuntimeEnvironment; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.AbstractID; -import eu.stratosphere.nephele.io.OutputGate; -import eu.stratosphere.nephele.io.channels.Buffer; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; -import eu.stratosphere.nephele.taskmanager.RuntimeTask; -import eu.stratosphere.nephele.taskmanager.bufferprovider.AsynchronousEventListener; -import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; -import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; +import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; -import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; -import eu.stratosphere.nephele.types.Record; - -final class TaskContext implements BufferProvider, LocalBufferPoolOwner, AsynchronousEventListener { - - private final LocalBufferPool localBufferPool; - - private final RuntimeTask task; - - private final AsynchronousEventListener[] subEventListener; - - private final int numberOfOutputChannels; - - final TransferEnvelopeDispatcher transferEnvelopeDispatcher; - - private final EphemeralCheckpoint ephemeralCheckpoint; - - private final boolean forwardTransferEnvelopes; - - /** - * Stores whether the initial exhaustion of memory buffers has already been reported - */ - private boolean initialExhaustionOfMemoryBuffersReported = false; - - TaskContext(final RuntimeTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, - final Map tasksWithUndecidedCheckpoints) { - - this.localBufferPool = new LocalBufferPool(1, false, this); - this.task = task; - - final RuntimeEnvironment environment = task.getRuntimeEnvironment(); - - // Compute number of output input channels - int nooc = 0; - boolean ephemeral = true; - for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - final OutputGate outputGate = environment.getOutputGate(i); - nooc += outputGate.getNumberOfOutputChannels(); - if (outputGate.getChannelType() == ChannelType.FILE) { - ephemeral = false; - } - } - this.numberOfOutputChannels = nooc; - this.forwardTransferEnvelopes = ephemeral; - - this.ephemeralCheckpoint = new EphemeralCheckpoint(task, ephemeral); - if (ephemeral) { - tasksWithUndecidedCheckpoints.put(task.getVertexID(), this); - } - - this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; - - // Each output gate context will register as a sub event listener - this.subEventListener = new AsynchronousEventListener[environment.getNumberOfOutputGates()]; - } - - void registerAsynchronousEventListener(final int index, final AsynchronousEventListener eventListener) { - - if (index >= this.subEventListener.length || index < 0) { - throw new IllegalArgumentException("Argument index has invalid value " + index); - } - - if (eventListener == null) { - throw new IllegalArgumentException("Argument eventListener must not be null"); - } - - if (this.subEventListener[index] != null) { - throw new IllegalStateException("There is already an event listener with index " + index + " registered"); - } - - this.subEventListener[index] = eventListener; - } - - /** - * {@inheritDoc} - */ - @Override - public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException { - - return this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer); - } - - /** - * {@inheritDoc} - */ - @Override - public Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException, - InterruptedException { - - return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer); - } - - /** - * {@inheritDoc} - */ - @Override - public int getMaximumBufferSize() { - - return this.localBufferPool.getMaximumBufferSize(); - } - - /** - * {@inheritDoc} - */ - @Override - public void clearLocalBufferPool() { - - // Clear the buffer cache - this.localBufferPool.clear(); - } - - /** - * {@inheritDoc} - */ - @Override - public boolean isShared() { - - return false; - } - - /** - * {@inheritDoc} - */ - @Override - public void logBufferUtilization() { - - final int ava = this.localBufferPool.getNumberOfAvailableBuffers(); - final int req = this.localBufferPool.getRequestedNumberOfBuffers(); - final int des = this.localBufferPool.getDesignatedNumberOfBuffers(); - - final RuntimeEnvironment environment = this.task.getRuntimeEnvironment(); - - System.out.println("\t\t" + environment.getTaskName() + ": " + ava + " available, " + req + " requested, " - + des + " designated"); - } - - /** - * Called by an {@link OutputGateContext} to indicate that the task has temporarily run out of memory buffers. - */ - void reportExhaustionOfMemoryBuffers() { - - if (!this.initialExhaustionOfMemoryBuffersReported) { - - this.task.initialExecutionResourcesExhausted(); - this.initialExhaustionOfMemoryBuffersReported = true; - } - } - - /** - * {@inheritDoc} - */ - @Override - public void reportAsynchronousEvent() { - - this.localBufferPool.reportAsynchronousEvent(); - } - - /** - * {@inheritDoc} - */ - @Override - public void asynchronousEventOccurred() throws IOException, InterruptedException { - - // First, notify all the listeners about the asynchronous event - for (int i = 0; i < this.subEventListener.length; ++i) { - - if (this.subEventListener[i] == null) { - throw new IllegalStateException("Event listener at index " + i + " is null"); - } - - this.subEventListener[i].asynchronousEventOccurred(); - } - - // Second, check if the checkpoint decision changed - this.ephemeralCheckpoint.checkAsynchronousCheckpointDecision(); - } - - /** - * {@inheritDoc} - */ - @Override - public int getNumberOfChannels() { - - return this.numberOfOutputChannels; - } - - /** - * {@inheritDoc} - */ - @Override - public void setDesignatedNumberOfBuffers(int numberOfBuffers) { - - this.localBufferPool.setDesignatedNumberOfBuffers(numberOfBuffers); - } - - /** - * Called by the attached output gate context to forward a {@link TransferEnvelope} object - * to its final destination. Within this method the provided transfer envelope is possibly also - * forwarded to the assigned ephemeral checkpoint. - * - * @param outgoingTransferEnvelope - * the transfer envelope to be forwarded - * @throws IOException - * thrown if an I/O error occurs while processing the envelope - * @throws InterruptedException - * thrown if the thread is interrupted while waiting for the envelope to be processed - */ - void processEnvelope(final TransferEnvelope outgoingTransferEnvelope) throws IOException, InterruptedException { - - if (!this.ephemeralCheckpoint.isDiscarded()) { - final TransferEnvelope dup = outgoingTransferEnvelope.duplicate(); - this.ephemeralCheckpoint.addTransferEnvelope(dup); - } - - if (this.forwardTransferEnvelopes) { - // Immediately forward the envelope - this.transferEnvelopeDispatcher.processEnvelopeFromOutputChannel(outgoingTransferEnvelope); - } else { - // Simply discard the envelope - final Buffer buffer = outgoingTransferEnvelope.getBuffer(); - if (buffer != null) { - buffer.recycleBuffer(); - } - } - } - - AbstractID getFileOwnerID() { - - return this.task.getVertexID(); - } - void setCheckpointDecisionAsynchronously(final boolean checkpointDecision) { +public interface TaskContext { - // Simply delegate call - this.ephemeralCheckpoint.setCheckpointDecisionAsynchronously(checkpointDecision); - } + OutputGateContext createOutputGateContext(GateID gateID); - /** - * Registers the given spilling queue with a network connection. The network connection is in charge of polling the - * remaining elements from the queue. - * - * @param sourceChannelID - * the ID of the source channel which is associated with the spilling queue - * @param spillingQueue - * the spilling queue to be registered - * @return true if the has been successfully registered with the network connection, false - * if the receiver runs within the same task manager and there is no network operation required to transfer - * the queued data - * @throws IOException - * thrown if an I/O error occurs while looking up the destination of the queued envelopes - * @throws InterruptedException - * thrown if the thread is interrupted while looking up the destination of the queued envelopes - */ - boolean registerSpillingQueueWithNetworkConnection(final ChannelID sourceChannelID, - final SpillingQueue spillingQueue) throws IOException, InterruptedException { + InputGateContext createInputGateContext(GateID gateID); - return this.transferEnvelopeDispatcher.registerSpillingQueueWithNetworkConnection(this.task.getJobID(), - sourceChannelID, spillingQueue); - } + LocalBufferPoolOwner getLocalBufferPoolOwner(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java new file mode 100644 index 0000000000000..807c813bdc561 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java @@ -0,0 +1,337 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.runtime; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Iterator; +import java.util.Queue; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.event.task.EventList; +import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.BufferPairResponse; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedInputChannelBroker; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.taskmanager.bytebuffered.InputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; +import eu.stratosphere.nephele.util.StringUtils; + +final class RuntimeInputChannelContext implements InputChannelContext, ByteBufferedInputChannelBroker { + + private static final Log LOG = LogFactory.getLog(RuntimeInputChannelContext.class); + + private final RuntimeInputGateContext inputGateContext; + + private final AbstractByteBufferedInputChannel byteBufferedInputChannel; + + private final TransferEnvelopeDispatcher transferEnvelopeDispatcher; + + private final Queue queuedEnvelopes = new ArrayDeque(); + + private int lastReceivedEnvelope = -1; + + RuntimeInputChannelContext(final RuntimeInputGateContext inputGateContext, + final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + final AbstractByteBufferedInputChannel byteBufferedInputChannel) { + + this.inputGateContext = inputGateContext; + this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; + this.byteBufferedInputChannel = byteBufferedInputChannel; + this.byteBufferedInputChannel.setInputChannelBroker(this); + } + + @Override + public BufferPairResponse getReadBufferToConsume() { + + TransferEnvelope transferEnvelope = null; + + synchronized (this.queuedEnvelopes) { + + if (this.queuedEnvelopes.isEmpty()) { + return null; + } + + transferEnvelope = this.queuedEnvelopes.peek(); + + // If envelope does not have a buffer, remove it immediately + if (transferEnvelope.getBuffer() == null) { + this.queuedEnvelopes.poll(); + } + } + + // Make sure we have all necessary buffers before we go on + if (transferEnvelope.getBuffer() == null) { + + // No buffers necessary + final EventList eventList = transferEnvelope.getEventList(); + if (eventList != null) { + if (!eventList.isEmpty()) { + final Iterator it = eventList.iterator(); + while (it.hasNext()) { + this.byteBufferedInputChannel.processEvent(it.next()); + } + } + } + + return null; + } + + // TODO: Fix implementation breaks compression, fix it later on + final BufferPairResponse response = new BufferPairResponse(null, transferEnvelope.getBuffer()); // No need to + + // Moved event processing to releaseConsumedReadBuffer method // copy anything + + return response; + } + + @Override + public void releaseConsumedReadBuffer() { + + TransferEnvelope transferEnvelope = null; + synchronized (this.queuedEnvelopes) { + + if (this.queuedEnvelopes.isEmpty()) { + LOG.error("Inconsistency: releaseConsumedReadBuffer called on empty queue!"); + return; + } + + transferEnvelope = this.queuedEnvelopes.poll(); + } + + // Process events + final EventList eventList = transferEnvelope.getEventList(); + if (eventList != null) { + if (!eventList.isEmpty()) { + final Iterator it = eventList.iterator(); + while (it.hasNext()) { + this.byteBufferedInputChannel.processEvent(it.next()); + } + } + } + + final Buffer consumedBuffer = transferEnvelope.getBuffer(); + if (consumedBuffer == null) { + LOG.error("Inconsistency: consumed read buffer is null!"); + return; + } + + if (consumedBuffer.remaining() > 0) { + LOG.error("consumedReadBuffer has " + consumedBuffer.remaining() + " unconsumed bytes left!!"); + } + + // Recycle consumed read buffer + consumedBuffer.recycleBuffer(); + } + + /** + * {@inheritDoc} + */ + @Override + public void transferEventToOutputChannel(AbstractEvent event) throws IOException, InterruptedException { + + final TransferEnvelope ephemeralTransferEnvelope = new TransferEnvelope(0, getJobID(), getChannelID()); + + ephemeralTransferEnvelope.addEvent(event); + this.transferEnvelopeDispatcher.processEnvelopeFromInputChannel(ephemeralTransferEnvelope); + } + + /** + * {@inheritDoc} + */ + @Override + public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { + + // The sequence number of the envelope to be queued + final int sequenceNumber = transferEnvelope.getSequenceNumber(); + + synchronized (this.queuedEnvelopes) { + + final int expectedSequenceNumber = this.lastReceivedEnvelope + 1; + if (sequenceNumber != expectedSequenceNumber) { + + if (sequenceNumber > expectedSequenceNumber) { + + // This is a problem, now we are actually missing some data + this.byteBufferedInputChannel.reportIOException(new IOException("Expected data packet " + + expectedSequenceNumber + " but received " + sequenceNumber)); + this.byteBufferedInputChannel.checkForNetworkEvents(); + } else { + + // Tell the sender that we are expecting an envelope with a higher sequence number + try { + transferEventToOutputChannel(new UnexpectedEnvelopeEvent(expectedSequenceNumber)); + } catch (Exception e) { + LOG.error(StringUtils.stringifyException(e)); + } + } + + LOG.warn("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber + + " but received " + sequenceNumber); + + final Buffer buffer = transferEnvelope.getBuffer(); + if (buffer != null) { + buffer.recycleBuffer(); + } + + return; + } + + this.queuedEnvelopes.add(transferEnvelope); + + this.lastReceivedEnvelope = sequenceNumber; + } + + // Notify the channel about the new data + this.byteBufferedInputChannel.checkForNetworkEvents(); + } + + @Override + public ChannelID getChannelID() { + + return this.byteBufferedInputChannel.getID(); + } + + @Override + public ChannelID getConnectedChannelID() { + + return this.byteBufferedInputChannel.getConnectedChannelID(); + } + + @Override + public JobID getJobID() { + + return this.byteBufferedInputChannel.getJobID(); + } + + @Override + public boolean isInputChannel() { + + return this.byteBufferedInputChannel.isInputChannel(); + } + + public void releaseAllResources() { + + final Queue buffersToRecycle = new ArrayDeque(); + + synchronized (this.queuedEnvelopes) { + + while (!this.queuedEnvelopes.isEmpty()) { + final TransferEnvelope envelope = this.queuedEnvelopes.poll(); + if (envelope.getBuffer() != null) { + buffersToRecycle.add(envelope.getBuffer()); + } + } + } + + while (!buffersToRecycle.isEmpty()) { + buffersToRecycle.poll().recycleBuffer(); + } + } + + @Override + public int getNumberOfQueuedEnvelopes() { + + synchronized (this.queuedEnvelopes) { + + return this.queuedEnvelopes.size(); + } + } + + @Override + public int getNumberOfQueuedMemoryBuffers() { + + synchronized (this.queuedEnvelopes) { + + int count = 0; + + final Iterator it = this.queuedEnvelopes.iterator(); + while (it.hasNext()) { + + final TransferEnvelope envelope = it.next(); + if (envelope.getBuffer() != null) { + if (envelope.getBuffer().isBackedByMemory()) { + ++count; + } + } + } + + return count; + } + } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException { + + throw new IllegalStateException("requestEmptyBuffer called on InputChannelContext"); + } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException { + + return this.inputGateContext.requestEmptyBufferBlocking(minimumSizeOfBuffer); + } + + /** + * {@inheritDoc} + */ + @Override + public int getMaximumBufferSize() { + + return this.inputGateContext.getMaximumBufferSize(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isShared() { + + return this.inputGateContext.isShared(); + } + + /** + * {@inheritDoc} + */ + @Override + public void reportAsynchronousEvent() { + + this.inputGateContext.reportAsynchronousEvent(); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelType getType() { + + return this.byteBufferedInputChannel.getType(); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java new file mode 100644 index 0000000000000..abd95848819b0 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -0,0 +1,180 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.runtime; + +import java.io.IOException; + +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.channels.AbstractInputChannel; +import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; +import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; +import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; +import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; +import eu.stratosphere.nephele.taskmanager.bytebuffered.InputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; +import eu.stratosphere.nephele.types.Record; + +final class RuntimeInputGateContext implements BufferProvider, InputGateContext, LocalBufferPoolOwner { + + private final LocalBufferPool localBufferPool; + + private final TransferEnvelopeDispatcher transferEnvelopeDispatcher; + + private final InputGate inputGate; + + RuntimeInputGateContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + final InputGate inputGate) { + + this.localBufferPool = new LocalBufferPool(1, false); + + this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; + this.inputGate = inputGate; + } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException { + + return this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer); + } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException { + + return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer); + } + + /** + * {@inheritDoc} + */ + @Override + public int getMaximumBufferSize() { + + return this.localBufferPool.getMaximumBufferSize(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isShared() { + + return this.localBufferPool.isShared(); + } + + /** + * {@inheritDoc} + */ + @Override + public void reportAsynchronousEvent() { + + this.localBufferPool.reportAsynchronousEvent(); + } + + @Override + public int getNumberOfChannels() { + + return this.inputGate.getNumberOfInputChannels(); + } + + /** + * {@inheritDoc} + */ + @Override + public void setDesignatedNumberOfBuffers(int numberOfBuffers) { + + this.localBufferPool.setDesignatedNumberOfBuffers(numberOfBuffers); + } + + /** + * {@inheritDoc} + */ + @Override + public void clearLocalBufferPool() { + + this.localBufferPool.clear(); + } + + /** + * {@inheritDoc} + */ + @Override + public void logBufferUtilization() { + + final int ava = this.localBufferPool.getNumberOfAvailableBuffers(); + final int req = this.localBufferPool.getRequestedNumberOfBuffers(); + final int des = this.localBufferPool.getDesignatedNumberOfBuffers(); + + System.out + .println("\t\tInputGateContext: " + ava + " available, " + req + " requested, " + des + " designated"); + } + + /** + * {@inheritDoc} + */ + @Override + public GateID getGateID() { + + return this.inputGate.getGateID(); + } + + /** + * {@inheritDoc} + */ + @Override + public InputChannelContext createInputChannelContext(final ChannelID channelID, + final InputChannelContext previousContext) { + + AbstractInputChannel channel = null; + for (int i = 0; i < this.inputGate.getNumberOfInputChannels(); ++i) { + AbstractInputChannel candidateChannel = this.inputGate.getInputChannel(i); + if (candidateChannel.getID().equals(channelID)) { + channel = candidateChannel; + break; + } + } + + if (channel == null) { + throw new IllegalArgumentException("Cannot find input channel with ID " + channelID); + } + + if (!(channel instanceof AbstractByteBufferedInputChannel)) { + throw new IllegalStateException("Channel with ID" + channelID + + " is not of type AbstractByteBufferedInputChannel"); + } + + return new RuntimeInputChannelContext(this, this.transferEnvelopeDispatcher, + (AbstractByteBufferedInputChannel) channel); + } + + /** + * {@inheritDoc} + */ + @Override + public LocalBufferPoolOwner getLocalBufferPoolOwner() { + + return this; + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java new file mode 100644 index 0000000000000..4f2e91964daf1 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java @@ -0,0 +1,408 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.runtime; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Queue; +import java.util.concurrent.LinkedBlockingDeque; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.io.AbstractID; +import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.BufferPairResponse; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelActivateEvent; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedOutputChannelBroker; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; +import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +public final class RuntimeOutputChannelContext implements ByteBufferedOutputChannelBroker, OutputChannelContext { + + /** + * The static object used for logging. + */ + private static final Log LOG = LogFactory.getLog(RuntimeOutputChannelContext.class); + + /** + * The byte buffered output channel this context belongs to. + */ + private final AbstractByteBufferedOutputChannel byteBufferedOutputChannel; + + /** + * The output gate context associated with this context. + */ + private final RuntimeOutputGateContext outputGateContext; + + /** + * Points to the {@link TransferEnvelope} object that will be passed to the framework upon + * the next releaseWriteBuffers call. + */ + private TransferEnvelope outgoingTransferEnvelope = null; + + /** + * Indicates whether the receiver of an envelope is currently running. + */ + private boolean isReceiverRunning = false; + + /** + * Stores whether the receiver has acknowledged the close request from this channel. + */ + private boolean closeAcknowledgementReceived = false; + + /** + * Queue to store outgoing transfer envelope in case the receiver of the envelopes is not yet running. + */ + private final SpillingQueue queuedOutgoingEnvelopes; + + /** + * Stores incoming events for this output channel. + */ + private final Queue incomingEventQueue = new LinkedBlockingDeque(); + + /** + * The sequence number for the next {@link TransferEnvelope} to be created. + */ + private int sequenceNumber = 0; + + /** + * Stores if the flushing the of spilling queue has already been triggered. + */ + private boolean spillingQueueAlreadyFlushed = false; + + RuntimeOutputChannelContext(final RuntimeOutputGateContext outputGateContext, + final AbstractByteBufferedOutputChannel byteBufferedOutputChannel, final boolean isReceiverRunning, + final boolean mergeSpilledBuffers) { + + this.outputGateContext = outputGateContext; + this.byteBufferedOutputChannel = byteBufferedOutputChannel; + this.byteBufferedOutputChannel.setByteBufferedOutputChannelBroker(this); + this.isReceiverRunning = isReceiverRunning; + + final AbstractID fileOwnerID = mergeSpilledBuffers ? outputGateContext.getFileOwnerID() + : byteBufferedOutputChannel.getID(); + this.queuedOutgoingEnvelopes = new SpillingQueue(fileOwnerID, this.outputGateContext); + + // Register as inactive channel so queue can be spilled to disk when we run out of memory buffers + if (!isReceiverRunning) { + this.outputGateContext.registerInactiveOutputChannel(this); + } + } + + /** + * {@inheritDoc} + */ + @Override + public BufferPairResponse requestEmptyWriteBuffers() throws InterruptedException, IOException { + + if (this.outgoingTransferEnvelope == null) { + this.outgoingTransferEnvelope = createNewOutgoingTransferEnvelope(); + } else { + if (this.outgoingTransferEnvelope.getBuffer() != null) { + LOG.error("Channel " + this.byteBufferedOutputChannel.getID() + + "'s transfer envelope already has a buffer attached"); + return null; + } + } + + final int uncompressedBufferSize = calculateBufferSize(); + + // TODO: This implementation breaks compression, we have to fix it later + final Buffer buffer = this.outputGateContext.requestEmptyBufferBlocking(uncompressedBufferSize); + final BufferPairResponse bufferResponse = new BufferPairResponse(null, buffer); + + // Put the buffer into the transfer envelope + this.outgoingTransferEnvelope.setBuffer(bufferResponse.getUncompressedDataBuffer()); + + return bufferResponse; + } + + /** + * Creates a new {@link TransferEnvelope} object. The method assigns + * and increases the sequence number. Moreover, it will look up the list of receivers for this transfer envelope. + * This method will block until the lookup is completed. + * + * @return a new {@link TransferEnvelope} object containing the correct sequence number and receiver list + */ + private TransferEnvelope createNewOutgoingTransferEnvelope() { + + final TransferEnvelope transferEnvelope = new TransferEnvelope(this.sequenceNumber++, + this.byteBufferedOutputChannel.getJobID(), + this.byteBufferedOutputChannel.getID()); + + return transferEnvelope; + } + + /** + * {@inheritDoc} + */ + @Override + public void releaseWriteBuffers() throws IOException, InterruptedException { + + processIncomingEvents(); + + if (this.outgoingTransferEnvelope == null) { + LOG.error("Cannot find transfer envelope for channel with ID " + this.byteBufferedOutputChannel.getID()); + return; + } + + // Consistency check + if (this.outgoingTransferEnvelope.getBuffer() == null) { + LOG.error("Channel " + this.byteBufferedOutputChannel.getID() + " has no buffer attached"); + return; + } + + // Finish the write phase of the buffer + final Buffer buffer = this.outgoingTransferEnvelope.getBuffer(); + buffer.finishWritePhase(); + + // TODO: Add to checkpoint + + if (!this.isReceiverRunning) { + this.queuedOutgoingEnvelopes.add(this.outgoingTransferEnvelope); + } else { + + if (this.queuedOutgoingEnvelopes.isEmpty()) { + this.outputGateContext.processEnvelope(this, this.outgoingTransferEnvelope); + } else { + this.queuedOutgoingEnvelopes.add(this.outgoingTransferEnvelope); + flushQueuedOutgoingEnvelopes(); + } + } + + this.outgoingTransferEnvelope = null; + } + + /** + * {@inheritDoc} + */ + @Override + public void transferEventToInputChannel(final AbstractEvent event) throws IOException, InterruptedException { + + if (this.outgoingTransferEnvelope != null) { + this.outgoingTransferEnvelope.addEvent(event); + } else { + + final TransferEnvelope ephemeralTransferEnvelope = createNewOutgoingTransferEnvelope(); + ephemeralTransferEnvelope.addEvent(event); + + // TODO: Add to checkpoint + + if (!this.isReceiverRunning) { + this.queuedOutgoingEnvelopes.add(ephemeralTransferEnvelope); + } else { + + if (this.queuedOutgoingEnvelopes.isEmpty()) { + this.outputGateContext.processEnvelope(this, ephemeralTransferEnvelope); + } else { + this.queuedOutgoingEnvelopes.add(ephemeralTransferEnvelope); + flushQueuedOutgoingEnvelopes(); + } + } + } + } + + /** + * Calculates the recommended size of the next buffer to be + * handed to the attached channel object in bytes. + * + * @return the recommended size of the next buffer in bytes + */ + private int calculateBufferSize() { + + // TODO: Include latency considerations + return this.outputGateContext.getMaximumBufferSize(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isInputChannel() { + + return false; + } + + /** + * Checks if this channel is active, that means the receiver of the channel's data is able to able to accept the + * data. + * + * @return true if the channel is active, false otherwise + */ + boolean isChannelActive() { + + return this.isReceiverRunning; + } + + /** + * Called by the framework to report events to + * the attached channel object. + * + * @param abstractEvent + * the event to be reported + */ + void processEvent(final AbstractEvent abstractEvent) { + + this.incomingEventQueue.offer(abstractEvent); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelID getChannelID() { + + return this.byteBufferedOutputChannel.getID(); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelID getConnectedChannelID() { + + return this.byteBufferedOutputChannel.getConnectedChannelID(); + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + + return this.byteBufferedOutputChannel.getJobID(); + } + + /** + * {@inheritDoc} + */ + @Override + public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { + + if (transferEnvelope.getBuffer() != null) { + LOG.error("Transfer envelope for output channel has buffer attached"); + } + + final Iterator it = transferEnvelope.getEventList().iterator(); + while (it.hasNext()) { + processEvent(it.next()); + } + } + + void flushQueuedOutgoingEnvelopes() throws IOException, InterruptedException { + + if (this.spillingQueueAlreadyFlushed) { + return; + } + + if (!this.queuedOutgoingEnvelopes.isEmpty()) { + + // TODO: Make this mechanisms smarter + this.queuedOutgoingEnvelopes.spillSynchronouslyIncludingHead(); + this.queuedOutgoingEnvelopes.printSpillingState(); + + if (!this.outputGateContext.registerSpillingQueueWithNetworkConnection( + this.byteBufferedOutputChannel.getID(), this.queuedOutgoingEnvelopes)) { + + // Direct connection, spill the queue but make sure we do not copy data back to main memory + this.queuedOutgoingEnvelopes.disableAsynchronousUnspilling(); + + while (!this.queuedOutgoingEnvelopes.isEmpty()) { + this.outputGateContext.processEnvelope(this, this.queuedOutgoingEnvelopes.poll()); + } + } + } + + this.spillingQueueAlreadyFlushed = true; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasDataLeftToTransmit() throws IOException, InterruptedException { + + processIncomingEvents(); + + if (!this.isReceiverRunning) { + return true; + } + + flushQueuedOutgoingEnvelopes(); + + return (!this.closeAcknowledgementReceived); + } + + long getAmountOfMainMemoryInQueue() { + + return this.queuedOutgoingEnvelopes.getAmountOfMainMemoryInQueue(); + } + + /** + * Triggers the encapsulated output channel to flush and release its internal working buffers. + * + * @throws IOException + * thrown if an I/O error occurs while flushing the buffers + * @throws InterruptedException + * thrown if the thread is interrupted while waiting for the channel to flush + */ + void flush() throws IOException, InterruptedException { + + this.byteBufferedOutputChannel.flush(); + } + + long spillQueueWithOutgoingEnvelopes() throws IOException { + + return this.queuedOutgoingEnvelopes.spillSynchronouslyIncludingHead(); + } + + /** + * Processes all queues incoming events. + */ + private void processIncomingEvents() { + + AbstractEvent event = this.incomingEventQueue.poll(); + while (event != null) { + + if (event instanceof ByteBufferedChannelCloseEvent) { + this.closeAcknowledgementReceived = true; + } else if (event instanceof ByteBufferedChannelActivateEvent) { + this.isReceiverRunning = true; + } else { + this.byteBufferedOutputChannel.processEvent(event); + } + + event = this.incomingEventQueue.poll(); + } + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelType getType() { + + return this.byteBufferedOutputChannel.getType(); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java new file mode 100644 index 0000000000000..714e19bfcd1b5 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java @@ -0,0 +1,273 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.runtime; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.event.task.EventList; +import eu.stratosphere.nephele.io.AbstractID; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; +import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; +import eu.stratosphere.nephele.taskmanager.bufferprovider.AsynchronousEventListener; +import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; +import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; +import eu.stratosphere.nephele.types.Record; + +final class RuntimeOutputGateContext implements AsynchronousEventListener, BufferProvider, OutputGateContext { + + private final RuntimeTaskContext taskContext; + + private final OutputGate outputGate; + + private final Set inactiveOutputChannels; + + RuntimeOutputGateContext(final RuntimeTaskContext taskContext, final OutputGate outputGate) { + + this.taskContext = taskContext; + this.outputGate = outputGate; + + this.inactiveOutputChannels = new HashSet(); + + this.taskContext.registerAsynchronousEventListener(this.outputGate.getIndex(), this); + } + + void registerInactiveOutputChannel(final RuntimeOutputChannelContext outputChannelContext) { + + this.inactiveOutputChannels.add(outputChannelContext); + } + + AbstractID getFileOwnerID() { + + return this.taskContext.getFileOwnerID(); + } + + private long spillQueueWithLargestAmountOfMainMemory() throws IOException { + + if (this.inactiveOutputChannels.isEmpty()) { + return 0L; + } + + final Iterator it = this.inactiveOutputChannels.iterator(); + + long maxMainMemory = 0; + RuntimeOutputChannelContext maxContext = null; + + while (it.hasNext()) { + + final RuntimeOutputChannelContext context = it.next(); + final long mm = context.getAmountOfMainMemoryInQueue(); + + if (mm > maxMainMemory) { + maxMainMemory = mm; + maxContext = context; + } + } + + if (maxContext != null) { + return maxContext.spillQueueWithOutgoingEnvelopes(); + } + + return 0L; + } + + private void checkForActiveOutputChannels() throws IOException, InterruptedException { + + final Iterator it = this.inactiveOutputChannels.iterator(); + while (it.hasNext()) { + final RuntimeOutputChannelContext channelContext = it.next(); + if (channelContext.isChannelActive()) { + channelContext.flushQueuedOutgoingEnvelopes(); + it.remove(); + } + } + } + + /** + * {@inheritDoc} + */ + @Override + public int getMaximumBufferSize() { + + return this.taskContext.getMaximumBufferSize(); + } + + /** + * Called by the attached output channel wrapper to forward a {@link TransferEnvelope} object + * to its final destination. Within this method the provided transfer envelope is possibly also + * forwarded to the assigned ephemeral checkpoint. + * + * @param caller + * the output channel context calling this method + * @param outgoingTransferEnvelope + * the transfer envelope to be forwarded + * @throws IOException + * thrown if an I/O error occurs while processing the envelope + * @throws InterruptedException + * thrown if the thread is interrupted while waiting for the envelope to be processed + */ + void processEnvelope(final RuntimeOutputChannelContext caller, final TransferEnvelope outgoingTransferEnvelope) + throws IOException, InterruptedException { + + this.taskContext.processEnvelope(outgoingTransferEnvelope); + + if (this.outputGate.getChannelType() == ChannelType.FILE) { + // Check if the event list of the envelope contains a close event and acknowledge it + final EventList eventList = outgoingTransferEnvelope.getEventList(); + if (eventList != null) { + final Iterator it = eventList.iterator(); + while (it.hasNext()) { + final AbstractEvent event = it.next(); + if (event instanceof ByteBufferedChannelCloseEvent) { + caller.processEvent(event); + } + } + } + } + } + + /** + * {@inheritDoc} + */ + @Override + public void asynchronousEventOccurred() throws IOException, InterruptedException { + + checkForActiveOutputChannels(); + } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBuffer(int minimumSizeOfBuffer) throws IOException { + + return this.taskContext.requestEmptyBuffer(minimumSizeOfBuffer); + } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException, InterruptedException { + + Buffer buffer = this.taskContext.requestEmptyBuffer(minimumSizeOfBuffer); + + // No memory-based buffer available + if (buffer == null) { + + // Report exhaustion of memory buffers to the task context + this.taskContext.reportExhaustionOfMemoryBuffers(); + + // Spill queue that contains the largest amount of main memory, encapsulated in the queued buffers, to disk + spillQueueWithLargestAmountOfMainMemory(); + + // Wait until a memory-based buffer is available + buffer = this.taskContext.requestEmptyBufferBlocking(minimumSizeOfBuffer); + } + + return buffer; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isShared() { + + return this.taskContext.isShared(); + } + + /** + * {@inheritDoc} + */ + @Override + public void reportAsynchronousEvent() { + + this.taskContext.reportAsynchronousEvent(); + } + + /** + * Registers the given spilling queue with a network connection. The network connection is in charge of polling the + * remaining elements from the queue. + * + * @param sourceChannelID + * the ID of the source channel which is associated with the spilling queue + * @param spillingQueue + * the spilling queue to be registered + * @return true if the has been successfully registered with the network connection, false + * if the receiver runs within the same task manager and there is no network operation required to transfer + * the queued data + * @throws IOException + * thrown if an I/O error occurs while looking up the destination of the queued envelopes + * @throws InterruptedException + * thrown if the thread is interrupted while looking up the destination of the queued envelopes + */ + boolean registerSpillingQueueWithNetworkConnection(final ChannelID sourceChannelID, + final SpillingQueue spillingQueue) throws IOException, InterruptedException { + + return this.taskContext.registerSpillingQueueWithNetworkConnection(sourceChannelID, spillingQueue); + } + + /** + * {@inheritDoc} + */ + @Override + public GateID getGateID() { + + return this.outputGate.getGateID(); + } + + /** + * {@inheritDoc} + */ + @Override + public OutputChannelContext createOutputChannelContext(ChannelID channelID, OutputChannelContext previousContext, + boolean isReceiverRunning, boolean mergeSpillBuffers) { + + AbstractOutputChannel channel = null; + for (int i = 0; i < this.outputGate.getNumberOfOutputChannels(); ++i) { + AbstractOutputChannel candidateChannel = this.outputGate.getOutputChannel(i); + if (candidateChannel.getID().equals(channelID)) { + channel = candidateChannel; + break; + } + } + + if (channel == null) { + throw new IllegalArgumentException("Cannot find output channel with ID " + channelID); + } + + if (!(channel instanceof AbstractByteBufferedOutputChannel)) { + throw new IllegalStateException("Channel with ID" + channelID + + " is not of type AbstractByteBufferedOutputChannel"); + } + + return new RuntimeOutputChannelContext(this, (AbstractByteBufferedOutputChannel) channel, + isReceiverRunning, mergeSpillBuffers); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java similarity index 94% rename from nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/RuntimeTask.java rename to nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index 6098342f3a8b9..b75db8f37ac1f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -13,7 +13,7 @@ * **********************************************************************************************************************/ -package eu.stratosphere.nephele.taskmanager; +package eu.stratosphere.nephele.taskmanager.runtime; import java.lang.management.ManagementFactory; import java.lang.management.ThreadMXBean; @@ -48,6 +48,10 @@ import eu.stratosphere.nephele.profiling.TaskManagerProfiler; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; +import eu.stratosphere.nephele.taskmanager.Task; +import eu.stratosphere.nephele.taskmanager.TaskManager; +import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; import eu.stratosphere.nephele.template.AbstractInvokable; import eu.stratosphere.nephele.template.InputSplitProvider; import eu.stratosphere.nephele.types.Record; @@ -82,7 +86,7 @@ public final class RuntimeTask implements Task, ExecutionObserver { private long startTime; - RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, final TaskManager taskManager) { + public RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, final TaskManager taskManager) { this.vertexID = vertexID; this.environment = environment; @@ -473,4 +477,14 @@ public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) { taskManagerProfiler.unregisterExecutionListener(this.vertexID); } } + + /** + * {@inheritDoc} + */ + @Override + public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + final Map tasksWithUndecidedCheckpoints) { + + return new RuntimeTaskContext(this, transferEnvelopeDispatcher, tasksWithUndecidedCheckpoints); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java new file mode 100644 index 0000000000000..11606444abb01 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -0,0 +1,362 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.runtime; + +import java.io.IOException; +import java.util.Map; + +import eu.stratosphere.nephele.checkpointing.EphemeralCheckpoint; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.AbstractID; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.taskmanager.bufferprovider.AsynchronousEventListener; +import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; +import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; +import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; +import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; +import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; +import eu.stratosphere.nephele.types.Record; + +public final class RuntimeTaskContext implements BufferProvider, AsynchronousEventListener, LocalBufferPoolOwner, TaskContext { + + private final LocalBufferPool localBufferPool; + + private final RuntimeTask task; + + private final AsynchronousEventListener[] subEventListener; + + private final int numberOfOutputChannels; + + final TransferEnvelopeDispatcher transferEnvelopeDispatcher; + + private final EphemeralCheckpoint ephemeralCheckpoint; + + private final boolean forwardTransferEnvelopes; + + /** + * Stores whether the initial exhaustion of memory buffers has already been reported + */ + private boolean initialExhaustionOfMemoryBuffersReported = false; + + RuntimeTaskContext(final RuntimeTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + final Map tasksWithUndecidedCheckpoints) { + + this.localBufferPool = new LocalBufferPool(1, false, this); + this.task = task; + + final RuntimeEnvironment environment = task.getRuntimeEnvironment(); + + // Compute number of output input channels + int nooc = 0; + boolean ephemeral = true; + for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { + final OutputGate outputGate = environment.getOutputGate(i); + nooc += outputGate.getNumberOfOutputChannels(); + if (outputGate.getChannelType() == ChannelType.FILE) { + ephemeral = false; + } + } + this.numberOfOutputChannels = nooc; + this.forwardTransferEnvelopes = ephemeral; + + this.ephemeralCheckpoint = new EphemeralCheckpoint(task, ephemeral); + if (ephemeral) { + tasksWithUndecidedCheckpoints.put(task.getVertexID(), this); + } + + this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; + + // Each output gate context will register as a sub event listener + this.subEventListener = new AsynchronousEventListener[environment.getNumberOfOutputGates()]; + } + + void registerAsynchronousEventListener(final int index, final AsynchronousEventListener eventListener) { + + if (index >= this.subEventListener.length || index < 0) { + throw new IllegalArgumentException("Argument index has invalid value " + index); + } + + if (eventListener == null) { + throw new IllegalArgumentException("Argument eventListener must not be null"); + } + + if (this.subEventListener[index] != null) { + throw new IllegalStateException("There is already an event listener with index " + index + " registered"); + } + + this.subEventListener[index] = eventListener; + } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException { + + return this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer); + } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException, + InterruptedException { + + return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer); + } + + /** + * {@inheritDoc} + */ + @Override + public int getMaximumBufferSize() { + + return this.localBufferPool.getMaximumBufferSize(); + } + + /** + * {@inheritDoc} + */ + @Override + public void clearLocalBufferPool() { + + // Clear the buffer cache + this.localBufferPool.clear(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isShared() { + + return false; + } + + /** + * {@inheritDoc} + */ + @Override + public void logBufferUtilization() { + + final int ava = this.localBufferPool.getNumberOfAvailableBuffers(); + final int req = this.localBufferPool.getRequestedNumberOfBuffers(); + final int des = this.localBufferPool.getDesignatedNumberOfBuffers(); + + final RuntimeEnvironment environment = this.task.getRuntimeEnvironment(); + + System.out.println("\t\t" + environment.getTaskName() + ": " + ava + " available, " + req + " requested, " + + des + " designated"); + } + + /** + * Called by an {@link OutputGateContext} to indicate that the task has temporarily run out of memory buffers. + */ + void reportExhaustionOfMemoryBuffers() { + + if (!this.initialExhaustionOfMemoryBuffersReported) { + + this.task.initialExecutionResourcesExhausted(); + this.initialExhaustionOfMemoryBuffersReported = true; + } + } + + /** + * {@inheritDoc} + */ + @Override + public void reportAsynchronousEvent() { + + this.localBufferPool.reportAsynchronousEvent(); + } + + /** + * {@inheritDoc} + */ + @Override + public void asynchronousEventOccurred() throws IOException, InterruptedException { + + // First, notify all the listeners about the asynchronous event + for (int i = 0; i < this.subEventListener.length; ++i) { + + if (this.subEventListener[i] == null) { + throw new IllegalStateException("Event listener at index " + i + " is null"); + } + + this.subEventListener[i].asynchronousEventOccurred(); + } + + // Second, check if the checkpoint decision changed + this.ephemeralCheckpoint.checkAsynchronousCheckpointDecision(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfChannels() { + + return this.numberOfOutputChannels; + } + + /** + * {@inheritDoc} + */ + @Override + public void setDesignatedNumberOfBuffers(int numberOfBuffers) { + + this.localBufferPool.setDesignatedNumberOfBuffers(numberOfBuffers); + } + + /** + * Called by the attached output gate context to forward a {@link TransferEnvelope} object + * to its final destination. Within this method the provided transfer envelope is possibly also + * forwarded to the assigned ephemeral checkpoint. + * + * @param outgoingTransferEnvelope + * the transfer envelope to be forwarded + * @throws IOException + * thrown if an I/O error occurs while processing the envelope + * @throws InterruptedException + * thrown if the thread is interrupted while waiting for the envelope to be processed + */ + void processEnvelope(final TransferEnvelope outgoingTransferEnvelope) throws IOException, InterruptedException { + + if (!this.ephemeralCheckpoint.isDiscarded()) { + final TransferEnvelope dup = outgoingTransferEnvelope.duplicate(); + this.ephemeralCheckpoint.addTransferEnvelope(dup); + } + + if (this.forwardTransferEnvelopes) { + // Immediately forward the envelope + this.transferEnvelopeDispatcher.processEnvelopeFromOutputChannel(outgoingTransferEnvelope); + } else { + // Simply discard the envelope + final Buffer buffer = outgoingTransferEnvelope.getBuffer(); + if (buffer != null) { + buffer.recycleBuffer(); + } + } + } + + AbstractID getFileOwnerID() { + + return this.task.getVertexID(); + } + + public void setCheckpointDecisionAsynchronously(final boolean checkpointDecision) { + + // Simply delegate call + this.ephemeralCheckpoint.setCheckpointDecisionAsynchronously(checkpointDecision); + } + + /** + * Registers the given spilling queue with a network connection. The network connection is in charge of polling the + * remaining elements from the queue. + * + * @param sourceChannelID + * the ID of the source channel which is associated with the spilling queue + * @param spillingQueue + * the spilling queue to be registered + * @return true if the has been successfully registered with the network connection, false + * if the receiver runs within the same task manager and there is no network operation required to transfer + * the queued data + * @throws IOException + * thrown if an I/O error occurs while looking up the destination of the queued envelopes + * @throws InterruptedException + * thrown if the thread is interrupted while looking up the destination of the queued envelopes + */ + boolean registerSpillingQueueWithNetworkConnection(final ChannelID sourceChannelID, + final SpillingQueue spillingQueue) throws IOException, InterruptedException { + + return this.transferEnvelopeDispatcher.registerSpillingQueueWithNetworkConnection(this.task.getJobID(), + sourceChannelID, spillingQueue); + } + + /** + * {@inheritDoc} + */ + @Override + public OutputGateContext createOutputGateContext(final GateID gateID) { + + if (gateID == null) { + throw new IllegalArgumentException("Argument gateID must not be null"); + } + + OutputGate outputGate = null; + final RuntimeEnvironment re = this.task.getRuntimeEnvironment(); + for (int i = 0; i < re.getNumberOfOutputGates(); ++i) { + final OutputGate candidateGate = re.getOutputGate(i); + if (candidateGate.getGateID().equals(gateID)) { + outputGate = candidateGate; + break; + } + } + + if (outputGate == null) { + throw new IllegalStateException("Cannot find output gate with ID " + gateID); + } + + return new RuntimeOutputGateContext(this, outputGate); + } + + /** + * {@inheritDoc} + */ + @Override + public InputGateContext createInputGateContext(final GateID gateID) { + + if (gateID == null) { + throw new IllegalArgumentException("Argument gateID must not be null"); + } + + InputGate inputGate = null; + final RuntimeEnvironment re = this.task.getRuntimeEnvironment(); + for (int i = 0; i < re.getNumberOfInputGates(); ++i) { + final InputGate candidateGate = re.getInputGate(i); + if (candidateGate.getGateID().equals(gateID)) { + inputGate = candidateGate; + break; + } + } + + if (inputGate == null) { + throw new IllegalStateException("Cannot find input gate with ID " + gateID); + } + + return new RuntimeInputGateContext(this.transferEnvelopeDispatcher, inputGate); + } + + /** + * {@inheritDoc} + */ + @Override + public LocalBufferPoolOwner getLocalBufferPoolOwner() { + + return this; + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java index d9894f7e90645..573de389e56e8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java @@ -23,9 +23,9 @@ import java.util.List; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; -import eu.stratosphere.nephele.io.channels.AbstractChannel; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.taskmanager.bytebuffered.ChannelContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse; /** @@ -58,15 +58,15 @@ public TransferEnvelopeReceiverList(final ConnectionInfoLookupResponse cilr) { this.remoteReceivers = Collections.unmodifiableList(tmpList); } - public TransferEnvelopeReceiverList(final AbstractChannel inMemoryChannel) { + public TransferEnvelopeReceiverList(final ChannelContext channelContext) { - if (inMemoryChannel.getType() != ChannelType.INMEMORY) { + if (channelContext.getType() != ChannelType.INMEMORY) { throw new IllegalArgumentException( "Transfer envelope receiver lists can only be constructed from in-memory channels."); } final List lr = new ArrayList(1); - lr.add(inMemoryChannel.getConnectedChannelID()); + lr.add(channelContext.getConnectedChannelID()); this.localReceivers = Collections.unmodifiableList(lr); this.remoteReceivers = Collections.emptyList(); From 5da5783b92ac8c1456821a9c31b612314cd8790f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 31 Jan 2012 21:13:05 +0100 Subject: [PATCH 187/310] Cleaned up task manager operation protocol --- .../nephele/execution/ExecutionState.java | 7 +- .../queue/TestDeploymentManager.java | 10 - .../CheckpointReplayRequest.java | 121 ------------ .../checkpointing/CheckpointReplayResult.java | 41 ----- .../nephele/instance/AbstractInstance.java | 8 - .../nephele/jobmanager/DeploymentManager.java | 13 -- .../nephele/jobmanager/JobManager.java | 50 ----- .../scheduler/AbstractScheduler.java | 24 +-- .../jobmanager/scheduler/RecoveryLogic.java | 63 +------ .../protocols/TaskOperationProtocol.java | 4 - .../nephele/taskmanager/TaskManager.java | 174 ++++++------------ .../ByteBufferedChannelManager.java | 4 +- 12 files changed, 76 insertions(+), 443 deletions(-) delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayRequest.java delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayResult.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java index a6bcd3cb47482..a1ea7068f32d2 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionState.java @@ -75,5 +75,10 @@ public enum ExecutionState { /** * The task has been aborted due to a failure during execution. */ - FAILED; + FAILED, + + /** + * The task is being replayed from a checkpoint (the original task may still be running). + */ + REPLAYING; } diff --git a/nephele/nephele-queuescheduler/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java b/nephele/nephele-queuescheduler/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java index e6d598ed62934..4a71d9c51a536 100644 --- a/nephele/nephele-queuescheduler/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java +++ b/nephele/nephele-queuescheduler/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/TestDeploymentManager.java @@ -17,7 +17,6 @@ import java.util.List; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.jobgraph.JobID; @@ -63,15 +62,6 @@ public void deploy(final JobID jobID, final AbstractInstance instance, } } - /** - * {@inheritDoc} - */ - @Override - public void replayCheckpoints(final JobID jobID, final AbstractInstance instance, - final List replayRequests) { - throw new IllegalStateException("replayCheckpoints called on TestDeploymentManager"); - } - /** * Returns the ID of the last deployed job. */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayRequest.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayRequest.java deleted file mode 100644 index 1992ae5c82012..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayRequest.java +++ /dev/null @@ -1,121 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.checkpointing; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.Collections; -import java.util.Set; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.IOReadableWritable; -import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.util.SerializableHashSet; - -/** - * A checkpoint replay request can be used to trigger the replay of a checkpoint at a remote task manager. - * - * @author warneke - */ -public final class CheckpointReplayRequest implements IOReadableWritable { - - /** - * The vertex ID which identifies the checkpoint to be replayed. - */ - private ExecutionVertexID vertexID; - - /** - * The IDs of the output channels the replay task will be in charge of. - */ - private SerializableHashSet outputChannelIDs = new SerializableHashSet(); - - /** - * Constructs a new checkpoint replay request. - * - * @param vertexID - * the vertex ID identifying the checkpoint to be replayed - */ - public CheckpointReplayRequest(final ExecutionVertexID vertexID) { - - if (vertexID == null) { - throw new IllegalArgumentException("Argument vertexID must no be null"); - } - - this.vertexID = vertexID; - } - - /** - * Default constructor for serialization/deserialization. - */ - public CheckpointReplayRequest() { - - this.vertexID = new ExecutionVertexID(); - } - - /** - * Adds a set of channel IDs which identify the output channels the replay task will be in charge of. - * - * @param outputChannelIDs - * the IDs of the output channels the replay task will be in charge of - */ - public void addOutputChannelIDs(final Set outputChannelIDs) { - - this.outputChannelIDs.addAll(outputChannelIDs); - } - - /** - * Returns the set of channel IDs which identify the output channels the replay task will be in charge of. - * - * @return the IDs of the output channels the replay task will be in charge of - */ - public Set getOutputChannelIDs() { - - return Collections.unmodifiableSet(this.outputChannelIDs); - } - - /** - * Returns the vertex ID identifying the checkpoint to be replayed. - * - * @return the vertex ID identifying the checkpoint to be replayed - */ - public ExecutionVertexID getVertexID() { - - return this.vertexID; - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - - this.vertexID.write(out); - - this.outputChannelIDs.write(out); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - - this.vertexID.read(in); - - this.outputChannelIDs.read(in); - } -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayResult.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayResult.java deleted file mode 100644 index cb40e77ece336..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayResult.java +++ /dev/null @@ -1,41 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.checkpointing; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.taskmanager.AbstractTaskResult; - -public class CheckpointReplayResult extends AbstractTaskResult { - - /** - * Constructs a new checkpoint replay result. - * - * @param vertexID - * the task ID this result belongs to - * @param returnCode - * the return code of the submission - */ - public CheckpointReplayResult(final ExecutionVertexID vertexID, final ReturnCode returnCode) { - super(vertexID, returnCode); - } - - /** - * Constructs an empty checkpoint replay result. - */ - public CheckpointReplayResult() { - super(); - } -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index 55cc2d222c9d5..57e80440d0ae7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -21,8 +21,6 @@ import java.util.Set; import eu.stratosphere.nephele.checkpointing.CheckpointDecision; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.RuntimeEnvironment; @@ -239,12 +237,6 @@ public synchronized List submitTasks(final List replayCheckpoints(final List replayRequests) - throws IOException { - - return getTaskManager().replayCheckpoints(replayRequests); - } - public synchronized void propagateCheckpointDecisions(final List checkpointDecisions) throws IOException { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java index 0222356b641bc..9a053617b2656 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/DeploymentManager.java @@ -17,7 +17,6 @@ import java.util.List; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.jobgraph.JobID; @@ -43,16 +42,4 @@ public interface DeploymentManager { * the list of vertices to be deployed */ void deploy(JobID jobID, AbstractInstance instance, List verticesToBeDeployed); - - /** - * Initializes a list of checkpoints to be replayed on a given {@link AbstractInstance}. - * - * @param jobID - * the ID of the job the checkpoints to be replayed belongs to - * @param instance - * the instance on which the checkpoints shall be replayed - * @param replayRequests - * the list of replay requests - */ - void replayCheckpoints(JobID jobID, AbstractInstance instance, List replayRequests); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 697943c6d1336..da8af2aa1d8ba 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -59,8 +59,6 @@ import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.checkpointing.CheckpointDecisionCoordinator; import eu.stratosphere.nephele.checkpointing.CheckpointDecisionPropagator; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; import eu.stratosphere.nephele.client.AbstractJobResult; import eu.stratosphere.nephele.client.JobCancelResult; import eu.stratosphere.nephele.client.JobProgressResult; @@ -1234,54 +1232,6 @@ public void run() { this.executorService.execute(deploymentRunnable); } - /** - * {@inheritDoc} - */ - @Override - public void replayCheckpoints(final JobID jobID, final AbstractInstance instance, - final List replayRequests) { - - if (replayRequests.isEmpty()) { - LOG.error("Method 'replayCheckpoints' called but list of checkpoints to be replayed is empty"); - return; - } - - // Create a new runnable and pass it the executor service - final Runnable deploymentRunnable = new Runnable() { - - /** - * {@inheritDoc} - */ - @Override - public void run() { - - List checkpointResultList = null; - - try { - checkpointResultList = instance.replayCheckpoints(replayRequests); - } catch (final IOException ioe) { - final String errorMsg = StringUtils.stringifyException(ioe); - // TODO: Handle this correctly - LOG.error(errorMsg); - } - - if (replayRequests.size() != checkpointResultList.size()) { - LOG.error("size of submission result list does not match size of list with checkpoints to be deployed"); - } - - for (final CheckpointReplayResult ccr : checkpointResultList) { - - if (ccr.getReturnCode() == AbstractTaskResult.ReturnCode.ERROR) { - // TODO: Handle this correctly - LOG.error(ccr.getDescription()); - } - } - } - }; - - this.executorService.execute(deploymentRunnable); - } - /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index 7fc485f4bd022..4581c8f4b6e66 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -26,7 +26,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.util.StringUtils; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.execution.RuntimeEnvironment; @@ -52,7 +51,6 @@ import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobmanager.DeploymentManager; import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.util.SerializableArrayList; /** * This abstract scheduler must be extended by a scheduler implementations for Nephele. The abstract class defines the @@ -472,31 +470,13 @@ protected void replayCheckpointsFromPreviousStage(final ExecutionGraph execution final int currentStageIndex = executionGraph.getIndexOfCurrentExecutionStage(); final ExecutionStage previousStage = executionGraph.getStage(currentStageIndex - 1); - final Map> checkpointsToReplay = new HashMap>(); - for (int i = 0; i < previousStage.getNumberOfOutputExecutionVertices(); ++i) { final ExecutionVertex vertex = previousStage.getOutputExecutionVertex(i); - final AbstractInstance instance = vertex.getAllocatedResource().getInstance(); - - List replayRequests = checkpointsToReplay.get(instance); - if (replayRequests == null) { - replayRequests = new SerializableArrayList(); - checkpointsToReplay.put(instance, replayRequests); - } - - final CheckpointReplayRequest replayRequest = new CheckpointReplayRequest(vertex.getID()); - replayRequest.addOutputChannelIDs(vertex.getEnvironment().getOutputChannelIDs()); - - replayRequests.add(replayRequest); + vertex.updateExecutionState(ExecutionState.ASSIGNED); } - final Iterator>> it = checkpointsToReplay.entrySet() - .iterator(); - while (it.hasNext()) { - final Map.Entry> entry = it.next(); - this.deploymentManager.replayCheckpoints(executionGraph.getJobID(), entry.getKey(), entry.getValue()); - } + deployAssignedVertices(executionGraph); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index 6d36bf760d778..706847643ac94 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -19,7 +19,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Queue; import java.util.Set; @@ -27,8 +26,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; @@ -46,7 +43,6 @@ import eu.stratosphere.nephele.taskmanager.TaskCancelResult; import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; import eu.stratosphere.nephele.types.Record; -import eu.stratosphere.nephele.util.SerializableArrayList; import eu.stratosphere.nephele.util.SerializableHashSet; import eu.stratosphere.nephele.util.StringUtils; @@ -79,7 +75,7 @@ public static boolean recover(final ExecutionVertex failedVertex, final Set verticesToBeCanceled = new HashSet(); - final Map> checkpointsToBeReplayed = new HashMap>(); + final Set checkpointsToBeReplayed = new HashSet(); findVerticesToRestart(failedVertex, verticesToBeCanceled, checkpointsToBeReplayed); @@ -105,28 +101,11 @@ public static boolean recover(final ExecutionVertex failedVertex, } // Replay all necessary checkpoints - final Iterator>> checkpointIterator = checkpointsToBeReplayed - .entrySet().iterator(); + final Iterator checkpointIterator = checkpointsToBeReplayed.iterator(); while (checkpointIterator.hasNext()) { - final Map.Entry> entry = checkpointIterator.next(); - final AbstractInstance instance = entry.getKey(); - - try { - final List results = instance.replayCheckpoints(toListOfReplayRequests( - failedVertex.getExecutionGraph(), entry.getValue())); - for (final CheckpointReplayResult result : results) { - if (result.getReturnCode() != ReturnCode.SUCCESS) { - LOG.error(result.getDescription()); - return false; - } - } - - } catch (IOException ioe) { - LOG.error(StringUtils.stringifyException(ioe)); - return false; - } + checkpointIterator.next().updateExecutionState(ExecutionState.ASSIGNED); } // Restart failed vertex @@ -139,32 +118,9 @@ public static boolean recover(final ExecutionVertex failedVertex, return true; } - private static List toListOfReplayRequests(final ExecutionGraph executionGraph, - final Set vertexIDs) { - - final List replayRequests = new SerializableArrayList(); - final Iterator it = vertexIDs.iterator(); - while (it.hasNext()) { - - final ExecutionVertexID vertexID = it.next(); - final ExecutionVertex vertex = executionGraph.getVertexByID(vertexID); - - if (vertex == null) { - LOG.error("Cannot find execution vertex with ID " + vertexID); - continue; - } - - final CheckpointReplayRequest replayRequest = new CheckpointReplayRequest(vertexID); - replayRequest.addOutputChannelIDs(vertex.getEnvironment().getOutputChannelIDs()); - replayRequests.add(replayRequest); - } - - return replayRequests; - } - private static void findVerticesToRestart(final ExecutionVertex failedVertex, final Set verticesToBeCanceled, - final Map> checkpointsToBeReplayed) { + final Set checkpointsToBeReplayed) { final Queue verticesToTest = new ArrayDeque(); final Set visited = new HashSet(); @@ -189,16 +145,7 @@ private static void findVerticesToRestart(final ExecutionVertex failedVertex, verticesToTest.add(predecessor); } } else { - - // Group IDs by instance - final AbstractInstance instance = predecessor.getAllocatedResource().getInstance(); - Set checkpointIDs = checkpointsToBeReplayed.get(instance); - if (checkpointIDs == null) { - checkpointIDs = new HashSet(); - checkpointsToBeReplayed.put(instance, checkpointIDs); - } - - checkpointIDs.add(predecessor.getID()); + checkpointsToBeReplayed.add(predecessor); } } visited.add(vertex); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index 82b9f7588058f..ed460c6b5fb76 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -20,8 +20,6 @@ import java.util.Set; import eu.stratosphere.nephele.checkpointing.CheckpointDecision; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest; @@ -120,8 +118,6 @@ List submitTasks(List tasks) */ void updateLibraryCache(LibraryCacheUpdate update) throws IOException; - List replayCheckpoints(List replayRequests) throws IOException; - void propagateCheckpointDecisions(List checkpointDecisions) throws IOException; /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index ceddaac43c8ff..36b318989aa36 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -20,13 +20,13 @@ import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.ArrayList; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; @@ -40,8 +40,6 @@ import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.checkpointing.CheckpointReplayManager; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayRequest; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayResult; import eu.stratosphere.nephele.configuration.ConfigConstants; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.configuration.GlobalConfiguration; @@ -79,7 +77,6 @@ import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager; -import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; import eu.stratosphere.nephele.taskmanager.bytebuffered.ByteBufferedChannelManager; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.util.SerializableArrayList; @@ -114,7 +111,7 @@ public class TaskManager implements TaskOperationProtocol, PluginCommunicationPr * is stored inside this map and its thread status is TERMINATED, this indicates a virtual machine error. * As a result, task status will switch to FAILED and reported to the {@link JobManager}. */ - private final Map runningTasks = new HashMap(); + private final Map runningTasks = new ConcurrentHashMap(); private final InstanceConnectionInfo localInstanceConnectionInfo; @@ -416,21 +413,14 @@ public void runIOLoop() { @Override public TaskCancelResult cancelTask(final ExecutionVertexID id) throws IOException { - // Check if the task is registered with our task manager - Task tmpTask; - - synchronized (this.runningTasks) { - - tmpTask = this.runningTasks.get(id); + final Task task = this.runningTasks.get(id); - if (tmpTask == null) { - final TaskCancelResult taskCancelResult = new TaskCancelResult(id, AbstractTaskResult.ReturnCode.ERROR); - taskCancelResult.setDescription("No task with ID + " + id + " is currently running"); - return taskCancelResult; - } + if (task == null) { + final TaskCancelResult taskCancelResult = new TaskCancelResult(id, AbstractTaskResult.ReturnCode.ERROR); + taskCancelResult.setDescription("No task with ID + " + id + " is currently running"); + return taskCancelResult; } - final Task task = tmpTask; // Execute call in a new thread so IPC thread can return immediately final Thread tmpThread = new Thread(new Runnable() { @@ -490,13 +480,6 @@ public TaskSubmissionResult submitTask(final ExecutionVertexID id, final Configu final RuntimeEnvironment re, final Set activeOutputChannels) throws IOException { - // Register task manager components in environment - re.setMemoryManager(this.memoryManager); - re.setIOManager(this.ioManager); - - // Register a new task input split provider - re.setInputSplitProvider(new TaskInputSplitProvider(re.getJobID(), id, this.globalInputSplitProvider)); - // Create task object and register it with the environment final RuntimeTask task = new RuntimeTask(id, re, this); re.setExecutionObserver(task); @@ -581,87 +564,49 @@ private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Conf throw new IllegalArgumentException("Argument task is null"); } - // Check if the task is already running - synchronized (this.runningTasks) { - if (this.runningTasks.containsKey(id)) { - LOG.debug("Task with ID " + id + " is already running"); - TaskSubmissionResult result = new TaskSubmissionResult(id, AbstractTaskResult.ReturnCode.ERROR); + // Task registration must be atomic + synchronized (this) { + + final Task runningTask = this.runningTasks.get(id); + if (runningTask != null) { + LOG.error("Task with ID " + id + " is already running"); + final TaskSubmissionResult result = new TaskSubmissionResult(id, AbstractTaskResult.ReturnCode.ERROR); result.setDescription("Task with ID " + id + " is already running"); return result; } - } - final Environment ee = task.getEnvironment(); + final Environment ee = task.getEnvironment(); - // Register task manager components with the task - task.registerMemoryManager(this.memoryManager); - task.registerIOManager(this.ioManager); - task.registerInputSplitProvider(new TaskInputSplitProvider(ee.getJobID(), id, this.globalInputSplitProvider)); + // Register task manager components with the task + task.registerMemoryManager(this.memoryManager); + task.registerIOManager(this.ioManager); + task.registerInputSplitProvider(new TaskInputSplitProvider(ee.getJobID(), id, this.globalInputSplitProvider)); - // Register the task with the byte buffered channel manager - this.byteBufferedChannelManager.register(task, activeOutputChannels); + // Register the task with the byte buffered channel manager + this.byteBufferedChannelManager.register(task, activeOutputChannels); - boolean enableProfiling = false; - if (this.profiler != null && jobConfiguration.getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) { - enableProfiling = true; - } - - // Register environment, input, and output gates for profiling - if (enableProfiling) { - task.registerProfiler(this.profiler, jobConfiguration); - } - - // Allow plugins to register their listeners for this task - if (!this.taskManagerPlugins.isEmpty()) { - final Iterator it = this.taskManagerPlugins.values().iterator(); - while (it.hasNext()) { - it.next().registerTask(id, jobConfiguration, ee); + boolean enableProfiling = false; + if (this.profiler != null && jobConfiguration.getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) { + enableProfiling = true; } - } - - // The environment itself will put the task into the running task map - return null; - } + // Register environment, input, and output gates for profiling + if (enableProfiling) { + task.registerProfiler(this.profiler, jobConfiguration); + } - /** - * {@inheritDoc} - */ - @Override - public SerializableArrayList replayCheckpoints( - final List replayRequests) throws IOException { - - final SerializableArrayList checkpointResultList = new SerializableArrayList(); - - for (final CheckpointReplayRequest replayRequest : replayRequests) { - - if (!this.checkpointManager.hasCompleteCheckpointAvailable(vertexID)) { - - if (this.checkpointManager.hasPartialCheckpointAvailable(vertexID)) { - synchronized (this.runningTasks) { - if (!this.runningTasks.containsKey(vertexID)) { - final CheckpointReplayResult result = new CheckpointReplayResult(vertexID, ReturnCode.ERROR); - result - .setDescription("Checkpoint is only partial and corresponding task is no longer running"); - checkpointResultList.add(result); - continue; - } - } - } else { - final CheckpointReplayResult result = new CheckpointReplayResult(vertexID, ReturnCode.ERROR); - result.setDescription("No checkpoint found"); - checkpointResultList.add(result); - continue; + // Allow plugins to register their listeners for this task + if (!this.taskManagerPlugins.isEmpty()) { + final Iterator it = this.taskManagerPlugins.values().iterator(); + while (it.hasNext()) { + it.next().registerTask(id, jobConfiguration, ee); } } - // Replay the checkpoint - this.checkpointManager.replayCheckpoint(vertexID); - - checkpointResultList.add(new CheckpointReplayResult(vertexID, ReturnCode.SUCCESS)); + this.runningTasks.put(id, task); } - return checkpointResultList; + return null; } /** @@ -674,26 +619,28 @@ public SerializableArrayList replayCheckpoints( */ private void unregisterTask(final ExecutionVertexID id, final Task task) { - // Unregister task from the byte buffered channel manager - this.byteBufferedChannelManager.unregister(id, task); + // Task deregistration must be atomic + synchronized (this) { - // Unregister task from profiling - task.unregisterProfiler(this.profiler); + // Unregister task from the byte buffered channel manager + this.byteBufferedChannelManager.unregister(id, task); - // Unregister task from memory manager - task.unregisterMemoryManager(this.memoryManager); + // Unregister task from profiling + task.unregisterProfiler(this.profiler); - // Allow plugins to unregister their listeners for this task - if (!this.taskManagerPlugins.isEmpty()) { - final Iterator it = this.taskManagerPlugins.values().iterator(); - while (it.hasNext()) { - it.next().unregisterTask(id, task.getEnvironment()); + // Unregister task from memory manager + task.unregisterMemoryManager(this.memoryManager); + + // Allow plugins to unregister their listeners for this task + if (!this.taskManagerPlugins.isEmpty()) { + final Iterator it = this.taskManagerPlugins.values().iterator(); + while (it.hasNext()) { + it.next().unregisterTask(id, task.getEnvironment()); + } } - } - // Check if there are still vertices running that belong to the same job - int numberOfVerticesBelongingToThisJob = 0; - synchronized (this.runningTasks) { + // Check if there are still vertices running that belong to the same job + int numberOfVerticesBelongingToThisJob = 0; final Iterator iterator = this.runningTasks.values().iterator(); while (iterator.hasNext()) { final Task candidateTask = iterator.next(); @@ -701,14 +648,14 @@ private void unregisterTask(final ExecutionVertexID id, final Task task) { numberOfVerticesBelongingToThisJob++; } } - } - // If there are no other vertices belonging to the same job, we can unregister the job's class loader - if (numberOfVerticesBelongingToThisJob == 0) { - try { - LibraryCacheManager.unregister(task.getJobID()); - } catch (IOException e) { - LOG.debug("Unregistering the job vertex ID " + id + " caused an IOException"); + // If there are no other vertices belonging to the same job, we can unregister the job's class loader + if (numberOfVerticesBelongingToThisJob == 0) { + try { + LibraryCacheManager.unregister(task.getJobID()); + } catch (IOException e) { + LOG.debug("Unregistering the job vertex ID " + id + " caused an IOException"); + } } } } @@ -797,7 +744,8 @@ public void initialExecutionResourcesExhausted(final JobID jobID, final Executio } } - public void checkpointStateChanged(final JobID jobID, final ExecutionVertexID id, final CheckpointState newCheckpointState) { + public void checkpointStateChanged(final JobID jobID, final ExecutionVertexID id, + final CheckpointState newCheckpointState) { synchronized (this.jobManager) { try { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 4053b696ad854..3e5baa18949ab 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -129,7 +129,7 @@ public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupServi * @param the * set of output channels which are initially active */ - public synchronized void register(final Task task, final Set activeOutputChannels) { + public void register(final Task task, final Set activeOutputChannels) { final Environment environment = task.getEnvironment(); @@ -218,7 +218,7 @@ public synchronized void register(final Task task, final Set activeOu * @param task * the task to be unregistered */ - public synchronized void unregister(final ExecutionVertexID vertexID, final Task task) { + public void unregister(final ExecutionVertexID vertexID, final Task task) { final Environment environment = task.getEnvironment(); From 75c9c124d2eac42c5c1300ade85f001e08b59e1b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 1 Feb 2012 20:28:55 +0100 Subject: [PATCH 188/310] Worked on proper event dispatching in case a replay task runs concurrently with the original task --- .../AbstractReplayGateContext.java | 23 ++ .../checkpointing/CheckpointEnvironment.java | 324 +++++++++++++++++ .../CheckpointReplayManager.java | 91 +---- .../checkpointing/CheckpointReplayTask.java | 155 -------- .../checkpointing/CheckpointUtils.java | 101 ++++++ .../checkpointing/EphemeralCheckpoint.java | 100 +++--- .../checkpointing/ReplayFinishedNotifier.java | 36 -- .../ReplayInputChannelContext.java | 137 +++++++ .../checkpointing/ReplayInputGateContext.java | 33 ++ .../ReplayOutputChannelContext.java | 63 ++++ .../ReplayOutputGateContext.java | 36 ++ .../nephele/checkpointing/ReplayTask.java | 181 ++++++++++ .../checkpointing/ReplayTaskContext.java | 35 ++ .../executiongraph/ExecutionVertex.java | 15 +- .../nephele/instance/AbstractInstance.java | 26 -- .../protocols/TaskOperationProtocol.java | 21 -- .../taskmanager/CheckpointReplayTask.java | 5 - .../nephele/taskmanager/TaskManager.java | 148 ++++---- .../AbstractOutputChannelContext.java | 40 +++ .../bytebuffered/IncomingEventQueue.java | 31 ++ .../bytebuffered/OutputChannelForwarder.java | 15 + .../OutputChannelForwardingChain.java | 77 ++++ .../runtime/ForwardingBarrier.java | 71 ++++ .../runtime/RuntimeDispatcher.java | 43 +++ .../runtime/RuntimeOutputChannelBroker.java | 216 +++++++++++ .../runtime/RuntimeOutputChannelContext.java | 340 +----------------- .../runtime/RuntimeOutputGateContext.java | 159 ++------ .../taskmanager/runtime/RuntimeTask.java | 2 + .../runtime/RuntimeTaskContext.java | 102 +----- .../taskmanager/runtime/SpillingBarrier.java | 64 ++++ 30 files changed, 1669 insertions(+), 1021 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/AbstractReplayGateContext.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayFinishedNotifier.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/CheckpointReplayTask.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingEventQueue.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/AbstractReplayGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/AbstractReplayGateContext.java new file mode 100644 index 0000000000000..f723c5e5b5f34 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/AbstractReplayGateContext.java @@ -0,0 +1,23 @@ +package eu.stratosphere.nephele.checkpointing; + +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.taskmanager.bytebuffered.GateContext; + +abstract class AbstractReplayGateContext implements GateContext { + + private final GateID gateID; + + AbstractReplayGateContext(final GateID gateID) { + this.gateID = gateID; + } + + /** + * {@inheritDoc} + */ + @Override + public GateID getGateID() { + + return this.gateID; + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java new file mode 100644 index 0000000000000..c56be1a697871 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java @@ -0,0 +1,324 @@ +package eu.stratosphere.nephele.checkpointing; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.io.ChannelSelector; +import eu.stratosphere.nephele.io.DistributionPattern; +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.RecordDeserializer; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.services.iomanager.IOManager; +import eu.stratosphere.nephele.services.memorymanager.MemoryManager; +import eu.stratosphere.nephele.template.InputSplitProvider; +import eu.stratosphere.nephele.types.Record; + +final class CheckpointEnvironment implements Environment { + + private final JobID jobID; + + private final Map> outputChannelIDs; + + private final Map> inputChannelIDs; + + private CheckpointEnvironment(final JobID jobID, Map> outputChannelIDs, + Map> inputChannelIDs) { + + this.jobID = jobID; + this.outputChannelIDs = outputChannelIDs; + this.inputChannelIDs = inputChannelIDs; + } + + static CheckpointEnvironment createFromEnvironment(final Environment environment) { + + final JobID jobID = environment.getJobID(); + + final Map> outputChannelIDs = new HashMap>(); + + final Map> inputChannelIDs = new HashMap>(); + + Iterator gateIt = environment.getOutputGateIDs().iterator(); + while (gateIt.hasNext()) { + + final GateID gateID = gateIt.next(); + outputChannelIDs.put(gateID, environment.getOutputChannelIDsOfGate(gateID)); + } + + gateIt = environment.getInputGateIDs().iterator(); + while (gateIt.hasNext()) { + + final GateID gateID = gateIt.next(); + inputChannelIDs.put(gateID, environment.getInputChannelIDsOfGate(gateID)); + } + + return new CheckpointEnvironment(jobID, Collections.unmodifiableMap(outputChannelIDs), + Collections.unmodifiableMap(inputChannelIDs)); + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + + return this.jobID; + } + + /** + * {@inheritDoc} + */ + @Override + public Configuration getTaskConfiguration() { + + throw new UnsupportedOperationException("Method getTaskConfiguration is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public Configuration getJobConfiguration() { + + throw new UnsupportedOperationException("Method getJobConfiguration is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public int getCurrentNumberOfSubtasks() { + + throw new UnsupportedOperationException( + "Method getCurrentNumberOfSubtasks is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public int getIndexInSubtaskGroup() { + + throw new UnsupportedOperationException("Method getIndexInSubtaskGroup is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadStarted(final Thread userThread) { + + throw new UnsupportedOperationException("Method userThreadStarted is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadFinished(final Thread userThread) { + + throw new UnsupportedOperationException("Method userThreadFinished is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public InputSplitProvider getInputSplitProvider() { + + throw new UnsupportedOperationException("Method getInputSplitProvider is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public IOManager getIOManager() { + + throw new UnsupportedOperationException("Method getIOManager is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public MemoryManager getMemoryManager() { + + throw new UnsupportedOperationException("Method getMemoryManager is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public String getTaskName() { + + throw new UnsupportedOperationException("Method getTaskName is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public GateID getNextUnboundInputGateID() { + + throw new UnsupportedOperationException("Method getNextUnboundInputGateID is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public GateID getNextUnboundOutputGateID() { + + throw new UnsupportedOperationException( + "Method getNextUnboundOutputGateID is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfOutputGates() { + + throw new UnsupportedOperationException("Method getNumberOfOutputGates is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfInputGates() { + + throw new UnsupportedOperationException("Method getNumberOfInputGates is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public OutputGate createOutputGate(final GateID gateID, Class outputClass, + final ChannelSelector selector, final boolean isBroadcast) { + + throw new UnsupportedOperationException("Method createOutputGate is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public InputGate createInputGate(final GateID gateID, + final RecordDeserializer deserializer, final DistributionPattern distributionPattern) { + + throw new UnsupportedOperationException("Method createInputGate is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerOutputGate(final OutputGate outputGate) { + + throw new UnsupportedOperationException("Method registerOutputGate is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public void registerInputGate(final InputGate inputGate) { + + throw new UnsupportedOperationException("Method registerInputGate is not supported by this environment"); + } + + /** + * {@inheritDoc} + */ + @Override + public Set getOutputChannelIDs() { + + final Set channelIDs = new HashSet(); + final Iterator>> it = this.outputChannelIDs.entrySet().iterator(); + while (it.hasNext()) { + + final Map.Entry> entry = it.next(); + channelIDs.addAll(entry.getValue()); + + } + + return Collections.unmodifiableSet(channelIDs); + } + + /** + * {@inheritDoc} + */ + @Override + public Set getInputChannelIDs() { + + final Set channelIDs = new HashSet(); + final Iterator>> it = this.inputChannelIDs.entrySet().iterator(); + while (it.hasNext()) { + + final Map.Entry> entry = it.next(); + channelIDs.addAll(entry.getValue()); + + } + + return Collections.unmodifiableSet(channelIDs); + } + + /** + * {@inheritDoc} + */ + @Override + public Set getOutputGateIDs() { + + return this.outputChannelIDs.keySet(); + } + + /** + * {@inheritDoc} + */ + @Override + public Set getInputGateIDs() { + + return this.inputChannelIDs.keySet(); + } + + /** + * {@inheritDoc} + */ + @Override + public Set getOutputChannelIDsOfGate(final GateID gateID) { + + final Set channelIDs = this.outputChannelIDs.get(gateID); + if (channelIDs == null) { + throw new IllegalStateException("Cannot find channel IDs for output gate with ID " + gateID); + } + + return channelIDs; + } + + /** + * {@inheritDoc} + */ + @Override + public Set getInputChannelIDsOfGate(final GateID gateID) { + + final Set channelIDs = this.inputChannelIDs.get(gateID); + if (channelIDs == null) { + throw new IllegalStateException("Cannot find channel IDs for input gate with ID " + gateID); + } + + return channelIDs; + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java index 8d3978de22c9d..8444011524853 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java @@ -15,73 +15,25 @@ package eu.stratosphere.nephele.checkpointing; -import java.io.File; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.configuration.GlobalConfiguration; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; -public class CheckpointReplayManager implements ReplayFinishedNotifier { - +public class CheckpointReplayManager { + private static final Log LOG = LogFactory.getLog(CheckpointReplayManager.class); - public static final String CHECKPOINT_DIRECTORY_KEY = "channel.checkpoint.directory"; - - public static final String DEFAULT_CHECKPOINT_DIRECTORY = "/tmp"; - - /** - * The prefix for the name of the file containing the checkpoint meta data. - */ - public static final String METADATA_PREFIX = "checkpoint"; - private final TransferEnvelopeDispatcher transferEnvelopeDispatcher; - private final String checkpointDirectory; - - private final ConcurrentMap runningReplayTasks; - public CheckpointReplayManager(final TransferEnvelopeDispatcher transferEnvelopeDispatcher) { this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; - - this.checkpointDirectory = GlobalConfiguration - .getString(CHECKPOINT_DIRECTORY_KEY, DEFAULT_CHECKPOINT_DIRECTORY); - - this.runningReplayTasks = new ConcurrentHashMap(); } - public boolean hasCompleteCheckpointAvailable(final ExecutionVertexID vertexID) { - - final File file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID - + "_final"); - if (file.exists()) { - return true; - } + - return false; - } - - public boolean hasPartialCheckpointAvailable(final ExecutionVertexID vertexID) { - - File file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_0"); - if (file.exists()) { - return true; - } - - file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_part"); - if (file.exists()) { - return true; - } - - return false; - } - - public void replayCheckpoint(final ExecutionVertexID vertexID) { + /*public void replayCheckpoint(final ExecutionVertexID vertexID) { final CheckpointReplayTask newReplayTask = new CheckpointReplayTask(this, vertexID, this.checkpointDirectory, this.transferEnvelopeDispatcher, hasCompleteCheckpointAvailable(vertexID)); @@ -94,38 +46,7 @@ public void replayCheckpoint(final ExecutionVertexID vertexID) { LOG.info("Replaying checkpoint for vertex " + vertexID); newReplayTask.start(); - } - - /** - * Removes the checkpoint of the vertex with the given ID. All files contained in the checkpoint are deleted. - * - * @param vertexID - * the vertex whose checkpoint shall be removed - */ - public void removeCheckpoint(final ExecutionVertexID vertexID) { - File file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID - + "_final"); - if (file.exists()) { - file.delete(); - return; - } - file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_0"); - if (file.exists()) { - file.delete(); - } + }*/ - file = new File(this.checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_part"); - if (file.exists()) { - file.delete(); - } - } - - /** - * {@inheritDoc} - */ - @Override - public void replayFinished(final ExecutionVertexID vertexID) { - - this.runningReplayTasks.remove(vertexID); - } + } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java deleted file mode 100644 index 738da7558954e..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayTask.java +++ /dev/null @@ -1,155 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.checkpointing; - -import java.io.EOFException; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.channels.FileChannel; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.taskmanager.transferenvelope.CheckpointDeserializer; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; - -class CheckpointReplayTask extends Thread { - - private final ReplayFinishedNotifier replayFinishedNotifier; - - private final ExecutionVertexID vertexID; - - private final String checkpointDirectory; - - private final TransferEnvelopeDispatcher transferEnvelopeDispatcher; - - private final boolean isCheckpointComplete; - - private volatile boolean isCanceled = false; - - CheckpointReplayTask(final ReplayFinishedNotifier replayFinishedNotifier, final ExecutionVertexID vertexID, - final String checkpointDirectory, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, - final boolean isCheckpointComplete) { - - super("Recovery task for " + vertexID); - - this.replayFinishedNotifier = replayFinishedNotifier; - this.vertexID = vertexID; - this.checkpointDirectory = checkpointDirectory; - this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; - this.isCheckpointComplete = isCheckpointComplete; - } - - /** - * {@inheritDoc} - */ - @Override - public void run() { - - try { - replayCheckpoint(); - } catch (IOException ioe) { - // TODO: Handle this correctly - ioe.printStackTrace(); - } - - // Notify the checkpoint replay manager that the replay has been finished - this.replayFinishedNotifier.replayFinished(this.vertexID); - } - - void cancelAndWait() { - - this.isCanceled = true; - interrupt(); - - try { - join(); - } catch (InterruptedException ie) { - ie.printStackTrace(); - } - } - - private void replayCheckpoint() throws IOException { - - final CheckpointDeserializer deserializer = new CheckpointDeserializer(this.vertexID); - - int metaDataIndex = 0; - - while (true) { - - // Try to locate the meta data file - final File metaDataFile = new File(this.checkpointDirectory + File.separator - + CheckpointReplayManager.METADATA_PREFIX + "_" + this.vertexID + "_" + metaDataIndex); - - while (!metaDataFile.exists()) { - - // Try to locate the final meta data file - final File finalMetaDataFile = new File(this.checkpointDirectory + File.separator - + CheckpointReplayManager.METADATA_PREFIX + "_" + this.vertexID + "_final"); - - if (finalMetaDataFile.exists()) { - return; - } - - if (this.isCheckpointComplete) { - throw new FileNotFoundException("Cannot find meta data file " + metaDataIndex - + " for checkpoint of vertex " + this.vertexID); - } - - // Wait for the file to be created - try { - Thread.sleep(100); - } catch (InterruptedException e) { - // Ignore exception - } - - if (this.isCanceled) { - return; - } - } - - FileInputStream fis = null; - - try { - - fis = new FileInputStream(metaDataFile); - final FileChannel fileChannel = fis.getChannel(); - - while (!this.isCanceled) { - try { - deserializer.read(fileChannel); - - final TransferEnvelope transferEnvelope = deserializer.getFullyDeserializedTransferEnvelope(); - if (transferEnvelope != null) { - this.transferEnvelopeDispatcher.processEnvelopeFromOutputChannel(transferEnvelope); - } - } catch (EOFException eof) { - // Close the file channel - fileChannel.close(); - // Increase the index of the meta data file - ++metaDataIndex; - break; - } - } - } finally { - if (fis != null) { - fis.close(); - } - } - } - } -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java new file mode 100644 index 0000000000000..72196ca1bbdd8 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java @@ -0,0 +1,101 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; + +import java.io.File; + +import eu.stratosphere.nephele.configuration.GlobalConfiguration; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; + +public final class CheckpointUtils { + + /** + * The prefix for the name of the file containing the checkpoint meta data. + */ + public static final String METADATA_PREFIX = "checkpoint"; + + public static final String CHECKPOINT_DIRECTORY_KEY = "channel.checkpoint.directory"; + + public static final String DEFAULT_CHECKPOINT_DIRECTORY = "/tmp"; + + private static String CHECKPOINT_DIRECTORY = null; + + private CheckpointUtils() { + } + + static String getCheckpointDirectory() { + + if (CHECKPOINT_DIRECTORY == null) { + CHECKPOINT_DIRECTORY = GlobalConfiguration + .getString(CHECKPOINT_DIRECTORY_KEY, DEFAULT_CHECKPOINT_DIRECTORY); + } + + return CHECKPOINT_DIRECTORY; + } + + public static boolean hasCompleteCheckpointAvailable(final ExecutionVertexID vertexID) { + + final File file = new File(getCheckpointDirectory() + File.separator + METADATA_PREFIX + "_" + vertexID + + "_final"); + if (file.exists()) { + return true; + } + + return false; + } + + public static boolean hasPartialCheckpointAvailable(final ExecutionVertexID vertexID) { + + File file = new File(getCheckpointDirectory() + File.separator + METADATA_PREFIX + "_" + vertexID + "_0"); + if (file.exists()) { + return true; + } + + file = new File(getCheckpointDirectory() + File.separator + METADATA_PREFIX + "_" + vertexID + "_part"); + if (file.exists()) { + return true; + } + + return false; + } + + /** + * Removes the checkpoint of the vertex with the given ID. All files contained in the checkpoint are deleted. + * + * @param vertexID + * the vertex whose checkpoint shall be removed + */ + public static void removeCheckpoint(final ExecutionVertexID vertexID) { + + final String checkpointDirectory = getCheckpointDirectory(); + + File file = new File(checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + + "_final"); + if (file.exists()) { + file.delete(); + return; + } + file = new File(checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_0"); + if (file.exists()) { + file.delete(); + } + + file = new File(checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_part"); + if (file.exists()) { + file.delete(); + } + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 30ea85f0ebf1d..602338bf6d80c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -26,6 +26,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.taskmanager.transferenvelope.CheckpointSerializer; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; @@ -52,7 +53,7 @@ * * @author warneke */ -public class EphemeralCheckpoint { +public class EphemeralCheckpoint implements OutputChannelForwarder { /** * The log object used to report problems. @@ -152,52 +153,6 @@ public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { } } - /** - * Adds a transfer envelope to the checkpoint. - * - * @param transferEnvelope - * the transfer envelope to be added - * @throws IOException - * thrown when an I/O error occurs while writing the envelope to disk - */ - public void addTransferEnvelope(TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - - if (this.checkpointingDecision == CheckpointingDecisionState.NO_CHECKPOINTING) { - final Buffer buffer = transferEnvelope.getBuffer(); - if (buffer != null) { - buffer.recycleBuffer(); - } - - return; - } - - if (this.checkpointingDecision == CheckpointingDecisionState.UNDECIDED) { - this.queuedEnvelopes.add(transferEnvelope); - return; - } - - writeTransferEnvelope(transferEnvelope); - } - - /** - * Returns whether the checkpoint is persistent. - * - * @return true if the checkpoint is persistent, false otherwise - */ - public boolean isPersistent() { - - return (this.checkpointingDecision == CheckpointingDecisionState.CHECKPOINTING); - } - - public boolean isDecided() { - return this.checkpointingDecision != CheckpointingDecisionState.UNDECIDED; - } - - public boolean isDiscarded() { - - return this.checkpointingDecision == CheckpointingDecisionState.NO_CHECKPOINTING; - } - private void destroy() { while (!this.queuedEnvelopes.isEmpty()) { @@ -220,11 +175,11 @@ private void write() throws IOException, InterruptedException { private boolean renameCheckpointPart(final String checkpointDir) { final File oldFile = new File(checkpointDir + File.separator - + CheckpointReplayManager.METADATA_PREFIX + "_" + + CheckpointUtils.METADATA_PREFIX + "_" + this.task.getVertexID() + "_part"); final File newFile = new File(checkpointDir + File.separator - + CheckpointReplayManager.METADATA_PREFIX + "_" + + CheckpointUtils.METADATA_PREFIX + "_" + this.task.getVertexID() + "_" + this.metaDataSuffix); if (!oldFile.renameTo(newFile)) { @@ -239,8 +194,8 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro InterruptedException { final String checkpointDir = GlobalConfiguration.getString( - CheckpointReplayManager.CHECKPOINT_DIRECTORY_KEY, - CheckpointReplayManager.DEFAULT_CHECKPOINT_DIRECTORY); + CheckpointUtils.CHECKPOINT_DIRECTORY_KEY, + CheckpointUtils.DEFAULT_CHECKPOINT_DIRECTORY); final Buffer buffer = transferEnvelope.getBuffer(); if (buffer != null) { @@ -276,7 +231,7 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro LOG.debug("Writing checkpointing meta data to directory " + checkpointDir); } final FileOutputStream fos = new FileOutputStream(checkpointDir + File.separator - + CheckpointReplayManager.METADATA_PREFIX + + CheckpointUtils.METADATA_PREFIX + "_" + this.task.getVertexID() + "_part"); this.metaDataFileChannel = fos.getChannel(); } @@ -308,7 +263,7 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro renameCheckpointPart(checkpointDir); } - new FileOutputStream(checkpointDir + File.separator + CheckpointReplayManager.METADATA_PREFIX + "_" + new FileOutputStream(checkpointDir + File.separator + CheckpointUtils.METADATA_PREFIX + "_" + this.task.getVertexID() + "_final").close(); // Since it is unclear whether the underlying physical file will ever be read, we force to close it. @@ -352,4 +307,43 @@ public void checkAsynchronousCheckpointDecision() throws IOException, Interrupte this.checkpointingDecision = this.asynchronousCheckpointingDecision; } + + /** + * {@inheritDoc} + */ + @Override + public boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + + if (this.checkpointingDecision == CheckpointingDecisionState.NO_CHECKPOINTING) { + return true; + } + + final TransferEnvelope dup = transferEnvelope.duplicate(); + + if (this.checkpointingDecision == CheckpointingDecisionState.UNDECIDED) { + this.queuedEnvelopes.add(dup); + } else { + writeTransferEnvelope(dup); + } + + return true; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasDataLeft() { + + return false; + } + + /** + * {@inheritDoc} + */ + @Override + public void processEvent(final AbstractEvent event) { + // TODO Auto-generated method stub + + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayFinishedNotifier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayFinishedNotifier.java deleted file mode 100644 index b7ef56da97034..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayFinishedNotifier.java +++ /dev/null @@ -1,36 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.checkpointing; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; - -/** - * This notifier will be called by a {@link CheckpointReplayTask} after the replay of a checkpoint has been finished, - * either because all data has been replayed, the same checkpoint shall be replayed by another - * {@link CheckpointReplayTask} object, or an error occurred. - * - * @author warneke - */ -public interface ReplayFinishedNotifier { - - /** - * Indicates the {@link CheckpointReplayTask} for the task represented by the given vertex ID has finished. - * - * @param vertexID - * the ID identifying the {@link CheckpointReplayTask} that has finished - */ - public void replayFinished(ExecutionVertexID vertexID); -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java new file mode 100644 index 0000000000000..22c40c32062bb --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java @@ -0,0 +1,137 @@ +package eu.stratosphere.nephele.checkpointing; + +import java.io.IOException; + +import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.taskmanager.bytebuffered.InputChannelContext; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +final class ReplayInputChannelContext implements InputChannelContext { + + private final InputChannelContext encapsulatedContext; + + ReplayInputChannelContext(final InputChannelContext encapsulatedContext) { + this.encapsulatedContext = encapsulatedContext; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isInputChannel() { + + return true; + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + + return this.encapsulatedContext.getJobID(); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelID getChannelID() { + + return this.encapsulatedContext.getChannelID(); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelID getConnectedChannelID() { + + return this.encapsulatedContext.getConnectedChannelID(); + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelType getType() { + + return this.encapsulatedContext.getType(); + } + + /** + * {@inheritDoc} + */ + @Override + public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { + + this.encapsulatedContext.queueTransferEnvelope(transferEnvelope); + } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException { + + return this.encapsulatedContext.requestEmptyBuffer(minimumSizeOfBuffer); + } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException { + + return this.encapsulatedContext.requestEmptyBufferBlocking(minimumSizeOfBuffer); + } + + /** + * {@inheritDoc} + */ + @Override + public int getMaximumBufferSize() { + + return this.encapsulatedContext.getMaximumBufferSize(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isShared() { + + return this.encapsulatedContext.isShared(); + } + + /** + * {@inheritDoc} + */ + @Override + public void reportAsynchronousEvent() { + + this.encapsulatedContext.reportAsynchronousEvent(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfQueuedEnvelopes() { + + return this.encapsulatedContext.getNumberOfQueuedEnvelopes(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfQueuedMemoryBuffers() { + + return this.encapsulatedContext.getNumberOfQueuedMemoryBuffers(); + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java new file mode 100644 index 0000000000000..d163b9b6f7172 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java @@ -0,0 +1,33 @@ +package eu.stratosphere.nephele.checkpointing; + +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; +import eu.stratosphere.nephele.taskmanager.bytebuffered.InputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext; + +final class ReplayInputGateContext extends AbstractReplayGateContext implements InputGateContext { + + ReplayInputGateContext(final GateID gateID) { + super(gateID); + } + + /** + * {@inheritDoc} + */ + @Override + public InputChannelContext createInputChannelContext(ChannelID channelID, InputChannelContext previousContext) { + + return new ReplayInputChannelContext(previousContext); + } + + /** + * {@inheritDoc} + */ + @Override + public LocalBufferPoolOwner getLocalBufferPoolOwner() { + // TODO Auto-generated method stub + return null; + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java new file mode 100644 index 0000000000000..837bf523f08dd --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java @@ -0,0 +1,63 @@ +package eu.stratosphere.nephele.checkpointing; + +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +public final class ReplayOutputChannelContext implements OutputChannelContext { + + private final ChannelID channelID; + + private final OutputChannelContext encapsulatedContext; + + ReplayOutputChannelContext(final ChannelID channelID, final OutputChannelContext encapsulatedContext) { + + this.channelID = channelID; + this.encapsulatedContext = encapsulatedContext; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isInputChannel() { + // TODO Auto-generated method stub + return true; + } + + @Override + public JobID getJobID() { + // TODO Auto-generated method stub + return null; + } + + /** + * {@inheritDoc} + */ + @Override + public ChannelID getChannelID() { + + return this.channelID; + } + + @Override + public ChannelID getConnectedChannelID() { + // TODO Auto-generated method stub + return null; + } + + @Override + public ChannelType getType() { + // TODO Auto-generated method stub + return null; + } + + @Override + public void queueTransferEnvelope(TransferEnvelope transferEnvelope) { + // TODO Auto-generated method stub + + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java new file mode 100644 index 0000000000000..d40b1619e8eb6 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java @@ -0,0 +1,36 @@ +package eu.stratosphere.nephele.checkpointing; + +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +final class ReplayOutputGateContext extends AbstractReplayGateContext implements OutputGateContext { + + ReplayOutputGateContext(final GateID gateID) { + super(gateID); + } + + @Override + public OutputChannelContext createOutputChannelContext(ChannelID channelID, OutputChannelContext previousContext, + boolean isReceiverRunning, boolean mergeSpillBuffers) { + + if (previousContext != null) { + activateForwardingBarrier(previousContext); + } + + return new ReplayOutputChannelContext(channelID, previousContext); + } + + private static void activateForwardingBarrier(final OutputChannelContext previousContext) { + + final TransferEnvelope transferEnvelope = new TransferEnvelope(0, previousContext.getJobID(), + previousContext.getConnectedChannelID()); + + transferEnvelope.addEvent(new UnexpectedEnvelopeEvent(Integer.MAX_VALUE)); + previousContext.queueTransferEnvelope(transferEnvelope); + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java new file mode 100644 index 0000000000000..30f3ffcb08913 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -0,0 +1,181 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; + +import java.util.Map; + +import eu.stratosphere.nephele.configuration.Configuration; +import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.profiling.TaskManagerProfiler; +import eu.stratosphere.nephele.services.iomanager.IOManager; +import eu.stratosphere.nephele.services.memorymanager.MemoryManager; +import eu.stratosphere.nephele.taskmanager.Task; +import eu.stratosphere.nephele.taskmanager.TaskManager; +import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; +import eu.stratosphere.nephele.template.InputSplitProvider; + +public final class ReplayTask implements Task { + + private final ExecutionVertexID vertexID; + + private final CheckpointEnvironment environment; + + private final Task encapsulatedTask; + + public ReplayTask(final ExecutionVertexID vertexID, final Environment environment, + final TaskManager taskManager) { + + this.vertexID = vertexID; + this.environment = CheckpointEnvironment.createFromEnvironment(environment); + + this.encapsulatedTask = null; + } + + public ReplayTask(final Task encapsulatedTask) { + + this.vertexID = encapsulatedTask.getVertexID(); + this.environment = CheckpointEnvironment.createFromEnvironment(encapsulatedTask.getEnvironment()); + + this.encapsulatedTask = encapsulatedTask; + } + + /** + * {@inheritDoc} + */ + @Override + public JobID getJobID() { + + return this.environment.getJobID(); + } + + /** + * {@inheritDoc} + */ + @Override + public ExecutionVertexID getVertexID() { + + return this.vertexID; + } + + /** + * {@inheritDoc} + */ + @Override + public Environment getEnvironment() { + + return this.environment; + } + + @Override + public void markAsFailed() { + // TODO Auto-generated method stub + + } + + @Override + public boolean isTerminated() { + // TODO Auto-generated method stub + return false; + } + + @Override + public void startExecution() { + // TODO Auto-generated method stub + + System.out.println("Checkpoint replay task started"); + } + + @Override + public void cancelExecution() { + // TODO Auto-generated method stub + + } + + @Override + public void killExecution() { + // TODO Auto-generated method stub + + } + + /** + * {@inheritDoc} + */ + @Override + public void registerMemoryManager(final MemoryManager memoryManager) { + // Nothing to do here + } + + /** + * {@inheritDoc} + */ + @Override + public void registerIOManager(final IOManager ioManager) { + // Nothing to do here + } + + /** + * {@inheritDoc} + */ + @Override + public void registerInputSplitProvider(final InputSplitProvider inputSplitProvider) { + // Nothing to do here + } + + /** + * {@inheritDoc} + */ + @Override + public void registerProfiler(final TaskManagerProfiler taskManagerProfiler, final Configuration jobConfiguration) { + // Nothing to do here + } + + /** + * {@inheritDoc} + */ + @Override + public void unregisterMemoryManager(final MemoryManager memoryManager) { + + if (this.encapsulatedTask != null) { + this.encapsulatedTask.unregisterMemoryManager(memoryManager); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) { + + if (this.encapsulatedTask != null) { + this.encapsulatedTask.unregisterProfiler(taskManagerProfiler); + } + } + + /** + * {@inheritDoc} + */ + @Override + public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + final Map tasksWithUndecidedCheckpoints) { + + return new ReplayTaskContext(); + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java new file mode 100644 index 0000000000000..b159afce08e51 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java @@ -0,0 +1,35 @@ +package eu.stratosphere.nephele.checkpointing; + +import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; +import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; + +final class ReplayTaskContext implements TaskContext { + + /** + * {@inheritDoc} + */ + @Override + public OutputGateContext createOutputGateContext(final GateID gateID) { + + return new ReplayOutputGateContext(gateID); + } + + @Override + public InputGateContext createInputGateContext(final GateID gateID) { + // TODO Auto-generated method stub + return null; + } + + /** + * {@inheritDoc} + */ + @Override + public LocalBufferPoolOwner getLocalBufferPoolOwner() { + + return null; + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 386c7e7d812b3..76472bdfd9bb6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -17,6 +17,7 @@ import java.io.IOException; import java.util.Iterator; +import java.util.List; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; @@ -46,9 +47,11 @@ import eu.stratosphere.nephele.taskmanager.TaskCancelResult; import eu.stratosphere.nephele.taskmanager.TaskKillResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult; +import eu.stratosphere.nephele.taskmanager.TaskSubmissionWrapper; import eu.stratosphere.nephele.template.AbstractInvokable; import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.util.AtomicEnum; +import eu.stratosphere.nephele.util.SerializableArrayList; import eu.stratosphere.nephele.util.SerializableHashSet; /** @@ -582,10 +585,16 @@ public TaskSubmissionResult startTask() { final SerializableHashSet activeOutputChannels = constructInitialActiveOutputChannelsSet(); + final List tasks = new SerializableArrayList(); + final TaskSubmissionWrapper tsw = new TaskSubmissionWrapper(this.vertexID, this.environment, + this.executionGraph.getJobConfiguration(), activeOutputChannels); + tasks.add(tsw); + try { - return this.allocatedResource.getInstance().submitTask(this.vertexID, - this.executionGraph.getJobConfiguration(), this.environment, - activeOutputChannels); + final List results = this.allocatedResource.getInstance().submitTasks(tasks); + + return results.get(0); + } catch (IOException e) { final TaskSubmissionResult result = new TaskSubmissionResult(getID(), AbstractTaskResult.ReturnCode.ERROR); result.setDescription(StringUtils.stringifyException(e)); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index 57e80440d0ae7..722f6c7428703 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -21,9 +21,6 @@ import java.util.Set; import eu.stratosphere.nephele.checkpointing.CheckpointDecision; -import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.Environment; -import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse; @@ -199,29 +196,6 @@ public synchronized void checkLibraryAvailability(final JobID jobID) throws IOEx } } - /** - * Submits the task represented by the given {@link Environment} object to the instance's - * {@link eu.stratosphere.nephele.taskmanager.TaskManager}. - * - * @param id - * the ID of the vertex to be submitted - * @param jobConfiguration - * the configuration of the overall job - * @param environment - * the environment encapsulating the task - * @param activeOutputChannels - * the set of initially active output channels - * @return the result of the submission attempt - * @throws IOException - * thrown if an error occurs while transmitting the task - */ - public synchronized TaskSubmissionResult submitTask(final ExecutionVertexID id, - final Configuration jobConfiguration, final RuntimeEnvironment environment, - final Set activeOutputChannels) throws IOException { - - return getTaskManager().submitTask(id, jobConfiguration, environment, activeOutputChannels); - } - /** * Submits a list of tasks to the instance's {@link eu.stratosphere.nephele.taskmanager.TaskManager}. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index ed460c6b5fb76..2654c20cae1c0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -20,8 +20,6 @@ import java.util.Set; import eu.stratosphere.nephele.checkpointing.CheckpointDecision; -import eu.stratosphere.nephele.configuration.Configuration; -import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate; @@ -42,25 +40,6 @@ */ public interface TaskOperationProtocol extends VersionedProtocol { - /** - * Submits a task to the task manager. - * - * @param id - * the ID of the corresponding execution vertex - * @param jobConfiguration - * the job configuration that has been attached to the original job graph - * @param re - * the runtime environment containing the task - * @param activeOutputChannels - * the set of initially active output channels - * @return the result of the task submission - * @throws IOException - * thrown if an error occurs during this remote procedure call - */ - TaskSubmissionResult submitTask(ExecutionVertexID id, Configuration jobConfiguration, RuntimeEnvironment re, - Set activeOutputChannels) - throws IOException; - /** * Submits a list of tasks to the task manager. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/CheckpointReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/CheckpointReplayTask.java deleted file mode 100644 index 97d967962bf9f..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/CheckpointReplayTask.java +++ /dev/null @@ -1,5 +0,0 @@ -package eu.stratosphere.nephele.taskmanager; - -public class CheckpointReplayTask { - -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 36b318989aa36..f920c9c174894 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -39,7 +39,8 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.checkpointing.CheckpointDecision; -import eu.stratosphere.nephele.checkpointing.CheckpointReplayManager; +import eu.stratosphere.nephele.checkpointing.ReplayTask; +import eu.stratosphere.nephele.checkpointing.CheckpointUtils; import eu.stratosphere.nephele.configuration.ConfigConstants; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.configuration.GlobalConfiguration; @@ -125,12 +126,6 @@ public class TaskManager implements TaskOperationProtocol, PluginCommunicationPr */ private final ByteBufferedChannelManager byteBufferedChannelManager; - /** - * The instance of the {@link CheckpointReplayManager} to restore - * previously written checkpoints. - */ - private final CheckpointReplayManager checkpointManager; - /** * Instance of the task manager profile if profiling is enabled. */ @@ -292,9 +287,6 @@ public TaskManager(String configDir) throws Exception { } this.byteBufferedChannelManager = byteBufferedChannelManager; - // Initialize the checkpoint manager - this.checkpointManager = new CheckpointReplayManager(this.byteBufferedChannelManager); - // Determine hardware description HardwareDescription hardware = HardwareDescriptionFactory.extractFromSystem(); if (hardware == null) { @@ -472,33 +464,6 @@ public void run() { return new TaskKillResult(id, AbstractTaskResult.ReturnCode.SUCCESS); } - /** - * {@inheritDoc} - */ - @Override - public TaskSubmissionResult submitTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final RuntimeEnvironment re, final Set activeOutputChannels) - throws IOException { - - // Create task object and register it with the environment - final RuntimeTask task = new RuntimeTask(id, re, this); - re.setExecutionObserver(task); - - // Register the task - TaskSubmissionResult result = registerTask(id, jobConfiguration, task, activeOutputChannels); - if (result != null) { // If result is non-null, an error occurred during task registration - return result; - } - - // Start execution - if (LOG.isDebugEnabled()) { - LOG.debug("Starting execution of task with ID " + id); - } - task.startExecution(); - - return new TaskSubmissionResult(id, AbstractTaskResult.ReturnCode.SUCCESS); - } - /** * {@inheritDoc} */ @@ -516,13 +481,12 @@ public List submitTasks(final List final Configuration jobConfiguration = tsw.getConfiguration(); final Set activeOutputChannels = tsw.getActiveOutputChannels(); - // Create task object and register it with the environment - final RuntimeTask task = new RuntimeTask(id, re, this); - re.setExecutionObserver(task); - // Register the task - TaskSubmissionResult result = registerTask(id, jobConfiguration, task, activeOutputChannels); - if (result != null) { // If result is non-null, an error occurred during task registration + final Task task = createAndRegisterTask(id, jobConfiguration, re, activeOutputChannels); + if (task == null) { + final TaskSubmissionResult result = new TaskSubmissionResult(id, AbstractTaskResult.ReturnCode.ERROR); + result.setDescription("Task with ID " + id + " was already running"); + LOG.error(result.getDescription()); submissionResultList.add(result); } else { submissionResultList.add(new TaskSubmissionResult(id, AbstractTaskResult.ReturnCode.SUCCESS)); @@ -546,67 +510,91 @@ public List submitTasks(final List * the ID of the task to register * @param jobConfiguration * the job configuration that has been attached to the original job graph - * @param task - * the task to register + * @param environment + * the environment of the task to be registered * @param activeOutputChannels * the set of initially active output channels - * @return null if the registration has been successful or a {@link TaskSubmissionResult} containing - * the error that occurred + * @return the task to be started or null if a task with the same ID was already running */ - private TaskSubmissionResult registerTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final Task task, final Set activeOutputChannels) { + private Task createAndRegisterTask(final ExecutionVertexID id, final Configuration jobConfiguration, + final RuntimeEnvironment environment, final Set activeOutputChannels) { if (id == null) { throw new IllegalArgumentException("Argument id is null"); } - if (task == null) { - throw new IllegalArgumentException("Argument task is null"); + if (environment == null) { + throw new IllegalArgumentException("Argument environment is null"); } - // Task registration must be atomic + // Task creation and registration must be atomic + Task task = null; + synchronized (this) { final Task runningTask = this.runningTasks.get(id); - if (runningTask != null) { - LOG.error("Task with ID " + id + " is already running"); - final TaskSubmissionResult result = new TaskSubmissionResult(id, AbstractTaskResult.ReturnCode.ERROR); - result.setDescription("Task with ID " + id + " is already running"); - return result; + boolean registerTask = true; + if (runningTask == null) { + + // Is there a complete checkpoint for this task + if (CheckpointUtils.hasCompleteCheckpointAvailable(id)) { + task = new ReplayTask(id, environment, this); + } else { + task = new RuntimeTask(id, environment, this); + } + } else { + + if (runningTask instanceof RuntimeTask) { + + // Check if there at least a partial checkpoint available + if (CheckpointUtils.hasPartialCheckpointAvailable(id)) { + task = new ReplayTask((RuntimeTask) runningTask); + } else { + // Task is already running + return null; + } + } else { + // There is already a replay task running, we will simply restart it + registerTask = false; + } + } final Environment ee = task.getEnvironment(); - // Register task manager components with the task - task.registerMemoryManager(this.memoryManager); - task.registerIOManager(this.ioManager); - task.registerInputSplitProvider(new TaskInputSplitProvider(ee.getJobID(), id, this.globalInputSplitProvider)); + if (registerTask) { + // Register task manager components with the task + task.registerMemoryManager(this.memoryManager); + task.registerIOManager(this.ioManager); + task.registerInputSplitProvider(new TaskInputSplitProvider(ee.getJobID(), id, + this.globalInputSplitProvider)); - // Register the task with the byte buffered channel manager - this.byteBufferedChannelManager.register(task, activeOutputChannels); + // Register the task with the byte buffered channel manager + this.byteBufferedChannelManager.register(task, activeOutputChannels); - boolean enableProfiling = false; - if (this.profiler != null && jobConfiguration.getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) { - enableProfiling = true; - } + boolean enableProfiling = false; + if (this.profiler != null && jobConfiguration.getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) { + enableProfiling = true; + } - // Register environment, input, and output gates for profiling - if (enableProfiling) { - task.registerProfiler(this.profiler, jobConfiguration); - } + // Register environment, input, and output gates for profiling + if (enableProfiling) { + task.registerProfiler(this.profiler, jobConfiguration); + } - // Allow plugins to register their listeners for this task - if (!this.taskManagerPlugins.isEmpty()) { - final Iterator it = this.taskManagerPlugins.values().iterator(); - while (it.hasNext()) { - it.next().registerTask(id, jobConfiguration, ee); + // Allow plugins to register their listeners for this task + if (!this.taskManagerPlugins.isEmpty()) { + final Iterator it = this.taskManagerPlugins.values().iterator(); + while (it.hasNext()) { + it.next().registerTask(id, jobConfiguration, ee); + } } - } - this.runningTasks.put(id, task); + this.runningTasks.put(id, task); + } } - return null; + return task; } /** @@ -849,7 +837,7 @@ public void removeCheckpoints(List listOfVertexIDs) throws IO final Iterator it = listOfVertexIDs.iterator(); while (it.hasNext()) { - this.checkpointManager.removeCheckpoint(it.next()); + CheckpointUtils.removeCheckpoint(it.next()); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java new file mode 100644 index 0000000000000..86e4434fb475a --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java @@ -0,0 +1,40 @@ +package eu.stratosphere.nephele.taskmanager.bytebuffered; + +import java.util.Iterator; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +public abstract class AbstractOutputChannelContext implements OutputChannelContext { + + /** + * Stores incoming events for this output channel. + */ + private final IncomingEventQueue incomingEventQueue; + + public AbstractOutputChannelContext(final IncomingEventQueue incomingEventQueue) { + + this.incomingEventQueue = incomingEventQueue; + } + + public static IncomingEventQueue createIncomingEventQueue(final OutputChannelForwardingChain forwardingChain) { + + return new IncomingEventQueue(forwardingChain); + } + + /** + * {@inheritDoc} + */ + @Override + public final void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { + + if (transferEnvelope.getBuffer() != null) { + throw new IllegalStateException("Transfer envelope for output channel has buffer attached"); + } + + final Iterator it = transferEnvelope.getEventList().iterator(); + while (it.hasNext()) { + this.incomingEventQueue.offer(it.next()); + } + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingEventQueue.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingEventQueue.java new file mode 100644 index 0000000000000..c8bbb4093c32b --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingEventQueue.java @@ -0,0 +1,31 @@ +package eu.stratosphere.nephele.taskmanager.bytebuffered; + +import java.util.Queue; +import java.util.concurrent.LinkedBlockingDeque; + +import eu.stratosphere.nephele.event.task.AbstractEvent; + +public final class IncomingEventQueue { + + private final Queue incomingEventQueue = new LinkedBlockingDeque(); + + private final OutputChannelForwardingChain forwardingChain; + + IncomingEventQueue(final OutputChannelForwardingChain forwardingChain) { + this.forwardingChain = forwardingChain; + } + + public void processQueuedEvents() { + + AbstractEvent event = this.incomingEventQueue.poll(); + while (event != null) { + + this.forwardingChain.processEvent(event); + event = this.incomingEventQueue.poll(); + } + } + + void offer(final AbstractEvent event) { + this.incomingEventQueue.offer(event); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java new file mode 100644 index 0000000000000..41a75128d3020 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java @@ -0,0 +1,15 @@ +package eu.stratosphere.nephele.taskmanager.bytebuffered; + +import java.io.IOException; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +public interface OutputChannelForwarder { + + boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException; + + boolean hasDataLeft(); + + void processEvent(AbstractEvent event); +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java new file mode 100644 index 0000000000000..a8422498f8c11 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java @@ -0,0 +1,77 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.bytebuffered; + +import java.io.IOException; +import java.util.Iterator; +import java.util.concurrent.CopyOnWriteArrayList; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +public final class OutputChannelForwardingChain { + + private final CopyOnWriteArrayList forwardingChain = new CopyOnWriteArrayList(); + + public void addForwarder(final OutputChannelForwarder forwarder) { + + this.forwardingChain.add(forwarder); + } + + public void forwardEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + + final Iterator it = this.forwardingChain.iterator(); + while (it.hasNext()) { + + if (!it.next().forward(transferEnvelope)) { + recycleEnvelope(transferEnvelope); + break; + } + + } + } + + public void processEvent(final AbstractEvent event) { + + final Iterator it = this.forwardingChain.iterator(); + while (it.hasNext()) { + it.next().processEvent(event); + } + } + + public boolean anyForwarderHasDataLeft() { + + final Iterator it = this.forwardingChain.iterator(); + while (it.hasNext()) { + + if (it.next().hasDataLeft()) { + return true; + } + } + + return false; + } + + private void recycleEnvelope(final TransferEnvelope transferEnvelope) { + + final Buffer buffer = transferEnvelope.getBuffer(); + if (buffer != null) { + buffer.recycleBuffer(); + } + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java new file mode 100644 index 0000000000000..5212dda1cb557 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java @@ -0,0 +1,71 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.runtime; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; +import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +public final class ForwardingBarrier implements OutputChannelForwarder { + + private static final Log LOG = LogFactory.getLog(ForwardingBarrier.class); + + private final ChannelID outputChannelID; + + private int forwardingBarrier = -1; + + ForwardingBarrier(final ChannelID outputChannelID) { + this.outputChannelID = outputChannelID; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean forward(final TransferEnvelope transferEnvelope) { + + if (transferEnvelope.getSequenceNumber() < this.forwardingBarrier) { + return false; + } + + return true; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasDataLeft() { + + return false; + } + + @Override + public void processEvent(final AbstractEvent event) { + + if (event instanceof UnexpectedEnvelopeEvent) { + + final UnexpectedEnvelopeEvent uee = (UnexpectedEnvelopeEvent) event; + this.forwardingBarrier = uee.getExpectedSequenceNumber(); + LOG.info("Setting forwarding barrier to sequence number " + this.forwardingBarrier + " for output channel " + this.outputChannelID); + } + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java new file mode 100644 index 0000000000000..4428225b99392 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java @@ -0,0 +1,43 @@ +package eu.stratosphere.nephele.taskmanager.runtime; + +import java.io.IOException; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; + +public final class RuntimeDispatcher implements OutputChannelForwarder { + + private final TransferEnvelopeDispatcher dispatcher; + + RuntimeDispatcher(final TransferEnvelopeDispatcher dispatcher) { + + this.dispatcher = dispatcher; + } + + @Override + public boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + + dispatcher.processEnvelopeFromOutputChannel(transferEnvelope); + + return true; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasDataLeft() { + + return false; + } + + /** + * {@inheritDoc} + */ + @Override + public void processEvent(final AbstractEvent event) { + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java new file mode 100644 index 0000000000000..25b8091677fad --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java @@ -0,0 +1,216 @@ +package eu.stratosphere.nephele.taskmanager.runtime; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.BufferPairResponse; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedOutputChannelBroker; +import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; +import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +final class RuntimeOutputChannelBroker implements ByteBufferedOutputChannelBroker, OutputChannelForwarder { + + /** + * The static object used for logging. + */ + private static final Log LOG = LogFactory.getLog(RuntimeOutputChannelBroker.class); + + /** + * The byte buffered output channel this context belongs to. + */ + private final AbstractByteBufferedOutputChannel byteBufferedOutputChannel; + + /** + * Reference to the queue with the incoming events. + */ + private final IncomingEventQueue incomingEventQueue; + + /** + * The buffer provider this channel broker to obtain buffers from. + */ + private final BufferProvider bufferProvider; + + /** + * The output channel forwarder which will take care of the produced transfer envelopes. + */ + private final OutputChannelForwardingChain forwarder; + + /** + * Points to the {@link TransferEnvelope} object that will be passed to the framework upon + * the next releaseWriteBuffers call. + */ + private TransferEnvelope outgoingTransferEnvelope = null; + + /** + * Stores whether the receiver has acknowledged the close request from this channel. + */ + private boolean closeAcknowledgementReceived = false; + + /** + * The sequence number for the next {@link TransferEnvelope} to be created. + */ + private int sequenceNumber = 0; + + RuntimeOutputChannelBroker(final BufferProvider bufferProvider, final OutputChannelForwardingChain forwarder, + final IncomingEventQueue incomingEventQueue, + final AbstractByteBufferedOutputChannel byteBufferedOutputChannel) { + + this.bufferProvider = bufferProvider; + this.forwarder = forwarder; + this.incomingEventQueue = incomingEventQueue; + this.byteBufferedOutputChannel = byteBufferedOutputChannel; + this.byteBufferedOutputChannel.setByteBufferedOutputChannelBroker(this); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean forward(final TransferEnvelope transferEnvelope) { + + // Nothing to do here + + return true; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasDataLeft() { + + return (!this.closeAcknowledgementReceived); + } + + /** + * {@inheritDoc} + */ + @Override + public void processEvent(final AbstractEvent event) { + + if (event instanceof ByteBufferedChannelCloseEvent) { + this.closeAcknowledgementReceived = true; + } else { + this.byteBufferedOutputChannel.processEvent(event); + } + } + + @Override + public BufferPairResponse requestEmptyWriteBuffers() throws InterruptedException, IOException { + + if (this.outgoingTransferEnvelope == null) { + this.outgoingTransferEnvelope = createNewOutgoingTransferEnvelope(); + } else { + if (this.outgoingTransferEnvelope.getBuffer() != null) { + LOG.error("Channel " + this.byteBufferedOutputChannel.getID() + + "'s transfer envelope already has a buffer attached"); + return null; + } + } + + final int uncompressedBufferSize = calculateBufferSize(); + + // TODO: This implementation breaks compression, we have to fix it later + final Buffer buffer = this.bufferProvider.requestEmptyBufferBlocking(uncompressedBufferSize); + final BufferPairResponse bufferResponse = new BufferPairResponse(null, buffer); + + // Put the buffer into the transfer envelope + this.outgoingTransferEnvelope.setBuffer(bufferResponse.getUncompressedDataBuffer()); + + return bufferResponse; + } + + /** + * Creates a new {@link TransferEnvelope} object. The method assigns + * and increases the sequence number. Moreover, it will look up the list of receivers for this transfer envelope. + * This method will block until the lookup is completed. + * + * @return a new {@link TransferEnvelope} object containing the correct sequence number and receiver list + */ + private TransferEnvelope createNewOutgoingTransferEnvelope() { + + final TransferEnvelope transferEnvelope = new TransferEnvelope(this.sequenceNumber++, + this.byteBufferedOutputChannel.getJobID(), + this.byteBufferedOutputChannel.getID()); + + return transferEnvelope; + } + + /** + * Calculates the recommended size of the next buffer to be + * handed to the attached channel object in bytes. + * + * @return the recommended size of the next buffer in bytes + */ + private int calculateBufferSize() { + + // TODO: Include latency considerations + return this.bufferProvider.getMaximumBufferSize(); + } + + /** + * {@inheritDoc} + */ + @Override + public void releaseWriteBuffers() throws IOException, InterruptedException { + + // Check for events + this.incomingEventQueue.processQueuedEvents(); + + if (this.outgoingTransferEnvelope == null) { + LOG.error("Cannot find transfer envelope for channel with ID " + this.byteBufferedOutputChannel.getID()); + return; + } + + // Consistency check + if (this.outgoingTransferEnvelope.getBuffer() == null) { + LOG.error("Channel " + this.byteBufferedOutputChannel.getID() + " has no buffer attached"); + return; + } + + // Finish the write phase of the buffer + final Buffer buffer = this.outgoingTransferEnvelope.getBuffer(); + buffer.finishWritePhase(); + + this.forwarder.forwardEnvelope(this.outgoingTransferEnvelope); + this.outgoingTransferEnvelope = null; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasDataLeftToTransmit() throws IOException, InterruptedException { + + // Check for events + this.incomingEventQueue.processQueuedEvents(); + + return this.forwarder.anyForwarderHasDataLeft(); + } + + /** + * {@inheritDoc} + */ + @Override + public void transferEventToInputChannel(final AbstractEvent event) throws IOException, InterruptedException { + + if (this.outgoingTransferEnvelope != null) { + this.outgoingTransferEnvelope.addEvent(event); + } else { + + final TransferEnvelope ephemeralTransferEnvelope = createNewOutgoingTransferEnvelope(); + ephemeralTransferEnvelope.addEvent(event); + + this.forwarder.forwardEnvelope(ephemeralTransferEnvelope); + } + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java index 4f2e91964daf1..563d9ac640a11 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java @@ -15,224 +15,22 @@ package eu.stratosphere.nephele.taskmanager.runtime; -import java.io.IOException; -import java.util.Iterator; -import java.util.Queue; -import java.util.concurrent.LinkedBlockingDeque; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.event.task.AbstractEvent; -import eu.stratosphere.nephele.io.AbstractID; -import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.BufferPairResponse; -import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelActivateEvent; -import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; -import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedOutputChannelBroker; import eu.stratosphere.nephele.jobgraph.JobID; -import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; -import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; - -public final class RuntimeOutputChannelContext implements ByteBufferedOutputChannelBroker, OutputChannelContext { +import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; - /** - * The static object used for logging. - */ - private static final Log LOG = LogFactory.getLog(RuntimeOutputChannelContext.class); +public final class RuntimeOutputChannelContext extends AbstractOutputChannelContext { - /** - * The byte buffered output channel this context belongs to. - */ private final AbstractByteBufferedOutputChannel byteBufferedOutputChannel; - /** - * The output gate context associated with this context. - */ - private final RuntimeOutputGateContext outputGateContext; - - /** - * Points to the {@link TransferEnvelope} object that will be passed to the framework upon - * the next releaseWriteBuffers call. - */ - private TransferEnvelope outgoingTransferEnvelope = null; - - /** - * Indicates whether the receiver of an envelope is currently running. - */ - private boolean isReceiverRunning = false; - - /** - * Stores whether the receiver has acknowledged the close request from this channel. - */ - private boolean closeAcknowledgementReceived = false; - - /** - * Queue to store outgoing transfer envelope in case the receiver of the envelopes is not yet running. - */ - private final SpillingQueue queuedOutgoingEnvelopes; + RuntimeOutputChannelContext(final AbstractByteBufferedOutputChannel byteBufferedOutputChannel, + final IncomingEventQueue incomingEventQueue) { + super(incomingEventQueue); - /** - * Stores incoming events for this output channel. - */ - private final Queue incomingEventQueue = new LinkedBlockingDeque(); - - /** - * The sequence number for the next {@link TransferEnvelope} to be created. - */ - private int sequenceNumber = 0; - - /** - * Stores if the flushing the of spilling queue has already been triggered. - */ - private boolean spillingQueueAlreadyFlushed = false; - - RuntimeOutputChannelContext(final RuntimeOutputGateContext outputGateContext, - final AbstractByteBufferedOutputChannel byteBufferedOutputChannel, final boolean isReceiverRunning, - final boolean mergeSpilledBuffers) { - - this.outputGateContext = outputGateContext; this.byteBufferedOutputChannel = byteBufferedOutputChannel; - this.byteBufferedOutputChannel.setByteBufferedOutputChannelBroker(this); - this.isReceiverRunning = isReceiverRunning; - - final AbstractID fileOwnerID = mergeSpilledBuffers ? outputGateContext.getFileOwnerID() - : byteBufferedOutputChannel.getID(); - this.queuedOutgoingEnvelopes = new SpillingQueue(fileOwnerID, this.outputGateContext); - - // Register as inactive channel so queue can be spilled to disk when we run out of memory buffers - if (!isReceiverRunning) { - this.outputGateContext.registerInactiveOutputChannel(this); - } - } - - /** - * {@inheritDoc} - */ - @Override - public BufferPairResponse requestEmptyWriteBuffers() throws InterruptedException, IOException { - - if (this.outgoingTransferEnvelope == null) { - this.outgoingTransferEnvelope = createNewOutgoingTransferEnvelope(); - } else { - if (this.outgoingTransferEnvelope.getBuffer() != null) { - LOG.error("Channel " + this.byteBufferedOutputChannel.getID() - + "'s transfer envelope already has a buffer attached"); - return null; - } - } - - final int uncompressedBufferSize = calculateBufferSize(); - - // TODO: This implementation breaks compression, we have to fix it later - final Buffer buffer = this.outputGateContext.requestEmptyBufferBlocking(uncompressedBufferSize); - final BufferPairResponse bufferResponse = new BufferPairResponse(null, buffer); - - // Put the buffer into the transfer envelope - this.outgoingTransferEnvelope.setBuffer(bufferResponse.getUncompressedDataBuffer()); - - return bufferResponse; - } - - /** - * Creates a new {@link TransferEnvelope} object. The method assigns - * and increases the sequence number. Moreover, it will look up the list of receivers for this transfer envelope. - * This method will block until the lookup is completed. - * - * @return a new {@link TransferEnvelope} object containing the correct sequence number and receiver list - */ - private TransferEnvelope createNewOutgoingTransferEnvelope() { - - final TransferEnvelope transferEnvelope = new TransferEnvelope(this.sequenceNumber++, - this.byteBufferedOutputChannel.getJobID(), - this.byteBufferedOutputChannel.getID()); - - return transferEnvelope; - } - - /** - * {@inheritDoc} - */ - @Override - public void releaseWriteBuffers() throws IOException, InterruptedException { - - processIncomingEvents(); - - if (this.outgoingTransferEnvelope == null) { - LOG.error("Cannot find transfer envelope for channel with ID " + this.byteBufferedOutputChannel.getID()); - return; - } - - // Consistency check - if (this.outgoingTransferEnvelope.getBuffer() == null) { - LOG.error("Channel " + this.byteBufferedOutputChannel.getID() + " has no buffer attached"); - return; - } - - // Finish the write phase of the buffer - final Buffer buffer = this.outgoingTransferEnvelope.getBuffer(); - buffer.finishWritePhase(); - - // TODO: Add to checkpoint - - if (!this.isReceiverRunning) { - this.queuedOutgoingEnvelopes.add(this.outgoingTransferEnvelope); - } else { - - if (this.queuedOutgoingEnvelopes.isEmpty()) { - this.outputGateContext.processEnvelope(this, this.outgoingTransferEnvelope); - } else { - this.queuedOutgoingEnvelopes.add(this.outgoingTransferEnvelope); - flushQueuedOutgoingEnvelopes(); - } - } - - this.outgoingTransferEnvelope = null; - } - - /** - * {@inheritDoc} - */ - @Override - public void transferEventToInputChannel(final AbstractEvent event) throws IOException, InterruptedException { - - if (this.outgoingTransferEnvelope != null) { - this.outgoingTransferEnvelope.addEvent(event); - } else { - - final TransferEnvelope ephemeralTransferEnvelope = createNewOutgoingTransferEnvelope(); - ephemeralTransferEnvelope.addEvent(event); - - // TODO: Add to checkpoint - - if (!this.isReceiverRunning) { - this.queuedOutgoingEnvelopes.add(ephemeralTransferEnvelope); - } else { - - if (this.queuedOutgoingEnvelopes.isEmpty()) { - this.outputGateContext.processEnvelope(this, ephemeralTransferEnvelope); - } else { - this.queuedOutgoingEnvelopes.add(ephemeralTransferEnvelope); - flushQueuedOutgoingEnvelopes(); - } - } - } - } - - /** - * Calculates the recommended size of the next buffer to be - * handed to the attached channel object in bytes. - * - * @return the recommended size of the next buffer in bytes - */ - private int calculateBufferSize() { - - // TODO: Include latency considerations - return this.outputGateContext.getMaximumBufferSize(); } /** @@ -244,29 +42,6 @@ public boolean isInputChannel() { return false; } - /** - * Checks if this channel is active, that means the receiver of the channel's data is able to able to accept the - * data. - * - * @return true if the channel is active, false otherwise - */ - boolean isChannelActive() { - - return this.isReceiverRunning; - } - - /** - * Called by the framework to report events to - * the attached channel object. - * - * @param abstractEvent - * the event to be reported - */ - void processEvent(final AbstractEvent abstractEvent) { - - this.incomingEventQueue.offer(abstractEvent); - } - /** * {@inheritDoc} */ @@ -294,109 +69,6 @@ public JobID getJobID() { return this.byteBufferedOutputChannel.getJobID(); } - /** - * {@inheritDoc} - */ - @Override - public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { - - if (transferEnvelope.getBuffer() != null) { - LOG.error("Transfer envelope for output channel has buffer attached"); - } - - final Iterator it = transferEnvelope.getEventList().iterator(); - while (it.hasNext()) { - processEvent(it.next()); - } - } - - void flushQueuedOutgoingEnvelopes() throws IOException, InterruptedException { - - if (this.spillingQueueAlreadyFlushed) { - return; - } - - if (!this.queuedOutgoingEnvelopes.isEmpty()) { - - // TODO: Make this mechanisms smarter - this.queuedOutgoingEnvelopes.spillSynchronouslyIncludingHead(); - this.queuedOutgoingEnvelopes.printSpillingState(); - - if (!this.outputGateContext.registerSpillingQueueWithNetworkConnection( - this.byteBufferedOutputChannel.getID(), this.queuedOutgoingEnvelopes)) { - - // Direct connection, spill the queue but make sure we do not copy data back to main memory - this.queuedOutgoingEnvelopes.disableAsynchronousUnspilling(); - - while (!this.queuedOutgoingEnvelopes.isEmpty()) { - this.outputGateContext.processEnvelope(this, this.queuedOutgoingEnvelopes.poll()); - } - } - } - - this.spillingQueueAlreadyFlushed = true; - } - - /** - * {@inheritDoc} - */ - @Override - public boolean hasDataLeftToTransmit() throws IOException, InterruptedException { - - processIncomingEvents(); - - if (!this.isReceiverRunning) { - return true; - } - - flushQueuedOutgoingEnvelopes(); - - return (!this.closeAcknowledgementReceived); - } - - long getAmountOfMainMemoryInQueue() { - - return this.queuedOutgoingEnvelopes.getAmountOfMainMemoryInQueue(); - } - - /** - * Triggers the encapsulated output channel to flush and release its internal working buffers. - * - * @throws IOException - * thrown if an I/O error occurs while flushing the buffers - * @throws InterruptedException - * thrown if the thread is interrupted while waiting for the channel to flush - */ - void flush() throws IOException, InterruptedException { - - this.byteBufferedOutputChannel.flush(); - } - - long spillQueueWithOutgoingEnvelopes() throws IOException { - - return this.queuedOutgoingEnvelopes.spillSynchronouslyIncludingHead(); - } - - /** - * Processes all queues incoming events. - */ - private void processIncomingEvents() { - - AbstractEvent event = this.incomingEventQueue.poll(); - while (event != null) { - - if (event instanceof ByteBufferedChannelCloseEvent) { - this.closeAcknowledgementReceived = true; - } else if (event instanceof ByteBufferedChannelActivateEvent) { - this.isReceiverRunning = true; - } else { - this.byteBufferedOutputChannel.processEvent(event); - } - - event = this.incomingEventQueue.poll(); - } - } - /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java index 714e19bfcd1b5..a3cc0e4c79db0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java @@ -16,50 +16,33 @@ package eu.stratosphere.nephele.taskmanager.runtime; import java.io.IOException; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Set; -import eu.stratosphere.nephele.event.task.AbstractEvent; -import eu.stratosphere.nephele.event.task.EventList; +import eu.stratosphere.nephele.checkpointing.EphemeralCheckpoint; import eu.stratosphere.nephele.io.AbstractID; import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; -import eu.stratosphere.nephele.taskmanager.bufferprovider.AsynchronousEventListener; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; +import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; -import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.types.Record; -final class RuntimeOutputGateContext implements AsynchronousEventListener, BufferProvider, OutputGateContext { +final class RuntimeOutputGateContext implements BufferProvider, OutputGateContext { private final RuntimeTaskContext taskContext; private final OutputGate outputGate; - private final Set inactiveOutputChannels; - RuntimeOutputGateContext(final RuntimeTaskContext taskContext, final OutputGate outputGate) { this.taskContext = taskContext; this.outputGate = outputGate; - - this.inactiveOutputChannels = new HashSet(); - - this.taskContext.registerAsynchronousEventListener(this.outputGate.getIndex(), this); - } - - void registerInactiveOutputChannel(final RuntimeOutputChannelContext outputChannelContext) { - - this.inactiveOutputChannels.add(outputChannelContext); } AbstractID getFileOwnerID() { @@ -67,47 +50,6 @@ AbstractID getFileOwnerID() { return this.taskContext.getFileOwnerID(); } - private long spillQueueWithLargestAmountOfMainMemory() throws IOException { - - if (this.inactiveOutputChannels.isEmpty()) { - return 0L; - } - - final Iterator it = this.inactiveOutputChannels.iterator(); - - long maxMainMemory = 0; - RuntimeOutputChannelContext maxContext = null; - - while (it.hasNext()) { - - final RuntimeOutputChannelContext context = it.next(); - final long mm = context.getAmountOfMainMemoryInQueue(); - - if (mm > maxMainMemory) { - maxMainMemory = mm; - maxContext = context; - } - } - - if (maxContext != null) { - return maxContext.spillQueueWithOutgoingEnvelopes(); - } - - return 0L; - } - - private void checkForActiveOutputChannels() throws IOException, InterruptedException { - - final Iterator it = this.inactiveOutputChannels.iterator(); - while (it.hasNext()) { - final RuntimeOutputChannelContext channelContext = it.next(); - if (channelContext.isChannelActive()) { - channelContext.flushQueuedOutgoingEnvelopes(); - it.remove(); - } - } - } - /** * {@inheritDoc} */ @@ -117,49 +59,6 @@ public int getMaximumBufferSize() { return this.taskContext.getMaximumBufferSize(); } - /** - * Called by the attached output channel wrapper to forward a {@link TransferEnvelope} object - * to its final destination. Within this method the provided transfer envelope is possibly also - * forwarded to the assigned ephemeral checkpoint. - * - * @param caller - * the output channel context calling this method - * @param outgoingTransferEnvelope - * the transfer envelope to be forwarded - * @throws IOException - * thrown if an I/O error occurs while processing the envelope - * @throws InterruptedException - * thrown if the thread is interrupted while waiting for the envelope to be processed - */ - void processEnvelope(final RuntimeOutputChannelContext caller, final TransferEnvelope outgoingTransferEnvelope) - throws IOException, InterruptedException { - - this.taskContext.processEnvelope(outgoingTransferEnvelope); - - if (this.outputGate.getChannelType() == ChannelType.FILE) { - // Check if the event list of the envelope contains a close event and acknowledge it - final EventList eventList = outgoingTransferEnvelope.getEventList(); - if (eventList != null) { - final Iterator it = eventList.iterator(); - while (it.hasNext()) { - final AbstractEvent event = it.next(); - if (event instanceof ByteBufferedChannelCloseEvent) { - caller.processEvent(event); - } - } - } - } - } - - /** - * {@inheritDoc} - */ - @Override - public void asynchronousEventOccurred() throws IOException, InterruptedException { - - checkForActiveOutputChannels(); - } - /** * {@inheritDoc} */ @@ -183,9 +82,6 @@ public Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOExcep // Report exhaustion of memory buffers to the task context this.taskContext.reportExhaustionOfMemoryBuffers(); - // Spill queue that contains the largest amount of main memory, encapsulated in the queued buffers, to disk - spillQueueWithLargestAmountOfMainMemory(); - // Wait until a memory-based buffer is available buffer = this.taskContext.requestEmptyBufferBlocking(minimumSizeOfBuffer); } @@ -211,28 +107,6 @@ public void reportAsynchronousEvent() { this.taskContext.reportAsynchronousEvent(); } - /** - * Registers the given spilling queue with a network connection. The network connection is in charge of polling the - * remaining elements from the queue. - * - * @param sourceChannelID - * the ID of the source channel which is associated with the spilling queue - * @param spillingQueue - * the spilling queue to be registered - * @return true if the has been successfully registered with the network connection, false - * if the receiver runs within the same task manager and there is no network operation required to transfer - * the queued data - * @throws IOException - * thrown if an I/O error occurs while looking up the destination of the queued envelopes - * @throws InterruptedException - * thrown if the thread is interrupted while looking up the destination of the queued envelopes - */ - boolean registerSpillingQueueWithNetworkConnection(final ChannelID sourceChannelID, - final SpillingQueue spillingQueue) throws IOException, InterruptedException { - - return this.taskContext.registerSpillingQueueWithNetworkConnection(sourceChannelID, spillingQueue); - } - /** * {@inheritDoc} */ @@ -249,6 +123,10 @@ public GateID getGateID() { public OutputChannelContext createOutputChannelContext(ChannelID channelID, OutputChannelContext previousContext, boolean isReceiverRunning, boolean mergeSpillBuffers) { + if (previousContext != null) { + throw new IllegalStateException("Found previous output context for channel " + channelID); + } + AbstractOutputChannel channel = null; for (int i = 0; i < this.outputGate.getNumberOfOutputChannels(); ++i) { AbstractOutputChannel candidateChannel = this.outputGate.getOutputChannel(i); @@ -267,7 +145,22 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp + " is not of type AbstractByteBufferedOutputChannel"); } - return new RuntimeOutputChannelContext(this, (AbstractByteBufferedOutputChannel) channel, - isReceiverRunning, mergeSpillBuffers); + AbstractByteBufferedOutputChannel outputChannel = (AbstractByteBufferedOutputChannel) channel; + + // Construct forwarding chain for this output channel + final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(); + final IncomingEventQueue incomingEventQueue = AbstractOutputChannelContext + .createIncomingEventQueue(forwardingChain); + forwardingChain.addForwarder(new RuntimeOutputChannelBroker(this, forwardingChain, incomingEventQueue, + outputChannel)); + final EphemeralCheckpoint checkpoint = this.taskContext.getEphemeralCheckpoint(); + if (checkpoint != null) { + forwardingChain.addForwarder(checkpoint); + } + forwardingChain.addForwarder(new ForwardingBarrier(channelID)); + forwardingChain.addForwarder(new SpillingBarrier(isReceiverRunning)); + forwardingChain.addForwarder(this.taskContext.getRuntimeDispatcher()); + + return new RuntimeOutputChannelContext(outputChannel, incomingEventQueue); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index b75db8f37ac1f..bbd0934749898 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -91,6 +91,8 @@ public RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment en this.vertexID = vertexID; this.environment = environment; this.taskManager = taskManager; + + this.environment.setExecutionObserver(this); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 11606444abb01..49d18e150bddd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -26,7 +26,6 @@ import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; import eu.stratosphere.nephele.io.channels.Buffer; -import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.taskmanager.bufferprovider.AsynchronousEventListener; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; @@ -35,26 +34,23 @@ import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; -import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; import eu.stratosphere.nephele.types.Record; -public final class RuntimeTaskContext implements BufferProvider, AsynchronousEventListener, LocalBufferPoolOwner, TaskContext { +public final class RuntimeTaskContext implements BufferProvider, AsynchronousEventListener, LocalBufferPoolOwner, + TaskContext { private final LocalBufferPool localBufferPool; private final RuntimeTask task; - private final AsynchronousEventListener[] subEventListener; - private final int numberOfOutputChannels; - final TransferEnvelopeDispatcher transferEnvelopeDispatcher; + private final TransferEnvelopeDispatcher transferEnvelopeDispatcher; - private final EphemeralCheckpoint ephemeralCheckpoint; + private final RuntimeDispatcher runtimeDispatcher; - private final boolean forwardTransferEnvelopes; + private final EphemeralCheckpoint ephemeralCheckpoint; /** * Stores whether the initial exhaustion of memory buffers has already been reported @@ -80,7 +76,6 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve } } this.numberOfOutputChannels = nooc; - this.forwardTransferEnvelopes = ephemeral; this.ephemeralCheckpoint = new EphemeralCheckpoint(task, ephemeral); if (ephemeral) { @@ -88,26 +83,17 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve } this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; - - // Each output gate context will register as a sub event listener - this.subEventListener = new AsynchronousEventListener[environment.getNumberOfOutputGates()]; + this.runtimeDispatcher = new RuntimeDispatcher(transferEnvelopeDispatcher); } - void registerAsynchronousEventListener(final int index, final AsynchronousEventListener eventListener) { + RuntimeDispatcher getRuntimeDispatcher() { - if (index >= this.subEventListener.length || index < 0) { - throw new IllegalArgumentException("Argument index has invalid value " + index); - } - - if (eventListener == null) { - throw new IllegalArgumentException("Argument eventListener must not be null"); - } + return this.runtimeDispatcher; + } - if (this.subEventListener[index] != null) { - throw new IllegalStateException("There is already an event listener with index " + index + " registered"); - } + EphemeralCheckpoint getEphemeralCheckpoint() { - this.subEventListener[index] = eventListener; + return this.ephemeralCheckpoint; } /** @@ -200,17 +186,7 @@ public void reportAsynchronousEvent() { @Override public void asynchronousEventOccurred() throws IOException, InterruptedException { - // First, notify all the listeners about the asynchronous event - for (int i = 0; i < this.subEventListener.length; ++i) { - - if (this.subEventListener[i] == null) { - throw new IllegalStateException("Event listener at index " + i + " is null"); - } - - this.subEventListener[i].asynchronousEventOccurred(); - } - - // Second, check if the checkpoint decision changed + // Check if the checkpoint decision changed this.ephemeralCheckpoint.checkAsynchronousCheckpointDecision(); } @@ -232,37 +208,6 @@ public void setDesignatedNumberOfBuffers(int numberOfBuffers) { this.localBufferPool.setDesignatedNumberOfBuffers(numberOfBuffers); } - /** - * Called by the attached output gate context to forward a {@link TransferEnvelope} object - * to its final destination. Within this method the provided transfer envelope is possibly also - * forwarded to the assigned ephemeral checkpoint. - * - * @param outgoingTransferEnvelope - * the transfer envelope to be forwarded - * @throws IOException - * thrown if an I/O error occurs while processing the envelope - * @throws InterruptedException - * thrown if the thread is interrupted while waiting for the envelope to be processed - */ - void processEnvelope(final TransferEnvelope outgoingTransferEnvelope) throws IOException, InterruptedException { - - if (!this.ephemeralCheckpoint.isDiscarded()) { - final TransferEnvelope dup = outgoingTransferEnvelope.duplicate(); - this.ephemeralCheckpoint.addTransferEnvelope(dup); - } - - if (this.forwardTransferEnvelopes) { - // Immediately forward the envelope - this.transferEnvelopeDispatcher.processEnvelopeFromOutputChannel(outgoingTransferEnvelope); - } else { - // Simply discard the envelope - final Buffer buffer = outgoingTransferEnvelope.getBuffer(); - if (buffer != null) { - buffer.recycleBuffer(); - } - } - } - AbstractID getFileOwnerID() { return this.task.getVertexID(); @@ -274,29 +219,6 @@ public void setCheckpointDecisionAsynchronously(final boolean checkpointDecision this.ephemeralCheckpoint.setCheckpointDecisionAsynchronously(checkpointDecision); } - /** - * Registers the given spilling queue with a network connection. The network connection is in charge of polling the - * remaining elements from the queue. - * - * @param sourceChannelID - * the ID of the source channel which is associated with the spilling queue - * @param spillingQueue - * the spilling queue to be registered - * @return true if the has been successfully registered with the network connection, false - * if the receiver runs within the same task manager and there is no network operation required to transfer - * the queued data - * @throws IOException - * thrown if an I/O error occurs while looking up the destination of the queued envelopes - * @throws InterruptedException - * thrown if the thread is interrupted while looking up the destination of the queued envelopes - */ - boolean registerSpillingQueueWithNetworkConnection(final ChannelID sourceChannelID, - final SpillingQueue spillingQueue) throws IOException, InterruptedException { - - return this.transferEnvelopeDispatcher.registerSpillingQueueWithNetworkConnection(this.task.getJobID(), - sourceChannelID, spillingQueue); - } - /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java new file mode 100644 index 0000000000000..10ec1a823f48b --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java @@ -0,0 +1,64 @@ +package eu.stratosphere.nephele.taskmanager.runtime; + +import java.io.IOException; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelActivateEvent; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; +import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +public final class SpillingBarrier implements OutputChannelForwarder { + + /** + * Queue to store outgoing transfer envelope in case the receiver of the envelopes is not yet running. + */ + private final SpillingQueue queuedOutgoingEnvelopes; + + /** + * Indicates whether the receiver of an envelope is currently running. + */ + private boolean isReceiverRunning = false; + + SpillingBarrier(final boolean isReceiverRunning) { + this.isReceiverRunning = isReceiverRunning; + this.queuedOutgoingEnvelopes = null; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + + if (!this.isReceiverRunning) { + + // TODO: Add this to the spilling queue + + return false; + } + + return true; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasDataLeft() { + + return false; + } + + /** + * {@inheritDoc} + */ + @Override + public void processEvent(final AbstractEvent event) { + + if (event instanceof ByteBufferedChannelActivateEvent) { + this.isReceiverRunning = true; + } + } + +} From 245daf16e1591b27abd619d93992979a866e7c4a Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 2 Feb 2012 13:40:06 +0100 Subject: [PATCH 189/310] Fixed resource leaks during task clean-up --- .../nephele/io/channels/AbstractChannel.java | 2 +- .../AbstractByteBufferedInputChannel.java | 2 +- .../AbstractByteBufferedOutputChannel.java | 5 +++-- .../checkpointing/ReplayInputChannelContext.java | 9 +++++++++ .../checkpointing/ReplayOutputChannelContext.java | 6 ++++++ .../eu/stratosphere/nephele/io/RuntimeInputGate.java | 2 +- .../stratosphere/nephele/io/RuntimeOutputGate.java | 2 +- .../bytebuffered/ByteBufferedChannelManager.java | 12 +++++++++--- .../taskmanager/bytebuffered/ChannelContext.java | 12 +++++++----- .../runtime/RuntimeInputChannelContext.java | 4 ++++ .../runtime/RuntimeOutputChannelContext.java | 9 +++++++++ 11 files changed, 51 insertions(+), 14 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java index 19cc9c2af28d3..77eaeb31cea38 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java @@ -171,7 +171,7 @@ public CompressionLevel getCompressionLevel() { * Releases all resources (especially buffers) which are currently allocated by this channel. This method should be * called in case of a task error or as a result of a cancel operation. */ - public abstract void releaseResources(); + public abstract void releaseAllResources(); /** * Returns the number of bytes which have been transmitted through this channel since its instantiation. diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java index 56f1cfe5b53c3..f99af0fff0fa7 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java @@ -291,7 +291,7 @@ public void reportIOException(IOException ioe) { * {@inheritDoc} */ @Override - public void releaseResources() { + public void releaseAllResources() { synchronized (this.synchronisationObject) { this.brokerAggreedToCloseChannel = true; diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java index 064f329b9de00..08e291f0d79d9 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java @@ -117,7 +117,7 @@ public void requestClose() throws IOException, InterruptedException { if (!this.closeRequested) { this.closeRequested = true; if (this.serializationBuffer.dataLeftFromPreviousSerialization()) { - //make sure we serialized all data before we send the close event + // make sure we serialized all data before we send the close event flush(); } @@ -326,8 +326,9 @@ public void flush() throws IOException, InterruptedException { * {@inheritDoc} */ @Override - public void releaseResources() { + public void releaseAllResources() { + // TODO: Reconsider release of broker's resources here this.closeRequested = true; this.serializationBuffer.clear(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java index 22c40c32062bb..dc3a597fb7cf0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java @@ -134,4 +134,13 @@ public int getNumberOfQueuedMemoryBuffers() { return this.encapsulatedContext.getNumberOfQueuedMemoryBuffers(); } + /** + * {@inheritDoc} + */ + @Override + public void releaseAllResources() { + + this.encapsulatedContext.releaseAllResources(); + } + } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java index 837bf523f08dd..df1f335c5d85d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java @@ -60,4 +60,10 @@ public void queueTransferEnvelope(TransferEnvelope transferEnvelope) { } + @Override + public void releaseAllResources() { + // TODO Auto-generated method stub + + } + } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java index feacd67ab8400..0c4cf09e02b39 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java @@ -487,7 +487,7 @@ public void releaseAllChannelResources() { final Iterator> it = this.inputChannels.iterator(); while (it.hasNext()) { - it.next().releaseResources(); + it.next().releaseAllResources(); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java index c843cca1955ba..35ed0417b6295 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java @@ -468,7 +468,7 @@ public void releaseAllChannelResources() { final Iterator> it = this.outputChannels.iterator(); while (it.hasNext()) { - it.next().releaseResources(); + it.next().releaseAllResources(); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 3e5baa18949ab..6556892e44d77 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -148,7 +148,7 @@ public void register(final Task task, final Set activeOutputChannels) .get(channelID); final boolean isActive = activeOutputChannels.contains(channelID); - + final OutputChannelContext outputChannelContext = outputGateContext.createOutputChannelContext( channelID, previousContext, isActive, this.mergeSpilledBuffers); @@ -230,7 +230,10 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { while (channelIterator.hasNext()) { final ChannelID outputChannelID = channelIterator.next(); - this.registeredChannels.remove(outputChannelID); + final ChannelContext context = this.registeredChannels.remove(outputChannelID); + if (context != null) { + context.releaseAllResources(); + } this.receiverCache.remove(outputChannelID); } @@ -239,7 +242,10 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { while (channelIterator.hasNext()) { final ChannelID outputChannelID = channelIterator.next(); - this.registeredChannels.remove(outputChannelID); + final ChannelContext context = this.registeredChannels.remove(outputChannelID); + if (context != null) { + context.releaseAllResources(); + } this.receiverCache.remove(outputChannelID); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java index 203e2bf142561..6346b2a478dd0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java @@ -24,13 +24,15 @@ public interface ChannelContext { boolean isInputChannel(); - public JobID getJobID(); + JobID getJobID(); - public ChannelID getChannelID(); + ChannelID getChannelID(); - public ChannelID getConnectedChannelID(); + ChannelID getConnectedChannelID(); - public ChannelType getType(); + ChannelType getType(); - public void queueTransferEnvelope(TransferEnvelope transferEnvelope); + void queueTransferEnvelope(TransferEnvelope transferEnvelope); + + void releaseAllResources(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java index 807c813bdc561..756cfb819be01 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java @@ -231,6 +231,10 @@ public boolean isInputChannel() { return this.byteBufferedInputChannel.isInputChannel(); } + /** + * {@inheritDoc} + */ + @Override public void releaseAllResources() { final Queue buffersToRecycle = new ArrayDeque(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java index 563d9ac640a11..431b19f503ead 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java @@ -77,4 +77,13 @@ public ChannelType getType() { return this.byteBufferedOutputChannel.getType(); } + + /** + * {@inheritDoc} + */ + @Override + public void releaseAllResources() { + // TODO Auto-generated method stub + + } } From 249358b91a058dbc54afe464aa8057016a4e24f4 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 2 Feb 2012 13:43:02 +0100 Subject: [PATCH 190/310] Improved robustness of recovery phase --- .../taskmanager/runtime/RuntimeInputChannelContext.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java index 756cfb819be01..7072d21ac5099 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java @@ -173,10 +173,12 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { if (sequenceNumber > expectedSequenceNumber) { - // This is a problem, now we are actually missing some data - this.byteBufferedInputChannel.reportIOException(new IOException("Expected data packet " + if (expectedSequenceNumber > 0) { + // This is a problem, now we are actually missing some data + this.byteBufferedInputChannel.reportIOException(new IOException("Expected data packet " + expectedSequenceNumber + " but received " + sequenceNumber)); - this.byteBufferedInputChannel.checkForNetworkEvents(); + this.byteBufferedInputChannel.checkForNetworkEvents(); + } } else { // Tell the sender that we are expecting an envelope with a higher sequence number From dac63173672da25884dec32cec655b824a8e887a Mon Sep 17 00:00:00 2001 From: marrus Date: Thu, 2 Feb 2012 16:18:16 +0100 Subject: [PATCH 191/310] changed cpu usage intervall --- .../eu/stratosphere/nephele/io/InputGate.java | 12 ++++++++++++ .../CheckpointDecisionCoordinator.java | 16 ++++++++++++---- .../stratosphere/nephele/taskmanager/Task.java | 6 +++++- 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java index c327362a839ee..2f5e120429c99 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java @@ -109,6 +109,10 @@ public class InputGate extends AbstractGate implements IORe * The thread which executes the task connected to the input gate. */ private Thread executingThread = null; + /** + * The Systemtime at the first arrival of a record + */ + private long executionstart = -1; /** * Constructs a new input gate. @@ -351,6 +355,10 @@ public T readRecord(final T target) throws IOException, InterruptedException { this.channelToReadFrom = waitForAnyChannelToBecomeAvailable(); } try { + if(this.executionstart == -1){ + //save time for arrival of first record + this.executionstart = System.currentTimeMillis(); + } record = this.getInputChannel(this.channelToReadFrom).readRecord(target); } catch (EOFException e) { // System.out.println("### Caught EOF exception at channel " + channelToReadFrom + "(" + @@ -635,4 +643,8 @@ void registerRecordAvailabilityListener(final RecordAvailabilityListener list this.recordAvailabilityListener = listener; } } + + public long getExecutionStart(){ + return this.executionstart; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index 61ca9b261aa3f..169f71ae420bf 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -106,6 +106,7 @@ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtil synchronized (graph) { ExecutionGroupVertex groupVertex = vertex.getGroupVertex(); + LOG.info("Forcing decision to " + checkpointDecision + " for all of " + groupVertex.getName()); //force decision to all groupVertex members for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); i++) { ExecutionVertex member = groupVertex.getGroupMember(i); @@ -148,31 +149,38 @@ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtil private boolean getDecision(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { // This implementation always creates the checkpoint if(rus.getForced() == null){ - if(rus.getTotalInputAmount() != 0 && (rus.getTotalOutputAmount() * 1.0 / rus.getTotalInputAmount() > 2.0)){ + if(rus.getTotalInputAmount() != 0 ){ + LOG.info("selektivity is " + (double)rus.getTotalOutputAmount() / rus.getTotalInputAmount()); + LOG.info("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); + } + if(rus.getTotalInputAmount() != 0 && ((double)rus.getTotalOutputAmount() / rus.getTotalInputAmount() > 2.0)){ //estimated size of checkpoint //TODO progress estimation would make sense here - LOG.info("Chechpoint to large selektivity " + (rus.getTotalOutputAmount() * 1.0 / rus.getTotalInputAmount() > 2.0)); + LOG.info(vertex.getEnvironment().getTaskName() + "Chechpoint to large selektivity " + ((double)rus.getTotalOutputAmount()/ rus.getTotalInputAmount() > 2.0)); return false; } if (rus.getUserCPU() >= 90) { - LOG.info("CPU-Bottleneck"); + LOG.info(vertex.getEnvironment().getTaskName() + "CPU-Bottleneck"); //CPU bottleneck return true; } if ( vertex.getNumberOfSuccessors() != 0 && vertex.getNumberOfPredecessors() * 1.0 / vertex.getNumberOfSuccessors() > 1.5) { - LOG.info("vertex.getNumberOfPredecessors()/ vertex.getNumberOfSuccessors() > 1.5"); + + LOG.info(vertex.getEnvironment().getTaskName() + " vertex.getNumberOfPredecessors() " + vertex.getNumberOfPredecessors() +" / vertex.getNumberOfSuccessors() " + vertex.getNumberOfSuccessors() +" > 1.5"); //less output-channels than input-channels //checkpoint at this position probably saves network-traffic return true; } }else{ + LOG.info("Checkpoint decision was forced"); //checkpoint decision was forced by the user return rus.getForced(); } + LOG.info("always create Checkpoint for testing"); //FIXME always create checkpoint for testing return true; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index 173c16a074b9f..9be3b8b42222e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -278,11 +278,14 @@ public void initialExecutionResourcesExhausted() { // Construct a resource utilization snapshot final long timestamp = System.currentTimeMillis(); + if(this.environment.getInputGate(0) != null && this.environment.getInputGate(0).getExecutionStart() < timestamp ){ + this.startTime = this.environment.getInputGate(0).getExecutionStart(); + } // Get CPU-Usertime in percent ThreadMXBean threadBean = ManagementFactory.getThreadMXBean(); long userCPU = (threadBean.getCurrentThreadUserTime() / NANO_TO_MILLISECONDS) * 100 / (timestamp - this.startTime); - + LOG.info("USER CPU for " + this.getTaskName() + " : " + userCPU); // collect outputChannelUtilization final Map channelUtilization = new HashMap(); long totalOutputAmount = 0; @@ -295,6 +298,7 @@ public void initialExecutionResourcesExhausted() { totalOutputAmount += outputChannel.getAmountOfDataTransmitted(); } } + //FIXME (marrus) it is not about what we received but what we processed yet long totalInputAmount = 0; for (int i = 0; i < this.environment.getNumberOfInputGates(); ++i) { final InputGate inputGate = this.environment.getInputGate(i); From 7d85f201d0511a66c59492c1c1557620253c987a Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 2 Feb 2012 18:03:20 +0100 Subject: [PATCH 192/310] Worked on replay task implementation --- .../checkpointing/CheckpointEnvironment.java | 161 ++++++------- .../checkpointing/ReplayOutputBroker.java | 72 ++++++ .../nephele/checkpointing/ReplayTask.java | 162 ++++++++++++- .../nephele/checkpointing/ReplayThread.java | 214 ++++++++++++++++++ .../nephele/taskmanager/Task.java | 8 + .../nephele/taskmanager/TaskManager.java | 31 +-- .../bufferprovider/LocalBufferPool.java | 3 +- .../taskmanager/runtime/RuntimeTask.java | 20 +- 8 files changed, 548 insertions(+), 123 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java index c56be1a697871..68722f0b52f4b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java @@ -1,14 +1,12 @@ package eu.stratosphere.nephele.checkpointing; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; import java.util.Map; import java.util.Set; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.ExecutionObserver; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.ChannelSelector; import eu.stratosphere.nephele.io.DistributionPattern; import eu.stratosphere.nephele.io.GateID; @@ -24,44 +22,39 @@ final class CheckpointEnvironment implements Environment { - private final JobID jobID; + private final ExecutionVertexID vertexID; - private final Map> outputChannelIDs; + private final Environment environment; - private final Map> inputChannelIDs; + private final boolean hasCompleteCheckpoint; - private CheckpointEnvironment(final JobID jobID, Map> outputChannelIDs, - Map> inputChannelIDs) { + private final Map outputBrokerMap; - this.jobID = jobID; - this.outputChannelIDs = outputChannelIDs; - this.inputChannelIDs = inputChannelIDs; - } - - static CheckpointEnvironment createFromEnvironment(final Environment environment) { - - final JobID jobID = environment.getJobID(); - - final Map> outputChannelIDs = new HashMap>(); - - final Map> inputChannelIDs = new HashMap>(); - - Iterator gateIt = environment.getOutputGateIDs().iterator(); - while (gateIt.hasNext()) { + /** + * The observer object for the task's execution. + */ + private volatile ExecutionObserver executionObserver = null; - final GateID gateID = gateIt.next(); - outputChannelIDs.put(gateID, environment.getOutputChannelIDsOfGate(gateID)); - } + private volatile ReplayThread executingThread = null; - gateIt = environment.getInputGateIDs().iterator(); - while (gateIt.hasNext()) { + CheckpointEnvironment(final ExecutionVertexID vertexID, final Environment environment, + final boolean hasCompleteCheckpoint, + final Map outputBrokerMap) { - final GateID gateID = gateIt.next(); - inputChannelIDs.put(gateID, environment.getInputChannelIDsOfGate(gateID)); - } + this.vertexID = vertexID; + this.environment = environment; + this.hasCompleteCheckpoint = hasCompleteCheckpoint; + this.outputBrokerMap = outputBrokerMap; + } - return new CheckpointEnvironment(jobID, Collections.unmodifiableMap(outputChannelIDs), - Collections.unmodifiableMap(inputChannelIDs)); + /** + * Sets the execution observer for this environment. + * + * @param executionObserver + * the execution observer for this environment + */ + void setExecutionObserver(final ExecutionObserver executionObserver) { + this.executionObserver = executionObserver; } /** @@ -70,7 +63,7 @@ static CheckpointEnvironment createFromEnvironment(final Environment environment @Override public JobID getJobID() { - return this.jobID; + return this.environment.getJobID(); } /** @@ -79,7 +72,7 @@ public JobID getJobID() { @Override public Configuration getTaskConfiguration() { - throw new UnsupportedOperationException("Method getTaskConfiguration is not supported by this environment"); + return this.environment.getTaskConfiguration(); } /** @@ -88,7 +81,7 @@ public Configuration getTaskConfiguration() { @Override public Configuration getJobConfiguration() { - throw new UnsupportedOperationException("Method getJobConfiguration is not supported by this environment"); + return this.environment.getJobConfiguration(); } /** @@ -97,8 +90,7 @@ public Configuration getJobConfiguration() { @Override public int getCurrentNumberOfSubtasks() { - throw new UnsupportedOperationException( - "Method getCurrentNumberOfSubtasks is not supported by this environment"); + return this.environment.getCurrentNumberOfSubtasks(); } /** @@ -107,7 +99,7 @@ public int getCurrentNumberOfSubtasks() { @Override public int getIndexInSubtaskGroup() { - throw new UnsupportedOperationException("Method getIndexInSubtaskGroup is not supported by this environment"); + return this.environment.getIndexInSubtaskGroup(); } /** @@ -116,7 +108,7 @@ public int getIndexInSubtaskGroup() { @Override public void userThreadStarted(final Thread userThread) { - throw new UnsupportedOperationException("Method userThreadStarted is not supported by this environment"); + throw new IllegalStateException("Checkpoint replay task called userThreadStarted"); } /** @@ -125,7 +117,7 @@ public void userThreadStarted(final Thread userThread) { @Override public void userThreadFinished(final Thread userThread) { - throw new UnsupportedOperationException("Method userThreadFinished is not supported by this environment"); + throw new IllegalStateException("Checkpoint replay task called userThreadFinished"); } /** @@ -134,7 +126,7 @@ public void userThreadFinished(final Thread userThread) { @Override public InputSplitProvider getInputSplitProvider() { - throw new UnsupportedOperationException("Method getInputSplitProvider is not supported by this environment"); + throw new IllegalStateException("Checkpoint replay task called getInputSplitProvider"); } /** @@ -143,7 +135,7 @@ public InputSplitProvider getInputSplitProvider() { @Override public IOManager getIOManager() { - throw new UnsupportedOperationException("Method getIOManager is not supported by this environment"); + throw new IllegalStateException("Checkpoint replay task called getIOManager"); } /** @@ -152,7 +144,7 @@ public IOManager getIOManager() { @Override public MemoryManager getMemoryManager() { - throw new UnsupportedOperationException("Method getMemoryManager is not supported by this environment"); + throw new IllegalStateException("Checkpoint replay task called getMemoryManager"); } /** @@ -161,7 +153,7 @@ public MemoryManager getMemoryManager() { @Override public String getTaskName() { - throw new UnsupportedOperationException("Method getTaskName is not supported by this environment"); + return this.environment.getTaskName(); } /** @@ -170,7 +162,7 @@ public String getTaskName() { @Override public GateID getNextUnboundInputGateID() { - throw new UnsupportedOperationException("Method getNextUnboundInputGateID is not supported by this environment"); + throw new IllegalStateException("Checkpoint replay task called getNextUnboundInputGateID"); } /** @@ -179,8 +171,7 @@ public GateID getNextUnboundInputGateID() { @Override public GateID getNextUnboundOutputGateID() { - throw new UnsupportedOperationException( - "Method getNextUnboundOutputGateID is not supported by this environment"); + throw new IllegalStateException("Checkpoint replay task called getNextUnboundOutputGateID"); } /** @@ -189,7 +180,7 @@ public GateID getNextUnboundOutputGateID() { @Override public int getNumberOfOutputGates() { - throw new UnsupportedOperationException("Method getNumberOfOutputGates is not supported by this environment"); + return this.environment.getNumberOfOutputGates(); } /** @@ -198,17 +189,18 @@ public int getNumberOfOutputGates() { @Override public int getNumberOfInputGates() { - throw new UnsupportedOperationException("Method getNumberOfInputGates is not supported by this environment"); + return this.environment.getNumberOfInputGates(); } /** * {@inheritDoc} */ @Override - public OutputGate createOutputGate(final GateID gateID, Class outputClass, + public OutputGate createOutputGate(final GateID gateID, + final Class outputClass, final ChannelSelector selector, final boolean isBroadcast) { - throw new UnsupportedOperationException("Method createOutputGate is not supported by this environment"); + throw new IllegalStateException("Checkpoint replay task called createOutputGate"); } /** @@ -218,7 +210,7 @@ public OutputGate createOutputGate(final GateID gateID, Class< public InputGate createInputGate(final GateID gateID, final RecordDeserializer deserializer, final DistributionPattern distributionPattern) { - throw new UnsupportedOperationException("Method createInputGate is not supported by this environment"); + throw new IllegalStateException("Checkpoint replay task called createInputGate"); } /** @@ -227,7 +219,7 @@ public InputGate createInputGate(final GateID gateID, @Override public void registerOutputGate(final OutputGate outputGate) { - throw new UnsupportedOperationException("Method registerOutputGate is not supported by this environment"); + throw new IllegalStateException("Checkpoint replay task called registerOutputGate"); } /** @@ -236,7 +228,7 @@ public void registerOutputGate(final OutputGate outputGate) { @Override public void registerInputGate(final InputGate inputGate) { - throw new UnsupportedOperationException("Method registerInputGate is not supported by this environment"); + throw new IllegalStateException("Checkpoint replay task called registerInputGate"); } /** @@ -245,16 +237,7 @@ public void registerInputGate(final InputGate inputGate) { @Override public Set getOutputChannelIDs() { - final Set channelIDs = new HashSet(); - final Iterator>> it = this.outputChannelIDs.entrySet().iterator(); - while (it.hasNext()) { - - final Map.Entry> entry = it.next(); - channelIDs.addAll(entry.getValue()); - - } - - return Collections.unmodifiableSet(channelIDs); + return this.environment.getOutputChannelIDs(); } /** @@ -263,16 +246,7 @@ public Set getOutputChannelIDs() { @Override public Set getInputChannelIDs() { - final Set channelIDs = new HashSet(); - final Iterator>> it = this.inputChannelIDs.entrySet().iterator(); - while (it.hasNext()) { - - final Map.Entry> entry = it.next(); - channelIDs.addAll(entry.getValue()); - - } - - return Collections.unmodifiableSet(channelIDs); + return this.environment.getInputChannelIDs(); } /** @@ -281,7 +255,7 @@ public Set getInputChannelIDs() { @Override public Set getOutputGateIDs() { - return this.outputChannelIDs.keySet(); + return this.environment.getOutputGateIDs(); } /** @@ -290,7 +264,7 @@ public Set getOutputGateIDs() { @Override public Set getInputGateIDs() { - return this.inputChannelIDs.keySet(); + return this.environment.getInputGateIDs(); } /** @@ -299,12 +273,7 @@ public Set getInputGateIDs() { @Override public Set getOutputChannelIDsOfGate(final GateID gateID) { - final Set channelIDs = this.outputChannelIDs.get(gateID); - if (channelIDs == null) { - throw new IllegalStateException("Cannot find channel IDs for output gate with ID " + gateID); - } - - return channelIDs; + return this.environment.getOutputChannelIDsOfGate(gateID); } /** @@ -313,12 +282,24 @@ public Set getOutputChannelIDsOfGate(final GateID gateID) { @Override public Set getInputChannelIDsOfGate(final GateID gateID) { - final Set channelIDs = this.inputChannelIDs.get(gateID); - if (channelIDs == null) { - throw new IllegalStateException("Cannot find channel IDs for input gate with ID " + gateID); - } - - return channelIDs; + return this.environment.getInputChannelIDsOfGate(gateID); } + /** + * Returns the thread which is assigned to executes the replay task + * + * @return the thread which is assigned to execute the replay task + */ + public ReplayThread getExecutingThread() { + + synchronized (this) { + + if (this.executingThread == null) { + this.executingThread = new ReplayThread(this.vertexID, this.executionObserver, getTaskName(), + this.hasCompleteCheckpoint, this.outputBrokerMap); + } + + return this.executingThread; + } + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java new file mode 100644 index 0000000000000..dba2f08d26099 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java @@ -0,0 +1,72 @@ +package eu.stratosphere.nephele.checkpointing; + +import java.io.IOException; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; +import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +final class ReplayOutputBroker implements OutputChannelForwarder { + + private final OutputChannelForwardingChain forwardingChain; + + private final IncomingEventQueue incomingEventQueue; + + /** + * Stores whether the receiver has acknowledged the close request from this channel. + */ + private boolean closeAcknowledgementReceived = false; + + ReplayOutputBroker(final OutputChannelForwardingChain forwardingChain, final IncomingEventQueue incomingEventQueue) { + this.forwardingChain = forwardingChain; + this.incomingEventQueue = incomingEventQueue; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + + // Nothing to do here + + return true; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasDataLeft() { + + return (!this.closeAcknowledgementReceived); + } + + /** + * {@inheritDoc} + */ + @Override + public void processEvent(final AbstractEvent event) { + + if (event instanceof ByteBufferedChannelCloseEvent) { + this.closeAcknowledgementReceived = true; + } else { + System.out.println("Received unknown event: " + event); + } + } + + void outputEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + + this.incomingEventQueue.processQueuedEvents(); + + this.forwardingChain.forwardEnvelope(transferEnvelope); + } + + boolean hasFinished() { + + return (!this.forwardingChain.anyForwarderHasDataLeft()); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index 30f3ffcb08913..cd0643231c453 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -16,10 +16,19 @@ package eu.stratosphere.nephele.checkpointing; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.io.OutputBuffer; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.ExecutionObserver; +import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.profiling.TaskManagerProfiler; import eu.stratosphere.nephele.services.iomanager.IOManager; @@ -27,33 +36,129 @@ import eu.stratosphere.nephele.taskmanager.Task; import eu.stratosphere.nephele.taskmanager.TaskManager; import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; import eu.stratosphere.nephele.template.InputSplitProvider; public final class ReplayTask implements Task { + private final class ReplayTaskExecutionObserver implements ExecutionObserver { + + private final RuntimeTask encapsulatedTask; + + private ReplayTaskExecutionObserver(final RuntimeTask encapsulatedTask) { + this.encapsulatedTask = encapsulatedTask; + } + + /** + * {@inheritDoc} + */ + @Override + public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) { + + if (this.encapsulatedTask == null) { + replayTaskExecutionState = newExecutionState; + } else { + encapsulatedExecutionState = newExecutionState; + } + + reportExecutionStateChange((this.encapsulatedTask == null), optionalMessage); + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadStarted(final Thread userThread) { + + if (this.encapsulatedTask != null) { + this.encapsulatedTask.userThreadStarted(userThread); + } else { + LOG.error("userThreadStarted called although there is no encapsulated task"); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void userThreadFinished(final Thread userThread) { + + if (this.encapsulatedTask != null) { + this.encapsulatedTask.userThreadFinished(userThread); + } else { + LOG.error("userThreadFinished called although there is no encapsulated task"); + } + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isCanceled() { + + if (this.encapsulatedTask != null) { + if (this.encapsulatedTask.isCanceled()) { + return true; + } + } + + return isCanceled; + } + } + + /** + * The log object used for debugging. + */ + private static final Log LOG = LogFactory.getLog(ReplayTask.class); + private final ExecutionVertexID vertexID; private final CheckpointEnvironment environment; - private final Task encapsulatedTask; + private final RuntimeTask encapsulatedTask; + + private final TaskManager taskManager; + + private volatile ExecutionState encapsulatedExecutionState = null; + + private volatile ExecutionState replayTaskExecutionState = ExecutionState.STARTING; + + private AtomicBoolean replayThreadStarted = new AtomicBoolean(false); + + /** + * Stores whether the task has been canceled. + */ + private volatile boolean isCanceled = false; + + private final Map outputBrokerMap = new ConcurrentHashMap(); public ReplayTask(final ExecutionVertexID vertexID, final Environment environment, final TaskManager taskManager) { this.vertexID = vertexID; - this.environment = CheckpointEnvironment.createFromEnvironment(environment); + this.environment = new CheckpointEnvironment(this.vertexID, environment, + CheckpointUtils.hasCompleteCheckpointAvailable(vertexID), this.outputBrokerMap); + this.environment.setExecutionObserver(new ReplayTaskExecutionObserver(null)); this.encapsulatedTask = null; + this.taskManager = taskManager; } - public ReplayTask(final Task encapsulatedTask) { + public ReplayTask(final RuntimeTask encapsulatedTask, final TaskManager taskManager) { this.vertexID = encapsulatedTask.getVertexID(); - this.environment = CheckpointEnvironment.createFromEnvironment(encapsulatedTask.getEnvironment()); + this.environment = new CheckpointEnvironment(this.vertexID, encapsulatedTask.getEnvironment(), + CheckpointUtils.hasCompleteCheckpointAvailable(vertexID), this.outputBrokerMap); + this.environment.setExecutionObserver(new ReplayTaskExecutionObserver(null)); this.encapsulatedTask = encapsulatedTask; + // Redirect all state change notifications to this task + this.encapsulatedTask.getRuntimeEnvironment().setExecutionObserver( + new ReplayTaskExecutionObserver(this.encapsulatedTask)); + this.encapsulatedExecutionState = this.encapsulatedTask.getExecutionState(); + this.taskManager = taskManager; } /** @@ -83,10 +188,14 @@ public Environment getEnvironment() { return this.environment; } + /** + * {@inheritDoc} + */ @Override public void markAsFailed() { - // TODO Auto-generated method stub + this.replayTaskExecutionState = ExecutionState.FAILED; + reportExecutionStateChange(true, "Execution thread died unexpectedly"); } @Override @@ -95,11 +204,18 @@ public boolean isTerminated() { return false; } + /** + * {@inheritDoc} + */ @Override public void startExecution() { - // TODO Auto-generated method stub - System.out.println("Checkpoint replay task started"); + final ReplayThread thread = this.environment.getExecutingThread(); + if (this.replayThreadStarted.compareAndSet(false, true)) { + thread.start(); + } else { + thread.restart(); + } } @Override @@ -178,4 +294,36 @@ public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEn return new ReplayTaskContext(); } + private void reportExecutionStateChange(final boolean replayTaskStateChanged, final String optionalMessage) { + + final JobID jobID = this.environment.getJobID(); + + if (replayTaskStateChanged) { + + if (this.replayTaskExecutionState == ExecutionState.REPLAYING) { + this.taskManager.executionStateChanged(jobID, this.vertexID, this.replayTaskExecutionState, + optionalMessage); + } + + } else { + + + + } + } + + void registerReplayOutputBroker(final ChannelID channelID, final ReplayOutputBroker outputBroker) { + + this.outputBrokerMap.put(channelID, outputBroker); + } + + /** + * {@inheritDoc} + */ + @Override + public ExecutionState getExecutionState() { + + return null; + } + } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java new file mode 100644 index 0000000000000..4a3727afc6b6d --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -0,0 +1,214 @@ +package eu.stratosphere.nephele.checkpointing; + +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +import eu.stratosphere.nephele.execution.ExecutionObserver; +import eu.stratosphere.nephele.execution.ExecutionState; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.taskmanager.transferenvelope.CheckpointDeserializer; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; +import eu.stratosphere.nephele.util.StringUtils; + +final class ReplayThread extends Thread { + + private static final String REPLAY_SUFFIX = " (Replay)"; + + private final ExecutionVertexID vertexID; + + private final ExecutionObserver executionObserver; + + private final boolean isCheckpointComplete; + + private final Map outputBrokerMap; + + /** + * The interval to sleep in case a communication channel is not yet entirely set up (in milliseconds). + */ + private static final int SLEEPINTERVAL = 100; + + private final AtomicBoolean restartRequested = new AtomicBoolean(false); + + ReplayThread(final ExecutionVertexID vertexID, final ExecutionObserver executionObserver, final String taskName, + final boolean isCheckpointComplete, final Map outputBrokerMap) { + super((taskName == null ? "Unkown" : taskName) + REPLAY_SUFFIX); + + this.vertexID = vertexID; + this.executionObserver = executionObserver; + this.isCheckpointComplete = isCheckpointComplete; + this.outputBrokerMap = outputBrokerMap; + } + + /** + * {@inheritDoc} + */ + @Override + public void run() { + + // Now the actual program starts to run + changeExecutionState(ExecutionState.REPLAYING, null); + + // If the task has been canceled in the mean time, do not even start it + if (this.executionObserver.isCanceled()) { + changeExecutionState(ExecutionState.CANCELED, null); + return; + } + + try { + + replayCheckpoint(); + + // Make sure, we enter the catch block when the task has been canceled + if (this.executionObserver.isCanceled()) { + throw new InterruptedException(); + } + + } catch (Exception e) { + + if (this.executionObserver.isCanceled()) { + changeExecutionState(ExecutionState.CANCELED, null); + } else { + changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(e)); + } + + return; + } + + // Task finished running, but there may be some unconsumed data in the brokers + changeExecutionState(ExecutionState.FINISHING, null); + + try { + // Wait until all output broker have sent all of their data + waitForAllOutputBrokerToFinish(); + } catch (Exception e) { + + if (this.executionObserver.isCanceled()) { + changeExecutionState(ExecutionState.CANCELED, null); + } else { + changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(e)); + } + + return; + } + + // Finally, switch execution state to FINISHED and report to job manager + changeExecutionState(ExecutionState.FINISHED, null); + } + + private void waitForAllOutputBrokerToFinish() throws IOException, InterruptedException { + + while (true) { + boolean finished = true; + final Iterator it = this.outputBrokerMap.values().iterator(); + while (it.hasNext()) { + + if (it.next().hasFinished()) { + finished = false; + } + } + + if (finished) { + break; + } + + Thread.sleep(SLEEPINTERVAL); + } + } + + private void changeExecutionState(final ExecutionState newExecutionState, final String optionalMessage) { + + if (this.executionObserver != null) { + this.executionObserver.executionStateChanged(newExecutionState, optionalMessage); + } + } + + void restart() { + + this.restartRequested.set(true); + } + + private void replayCheckpoint() throws Exception { + + final CheckpointDeserializer deserializer = new CheckpointDeserializer(this.vertexID); + + int metaDataIndex = 0; + + while (true) { + + if (this.restartRequested.compareAndSet(true, false)) { + metaDataIndex = 0; + } + + // Try to locate the meta data file + final File metaDataFile = new File(CheckpointUtils.getCheckpointDirectory() + File.separator + + CheckpointUtils.METADATA_PREFIX + "_" + this.vertexID + "_" + metaDataIndex); + + while (!metaDataFile.exists()) { + + // Try to locate the final meta data file + final File finalMetaDataFile = new File(CheckpointUtils.getCheckpointDirectory() + File.separator + + CheckpointUtils.METADATA_PREFIX + "_" + this.vertexID + "_final"); + + if (finalMetaDataFile.exists()) { + return; + } + + if (this.isCheckpointComplete) { + throw new FileNotFoundException("Cannot find meta data file " + metaDataIndex + + " for checkpoint of vertex " + this.vertexID); + } + + // Wait for the file to be created + Thread.sleep(100); + + } + + FileInputStream fis = null; + + try { + + fis = new FileInputStream(metaDataFile); + final FileChannel fileChannel = fis.getChannel(); + + while (true) { + try { + deserializer.read(fileChannel); + + final TransferEnvelope transferEnvelope = deserializer.getFullyDeserializedTransferEnvelope(); + if (transferEnvelope != null) { + outputEnvelope(transferEnvelope); + } + } catch (EOFException eof) { + // Close the file channel + fileChannel.close(); + // Increase the index of the meta data file + ++metaDataIndex; + break; + } + } + } finally { + if (fis != null) { + fis.close(); + } + } + } + } + + private void outputEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + + final ReplayOutputBroker outputBroker = this.outputBrokerMap.get(transferEnvelope.getSource()); + if (outputBroker == null) { + throw new IOException("Cannot find output broker for channel " + transferEnvelope.getSource()); + } + + outputBroker.outputEnvelope(transferEnvelope); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index 70b3e7a06e786..d249430147b73 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -19,6 +19,7 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; +import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.profiling.TaskManagerProfiler; @@ -130,6 +131,13 @@ public interface Task { */ void unregisterProfiler(TaskManagerProfiler taskManagerProfiler); + /** + * Returns the current execution state of the task. + * + * @return the current execution state of the task + */ + ExecutionState getExecutionState(); + TaskContext createTaskContext(TransferEnvelopeDispatcher transferEnvelopeDispatcher, Map tasksWithUndecidedCheckpoints); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index f920c9c174894..94415635728b7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -548,7 +548,7 @@ private Task createAndRegisterTask(final ExecutionVertexID id, final Configurati // Check if there at least a partial checkpoint available if (CheckpointUtils.hasPartialCheckpointAvailable(id)) { - task = new ReplayTask((RuntimeTask) runningTask); + task = new ReplayTask((RuntimeTask) runningTask, this); } else { // Task is already running return null; @@ -602,14 +602,18 @@ private Task createAndRegisterTask(final ExecutionVertexID id, final Configurati * * @param id * the ID of the task to be unregistered - * @param task - * the {@link Task} to be unregistered */ - private void unregisterTask(final ExecutionVertexID id, final Task task) { + private void unregisterTask(final ExecutionVertexID id) { // Task deregistration must be atomic synchronized (this) { + final Task task = this.runningTasks.remove(id); + if (task == null) { + LOG.error("Cannot find task with ID " + id + " to unregister"); + return; + } + // Unregister task from the byte buffered channel manager this.byteBufferedChannelManager.unregister(id, task); @@ -676,27 +680,14 @@ public void updateLibraryCache(LibraryCacheUpdate update) throws IOException { // Nothing to to here } - public void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final Task task, + public void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final ExecutionState newExecutionState, final String optionalDescription) { - if (newExecutionState == ExecutionState.RUNNING || newExecutionState == ExecutionState.RUNNING) { - // Mark task as running by putting it in the corresponding map - synchronized (this.runningTasks) { - this.runningTasks.put(id, task); - } - } - if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED || newExecutionState == ExecutionState.FAILED) { - // In any of these states the task's thread will be terminated, so we remove the task from the running tasks - // map - synchronized (this.runningTasks) { - this.runningTasks.remove(id); - } - - // Unregister the task (free all buffers, remove all channels, task-specific class loaders, etc... - unregisterTask(id, task); + // Unregister the task (free all buffers, remove all channels, task-specific class loaders, etc...) + unregisterTask(id); } // Get lock on the jobManager object and propagate the state change synchronized (this.jobManager) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java index 2804a402c1095..45bf09e255e0d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java @@ -190,7 +190,8 @@ public void clear() { synchronized (this.buffers) { if (this.requestedNumberOfBuffers != this.buffers.size()) { - LOG.error("Clear is called, but some buffers are still missing..."); + LOG.error("Possible resource leak: Requested number of buffers is " + this.requestedNumberOfBuffers + + ", but only " + this.buffers.size() + " buffers in local pool"); } while (!this.buffers.isEmpty()) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index bbd0934749898..81c6047342d0b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -62,7 +62,7 @@ public final class RuntimeTask implements Task, ExecutionObserver { /** * The log object used for debugging. */ - private static final Log LOG = LogFactory.getLog(Task.class); + private static final Log LOG = LogFactory.getLog(RuntimeTask.class); private static final long NANO_TO_MILLISECONDS = 1000 * 1000; @@ -86,12 +86,13 @@ public final class RuntimeTask implements Task, ExecutionObserver { private long startTime; - public RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, final TaskManager taskManager) { + public RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, + final TaskManager taskManager) { this.vertexID = vertexID; this.environment = environment; this.taskManager = taskManager; - + this.environment.setExecutionObserver(this); } @@ -120,8 +121,8 @@ public void executionStateChanged(final ExecutionState newExecutionState, final this.executionState = newExecutionState; // Finally propagate the state change to the job manager - this.taskManager.executionStateChanged(this.environment.getJobID(), this.vertexID, this, - newExecutionState, optionalMessage); + this.taskManager.executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState, + optionalMessage); } /** @@ -489,4 +490,13 @@ public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEn return new RuntimeTaskContext(this, transferEnvelopeDispatcher, tasksWithUndecidedCheckpoints); } + + /** + * {@inheritDoc} + */ + @Override + public ExecutionState getExecutionState() { + + return this.executionState; + } } From e6a1cccee4f8cb5e885d853c54680a66e496a6bf Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 2 Feb 2012 18:46:32 +0100 Subject: [PATCH 193/310] Implemented ReplayOutputChannelContext --- .../ReplayOutputChannelContext.java | 46 ++++++++++++++----- .../ReplayOutputGateContext.java | 25 +++++++++- .../nephele/checkpointing/ReplayTask.java | 7 ++- .../checkpointing/ReplayTaskContext.java | 26 ++++++++++- .../nephele/checkpointing/ReplayThread.java | 3 +- .../jobmanager/scheduler/RecoveryLogic.java | 3 ++ .../AbstractOutputChannelContext.java | 2 +- .../runtime/ForwardingBarrier.java | 2 +- .../runtime/RuntimeDispatcher.java | 2 +- .../runtime/RuntimeInputGateContext.java | 2 + .../runtime/RuntimeOutputGateContext.java | 2 +- .../runtime/RuntimeTaskContext.java | 2 + .../taskmanager/runtime/SpillingBarrier.java | 2 +- 13 files changed, 99 insertions(+), 25 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java index df1f335c5d85d..e52229d505016 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java @@ -3,17 +3,24 @@ import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -public final class ReplayOutputChannelContext implements OutputChannelContext { +public final class ReplayOutputChannelContext extends AbstractOutputChannelContext implements OutputChannelContext { + + private final JobID jobID; private final ChannelID channelID; - + private final OutputChannelContext encapsulatedContext; - ReplayOutputChannelContext(final ChannelID channelID, final OutputChannelContext encapsulatedContext) { - + ReplayOutputChannelContext(final JobID jobID, final ChannelID channelID, + final IncomingEventQueue incomingEventQueue, final OutputChannelContext encapsulatedContext) { + super(incomingEventQueue); + + this.jobID = jobID; this.channelID = channelID; this.encapsulatedContext = encapsulatedContext; } @@ -23,14 +30,17 @@ public final class ReplayOutputChannelContext implements OutputChannelContext { */ @Override public boolean isInputChannel() { - // TODO Auto-generated method stub + return true; } + /** + * {@inheritDoc} + */ @Override public JobID getJobID() { - // TODO Auto-generated method stub - return null; + + return this.jobID; } /** @@ -38,7 +48,7 @@ public JobID getJobID() { */ @Override public ChannelID getChannelID() { - + return this.channelID; } @@ -54,16 +64,28 @@ public ChannelType getType() { return null; } + /** + * {@inheritDoc} + */ @Override - public void queueTransferEnvelope(TransferEnvelope transferEnvelope) { - // TODO Auto-generated method stub + public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { + + super.queueTransferEnvelope(transferEnvelope); + if (this.encapsulatedContext != null) { + this.encapsulatedContext.queueTransferEnvelope(transferEnvelope); + } } + /** + * {@inheritDoc} + */ @Override public void releaseAllResources() { - // TODO Auto-generated method stub - + + if (this.encapsulatedContext != null) { + this.encapsulatedContext.releaseAllResources(); + } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java index d40b1619e8eb6..f4cc2f19d319b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java @@ -2,15 +2,24 @@ import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent; +import eu.stratosphere.nephele.taskmanager.runtime.ForwardingBarrier; +import eu.stratosphere.nephele.taskmanager.runtime.SpillingBarrier; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; final class ReplayOutputGateContext extends AbstractReplayGateContext implements OutputGateContext { - ReplayOutputGateContext(final GateID gateID) { + private final ReplayTaskContext taskContext; + + ReplayOutputGateContext(final ReplayTaskContext taskContext, final GateID gateID) { super(gateID); + + this.taskContext = taskContext; } @Override @@ -21,7 +30,19 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp activateForwardingBarrier(previousContext); } - return new ReplayOutputChannelContext(channelID, previousContext); + // Construct new forwarding chain for the replay output channel context + final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(); + final IncomingEventQueue incomingEventQueue = AbstractOutputChannelContext + .createIncomingEventQueue(forwardingChain); + final ReplayOutputBroker outputBroker = new ReplayOutputBroker(forwardingChain, incomingEventQueue); + forwardingChain.addForwarder(outputBroker); + forwardingChain.addForwarder(new ForwardingBarrier(channelID)); + forwardingChain.addForwarder(new SpillingBarrier(isReceiverRunning, mergeSpillBuffers)); + forwardingChain.addForwarder(this.taskContext.getRuntimeDispatcher()); + + // Register output broker + + return new ReplayOutputChannelContext(null, channelID, incomingEventQueue, previousContext); } private static void activateForwardingBarrier(final OutputChannelContext previousContext) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index cd0643231c453..1f8ae9ddf96ff 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -21,7 +21,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.io.OutputBuffer; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; @@ -57,6 +56,8 @@ private ReplayTaskExecutionObserver(final RuntimeTask encapsulatedTask) { @Override public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) { + System.out.println("Execution state changed to " + newExecutionState + ", " + optionalMessage); + if (this.encapsulatedTask == null) { replayTaskExecutionState = newExecutionState; } else { @@ -291,7 +292,7 @@ public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) { public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final Map tasksWithUndecidedCheckpoints) { - return new ReplayTaskContext(); + return new ReplayTaskContext(this, transferEnvelopeDispatcher); } private void reportExecutionStateChange(final boolean replayTaskStateChanged, final String optionalMessage) { @@ -307,8 +308,6 @@ private void reportExecutionStateChange(final boolean replayTaskStateChanged, fi } else { - - } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java index b159afce08e51..899913e71b0d0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java @@ -1,22 +1,37 @@ package eu.stratosphere.nephele.checkpointing; import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeDispatcher; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; final class ReplayTaskContext implements TaskContext { + private final ReplayTask task; + + private final RuntimeDispatcher runtimeDispatcher; + + ReplayTaskContext(final ReplayTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher) { + this.task = task; + this.runtimeDispatcher = new RuntimeDispatcher(transferEnvelopeDispatcher); + } + /** * {@inheritDoc} */ @Override public OutputGateContext createOutputGateContext(final GateID gateID) { - return new ReplayOutputGateContext(gateID); + return new ReplayOutputGateContext(this, gateID); } + /** + * {@inheritDoc} + */ @Override public InputGateContext createInputGateContext(final GateID gateID) { // TODO Auto-generated method stub @@ -32,4 +47,13 @@ public LocalBufferPoolOwner getLocalBufferPoolOwner() { return null; } + void registerReplayOutputBroker(final ChannelID channelID, final ReplayOutputBroker outputBroker) { + + this.task.registerReplayOutputBroker(channelID, outputBroker); + } + + RuntimeDispatcher getRuntimeDispatcher() { + + return this.runtimeDispatcher; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index 4a3727afc6b6d..58ec3d2e69cac 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -136,7 +136,7 @@ void restart() { } private void replayCheckpoint() throws Exception { - + final CheckpointDeserializer deserializer = new CheckpointDeserializer(this.vertexID); int metaDataIndex = 0; @@ -209,6 +209,7 @@ private void outputEnvelope(final TransferEnvelope transferEnvelope) throws IOEx throw new IOException("Cannot find output broker for channel " + transferEnvelope.getSource()); } + System.out.println("Emitting envelope " + transferEnvelope.getSequenceNumber()); outputBroker.outputEnvelope(transferEnvelope); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index 706847643ac94..ed4f4b9ad7fed 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -126,6 +126,8 @@ private static void findVerticesToRestart(final ExecutionVertex failedVertex, final Set visited = new HashSet(); verticesToTest.add(failedVertex); + System.out.println("++++" + failedVertex + " failed"); + while (!verticesToTest.isEmpty()) { final ExecutionVertex vertex = verticesToTest.poll(); @@ -137,6 +139,7 @@ private static void findVerticesToRestart(final ExecutionVertex failedVertex, // Predecessors must be either checkpoints or need to be restarted, too for (int j = 0; j < vertex.getNumberOfPredecessors(); j++) { final ExecutionVertex predecessor = vertex.getPredecessor(j); + System.out.println("++++ Predecessor " + predecessor + " has checkpoint state " + predecessor.getCheckpointState()); if (predecessor.getCheckpointState() != CheckpointState.PARTIAL && predecessor.getCheckpointState() != CheckpointState.COMPLETE) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java index 86e4434fb475a..f59474883b95b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java @@ -26,7 +26,7 @@ public static IncomingEventQueue createIncomingEventQueue(final OutputChannelFor * {@inheritDoc} */ @Override - public final void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { + public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { if (transferEnvelope.getBuffer() != null) { throw new IllegalStateException("Transfer envelope for output channel has buffer attached"); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java index 5212dda1cb557..d1c3d2b9e4c1b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java @@ -32,7 +32,7 @@ public final class ForwardingBarrier implements OutputChannelForwarder { private int forwardingBarrier = -1; - ForwardingBarrier(final ChannelID outputChannelID) { + public ForwardingBarrier(final ChannelID outputChannelID) { this.outputChannelID = outputChannelID; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java index 4428225b99392..53c8c44251e67 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java @@ -11,7 +11,7 @@ public final class RuntimeDispatcher implements OutputChannelForwarder { private final TransferEnvelopeDispatcher dispatcher; - RuntimeDispatcher(final TransferEnvelopeDispatcher dispatcher) { + public RuntimeDispatcher(final TransferEnvelopeDispatcher dispatcher) { this.dispatcher = dispatcher; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index abd95848819b0..f27d8b04f8844 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -114,6 +114,8 @@ public void setDesignatedNumberOfBuffers(int numberOfBuffers) { @Override public void clearLocalBufferPool() { + System.out.println("Clearing local input gate context: " + this.localBufferPool.getNumberOfAvailableBuffers()); + this.localBufferPool.clear(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java index a3cc0e4c79db0..93a017eb4220e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java @@ -158,7 +158,7 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp forwardingChain.addForwarder(checkpoint); } forwardingChain.addForwarder(new ForwardingBarrier(channelID)); - forwardingChain.addForwarder(new SpillingBarrier(isReceiverRunning)); + forwardingChain.addForwarder(new SpillingBarrier(isReceiverRunning, mergeSpillBuffers)); forwardingChain.addForwarder(this.taskContext.getRuntimeDispatcher()); return new RuntimeOutputChannelContext(outputChannel, incomingEventQueue); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 49d18e150bddd..194b248a4631c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -166,6 +166,8 @@ void reportExhaustionOfMemoryBuffers() { if (!this.initialExhaustionOfMemoryBuffersReported) { + System.out.println(this.task.getEnvironment().getTaskName() + " has buffers exhausted"); + this.task.initialExecutionResourcesExhausted(); this.initialExhaustionOfMemoryBuffersReported = true; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java index 10ec1a823f48b..b146bce097eae 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java @@ -20,7 +20,7 @@ public final class SpillingBarrier implements OutputChannelForwarder { */ private boolean isReceiverRunning = false; - SpillingBarrier(final boolean isReceiverRunning) { + public SpillingBarrier(final boolean isReceiverRunning, final boolean mergeSpillBuffers) { this.isReceiverRunning = isReceiverRunning; this.queuedOutgoingEnvelopes = null; } From ca0c7c62fc87df08fcaefd0a8e093c6bc2d31fd7 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 2 Feb 2012 19:12:29 +0100 Subject: [PATCH 194/310] Improved robustness of forwarding barrier --- .../nephele/taskmanager/runtime/ForwardingBarrier.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java index d1c3d2b9e4c1b..febc3fdb5bd9f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java @@ -45,7 +45,7 @@ public boolean forward(final TransferEnvelope transferEnvelope) { if (transferEnvelope.getSequenceNumber() < this.forwardingBarrier) { return false; } - + return true; } @@ -64,8 +64,11 @@ public void processEvent(final AbstractEvent event) { if (event instanceof UnexpectedEnvelopeEvent) { final UnexpectedEnvelopeEvent uee = (UnexpectedEnvelopeEvent) event; - this.forwardingBarrier = uee.getExpectedSequenceNumber(); - LOG.info("Setting forwarding barrier to sequence number " + this.forwardingBarrier + " for output channel " + this.outputChannelID); + if (uee.getExpectedSequenceNumber() > this.forwardingBarrier) { + this.forwardingBarrier = uee.getExpectedSequenceNumber(); + LOG.info("Setting forwarding barrier to sequence number " + this.forwardingBarrier + + " for output channel " + this.outputChannelID); + } } } } From 91a3530c89f11e0b8f394a530225e524ad955500 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 2 Feb 2012 21:21:31 +0100 Subject: [PATCH 195/310] Implemented execution state update for ReplayTask --- .../CheckpointReplayManager.java | 52 ------- .../nephele/checkpointing/ReplayTask.java | 139 ++++++++++++++++-- .../executiongraph/ExecutionVertex.java | 2 +- 3 files changed, 128 insertions(+), 65 deletions(-) delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java deleted file mode 100644 index 8444011524853..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointReplayManager.java +++ /dev/null @@ -1,52 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.checkpointing; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; - -public class CheckpointReplayManager { - - private static final Log LOG = LogFactory.getLog(CheckpointReplayManager.class); - - private final TransferEnvelopeDispatcher transferEnvelopeDispatcher; - - public CheckpointReplayManager(final TransferEnvelopeDispatcher transferEnvelopeDispatcher) { - - this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; - } - - - - /*public void replayCheckpoint(final ExecutionVertexID vertexID) { - - final CheckpointReplayTask newReplayTask = new CheckpointReplayTask(this, vertexID, this.checkpointDirectory, - this.transferEnvelopeDispatcher, hasCompleteCheckpointAvailable(vertexID)); - - final CheckpointReplayTask runningReplayTask = this.runningReplayTasks.put(vertexID, newReplayTask); - if (runningReplayTask != null) { - LOG.info("There is already a replay task running for task " + vertexID + ", cancelling it first..."); - runningReplayTask.cancelAndWait(); - } - - LOG.info("Replaying checkpoint for vertex " + vertexID); - newReplayTask.start(); - }*/ - - -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index 1f8ae9ddf96ff..2f13c2845d09b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -18,6 +18,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -38,7 +39,9 @@ import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; +import eu.stratosphere.nephele.template.AbstractInvokable; import eu.stratosphere.nephele.template.InputSplitProvider; +import eu.stratosphere.nephele.util.StringUtils; public final class ReplayTask implements Task { @@ -56,8 +59,6 @@ private ReplayTaskExecutionObserver(final RuntimeTask encapsulatedTask) { @Override public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) { - System.out.println("Execution state changed to " + newExecutionState + ", " + optionalMessage); - if (this.encapsulatedTask == null) { replayTaskExecutionState = newExecutionState; } else { @@ -126,6 +127,9 @@ public boolean isCanceled() { private volatile ExecutionState replayTaskExecutionState = ExecutionState.STARTING; + private final AtomicReference overallExecutionState = new AtomicReference( + ExecutionState.STARTING); + private AtomicBoolean replayThreadStarted = new AtomicBoolean(false); /** @@ -219,16 +223,90 @@ public void startExecution() { } } + /** + * {@inheritDoc} + */ @Override public void cancelExecution() { - // TODO Auto-generated method stub + System.out.println("+++++++ Cancel called"); + + cancelOrKillExecution(true); } + /** + * {@inheritDoc} + */ @Override public void killExecution() { - // TODO Auto-generated method stub + cancelOrKillExecution(false); + } + + /** + * Cancels or kills the task. + * + * @param cancel + * true/code> if the task shall be cancelled, false if it shall be killed + */ + private void cancelOrKillExecution(final boolean cancel) { + + final Thread replayThread = this.environment.getExecutingThread(); + Thread encapsulatedThread = null; + if (this.encapsulatedTask != null) { + encapsulatedThread = this.encapsulatedTask.getRuntimeEnvironment().getExecutingThread(); + } + + if (replayThread == null && encapsulatedThread == null) { + return; + } + + if (cancel) { + this.isCanceled = true; + this.replayTaskExecutionState = ExecutionState.CANCELING; + if (this.encapsulatedExecutionState != null) { + this.encapsulatedExecutionState = ExecutionState.CANCELING; + } + reportExecutionStateChange(true, null); + } + + // Request user code to shut down + if (this.encapsulatedTask != null) { + + try { + final AbstractInvokable invokable = this.encapsulatedTask.getRuntimeEnvironment().getInvokable(); + if (invokable != null) { + invokable.cancel(); + } + } catch (Throwable e) { + LOG.error(StringUtils.stringifyException(e)); + } + } + + // Continuously interrupt the threads until it changed to state CANCELED + while (true) { + + replayThread.interrupt(); + if (encapsulatedThread != null) { + encapsulatedThread.interrupt(); + } + + if (cancel) { + if (this.overallExecutionState.get() == ExecutionState.CANCELED) { + break; + } + } else { + if (this.overallExecutionState.get() == ExecutionState.FAILED) { + break; + } + } + + try { + Thread.sleep(100); + } catch (InterruptedException e) { + break; + } + } } /** @@ -297,17 +375,22 @@ public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEn private void reportExecutionStateChange(final boolean replayTaskStateChanged, final String optionalMessage) { - final JobID jobID = this.environment.getJobID(); - + ExecutionState candidateState; if (replayTaskStateChanged) { - - if (this.replayTaskExecutionState == ExecutionState.REPLAYING) { - this.taskManager.executionStateChanged(jobID, this.vertexID, this.replayTaskExecutionState, - optionalMessage); - } - + candidateState = determineOverallExecutionState(this.encapsulatedExecutionState, + this.replayTaskExecutionState); } else { + candidateState = determineOverallExecutionState(this.replayTaskExecutionState, + this.encapsulatedExecutionState); + } + if (candidateState == null) { + return; + } + + if (this.overallExecutionState.getAndSet(candidateState) != candidateState) { + this.taskManager.executionStateChanged(this.environment.getJobID(), this.vertexID, candidateState, + optionalMessage); } } @@ -325,4 +408,36 @@ public ExecutionState getExecutionState() { return null; } + private static ExecutionState determineOverallExecutionState(final ExecutionState unchangedExecutionState, + final ExecutionState changedExecutionState) { + + if (changedExecutionState == null) { + return changedExecutionState; + } + + if (changedExecutionState == ExecutionState.REPLAYING) { + + if (unchangedExecutionState == ExecutionState.RUNNING) { + return ExecutionState.REPLAYING; + } else { + return unchangedExecutionState; + } + } + + if (changedExecutionState == ExecutionState.CANCELING) { + return ExecutionState.CANCELING; + } + + if (changedExecutionState == ExecutionState.CANCELED && unchangedExecutionState == ExecutionState.CANCELED) { + return ExecutionState.CANCELED; + } + + if (changedExecutionState == ExecutionState.FINISHING && unchangedExecutionState == ExecutionState.FINISHING) { + return ExecutionState.FINISHING; + } + + System.out.println("Changed: " + changedExecutionState + " unchanged: " + unchangedExecutionState); + + return null; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 76472bdfd9bb6..85958401a0f69 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -659,7 +659,7 @@ public TaskCancelResult cancelTask() { } if (state != ExecutionState.RUNNING && state != ExecutionState.STARTING - && state != ExecutionState.FINISHING) { + && state != ExecutionState.FINISHING && state != ExecutionState.REPLAYING) { // Set to canceled directly updateExecutionState(ExecutionState.CANCELED, null); return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); From 1103ec2000a3cd8f0a0c450e5f2ae52ad76a1d10 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 2 Feb 2012 21:44:53 +0100 Subject: [PATCH 196/310] Improved robustness of task manager profiling --- .../profiling/impl/EnvironmentThreadSet.java | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentThreadSet.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentThreadSet.java index 24c5d26dcbc7f..e5179399e5094 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentThreadSet.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentThreadSet.java @@ -15,6 +15,7 @@ package eu.stratosphere.nephele.profiling.impl; +import java.lang.management.ThreadInfo; import java.lang.management.ThreadMXBean; import java.util.HashMap; import java.util.Iterator; @@ -117,10 +118,14 @@ private CPUUtilizationSnapshot createCPUUtilizationSnapshot(ThreadMXBean tmx, Th final long threadId = thread.getId(); + final ThreadInfo threadInfo = tmx.getThreadInfo(threadId); + if(threadInfo == null) { + return null; + } + return new CPUUtilizationSnapshot(timestamp, tmx.getThreadCpuTime(threadId) / NANO_TO_MILLISECONDS, tmx - .getThreadUserTime(threadId) - / NANO_TO_MILLISECONDS, tmx.getThreadInfo(threadId).getWaitedTime(), tmx.getThreadInfo(threadId) - .getBlockedTime()); + .getThreadUserTime(threadId) / NANO_TO_MILLISECONDS, threadInfo.getWaitedTime(), + threadInfo.getBlockedTime()); } public InternalExecutionVertexThreadProfilingData captureCPUUtilization(JobID jobID, ThreadMXBean tmx, @@ -129,7 +134,11 @@ public InternalExecutionVertexThreadProfilingData captureCPUUtilization(JobID jo synchronized (this.userThreads) { // Calculate utilization for main thread first - CPUUtilizationSnapshot newMainThreadSnapshot = createCPUUtilizationSnapshot(tmx, this.mainThread, timestamp); + final CPUUtilizationSnapshot newMainThreadSnapshot = createCPUUtilizationSnapshot(tmx, this.mainThread, timestamp); + if(newMainThreadSnapshot == null) { + return null; + } + final long mainInterval = newMainThreadSnapshot.getTimestamp() - this.mainThreadSnapshot.getTimestamp(); if (mainInterval == 0) { From 805a014a45e4951143e8377a690a7b394706ef7c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 2 Feb 2012 21:51:43 +0100 Subject: [PATCH 197/310] Finished implementation of ReplayTask --- .../nephele/checkpointing/ReplayTask.java | 24 +++++++++++++++---- .../nephele/checkpointing/ReplayThread.java | 1 - 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index 2f13c2845d09b..acf74959e2c99 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -130,7 +130,7 @@ public boolean isCanceled() { private final AtomicReference overallExecutionState = new AtomicReference( ExecutionState.STARTING); - private AtomicBoolean replayThreadStarted = new AtomicBoolean(false); + private final AtomicBoolean replayThreadStarted = new AtomicBoolean(false); /** * Stores whether the task has been canceled. @@ -203,9 +203,23 @@ public void markAsFailed() { reportExecutionStateChange(true, "Execution thread died unexpectedly"); } + /** + * {@inheritDoc} + */ @Override public boolean isTerminated() { - // TODO Auto-generated method stub + + if (this.encapsulatedTask != null) { + if (this.encapsulatedTask.isTerminated()) { + return true; + } + } + + final Thread executingThread = this.environment.getExecutingThread(); + if (executingThread.getState() == Thread.State.TERMINATED) { + return true; + } + return false; } @@ -229,8 +243,6 @@ public void startExecution() { @Override public void cancelExecution() { - System.out.println("+++++++ Cancel called"); - cancelOrKillExecution(true); } @@ -436,7 +448,9 @@ private static ExecutionState determineOverallExecutionState(final ExecutionStat return ExecutionState.FINISHING; } - System.out.println("Changed: " + changedExecutionState + " unchanged: " + unchangedExecutionState); + if (changedExecutionState == ExecutionState.FINISHED && unchangedExecutionState == ExecutionState.FINISHED) { + return ExecutionState.FINISHED; + } return null; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index 58ec3d2e69cac..0f15fb29eb30a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -209,7 +209,6 @@ private void outputEnvelope(final TransferEnvelope transferEnvelope) throws IOEx throw new IOException("Cannot find output broker for channel " + transferEnvelope.getSource()); } - System.out.println("Emitting envelope " + transferEnvelope.getSequenceNumber()); outputBroker.outputEnvelope(transferEnvelope); } } From 86be814637ab676cf49f35ffe39b8bfb702ab312 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 2 Feb 2012 22:43:08 +0100 Subject: [PATCH 198/310] Fixed bug related to detection of failed tasks --- .../nephele/checkpointing/ReplayTask.java | 56 ++++++++++++++----- .../nephele/taskmanager/TaskManager.java | 24 ++------ 2 files changed, 48 insertions(+), 32 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index acf74959e2c99..669854506065b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -47,10 +47,10 @@ public final class ReplayTask implements Task { private final class ReplayTaskExecutionObserver implements ExecutionObserver { - private final RuntimeTask encapsulatedTask; + private final RuntimeTask encapsulatedRuntimeTask; - private ReplayTaskExecutionObserver(final RuntimeTask encapsulatedTask) { - this.encapsulatedTask = encapsulatedTask; + private ReplayTaskExecutionObserver(final RuntimeTask encapsulatedRuntimeTask) { + this.encapsulatedRuntimeTask = encapsulatedRuntimeTask; } /** @@ -59,13 +59,24 @@ private ReplayTaskExecutionObserver(final RuntimeTask encapsulatedTask) { @Override public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) { - if (this.encapsulatedTask == null) { + if (this.encapsulatedRuntimeTask == null) { replayTaskExecutionState = newExecutionState; + + if (newExecutionState == ExecutionState.FAILED) { + if (encapsulatedTask != null) { + encapsulatedTask.killExecution(); + } + } + } else { encapsulatedExecutionState = newExecutionState; + + if (newExecutionState == ExecutionState.FAILED) { + killExecution(); + } } - reportExecutionStateChange((this.encapsulatedTask == null), optionalMessage); + reportExecutionStateChange((this.encapsulatedRuntimeTask == null), optionalMessage); } /** @@ -74,8 +85,8 @@ public void executionStateChanged(final ExecutionState newExecutionState, final @Override public void userThreadStarted(final Thread userThread) { - if (this.encapsulatedTask != null) { - this.encapsulatedTask.userThreadStarted(userThread); + if (this.encapsulatedRuntimeTask != null) { + this.encapsulatedRuntimeTask.userThreadStarted(userThread); } else { LOG.error("userThreadStarted called although there is no encapsulated task"); } @@ -87,8 +98,8 @@ public void userThreadStarted(final Thread userThread) { @Override public void userThreadFinished(final Thread userThread) { - if (this.encapsulatedTask != null) { - this.encapsulatedTask.userThreadFinished(userThread); + if (this.encapsulatedRuntimeTask != null) { + this.encapsulatedRuntimeTask.userThreadFinished(userThread); } else { LOG.error("userThreadFinished called although there is no encapsulated task"); } @@ -100,8 +111,8 @@ public void userThreadFinished(final Thread userThread) { @Override public boolean isCanceled() { - if (this.encapsulatedTask != null) { - if (this.encapsulatedTask.isCanceled()) { + if (this.encapsulatedRuntimeTask != null) { + if (this.encapsulatedRuntimeTask.isCanceled()) { return true; } } @@ -199,6 +210,9 @@ public Environment getEnvironment() { @Override public void markAsFailed() { + if (this.encapsulatedTask != null) { + this.encapsulatedTask.killExecution(); + } this.replayTaskExecutionState = ExecutionState.FAILED; reportExecutionStateChange(true, "Execution thread died unexpectedly"); } @@ -211,13 +225,25 @@ public boolean isTerminated() { if (this.encapsulatedTask != null) { if (this.encapsulatedTask.isTerminated()) { - return true; + + if (this.encapsulatedExecutionState != ExecutionState.FINISHED + && this.encapsulatedExecutionState != ExecutionState.CANCELED + && this.encapsulatedExecutionState != ExecutionState.FAILED) { + + return true; + } } } final Thread executingThread = this.environment.getExecutingThread(); if (executingThread.getState() == Thread.State.TERMINATED) { - return true; + + if (this.replayTaskExecutionState != ExecutionState.FINISHED + && this.replayTaskExecutionState != ExecutionState.CANCELED + && this.replayTaskExecutionState != ExecutionState.FAILED) { + + return true; + } } return false; @@ -452,6 +478,10 @@ private static ExecutionState determineOverallExecutionState(final ExecutionStat return ExecutionState.FINISHED; } + if (changedExecutionState == ExecutionState.FAILED && unchangedExecutionState == ExecutionState.FAILED) { + return ExecutionState.FAILED; + } + return null; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 94415635728b7..bcf1ed1992102 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -796,28 +796,14 @@ public synchronized boolean isShutDown() { */ private void checkTaskExecution() { - final List failedTasks = new ArrayList(); - - synchronized (this.runningTasks) { - - final Iterator it = this.runningTasks.keySet().iterator(); - while (it.hasNext()) { - final ExecutionVertexID executionVertexID = it.next(); - final Task task = this.runningTasks.get(executionVertexID); + final Iterator it = this.runningTasks.values().iterator(); + while (it.hasNext()) { + final Task task = it.next(); - if (task.isTerminated()) { - // Remove entry from the running tasks map - it.remove(); - // Don't to IPC call while holding a lock on the runningTasks map - failedTasks.add(task); - } + if (task.isTerminated()) { + task.markAsFailed(); } } - - final Iterator it2 = failedTasks.iterator(); - while (it2.hasNext()) { - it2.next().markAsFailed(); - } } /** From eb999a571e12a58c5163fe4867987e630ecf9589 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 3 Feb 2012 14:42:25 +0100 Subject: [PATCH 199/310] Implemented visual feedback for execution state REPLAYING --- .../visualization/swt/ColorScheme.java | 21 +++ .../nephele/visualization/swt/SWTGate.java | 6 + .../nephele/visualization/swt/SWTVertex.java | 162 +++++++++++++++++- 3 files changed, 185 insertions(+), 4 deletions(-) diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/ColorScheme.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/ColorScheme.java index 0972111e5d85a..bdbae755f6ffd 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/ColorScheme.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/ColorScheme.java @@ -29,6 +29,8 @@ public class ColorScheme { private final Color VERTEXRUNNINGBACKGROUNDCOLOR; + private final Color VERTEXREPLAYINGBACKGROUNDCOLOR; + private final Color VERTEXFINISHINGBACKGROUNDCOLOR; private final Color VERTEXFINISHEDBACKGROUNDCOLOR; @@ -46,6 +48,10 @@ public class ColorScheme { private final Color GATERUNNINGBACKGROUNDCOLOR; + private final Color GATEREPLAYINGBORDERCOLOR; + + private final Color GATEREPLAYINGBACKGROUNDCOLOR; + private final Color GATEFINISHINGBORDERCOLOR; private final Color GATEFINISHINGBACKGROUNDCOLOR; @@ -85,6 +91,7 @@ private ColorScheme(Device device) { this.VERTEXDEFAULTBACKGROUNDCOLOR = new Color(device, 162, 162, 162); this.VERTEXRUNNINGBACKGROUNDCOLOR = new Color(device, 155, 187, 89); + this.VERTEXREPLAYINGBACKGROUNDCOLOR = new Color(device, 155, 187, 89); this.VERTEXFINISHINGBACKGROUNDCOLOR = new Color(device, 130, 190, 255); this.VERTEXFINISHEDBACKGROUNDCOLOR = new Color(device, 93, 121, 246); this.VERTEXCANCELBACKGROUNDCOLOR = new Color(device, 247, 150, 70); @@ -94,6 +101,8 @@ private ColorScheme(Device device) { this.GATEDEFAULTBACKGROUNDCOLOR = new Color(device, 104, 104, 104); this.GATERUNNINGBORDERCOLOR = new Color(device, 155, 187, 89); this.GATERUNNINGBACKGROUNDCOLOR = new Color(device, 95, 114, 54); + this.GATEREPLAYINGBORDERCOLOR = new Color(device, 155, 187, 89); + this.GATEREPLAYINGBACKGROUNDCOLOR = new Color(device, 95, 114, 54); this.GATEFINISHINGBORDERCOLOR = new Color(device, 130, 190, 255); this.GATEFINISHINGBACKGROUNDCOLOR = new Color(device, 95, 135, 250); this.GATEFINISHEDBORDERCOLOR = new Color(device, 93, 121, 246); @@ -150,6 +159,10 @@ public static Color getVertexRunningBackgroundColor(Device device) { return getInstance(device).VERTEXRUNNINGBACKGROUNDCOLOR; } + public static Color getVertexReplayingBackgroundColor(Device device) { + return getInstance(device).VERTEXREPLAYINGBACKGROUNDCOLOR; + } + public static Color getGraphBackgroundColor(Device device) { return getInstance(device).GRAPHBACKGROUNDCOLOR; } @@ -170,6 +183,14 @@ public static Color getGateRunningBackgroundColor(Device device) { return getInstance(device).GATERUNNINGBACKGROUNDCOLOR; } + public static Color getGateReplayingBorderColor(Device device) { + return getInstance(device).GATEREPLAYINGBORDERCOLOR; + } + + public static Color getGateReplayingBackgroundColor(Device device) { + return getInstance(device).GATEREPLAYINGBACKGROUNDCOLOR; + } + public static Color getGateFinishingBorderColor(Device device) { return getInstance(device).GATEFINISHINGBORDERCOLOR; } diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTGate.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTGate.java index e377d1b0fa4ef..6d35be3c1ddbb 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTGate.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTGate.java @@ -78,6 +78,9 @@ private Color getBorderColor(Device device) { case RUNNING: returnColor = ColorScheme.getGateRunningBorderColor(device); break; + case REPLAYING: + returnColor = ColorScheme.getGateReplayingBorderColor(device); + break; case FINISHING: returnColor = ColorScheme.getGateFinishingBorderColor(device); break; @@ -108,6 +111,9 @@ private Color getBackgroundColor(Device device) { case RUNNING: returnColor = ColorScheme.getGateRunningBackgroundColor(device); break; + case REPLAYING: + returnColor = ColorScheme.getGateReplayingBackgroundColor(device); + break; case FINISHING: returnColor = ColorScheme.getGateFinishingBackgroundColor(device); break; diff --git a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVertex.java b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVertex.java index 82de4d22aeecd..b3f6bd25dc966 100644 --- a/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVertex.java +++ b/nephele/nephele-visualization/src/main/java/eu/stratosphere/nephele/visualization/swt/SWTVertex.java @@ -22,14 +22,21 @@ import org.eclipse.swt.graphics.GC; import org.eclipse.swt.widgets.Shell; +import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.managementgraph.ManagementVertex; public class SWTVertex extends AbstractSWTVertex { private final static int SPACEFORGATESINPERCENT = 20; + private final static int MINIMUMSIZEOFREPLAYICON = 40; + + private int gateHeight = 0; + private final ManagementVertex managementVertex; + private boolean replayMode = false; + public SWTVertex(AbstractSWTComponent parent, ManagementVertex managementVertex) { super(parent); @@ -67,17 +74,17 @@ public void layout() { outputGateWidth = getWidth() / numberOfOutputGates; } - int height = (int) ((double) getHeight() * (SPACEFORGATESINPERCENT / 100.0f)); + this.gateHeight = (int) ((double) getHeight() * (SPACEFORGATESINPERCENT / 100.0f)); it = getChildren(); while (it.hasNext()) { AbstractSWTComponent child = it.next(); if (child instanceof SWTGate) { SWTGate gate = (SWTGate) child; - gate.setHeight(height); + gate.setHeight(this.gateHeight); if (gate.isInputGate()) { gate.setX(getX() + numberOfLayoutedInputGates * inputGateWidth); - gate.setY(getY() + getHeight() - height); + gate.setY(getY() + getHeight() - this.gateHeight); gate.setWidth(inputGateWidth); numberOfLayoutedInputGates++; @@ -99,7 +106,7 @@ public boolean isSelectable() { } @Override - protected void paintInternal(GC gc, Device device) { + protected void paintInternal(final GC gc, final Device device) { final GroupVertexVisualizationData groupVertexVisualizationData = (GroupVertexVisualizationData) this.managementVertex .getGroupVertex().getAttachment(); @@ -112,6 +119,150 @@ protected void paintInternal(GC gc, Device device) { } gc.fillRectangle(this.rect.x, this.rect.y, this.rect.width, this.rect.height); + + // Check for replay mode update + final ExecutionState executionState = this.managementVertex.getExecutionState(); + switch (executionState) { + case REPLAYING: + this.replayMode = true; + break; + case RUNNING: + case FINISHED: + case CANCELED: + case FAILED: + this.replayMode = false; + break; + } + + if (this.replayMode) { + paintReplayIcon(gc, device); + } + } + + private void paintReplayIcon(final GC gc, final Device device) { + + // Determine dimension of replay icon + int dimension = getHeight(); + if (this.managementVertex.getNumberOfInputGates() > 0) { + dimension -= this.gateHeight; + } + if (this.managementVertex.getNumberOfOutputGates() > 0) { + dimension -= this.gateHeight; + } + dimension = Math.round((float) Math.min(dimension, getWidth()) * 0.7f); + + // Icon would be too small, do not draw it + if (dimension < MINIMUMSIZEOFREPLAYICON) { + return; + } + + // Determine coordinates of icon + final int x = getX() + (getWidth() / 2 - dimension / 2); + int height = getHeight(); + final int numberOfInputGates = this.managementVertex.getNumberOfInputGates(); + final int numberOfOutputGates = this.managementVertex.getNumberOfOutputGates(); + if ((numberOfInputGates > 0 && numberOfOutputGates == 0) + || (numberOfInputGates == 0 && numberOfOutputGates > 0)) { + height -= this.gateHeight; + } + int y = getY() + (height / 2 - dimension / 2); + if (this.managementVertex.getNumberOfOutputGates() > 0) { + y += this.gateHeight; + } + + gc.setBackground(getReplayIconBackgroundColor(device)); + gc.fillOval(x, y, dimension, dimension); + + final int triangleWidth = Math.round((float) (dimension / 2) * 0.6f); + final int triangleHeight = Math.round(dimension * 0.6f); + + final int triangleY = y + (dimension / 2 - triangleHeight / 2); + final int leftTriangleX = x + (dimension / 2 - triangleWidth); + final int rightTriangleX = x + (dimension / 2); + + paintReplayTriangle(gc, device, leftTriangleX, triangleY, triangleWidth, triangleHeight); + paintReplayTriangle(gc, device, rightTriangleX, triangleY, triangleWidth, triangleHeight); + } + + private void paintReplayTriangle(final GC gc, final Device device, final int x, final int y, final int width, + final int height) { + + gc.setBackground(getReplayIconForegroundColor(device)); + + final int[] polygon = new int[6]; + + polygon[0] = x; + polygon[1] = y + (height / 2); + polygon[2] = x + width; + polygon[3] = y; + polygon[4] = x + width; + polygon[5] = y + height; + + gc.fillPolygon(polygon); + } + + private Color getReplayIconForegroundColor(Device device) { + + Color returnColor = null; + + switch (this.managementVertex.getExecutionState()) { + case RUNNING: + returnColor = ColorScheme.getGateRunningBorderColor(device); + break; + case REPLAYING: + returnColor = ColorScheme.getGateReplayingBorderColor(device); + break; + case FINISHING: + returnColor = ColorScheme.getGateFinishingBorderColor(device); + break; + case FINISHED: + returnColor = ColorScheme.getGateFinishedBorderColor(device); + break; + case CANCELING: + case CANCELED: + returnColor = ColorScheme.getGateCancelBorderColor(device); + break; + case FAILED: + returnColor = ColorScheme.getGateFailedBorderColor(device); + break; + default: + returnColor = ColorScheme.getGateDefaultBorderColor(device); + break; + } + + return returnColor; + } + + private Color getReplayIconBackgroundColor(Device device) { + + Color returnColor = null; + + switch (this.managementVertex.getExecutionState()) { + case RUNNING: + returnColor = ColorScheme.getGateRunningBackgroundColor(device); + break; + case REPLAYING: + returnColor = ColorScheme.getGateReplayingBackgroundColor(device); + break; + case FINISHING: + returnColor = ColorScheme.getGateFinishingBackgroundColor(device); + break; + case FINISHED: + returnColor = ColorScheme.getGateFinishedBackgroundColor(device); + break; + case CANCELING: + case CANCELED: + returnColor = ColorScheme.getGateCancelBackgroundColor(device); + break; + case FAILED: + returnColor = ColorScheme.getGateFailedBackgroundColor(device); + break; + default: + returnColor = ColorScheme.getGateDefaultBackgroundColor(device); + break; + } + + return returnColor; } private Color getBackgroundColor(Device device) { @@ -122,6 +273,9 @@ private Color getBackgroundColor(Device device) { case RUNNING: returnColor = ColorScheme.getVertexRunningBackgroundColor(device); break; + case REPLAYING: + returnColor = ColorScheme.getVertexReplayingBackgroundColor(device); + break; case FINISHING: returnColor = ColorScheme.getVertexFinishingBackgroundColor(device); break; From 31e66590fdbbec663478b4e0e63a3c67f941e36d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 3 Feb 2012 14:56:47 +0100 Subject: [PATCH 200/310] Modified ExecutionStateTransition to reflect new state changes during task replay --- .../nephele/execution/ExecutionStateTransition.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java index 1384694c3ac74..af2b343a99827 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java @@ -60,9 +60,15 @@ public static void checkTransition(final String taskName, final ExecutionState o if (oldState == ExecutionState.STARTING && newState == ExecutionState.RUNNING) { unexpectedStateChange = false; } + if (oldState == ExecutionState.STARTING && newState == ExecutionState.REPLAYING) { + unexpectedStateChange = false; + } if (oldState == ExecutionState.RUNNING && newState == ExecutionState.FINISHING) { unexpectedStateChange = false; } + if (oldState == ExecutionState.REPLAYING && newState == ExecutionState.FINISHING) { + unexpectedStateChange = false; + } if (oldState == ExecutionState.FINISHING && newState == ExecutionState.FINISHED) { unexpectedStateChange = false; } @@ -104,6 +110,12 @@ public static void checkTransition(final String taskName, final ExecutionState o */ unexpectedStateChange = false; } + if (oldState == ExecutionState.RUNNING && newState == ExecutionState.ASSIGNED) { + /** + * This is a regular transition in case a task replay is triggered. + */ + unexpectedStateChange = false; + } if (oldState == ExecutionState.FINISHING && newState == ExecutionState.FAILED) { /** * This is a regular transition in case of a task error. From 3912a967439a7e4a31a4563811c2e3a4d2ed47a6 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 3 Feb 2012 18:15:10 +0100 Subject: [PATCH 201/310] Fixed possible resource leak during task recovery --- .../checkpointing/ReplayInputChannelContext.java | 4 ++-- .../checkpointing/ReplayOutputChannelContext.java | 4 ++-- .../checkpointing/ReplayOutputGateContext.java | 1 + .../bufferprovider/LocalBufferPool.java | 12 ++++++++---- .../bytebuffered/ByteBufferedChannelManager.java | 6 +++--- .../taskmanager/bytebuffered/ChannelContext.java | 2 +- .../runtime/RuntimeInputChannelContext.java | 14 +++++++++++++- .../runtime/RuntimeInputGateContext.java | 9 +++++---- .../runtime/RuntimeOutputChannelContext.java | 2 +- .../taskmanager/runtime/RuntimeTaskContext.java | 10 ++++++---- 10 files changed, 42 insertions(+), 22 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java index dc3a597fb7cf0..7435b3ac1b524 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java @@ -138,9 +138,9 @@ public int getNumberOfQueuedMemoryBuffers() { * {@inheritDoc} */ @Override - public void releaseAllResources() { + public void destroy() { - this.encapsulatedContext.releaseAllResources(); + this.encapsulatedContext.destroy(); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java index e52229d505016..da3b149077ca6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java @@ -81,10 +81,10 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { * {@inheritDoc} */ @Override - public void releaseAllResources() { + public void destroy() { if (this.encapsulatedContext != null) { - this.encapsulatedContext.releaseAllResources(); + this.encapsulatedContext.destroy(); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java index f4cc2f19d319b..5049a5f4456a9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java @@ -41,6 +41,7 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp forwardingChain.addForwarder(this.taskContext.getRuntimeDispatcher()); // Register output broker + this.taskContext.registerReplayOutputBroker(channelID, outputBroker); return new ReplayOutputChannelContext(null, channelID, incomingEventQueue, previousContext); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java index 45bf09e255e0d..889d56620fd81 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java @@ -30,6 +30,8 @@ public final class LocalBufferPool implements BufferProvider { private final static Log LOG = LogFactory.getLog(LocalBufferPool.class); + private final String ownerName; + private final GlobalBufferPool globalBufferPool; private final int maximumBufferSize; @@ -46,9 +48,10 @@ public final class LocalBufferPool implements BufferProvider { private final Queue buffers = new ArrayDeque(); - public LocalBufferPool(final int designatedNumberOfBuffers, final boolean isShared, + public LocalBufferPool(final String ownerName, final int designatedNumberOfBuffers, final boolean isShared, final AsynchronousEventListener eventListener) { + this.ownerName = ownerName; this.globalBufferPool = GlobalBufferPool.getInstance(); this.maximumBufferSize = this.globalBufferPool.getMaximumBufferSize(); this.designatedNumberOfBuffers = designatedNumberOfBuffers; @@ -56,8 +59,8 @@ public LocalBufferPool(final int designatedNumberOfBuffers, final boolean isShar this.eventListener = eventListener; } - public LocalBufferPool(final int designatedNumberOfBuffers, final boolean isShared) { - this(designatedNumberOfBuffers, isShared, null); + public LocalBufferPool(final String ownerName, final int designatedNumberOfBuffers, final boolean isShared) { + this(ownerName, designatedNumberOfBuffers, isShared, null); } /** @@ -190,7 +193,8 @@ public void clear() { synchronized (this.buffers) { if (this.requestedNumberOfBuffers != this.buffers.size()) { - LOG.error("Possible resource leak: Requested number of buffers is " + this.requestedNumberOfBuffers + + LOG.error(this.ownerName + ": Requested number of buffers is " + this.requestedNumberOfBuffers + ", but only " + this.buffers.size() + " buffers in local pool"); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 6556892e44d77..94e3b55168320 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -105,7 +105,7 @@ public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupServi GlobalBufferPool.getInstance(); // Initialize the transit buffer pool - this.transitBufferPool = new LocalBufferPool(128, true); + this.transitBufferPool = new LocalBufferPool("Transit buffer pool", 128, true); this.networkConnectionManager = new NetworkConnectionManager(this, localInstanceConnectionInfo.getAddress(), localInstanceConnectionInfo.getDataPort()); @@ -232,7 +232,7 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { final ChannelID outputChannelID = channelIterator.next(); final ChannelContext context = this.registeredChannels.remove(outputChannelID); if (context != null) { - context.releaseAllResources(); + context.destroy(); } this.receiverCache.remove(outputChannelID); } @@ -244,7 +244,7 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { final ChannelID outputChannelID = channelIterator.next(); final ChannelContext context = this.registeredChannels.remove(outputChannelID); if (context != null) { - context.releaseAllResources(); + context.destroy(); } this.receiverCache.remove(outputChannelID); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java index 6346b2a478dd0..0ec2b6dc3885c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java @@ -34,5 +34,5 @@ public interface ChannelContext { void queueTransferEnvelope(TransferEnvelope transferEnvelope); - void releaseAllResources(); + void destroy(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java index 7072d21ac5099..93b67e8e2a3aa 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java @@ -52,6 +52,8 @@ final class RuntimeInputChannelContext implements InputChannelContext, ByteBuffe private int lastReceivedEnvelope = -1; + private boolean destroyCalled = false; + RuntimeInputChannelContext(final RuntimeInputGateContext inputGateContext, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final AbstractByteBufferedInputChannel byteBufferedInputChannel) { @@ -168,6 +170,14 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { synchronized (this.queuedEnvelopes) { + if (this.destroyCalled) { + final Buffer buffer = transferEnvelope.getBuffer(); + if (buffer != null) { + buffer.recycleBuffer(); + } + return; + } + final int expectedSequenceNumber = this.lastReceivedEnvelope + 1; if (sequenceNumber != expectedSequenceNumber) { @@ -237,12 +247,14 @@ public boolean isInputChannel() { * {@inheritDoc} */ @Override - public void releaseAllResources() { + public void destroy() { final Queue buffersToRecycle = new ArrayDeque(); synchronized (this.queuedEnvelopes) { + this.destroyCalled = true; + while (!this.queuedEnvelopes.isEmpty()) { final TransferEnvelope envelope = this.queuedEnvelopes.poll(); if (envelope.getBuffer() != null) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index f27d8b04f8844..6eb61c8774122 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -39,10 +39,13 @@ final class RuntimeInputGateContext implements BufferProvider, InputGateContext, private final InputGate inputGate; - RuntimeInputGateContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + RuntimeInputGateContext(final String taskName, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final InputGate inputGate) { - this.localBufferPool = new LocalBufferPool(1, false); + final String poolOwnerName = (taskName == null ? "Unknown task" : taskName + " (Input Gate " + + inputGate.getIndex() + ")"); + + this.localBufferPool = new LocalBufferPool(poolOwnerName, 1, false); this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.inputGate = inputGate; @@ -114,8 +117,6 @@ public void setDesignatedNumberOfBuffers(int numberOfBuffers) { @Override public void clearLocalBufferPool() { - System.out.println("Clearing local input gate context: " + this.localBufferPool.getNumberOfAvailableBuffers()); - this.localBufferPool.clear(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java index 431b19f503ead..a106f7f255225 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java @@ -82,7 +82,7 @@ public ChannelType getType() { * {@inheritDoc} */ @Override - public void releaseAllResources() { + public void destroy() { // TODO Auto-generated method stub } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 194b248a4631c..001a7a2589fea 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -60,7 +60,10 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve RuntimeTaskContext(final RuntimeTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final Map tasksWithUndecidedCheckpoints) { - this.localBufferPool = new LocalBufferPool(1, false, this); + final String poolOwnerName = (task.getEnvironment().getTaskName() == null ? "Unkown task" : task + .getEnvironment().getTaskName()); + + this.localBufferPool = new LocalBufferPool(poolOwnerName, 1, false, this); this.task = task; final RuntimeEnvironment environment = task.getRuntimeEnvironment(); @@ -166,8 +169,6 @@ void reportExhaustionOfMemoryBuffers() { if (!this.initialExhaustionOfMemoryBuffersReported) { - System.out.println(this.task.getEnvironment().getTaskName() + " has buffers exhausted"); - this.task.initialExecutionResourcesExhausted(); this.initialExhaustionOfMemoryBuffersReported = true; } @@ -272,7 +273,8 @@ public InputGateContext createInputGateContext(final GateID gateID) { throw new IllegalStateException("Cannot find input gate with ID " + gateID); } - return new RuntimeInputGateContext(this.transferEnvelopeDispatcher, inputGate); + return new RuntimeInputGateContext(this.task.getEnvironment().getTaskName(), this.transferEnvelopeDispatcher, + inputGate); } /** From 09c443fdc5e478160e556b7d253a7b952762c535 Mon Sep 17 00:00:00 2001 From: marrus Date: Fri, 3 Feb 2012 18:45:34 +0100 Subject: [PATCH 202/310] implementation auf pigMix1 --- .../stratosphere/pact/example/pigmix/L1.java | 101 ++++++++++ .../stratosphere/pact/example/pigmix/L10.java | 66 +++++++ .../stratosphere/pact/example/pigmix/L11.java | 121 ++++++++++++ .../stratosphere/pact/example/pigmix/L12.java | 187 ++++++++++++++++++ .../stratosphere/pact/example/pigmix/L3.java | 118 +++++++++++ .../stratosphere/pact/example/pigmix/L4.java | 84 ++++++++ .../stratosphere/pact/example/pigmix/L5.java | 100 ++++++++++ .../stratosphere/pact/example/pigmix/L6.java | 82 ++++++++ .../stratosphere/pact/example/pigmix/L7.java | 95 +++++++++ .../stratosphere/pact/example/pigmix/L8.java | 90 +++++++++ .../stratosphere/pact/example/pigmix/L9.java | 64 ++++++ .../pact/example/pigmix/MultiOrderKey.java | 66 +++++++ 12 files changed, 1174 insertions(+) create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L1.java create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L10.java create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L11.java create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L12.java create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L3.java create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L4.java create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L5.java create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L6.java create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L7.java create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L8.java create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L9.java create mode 100644 pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/MultiOrderKey.java diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L1.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L1.java new file mode 100644 index 0000000000000..457650994ede0 --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L1.java @@ -0,0 +1,101 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.util.Iterator; +import java.util.List; + +import eu.stratosphere.pact.common.contract.FileDataSink; +import eu.stratosphere.pact.common.contract.FileDataSource; +import eu.stratosphere.pact.common.contract.MapContract; +import eu.stratosphere.pact.common.contract.MatchContract; +import eu.stratosphere.pact.common.contract.ReduceContract; +import eu.stratosphere.pact.common.io.RecordOutputFormat; +import eu.stratosphere.pact.common.io.TextInputFormat; +import eu.stratosphere.pact.common.plan.Plan; +import eu.stratosphere.pact.common.plan.PlanAssembler; +import eu.stratosphere.pact.common.stubs.Collector; +import eu.stratosphere.pact.common.stubs.MapStub; +import eu.stratosphere.pact.common.stubs.MatchStub; +import eu.stratosphere.pact.common.stubs.ReduceStub; +import eu.stratosphere.pact.common.type.PactRecord; +import eu.stratosphere.pact.common.type.base.PactInteger; +import eu.stratosphere.pact.common.type.base.PactString; + + +public class L1 implements PlanAssembler{ + + public static class ReadPageViews extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + if(fields.size() != 9){ + return; + } + int cnt = 0; + if (fields.get(1).toString() == "1") { + PactString throwAway = Library.mapLookup(fields.get(7), new PactString("a")); + cnt++; + } else { + List le = Library.splitLine(fields.get(8), ''); + for (PactString e : le) { + PactString throwAway = Library.mapLookup(e, new PactString("b")); + cnt++; + } + } + out.collect(new PactRecord(fields.get(0), new PactInteger(cnt))); + + } + } + } + + public static class Group extends ReduceStub{ + + @Override + public void reduce(Iterator records, Collector out) + throws Exception { + PactRecord rec = null; + int cnt = 0; + while (records.hasNext()) { + rec = records.next(); + cnt += rec.getField(1, PactInteger.class).getValue(); + } + out.collect(new PactRecord(rec.getField(0, PactString.class), new PactInteger(cnt))); + + } + + } + /* (non-Javadoc) + * @see eu.stratosphere.pact.common.plan.PlanAssembler#getPlan(java.lang.String[]) + */ + @Override + public Plan getPlan(String... args) + { + final int parallelism = args.length > 0 ? Integer.parseInt(args[0]) : 1; + final String pageViewsFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/page_views"; + + FileDataSource pageViews = new FileDataSource(TextInputFormat.class, pageViewsFile, "Read PageViews"); + pageViews.setDegreeOfParallelism(parallelism); + + + MapContract readPageViews = new MapContract(ReadPageViews.class, pageViews, "Project Page Views"); + readPageViews.setDegreeOfParallelism(parallelism); + + + ReduceContract group = new ReduceContract(Group.class, PactString.class, 0, readPageViews, "Group"); + group.setDegreeOfParallelism(40); + + FileDataSink sink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L1", group, "Result"); + sink.setDegreeOfParallelism(parallelism); + sink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 2); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, PactString.class); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, PactInteger.class); + + Plan plan = new Plan(sink, "L1 Load Page Views"); + return plan; + } +} diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L10.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L10.java new file mode 100644 index 0000000000000..96035f0d6307f --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L10.java @@ -0,0 +1,66 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.util.List; + +import eu.stratosphere.pact.common.contract.FileDataSink; +import eu.stratosphere.pact.common.contract.FileDataSource; +import eu.stratosphere.pact.common.contract.MapContract; +import eu.stratosphere.pact.common.contract.Order; +import eu.stratosphere.pact.common.io.RecordOutputFormat; +import eu.stratosphere.pact.common.io.TextInputFormat; +import eu.stratosphere.pact.common.plan.Plan; +import eu.stratosphere.pact.common.plan.PlanAssembler; +import eu.stratosphere.pact.common.stubs.Collector; +import eu.stratosphere.pact.common.stubs.MapStub; +import eu.stratosphere.pact.common.type.PactRecord; +import eu.stratosphere.pact.common.type.base.PactDouble; +import eu.stratosphere.pact.common.type.base.PactInteger; +import eu.stratosphere.pact.common.type.base.PactString; + +public class L10 implements PlanAssembler{ + + public static class ProjectPageViews extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + + MultiOrderKey mulitOrder = new MultiOrderKey(fields.get(3).getValue(), fields.get(6).getValue(), fields.get(2).getValue()); + rec.setField(0, mulitOrder); + out.collect(rec); + } + } + } + + /* (non-Javadoc) + * @see eu.stratosphere.pact.common.plan.PlanAssembler#getPlan(java.lang.String[]) + */ + @Override + public Plan getPlan(String... args) + { + final int parallelism = args.length > 0 ? Integer.parseInt(args[0]) : 1; + final String pageViewsFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/page_views"; + + FileDataSource pageViews = new FileDataSource(TextInputFormat.class, pageViewsFile, "Read PageViews"); + pageViews.setDegreeOfParallelism(parallelism); + + + MapContract projectPageViews = new MapContract(ProjectPageViews.class, pageViews, "Project Page Views"); + projectPageViews.setDegreeOfParallelism(parallelism); + + + FileDataSink sink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L10", projectPageViews, "Result"); + sink.setDegreeOfParallelism(parallelism); + sink.setGlobalOrder(Order.ASCENDING); + sink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 1); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, MultiOrderKey.class); + + Plan plan = new Plan(sink, "L10 order by multiple fields"); + return plan; + } +} diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L11.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L11.java new file mode 100644 index 0000000000000..c7c32df733f5e --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L11.java @@ -0,0 +1,121 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.util.Iterator; +import java.util.List; + +import eu.stratosphere.pact.common.contract.CoGroupContract; +import eu.stratosphere.pact.common.contract.FileDataSink; +import eu.stratosphere.pact.common.contract.FileDataSource; +import eu.stratosphere.pact.common.contract.MapContract; +import eu.stratosphere.pact.common.contract.ReduceContract; +import eu.stratosphere.pact.common.io.RecordOutputFormat; +import eu.stratosphere.pact.common.io.TextInputFormat; +import eu.stratosphere.pact.common.plan.Plan; +import eu.stratosphere.pact.common.plan.PlanAssembler; +import eu.stratosphere.pact.common.stubs.CoGroupStub; +import eu.stratosphere.pact.common.stubs.Collector; +import eu.stratosphere.pact.common.stubs.MapStub; +import eu.stratosphere.pact.common.stubs.ReduceStub; +import eu.stratosphere.pact.common.type.PactRecord; +import eu.stratosphere.pact.common.type.base.PactDouble; +import eu.stratosphere.pact.common.type.base.PactInteger; +import eu.stratosphere.pact.common.type.base.PactString; + + +public class L11 implements PlanAssembler{ + public static class ProjectPageViews extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + + rec.setField(0, fields.get(0)); + out.collect(rec); + } + } + } + public static class ProjectWideRow extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + + rec.setField(0, fields.get(0)); + out.collect(rec); + } + } + } + + + public static class Distinct extends ReduceStub{ + + @Override + public void reduce(Iterator records, Collector out) + throws Exception { + + out.collect(records.next()); + + } + } + + public static class UnionDistinct extends CoGroupStub { + + @Override + public void coGroup(Iterator records1, + Iterator records2, Collector out) { + if(!records1.hasNext()){ + out.collect(records2.next()); + }else{ + out.collect(records1.next()); + } + + } + } + + + /* (non-Javadoc) + * @see eu.stratosphere.pact.common.plan.PlanAssembler#getPlan(java.lang.String[]) + */ + @Override + public Plan getPlan(String... args) + { + final int parallelism = args.length > 0 ? Integer.parseInt(args[0]) : 1; + final String pageViewsFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/page_views"; + final String wideRowFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/widerow"; + FileDataSource pageViews = new FileDataSource(TextInputFormat.class, pageViewsFile, "Read PageViews"); + pageViews.setDegreeOfParallelism(parallelism); + FileDataSource wideRow = new FileDataSource(TextInputFormat.class, wideRowFile, "Read WideRow"); + wideRow.setDegreeOfParallelism(parallelism); + + MapContract projectPageViews = new MapContract(ProjectPageViews.class, pageViews, "Project Page Views"); + projectPageViews.setDegreeOfParallelism(parallelism); + MapContract projectWideRow = new MapContract(ProjectWideRow.class, pageViews, "Project Page Views"); + projectWideRow.setDegreeOfParallelism(parallelism); + + ReduceContract distinctPageView = new ReduceContract(Distinct.class, PactString.class, 0, projectPageViews, "Distinct PageView"); + distinctPageView.setDegreeOfParallelism(40); + ReduceContract distinctWideRow = new ReduceContract(Distinct.class, PactString.class, 0, projectWideRow, "Distinct Wide Row"); + distinctWideRow .setDegreeOfParallelism(40); + + CoGroupContract unionDistinct = new CoGroupContract(UnionDistinct.class, PactString.class, 0, 0, distinctPageView, distinctWideRow, "Join"); + unionDistinct.setDegreeOfParallelism(40); + FileDataSink sink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L11", unionDistinct, "Result"); + sink.setDegreeOfParallelism(parallelism); + sink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 2); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, PactInteger.class); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, PactDouble.class); + + Plan plan = new Plan(sink, "L11 distinct, union and widerow"); + return plan; + } +} diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L12.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L12.java new file mode 100644 index 0000000000000..bb05ccf7c1a30 --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L12.java @@ -0,0 +1,187 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.util.Iterator; +import java.util.List; + +import eu.stratosphere.pact.common.contract.FileDataSink; +import eu.stratosphere.pact.common.contract.FileDataSource; +import eu.stratosphere.pact.common.contract.MapContract; +import eu.stratosphere.pact.common.contract.ReduceContract; +import eu.stratosphere.pact.common.io.RecordOutputFormat; +import eu.stratosphere.pact.common.io.TextInputFormat; +import eu.stratosphere.pact.common.plan.Plan; +import eu.stratosphere.pact.common.plan.PlanAssembler; +import eu.stratosphere.pact.common.stubs.Collector; +import eu.stratosphere.pact.common.stubs.MapStub; +import eu.stratosphere.pact.common.stubs.ReduceStub; +import eu.stratosphere.pact.common.type.PactRecord; +import eu.stratosphere.pact.common.type.base.PactDouble; +import eu.stratosphere.pact.common.type.base.PactInteger; +import eu.stratosphere.pact.common.type.base.PactString; + + +public class L12 implements PlanAssembler{ + + public static class ProjectintoD extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + + if(fields.get(0).length() != 0 && fields.get(3).length() != 0){ + rec.setField(0, fields.get(0)); + rec.setField(1, fields.get(1)); + rec.setField(2, new PactInteger(Integer.parseInt(fields.get(2).getValue()))); + rec.setField(3, fields.get(3)); + rec.setField(4, new PactDouble(Double.parseDouble(fields.get(6).getValue()))); + out.collect(rec); + } + + } + } + } + + public static class HighestValuePagePerUser extends ReduceStub{ + + @Override + public void reduce(Iterator records, Collector out) + throws Exception { + double max = 0; + PactRecord rec = new PactRecord(); + while (records.hasNext()) { + rec = records.next(); + double d = rec.getField(4, PactDouble.class).getValue(); + max = max > d ? max : d; + } + + + out.collect(new PactRecord(rec.getField(0, PactString.class), new PactDouble(max))); + + } + } + public static class ProjectintoAleph extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + + if(fields.get(0).length() != 0 && fields.get(3).length() == 0){ + rec.setField(0, fields.get(1)); + out.collect(rec); + } + + } + } + } + + public static class QueriesPerAction extends ReduceStub{ + + @Override + public void reduce(Iterator records, Collector out) + throws Exception { + int count = 0; + PactRecord rec = new PactRecord(); + while (records.hasNext()) { + rec = records.next(); + count ++; + } + + out.collect(new PactRecord(rec.getField(0, PactString.class), new PactInteger(count))); + + } + } + public static class ProjectintoAlpha extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + + if(fields.get(0).length() == 0 ){ + rec.setField(0, fields.get(3)); + rec.setField(1, new PactInteger(Integer.parseInt(fields.get(2).getValue()))); + out.collect(rec); + } + + } + } + } + + public static class TotalTimespentPerTerm extends ReduceStub{ + + @Override + public void reduce(Iterator records, Collector out) + throws Exception { + double sum = 0; + PactRecord rec = new PactRecord(); + while (records.hasNext()) { + rec = records.next(); + sum += rec.getField(1, PactDouble.class).getValue(); + } + + out.collect(new PactRecord(rec.getField(0, PactString.class), new PactDouble(sum))); + + } + } + + + /* (non-Javadoc) + * @see eu.stratosphere.pact.common.plan.PlanAssembler#getPlan(java.lang.String[]) + */ + @Override + public Plan getPlan(String... args) + { + final int parallelism = args.length > 0 ? Integer.parseInt(args[0]) : 1; + final String pageViewsFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/page_views"; + FileDataSource pageViews = new FileDataSource(TextInputFormat.class, pageViewsFile, "Read PageViews"); + pageViews.setDegreeOfParallelism(parallelism); + + + MapContract projectintoD = new MapContract(ProjectintoD.class, pageViews, "Project into D"); + projectintoD.setDegreeOfParallelism(parallelism); + ReduceContract highestValuePagePerUser = new ReduceContract(HighestValuePagePerUser.class, PactString.class, 0, projectintoD, "HighestValuePagePerUser"); + highestValuePagePerUser.setDegreeOfParallelism(40); + FileDataSink sink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L12_HighestValuePagePerUser", highestValuePagePerUser, "Result"); + sink.setDegreeOfParallelism(parallelism); + sink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 2); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, PactString.class); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, PactDouble.class); + + MapContract projectintoaleph = new MapContract(ProjectintoAleph.class, pageViews, "Project into aleph"); + projectintoaleph.setDegreeOfParallelism(parallelism); + ReduceContract queriesPerAction = new ReduceContract(QueriesPerAction.class, PactString.class, 0, projectintoaleph, "QueriesPerAction"); + queriesPerAction.setDegreeOfParallelism(40); + FileDataSink querysink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L12_QueriesPerAction", queriesPerAction, "Result"); + querysink.setDegreeOfParallelism(parallelism); + querysink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 2); + querysink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, PactString.class); + querysink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, PactInteger.class); + + MapContract projectintoalpha = new MapContract(ProjectintoAlpha.class, pageViews, "Project into alpha"); + projectintoalpha.setDegreeOfParallelism(parallelism); + ReduceContract totalTimespentPerTerm = new ReduceContract(TotalTimespentPerTerm.class, PactString.class, 0, projectintoalpha, "TotalTimespentPerTerm"); + totalTimespentPerTerm.setDegreeOfParallelism(40); + FileDataSink timespentsink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L12_TotalTimespentPerTerm", totalTimespentPerTerm, "Result"); + timespentsink.setDegreeOfParallelism(parallelism); + timespentsink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 2); + timespentsink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, PactString.class); + timespentsink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, PactInteger.class); + + Plan plan = new Plan(sink, "L11 distinct, union and widerow"); + return plan; + } +} diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L3.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L3.java new file mode 100644 index 0000000000000..da184a1db4433 --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L3.java @@ -0,0 +1,118 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.util.Iterator; +import java.util.List; + +import eu.stratosphere.pact.common.contract.FileDataSink; +import eu.stratosphere.pact.common.contract.FileDataSource; +import eu.stratosphere.pact.common.contract.MapContract; +import eu.stratosphere.pact.common.contract.MatchContract; +import eu.stratosphere.pact.common.contract.ReduceContract; +import eu.stratosphere.pact.common.io.RecordOutputFormat; +import eu.stratosphere.pact.common.io.TextInputFormat; +import eu.stratosphere.pact.common.plan.Plan; +import eu.stratosphere.pact.common.plan.PlanAssembler; +import eu.stratosphere.pact.common.stubs.Collector; +import eu.stratosphere.pact.common.stubs.MapStub; +import eu.stratosphere.pact.common.stubs.MatchStub; +import eu.stratosphere.pact.common.stubs.ReduceStub; +import eu.stratosphere.pact.common.type.PactRecord; +import eu.stratosphere.pact.common.type.base.PactDouble; +import eu.stratosphere.pact.common.type.base.PactString; + +public class L3 implements PlanAssembler{ + public static class ProjectPageViews extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + + rec.setField(0, fields.get(0)); + rec.setField(1, new PactDouble(Double.parseDouble(fields.get(6).getValue()))); + out.collect(rec); + } + } + } + + public static class ProjectPowerUsers extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + rec.setField(0, fields.get(0)); + out.collect(rec); + } + } + } + + + public static class Join extends MatchStub { + @Override + public void match(PactRecord value1, PactRecord value2, Collector out) throws Exception + { + out.collect(value2); + } + } + public static class Group extends ReduceStub{ + + @Override + public void reduce(Iterator records, Collector out) + throws Exception { + PactRecord rec = null; + double cnt = 0; + while (records.hasNext()) { + rec = records.next(); + cnt += rec.getField(1, PactDouble.class).getValue(); + } + out.collect(new PactRecord(rec.getField(0, PactString.class), new PactDouble(cnt))); + + } + } + /* (non-Javadoc) + * @see eu.stratosphere.pact.common.plan.PlanAssembler#getPlan(java.lang.String[]) + */ + @Override + public Plan getPlan(String... args) + { + final int parallelism = args.length > 0 ? Integer.parseInt(args[0]) : 1; + final String pageViewsFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/page_views"; + final String powerUsersFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/power_users"; + + FileDataSource pageViews = new FileDataSource(TextInputFormat.class, pageViewsFile, "Read PageViews"); + pageViews.setDegreeOfParallelism(parallelism); + + FileDataSource powerUsers = new FileDataSource(TextInputFormat.class, powerUsersFile, "Read PowerUsers"); + powerUsers.setDegreeOfParallelism(parallelism); + + MapContract projectPageViews = new MapContract(ProjectPageViews.class, pageViews, "Project Page Views"); + projectPageViews.setDegreeOfParallelism(parallelism); + + MapContract projectPowerUsers = new MapContract(ProjectPowerUsers.class, powerUsers, "Project Power Users"); + projectPowerUsers.setDegreeOfParallelism(parallelism); + + MatchContract joiner = new MatchContract(Join.class, PactString.class, 0, 0, projectPowerUsers, projectPageViews, "Join"); + joiner.setDegreeOfParallelism(parallelism); + + ReduceContract group = new ReduceContract(Group.class, PactString.class, 0, joiner, "Group"); + group.setDegreeOfParallelism(40); + + FileDataSink sink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L3", group, "Result"); + sink.setDegreeOfParallelism(parallelism); + sink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 2); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, PactString.class); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, PactString.class); + + Plan plan = new Plan(sink, "L3 Big Join with Grouping"); + return plan; + } +} diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L4.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L4.java new file mode 100644 index 0000000000000..7e0d3f7e9e4ae --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L4.java @@ -0,0 +1,84 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; + +import eu.stratosphere.pact.common.contract.FileDataSink; +import eu.stratosphere.pact.common.contract.FileDataSource; +import eu.stratosphere.pact.common.contract.MapContract; +import eu.stratosphere.pact.common.contract.ReduceContract; +import eu.stratosphere.pact.common.io.RecordOutputFormat; +import eu.stratosphere.pact.common.io.TextInputFormat; +import eu.stratosphere.pact.common.plan.Plan; +import eu.stratosphere.pact.common.plan.PlanAssembler; +import eu.stratosphere.pact.common.stubs.Collector; +import eu.stratosphere.pact.common.stubs.MapStub; +import eu.stratosphere.pact.common.stubs.ReduceStub; +import eu.stratosphere.pact.common.type.PactRecord; +import eu.stratosphere.pact.common.type.base.PactInteger; +import eu.stratosphere.pact.common.type.base.PactString; + +public class L4 implements PlanAssembler{ + + public static class ProjectPageViews extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + + rec.setField(0, fields.get(0)); + rec.setField(1, fields.get(1)); + out.collect(rec); + } + } + } + public static class Group extends ReduceStub{ + + @Override + public void reduce(Iterator records, Collector out) + throws Exception { + PactRecord rec = null; + HashSet hash = new HashSet(); + while (records.hasNext()) { + rec = records.next(); + hash.add(rec.getField(1, PactString.class)); + } + out.collect(new PactRecord(rec.getField(0, PactString.class), new PactInteger(hash.size()))); + + } + } + /* (non-Javadoc) + * @see eu.stratosphere.pact.common.plan.PlanAssembler#getPlan(java.lang.String[]) + */ + @Override + public Plan getPlan(String... args) + { + final int parallelism = args.length > 0 ? Integer.parseInt(args[0]) : 1; + final String pageViewsFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/page_views"; + + FileDataSource pageViews = new FileDataSource(TextInputFormat.class, pageViewsFile, "Read PageViews"); + pageViews.setDegreeOfParallelism(parallelism); + + + MapContract projectPageViews = new MapContract(ProjectPageViews.class, pageViews, "Project Page Views"); + projectPageViews.setDegreeOfParallelism(parallelism); + + ReduceContract group = new ReduceContract(Group.class, PactString.class, 0, projectPageViews, "Group"); + group.setDegreeOfParallelism(40); + + FileDataSink sink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L4", group, "Result"); + sink.setDegreeOfParallelism(parallelism); + sink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 2); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, PactString.class); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, PactInteger.class); + + Plan plan = new Plan(sink, "L4 foreach + nested distinct"); + return plan; + } +} diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L5.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L5.java new file mode 100644 index 0000000000000..34d779b9c4542 --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L5.java @@ -0,0 +1,100 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.util.Iterator; +import java.util.List; + +import eu.stratosphere.pact.common.contract.CoGroupContract; +import eu.stratosphere.pact.common.contract.FileDataSink; +import eu.stratosphere.pact.common.contract.FileDataSource; +import eu.stratosphere.pact.common.contract.MapContract; +import eu.stratosphere.pact.common.io.RecordOutputFormat; +import eu.stratosphere.pact.common.io.TextInputFormat; +import eu.stratosphere.pact.common.plan.Plan; +import eu.stratosphere.pact.common.plan.PlanAssembler; +import eu.stratosphere.pact.common.stubs.CoGroupStub; +import eu.stratosphere.pact.common.stubs.Collector; +import eu.stratosphere.pact.common.stubs.MapStub; +import eu.stratosphere.pact.common.type.PactRecord; +import eu.stratosphere.pact.common.type.base.PactString; + +public class L5 implements PlanAssembler{ + public static class ProjectPageViews extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + rec.setField(0, fields.get(0)); + out.collect(rec); + } + } + } + + public static class ProjectPowerUsers extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + rec.setField(0, fields.get(0)); + out.collect(rec); + } + } + } + + + public static class Join extends CoGroupStub { + + @Override + public void coGroup(Iterator records1, + Iterator records2, Collector out) { + if(!records1.hasNext()){ + out.collect(records2.next()); + } + + } + } + + + /* (non-Javadoc) + * @see eu.stratosphere.pact.common.plan.PlanAssembler#getPlan(java.lang.String[]) + */ + @Override + public Plan getPlan(String... args) + { + final int parallelism = args.length > 0 ? Integer.parseInt(args[0]) : 1; + final String pageViewsFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/page_views"; + final String powerUsersFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/power_users"; + + FileDataSource pageViews = new FileDataSource(TextInputFormat.class, pageViewsFile, "Read PageViews"); + pageViews.setDegreeOfParallelism(parallelism); + + FileDataSource powerUsers = new FileDataSource(TextInputFormat.class, powerUsersFile, "Read PowerUsers"); + powerUsers.setDegreeOfParallelism(parallelism); + + MapContract projectPageViews = new MapContract(ProjectPageViews.class, pageViews, "Project Page Views"); + projectPageViews.setDegreeOfParallelism(parallelism); + + MapContract projectPowerUsers = new MapContract(ProjectPowerUsers.class, powerUsers, "Project Power Users"); + projectPowerUsers.setDegreeOfParallelism(parallelism); + + CoGroupContract joiner = new CoGroupContract(Join.class, PactString.class, 0, 0, projectPageViews, projectPowerUsers, "Join"); + joiner.setDegreeOfParallelism(parallelism); + + FileDataSink sink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L5", joiner, "Result"); + sink.setDegreeOfParallelism(parallelism); + sink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 1); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, PactString.class); + + Plan plan = new Plan(sink, "L5 Anti Join wihth CoGroup"); + return plan; + } +} diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L6.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L6.java new file mode 100644 index 0000000000000..31eeed7421c5c --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L6.java @@ -0,0 +1,82 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.util.Iterator; +import java.util.List; + +import eu.stratosphere.pact.common.contract.FileDataSink; +import eu.stratosphere.pact.common.contract.FileDataSource; +import eu.stratosphere.pact.common.contract.MapContract; +import eu.stratosphere.pact.common.contract.ReduceContract; +import eu.stratosphere.pact.common.io.RecordOutputFormat; +import eu.stratosphere.pact.common.io.TextInputFormat; +import eu.stratosphere.pact.common.plan.Plan; +import eu.stratosphere.pact.common.plan.PlanAssembler; +import eu.stratosphere.pact.common.stubs.Collector; +import eu.stratosphere.pact.common.stubs.MapStub; +import eu.stratosphere.pact.common.stubs.ReduceStub; +import eu.stratosphere.pact.common.type.PactRecord; +import eu.stratosphere.pact.common.type.base.PactInteger; +import eu.stratosphere.pact.common.type.base.PactString; + +public class L6 implements PlanAssembler{ + public static class ProjectTimeSpent extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + String key = fields.get(0).getValue() + ""+ fields.get(3).getValue() +""+ fields.get(4).getValue() + ""+fields.get(5).getValue(); + rec.setField(0,new PactString( key)); + rec.setField(1, new PactInteger(Integer.parseInt(fields.get(2).getValue()))); + out.collect(rec); + } + } + } + public static class Group extends ReduceStub{ + + @Override + public void reduce(Iterator records, Collector out) + throws Exception { + PactRecord rec = null; + int sum = 0; + while (records.hasNext()) { + rec = records.next(); + sum += rec.getField(1, PactInteger.class).getValue(); + } + out.collect(new PactRecord(rec.getField(0, PactString.class), new PactInteger(sum))); + + } + } + /* (non-Javadoc) + * @see eu.stratosphere.pact.common.plan.PlanAssembler#getPlan(java.lang.String[]) + */ + @Override + public Plan getPlan(String... args) + { + final int parallelism = args.length > 0 ? Integer.parseInt(args[0]) : 1; + final String pageViewsFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/page_views"; + + FileDataSource pageViews = new FileDataSource(TextInputFormat.class, pageViewsFile, "Read PageViews"); + pageViews.setDegreeOfParallelism(parallelism); + + + MapContract projectTimeSpent = new MapContract(ProjectTimeSpent.class, pageViews, "Project Time Spent"); + projectTimeSpent.setDegreeOfParallelism(parallelism); + + ReduceContract group = new ReduceContract(Group.class, PactString.class, 0, projectTimeSpent, "Group"); + group.setDegreeOfParallelism(40); + + FileDataSink sink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L6", group, "Result"); + sink.setDegreeOfParallelism(parallelism); + sink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 2); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, PactString.class); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, PactInteger.class); + + Plan plan = new Plan(sink, "L6 group with long key"); + return plan; + } +} diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L7.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L7.java new file mode 100644 index 0000000000000..a7355181908e1 --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L7.java @@ -0,0 +1,95 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.util.Iterator; +import java.util.List; + +import eu.stratosphere.pact.common.contract.FileDataSink; +import eu.stratosphere.pact.common.contract.FileDataSource; +import eu.stratosphere.pact.common.contract.MapContract; +import eu.stratosphere.pact.common.contract.ReduceContract; +import eu.stratosphere.pact.common.io.RecordOutputFormat; +import eu.stratosphere.pact.common.io.TextInputFormat; +import eu.stratosphere.pact.common.plan.Plan; +import eu.stratosphere.pact.common.plan.PlanAssembler; +import eu.stratosphere.pact.common.stubs.Collector; +import eu.stratosphere.pact.common.stubs.MapStub; +import eu.stratosphere.pact.common.stubs.ReduceStub; +import eu.stratosphere.pact.common.type.PactRecord; +import eu.stratosphere.pact.common.type.base.PactInteger; +import eu.stratosphere.pact.common.type.base.PactString; + +public class L7 implements PlanAssembler{ + + public static class ProjectPageViews extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + + rec.setField(0, fields.get(0)); + rec.setField(1, fields.get(5)); + out.collect(rec); + } + } + } + public static class Group extends ReduceStub{ + + @Override + public void reduce(Iterator records, Collector out) + throws Exception { + PactRecord rec = null; + int morning = 0; + int afternoon = 0; + while (records.hasNext()) { + rec = records.next(); + if (Integer.parseInt(rec.getField(1, PactString.class).getValue()) > 43200){ + morning++; + } + else{ + afternoon++; + } + } + PactRecord output = new PactRecord(); + output.setNumFields(3); + output.setField(0, rec.getField(0, PactString.class)); + output.setField(1, new PactInteger(morning)); + output.setField(2, new PactInteger(afternoon)); + out.collect(output); + + } + } + /* (non-Javadoc) + * @see eu.stratosphere.pact.common.plan.PlanAssembler#getPlan(java.lang.String[]) + */ + @Override + public Plan getPlan(String... args) + { + final int parallelism = args.length > 0 ? Integer.parseInt(args[0]) : 1; + final String pageViewsFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/page_views"; + + FileDataSource pageViews = new FileDataSource(TextInputFormat.class, pageViewsFile, "Read PageViews"); + pageViews.setDegreeOfParallelism(parallelism); + + + MapContract projectPageViews = new MapContract(ProjectPageViews.class, pageViews, "Project Page Views"); + projectPageViews.setDegreeOfParallelism(parallelism); + + ReduceContract group = new ReduceContract(Group.class, PactString.class, 0, projectPageViews, "Group"); + group.setDegreeOfParallelism(40); + + FileDataSink sink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L7", group, "Result"); + sink.setDegreeOfParallelism(parallelism); + sink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 2); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, PactString.class); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, PactInteger.class); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 2, PactInteger.class); + + Plan plan = new Plan(sink, "L7 nested plan with splits"); + return plan; + } +} diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L8.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L8.java new file mode 100644 index 0000000000000..9b0ef8cff2bda --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L8.java @@ -0,0 +1,90 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; + +import eu.stratosphere.pact.common.contract.FileDataSink; +import eu.stratosphere.pact.common.contract.FileDataSource; +import eu.stratosphere.pact.common.contract.MapContract; +import eu.stratosphere.pact.common.contract.ReduceContract; +import eu.stratosphere.pact.common.io.RecordOutputFormat; +import eu.stratosphere.pact.common.io.TextInputFormat; +import eu.stratosphere.pact.common.plan.Plan; +import eu.stratosphere.pact.common.plan.PlanAssembler; +import eu.stratosphere.pact.common.stubs.Collector; +import eu.stratosphere.pact.common.stubs.MapStub; +import eu.stratosphere.pact.common.stubs.ReduceStub; +import eu.stratosphere.pact.common.type.PactRecord; +import eu.stratosphere.pact.common.type.base.PactDouble; +import eu.stratosphere.pact.common.type.base.PactInteger; +import eu.stratosphere.pact.common.type.base.PactString; + +public class L8 implements PlanAssembler{ + public static class ProjectPageViews extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + + rec.setField(0, new PactString("all")); + rec.setField(1, new PactInteger(Integer.parseInt(fields.get(2).getValue()))); + rec.setField(2, new PactDouble(Double.parseDouble(fields.get(6).getValue()))); + out.collect(rec); + } + } + } + public static class Group extends ReduceStub{ + + @Override + public void reduce(Iterator records, Collector out) + throws Exception { + PactRecord rec = null; + int timespent = 0; + double revenue = 0; + int nrevenue = 0; + while (records.hasNext()) { + rec = records.next(); + timespent += rec.getField(1, PactInteger.class).getValue(); + revenue += rec.getField(2, PactDouble.class).getValue(); + nrevenue++; + } + PactRecord output = new PactRecord(new PactInteger(timespent), new PactDouble(revenue/nrevenue)); + out.collect(output); + + } + } + /* (non-Javadoc) + * @see eu.stratosphere.pact.common.plan.PlanAssembler#getPlan(java.lang.String[]) + */ + @Override + public Plan getPlan(String... args) + { + final int parallelism = args.length > 0 ? Integer.parseInt(args[0]) : 1; + final String pageViewsFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/page_views"; + + FileDataSource pageViews = new FileDataSource(TextInputFormat.class, pageViewsFile, "Read PageViews"); + pageViews.setDegreeOfParallelism(parallelism); + + + MapContract projectPageViews = new MapContract(ProjectPageViews.class, pageViews, "Project Page Views"); + projectPageViews.setDegreeOfParallelism(parallelism); + + ReduceContract group = new ReduceContract(Group.class, PactString.class, 0, projectPageViews, "Group all"); + group.setDegreeOfParallelism(40); + + FileDataSink sink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L8", group, "Result"); + sink.setDegreeOfParallelism(parallelism); + sink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 2); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, PactInteger.class); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, PactDouble.class); + + Plan plan = new Plan(sink, "L8 group all"); + return plan; + } +} diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L9.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L9.java new file mode 100644 index 0000000000000..fd7f05b52e691 --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/L9.java @@ -0,0 +1,64 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.util.List; + +import eu.stratosphere.pact.common.contract.FileDataSink; +import eu.stratosphere.pact.common.contract.FileDataSource; +import eu.stratosphere.pact.common.contract.MapContract; +import eu.stratosphere.pact.common.contract.Order; +import eu.stratosphere.pact.common.io.RecordOutputFormat; +import eu.stratosphere.pact.common.io.TextInputFormat; +import eu.stratosphere.pact.common.plan.Plan; +import eu.stratosphere.pact.common.plan.PlanAssembler; +import eu.stratosphere.pact.common.stubs.Collector; +import eu.stratosphere.pact.common.stubs.MapStub; +import eu.stratosphere.pact.common.type.PactRecord; +import eu.stratosphere.pact.common.type.base.PactString; + +public class L9 implements PlanAssembler{ + public static class ProjectPageViews extends MapStub + { + private final PactRecord rec = new PactRecord(); + + @Override + public void map(PactRecord record, Collector out) throws Exception + { + PactString str = record.getField(0, PactString.class); + if (str.length() > 0) { + List fields = Library.splitLine(str, ''); + + + rec.setField(0, fields.get(3)); + rec.setField(1, record.getField(0, PactString.class)); + out.collect(rec); + } + } + } + + /* (non-Javadoc) + * @see eu.stratosphere.pact.common.plan.PlanAssembler#getPlan(java.lang.String[]) + */ + @Override + public Plan getPlan(String... args) + { + final int parallelism = args.length > 0 ? Integer.parseInt(args[0]) : 1; + final String pageViewsFile = "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/pigmix625k/page_views"; + + FileDataSource pageViews = new FileDataSource(TextInputFormat.class, pageViewsFile, "Read PageViews"); + pageViews.setDegreeOfParallelism(parallelism); + + + MapContract projectPageViews = new MapContract(ProjectPageViews.class, pageViews, "Project Page Views"); + projectPageViews.setDegreeOfParallelism(parallelism); + + + FileDataSink sink = new FileDataSink(RecordOutputFormat.class, "hdfs://cloud-7.dima.tu-berlin.de:40010/pigmix/result_L9", projectPageViews, "Result"); + sink.setDegreeOfParallelism(parallelism); + sink.setGlobalOrder(Order.ASCENDING); + sink.getParameters().setInteger(RecordOutputFormat.NUM_FIELDS_PARAMETER, 1); + sink.getParameters().setClass(RecordOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, PactString.class); + + Plan plan = new Plan(sink, "L9 order by"); + return plan; + } +} diff --git a/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/MultiOrderKey.java b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/MultiOrderKey.java new file mode 100644 index 0000000000000..f0a87b49a7497 --- /dev/null +++ b/pact/pact-examples/src/main/java/eu/stratosphere/pact/example/pigmix/MultiOrderKey.java @@ -0,0 +1,66 @@ +package eu.stratosphere.pact.example.pigmix; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import eu.stratosphere.pact.common.type.Key; + +public class MultiOrderKey implements Key { + + public String query_term; + int timespent; + double estimated_revenue; + + public MultiOrderKey() { + query_term = null; + timespent = 0; + estimated_revenue = 0.0; + } + + public MultiOrderKey(String qt, String ts, String er) { + query_term = qt.toString(); + try { + timespent = Integer.valueOf(ts.toString()); + } catch (NumberFormatException nfe) { + timespent = 0; + } + try { + estimated_revenue = Double.valueOf(er.toString()); + } catch (NumberFormatException nfe) { + estimated_revenue = 0.0; + } + } + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(timespent); + out.writeDouble(estimated_revenue); + out.writeInt(query_term.length()); + out.writeBytes(query_term); + } + @Override + public void read(DataInput in) throws IOException { + timespent = in.readInt(); + estimated_revenue = in.readDouble(); + int len = in.readInt(); + byte[] b = new byte[len]; + in.readFully(b); + query_term = new String(b); + } + + @Override + public int compareTo(final Key other) { + if(!(other instanceof MultiOrderKey)){ + return 0; + } + final MultiOrderKey mokother = (MultiOrderKey) other; + int rc = query_term.compareTo(mokother.query_term); + if (rc != 0) return rc; + if (estimated_revenue < mokother.estimated_revenue) return 1; + else if (estimated_revenue > mokother.estimated_revenue) return -1; + if (timespent < mokother.timespent) return -1; + else if (timespent > mokother.timespent) return 1; + return 0; + } + +} \ No newline at end of file From b84f04bd4421177d4eb2c20eae6d3f9370e6b6b3 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 3 Feb 2012 19:00:41 +0100 Subject: [PATCH 203/310] Extended lookup protocol to reflect new state REPLAYING --- .../java/eu/stratosphere/nephele/jobmanager/JobManager.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index da8af2aa1d8ba..73d6868b7e52e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -779,7 +779,8 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio return ConnectionInfoLookupResponse.createReceiverFoundAndReady(); } - if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING) { + if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.REPLAYING + && executionState != ExecutionState.FINISHING) { return ConnectionInfoLookupResponse.createReceiverNotReady(); } @@ -808,7 +809,8 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio // Check execution state final ExecutionState executionState = targetVertex.getExecutionState(); - if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING) { + if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.REPLAYING + && executionState != ExecutionState.FINISHING) { return ConnectionInfoLookupResponse.createReceiverNotReady(); } From dc1cdb41f9ee89bc807b2a4d49b56d6d3e336d22 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 3 Feb 2012 19:43:15 +0100 Subject: [PATCH 204/310] Changed verbosity of RuntimeOutputChannelBroker --- .../taskmanager/runtime/RuntimeOutputChannelBroker.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java index 25b8091677fad..b87ae04d6ed67 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java @@ -6,6 +6,7 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.event.task.AbstractTaskEvent; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; import eu.stratosphere.nephele.io.channels.bytebuffered.BufferPairResponse; @@ -99,7 +100,7 @@ public void processEvent(final AbstractEvent event) { if (event instanceof ByteBufferedChannelCloseEvent) { this.closeAcknowledgementReceived = true; - } else { + } else if (event instanceof AbstractTaskEvent) { this.byteBufferedOutputChannel.processEvent(event); } } From e1f8f60d8b6743cf1718ae626abf81a554e8354b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 3 Feb 2012 20:51:36 +0100 Subject: [PATCH 205/310] Fixed bug in ReplayThread close logic --- .../eu/stratosphere/nephele/checkpointing/ReplayThread.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index 0f15fb29eb30a..0a5158c1c5d7f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -110,7 +110,7 @@ private void waitForAllOutputBrokerToFinish() throws IOException, InterruptedExc final Iterator it = this.outputBrokerMap.values().iterator(); while (it.hasNext()) { - if (it.next().hasFinished()) { + if (!it.next().hasFinished()) { finished = false; } } From 52adce0356d88e834ae3c80c3f8db618b98ddf2e Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 3 Feb 2012 21:08:36 +0100 Subject: [PATCH 206/310] Implemented logic to restart canceled tasks as part of recovery --- .../nephele/executiongraph/ExecutionGraph.java | 2 +- .../scheduler/AbstractExecutionListener.java | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index 319ca6dbb778b..22f98118de0f0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -1234,7 +1234,7 @@ private static InternalJobStatus determineNewJobStatus(final ExecutionGraph eg, } break; case RUNNING: - if (latestStateChange == ExecutionState.CANCELING || latestStateChange == ExecutionState.CANCELED) { + if (latestStateChange == ExecutionState.CANCELED) { return InternalJobStatus.CANCELING; } if (latestStateChange == ExecutionState.FAILED) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java index f76484e061594..063c910f3afe1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java @@ -86,6 +86,16 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver } } + if (newExecutionState == ExecutionState.CANCELED) { + if (this.scheduler.getVerticesToBeRestarted().remove(this.executionVertex.getID()) != null) { + this.executionVertex.updateExecutionState(ExecutionState.ASSIGNED, "Restart as part of recovery"); + + // Run through the deployment procedure + this.scheduler.deployAssignedVertices(eg); + return; + } + } + if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED || newExecutionState == ExecutionState.FAILED) { // Check if instance can be released From 3824e64f354ad726faaa6cd1925690bf6243f14b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 3 Feb 2012 21:33:37 +0100 Subject: [PATCH 207/310] Introduced sequence numbers for input split requests to handle recovery of input tasks --- .../eu/stratosphere/nephele/jobmanager/JobManager.java | 6 +++--- .../jobmanager/splitassigner/InputSplitManager.java | 4 +++- .../nephele/protocols/InputSplitProviderProtocol.java | 6 +++++- .../nephele/taskmanager/TaskInputSplitProvider.java | 7 +++++-- 4 files changed, 16 insertions(+), 7 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 73d6868b7e52e..2a0964062e9eb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -1238,8 +1238,8 @@ public void run() { * {@inheritDoc} */ @Override - public InputSplitWrapper requestNextInputSplit(final JobID jobID, final ExecutionVertexID vertexID) - throws IOException { + public InputSplitWrapper requestNextInputSplit(final JobID jobID, final ExecutionVertexID vertexID, + final IntegerRecord sequenceNumber) throws IOException { final ExecutionGraph graph = this.scheduler.getExecutionGraphByID(jobID); if (graph == null) { @@ -1253,7 +1253,7 @@ public InputSplitWrapper requestNextInputSplit(final JobID jobID, final Executio return null; } - return new InputSplitWrapper(jobID, this.inputSplitManager.getNextInputSplit(vertex)); + return new InputSplitWrapper(jobID, this.inputSplitManager.getNextInputSplit(vertex, sequenceNumber.getValue())); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java index cd55606527a9c..6a2fd57c2f171 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java @@ -159,9 +159,11 @@ public void unregisterJob(final ExecutionGraph executionGraph) { * * @param vertex * the vertex for which the next input split is to be determined + * @param sequenceNumber + * the sequence number of the vertex's request * @return the next input split to consume or null if the vertex shall consume no more input splits */ - public InputSplit getNextInputSplit(final ExecutionVertex vertex) { + public InputSplit getNextInputSplit(final ExecutionVertex vertex, final int sequenceNumber) { final ExecutionGroupVertex groupVertex = vertex.getGroupVertex(); final InputSplitAssigner inputSplitAssigner = this.assignerCache.get(groupVertex); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/InputSplitProviderProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/InputSplitProviderProtocol.java index 9de372041be95..0dba684b8b300 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/InputSplitProviderProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/InputSplitProviderProtocol.java @@ -20,6 +20,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobmanager.splitassigner.InputSplitWrapper; +import eu.stratosphere.nephele.types.IntegerRecord; /** * The input split provider protocol is used to facilitate RPC calls related to the lazy split assignment which Nephele @@ -36,10 +37,13 @@ public interface InputSplitProviderProtocol extends VersionedProtocol { * the ID of the job the task to retrieve the next input split for belongs to * @param vertexID * the ID of the task to retrieve the next input split for + * @param sequenceNumber + * a sequence number, starting at 0 and increased by the task on each request * @return a wrapper containing the next input split. The wrapped input split may also be null in case * no more input splits shall be consumed by the task with the given execution vertex ID * @throws IOException * thrown if an I/O error occurs while retrieving the new input split */ - InputSplitWrapper requestNextInputSplit(JobID jobID, ExecutionVertexID vertexID) throws IOException; + InputSplitWrapper requestNextInputSplit(JobID jobID, ExecutionVertexID vertexID, IntegerRecord sequenceNumber) + throws IOException; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskInputSplitProvider.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskInputSplitProvider.java index c1acf9c966952..3d01f6a97609a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskInputSplitProvider.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskInputSplitProvider.java @@ -16,6 +16,7 @@ package eu.stratosphere.nephele.taskmanager; import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; @@ -23,6 +24,7 @@ import eu.stratosphere.nephele.protocols.InputSplitProviderProtocol; import eu.stratosphere.nephele.template.InputSplit; import eu.stratosphere.nephele.template.InputSplitProvider; +import eu.stratosphere.nephele.types.IntegerRecord; import eu.stratosphere.nephele.util.StringUtils; /** @@ -42,6 +44,8 @@ public class TaskInputSplitProvider implements InputSplitProvider { private final InputSplitProviderProtocol globalInputSplitProvider; + private final AtomicInteger sequenceNumber = new AtomicInteger(0); + TaskInputSplitProvider(final JobID jobID, final ExecutionVertexID executionVertexID, final InputSplitProviderProtocol globalInputSplitProvider) { @@ -60,7 +64,7 @@ public InputSplit getNextInputSplit() { synchronized (this.globalInputSplitProvider) { final InputSplitWrapper wrapper = this.globalInputSplitProvider.requestNextInputSplit(this.jobID, - this.executionVertexID); + this.executionVertexID, new IntegerRecord(this.sequenceNumber.getAndIncrement())); return wrapper.getInputSplit(); } @@ -69,6 +73,5 @@ public InputSplit getNextInputSplit() { // rest throw new RuntimeException(StringUtils.stringifyException(ioe)); } - } } From b3af7c55f75a5ff666f43fc54fcb679805475056 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 4 Feb 2012 17:07:05 +0100 Subject: [PATCH 208/310] Resolved issues with close logic of replay tasks --- .../checkpointing/ReplayOutputBroker.java | 25 +++++++++++++------ .../nephele/checkpointing/ReplayTask.java | 6 +++-- .../nephele/checkpointing/ReplayThread.java | 2 +- 3 files changed, 22 insertions(+), 11 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java index dba2f08d26099..cce0bc297af47 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java @@ -2,6 +2,9 @@ import java.io.IOException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; @@ -11,15 +14,15 @@ final class ReplayOutputBroker implements OutputChannelForwarder { + /** + * The logger to report information and problems. + */ + private static final Log LOG = LogFactory.getLog(ReplayOutputBroker.class); + private final OutputChannelForwardingChain forwardingChain; private final IncomingEventQueue incomingEventQueue; - /** - * Stores whether the receiver has acknowledged the close request from this channel. - */ - private boolean closeAcknowledgementReceived = false; - ReplayOutputBroker(final OutputChannelForwardingChain forwardingChain, final IncomingEventQueue incomingEventQueue) { this.forwardingChain = forwardingChain; this.incomingEventQueue = incomingEventQueue; @@ -42,7 +45,10 @@ public boolean forward(final TransferEnvelope transferEnvelope) throws IOExcepti @Override public boolean hasDataLeft() { - return (!this.closeAcknowledgementReceived); + // A replay task will not wait for a close acknowledgement as it may have been sent to the corresponding runtime + // task before. + + return false; } /** @@ -52,9 +58,9 @@ public boolean hasDataLeft() { public void processEvent(final AbstractEvent event) { if (event instanceof ByteBufferedChannelCloseEvent) { - this.closeAcknowledgementReceived = true; + LOG.info("Replay output broker received event to close channel"); } else { - System.out.println("Received unknown event: " + event); + LOG.warn("Received unknown event: " + event); } } @@ -67,6 +73,9 @@ void outputEnvelope(final TransferEnvelope transferEnvelope) throws IOException, boolean hasFinished() { + // Check for events + this.incomingEventQueue.processQueuedEvents(); + return (!this.forwardingChain.anyForwarderHasDataLeft()); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index 669854506065b..9b2a523144e68 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -455,7 +455,8 @@ private static ExecutionState determineOverallExecutionState(final ExecutionStat if (changedExecutionState == ExecutionState.REPLAYING) { - if (unchangedExecutionState == ExecutionState.RUNNING) { + if (unchangedExecutionState == ExecutionState.RUNNING + || unchangedExecutionState == ExecutionState.FINISHING) { return ExecutionState.REPLAYING; } else { return unchangedExecutionState; @@ -470,7 +471,8 @@ private static ExecutionState determineOverallExecutionState(final ExecutionStat return ExecutionState.CANCELED; } - if (changedExecutionState == ExecutionState.FINISHING && unchangedExecutionState == ExecutionState.FINISHING) { + if (changedExecutionState == ExecutionState.FINISHING + && (unchangedExecutionState == ExecutionState.FINISHING || unchangedExecutionState == ExecutionState.FINISHED)) { return ExecutionState.FINISHING; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index 0a5158c1c5d7f..26314e9be5445 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -105,7 +105,7 @@ public void run() { private void waitForAllOutputBrokerToFinish() throws IOException, InterruptedException { - while (true) { + while (!this.executionObserver.isCanceled()) { boolean finished = true; final Iterator it = this.outputBrokerMap.values().iterator(); while (it.hasNext()) { From 10ba38274feb9756c22541aa2ecb4cb0dde5d305 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 4 Feb 2012 17:13:22 +0100 Subject: [PATCH 209/310] Fixed problem with lookup cache invalidation during recovery --- .../jobmanager/scheduler/RecoveryLogic.java | 62 +++++++++---------- 1 file changed, 31 insertions(+), 31 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index ed4f4b9ad7fed..67df792fa6025 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -127,7 +127,7 @@ private static void findVerticesToRestart(final ExecutionVertex failedVertex, verticesToTest.add(failedVertex); System.out.println("++++" + failedVertex + " failed"); - + while (!verticesToTest.isEmpty()) { final ExecutionVertex vertex = verticesToTest.poll(); @@ -139,7 +139,8 @@ private static void findVerticesToRestart(final ExecutionVertex failedVertex, // Predecessors must be either checkpoints or need to be restarted, too for (int j = 0; j < vertex.getNumberOfPredecessors(); j++) { final ExecutionVertex predecessor = vertex.getPredecessor(j); - System.out.println("++++ Predecessor " + predecessor + " has checkpoint state " + predecessor.getCheckpointState()); + System.out.println("++++ Predecessor " + predecessor + " has checkpoint state " + + predecessor.getCheckpointState()); if (predecessor.getCheckpointState() != CheckpointState.PARTIAL && predecessor.getCheckpointState() != CheckpointState.COMPLETE) { @@ -160,9 +161,35 @@ private static final boolean invalidateReceiverLookupCaches(final ExecutionVerte final Map> entriesToInvalidate = new HashMap>(); - final ExecutionGraph eg = failedVertex.getExecutionGraph(); + collectCacheEntriesToInvalidate(failedVertex, entriesToInvalidate); + for (final Iterator it = verticesToBeCanceled.iterator(); it.hasNext();) { + collectCacheEntriesToInvalidate(it.next(), entriesToInvalidate); + } + + final Iterator>> it = entriesToInvalidate.entrySet().iterator(); + + while (it.hasNext()) { + + final Map.Entry> entry = it.next(); + final AbstractInstance instance = entry.getKey(); - final RuntimeEnvironment env = failedVertex.getEnvironment(); + try { + instance.invalidateLookupCacheEntries(entry.getValue()); + } catch (IOException ioe) { + LOG.error(StringUtils.stringifyException(ioe)); + return false; + } + } + + return true; + } + + private static void collectCacheEntriesToInvalidate(final ExecutionVertex vertex, + final Map> entriesToInvalidate) { + + final ExecutionGraph eg = vertex.getExecutionGraph(); + + final RuntimeEnvironment env = vertex.getEnvironment(); for (int i = 0; i < env.getNumberOfOutputGates(); ++i) { final OutputGate outputGate = env.getOutputGate(i); @@ -181,11 +208,6 @@ private static final boolean invalidateReceiverLookupCaches(final ExecutionVerte continue; } - if (verticesToBeCanceled.contains(connectedVertex)) { - // Vertex will be canceled anyways - continue; - } - final AbstractInstance instance = connectedVertex.getAllocatedResource().getInstance(); Set channelIDs = entriesToInvalidate.get(instance); if (channelIDs == null) { @@ -215,11 +237,6 @@ private static final boolean invalidateReceiverLookupCaches(final ExecutionVerte continue; } - if (verticesToBeCanceled.contains(connectedVertex)) { - // Vertex will be canceled anyways - continue; - } - final AbstractInstance instance = connectedVertex.getAllocatedResource().getInstance(); Set channelIDs = entriesToInvalidate.get(instance); if (channelIDs == null) { @@ -230,22 +247,5 @@ private static final boolean invalidateReceiverLookupCaches(final ExecutionVerte channelIDs.add(connectedChannelID); } } - - final Iterator>> it = entriesToInvalidate.entrySet().iterator(); - - while (it.hasNext()) { - - final Map.Entry> entry = it.next(); - final AbstractInstance instance = entry.getKey(); - - try { - instance.invalidateLookupCacheEntries(entry.getValue()); - } catch (IOException ioe) { - LOG.error(StringUtils.stringifyException(ioe)); - return false; - } - } - - return true; } } From 73bd45ec956199f675bfbb571ce858b93d0d0ba5 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 4 Feb 2012 17:23:09 +0100 Subject: [PATCH 210/310] Minor fix to debug message --- .../java/eu/stratosphere/nephele/taskmanager/TaskManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index bcf1ed1992102..b4153013a3b95 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -409,7 +409,7 @@ public TaskCancelResult cancelTask(final ExecutionVertexID id) throws IOExceptio if (task == null) { final TaskCancelResult taskCancelResult = new TaskCancelResult(id, AbstractTaskResult.ReturnCode.ERROR); - taskCancelResult.setDescription("No task with ID + " + id + " is currently running"); + taskCancelResult.setDescription("No task with ID " + id + " is currently running"); return taskCancelResult; } From 1936afc6d65b5eb8219215a83ad9b72040f5a33c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 4 Feb 2012 19:28:54 +0100 Subject: [PATCH 211/310] Extended return code enumeration for IPC calls --- .../executiongraph/ExecutionVertex.java | 15 +++---- .../nephele/jobmanager/JobManager.java | 6 +-- .../jobmanager/scheduler/RecoveryLogic.java | 41 +++++++++++++------ .../taskmanager/AbstractTaskResult.java | 2 +- .../nephele/taskmanager/TaskManager.java | 9 ++-- 5 files changed, 47 insertions(+), 26 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 85958401a0f69..b82b1112451a4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -578,7 +578,7 @@ public TaskSubmissionResult startTask() { if (this.allocatedResource == null) { final TaskSubmissionResult result = new TaskSubmissionResult(getID(), - AbstractTaskResult.ReturnCode.ERROR); + AbstractTaskResult.ReturnCode.NO_INSTANCE); result.setDescription("Assigned instance of vertex " + this.toString() + " is null!"); return result; } @@ -596,7 +596,8 @@ public TaskSubmissionResult startTask() { return results.get(0); } catch (IOException e) { - final TaskSubmissionResult result = new TaskSubmissionResult(getID(), AbstractTaskResult.ReturnCode.ERROR); + final TaskSubmissionResult result = new TaskSubmissionResult(getID(), + AbstractTaskResult.ReturnCode.IPC_ERROR); result.setDescription(StringUtils.stringifyException(e)); return result; } @@ -615,13 +616,13 @@ public TaskKillResult killTask() { final ExecutionState state = this.executionState.get(); if (state != ExecutionState.RUNNING) { - final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.ERROR); + final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.ILLEGAL_STATE); result.setDescription("Vertex " + this.toString() + " is in state " + state); return result; } if (this.allocatedResource == null) { - final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.ERROR); + final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.NO_INSTANCE); result.setDescription("Assigned instance of vertex " + this.toString() + " is null!"); return result; } @@ -629,7 +630,7 @@ public TaskKillResult killTask() { try { return this.allocatedResource.getInstance().killTask(this.vertexID); } catch (IOException e) { - final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.ERROR); + final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.IPC_ERROR); result.setDescription(StringUtils.stringifyException(e)); return result; } @@ -666,7 +667,7 @@ public TaskCancelResult cancelTask() { } if (this.allocatedResource == null) { - final TaskCancelResult result = new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.ERROR); + final TaskCancelResult result = new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.NO_INSTANCE); result.setDescription("Assigned instance of vertex " + this.toString() + " is null!"); return result; } @@ -674,7 +675,7 @@ public TaskCancelResult cancelTask() { try { return this.allocatedResource.getInstance().cancelTask(this.vertexID); } catch (IOException e) { - final TaskCancelResult result = new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.ERROR); + final TaskCancelResult result = new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.IPC_ERROR); result.setDescription(StringUtils.stringifyException(e)); return result; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 2a0964062e9eb..4a468d3696f9a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -719,7 +719,7 @@ private TaskCancelResult cancelJob(final ExecutionGraph eg) { final ExecutionVertex vertex = it.next(); final TaskCancelResult result = vertex.cancelTask(); - if (result.getReturnCode() == AbstractTaskResult.ReturnCode.ERROR) { + if (result.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS) { errorResult = result; } } @@ -934,7 +934,7 @@ public void killTask(final JobID jobID, final ManagementVertexID id) throws IOEx public void run() { final TaskKillResult result = vertex.killTask(); - if (result.getReturnCode() == AbstractTaskResult.ReturnCode.ERROR) { + if (result.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS) { LOG.error(result.getDescription()); } } @@ -1223,7 +1223,7 @@ public void run() { } } - if (tsr.getReturnCode() == AbstractTaskResult.ReturnCode.ERROR) { + if (tsr.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS) { // Change the execution state to failed and let the scheduler deal with the rest vertex.updateExecutionState(ExecutionState.FAILED, tsr.getDescription()); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index 67df792fa6025..257e1d60362e8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -84,13 +84,25 @@ public static boolean recover(final ExecutionVertex failedVertex, while (cancelIterator.hasNext()) { final ExecutionVertex vertex = cancelIterator.next(); - LOG.info(vertex + " is canceled by recovery logic"); - final TaskCancelResult cancelResult = vertex.cancelTask(); - verticesToBeRestarted.put(vertex.getID(), vertex); - if (cancelResult.getReturnCode() != ReturnCode.SUCCESS) { - verticesToBeRestarted.remove(vertex.getID()); - LOG.error(cancelResult.getDescription()); - return false; + final ExecutionState state = vertex.getExecutionState(); + System.out.println("Canceling " + vertex + " with state " + vertex.getExecutionState()); + if (state == ExecutionState.FINISHED) { + // Restart vertex right away + restart(vertex); + } else { + + LOG.info(vertex + " is canceled by recovery logic"); + final TaskCancelResult cancelResult = vertex.cancelTask(); + verticesToBeRestarted.put(vertex.getID(), vertex); + if (cancelResult.getReturnCode() != ReturnCode.SUCCESS) { + verticesToBeRestarted.remove(vertex.getID()); + if (cancelResult.getReturnCode() == ReturnCode.TASK_NOT_FOUND) { + restart(vertex); + } else { + LOG.error(cancelResult.getDescription()); + return false; + } + } } } @@ -109,15 +121,20 @@ public static boolean recover(final ExecutionVertex failedVertex, } // Restart failed vertex - if (failedVertex.getAllocatedResource().getInstance() instanceof DummyInstance) { - failedVertex.updateExecutionState(ExecutionState.CREATED); - } else { - failedVertex.updateExecutionState(ExecutionState.ASSIGNED); - } + restart(failedVertex); return true; } + private static void restart(final ExecutionVertex vertex) { + + if (vertex.getAllocatedResource().getInstance() instanceof DummyInstance) { + vertex.updateExecutionState(ExecutionState.CREATED); + } else { + vertex.updateExecutionState(ExecutionState.ASSIGNED); + } + } + private static void findVerticesToRestart(final ExecutionVertex failedVertex, final Set verticesToBeCanceled, final Set checkpointsToBeReplayed) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/AbstractTaskResult.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/AbstractTaskResult.java index 157e2eae53aff..827ca5e103ab7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/AbstractTaskResult.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/AbstractTaskResult.java @@ -34,7 +34,7 @@ public abstract class AbstractTaskResult implements IOReadableWritable { public enum ReturnCode { - SUCCESS, ERROR + SUCCESS, IPC_ERROR, NO_INSTANCE, ILLEGAL_STATE, TASK_NOT_FOUND }; private ExecutionVertexID vertexID; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index b4153013a3b95..7941668962356 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -408,7 +408,8 @@ public TaskCancelResult cancelTask(final ExecutionVertexID id) throws IOExceptio final Task task = this.runningTasks.get(id); if (task == null) { - final TaskCancelResult taskCancelResult = new TaskCancelResult(id, AbstractTaskResult.ReturnCode.ERROR); + final TaskCancelResult taskCancelResult = new TaskCancelResult(id, + AbstractTaskResult.ReturnCode.TASK_NOT_FOUND); taskCancelResult.setDescription("No task with ID " + id + " is currently running"); return taskCancelResult; } @@ -442,7 +443,8 @@ public TaskKillResult killTask(final ExecutionVertexID id) throws IOException { tmpTask = this.runningTasks.get(id); if (tmpTask == null) { - final TaskKillResult taskKillResult = new TaskKillResult(id, AbstractTaskResult.ReturnCode.ERROR); + final TaskKillResult taskKillResult = new TaskKillResult(id, + AbstractTaskResult.ReturnCode.TASK_NOT_FOUND); taskKillResult.setDescription("No task with ID + " + id + " is currently running"); return taskKillResult; } @@ -484,7 +486,8 @@ public List submitTasks(final List // Register the task final Task task = createAndRegisterTask(id, jobConfiguration, re, activeOutputChannels); if (task == null) { - final TaskSubmissionResult result = new TaskSubmissionResult(id, AbstractTaskResult.ReturnCode.ERROR); + final TaskSubmissionResult result = new TaskSubmissionResult(id, + AbstractTaskResult.ReturnCode.TASK_NOT_FOUND); result.setDescription("Task with ID " + id + " was already running"); LOG.error(result.getDescription()); submissionResultList.add(result); From 330142690e5d3f50150ea5d9e139c15e7efd5ba4 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 5 Feb 2012 14:09:16 +0100 Subject: [PATCH 212/310] Improved logging of state transitions --- .../eu/stratosphere/nephele/executiongraph/ExecutionVertex.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index b82b1112451a4..2c55f63762625 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -312,7 +312,7 @@ public void updateExecutionState(final ExecutionState newExecutionState, final S } // Check the transition - ExecutionStateTransition.checkTransition(getName(), this.executionState.get(), newExecutionState); + ExecutionStateTransition.checkTransition(toString(), this.executionState.get(), newExecutionState); // Check and save the new execution state if (this.executionState.getAndSet(newExecutionState) == newExecutionState) { From 754881bef52bbfea782a8a4177231df00e5ec593 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 5 Feb 2012 16:01:00 +0100 Subject: [PATCH 213/310] Fixed misleading log messages in compareAndUpdateExecutionState method of ExecutionVertex --- .../nephele/executiongraph/ExecutionVertex.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 2c55f63762625..d0942ad4c9a50 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -333,12 +333,12 @@ public boolean compareAndUpdateExecutionState(final ExecutionState expected, fin throw new IllegalArgumentException("Argument update must not be null"); } - // Check the transition - ExecutionStateTransition.checkTransition(getName(), this.executionState.get(), update); - if (!this.executionState.compareAndSet(expected, update)) { return false; } + + // Check the transition + ExecutionStateTransition.checkTransition(toString(), this.executionState.get(), update); // Notify the listener objects final Iterator it = this.executionListeners.values().iterator(); From ea1201731535dd0a5f1d6d7610c1134af50a30d3 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 5 Feb 2012 16:15:05 +0100 Subject: [PATCH 214/310] Improved logging in utility class ExecutionStateTransition --- .../nephele/execution/ExecutionStateTransition.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java index af2b343a99827..c5138ae91a128 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java @@ -3,6 +3,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.stratosphere.nephele.util.StringUtils; + /** * This class is a utility class to check the consistency of Nephele's execution state model. * @@ -142,7 +144,11 @@ public static void checkTransition(final String taskName, final ExecutionState o } if (unexpectedStateChange) { - LOG.error("Unexpected state change: " + oldState + " -> " + newState); + try { + throw new IllegalStateException("Unexpected state change: " + oldState + " -> " + newState); + } catch (IllegalStateException e) { + LOG.error(StringUtils.stringifyException(e)); + } } } } From 506f305390e90eebc9fe900bb3c7e53abcb76a07 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 5 Feb 2012 21:47:17 +0100 Subject: [PATCH 215/310] Introduced new return value for routing lookups to reflect finished tasks during recovery --- .../nephele/jobmanager/JobManager.java | 5 ++++ .../ByteBufferedChannelManager.java | 10 ++++--- .../ConnectionInfoLookupResponse.java | 27 ++++++++++++++----- 3 files changed, 32 insertions(+), 10 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 4a468d3696f9a..d150680b2b805 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -809,6 +809,11 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio // Check execution state final ExecutionState executionState = targetVertex.getExecutionState(); + + if (executionState == ExecutionState.FINISHED) { + return ConnectionInfoLookupResponse.createReceiverFinished(); + } + if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.REPLAYING && executionState != ExecutionState.FINISHING) { return ConnectionInfoLookupResponse.createReceiverNotReady(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 94e3b55168320..1e313766b1140 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -494,15 +494,19 @@ private TransferEnvelopeReceiverList getReceiverList(final JobID jobID, final Ch continue; } + if (lookupResponse.receiverHasFinished()) { + // TODO: Send close notification here + break; + } + if (lookupResponse.receiverReady()) { receiverList = new TransferEnvelopeReceiverList(lookupResponse); break; } } - if (receiverList == null) { - LOG.error("Receiver list is null for source channel ID " + sourceChannelID); - } else { + if (receiverList != null) { + this.receiverCache.put(sourceChannelID, receiverList); if (LOG.isDebugEnabled()) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java index 21fcc73898bfc..bfa7dadf289af 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java @@ -30,7 +30,7 @@ public class ConnectionInfoLookupResponse implements IOReadableWritable { private enum ReturnCode { - NOT_FOUND, FOUND_AND_RECEIVER_READY, FOUND_BUT_RECEIVER_NOT_READY + NOT_FOUND, FOUND_AND_RECEIVER_READY, FOUND_BUT_RECEIVER_NOT_READY, FOUND_BUT_RECEIVER_FINISHED }; // was request successful? @@ -104,6 +104,11 @@ public boolean receiverReady() { return (this.returnCode == ReturnCode.FOUND_AND_RECEIVER_READY); } + public boolean receiverHasFinished() { + + return (this.returnCode == ReturnCode.FOUND_BUT_RECEIVER_FINISHED); + } + public static ConnectionInfoLookupResponse createReceiverFoundAndReady(ChannelID targetChannelID) { final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse(); @@ -121,9 +126,10 @@ public static ConnectionInfoLookupResponse createReceiverFoundAndReady(InstanceC return response; } - + /** * Constructor used to generate a plain ConnectionInfoLookupResponse object to be filled with multicast targets. + * * @return */ public static ConnectionInfoLookupResponse createReceiverFoundAndReady() { @@ -132,7 +138,7 @@ public static ConnectionInfoLookupResponse createReceiverFoundAndReady() { response.setReturnCode(ReturnCode.FOUND_AND_RECEIVER_READY); return response; - } + } public static ConnectionInfoLookupResponse createReceiverNotFound() { final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse(); @@ -147,16 +153,23 @@ public static ConnectionInfoLookupResponse createReceiverNotReady() { return response; } - + + public static ConnectionInfoLookupResponse createReceiverFinished() { + final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse(); + response.setReturnCode(ReturnCode.FOUND_BUT_RECEIVER_FINISHED); + + return response; + } + @Override - public String toString(){ + public String toString() { StringBuilder returnstring = new StringBuilder(); returnstring.append("local targets (total: " + this.localTargets.size() + "):\n"); - for(ChannelID i: this.localTargets){ + for (ChannelID i : this.localTargets) { returnstring.append(i + "\n"); } returnstring.append("remote targets: (total: " + this.remoteTargets.size() + "):\n"); - for(InstanceConnectionInfo i: this.remoteTargets){ + for (InstanceConnectionInfo i : this.remoteTargets) { returnstring.append(i + "\n"); } return returnstring.toString(); From 1216d33b6c9789032363e0aa9f95894ad4ae88ee Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 5 Feb 2012 21:56:33 +0100 Subject: [PATCH 216/310] Unexpected envelope event is not sent outside of the queue synchronization block --- .../runtime/RuntimeInputChannelContext.java | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java index 93b67e8e2a3aa..143e74f3499c0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java @@ -168,6 +168,8 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { // The sequence number of the envelope to be queued final int sequenceNumber = transferEnvelope.getSequenceNumber(); + AbstractEvent eventToSend = null; + synchronized (this.queuedEnvelopes) { if (this.destroyCalled) { @@ -192,11 +194,7 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { } else { // Tell the sender that we are expecting an envelope with a higher sequence number - try { - transferEventToOutputChannel(new UnexpectedEnvelopeEvent(expectedSequenceNumber)); - } catch (Exception e) { - LOG.error(StringUtils.stringifyException(e)); - } + eventToSend = new UnexpectedEnvelopeEvent(expectedSequenceNumber); } LOG.warn("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber @@ -206,17 +204,24 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { if (buffer != null) { buffer.recycleBuffer(); } + } else { - return; - } - - this.queuedEnvelopes.add(transferEnvelope); + this.queuedEnvelopes.add(transferEnvelope); - this.lastReceivedEnvelope = sequenceNumber; + this.lastReceivedEnvelope = sequenceNumber; + } } // Notify the channel about the new data this.byteBufferedInputChannel.checkForNetworkEvents(); + + if (eventToSend != null) { + try { + transferEventToOutputChannel(eventToSend); + } catch (Exception e) { + LOG.error(StringUtils.stringifyException(e)); + } + } } @Override From 5a805a89a095d74791e253842648419f8a1d8ee3 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 5 Feb 2012 22:01:38 +0100 Subject: [PATCH 217/310] Added convenience method to RuntimeEnvironment --- .../nephele/execution/RuntimeEnvironment.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index e6322b3f9e250..84d8c08d2110d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -943,6 +943,17 @@ public String getTaskName() { return this.taskName; } + /** + * Returns the name of the task with its index in the subtask group and the total number of subtasks. + * + * @return the name of the task with its index in the subtask group and the total number of subtasks + */ + public String getTaskNameWithIndex() { + + return this.taskName + " (" + (getIndexInSubtaskGroup() + 1) + "/" + + getCurrentNumberOfSubtasks() + ")"; + } + /** * Sets the execution observer for this environment. * From 0611b19b9572c6cba52b46607d64731192609ff1 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 5 Feb 2012 22:02:57 +0100 Subject: [PATCH 218/310] Temporarily disabled SpillingBarrier --- .../nephele/taskmanager/runtime/SpillingBarrier.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java index b146bce097eae..5ead352966605 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java @@ -31,12 +31,12 @@ public SpillingBarrier(final boolean isReceiverRunning, final boolean mergeSpill @Override public boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - if (!this.isReceiverRunning) { + /*if (!this.isReceiverRunning) { // TODO: Add this to the spilling queue return false; - } + }*/ return true; } From 2f16e490baf4cc48373923c59ec0c88b393a856d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 6 Feb 2012 11:23:31 +0100 Subject: [PATCH 219/310] Fixed bug in Task Manager --- .../eu/stratosphere/nephele/taskmanager/TaskManager.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 7941668962356..766b78844dfb8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -488,14 +488,13 @@ public List submitTasks(final List if (task == null) { final TaskSubmissionResult result = new TaskSubmissionResult(id, AbstractTaskResult.ReturnCode.TASK_NOT_FOUND); - result.setDescription("Task with ID " + id + " was already running"); + result.setDescription("Task " + re.getTaskNameWithIndex() + " (" + id + ") was already running"); LOG.error(result.getDescription()); submissionResultList.add(result); } else { submissionResultList.add(new TaskSubmissionResult(id, AbstractTaskResult.ReturnCode.SUCCESS)); + tasksToStart.add(task); } - - tasksToStart.add(task); } // Now start the tasks From 196fa56307b13e5a81331a0e651ec1342a421b0f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 6 Feb 2012 15:50:00 +0100 Subject: [PATCH 220/310] Worked on locking during recovery --- .../execution/ExecutionStateTransition.java | 113 ++++++++++-------- .../executiongraph/ExecutionVertex.java | 92 ++++++++------ .../scheduler/AbstractExecutionListener.java | 16 ++- .../jobmanager/scheduler/RecoveryLogic.java | 80 +++++++------ .../taskmanager/runtime/RuntimeTask.java | 22 ++-- 5 files changed, 183 insertions(+), 140 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java index c5138ae91a128..486fe78e205d0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java @@ -26,6 +26,9 @@ private ExecutionStateTransition() { /** * Checks the transition of the execution state and outputs an error in case of an unexpected state transition. * + * @param jobManager + * true to indicate the method is called by the job manager, + * false/ to indicate it is called by a task manager * @param taskName * the name of the task whose execution has changed * @param oldState @@ -33,16 +36,11 @@ private ExecutionStateTransition() { * @param newState * the new execution state */ - public static void checkTransition(final String taskName, final ExecutionState oldState, + public static void checkTransition(final boolean jobManager, final String taskName, final ExecutionState oldState, final ExecutionState newState) { - // Ignore state changes in final states - if (oldState == ExecutionState.CANCELED || oldState == ExecutionState.FINISHED - || oldState == ExecutionState.FAILED) { - return; - } - - LOG.info("ExecutionState set from " + oldState + " to " + newState + " for task " + taskName); + LOG.info((jobManager ? "JM: " : "TM: ") + "ExecutionState set from " + oldState + " to " + newState + + " for task " + taskName); boolean unexpectedStateChange = true; @@ -62,84 +60,101 @@ public static void checkTransition(final String taskName, final ExecutionState o if (oldState == ExecutionState.STARTING && newState == ExecutionState.RUNNING) { unexpectedStateChange = false; } - if (oldState == ExecutionState.STARTING && newState == ExecutionState.REPLAYING) { + if (oldState == ExecutionState.RUNNING && newState == ExecutionState.FINISHING) { unexpectedStateChange = false; } - if (oldState == ExecutionState.RUNNING && newState == ExecutionState.FINISHING) { + if (oldState == ExecutionState.FINISHING && newState == ExecutionState.FINISHED) { unexpectedStateChange = false; } - if (oldState == ExecutionState.REPLAYING && newState == ExecutionState.FINISHING) { + + // These transitions may occur during a task recovery + if (oldState == ExecutionState.FAILED && newState == ExecutionState.ASSIGNED) { unexpectedStateChange = false; } - if (oldState == ExecutionState.FINISHING && newState == ExecutionState.FINISHED) { + if (oldState == ExecutionState.FAILED && newState == ExecutionState.CREATED) { + unexpectedStateChange = false; + } + if (oldState == ExecutionState.FINISHED && newState == ExecutionState.ASSIGNED) { + unexpectedStateChange = false; + } + if (oldState == ExecutionState.FINISHED && newState == ExecutionState.CREATED) { + unexpectedStateChange = false; + } + if (oldState == ExecutionState.CANCELED && newState == ExecutionState.ASSIGNED) { + unexpectedStateChange = false; + } + if (oldState == ExecutionState.CANCELED && newState == ExecutionState.CREATED) { + unexpectedStateChange = false; + } + if (oldState == ExecutionState.STARTING && newState == ExecutionState.REPLAYING) { + unexpectedStateChange = false; + } + if (oldState == ExecutionState.REPLAYING && newState == ExecutionState.FINISHING) { unexpectedStateChange = false; } + // A vertex might skip the SCHEDULED state if its resource has been allocated in a previous stage. if (oldState == ExecutionState.CREATED && newState == ExecutionState.ASSIGNED) { - /** - * A vertex might skip the SCHEDULED state if its resource has been allocated in a previous stage. - */ unexpectedStateChange = false; } - if (oldState == ExecutionState.SCHEDULED && newState == ExecutionState.CANCELED) { - /** - * This transition can appear if a task in a stage which is not yet executed gets canceled. - */ + // This transition can appear if a task in a stage which is not yet executed gets canceled. + if (oldState == ExecutionState.SCHEDULED && newState == ExecutionState.CANCELING) { unexpectedStateChange = false; } - if (oldState == ExecutionState.ASSIGNED && newState == ExecutionState.CANCELED) { - /** - * This transition can appear if a task in a stage which is not yet executed gets canceled. - */ + + // This transition can appear if a task in a stage which is not yet executed gets canceled. + if (oldState == ExecutionState.ASSIGNED && newState == ExecutionState.CANCELING) { unexpectedStateChange = false; } - if (oldState == ExecutionState.READY && newState == ExecutionState.CANCELED) { - /** - * This transition can appear if a task is canceled that is not yet running on the task manager. - */ + + // This transition can appear if a task is canceled that is not yet running on the task manager. + if (oldState == ExecutionState.READY && newState == ExecutionState.CANCELING) { unexpectedStateChange = false; } + + // This transition can appear if a task cannot be deployed at the assigned task manager. if (oldState == ExecutionState.STARTING && newState == ExecutionState.FAILED) { - /** - * This transition can appear if a task cannot be deployed at the assigned task manager. - */ unexpectedStateChange = false; } + + // This is a regular transition in case of a task error. if (oldState == ExecutionState.RUNNING && newState == ExecutionState.FAILED) { - /** - * This is a regular transition in case of a task error. - */ unexpectedStateChange = false; } + + if (oldState == ExecutionState.FINISHING && newState == ExecutionState.FAILED) { + // This is a regular transition in case of a task error. + unexpectedStateChange = false; + } + + if (oldState == ExecutionState.REPLAYING && newState == ExecutionState.FAILED) { + // This is a regular transition in case of a task error. + unexpectedStateChange = false; + } + + // This is a regular transition in case a task replay is triggered. if (oldState == ExecutionState.RUNNING && newState == ExecutionState.ASSIGNED) { - /** - * This is a regular transition in case a task replay is triggered. - */ unexpectedStateChange = false; } - if (oldState == ExecutionState.FINISHING && newState == ExecutionState.FAILED) { - /** - * This is a regular transition in case of a task error. - */ + + // This is a regular transition in case a task replay is triggered. + if (oldState == ExecutionState.FINISHING && newState == ExecutionState.ASSIGNED) { unexpectedStateChange = false; } + + // This is a regular transition as a result of a cancel operation. if (oldState == ExecutionState.RUNNING && newState == ExecutionState.CANCELING) { - /** - * This is a regular transition as a result of a cancel operation. - */ unexpectedStateChange = false; } + + // This is a regular transition as a result of a cancel operation. if (oldState == ExecutionState.FINISHING && newState == ExecutionState.CANCELING) { - /** - * This is a regular transition as a result of a cancel operation. - */ unexpectedStateChange = false; } + + // This is a regular transition as a result of a cancel operation. if (oldState == ExecutionState.CANCELING && newState == ExecutionState.CANCELED) { - /** - * This is a regular transition as a result of a cancel operation. - */ unexpectedStateChange = false; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index d0942ad4c9a50..dd1ad91205b53 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -44,6 +44,7 @@ import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.taskmanager.AbstractTaskResult; +import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; import eu.stratosphere.nephele.taskmanager.TaskKillResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult; @@ -293,8 +294,8 @@ public ExecutionState getExecutionState() { * @param newExecutionState * the new execution state */ - public void updateExecutionState(final ExecutionState newExecutionState) { - updateExecutionState(newExecutionState, null); + public ExecutionState updateExecutionState(final ExecutionState newExecutionState) { + return updateExecutionState(newExecutionState, null); } /** @@ -305,26 +306,36 @@ public void updateExecutionState(final ExecutionState newExecutionState) { * @param optionalMessage * an optional message related to the state change */ - public void updateExecutionState(final ExecutionState newExecutionState, final String optionalMessage) { + public ExecutionState updateExecutionState(ExecutionState newExecutionState, final String optionalMessage) { if (newExecutionState == null) { throw new IllegalArgumentException("Argument newExecutionState must not be null"); } - // Check the transition - ExecutionStateTransition.checkTransition(toString(), this.executionState.get(), newExecutionState); + // Rewrite FINISHED to CANCELED if the task has been marked to be canceled + if (this.executionState.get() == ExecutionState.CANCELING && newExecutionState == ExecutionState.FINISHED) { + LOG.info("Received transition from CANCELING to FINISHED for vertex " + toString() + + ", converting it to CANCELED"); + newExecutionState = ExecutionState.CANCELED; + } // Check and save the new execution state - if (this.executionState.getAndSet(newExecutionState) == newExecutionState) { - return; + final ExecutionState previousState = this.executionState.getAndSet(newExecutionState); + if (previousState == newExecutionState) { + return previousState; } + // Check the transition + ExecutionStateTransition.checkTransition(true, toString(), previousState, newExecutionState); + // Notify the listener objects final Iterator it = this.executionListeners.values().iterator(); while (it.hasNext()) { it.next().executionStateChanged(this.executionGraph.getJobID(), this.vertexID, newExecutionState, optionalMessage); } + + return previousState; } public boolean compareAndUpdateExecutionState(final ExecutionState expected, final ExecutionState update) { @@ -336,9 +347,9 @@ public boolean compareAndUpdateExecutionState(final ExecutionState expected, fin if (!this.executionState.compareAndSet(expected, update)) { return false; } - + // Check the transition - ExecutionStateTransition.checkTransition(toString(), this.executionState.get(), update); + ExecutionStateTransition.checkTransition(true, toString(), expected, update); // Notify the listener objects final Iterator it = this.executionListeners.values().iterator(); @@ -646,39 +657,45 @@ public TaskKillResult killTask() { */ public TaskCancelResult cancelTask() { - final ExecutionState state = this.executionState.get(); + final ExecutionState previousState = this.executionState.get(); - if (this.groupVertex.getStageNumber() != this.executionGraph.getIndexOfCurrentExecutionStage()) { - // Set to canceled directly - updateExecutionState(ExecutionState.CANCELED, null); - return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); - } + if (updateExecutionState(ExecutionState.CANCELING) != ExecutionState.CANCELING) { - if (state == ExecutionState.FINISHED || state == ExecutionState.FAILED) { - // Ignore this call - return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); - } + if (this.groupVertex.getStageNumber() != this.executionGraph.getIndexOfCurrentExecutionStage()) { + // Set to canceled directly + updateExecutionState(ExecutionState.CANCELED, null); + return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); + } - if (state != ExecutionState.RUNNING && state != ExecutionState.STARTING - && state != ExecutionState.FINISHING && state != ExecutionState.REPLAYING) { - // Set to canceled directly - updateExecutionState(ExecutionState.CANCELED, null); - return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); - } + if (previousState == ExecutionState.FINISHED || previousState == ExecutionState.FAILED) { + // Ignore this call + return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); + } - if (this.allocatedResource == null) { - final TaskCancelResult result = new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.NO_INSTANCE); - result.setDescription("Assigned instance of vertex " + this.toString() + " is null!"); - return result; - } + if (previousState != ExecutionState.RUNNING && previousState != ExecutionState.STARTING + && previousState != ExecutionState.FINISHING && previousState != ExecutionState.REPLAYING) { + // Set to canceled directly + updateExecutionState(ExecutionState.CANCELED, null); + return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); + } - try { - return this.allocatedResource.getInstance().cancelTask(this.vertexID); - } catch (IOException e) { - final TaskCancelResult result = new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.IPC_ERROR); - result.setDescription(StringUtils.stringifyException(e)); - return result; + if (this.allocatedResource == null) { + final TaskCancelResult result = new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.NO_INSTANCE); + result.setDescription("Assigned instance of vertex " + this.toString() + " is null!"); + return result; + } + + try { + return this.allocatedResource.getInstance().cancelTask(this.vertexID); + + } catch (IOException e) { + final TaskCancelResult result = new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.IPC_ERROR); + result.setDescription(StringUtils.stringifyException(e)); + return result; + } } + + return new TaskCancelResult(getID(), ReturnCode.SUCCESS); } /** @@ -697,8 +714,7 @@ public ExecutionGraph getExecutionGraph() { @Override public String toString() { - return getName() + " (" + (this.environment.getIndexInSubtaskGroup() + 1) + "/" - + (this.environment.getCurrentNumberOfSubtasks()) + ")"; + return this.environment.getTaskNameWithIndex(); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java index 063c910f3afe1..d063363328aad 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java @@ -86,13 +86,17 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver } } - if (newExecutionState == ExecutionState.CANCELED) { - if (this.scheduler.getVerticesToBeRestarted().remove(this.executionVertex.getID()) != null) { - this.executionVertex.updateExecutionState(ExecutionState.ASSIGNED, "Restart as part of recovery"); + if (newExecutionState == ExecutionState.CANCELED || newExecutionState == ExecutionState.FINISHED) { - // Run through the deployment procedure - this.scheduler.deployAssignedVertices(eg); - return; + synchronized (this.executionVertex.getExecutionGraph()) { + + if (this.scheduler.getVerticesToBeRestarted().remove(this.executionVertex.getID()) != null) { + this.executionVertex.updateExecutionState(ExecutionState.ASSIGNED, "Restart as part of recovery"); + + // Run through the deployment procedure + this.scheduler.deployAssignedVertices(eg); + return; + } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index 257e1d60362e8..2b8a59cb7649f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -71,68 +71,72 @@ public static boolean recover(final ExecutionVertex failedVertex, return false; } - LOG.info("Starting recovery for failed vertex " + failedVertex); + final ExecutionGraph eg = failedVertex.getExecutionGraph(); + synchronized (eg) { - final Set verticesToBeCanceled = new HashSet(); + LOG.info("Starting recovery for failed vertex " + failedVertex); - final Set checkpointsToBeReplayed = new HashSet(); + final Set verticesToBeCanceled = new HashSet(); - findVerticesToRestart(failedVertex, verticesToBeCanceled, checkpointsToBeReplayed); + final Set checkpointsToBeReplayed = new HashSet(); - // Restart all predecessors without checkpoint - final Iterator cancelIterator = verticesToBeCanceled.iterator(); - while (cancelIterator.hasNext()) { + findVerticesToRestart(failedVertex, verticesToBeCanceled, checkpointsToBeReplayed); - final ExecutionVertex vertex = cancelIterator.next(); - final ExecutionState state = vertex.getExecutionState(); - System.out.println("Canceling " + vertex + " with state " + vertex.getExecutionState()); - if (state == ExecutionState.FINISHED) { - // Restart vertex right away - restart(vertex); - } else { + // Restart all predecessors without checkpoint + final Iterator cancelIterator = verticesToBeCanceled.iterator(); + while (cancelIterator.hasNext()) { + + final ExecutionVertex vertex = cancelIterator.next(); + + if (vertex.compareAndUpdateExecutionState(ExecutionState.FINISHED, getStateToUpdate(vertex))) { + LOG.info("Vertex " + vertex + " has already finished and will not be canceled"); + continue; + } LOG.info(vertex + " is canceled by recovery logic"); - final TaskCancelResult cancelResult = vertex.cancelTask(); verticesToBeRestarted.put(vertex.getID(), vertex); - if (cancelResult.getReturnCode() != ReturnCode.SUCCESS) { + final TaskCancelResult cancelResult = vertex.cancelTask(); + + if (cancelResult.getReturnCode() != ReturnCode.SUCCESS + && cancelResult.getReturnCode() != ReturnCode.TASK_NOT_FOUND) { + verticesToBeRestarted.remove(vertex.getID()); - if (cancelResult.getReturnCode() == ReturnCode.TASK_NOT_FOUND) { - restart(vertex); - } else { - LOG.error(cancelResult.getDescription()); - return false; - } + LOG.error(cancelResult.getDescription()); + return false; } } - } + LOG.info("Starting cache invalidation"); - // Invalidate the lookup caches - if (!invalidateReceiverLookupCaches(failedVertex, verticesToBeCanceled)) { - return false; - } + // Invalidate the lookup caches + if (!invalidateReceiverLookupCaches(failedVertex, verticesToBeCanceled)) { + return false; + } - // Replay all necessary checkpoints - final Iterator checkpointIterator = checkpointsToBeReplayed.iterator(); + LOG.info("Cache invalidation complete"); - while (checkpointIterator.hasNext()) { + // Replay all necessary checkpoints + final Iterator checkpointIterator = checkpointsToBeReplayed.iterator(); - checkpointIterator.next().updateExecutionState(ExecutionState.ASSIGNED); - } + while (checkpointIterator.hasNext()) { - // Restart failed vertex - restart(failedVertex); + checkpointIterator.next().updateExecutionState(ExecutionState.ASSIGNED); + } + + // Restart failed vertex + failedVertex.updateExecutionState(getStateToUpdate(failedVertex)); + } return true; } - private static void restart(final ExecutionVertex vertex) { + private static ExecutionState getStateToUpdate(final ExecutionVertex vertex) { if (vertex.getAllocatedResource().getInstance() instanceof DummyInstance) { - vertex.updateExecutionState(ExecutionState.CREATED); - } else { - vertex.updateExecutionState(ExecutionState.ASSIGNED); + return ExecutionState.CREATED; } + + return ExecutionState.ASSIGNED; } private static void findVerticesToRestart(final ExecutionVertex failedVertex, diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index 81c6047342d0b..c63d5d3db1453 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -103,7 +103,7 @@ public RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment en public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) { // Check the state transition - ExecutionStateTransition.checkTransition(getTaskName(), this.executionState, newExecutionState); + ExecutionStateTransition.checkTransition(false, getTaskName(), this.executionState, newExecutionState); // Make sure the reason for a transition to FAILED appears in the log files if (newExecutionState == ExecutionState.FAILED) { @@ -229,6 +229,13 @@ private void cancelOrKillExecution(final boolean cancel) { return; } + if (this.executionState != ExecutionState.RUNNING && this.executionState != ExecutionState.REPLAYING + && this.executionState != ExecutionState.FINISHING) { + return; + } + + LOG.info((cancel ? "Canceling " : "Killing ") + this.environment.getTaskNameWithIndex()); + if (cancel) { this.isCanceled = true; // Change state @@ -250,14 +257,8 @@ private void cancelOrKillExecution(final boolean cancel) { executingThread.interrupt(); - if (cancel) { - if (this.executionState == ExecutionState.CANCELED) { - break; - } - } else { - if (this.executionState == ExecutionState.FAILED) { - break; - } + if (!executingThread.isAlive()) { + break; } try { @@ -265,6 +266,9 @@ private void cancelOrKillExecution(final boolean cancel) { } catch (InterruptedException e) { break; } + + LOG.info((cancel == true ? "Canceling " : "Killing ") + this.environment.getTaskName() + + " with state " + this.executionState); } } From 44e14f6abb22ea8c5969ef2b38bb3d5f68e5963c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 6 Feb 2012 17:27:41 +0100 Subject: [PATCH 221/310] Fixed javadoc --- .../stratosphere/nephele/protocols/ChannelLookupProtocol.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java index c3795d287f569..1a413b6bf5b9f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java @@ -41,6 +41,8 @@ public interface ChannelLookupProtocol extends VersionedProtocol { * @param sourceChannelID * the ID of the channel to resolve * @return the lookup response containing the connection info and a return code + * @throws IOException + * thrown if an error occurs during the IPC call */ ConnectionInfoLookupResponse lookupConnectionInfo(InstanceConnectionInfo caller, JobID jobID, ChannelID sourceChannelID) throws IOException; From ef78fc49b58cf0e684704f2a262028b98045f4e3 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 6 Feb 2012 18:11:41 +0100 Subject: [PATCH 222/310] Fixed bug in channel close logic when receiving task has already finished --- .../nephele/jobmanager/JobManager.java | 6 +- .../nephele/taskmanager/TaskManager.java | 3 - .../ByteBufferedChannelManager.java | 51 ++++------------- .../ConnectionInfoLookupResponse.java | 14 +---- ...rEvent.java => ReceiverNotFoundEvent.java} | 56 +++++++++++++------ .../runtime/RuntimeOutputChannelBroker.java | 18 +++++- 6 files changed, 71 insertions(+), 77 deletions(-) rename nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/{UnknownReceiverEvent.java => ReceiverNotFoundEvent.java} (57%) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index d150680b2b805..d24be8283e3bd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -810,12 +810,8 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio // Check execution state final ExecutionState executionState = targetVertex.getExecutionState(); - if (executionState == ExecutionState.FINISHED) { - return ConnectionInfoLookupResponse.createReceiverFinished(); - } - if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.REPLAYING - && executionState != ExecutionState.FINISHING) { + && executionState != ExecutionState.FINISHING && executionState != ExecutionState.FINISHED) { return ConnectionInfoLookupResponse.createReceiverNotReady(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 766b78844dfb8..4566e024cbddd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -390,9 +390,6 @@ public void runIOLoop() { // Check the status of the task threads to detect unexpected thread terminations checkTaskExecution(); - - // Clean up set of recently unregistered channels - this.byteBufferedChannelManager.cleanUpRecentlyRemovedChannelIDSet(); } // Shutdown the individual components of the task manager diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 1e313766b1140..c28beed846516 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -71,8 +71,6 @@ public final class ByteBufferedChannelManager implements TransferEnvelopeDispatc private final NetworkConnectionManager networkConnectionManager; - private final RecentlyRemovedChannelIDSet recentlyRemovedChannelIDSet = new RecentlyRemovedChannelIDSet(); - private final ChannelLookupProtocol channelLookupService; private final InstanceConnectionInfo localConnectionInfo; @@ -222,9 +220,6 @@ public void unregister(final ExecutionVertexID vertexID, final Task task) { final Environment environment = task.getEnvironment(); - // Mark all channel IDs to be recently removed - this.recentlyRemovedChannelIDSet.add(environment); - Iterator channelIterator = environment.getOutputChannelIDs().iterator(); while (channelIterator.hasNext()) { @@ -290,15 +285,19 @@ private void recycleBuffer(final TransferEnvelope envelope) { } } - private void sendReceiverNotFoundEvent(final JobID jobID, final ChannelID unknownReceiver) { + private void sendReceiverNotFoundEvent(final TransferEnvelope envelope, final ChannelID unknownReceiver) { - if (ChannelID.SYSTEM_ID.equals(unknownReceiver)) { + if (ChannelID.SYSTEM_ID.equals(envelope.getSource())) { LOG.error("Requested to send unknown receiver event from the system, dropping request..."); return; } + final JobID jobID = envelope.getJobID(); + final TransferEnvelope transferEnvelope = new TransferEnvelope(0, jobID, ChannelID.SYSTEM_ID); - final UnknownReceiverEvent unknownReceiverEvent = new UnknownReceiverEvent(unknownReceiver); + + final ReceiverNotFoundEvent unknownReceiverEvent = new ReceiverNotFoundEvent(unknownReceiver, + envelope.getSequenceNumber()); transferEnvelope.addEvent(unknownReceiverEvent); final TransferEnvelopeReceiverList receiverList = getReceiverList(jobID, unknownReceiver); @@ -344,10 +343,7 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, final ChannelContext cc = this.registeredChannels.get(localReceiver); if (cc == null) { - if (!this.recentlyRemovedChannelIDSet.contains(localReceiver)) { - sendReceiverNotFoundEvent(transferEnvelope.getJobID(), localReceiver); - } - + sendReceiverNotFoundEvent(transferEnvelope, localReceiver); recycleBuffer(transferEnvelope); return; } @@ -374,10 +370,7 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, final ChannelContext cc = this.registeredChannels.get(localReceiver); if (cc == null) { - if (!this.recentlyRemovedChannelIDSet.contains(localReceiver)) { - sendReceiverNotFoundEvent(transferEnvelope.getJobID(), localReceiver); - } - + sendReceiverNotFoundEvent(transferEnvelope, localReceiver); continue; } @@ -443,10 +436,7 @@ private boolean processEnvelopeEnvelopeWithoutBuffer(final TransferEnvelope tran final ChannelContext channelContext = this.registeredChannels.get(localReceiver); if (channelContext == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Cannot find local receiver " + localReceiver + " for job " - + transferEnvelope.getJobID()); - } + sendReceiverNotFoundEvent(transferEnvelope, localReceiver); continue; } channelContext.queueTransferEnvelope(transferEnvelope); @@ -494,11 +484,6 @@ private TransferEnvelopeReceiverList getReceiverList(final JobID jobID, final Ch continue; } - if (lookupResponse.receiverHasFinished()) { - // TODO: Send close notification here - break; - } - if (lookupResponse.receiverReady()) { receiverList = new TransferEnvelopeReceiverList(lookupResponse); break; @@ -627,12 +612,8 @@ public BufferProvider getBufferProvider(final JobID jobID, final ChannelID sourc final ChannelContext cc = this.registeredChannels.get(localReceiver); if (cc == null) { - if (this.recentlyRemovedChannelIDSet.contains(localReceiver)) { - // Use the transit buffer for this purpose, data will be discarded in most cases anyway. - return this.transitBufferPool; - } else { - throw new IOException("Cannot find channel context for local receiver " + localReceiver); - } + // Use the transit buffer for this purpose, data will be discarded in most cases anyway. + return this.transitBufferPool; } if (!cc.isInputChannel()) { @@ -724,14 +705,6 @@ public boolean registerSpillingQueueWithNetworkConnection(final JobID jobID, fin return true; } - /** - * Triggers the clean-up method of the canceled channel ID set. - */ - public void cleanUpRecentlyRemovedChannelIDSet() { - - this.recentlyRemovedChannelIDSet.cleanup(); - } - /** * Invalidates the entries identified by the given channel IDs from the receiver lookup cache. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java index bfa7dadf289af..5d9fa571dc69a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java @@ -30,7 +30,7 @@ public class ConnectionInfoLookupResponse implements IOReadableWritable { private enum ReturnCode { - NOT_FOUND, FOUND_AND_RECEIVER_READY, FOUND_BUT_RECEIVER_NOT_READY, FOUND_BUT_RECEIVER_FINISHED + NOT_FOUND, FOUND_AND_RECEIVER_READY, FOUND_BUT_RECEIVER_NOT_READY }; // was request successful? @@ -104,11 +104,6 @@ public boolean receiverReady() { return (this.returnCode == ReturnCode.FOUND_AND_RECEIVER_READY); } - public boolean receiverHasFinished() { - - return (this.returnCode == ReturnCode.FOUND_BUT_RECEIVER_FINISHED); - } - public static ConnectionInfoLookupResponse createReceiverFoundAndReady(ChannelID targetChannelID) { final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse(); @@ -154,13 +149,6 @@ public static ConnectionInfoLookupResponse createReceiverNotReady() { return response; } - public static ConnectionInfoLookupResponse createReceiverFinished() { - final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse(); - response.setReturnCode(ReturnCode.FOUND_BUT_RECEIVER_FINISHED); - - return response; - } - @Override public String toString() { StringBuilder returnstring = new StringBuilder(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnknownReceiverEvent.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java similarity index 57% rename from nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnknownReceiverEvent.java rename to nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java index ec5893fb9a510..50c0e85ef3afa 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnknownReceiverEvent.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java @@ -28,44 +28,66 @@ * * @author warneke */ -public final class UnknownReceiverEvent extends AbstractEvent { +public final class ReceiverNotFoundEvent extends AbstractEvent { /** - * The ID of the unknown receiver. + * The ID of the receiver which could not be found */ - private ChannelID unknownReceiverID; + private ChannelID receiverID; + + /** + * The sequence number of the envelope this event refers to + */ + private int sequenceNumber; /** * Constructs a new unknown receiver event. * - * @param unknownReceiverID - * the ID of the unknown receiver + * @param receiverID + * the ID of the receiver which could not be found + * @param sequenceNumber + * the sequence number of the envelope this event refers to */ - public UnknownReceiverEvent(final ChannelID unknownReceiverID) { + public ReceiverNotFoundEvent(final ChannelID receiverID, final int sequenceNumber) { - if (unknownReceiverID == null) { + if (receiverID == null) { throw new IllegalArgumentException("Argument unknownReceiverID must not be null"); } - this.unknownReceiverID = unknownReceiverID; + if (sequenceNumber < 0) { + throw new IllegalArgumentException("Argument sequenceNumber must be non-negative"); + } + + this.receiverID = receiverID; + this.sequenceNumber = sequenceNumber; } /** * Default constructor for serialization/deserialization. */ - public UnknownReceiverEvent() { + public ReceiverNotFoundEvent() { + + this.receiverID = new ChannelID(); + } + + /** + * Returns the ID of the receiver which could not be found. + * + * @return the ID of the receiver which could not be found + */ + public ChannelID getReceiverID() { - this.unknownReceiverID = new ChannelID(); + return this.receiverID; } /** - * Returns the ID of the unknown receiver. + * Returns the sequence number of the envelope this event refers to. * - * @return the ID of the unknown receiver + * @return the sequence number of the envelope this event refers to */ - public ChannelID getUnknownReceiverID() { + public int getSequenceNumber() { - return this.unknownReceiverID; + return this.sequenceNumber; } /** @@ -74,7 +96,8 @@ public ChannelID getUnknownReceiverID() { @Override public void write(final DataOutput out) throws IOException { - this.unknownReceiverID.write(out); + this.receiverID.write(out); + out.writeInt(this.sequenceNumber); } /** @@ -83,7 +106,8 @@ public void write(final DataOutput out) throws IOException { @Override public void read(final DataInput in) throws IOException { - this.unknownReceiverID.read(in); + this.receiverID.read(in); + this.sequenceNumber = in.readInt(); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java index b87ae04d6ed67..3ce115bba0959 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java @@ -16,6 +16,7 @@ import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; +import eu.stratosphere.nephele.taskmanager.bytebuffered.ReceiverNotFoundEvent; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; final class RuntimeOutputChannelBroker implements ByteBufferedOutputChannelBroker, OutputChannelForwarder { @@ -56,6 +57,11 @@ final class RuntimeOutputChannelBroker implements ByteBufferedOutputChannelBroke */ private boolean closeAcknowledgementReceived = false; + /** + * Stores the last sequence number of the transfer envelope for which the receiver could not be found. + */ + private int lastSequenceNumberWithReceiverNotFound = -1; + /** * The sequence number for the next {@link TransferEnvelope} to be created. */ @@ -89,7 +95,15 @@ public boolean forward(final TransferEnvelope transferEnvelope) { @Override public boolean hasDataLeft() { - return (!this.closeAcknowledgementReceived); + if (this.closeAcknowledgementReceived) { + return false; + } + + if ((this.lastSequenceNumberWithReceiverNotFound + 1) == this.sequenceNumber) { + return false; + } + + return true; } /** @@ -100,6 +114,8 @@ public void processEvent(final AbstractEvent event) { if (event instanceof ByteBufferedChannelCloseEvent) { this.closeAcknowledgementReceived = true; + } else if (event instanceof ReceiverNotFoundEvent) { + this.lastSequenceNumberWithReceiverNotFound = ((ReceiverNotFoundEvent) event).getSequenceNumber(); } else if (event instanceof AbstractTaskEvent) { this.byteBufferedOutputChannel.processEvent(event); } From c6ece443307a4ed4eb065dce34f02ee3fa95f2e7 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 6 Feb 2012 19:06:52 +0100 Subject: [PATCH 223/310] Implemented input split tracker to make replay of input tasks deterministic --- .../splitassigner/InputSplitManager.java | 20 ++- .../splitassigner/InputSplitTracker.java | 164 ++++++++++++++++++ 2 files changed, 183 insertions(+), 1 deletion(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitTracker.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java index 6a2fd57c2f171..5078c5becb474 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitManager.java @@ -73,6 +73,12 @@ public final class InputSplitManager { */ private final Map, InputSplitAssigner> loadedAssigners = new HashMap, InputSplitAssigner>(); + /** + * The input split tracker makes sure that a vertex retrieves the same sequence of input splits after being + * restarted. + */ + private final InputSplitTracker inputSplitTracker = new InputSplitTracker(); + /** * The default input split assigner which is always used if a more specific assigner cannot be found. */ @@ -118,6 +124,8 @@ public void registerJob(final ExecutionGraph executionGraph) { assigner.registerGroupVertex(groupVertex); } + // Register job with the input split tracker + this.inputSplitTracker.registerJob(executionGraph); } /** @@ -151,6 +159,9 @@ public void unregisterJob(final ExecutionGraph executionGraph) { assigner.unregisterGroupVertex(groupVertex); } + + // Unregister job from input split tracker + this.inputSplitTracker.unregisterJob(executionGraph); } /** @@ -165,6 +176,12 @@ public void unregisterJob(final ExecutionGraph executionGraph) { */ public InputSplit getNextInputSplit(final ExecutionVertex vertex, final int sequenceNumber) { + InputSplit nextInputSplit = this.inputSplitTracker.getInputSplitFromLog(vertex, sequenceNumber); + if (nextInputSplit != null) { + LOG.info("Input split " + nextInputSplit.getSplitNumber() + " for vertex " + vertex + " replayed from log"); + return nextInputSplit; + } + final ExecutionGroupVertex groupVertex = vertex.getGroupVertex(); final InputSplitAssigner inputSplitAssigner = this.assignerCache.get(groupVertex); if (inputSplitAssigner == null) { @@ -173,8 +190,9 @@ public InputSplit getNextInputSplit(final ExecutionVertex vertex, final int sequ return null; } - final InputSplit nextInputSplit = inputSplitAssigner.getNextInputSplit(vertex); + nextInputSplit = inputSplitAssigner.getNextInputSplit(vertex); if (nextInputSplit != null) { + this.inputSplitTracker.addInputSplitToLog(vertex, sequenceNumber, nextInputSplit); LOG.info(vertex + " receives input split " + nextInputSplit.getSplitNumber()); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitTracker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitTracker.java new file mode 100644 index 0000000000000..6e9c1f7d5adcc --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitTracker.java @@ -0,0 +1,164 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.jobmanager.splitassigner; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; +import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; +import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertexIterator; +import eu.stratosphere.nephele.executiongraph.ExecutionVertex; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.template.InputSplit; + +/** + * The input split tracker maintains a log of all the input splits that are handed out to the individual input vertices. + * In case of an input vertex must be restarted the input split tracker makes sure that the vertex receives the same + * sequence of input splits as in its original run up to the point that it crashed. + *

+ * This class is thread-safe. + * + * @author warneke + */ +final class InputSplitTracker { + + /** + * The logging object which is used to report information and errors. + */ + private static final Log LOG = LogFactory.getLog(InputSplitTracker.class); + + /** + * The central split map which stores the logs of the individual input vertices. + */ + private final ConcurrentMap> splitMap = new ConcurrentHashMap>(); + + /** + * Constructor with package visibility only. + */ + InputSplitTracker() { + } + + /** + * Registers a new job with the input split tracker. + * + * @param eg + * the execution graph of the job to be registered + */ + void registerJob(final ExecutionGraph eg) { + + final Iterator it = new ExecutionGroupVertexIterator(eg, true, -1); + while (it.hasNext()) { + + final ExecutionGroupVertex groupVertex = it.next(); + final InputSplit[] inputSplits = groupVertex.getInputSplits(); + + if (inputSplits == null) { + continue; + } + + if (inputSplits.length == 0) { + continue; + } + + for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); ++i) { + final ExecutionVertex vertex = groupVertex.getGroupMember(i); + if (this.splitMap.put(vertex.getID(), new ArrayList()) != null) { + LOG.error("InputSplitTracker must keep track of two vertices with ID " + vertex.getID()); + } + } + } + } + + /** + * Unregisters a job from the input split tracker. + * + * @param eg + * the execution graph of the job to be unregistered + */ + void unregisterJob(final ExecutionGraph eg) { + + final Iterator it = new ExecutionGraphIterator(eg, true); + while (it.hasNext()) { + this.splitMap.remove(it.next().getID()); + } + } + + /** + * Returns the input split with the given sequence number from the specified vertex's log or null if no + * such input split exists. + * + * @param vertex + * the vertex for which the input split shall be returned from the log + * @param sequenceNumber + * the sequence number identifying the log entry + * @return the input split that was stored under the given sequence number of the vertex's log or null + * if no such input split exists + */ + InputSplit getInputSplitFromLog(final ExecutionVertex vertex, final int sequenceNumber) { + + final List inputSplitLog = this.splitMap.get(vertex.getID()); + if (inputSplitLog == null) { + LOG.error("Cannot find input split log for vertex " + vertex + " (" + vertex.getID() + ")"); + return null; + } + + synchronized (inputSplitLog) { + + if (sequenceNumber < inputSplitLog.size()) { + return inputSplitLog.get(sequenceNumber); + } + } + + return null; + } + + /** + * Adds the given input split to the vertex's log and stores it under the specified sequence number. + * + * @param vertex + * the vertex for which the input split shall be stored + * @param sequenceNumber + * the sequence number identifying the log entry under which the input split shall be stored + * @param inputSplit + * the input split to be stored + */ + void addInputSplitToLog(final ExecutionVertex vertex, final int sequenceNumber, final InputSplit inputSplit) { + + final List inputSplitLog = this.splitMap.get(vertex.getID()); + if (inputSplitLog == null) { + LOG.error("Cannot find input split log for vertex " + vertex + " (" + vertex.getID() + ")"); + return; + } + + synchronized (inputSplitLog) { + if (inputSplitLog.size() != sequenceNumber) { + LOG.error("Expected input split with sequence number " + inputSplitLog.size() + " for vertex " + vertex + + " (" + vertex.getID() + ") but received " + sequenceNumber + ", skipping..."); + return; + } + + inputSplitLog.add(inputSplit); + } + } +} From b0f17dde8560da5989a34b979b8a2b5f9ad9e74e Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 6 Feb 2012 20:54:18 +0100 Subject: [PATCH 224/310] Fixed bug related to channel lookup service --- .../nephele/io/channels/ChannelID.java | 16 --------- .../ByteBufferedChannelManager.java | 33 ++++++++++++++----- 2 files changed, 24 insertions(+), 25 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java index 699d8065c4ae8..999d8fc4b1519 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java @@ -24,26 +24,10 @@ */ public class ChannelID extends AbstractID { - /** - * Constructs a new channel ID from the provided byte array. - * - * @param bytes - * the byte array to construct the channel ID from - */ - private ChannelID(final byte[] bytes) { - super(bytes); - } - /** * Constructs a new, random channel ID. */ public ChannelID() { super(); } - - /** - * The channel ID used by the framework to send notifications about problems with the data delivery. - */ - public static final ChannelID SYSTEM_ID = new ChannelID(new byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, - 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 }); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index c28beed846516..cf082bf3eb7c3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -28,6 +28,8 @@ import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.configuration.GlobalConfiguration; +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.event.task.EventList; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; @@ -285,24 +287,31 @@ private void recycleBuffer(final TransferEnvelope envelope) { } } - private void sendReceiverNotFoundEvent(final TransferEnvelope envelope, final ChannelID unknownReceiver) { + private void sendReceiverNotFoundEvent(final TransferEnvelope envelope, final ChannelID receiver) { - if (ChannelID.SYSTEM_ID.equals(envelope.getSource())) { - LOG.error("Requested to send unknown receiver event from the system, dropping request..."); - return; + if (envelope.getBuffer() == null && envelope.getSequenceNumber() == 0) { + + final EventList eventList = envelope.getEventList(); + if (eventList.size() == 1) { + final AbstractEvent event = eventList.get(0); + if (event instanceof ReceiverNotFoundEvent) { + LOG.info("Dropping request to send ReceiverNotFoundEvent as response to ReceiverNotFoundEvent"); + return; + } + } } final JobID jobID = envelope.getJobID(); - final TransferEnvelope transferEnvelope = new TransferEnvelope(0, jobID, ChannelID.SYSTEM_ID); + final TransferEnvelope transferEnvelope = new TransferEnvelope(0, jobID, receiver); - final ReceiverNotFoundEvent unknownReceiverEvent = new ReceiverNotFoundEvent(unknownReceiver, + final ReceiverNotFoundEvent unknownReceiverEvent = new ReceiverNotFoundEvent(receiver, envelope.getSequenceNumber()); transferEnvelope.addEvent(unknownReceiverEvent); - final TransferEnvelopeReceiverList receiverList = getReceiverList(jobID, unknownReceiver); + final TransferEnvelopeReceiverList receiverList = getReceiverList(jobID, receiver); if (receiverList == null) { - LOG.error("Cannot determine receiver list for source channel ID " + unknownReceiver); + LOG.error("Cannot determine receiver list for source channel ID " + receiver); return; } @@ -471,8 +480,11 @@ private TransferEnvelopeReceiverList getReceiverList(final JobID jobID, final Ch try { while (true) { - final ConnectionInfoLookupResponse lookupResponse = this.channelLookupService.lookupConnectionInfo( + ConnectionInfoLookupResponse lookupResponse; + synchronized (this.channelLookupService) { + lookupResponse = this.channelLookupService.lookupConnectionInfo( this.localConnectionInfo, jobID, sourceChannelID); + } if (lookupResponse.receiverNotFound()) { throw new IOException("Cannot find task(s) waiting for data from source channel with ID " @@ -596,6 +608,9 @@ public void logBufferUtilization() { } } + /** + * {@inheritDoc} + */ @Override public BufferProvider getBufferProvider(final JobID jobID, final ChannelID sourceChannelID) throws IOException, InterruptedException { From 48b66dd9c673f091754fcfb3c2eab4a9a8e54174 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 7 Feb 2012 11:03:35 +0100 Subject: [PATCH 225/310] Minor performance improvement --- .../executiongraph/ExecutionStage.java | 26 +++++++++---------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java index 5433863918c45..4e29996cf8730 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java @@ -408,15 +408,14 @@ private void reconstructExecutionPipeline(final ExecutionVertex vertex, final bo .getConnectedChannelID()); boolean recurse = false; - - if(!alreadyVisited.contains(connectedVertex)) { - recurse = true; - } else if(channelType == ChannelType.INMEMORY && !pipeline.equals(connectedVertex.getExecutionPipeline())) { + + if (!alreadyVisited.contains(connectedVertex)) { recurse = true; - } - - if (channelType == ChannelType.INMEMORY) { + } else if (channelType == ChannelType.INMEMORY + && !pipeline.equals(connectedVertex.getExecutionPipeline())) { + connectedVertex.setExecutionPipeline(pipeline); + recurse = true; } if (recurse) { @@ -439,15 +438,14 @@ private void reconstructExecutionPipeline(final ExecutionVertex vertex, final bo .getConnectedChannelID()); boolean recurse = false; - - if(!alreadyVisited.contains(connectedVertex)) { - recurse = true; - } else if(channelType == ChannelType.INMEMORY && !pipeline.equals(connectedVertex.getExecutionPipeline())) { + + if (!alreadyVisited.contains(connectedVertex)) { recurse = true; - } - - if (channelType == ChannelType.INMEMORY) { + } else if (channelType == ChannelType.INMEMORY + && !pipeline.equals(connectedVertex.getExecutionPipeline())) { + connectedVertex.setExecutionPipeline(pipeline); + recurse = true; } if (recurse) { From f0b7ee3adcebe4d911e383de4e55783f9dde208a Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 7 Feb 2012 13:21:36 +0100 Subject: [PATCH 226/310] Fixed problem with memory leaks --- .../nephele/io/channels/BufferFactory.java | 7 +- .../nephele/io/channels/MemoryBuffer.java | 5 +- .../channels/MemoryBufferPoolConnector.java | 37 +++++++++++ .../io/channels/MemoryBufferRecycler.java | 51 ++++++--------- .../bufferprovider/LocalBufferPool.java | 64 +++++++++++++++---- .../ByteBufferedChannelManager.java | 2 +- .../runtime/RuntimeInputGateContext.java | 9 +-- .../runtime/RuntimeTaskContext.java | 10 +-- 8 files changed, 118 insertions(+), 67 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java index 33a87e3cbe8e1..1979b0388de2e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java @@ -16,7 +16,6 @@ package eu.stratosphere.nephele.io.channels; import java.nio.ByteBuffer; -import java.util.Queue; import eu.stratosphere.nephele.io.AbstractID; import eu.stratosphere.nephele.io.channels.Buffer; @@ -35,14 +34,14 @@ public static Buffer createFromCheckpoint(final int bufferSize, final FileID fil final AbstractID ownerID, final FileBufferManager fileBufferManager) { final InternalBuffer internalBuffer = new FileBuffer(bufferSize, fileID, offset, ownerID, fileBufferManager); - + return new Buffer(internalBuffer); } public static Buffer createFromMemory(final int bufferSize, final ByteBuffer byteBuffer, - final Queue queueForRecycledBuffers) { + final MemoryBufferPoolConnector bufferPoolConnector) { - final InternalBuffer internalBuffer = new MemoryBuffer(bufferSize, byteBuffer, queueForRecycledBuffers); + final InternalBuffer internalBuffer = new MemoryBuffer(bufferSize, byteBuffer, bufferPoolConnector); return new Buffer(internalBuffer); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java index 14743da11d736..b204488471eab 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java @@ -19,7 +19,6 @@ import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; -import java.util.Queue; import java.util.concurrent.atomic.AtomicBoolean; import eu.stratosphere.nephele.io.channels.InternalBuffer; @@ -32,14 +31,14 @@ public class MemoryBuffer implements InternalBuffer { private final AtomicBoolean writeMode = new AtomicBoolean(true); - MemoryBuffer(final int bufferSize, final ByteBuffer byteBuffer, final Queue queueForRecycledBuffers) { + MemoryBuffer(final int bufferSize, final ByteBuffer byteBuffer, final MemoryBufferPoolConnector bufferPoolConnector) { if (bufferSize > byteBuffer.capacity()) { throw new IllegalArgumentException("Requested buffer size is " + bufferSize + ", but provided byte buffer only has a capacity of " + byteBuffer.capacity()); } - this.bufferRecycler = new MemoryBufferRecycler(byteBuffer, queueForRecycledBuffers); + this.bufferRecycler = new MemoryBufferRecycler(byteBuffer, bufferPoolConnector); this.byteBuffer = byteBuffer; this.byteBuffer.position(0); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java new file mode 100644 index 0000000000000..e08936e22c1b1 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java @@ -0,0 +1,37 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.io.channels; + +import java.nio.ByteBuffer; + +import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; + +/** + * The memory buffer pool connector provides a connection between {@link MemoryBuffer} and the {@link LocalBufferPool} + * the memory buffer's encapsulated byte buffer has originally been taken from. + * + * @author warneke + */ +public interface MemoryBufferPoolConnector { + + /** + * Called by the {@link MemoryBufferRecycler} to return a buffer to its original buffer pool. + * + * @param byteBuffer + * the buffer to be recycled + */ + void recycle(ByteBuffer byteBuffer); +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.java index 1d8f88df45ac8..1a2a27f502d1f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.java @@ -16,7 +16,7 @@ package eu.stratosphere.nephele.io.channels; import java.nio.ByteBuffer; -import java.util.Queue; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -43,68 +43,53 @@ public final class MemoryBufferRecycler { private final ByteBuffer originalBuffer; /** - * The queue to which the byte buffer shall be appended after all copies have been processed. + * The connection to the pool from which the byte buffer has originally been taken. */ - private final Queue queueForRecycledBuffers; + private final MemoryBufferPoolConnector bufferPoolConnector; /** * The number of memory buffer objects which may still access the physical buffer. */ - private int referenceCounter = 1; - - /** - * Stores if the physical buffer has already been recycled. - */ - private boolean bufferAlreadyRecycled = false; + private final AtomicInteger referenceCounter = new AtomicInteger(1); /** * Constructs a new memory buffer recycler. * * @param originalBuffer * the original byte buffer - * @param queueForRecycledBuffers - * the queue to append the buffer for recycling + * @param bufferPoolConnector + * the connection to the pool from which the byte buffer has originally been taken */ - MemoryBufferRecycler(final ByteBuffer originalBuffer, final Queue queueForRecycledBuffers) { + MemoryBufferRecycler(final ByteBuffer originalBuffer, final MemoryBufferPoolConnector bufferPoolConnector) { this.originalBuffer = originalBuffer; - this.queueForRecycledBuffers = queueForRecycledBuffers; + this.bufferPoolConnector = bufferPoolConnector; } /** * Increases the number of references to the physical buffer by one. */ - synchronized void increaseReferenceCounter() { + void increaseReferenceCounter() { - if (this.bufferAlreadyRecycled) { - LOG.error("increaseReferenceCounter called although buffer has already been recycled"); + if (this.referenceCounter.getAndIncrement() == 0) { + LOG.error("Increasing reference counter from 0 to 1"); } - - ++this.referenceCounter; } /** * Decreases the number of references to the physical buffer by one. If the number of references becomes zero the * physical buffer is recycled. */ - synchronized void decreaseReferenceCounter() { + void decreaseReferenceCounter() { - if (this.bufferAlreadyRecycled) { - LOG.error("decreaseReferenceCounter called although buffer has already been recycled"); - } - - --this.referenceCounter; - - if (this.referenceCounter <= 0) { + final int val = this.referenceCounter.decrementAndGet(); + if (val == 0) { this.originalBuffer.clear(); - - synchronized (this.queueForRecycledBuffers) { - this.queueForRecycledBuffers.add(this.originalBuffer); - this.queueForRecycledBuffers.notify(); - } - - this.bufferAlreadyRecycled = true; + this.bufferPoolConnector.recycle(this.originalBuffer); + + } else if (val < 0) { + LOG.error("reference counter is negative"); } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java index 889d56620fd81..943d14f0d291f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java @@ -25,12 +25,30 @@ import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.BufferFactory; +import eu.stratosphere.nephele.io.channels.MemoryBufferPoolConnector; public final class LocalBufferPool implements BufferProvider { - private final static Log LOG = LogFactory.getLog(LocalBufferPool.class); + private static final class LocalBufferPoolConnector implements MemoryBufferPoolConnector { + + private final LocalBufferPool localBufferPool; + + private LocalBufferPoolConnector(final LocalBufferPool localBufferPool) { + this.localBufferPool = localBufferPool; + } + + /** + * {@inheritDoc} + */ + @Override + public void recycle(final ByteBuffer byteBuffer) { + + this.localBufferPool.recycleBuffer(byteBuffer); + } - private final String ownerName; + } + + private final static Log LOG = LogFactory.getLog(LocalBufferPool.class); private final GlobalBufferPool globalBufferPool; @@ -44,23 +62,27 @@ public final class LocalBufferPool implements BufferProvider { private boolean asynchronousEventOccurred = false; + private boolean isDestroyed = false; + private final AsynchronousEventListener eventListener; private final Queue buffers = new ArrayDeque(); - public LocalBufferPool(final String ownerName, final int designatedNumberOfBuffers, final boolean isShared, + private final LocalBufferPoolConnector bufferPoolConnector; + + public LocalBufferPool(final int designatedNumberOfBuffers, final boolean isShared, final AsynchronousEventListener eventListener) { - this.ownerName = ownerName; this.globalBufferPool = GlobalBufferPool.getInstance(); this.maximumBufferSize = this.globalBufferPool.getMaximumBufferSize(); this.designatedNumberOfBuffers = designatedNumberOfBuffers; this.isShared = isShared; this.eventListener = eventListener; + this.bufferPoolConnector = new LocalBufferPoolConnector(this); } - public LocalBufferPool(final String ownerName, final int designatedNumberOfBuffers, final boolean isShared) { - this(ownerName, designatedNumberOfBuffers, isShared, null); + public LocalBufferPool(final int designatedNumberOfBuffers, final boolean isShared) { + this(designatedNumberOfBuffers, isShared, null); } /** @@ -142,7 +164,7 @@ private Buffer requestBufferInternal(final int minimumSizeOfBuffer, final boolea if (!async) { final ByteBuffer byteBuffer = this.buffers.poll(); - return BufferFactory.createFromMemory(minimumSizeOfBuffer, byteBuffer, this.buffers); + return BufferFactory.createFromMemory(minimumSizeOfBuffer, byteBuffer, this.bufferPoolConnector); } } @@ -188,23 +210,23 @@ public void setDesignatedNumberOfBuffers(final int designatedNumberOfBuffers) { } } - public void clear() { + public void destroy() { synchronized (this.buffers) { - if (this.requestedNumberOfBuffers != this.buffers.size()) { - - LOG.error(this.ownerName + ": Requested number of buffers is " + this.requestedNumberOfBuffers - + ", but only " + this.buffers.size() + " buffers in local pool"); + if (this.isDestroyed) { + LOG.error("destroy is called on LocalBufferPool multiple times"); + return; } + this.isDestroyed = true; + while (!this.buffers.isEmpty()) { this.globalBufferPool.releaseGlobalBuffer(this.buffers.poll()); } this.requestedNumberOfBuffers = 0; } - } /** @@ -237,6 +259,22 @@ public int getRequestedNumberOfBuffers() { } } + private void recycleBuffer(final ByteBuffer byteBuffer) { + + synchronized (this.buffers) { + + if (this.isDestroyed) { + this.globalBufferPool.releaseGlobalBuffer(this.buffers.poll()); + this.requestedNumberOfBuffers--; + return; + } + + this.buffers.add(byteBuffer); + this.buffers.notify(); + } + + } + /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index cf082bf3eb7c3..505c36e9c954f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -105,7 +105,7 @@ public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupServi GlobalBufferPool.getInstance(); // Initialize the transit buffer pool - this.transitBufferPool = new LocalBufferPool("Transit buffer pool", 128, true); + this.transitBufferPool = new LocalBufferPool(128, true); this.networkConnectionManager = new NetworkConnectionManager(this, localInstanceConnectionInfo.getAddress(), localInstanceConnectionInfo.getDataPort()); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index 6eb61c8774122..85b0edf14de2f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -39,13 +39,10 @@ final class RuntimeInputGateContext implements BufferProvider, InputGateContext, private final InputGate inputGate; - RuntimeInputGateContext(final String taskName, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + RuntimeInputGateContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final InputGate inputGate) { - final String poolOwnerName = (taskName == null ? "Unknown task" : taskName + " (Input Gate " - + inputGate.getIndex() + ")"); - - this.localBufferPool = new LocalBufferPool(poolOwnerName, 1, false); + this.localBufferPool = new LocalBufferPool(1, false); this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.inputGate = inputGate; @@ -117,7 +114,7 @@ public void setDesignatedNumberOfBuffers(int numberOfBuffers) { @Override public void clearLocalBufferPool() { - this.localBufferPool.clear(); + this.localBufferPool.destroy(); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 001a7a2589fea..474bf930bd143 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -60,10 +60,7 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve RuntimeTaskContext(final RuntimeTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final Map tasksWithUndecidedCheckpoints) { - final String poolOwnerName = (task.getEnvironment().getTaskName() == null ? "Unkown task" : task - .getEnvironment().getTaskName()); - - this.localBufferPool = new LocalBufferPool(poolOwnerName, 1, false, this); + this.localBufferPool = new LocalBufferPool(1, false, this); this.task = task; final RuntimeEnvironment environment = task.getRuntimeEnvironment(); @@ -134,7 +131,7 @@ public int getMaximumBufferSize() { public void clearLocalBufferPool() { // Clear the buffer cache - this.localBufferPool.clear(); + this.localBufferPool.destroy(); } /** @@ -273,8 +270,7 @@ public InputGateContext createInputGateContext(final GateID gateID) { throw new IllegalStateException("Cannot find input gate with ID " + gateID); } - return new RuntimeInputGateContext(this.task.getEnvironment().getTaskName(), this.transferEnvelopeDispatcher, - inputGate); + return new RuntimeInputGateContext(this.transferEnvelopeDispatcher, inputGate); } /** From 50af9f7f5c8c284942d04d72e4e41676827f825f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 7 Feb 2012 14:16:28 +0100 Subject: [PATCH 227/310] Fixed bug in local buffer pool --- .../nephele/taskmanager/bufferprovider/LocalBufferPool.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java index 943d14f0d291f..2f6d2bf62a1f4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java @@ -264,7 +264,7 @@ private void recycleBuffer(final ByteBuffer byteBuffer) { synchronized (this.buffers) { if (this.isDestroyed) { - this.globalBufferPool.releaseGlobalBuffer(this.buffers.poll()); + this.globalBufferPool.releaseGlobalBuffer(byteBuffer); this.requestedNumberOfBuffers--; return; } From c0cc062cc11ab3fad136cc8354f6e8c27c46585f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 7 Feb 2012 15:52:53 +0100 Subject: [PATCH 228/310] Improved robustness of channel close procedure in presence of execution failures --- .../ByteBufferedChannelManager.java | 19 ++---- .../bytebuffered/ReceiverNotFoundEvent.java | 63 +++++++++++++++++++ .../runtime/ForwardingBarrier.java | 3 + .../runtime/RuntimeInputChannelContext.java | 44 ++++++++++++- 4 files changed, 112 insertions(+), 17 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 505c36e9c954f..7f30f9054572b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -28,8 +28,6 @@ import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.configuration.GlobalConfiguration; -import eu.stratosphere.nephele.event.task.AbstractEvent; -import eu.stratosphere.nephele.event.task.EventList; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; @@ -289,25 +287,16 @@ private void recycleBuffer(final TransferEnvelope envelope) { private void sendReceiverNotFoundEvent(final TransferEnvelope envelope, final ChannelID receiver) { - if (envelope.getBuffer() == null && envelope.getSequenceNumber() == 0) { + if (ReceiverNotFoundEvent.isReceiverNotFoundEvent(envelope)) { - final EventList eventList = envelope.getEventList(); - if (eventList.size() == 1) { - final AbstractEvent event = eventList.get(0); - if (event instanceof ReceiverNotFoundEvent) { - LOG.info("Dropping request to send ReceiverNotFoundEvent as response to ReceiverNotFoundEvent"); - return; - } - } + LOG.info("Dropping request to send ReceiverNotFoundEvent as response to ReceiverNotFoundEvent"); + return; } final JobID jobID = envelope.getJobID(); - final TransferEnvelope transferEnvelope = new TransferEnvelope(0, jobID, receiver); - - final ReceiverNotFoundEvent unknownReceiverEvent = new ReceiverNotFoundEvent(receiver, + final TransferEnvelope transferEnvelope = ReceiverNotFoundEvent.createEnvelopeWithEvent(jobID, receiver, envelope.getSequenceNumber()); - transferEnvelope.addEvent(unknownReceiverEvent); final TransferEnvelopeReceiverList receiverList = getReceiverList(jobID, receiver); if (receiverList == null) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java index 50c0e85ef3afa..40d9a4e39e4f7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java @@ -20,7 +20,10 @@ import java.io.IOException; import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.event.task.EventList; import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; /** * An unknown receiver event can be used by the framework to inform a sender task that the delivery of a @@ -30,6 +33,11 @@ */ public final class ReceiverNotFoundEvent extends AbstractEvent { + /** + * The sequence number that will be set for transfer envelopes which contain the receiver not found event. + */ + private static final int RECEIVER_NOT_FOUND_SEQUENCE_NUMBER = 0; + /** * The ID of the receiver which could not be found */ @@ -110,4 +118,59 @@ public void read(final DataInput in) throws IOException { this.sequenceNumber = in.readInt(); } + /** + * Creates a transfer envelope which only contains a ReceiverNotFoundEvent. + * + * @param jobID + * the ID of the job the event relates to. + * @param receiver + * the channel ID of the receiver that could not be found + * @param sequenceNumber + * the sequence number of the transfer envelope which caused the creation of this event + * @return a transfer envelope which only contains a ReceiverNotFoundEvent + */ + public static TransferEnvelope createEnvelopeWithEvent(final JobID jobID, final ChannelID receiver, + final int sequenceNumber) { + + final TransferEnvelope transferEnvelope = new TransferEnvelope(RECEIVER_NOT_FOUND_SEQUENCE_NUMBER, jobID, + receiver); + + final ReceiverNotFoundEvent unknownReceiverEvent = new ReceiverNotFoundEvent(receiver, sequenceNumber); + transferEnvelope.addEvent(unknownReceiverEvent); + + return transferEnvelope; + } + + /** + * Checks if the given envelope only contains a ReceiverNotFoundEvent. + * + * @param transferEnvelope + * the envelope to be checked + * @return true if the envelope only contains a ReceiverNotFoundEvent, false otherwise + */ + public static boolean isReceiverNotFoundEvent(final TransferEnvelope transferEnvelope) { + + if (transferEnvelope.getSequenceNumber() != RECEIVER_NOT_FOUND_SEQUENCE_NUMBER) { + return false; + } + + if (transferEnvelope.getBuffer() != null) { + return false; + } + + final EventList eventList = transferEnvelope.getEventList(); + if (eventList == null) { + return false; + } + + if (eventList.size() != 1) { + return false; + } + + if (!(eventList.get(0) instanceof ReceiverNotFoundEvent)) { + return false; + } + + return true; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java index febc3fdb5bd9f..879f929064b8f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java @@ -58,6 +58,9 @@ public boolean hasDataLeft() { return false; } + /** + * {@inheritDoc} + */ @Override public void processEvent(final AbstractEvent event) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java index 143e74f3499c0..e8c7ed23d96d6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java @@ -30,9 +30,11 @@ import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; import eu.stratosphere.nephele.io.channels.bytebuffered.BufferPairResponse; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedInputChannelBroker; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.taskmanager.bytebuffered.InputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.ReceiverNotFoundEvent; import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; @@ -170,6 +172,10 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { AbstractEvent eventToSend = null; + if (ReceiverNotFoundEvent.isReceiverNotFoundEvent(transferEnvelope)) { + return; + } + synchronized (this.queuedEnvelopes) { if (this.destroyCalled) { @@ -193,8 +199,13 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { } } else { - // Tell the sender that we are expecting an envelope with a higher sequence number - eventToSend = new UnexpectedEnvelopeEvent(expectedSequenceNumber); + eventToSend = lookForCloseEvent(transferEnvelope); + if (eventToSend == null) { + + // Tell the sender to skip all envelopes until the next envelope that could potentially include + // the close event + eventToSend = new UnexpectedEnvelopeEvent(expectedSequenceNumber - 1); + } } LOG.warn("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber @@ -224,6 +235,35 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { } } + /** + * Looks for a {@link ByteBufferedChannelCloseEvent} in the given envelope returns it if it is found. + * + * @param envelope + * the envelope to be inspected + * @return the found {@link ByteBufferedChannelCloseEvent} or null if no such event was stored inside + * the given envelope + */ + private AbstractEvent lookForCloseEvent(final TransferEnvelope envelope) { + + final EventList eventList = envelope.getEventList(); + if (eventList == null) { + return null; + } + + final Iterator it = eventList.iterator(); + while (it.hasNext()) { + + final AbstractEvent event = it.next(); + + if (event instanceof ByteBufferedChannelCloseEvent) { + LOG.info("Found close event in unexpected envelope"); + return event; + } + } + + return null; + } + @Override public ChannelID getChannelID() { From 2f67b4a4688a1bea53b251b875ee5946a5554e33 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 7 Feb 2012 18:12:38 +0100 Subject: [PATCH 229/310] Added missing implementation in ReplayTaskContext --- .../nephele/checkpointing/ReplayInputGateContext.java | 2 +- .../stratosphere/nephele/checkpointing/ReplayTaskContext.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java index d163b9b6f7172..5960085388bb9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java @@ -26,7 +26,7 @@ public InputChannelContext createInputChannelContext(ChannelID channelID, InputC */ @Override public LocalBufferPoolOwner getLocalBufferPoolOwner() { - // TODO Auto-generated method stub + return null; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java index 899913e71b0d0..d3a776b6cadcd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java @@ -34,8 +34,8 @@ public OutputGateContext createOutputGateContext(final GateID gateID) { */ @Override public InputGateContext createInputGateContext(final GateID gateID) { - // TODO Auto-generated method stub - return null; + + return new ReplayInputGateContext(gateID); } /** From 22e7291955f490a61a2edf53824c5140e7aad1cf Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 10 Feb 2012 13:42:46 +0100 Subject: [PATCH 230/310] Fixed unit tests --- .../SpillingQueueElementTest.java | 7 ++- .../TransferEnvelopeSerializerTest.java | 4 +- .../nephele/util/BufferPoolConnector.java | 58 +++++++++++++++++++ 3 files changed, 66 insertions(+), 3 deletions(-) create mode 100644 nephele/nephele-server/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElementTest.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElementTest.java index 45a2f77075eaf..ea9079640ce66 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElementTest.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElementTest.java @@ -16,6 +16,7 @@ import eu.stratosphere.nephele.io.channels.BufferFactory; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.jobgraph.JobID; +import eu.stratosphere.nephele.util.BufferPoolConnector; public class SpillingQueueElementTest { @@ -33,8 +34,10 @@ public void testSpillingQueueElement() { final JobID jobID = new JobID(); final ChannelID source = new ChannelID(); - final Buffer buf1 = BufferFactory.createFromMemory(BUFFER_SIZE, byteBuf1, queue); - final Buffer buf2 = BufferFactory.createFromMemory(BUFFER_SIZE, byteBuf2, queue); + final BufferPoolConnector connector = new BufferPoolConnector(queue); + + final Buffer buf1 = BufferFactory.createFromMemory(BUFFER_SIZE, byteBuf1, connector); + final Buffer buf2 = BufferFactory.createFromMemory(BUFFER_SIZE, byteBuf2, connector); final TransferEnvelope te1 = new TransferEnvelope(0, jobID, source); te1.setBuffer(buf1); diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeSerializerTest.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeSerializerTest.java index 235a550283058..7c4c8a498a9b9 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeSerializerTest.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeSerializerTest.java @@ -36,6 +36,7 @@ import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.taskmanager.transferenvelope.DefaultSerializer; +import eu.stratosphere.nephele.util.BufferPoolConnector; import eu.stratosphere.nephele.util.ServerTestUtils; /** @@ -147,7 +148,8 @@ private File generateDataStream() throws IOException { for (int i = 0; i < BUFFER_SIZE; i++) { - final Buffer buffer = BufferFactory.createFromMemory(i, recycleQueue.poll(), recycleQueue); + final Buffer buffer = BufferFactory.createFromMemory(i, recycleQueue.poll(), new BufferPoolConnector( + recycleQueue)); // Initialize buffer for (int j = 0; j < i; j++) { diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java new file mode 100644 index 0000000000000..ff66a07525081 --- /dev/null +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java @@ -0,0 +1,58 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.util; + +import java.nio.ByteBuffer; +import java.util.Queue; + +import eu.stratosphere.nephele.io.channels.MemoryBufferPoolConnector; + +/** + * This is a simple implementation of a {@link MemoryBufferPoolConnector} used for the server unit tests. + *

+ * This class is thread-safe. + * + * @author warneke + */ +public final class BufferPoolConnector implements MemoryBufferPoolConnector { + + /** + * Reference to the memory pool the byte buffer was originally taken from. + */ + private final Queue bufferPool; + + /** + * Constructs a new buffer pool connector + * + * @param bufferPool + * a reference to the memory pool the byte buffer was originally taken from + */ + public BufferPoolConnector(final Queue bufferPool) { + this.bufferPool = bufferPool; + } + + /** + * {@inheritDoc} + */ + @Override + public void recycle(final ByteBuffer byteBuffer) { + + synchronized (this.bufferPool) { + this.bufferPool.add(byteBuffer); + this.bufferPool.notify(); + } + } +} From 3680e27791b95e315dc404642337feb7c796961e Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 10 Feb 2012 16:19:23 +0100 Subject: [PATCH 231/310] Improved logging of buffer utilization --- .../taskmanager/runtime/RuntimeInputGateContext.java | 8 ++++++-- .../nephele/taskmanager/runtime/RuntimeTaskContext.java | 6 +++--- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index 85b0edf14de2f..69f944f4c791b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -33,15 +33,18 @@ final class RuntimeInputGateContext implements BufferProvider, InputGateContext, LocalBufferPoolOwner { + private final String taskName; + private final LocalBufferPool localBufferPool; private final TransferEnvelopeDispatcher transferEnvelopeDispatcher; private final InputGate inputGate; - RuntimeInputGateContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + RuntimeInputGateContext(final String taskName, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final InputGate inputGate) { + this.taskName = taskName; this.localBufferPool = new LocalBufferPool(1, false); this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; @@ -128,7 +131,8 @@ public void logBufferUtilization() { final int des = this.localBufferPool.getDesignatedNumberOfBuffers(); System.out - .println("\t\tInputGateContext: " + ava + " available, " + req + " requested, " + des + " designated"); + .println("\t\tInput gate " + this.inputGate.getIndex() + " of " + this.taskName + ": " + ava + + " available, " + req + " requested, " + des + " designated"); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 474bf930bd143..7284b4ecef487 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -155,8 +155,8 @@ public void logBufferUtilization() { final RuntimeEnvironment environment = this.task.getRuntimeEnvironment(); - System.out.println("\t\t" + environment.getTaskName() + ": " + ava + " available, " + req + " requested, " - + des + " designated"); + System.out.println("\t\t" + environment.getTaskNameWithIndex() + ": " + ava + " available, " + req + + " requested, " + des + " designated"); } /** @@ -270,7 +270,7 @@ public InputGateContext createInputGateContext(final GateID gateID) { throw new IllegalStateException("Cannot find input gate with ID " + gateID); } - return new RuntimeInputGateContext(this.transferEnvelopeDispatcher, inputGate); + return new RuntimeInputGateContext(re.getTaskNameWithIndex(), this.transferEnvelopeDispatcher, inputGate); } /** From 83d2dfef47001c4f8dca354675303e9cb5cbe823 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 10 Feb 2012 16:21:11 +0100 Subject: [PATCH 232/310] Improved logging --- .../eu/stratosphere/nephele/execution/RuntimeEnvironment.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 84d8c08d2110d..8b6ea40129e96 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -469,7 +469,7 @@ public Thread getExecutingThread() { if (this.taskName == null) { this.executingThread = new Thread(this); } else { - this.executingThread = new Thread(this, this.taskName); + this.executingThread = new Thread(this, getTaskNameWithIndex()); } } From f7ea8e628d98198c41da655fb13165f96f62aa22 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 11 Feb 2012 22:35:49 +0100 Subject: [PATCH 233/310] Implemented envelope consumption tracker to guarantee replay of envelopes in the correct order --- .../eu/stratosphere/nephele/io/InputGate.java | 11 + .../AbstractByteBufferedInputChannel.java | 9 + .../nephele/io/RuntimeInputGate.java | 12 +- .../runtime/EnvelopeConsumptionLog.java | 385 ++++++++++++++++++ .../runtime/EnvelopeConsumptionTracker.java | 46 +++ .../runtime/RuntimeInputChannelContext.java | 18 +- .../runtime/RuntimeInputGateContext.java | 7 +- .../runtime/RuntimeTaskContext.java | 10 +- 8 files changed, 489 insertions(+), 9 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java index 71c225cd6ead4..73077992765da 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java @@ -71,9 +71,20 @@ public interface InputGate extends Gate { /** * Notify the gate that the channel with the given index has * at least one record available. + * + * @param channelIndex + * the index of the channel which has at least one record available */ void notifyRecordIsAvailable(int channelIndex); + /** + * Notify the gate that is has consumed a data unit from the channel with the given index + * + * @param channelIndex + * the index of the channel from which a data unit has been consumed + */ + void notifyDataUnitConsumed(int channelIndex); + /** * Activates all of the task's input channels. * diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java index f99af0fff0fa7..cea3762e191fc 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java @@ -70,6 +70,7 @@ public abstract class AbstractByteBufferedInputChannel extends private Buffer uncompressedDataBuffer = null; private IOException ioException = null; + /** * Stores the number of bytes read through this input channel since its instantiation. */ @@ -323,4 +324,12 @@ public long getAmountOfDataTransmitted() { return this.amountOfDataTransmitted; } + + /** + * Notify the channel that a data unit has been consumed. + */ + public void notifyDataUnitConsumed() { + + this.getInputGate().notifyDataUnitConsumed(getChannelIndex()); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java index 0c4cf09e02b39..73ca50d1c0465 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java @@ -118,8 +118,7 @@ public class RuntimeInputGate extends AbstractGate implemen * the distribution pattern to determine the concrete wiring between to groups of vertices */ public RuntimeInputGate(final JobID jobID, final GateID gateID, final RecordDeserializer deserializer, - final int index, - final DistributionPattern distributionPattern) { + final int index, final DistributionPattern distributionPattern) { super(jobID, gateID, index); @@ -534,4 +533,13 @@ public boolean hasRecordAvailable() throws IOException, InterruptedException { return true; } + + /** + * {@inheritDoc} + */ + @Override + public void notifyDataUnitConsumed(final int channelIndex) { + + this.channelToReadFrom = -1; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java new file mode 100644 index 0000000000000..4f44c773d5b1a --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java @@ -0,0 +1,385 @@ +package eu.stratosphere.nephele.taskmanager.runtime; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.IntBuffer; +import java.nio.channels.FileChannel; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.configuration.ConfigConstants; +import eu.stratosphere.nephele.configuration.GlobalConfiguration; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.util.StringUtils; + +final class EnvelopeConsumptionLog { + + private static final Log LOG = LogFactory.getLog(EnvelopeConsumptionLog.class); + + private static final int LOG_WINDOW_SIZE = 8192; + + private static final int SIZE_OF_INTEGER = 4; + + private final File logFile; + + private final long numberOfInitialLogEntries; + + private final ByteBuffer outstandingEnvelopesAsByteBuffer; + + private final IntBuffer outstandingEnvelopesAsIntBuffer; + + private final ByteBuffer announcedEnvelopesAsByteBuffer; + + private final IntBuffer announcedEnvelopesAsIntBuffer; + + private final ExecutionVertexID vertexID; + + private final EnvelopeConsumptionTracker tracker; + + private long numberOfAnnouncedEnvelopes = 0L; + + private long numberOfEntriesReadFromLog = 0L; + + EnvelopeConsumptionLog(final ExecutionVertexID vertexID, final EnvelopeConsumptionTracker tracker) { + + this.vertexID = vertexID; + this.tracker = tracker; + + // Check if there is a log file from a previous execution + final String fileName = GlobalConfiguration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH) + File.separator + "cl_" + this.vertexID; + + this.logFile = new File(fileName); + + if (this.logFile.exists()) { + + final long length = this.logFile.length(); + if (length % SIZE_OF_INTEGER != 0) { + LOG.error("Channel consumption log " + fileName + " appears to be corrupt, discarding it..."); + this.logFile.delete(); + this.numberOfInitialLogEntries = 0L; + } else { + this.numberOfInitialLogEntries = length / SIZE_OF_INTEGER; + } + + System.out.println("PREVIOUS FILE FOUND with length " + length); + + } else { + this.numberOfInitialLogEntries = 0L; + } + + this.outstandingEnvelopesAsByteBuffer = ByteBuffer.allocate(LOG_WINDOW_SIZE); + this.outstandingEnvelopesAsIntBuffer = this.outstandingEnvelopesAsByteBuffer.asIntBuffer(); + + this.announcedEnvelopesAsByteBuffer = ByteBuffer.allocate(LOG_WINDOW_SIZE); + this.announcedEnvelopesAsIntBuffer = this.announcedEnvelopesAsByteBuffer.asIntBuffer(); + + this.outstandingEnvelopesAsIntBuffer.limit(0); + + if (this.numberOfInitialLogEntries > 0) { + loadNextOutstandingEnvelopes(); + } + } + + void add(final int gateIndex, final int channelIndex) { + + if (this.outstandingEnvelopesAsIntBuffer.hasRemaining()) { + addOutstandingEnvelope(gateIndex, channelIndex); + } else { + announce(gateIndex, channelIndex); + } + } + + void finish() { + + writeAnnouncedEnvelopesBufferToDisk(); + } + + private void addOutstandingEnvelope(final int gateIndex, final int channelIndex) { + + final int entryToTest = toEntry(gateIndex, channelIndex, false); + + boolean found = false; + + while (true) { + + for (int i = this.outstandingEnvelopesAsIntBuffer.position(); i < this.outstandingEnvelopesAsIntBuffer + .limit(); ++i) { + + if (this.outstandingEnvelopesAsIntBuffer.get(i) == entryToTest) { + // Mark data as available + this.outstandingEnvelopesAsIntBuffer.put(i, setDataAvailability(entryToTest, true)); + found = true; + break; + } + } + + if (!found) { + + System.out.println("Entry not found for channel " + channelIndex + " - " + + this.outstandingEnvelopesAsIntBuffer.limit() + ", " + + this.outstandingEnvelopesAsIntBuffer.capacity()); + + if (this.outstandingEnvelopesAsIntBuffer.limit() == this.outstandingEnvelopesAsIntBuffer.capacity()) { + loadNextOutstandingEnvelopes(); + showOustandingEnvelopeLog(); + continue; + } + + System.out.println("Adding new entry for channel " + channelIndex + " to index " + + this.outstandingEnvelopesAsIntBuffer.limit()); + final int newEntry = setDataAvailability(entryToTest, true); + final int limit = this.outstandingEnvelopesAsIntBuffer.limit(); + this.outstandingEnvelopesAsIntBuffer.limit(limit + 1); + this.outstandingEnvelopesAsIntBuffer.put(limit, newEntry); + + } + + break; + } + + showOustandingEnvelopeLog(); + + int newPosition = this.outstandingEnvelopesAsIntBuffer.position(); + int count = 0; + for (int i = this.outstandingEnvelopesAsIntBuffer.position(); i < this.outstandingEnvelopesAsIntBuffer.limit(); ++i) { + + final int entry = this.outstandingEnvelopesAsIntBuffer.get(i); + if (getDataAvailability(entry)) { + announce(getInputGate(entry), getInputChannel(entry)); + newPosition = i + 1; + ++count; + } else { + break; + } + } + + this.outstandingEnvelopesAsIntBuffer.position(Math.min(this.outstandingEnvelopesAsIntBuffer.limit(), + newPosition)); + + if (count > 0) { + System.out.println("Announced " + count + " buffers from log"); + System.out.println("Initial log entries: " + this.numberOfInitialLogEntries + ", announced " + + this.numberOfAnnouncedEnvelopes); + System.out.println("Outstanding buffer: " + this.outstandingEnvelopesAsIntBuffer.remaining()); + } + + if (!this.outstandingEnvelopesAsIntBuffer.hasRemaining()) { + loadNextOutstandingEnvelopes(); + } + } + + private void showOustandingEnvelopeLog() { + + int dataAvailableCounter = 0; + + final int pos = this.outstandingEnvelopesAsIntBuffer.position(); + final int limit = this.outstandingEnvelopesAsIntBuffer.limit(); + + for (int i = 0; i < this.outstandingEnvelopesAsIntBuffer.capacity(); ++i) { + + if (i < pos) { + System.out.print('_'); + continue; + } + + if (i >= limit) { + System.out.print('_'); + continue; + } + + final int entry = this.outstandingEnvelopesAsIntBuffer.get(i); + + final int channelIndex = getInputChannel(entry); + final boolean dataAvailable = getDataAvailability(entry); + if (dataAvailable) { + ++dataAvailableCounter; + } + + char ch = (char) (((int) 'A') + channelIndex + (dataAvailable ? 0 : 32)); + + System.out.print(ch); + + } + + System.out.println(""); + System.out.println("Data available: " + dataAvailableCounter); + } + + private void loadNextOutstandingEnvelopes() { + + System.out.println("++++++++++++ Entering loadNextOutstandingEnvelopes"); + + final int pos = this.outstandingEnvelopesAsIntBuffer.position(); + + if (pos > 0) { + + final int rem = this.outstandingEnvelopesAsIntBuffer.remaining(); + + for (int i = 0; i < rem; ++i) { + this.outstandingEnvelopesAsIntBuffer.put(i, this.outstandingEnvelopesAsIntBuffer.get(i + pos)); + } + + this.outstandingEnvelopesAsIntBuffer.position(0); + this.outstandingEnvelopesAsIntBuffer.limit(rem); + } + + if (this.numberOfEntriesReadFromLog == this.numberOfInitialLogEntries) { + return; + } + + FileChannel fc = null; + + try { + + this.outstandingEnvelopesAsByteBuffer.position(this.outstandingEnvelopesAsIntBuffer.limit() + * SIZE_OF_INTEGER); + this.outstandingEnvelopesAsByteBuffer.limit(this.outstandingEnvelopesAsByteBuffer.capacity()); + + fc = new FileInputStream(this.logFile).getChannel(); + fc.position(this.numberOfEntriesReadFromLog * SIZE_OF_INTEGER); + + int totalBytesRead = 0; + + while (this.outstandingEnvelopesAsByteBuffer.hasRemaining()) { + + final int bytesRead = fc.read(this.outstandingEnvelopesAsByteBuffer); + if (bytesRead == -1) { + break; + } + + totalBytesRead += bytesRead; + } + + System.out.println("Total bytes read: " + totalBytesRead); + + if (totalBytesRead % SIZE_OF_INTEGER != 0) { + LOG.error("Read " + totalBytesRead + " from " + this.logFile.getAbsolutePath() + + ", file may be corrupt"); + } + + final int numberOfNewEntries = totalBytesRead / SIZE_OF_INTEGER; + + this.outstandingEnvelopesAsIntBuffer.limit(this.outstandingEnvelopesAsIntBuffer.limit() + + numberOfNewEntries); + + this.numberOfEntriesReadFromLog += numberOfNewEntries; + + fc.close(); + + } catch (IOException ioe) { + LOG.error(StringUtils.stringifyException(ioe)); + } finally { + + if (fc != null) { + try { + fc.close(); + } catch (IOException ioe) { + } + } + } + } + + private void writeAnnouncedEnvelopesBufferToDisk() { + + FileChannel fc = null; + + try { + + this.announcedEnvelopesAsIntBuffer.flip(); + this.announcedEnvelopesAsByteBuffer.position(this.announcedEnvelopesAsIntBuffer.position() + * SIZE_OF_INTEGER); + this.announcedEnvelopesAsByteBuffer.limit(this.announcedEnvelopesAsIntBuffer.limit() * SIZE_OF_INTEGER); + + System.out.println("WRITING TO DISK " + this.announcedEnvelopesAsByteBuffer); + + fc = new FileOutputStream(this.logFile, true).getChannel(); + + while (this.announcedEnvelopesAsByteBuffer.hasRemaining()) { + fc.write(this.announcedEnvelopesAsByteBuffer); + } + + } catch (IOException ioe) { + LOG.error(StringUtils.stringifyException(ioe)); + } finally { + + if (fc != null) { + try { + fc.close(); + } catch (IOException ioe) { + } + } + + this.announcedEnvelopesAsIntBuffer.clear(); + this.announcedEnvelopesAsByteBuffer.clear(); + } + + } + + private void announce(final int gateIndex, final int channelIndex) { + + this.tracker.announceData(gateIndex, channelIndex); + + if (++this.numberOfAnnouncedEnvelopes < this.numberOfInitialLogEntries) { + return; + } + + this.announcedEnvelopesAsIntBuffer.put(toEntry(gateIndex, channelIndex, false)); + + if (!this.announcedEnvelopesAsIntBuffer.hasRemaining()) { + writeAnnouncedEnvelopesBufferToDisk(); + } + } + + private static int toEntry(final int gateIndex, final int channelIndex, final boolean dataAvailable) { + + int entry = 0; + entry = setInputGate(entry, gateIndex); + entry = setInputChannel(entry, channelIndex); + entry = setDataAvailability(entry, dataAvailable); + + return entry; + } + + private static int setInputGate(final int entry, final int gateIndex) { + + if ((gateIndex >>> 7) != 0) { + throw new IllegalArgumentException("Gate index " + gateIndex + " cannot be stored in 7 bits"); + } + + return ((entry & 0xffffff01) | ((gateIndex & 0x7f) << 1)); + } + + private static int getInputGate(final int entry) { + + return ((entry >>> 1) & 0x7f); + } + + private static int setInputChannel(final int entry, final int channelIndex) { + + if ((channelIndex >>> 24) != 0) { + throw new IllegalArgumentException("Channel index " + channelIndex + " cannot be stored in 24 bits"); + } + + return ((entry & 0xff) | ((channelIndex & 0xffffff) << 8)); + } + + private static int getInputChannel(final int entry) { + + return ((entry >>> 8) & 0xffffff); + } + + private static int setDataAvailability(final int entry, final boolean dataAvailable) { + + return ((entry & 0xfffffffe) | ((dataAvailable ? 1 : 0) & 0x01)); + } + + private static boolean getDataAvailability(final int entry) { + + return ((entry & 0x01) > 0); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java new file mode 100644 index 0000000000000..344fd8ea6920a --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java @@ -0,0 +1,46 @@ +package eu.stratosphere.nephele.taskmanager.runtime; + +import eu.stratosphere.nephele.execution.RuntimeEnvironment; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.channels.AbstractInputChannel; +import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; +import eu.stratosphere.nephele.types.Record; + +final class EnvelopeConsumptionTracker { + + private final EnvelopeConsumptionLog log; + + private final RuntimeEnvironment environment; + + EnvelopeConsumptionTracker(final ExecutionVertexID vertexID, final RuntimeEnvironment environment) { + + this.log = new EnvelopeConsumptionLog(vertexID, this); + this.environment = environment; + } + + public synchronized void reportEnvelopeAvailability( + final AbstractByteBufferedInputChannel inputChannel) { + + this.log.add(inputChannel.getInputGate().getIndex(), inputChannel.getChannelIndex()); + } + + public synchronized void finishLog() { + this.log.finish(); + } + + public void reportEnvelopeConsumed( + final AbstractByteBufferedInputChannel inputChannel) { + + inputChannel.notifyDataUnitConsumed(); + } + + void announceData(final int gateIndex, final int channelIndex) { + + final InputGate inputGate = this.environment.getInputGate(gateIndex); + + final AbstractInputChannel inputChannel = inputGate.getInputChannel(channelIndex); + + ((AbstractByteBufferedInputChannel) inputChannel).checkForNetworkEvents(); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java index e8c7ed23d96d6..4b7a33583f0ed 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java @@ -52,18 +52,22 @@ final class RuntimeInputChannelContext implements InputChannelContext, ByteBuffe private final Queue queuedEnvelopes = new ArrayDeque(); + private final EnvelopeConsumptionTracker envelopeConsumptionTracker; + private int lastReceivedEnvelope = -1; private boolean destroyCalled = false; RuntimeInputChannelContext(final RuntimeInputGateContext inputGateContext, - final TransferEnvelopeDispatcher transferEnvelopeDispatcher, - final AbstractByteBufferedInputChannel byteBufferedInputChannel) { + final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + final AbstractByteBufferedInputChannel byteBufferedInputChannel, + final EnvelopeConsumptionTracker envelopeConsumptionTracker) { this.inputGateContext = inputGateContext; this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.byteBufferedInputChannel = byteBufferedInputChannel; this.byteBufferedInputChannel.setInputChannelBroker(this); + this.envelopeConsumptionTracker = envelopeConsumptionTracker; } @Override @@ -99,6 +103,9 @@ public BufferPairResponse getReadBufferToConsume() { } } + // Notify the channel that an envelope has been consumed + this.envelopeConsumptionTracker.reportEnvelopeConsumed(this.byteBufferedInputChannel); + return null; } @@ -135,6 +142,9 @@ public void releaseConsumedReadBuffer() { } } + // Notify the channel that an envelope has been consumed + this.envelopeConsumptionTracker.reportEnvelopeConsumed(this.byteBufferedInputChannel); + final Buffer consumedBuffer = transferEnvelope.getBuffer(); if (consumedBuffer == null) { LOG.error("Inconsistency: consumed read buffer is null!"); @@ -195,7 +205,7 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { // This is a problem, now we are actually missing some data this.byteBufferedInputChannel.reportIOException(new IOException("Expected data packet " + expectedSequenceNumber + " but received " + sequenceNumber)); - this.byteBufferedInputChannel.checkForNetworkEvents(); + this.envelopeConsumptionTracker.reportEnvelopeAvailability(this.byteBufferedInputChannel); } } else { @@ -224,7 +234,7 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { } // Notify the channel about the new data - this.byteBufferedInputChannel.checkForNetworkEvents(); + this.envelopeConsumptionTracker.reportEnvelopeAvailability(this.byteBufferedInputChannel); if (eventToSend != null) { try { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index 69f944f4c791b..75e41132a99ff 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -41,14 +41,17 @@ final class RuntimeInputGateContext implements BufferProvider, InputGateContext, private final InputGate inputGate; + private final EnvelopeConsumptionTracker envelopeConsumptionTracker; + RuntimeInputGateContext(final String taskName, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, - final InputGate inputGate) { + final InputGate inputGate, final EnvelopeConsumptionTracker envelopeConsumptionTracker) { this.taskName = taskName; this.localBufferPool = new LocalBufferPool(1, false); this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.inputGate = inputGate; + this.envelopeConsumptionTracker = envelopeConsumptionTracker; } /** @@ -170,7 +173,7 @@ public InputChannelContext createInputChannelContext(final ChannelID channelID, } return new RuntimeInputChannelContext(this, this.transferEnvelopeDispatcher, - (AbstractByteBufferedInputChannel) channel); + (AbstractByteBufferedInputChannel) channel, this.envelopeConsumptionTracker); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 7284b4ecef487..fe3747a6b9abd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -52,6 +52,8 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve private final EphemeralCheckpoint ephemeralCheckpoint; + private final EnvelopeConsumptionTracker envelopeConsumptionTracker; + /** * Stores whether the initial exhaustion of memory buffers has already been reported */ @@ -84,6 +86,8 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.runtimeDispatcher = new RuntimeDispatcher(transferEnvelopeDispatcher); + + this.envelopeConsumptionTracker = new EnvelopeConsumptionTracker(task.getVertexID(), environment); } RuntimeDispatcher getRuntimeDispatcher() { @@ -132,6 +136,9 @@ public void clearLocalBufferPool() { // Clear the buffer cache this.localBufferPool.destroy(); + + // Finish the envelope consumption log + this.envelopeConsumptionTracker.finishLog(); } /** @@ -270,7 +277,8 @@ public InputGateContext createInputGateContext(final GateID gateID) { throw new IllegalStateException("Cannot find input gate with ID " + gateID); } - return new RuntimeInputGateContext(re.getTaskNameWithIndex(), this.transferEnvelopeDispatcher, inputGate); + return new RuntimeInputGateContext(re.getTaskNameWithIndex(), this.transferEnvelopeDispatcher, inputGate, + this.envelopeConsumptionTracker); } /** From a5267d517d958a40ef298fe032a9a7d76915ddae Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 11 Feb 2012 22:52:42 +0100 Subject: [PATCH 234/310] Reduced verbosity of EnvelopeConsumptionLog --- .../runtime/EnvelopeConsumptionLog.java | 20 +++---------------- 1 file changed, 3 insertions(+), 17 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java index 4f44c773d5b1a..7693d6c852a84 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java @@ -66,7 +66,8 @@ final class EnvelopeConsumptionLog { this.numberOfInitialLogEntries = length / SIZE_OF_INTEGER; } - System.out.println("PREVIOUS FILE FOUND with length " + length); + LOG.info("Found existing consumption log for task " + this.vertexID + " with a size of " + length + + " bytes"); } else { this.numberOfInitialLogEntries = 0L; @@ -120,30 +121,20 @@ private void addOutstandingEnvelope(final int gateIndex, final int channelIndex) if (!found) { - System.out.println("Entry not found for channel " + channelIndex + " - " - + this.outstandingEnvelopesAsIntBuffer.limit() + ", " - + this.outstandingEnvelopesAsIntBuffer.capacity()); - if (this.outstandingEnvelopesAsIntBuffer.limit() == this.outstandingEnvelopesAsIntBuffer.capacity()) { loadNextOutstandingEnvelopes(); - showOustandingEnvelopeLog(); continue; } - System.out.println("Adding new entry for channel " + channelIndex + " to index " - + this.outstandingEnvelopesAsIntBuffer.limit()); final int newEntry = setDataAvailability(entryToTest, true); final int limit = this.outstandingEnvelopesAsIntBuffer.limit(); this.outstandingEnvelopesAsIntBuffer.limit(limit + 1); this.outstandingEnvelopesAsIntBuffer.put(limit, newEntry); - } break; } - showOustandingEnvelopeLog(); - int newPosition = this.outstandingEnvelopesAsIntBuffer.position(); int count = 0; for (int i = this.outstandingEnvelopesAsIntBuffer.position(); i < this.outstandingEnvelopesAsIntBuffer.limit(); ++i) { @@ -166,6 +157,7 @@ private void addOutstandingEnvelope(final int gateIndex, final int channelIndex) System.out.println("Initial log entries: " + this.numberOfInitialLogEntries + ", announced " + this.numberOfAnnouncedEnvelopes); System.out.println("Outstanding buffer: " + this.outstandingEnvelopesAsIntBuffer.remaining()); + showOustandingEnvelopeLog(); } if (!this.outstandingEnvelopesAsIntBuffer.hasRemaining()) { @@ -212,8 +204,6 @@ private void showOustandingEnvelopeLog() { private void loadNextOutstandingEnvelopes() { - System.out.println("++++++++++++ Entering loadNextOutstandingEnvelopes"); - final int pos = this.outstandingEnvelopesAsIntBuffer.position(); if (pos > 0) { @@ -255,8 +245,6 @@ private void loadNextOutstandingEnvelopes() { totalBytesRead += bytesRead; } - System.out.println("Total bytes read: " + totalBytesRead); - if (totalBytesRead % SIZE_OF_INTEGER != 0) { LOG.error("Read " + totalBytesRead + " from " + this.logFile.getAbsolutePath() + ", file may be corrupt"); @@ -295,8 +283,6 @@ private void writeAnnouncedEnvelopesBufferToDisk() { * SIZE_OF_INTEGER); this.announcedEnvelopesAsByteBuffer.limit(this.announcedEnvelopesAsIntBuffer.limit() * SIZE_OF_INTEGER); - System.out.println("WRITING TO DISK " + this.announcedEnvelopesAsByteBuffer); - fc = new FileOutputStream(this.logFile, true).getChannel(); while (this.announcedEnvelopesAsByteBuffer.hasRemaining()) { From 07aafd17e6fd7a46d08b992b52e1ae23f812bcbf Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 12 Feb 2012 23:35:06 +0100 Subject: [PATCH 235/310] Fixed some bugs in the envelope consumption log --- .../bytebuffered/OutgoingConnection.java | 8 +++--- .../runtime/EnvelopeConsumptionLog.java | 26 ++++++++++++------- .../runtime/EnvelopeConsumptionTracker.java | 5 ++++ .../runtime/RuntimeInputChannelContext.java | 9 +++---- .../runtime/RuntimeInputGateContext.java | 16 ++++++++++++ 5 files changed, 46 insertions(+), 18 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java index 13e6470e4efa7..68ed49097efe3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java @@ -20,7 +20,9 @@ import java.nio.channels.SelectionKey; import java.nio.channels.SocketChannel; import java.nio.channels.WritableByteChannel; +import java.util.ArrayDeque; import java.util.Iterator; +import java.util.Queue; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -63,7 +65,7 @@ public class OutgoingConnection { /** * The queue of transfer envelopes to be transmitted. */ - private final TransferEnvelopeQueue queuedEnvelopes = new TransferEnvelopeQueue(); + private final Queue queuedEnvelopes = new ArrayDeque(); /** * The {@link DefaultSerializer} object used to transform the envelopes into a byte stream. @@ -554,9 +556,9 @@ public int getNumberOfQueuedWriteBuffers() { */ void registerSpillingQueue(final SpillingQueue spillingQueue) { - synchronized (this.queuedEnvelopes) { + /*synchronized (this.queuedEnvelopes) { checkConnection(); this.queuedEnvelopes.registerSpillingQueue(spillingQueue); - } + }*/ } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java index 7693d6c852a84..3bd2944942c9f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java @@ -20,7 +20,7 @@ final class EnvelopeConsumptionLog { private static final Log LOG = LogFactory.getLog(EnvelopeConsumptionLog.class); - private static final int LOG_WINDOW_SIZE = 8192; + private static final int LOG_WINDOW_SIZE = 65536; private static final int SIZE_OF_INTEGER = 4; @@ -100,6 +100,11 @@ void finish() { writeAnnouncedEnvelopesBufferToDisk(); } + boolean followsLog() { + + return (this.numberOfInitialLogEntries > 0L); + } + private void addOutstandingEnvelope(final int gateIndex, final int channelIndex) { final int entryToTest = toEntry(gateIndex, channelIndex, false); @@ -152,11 +157,11 @@ private void addOutstandingEnvelope(final int gateIndex, final int channelIndex) this.outstandingEnvelopesAsIntBuffer.position(Math.min(this.outstandingEnvelopesAsIntBuffer.limit(), newPosition)); - if (count > 0) { - System.out.println("Announced " + count + " buffers from log"); - System.out.println("Initial log entries: " + this.numberOfInitialLogEntries + ", announced " + if (count > 0 && LOG.isDebugEnabled()) { + LOG.debug("Announced " + count + " buffers from log"); + LOG.debug("Initial log entries: " + this.numberOfInitialLogEntries + ", announced " + this.numberOfAnnouncedEnvelopes); - System.out.println("Outstanding buffer: " + this.outstandingEnvelopesAsIntBuffer.remaining()); + LOG.debug("Outstanding buffer: " + this.outstandingEnvelopesAsIntBuffer.remaining()); showOustandingEnvelopeLog(); } @@ -172,15 +177,17 @@ private void showOustandingEnvelopeLog() { final int pos = this.outstandingEnvelopesAsIntBuffer.position(); final int limit = this.outstandingEnvelopesAsIntBuffer.limit(); + final StringBuilder sb = new StringBuilder(); + for (int i = 0; i < this.outstandingEnvelopesAsIntBuffer.capacity(); ++i) { if (i < pos) { - System.out.print('_'); + sb.append('_'); continue; } if (i >= limit) { - System.out.print('_'); + sb.append('_'); continue; } @@ -194,12 +201,11 @@ private void showOustandingEnvelopeLog() { char ch = (char) (((int) 'A') + channelIndex + (dataAvailable ? 0 : 32)); - System.out.print(ch); + sb.append(ch); } - System.out.println(""); - System.out.println("Data available: " + dataAvailableCounter); + LOG.debug(sb.toString()); } private void loadNextOutstandingEnvelopes() { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java index 344fd8ea6920a..4030fc1d9a844 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java @@ -24,6 +24,11 @@ public synchronized void reportEnvelopeAvailability( this.log.add(inputChannel.getInputGate().getIndex(), inputChannel.getChannelIndex()); } + + public boolean followsLog() { + + return this.log.followsLog(); + } public synchronized void finishLog() { this.log.finish(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java index 4b7a33583f0ed..d14c2b28c9dcd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java @@ -205,7 +205,7 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { // This is a problem, now we are actually missing some data this.byteBufferedInputChannel.reportIOException(new IOException("Expected data packet " + expectedSequenceNumber + " but received " + sequenceNumber)); - this.envelopeConsumptionTracker.reportEnvelopeAvailability(this.byteBufferedInputChannel); + this.byteBufferedInputChannel.checkForNetworkEvents(); } } else { @@ -228,14 +228,13 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { } else { this.queuedEnvelopes.add(transferEnvelope); - this.lastReceivedEnvelope = sequenceNumber; + + // Notify the channel about the new data + this.envelopeConsumptionTracker.reportEnvelopeAvailability(this.byteBufferedInputChannel); } } - // Notify the channel about the new data - this.envelopeConsumptionTracker.reportEnvelopeAvailability(this.byteBufferedInputChannel); - if (eventToSend != null) { try { transferEventToOutputChannel(eventToSend); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index 75e41132a99ff..b9054df315258 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -21,7 +21,10 @@ import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.channels.AbstractInputChannel; import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.BufferFactory; import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.io.channels.FileBufferManager; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; @@ -43,6 +46,8 @@ final class RuntimeInputGateContext implements BufferProvider, InputGateContext, private final EnvelopeConsumptionTracker envelopeConsumptionTracker; + private final FileBufferManager fileBufferManager; + RuntimeInputGateContext(final String taskName, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final InputGate inputGate, final EnvelopeConsumptionTracker envelopeConsumptionTracker) { @@ -52,6 +57,8 @@ final class RuntimeInputGateContext implements BufferProvider, InputGateContext, this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.inputGate = inputGate; this.envelopeConsumptionTracker = envelopeConsumptionTracker; + + this.fileBufferManager = FileBufferManager.getInstance(); } /** @@ -69,6 +76,15 @@ public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOExcepti @Override public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException { + final Buffer buffer = this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer); + if (buffer != null) { + return buffer; + } + + if (this.envelopeConsumptionTracker.followsLog()) { + return BufferFactory.createFromFile(minimumSizeOfBuffer, this.inputGate.getGateID(), fileBufferManager); + } + return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer); } From 80098e11cc55753e4721102e520887ffd22d5e22 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 12 Feb 2012 23:35:28 +0100 Subject: [PATCH 236/310] Quick workaround for buffer deadlock problem --- .../nephele/taskmanager/bufferprovider/LocalBufferPool.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java index 2f6d2bf62a1f4..02cbe914ca3cb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java @@ -156,7 +156,7 @@ private Buffer requestBufferInternal(final int minimumSizeOfBuffer, final boolea } if (block) { - this.buffers.wait(); + this.buffers.wait(100); } else { return null; } From f0c64bfd789c321871e34d825c13802bca1eb1d4 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 13 Feb 2012 15:18:34 +0100 Subject: [PATCH 237/310] Clean up of code --- .../runtime/EnvelopeConsumptionLog.java | 61 ++++++++++++++----- .../runtime/EnvelopeConsumptionTracker.java | 51 ---------------- .../runtime/RuntimeInputChannelContext.java | 12 ++-- .../runtime/RuntimeInputGateContext.java | 11 ++-- .../runtime/RuntimeTaskContext.java | 8 +-- 5 files changed, 60 insertions(+), 83 deletions(-) delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java index 3bd2944942c9f..f1d28fedd2921 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java @@ -13,7 +13,11 @@ import eu.stratosphere.nephele.configuration.ConfigConstants; import eu.stratosphere.nephele.configuration.GlobalConfiguration; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.InputGate; +import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; +import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.util.StringUtils; final class EnvelopeConsumptionLog { @@ -38,16 +42,16 @@ final class EnvelopeConsumptionLog { private final ExecutionVertexID vertexID; - private final EnvelopeConsumptionTracker tracker; + private final RuntimeEnvironment environment; private long numberOfAnnouncedEnvelopes = 0L; private long numberOfEntriesReadFromLog = 0L; - EnvelopeConsumptionLog(final ExecutionVertexID vertexID, final EnvelopeConsumptionTracker tracker) { + EnvelopeConsumptionLog(final ExecutionVertexID vertexID, final RuntimeEnvironment environment) { this.vertexID = vertexID; - this.tracker = tracker; + this.environment = environment; // Check if there is a log file from a previous execution final String fileName = GlobalConfiguration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, @@ -86,28 +90,44 @@ final class EnvelopeConsumptionLog { } } - void add(final int gateIndex, final int channelIndex) { + void reportEnvelopeAvailability(final AbstractByteBufferedInputChannel inputChannel) { - if (this.outstandingEnvelopesAsIntBuffer.hasRemaining()) { - addOutstandingEnvelope(gateIndex, channelIndex); - } else { - announce(gateIndex, channelIndex); + synchronized (this) { + + if (this.outstandingEnvelopesAsIntBuffer.hasRemaining()) { + addOutstandingEnvelope(inputChannel); + } else { + announce(inputChannel); + } } } void finish() { - writeAnnouncedEnvelopesBufferToDisk(); + synchronized (this) { + writeAnnouncedEnvelopesBufferToDisk(); + } } boolean followsLog() { - return (this.numberOfInitialLogEntries > 0L); + if (this.numberOfInitialLogEntries == 0) { + return false; + } + + synchronized (this) { + return this.announcedEnvelopesAsIntBuffer.hasRemaining(); + } + } + + void reportEnvelopeConsumed(final AbstractByteBufferedInputChannel inputChannel) { + + inputChannel.notifyDataUnitConsumed(); } - private void addOutstandingEnvelope(final int gateIndex, final int channelIndex) { + private void addOutstandingEnvelope(final AbstractByteBufferedInputChannel inputChannel) { - final int entryToTest = toEntry(gateIndex, channelIndex, false); + final int entryToTest = toEntry(inputChannel.getInputGate().getIndex(), inputChannel.getChannelIndex(), false); boolean found = false; @@ -146,7 +166,7 @@ private void addOutstandingEnvelope(final int gateIndex, final int channelIndex) final int entry = this.outstandingEnvelopesAsIntBuffer.get(i); if (getDataAvailability(entry)) { - announce(getInputGate(entry), getInputChannel(entry)); + announce(toInputChannel(getInputGate(entry), getInputChannel(entry))); newPosition = i + 1; ++count; } else { @@ -312,15 +332,24 @@ private void writeAnnouncedEnvelopesBufferToDisk() { } - private void announce(final int gateIndex, final int channelIndex) { + private AbstractByteBufferedInputChannel toInputChannel(final int gateIndex, + final int channelIndex) { + + final InputGate inputGate = this.environment.getInputGate(gateIndex); + + return (AbstractByteBufferedInputChannel) inputGate.getInputChannel(channelIndex); + } - this.tracker.announceData(gateIndex, channelIndex); + private void announce(final AbstractByteBufferedInputChannel inputChannel) { + inputChannel.checkForNetworkEvents(); + if (++this.numberOfAnnouncedEnvelopes < this.numberOfInitialLogEntries) { return; } - this.announcedEnvelopesAsIntBuffer.put(toEntry(gateIndex, channelIndex, false)); + this.announcedEnvelopesAsIntBuffer.put(toEntry(inputChannel.getInputGate().getIndex(), + inputChannel.getChannelIndex(), false)); if (!this.announcedEnvelopesAsIntBuffer.hasRemaining()) { writeAnnouncedEnvelopesBufferToDisk(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java deleted file mode 100644 index 4030fc1d9a844..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionTracker.java +++ /dev/null @@ -1,51 +0,0 @@ -package eu.stratosphere.nephele.taskmanager.runtime; - -import eu.stratosphere.nephele.execution.RuntimeEnvironment; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.InputGate; -import eu.stratosphere.nephele.io.channels.AbstractInputChannel; -import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; -import eu.stratosphere.nephele.types.Record; - -final class EnvelopeConsumptionTracker { - - private final EnvelopeConsumptionLog log; - - private final RuntimeEnvironment environment; - - EnvelopeConsumptionTracker(final ExecutionVertexID vertexID, final RuntimeEnvironment environment) { - - this.log = new EnvelopeConsumptionLog(vertexID, this); - this.environment = environment; - } - - public synchronized void reportEnvelopeAvailability( - final AbstractByteBufferedInputChannel inputChannel) { - - this.log.add(inputChannel.getInputGate().getIndex(), inputChannel.getChannelIndex()); - } - - public boolean followsLog() { - - return this.log.followsLog(); - } - - public synchronized void finishLog() { - this.log.finish(); - } - - public void reportEnvelopeConsumed( - final AbstractByteBufferedInputChannel inputChannel) { - - inputChannel.notifyDataUnitConsumed(); - } - - void announceData(final int gateIndex, final int channelIndex) { - - final InputGate inputGate = this.environment.getInputGate(gateIndex); - - final AbstractInputChannel inputChannel = inputGate.getInputChannel(channelIndex); - - ((AbstractByteBufferedInputChannel) inputChannel).checkForNetworkEvents(); - } -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java index d14c2b28c9dcd..d55118e9a6a05 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java @@ -52,7 +52,7 @@ final class RuntimeInputChannelContext implements InputChannelContext, ByteBuffe private final Queue queuedEnvelopes = new ArrayDeque(); - private final EnvelopeConsumptionTracker envelopeConsumptionTracker; + private final EnvelopeConsumptionLog envelopeConsumptionLog; private int lastReceivedEnvelope = -1; @@ -61,13 +61,13 @@ final class RuntimeInputChannelContext implements InputChannelContext, ByteBuffe RuntimeInputChannelContext(final RuntimeInputGateContext inputGateContext, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final AbstractByteBufferedInputChannel byteBufferedInputChannel, - final EnvelopeConsumptionTracker envelopeConsumptionTracker) { + final EnvelopeConsumptionLog envelopeConsumptionLog) { this.inputGateContext = inputGateContext; this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.byteBufferedInputChannel = byteBufferedInputChannel; this.byteBufferedInputChannel.setInputChannelBroker(this); - this.envelopeConsumptionTracker = envelopeConsumptionTracker; + this.envelopeConsumptionLog = envelopeConsumptionLog; } @Override @@ -104,7 +104,7 @@ public BufferPairResponse getReadBufferToConsume() { } // Notify the channel that an envelope has been consumed - this.envelopeConsumptionTracker.reportEnvelopeConsumed(this.byteBufferedInputChannel); + this.envelopeConsumptionLog.reportEnvelopeConsumed(this.byteBufferedInputChannel); return null; } @@ -143,7 +143,7 @@ public void releaseConsumedReadBuffer() { } // Notify the channel that an envelope has been consumed - this.envelopeConsumptionTracker.reportEnvelopeConsumed(this.byteBufferedInputChannel); + this.envelopeConsumptionLog.reportEnvelopeConsumed(this.byteBufferedInputChannel); final Buffer consumedBuffer = transferEnvelope.getBuffer(); if (consumedBuffer == null) { @@ -231,7 +231,7 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { this.lastReceivedEnvelope = sequenceNumber; // Notify the channel about the new data - this.envelopeConsumptionTracker.reportEnvelopeAvailability(this.byteBufferedInputChannel); + this.envelopeConsumptionLog.reportEnvelopeAvailability(this.byteBufferedInputChannel); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index b9054df315258..4e4895f550ba1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -23,7 +23,6 @@ import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.BufferFactory; import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.io.channels.FileBufferManager; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; @@ -44,19 +43,19 @@ final class RuntimeInputGateContext implements BufferProvider, InputGateContext, private final InputGate inputGate; - private final EnvelopeConsumptionTracker envelopeConsumptionTracker; + private final EnvelopeConsumptionLog envelopeConsumptionLog; private final FileBufferManager fileBufferManager; RuntimeInputGateContext(final String taskName, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, - final InputGate inputGate, final EnvelopeConsumptionTracker envelopeConsumptionTracker) { + final InputGate inputGate, final EnvelopeConsumptionLog envelopeConsumptionLog) { this.taskName = taskName; this.localBufferPool = new LocalBufferPool(1, false); this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.inputGate = inputGate; - this.envelopeConsumptionTracker = envelopeConsumptionTracker; + this.envelopeConsumptionLog = envelopeConsumptionLog; this.fileBufferManager = FileBufferManager.getInstance(); } @@ -81,7 +80,7 @@ public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws I return buffer; } - if (this.envelopeConsumptionTracker.followsLog()) { + if (this.envelopeConsumptionLog.followsLog()) { return BufferFactory.createFromFile(minimumSizeOfBuffer, this.inputGate.getGateID(), fileBufferManager); } @@ -189,7 +188,7 @@ public InputChannelContext createInputChannelContext(final ChannelID channelID, } return new RuntimeInputChannelContext(this, this.transferEnvelopeDispatcher, - (AbstractByteBufferedInputChannel) channel, this.envelopeConsumptionTracker); + (AbstractByteBufferedInputChannel) channel, this.envelopeConsumptionLog); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index fe3747a6b9abd..3be1585a09fef 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -52,7 +52,7 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve private final EphemeralCheckpoint ephemeralCheckpoint; - private final EnvelopeConsumptionTracker envelopeConsumptionTracker; + private final EnvelopeConsumptionLog envelopeConsumptionLog; /** * Stores whether the initial exhaustion of memory buffers has already been reported @@ -87,7 +87,7 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.runtimeDispatcher = new RuntimeDispatcher(transferEnvelopeDispatcher); - this.envelopeConsumptionTracker = new EnvelopeConsumptionTracker(task.getVertexID(), environment); + this.envelopeConsumptionLog = new EnvelopeConsumptionLog(task.getVertexID(), environment); } RuntimeDispatcher getRuntimeDispatcher() { @@ -138,7 +138,7 @@ public void clearLocalBufferPool() { this.localBufferPool.destroy(); // Finish the envelope consumption log - this.envelopeConsumptionTracker.finishLog(); + this.envelopeConsumptionLog.finish(); } /** @@ -278,7 +278,7 @@ public InputGateContext createInputGateContext(final GateID gateID) { } return new RuntimeInputGateContext(re.getTaskNameWithIndex(), this.transferEnvelopeDispatcher, inputGate, - this.envelopeConsumptionTracker); + this.envelopeConsumptionLog); } /** From 858666deddb0eee9bf9d1c675b66b656a65cdac7 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 14 Feb 2012 18:26:22 +0100 Subject: [PATCH 238/310] Fixed problem with unnecessary buffer copy attempt during recovery --- .../ByteBufferedChannelManager.java | 42 +++++++++++++------ 1 file changed, 29 insertions(+), 13 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index a58e728c02019..985d68ea91b2a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -380,21 +380,37 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, final InputChannelContext inputChannelContext = (InputChannelContext) cc; - Buffer destBuffer = null; - try { - destBuffer = inputChannelContext.requestEmptyBufferBlocking(srcBuffer.size()); - srcBuffer.copyToBuffer(destBuffer); - } catch (Exception e) { - LOG.error(StringUtils.stringifyException(e)); - if (destBuffer != null) { - destBuffer.recycleBuffer(); + if (srcBuffer.isBackedByMemory()) { + + Buffer destBuffer = null; + try { + destBuffer = inputChannelContext.requestEmptyBufferBlocking(srcBuffer.size()); + srcBuffer.copyToBuffer(destBuffer); + } catch (Exception e) { + LOG.error(StringUtils.stringifyException(e)); + if (destBuffer != null) { + destBuffer.recycleBuffer(); + } + continue; } - continue; + // TODO: See if we can save one duplicate step here + final TransferEnvelope dup = transferEnvelope.duplicateWithoutBuffer(); + dup.setBuffer(destBuffer); + inputChannelContext.queueTransferEnvelope(dup); + + } else { + + // This is a file buffer, we can simply duplicate the envelope + TransferEnvelope dup = null; + try { + dup = transferEnvelope.duplicate(); + } catch (Exception e) { + LOG.error(StringUtils.stringifyException(e)); + continue; + } + + inputChannelContext.queueTransferEnvelope(dup); } - // TODO: See if we can save one duplicate step here - final TransferEnvelope dup = transferEnvelope.duplicateWithoutBuffer(); - dup.setBuffer(destBuffer); - inputChannelContext.queueTransferEnvelope(dup); } } From 2d8ae9afea66a45bf7f3dad61c06ee95d3ef9a46 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 15 Feb 2012 18:59:47 +0100 Subject: [PATCH 239/310] Fixed bug in class ReplayOutputChannelContext --- .../nephele/checkpointing/ReplayOutputChannelContext.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java index da3b149077ca6..6634dc61dfbd1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java @@ -31,7 +31,7 @@ public final class ReplayOutputChannelContext extends AbstractOutputChannelConte @Override public boolean isInputChannel() { - return true; + return false; } /** From 28551bf1db31af90bcf9d8df936da57e7935d2ea Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 16 Feb 2012 14:07:33 +0100 Subject: [PATCH 240/310] Fixed problem with ClosedByInterrupedException during finish --- .../runtime/EnvelopeConsumptionLog.java | 39 +++++++++++++++++-- 1 file changed, 36 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java index f1d28fedd2921..06b0a6128904f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java @@ -24,7 +24,7 @@ final class EnvelopeConsumptionLog { private static final Log LOG = LogFactory.getLog(EnvelopeConsumptionLog.class); - private static final int LOG_WINDOW_SIZE = 65536; + private static final int LOG_WINDOW_SIZE = 262144; private static final int SIZE_OF_INTEGER = 4; @@ -105,7 +105,40 @@ void reportEnvelopeAvailability(final AbstractByteBufferedInputChannel toInputChannel(final private void announce(final AbstractByteBufferedInputChannel inputChannel) { inputChannel.checkForNetworkEvents(); - + if (++this.numberOfAnnouncedEnvelopes < this.numberOfInitialLogEntries) { return; } From 4a2abd4495c1e88d1a71fac1e470b408ea25ef07 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 16 Feb 2012 14:14:51 +0100 Subject: [PATCH 241/310] Temporarily switched ephemeral checkpoints to use one file per channel --- .../stratosphere/nephele/checkpointing/EphemeralCheckpoint.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 602338bf6d80c..5554680c1002c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -202,7 +202,7 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro if (buffer.isBackedByMemory()) { // Make sure we transfer the encapsulated buffer to a file and release the memory buffer again - final Buffer fileBuffer = BufferFactory.createFromFile(buffer.size(), this.task.getVertexID(), + final Buffer fileBuffer = BufferFactory.createFromFile(buffer.size(), transferEnvelope.getSource(), this.fileBufferManager); buffer.copyToBuffer(fileBuffer); transferEnvelope.setBuffer(fileBuffer); From 7a89de05f6f683b4b95aaec2c7b1c727a21c94df Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 16 Feb 2012 19:11:55 +0100 Subject: [PATCH 242/310] ReplayThread now reads back data to memory before emitting it --- .../checkpointing/EphemeralCheckpoint.java | 2 +- .../checkpointing/ReplayOutputBroker.java | 56 +++++++- .../ReplayOutputGateContext.java | 3 +- .../nephele/checkpointing/ReplayTask.java | 7 +- .../checkpointing/ReplayTaskContext.java | 129 ++++++++++++++++-- .../nephele/checkpointing/ReplayThread.java | 32 +++-- .../nephele/taskmanager/Task.java | 7 +- .../ByteBufferedChannelManager.java | 50 +++---- .../taskmanager/bytebuffered/TaskContext.java | 4 +- .../runtime/RuntimeInputChannelContext.java | 2 +- .../taskmanager/runtime/RuntimeTask.java | 13 +- .../runtime/RuntimeTaskContext.java | 16 +-- 12 files changed, 239 insertions(+), 82 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 5554680c1002c..602338bf6d80c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -202,7 +202,7 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro if (buffer.isBackedByMemory()) { // Make sure we transfer the encapsulated buffer to a file and release the memory buffer again - final Buffer fileBuffer = BufferFactory.createFromFile(buffer.size(), transferEnvelope.getSource(), + final Buffer fileBuffer = BufferFactory.createFromFile(buffer.size(), this.task.getVertexID(), this.fileBufferManager); buffer.copyToBuffer(fileBuffer); transferEnvelope.setBuffer(fileBuffer); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java index cce0bc297af47..cabd0512cfa0a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java @@ -6,24 +6,31 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; +import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -final class ReplayOutputBroker implements OutputChannelForwarder { +final class ReplayOutputBroker implements OutputChannelForwarder, BufferProvider { /** * The logger to report information and problems. */ private static final Log LOG = LogFactory.getLog(ReplayOutputBroker.class); + private final BufferProvider bufferProvider; + private final OutputChannelForwardingChain forwardingChain; private final IncomingEventQueue incomingEventQueue; - ReplayOutputBroker(final OutputChannelForwardingChain forwardingChain, final IncomingEventQueue incomingEventQueue) { + ReplayOutputBroker(final BufferProvider bufferProvider, final OutputChannelForwardingChain forwardingChain, + final IncomingEventQueue incomingEventQueue) { + + this.bufferProvider = bufferProvider; this.forwardingChain = forwardingChain; this.incomingEventQueue = incomingEventQueue; } @@ -78,4 +85,49 @@ boolean hasFinished() { return (!this.forwardingChain.anyForwarderHasDataLeft()); } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException { + + return this.bufferProvider.requestEmptyBuffer(minimumSizeOfBuffer); + } + + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException { + + return this.bufferProvider.requestEmptyBufferBlocking(minimumSizeOfBuffer); + } + + /** + * {@inheritDoc} + */ + @Override + public int getMaximumBufferSize() { + + return this.bufferProvider.getMaximumBufferSize(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isShared() { + + return this.bufferProvider.isShared(); + } + + /** + * {@inheritDoc} + */ + @Override + public void reportAsynchronousEvent() { + + this.bufferProvider.reportAsynchronousEvent(); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java index 5049a5f4456a9..59e650b09aa6c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java @@ -34,7 +34,8 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(); final IncomingEventQueue incomingEventQueue = AbstractOutputChannelContext .createIncomingEventQueue(forwardingChain); - final ReplayOutputBroker outputBroker = new ReplayOutputBroker(forwardingChain, incomingEventQueue); + final ReplayOutputBroker outputBroker = new ReplayOutputBroker(this.taskContext, forwardingChain, + incomingEventQueue); forwardingChain.addForwarder(outputBroker); forwardingChain.addForwarder(new ForwardingBarrier(channelID)); forwardingChain.addForwarder(new SpillingBarrier(isReceiverRunning, mergeSpillBuffers)); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index 9b2a523144e68..e9bb92dd66ce4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -35,6 +35,7 @@ import eu.stratosphere.nephele.services.memorymanager.MemoryManager; import eu.stratosphere.nephele.taskmanager.Task; import eu.stratosphere.nephele.taskmanager.TaskManager; +import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; @@ -406,9 +407,11 @@ public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) { */ @Override public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher, - final Map tasksWithUndecidedCheckpoints) { + final Map tasksWithUndecidedCheckpoints, + final LocalBufferPoolOwner previousBufferPoolOwner) { - return new ReplayTaskContext(this, transferEnvelopeDispatcher); + return new ReplayTaskContext(this, transferEnvelopeDispatcher, previousBufferPoolOwner, this.environment + .getOutputChannelIDs().size()); } private void reportExecutionStateChange(final boolean replayTaskStateChanged, final String optionalMessage) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java index d3a776b6cadcd..08bfbb9d1cf02 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java @@ -1,23 +1,47 @@ package eu.stratosphere.nephele.checkpointing; +import java.io.IOException; + +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.io.GateID; +import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.taskmanager.bufferprovider.AsynchronousEventListener; +import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; +import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeDispatcher; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; -final class ReplayTaskContext implements TaskContext { +final class ReplayTaskContext implements TaskContext, BufferProvider, AsynchronousEventListener { private final ReplayTask task; private final RuntimeDispatcher runtimeDispatcher; - ReplayTaskContext(final ReplayTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher) { + private final int numberOfChannels; + + private final LocalBufferPool localBufferPool; + + ReplayTaskContext(final ReplayTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, + final LocalBufferPoolOwner previousBufferPoolOwner, final int numberOfChannels) { this.task = task; this.runtimeDispatcher = new RuntimeDispatcher(transferEnvelopeDispatcher); + if (previousBufferPoolOwner == null) { + this.localBufferPool = new LocalBufferPool(1, false, this); + } else { + if (!(previousBufferPoolOwner instanceof RuntimeTaskContext)) { + throw new IllegalStateException("previousBufferPoolOwner is not of type RuntimeTaskContext"); + } + + final RuntimeTaskContext rtc = (RuntimeTaskContext) previousBufferPoolOwner; + this.localBufferPool = rtc.getLocalBufferPool(); + } + this.numberOfChannels = numberOfChannels; } /** @@ -38,22 +62,111 @@ public InputGateContext createInputGateContext(final GateID gateID) { return new ReplayInputGateContext(gateID); } + void registerReplayOutputBroker(final ChannelID channelID, final ReplayOutputBroker outputBroker) { + + this.task.registerReplayOutputBroker(channelID, outputBroker); + } + + RuntimeDispatcher getRuntimeDispatcher() { + + return this.runtimeDispatcher; + } + /** * {@inheritDoc} */ @Override - public LocalBufferPoolOwner getLocalBufferPoolOwner() { + public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException { - return null; + return this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer); } - void registerReplayOutputBroker(final ChannelID channelID, final ReplayOutputBroker outputBroker) { + /** + * {@inheritDoc} + */ + @Override + public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException { - this.task.registerReplayOutputBroker(channelID, outputBroker); + return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer); } - RuntimeDispatcher getRuntimeDispatcher() { + /** + * {@inheritDoc} + */ + @Override + public int getMaximumBufferSize() { - return this.runtimeDispatcher; + return this.localBufferPool.getMaximumBufferSize(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isShared() { + + return false; + } + + /** + * {@inheritDoc} + */ + @Override + public void reportAsynchronousEvent() { + + this.localBufferPool.reportAsynchronousEvent(); + } + + /** + * {@inheritDoc} + */ + @Override + public int getNumberOfChannels() { + + return this.numberOfChannels; + } + + /** + * {@inheritDoc} + */ + @Override + public void setDesignatedNumberOfBuffers(final int numberOfBuffers) { + + this.localBufferPool.setDesignatedNumberOfBuffers(numberOfBuffers); + } + + /** + * {@inheritDoc} + */ + @Override + public void clearLocalBufferPool() { + + // Clear the buffer cache + this.localBufferPool.destroy(); + } + + /** + * {@inheritDoc} + */ + @Override + public void logBufferUtilization() { + + final int ava = this.localBufferPool.getNumberOfAvailableBuffers(); + final int req = this.localBufferPool.getRequestedNumberOfBuffers(); + final int des = this.localBufferPool.getDesignatedNumberOfBuffers(); + + final Environment environment = this.task.getEnvironment(); + + System.out.println("\t\t" + environment.getTaskName() + " (Replay): " + ava + " available, " + req + + " requested, " + des + " designated"); + } + + /** + * {@inheritDoc} + */ + @Override + public void asynchronousEventOccurred() throws IOException, InterruptedException { + + throw new IllegalStateException("ReplayTaskContext received asynchronous event"); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index 26314e9be5445..d93f1deeaffb3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -13,6 +13,7 @@ import eu.stratosphere.nephele.execution.ExecutionObserver; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.taskmanager.transferenvelope.CheckpointDeserializer; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; @@ -136,7 +137,7 @@ void restart() { } private void replayCheckpoint() throws Exception { - + final CheckpointDeserializer deserializer = new CheckpointDeserializer(this.vertexID); int metaDataIndex = 0; @@ -167,7 +168,7 @@ private void replayCheckpoint() throws Exception { } // Wait for the file to be created - Thread.sleep(100); + Thread.sleep(10); } @@ -184,7 +185,22 @@ private void replayCheckpoint() throws Exception { final TransferEnvelope transferEnvelope = deserializer.getFullyDeserializedTransferEnvelope(); if (transferEnvelope != null) { - outputEnvelope(transferEnvelope); + + final ReplayOutputBroker broker = this.outputBrokerMap.get(transferEnvelope.getSource()); + if (broker == null) { + throw new IOException("Cannot find output broker for channel " + + transferEnvelope.getSource()); + } + + final Buffer srcBuffer = transferEnvelope.getBuffer(); + if (srcBuffer != null) { + final Buffer destBuffer = broker.requestEmptyBufferBlocking(srcBuffer.size()); + srcBuffer.copyToBuffer(destBuffer); + transferEnvelope.setBuffer(destBuffer); + srcBuffer.recycleBuffer(); + } + + broker.outputEnvelope(transferEnvelope); } } catch (EOFException eof) { // Close the file channel @@ -201,14 +217,4 @@ private void replayCheckpoint() throws Exception { } } } - - private void outputEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - - final ReplayOutputBroker outputBroker = this.outputBrokerMap.get(transferEnvelope.getSource()); - if (outputBroker == null) { - throw new IOException("Cannot find output broker for channel " + transferEnvelope.getSource()); - } - - outputBroker.outputEnvelope(transferEnvelope); - } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index 939e6345c75c0..c7dd480c262b2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -25,6 +25,7 @@ import eu.stratosphere.nephele.profiling.TaskManagerProfiler; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; +import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; @@ -97,7 +98,6 @@ public interface Task { */ void registerIOManager(IOManager ioManager); - /** * Registers the input splits provider with the task. * @@ -138,7 +138,8 @@ public interface Task { * @return the current execution state of the task */ ExecutionState getExecutionState(); - + TaskContext createTaskContext(TransferEnvelopeDispatcher transferEnvelopeDispatcher, - Map tasksWithUndecidedCheckpoints); + Map tasksWithUndecidedCheckpoints, + LocalBufferPoolOwner previousBufferPoolOwner); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 985d68ea91b2a..4f020416e0922 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -131,7 +131,8 @@ public void register(final Task task, final Set activeOutputChannels) final Environment environment = task.getEnvironment(); - final TaskContext taskContext = task.createTaskContext(this, this.tasksWithUndecidedCheckpoints); + final TaskContext taskContext = task.createTaskContext(this, this.tasksWithUndecidedCheckpoints, + this.localBufferPoolOwner.remove(task.getVertexID())); final Set outputGateIDs = environment.getOutputGateIDs(); for (final Iterator gateIt = outputGateIDs.iterator(); gateIt.hasNext();) { @@ -200,10 +201,7 @@ public void register(final Task task, final Set activeOutputChannels) } - final LocalBufferPoolOwner bufferPoolOwner = taskContext.getLocalBufferPoolOwner(); - if (bufferPoolOwner != null) { - this.localBufferPoolOwner.put(task.getVertexID(), bufferPoolOwner); - } + this.localBufferPoolOwner.put(task.getVertexID(), taskContext); redistributeGlobalBuffers(); } @@ -380,37 +378,21 @@ private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope, final InputChannelContext inputChannelContext = (InputChannelContext) cc; - if (srcBuffer.isBackedByMemory()) { - - Buffer destBuffer = null; - try { - destBuffer = inputChannelContext.requestEmptyBufferBlocking(srcBuffer.size()); - srcBuffer.copyToBuffer(destBuffer); - } catch (Exception e) { - LOG.error(StringUtils.stringifyException(e)); - if (destBuffer != null) { - destBuffer.recycleBuffer(); - } - continue; - } - // TODO: See if we can save one duplicate step here - final TransferEnvelope dup = transferEnvelope.duplicateWithoutBuffer(); - dup.setBuffer(destBuffer); - inputChannelContext.queueTransferEnvelope(dup); - - } else { - - // This is a file buffer, we can simply duplicate the envelope - TransferEnvelope dup = null; - try { - dup = transferEnvelope.duplicate(); - } catch (Exception e) { - LOG.error(StringUtils.stringifyException(e)); - continue; + Buffer destBuffer = null; + try { + destBuffer = inputChannelContext.requestEmptyBufferBlocking(srcBuffer.size()); + srcBuffer.copyToBuffer(destBuffer); + } catch (Exception e) { + LOG.error(StringUtils.stringifyException(e)); + if (destBuffer != null) { + destBuffer.recycleBuffer(); } - - inputChannelContext.queueTransferEnvelope(dup); + continue; } + // TODO: See if we can save one duplicate step here + final TransferEnvelope dup = transferEnvelope.duplicateWithoutBuffer(); + dup.setBuffer(destBuffer); + inputChannelContext.queueTransferEnvelope(dup); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java index d679e7d3eecab..d6cb3771890ff 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java @@ -18,11 +18,9 @@ import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; -public interface TaskContext { +public interface TaskContext extends LocalBufferPoolOwner { OutputGateContext createOutputGateContext(GateID gateID); InputGateContext createInputGateContext(GateID gateID); - - LocalBufferPoolOwner getLocalBufferPoolOwner(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java index d55118e9a6a05..5554d3e89d605 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java @@ -201,7 +201,7 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { if (sequenceNumber > expectedSequenceNumber) { - if (expectedSequenceNumber > 0) { + if (expectedSequenceNumber > 2000) { // This is a problem, now we are actually missing some data this.byteBufferedInputChannel.reportIOException(new IOException("Expected data packet " + expectedSequenceNumber + " but received " + sequenceNumber)); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index a7cfb26104ba5..d94c9a3f0071d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -50,6 +50,7 @@ import eu.stratosphere.nephele.services.memorymanager.MemoryManager; import eu.stratosphere.nephele.taskmanager.Task; import eu.stratosphere.nephele.taskmanager.TaskManager; +import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; import eu.stratosphere.nephele.template.AbstractInvokable; @@ -304,7 +305,8 @@ public void initialExecutionResourcesExhausted() { // Construct a resource utilization snapshot final long timestamp = System.currentTimeMillis(); - if(this.environment.getInputGate(0) != null && this.environment.getInputGate(0).getExecutionStart() < timestamp ){ + if (this.environment.getInputGate(0) != null + && this.environment.getInputGate(0).getExecutionStart() < timestamp) { this.startTime = this.environment.getInputGate(0).getExecutionStart(); } // Get CPU-Usertime in percent @@ -324,7 +326,7 @@ public void initialExecutionResourcesExhausted() { totalOutputAmount += outputChannel.getAmountOfDataTransmitted(); } } - //FIXME (marrus) it is not about what we received but what we processed yet + // FIXME (marrus) it is not about what we received but what we processed yet long totalInputAmount = 0; for (int i = 0; i < this.environment.getNumberOfInputGates(); ++i) { final InputGate inputGate = this.environment.getInputGate(i); @@ -494,7 +496,12 @@ public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) { */ @Override public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher, - final Map tasksWithUndecidedCheckpoints) { + final Map tasksWithUndecidedCheckpoints, + final LocalBufferPoolOwner previousBufferPoolOwner) { + + if (previousBufferPoolOwner != null) { + throw new IllegalStateException("Vertex " + this.vertexID + " has a previous buffer pool owner"); + } return new RuntimeTaskContext(this, transferEnvelopeDispatcher, tasksWithUndecidedCheckpoints); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 3be1585a09fef..8b4877c283f41 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -30,15 +30,13 @@ import eu.stratosphere.nephele.taskmanager.bufferprovider.AsynchronousEventListener; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; -import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; import eu.stratosphere.nephele.types.Record; -public final class RuntimeTaskContext implements BufferProvider, AsynchronousEventListener, LocalBufferPoolOwner, - TaskContext { +public final class RuntimeTaskContext implements BufferProvider, AsynchronousEventListener, TaskContext { private final LocalBufferPool localBufferPool; @@ -280,13 +278,9 @@ public InputGateContext createInputGateContext(final GateID gateID) { return new RuntimeInputGateContext(re.getTaskNameWithIndex(), this.transferEnvelopeDispatcher, inputGate, this.envelopeConsumptionLog); } - - /** - * {@inheritDoc} - */ - @Override - public LocalBufferPoolOwner getLocalBufferPoolOwner() { - - return this; + + public LocalBufferPool getLocalBufferPool() { + + return this.localBufferPool; } } From dc05b6fbef3d75707c1e3039ccc80d9ba3b82dfe Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 16 Feb 2012 20:14:08 +0100 Subject: [PATCH 243/310] Temporarily enabled receiver side spilling to prevent channel deadlocks --- .../nephele/taskmanager/runtime/RuntimeInputGateContext.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index 4e4895f550ba1..06a427c00330b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -23,6 +23,7 @@ import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.BufferFactory; import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.io.channels.FileBufferManager; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; @@ -80,7 +81,7 @@ public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws I return buffer; } - if (this.envelopeConsumptionLog.followsLog()) { + if (this.envelopeConsumptionLog.followsLog() || this.inputGate.getChannelType() == ChannelType.NETWORK) { return BufferFactory.createFromFile(minimumSizeOfBuffer, this.inputGate.getGateID(), fileBufferManager); } From 7576c86469d33129d39a47f4b9b4f5aed4532ba5 Mon Sep 17 00:00:00 2001 From: marrus Date: Thu, 16 Feb 2012 20:35:37 +0100 Subject: [PATCH 244/310] minor changes in decision components --- .../ResourceUtilizationSnapshot.java | 37 ++++++++++++++++++- .../taskmanager/runtime/RuntimeTask.java | 29 ++++++++++++++- 2 files changed, 63 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java index 1038e68c0e4a9..a23c8ccd708c8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java @@ -61,6 +61,10 @@ public final class ResourceUtilizationSnapshot implements IOReadableWritable { */ private long totalOutputAmount; + private long averageOutputRecordSize; + + private long averageInputRecordSize; + public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU) { @@ -99,6 +103,27 @@ public ResourceUtilizationSnapshot() { this.channelUtilization = new HashMap(); } + public ResourceUtilizationSnapshot(long timestamp, Map channelUtilization, long userCPU, + Boolean force, long totalInputAmount, long totalOutputAmount, long averageOutputRecordSize, + long averageInputRecordSize) { + if (timestamp <= 0L) { + throw new IllegalArgumentException("Argument timestamp must be larger than zero"); + } + + if (channelUtilization == null) { + throw new IllegalArgumentException("Argument channelUtilization is null"); + } + + this.timestamp = timestamp; + this.channelUtilization = channelUtilization; + this.userCPU = userCPU; + this.forced = force; + this.totalInputAmount = totalInputAmount; + this.totalOutputAmount = totalOutputAmount; + this.averageOutputRecordSize = averageOutputRecordSize; + this.averageInputRecordSize = averageInputRecordSize; + + } /** * {@inheritDoc} */ @@ -128,6 +153,8 @@ public void write(final DataOutput out) throws IOException { } out.writeLong(this.totalInputAmount); out.writeLong(this.totalOutputAmount); + out.writeLong(this.averageInputRecordSize); + out.writeLong(this.averageOutputRecordSize); } /** @@ -155,7 +182,8 @@ public void read(final DataInput in) throws IOException { } this.totalInputAmount = in.readLong(); this.totalOutputAmount = in.readLong(); - + this.averageInputRecordSize = in.readLong(); + this.averageOutputRecordSize = in.readLong(); } /** @@ -212,4 +240,11 @@ public long getTotalInputAmount() { public long getTotalOutputAmount() { return this.totalOutputAmount; } + public long getAverageOutputRecordSize() { + return averageOutputRecordSize; + } + public long getAverageInputRecordSize() { + return averageInputRecordSize; + } + } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index a7cfb26104ba5..fea382035bc69 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -15,6 +15,7 @@ package eu.stratosphere.nephele.taskmanager.runtime; +import java.io.IOException; import java.lang.management.ManagementFactory; import java.lang.management.ThreadMXBean; import java.util.HashMap; @@ -307,6 +308,7 @@ public void initialExecutionResourcesExhausted() { if(this.environment.getInputGate(0) != null && this.environment.getInputGate(0).getExecutionStart() < timestamp ){ this.startTime = this.environment.getInputGate(0).getExecutionStart(); } + LOG.info("Task " + this.getTaskName() + " started " + this.startTime); // Get CPU-Usertime in percent ThreadMXBean threadBean = ManagementFactory.getThreadMXBean(); long userCPU = (threadBean.getCurrentThreadUserTime() / NANO_TO_MILLISECONDS) * 100 @@ -315,8 +317,11 @@ public void initialExecutionResourcesExhausted() { // collect outputChannelUtilization final Map channelUtilization = new HashMap(); long totalOutputAmount = 0; + int numrec = 0; + long averageOutputRecordSize= 0; for (int i = 0; i < this.environment.getNumberOfOutputGates(); ++i) { final OutputGate outputGate = this.environment.getOutputGate(i); + numrec += outputGate.getNumRecords(); for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); channelUtilization.put(outputChannel.getID(), @@ -324,20 +329,39 @@ public void initialExecutionResourcesExhausted() { totalOutputAmount += outputChannel.getAmountOfDataTransmitted(); } } + if(numrec != 0){ + averageOutputRecordSize = totalOutputAmount/numrec; + } //FIXME (marrus) it is not about what we received but what we processed yet + boolean allClosed = true; + int numinrec = 0; long totalInputAmount = 0; + long averageInputRecordSize = 0; for (int i = 0; i < this.environment.getNumberOfInputGates(); ++i) { final InputGate inputGate = this.environment.getInputGate(i); + numrec += inputGate.getNumRecords(); for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); channelUtilization.put(inputChannel.getID(), Long.valueOf(inputChannel.getAmountOfDataTransmitted())); totalInputAmount += inputChannel.getAmountOfDataTransmitted(); + try { + if(!inputChannel.isClosed()){ + allClosed = false; + } + } catch (IOException e) { + e.printStackTrace(); + } catch (InterruptedException e) { + e.printStackTrace(); + } } } + if(numinrec != 0){ + averageInputRecordSize = totalInputAmount/numinrec; + } Boolean force = null; - + Boolean stateful = false; if (this.environment.getInvokable().getClass().isAnnotationPresent(Stateful.class) && !this.environment.getInvokable().getClass().isAnnotationPresent(Stateless.class)) { // Don't checkpoint stateful tasks @@ -349,8 +373,9 @@ public void initialExecutionResourcesExhausted() { force = forced.checkpoint(); } } + final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, channelUtilization, userCPU, - force, totalInputAmount, totalOutputAmount); + force, totalInputAmount, totalOutputAmount, averageOutputRecordSize, averageInputRecordSize); // Notify the listener objects final Iterator it = this.registeredListeners.iterator(); From 60d3d3adf37de87c9a30807ac87b6fb9198af194 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 16 Feb 2012 20:32:29 +0000 Subject: [PATCH 245/310] Integrated Stephan's new file buffer manager --- .../configuration/ConfigConstants.java | 23 + .../nephele/io/channels/Buffer.java | 133 ++--- .../nephele/io/channels/InternalBuffer.java | 43 -- .../checkpointing/EphemeralCheckpoint.java | 3 +- .../nephele/io/channels/BufferFactory.java | 29 +- .../{FileID.java => ChannelWithPosition.java} | 27 +- .../nephele/io/channels/FileBuffer.java | 301 ++++++------ .../io/channels/FileBufferManager.java | 464 +++++++++++------- .../nephele/io/channels/MemoryBuffer.java | 178 +++---- .../io/channels/ReadableSpillingFile.java | 88 ---- .../io/channels/WritableSpillingFile.java | 225 --------- .../io/compression/AbstractCompressor.java | 6 +- .../io/compression/AbstractDecompressor.java | 6 +- .../CheckpointDeserializer.java | 26 +- .../CheckpointSerializer.java | 41 +- 15 files changed, 625 insertions(+), 968 deletions(-) delete mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/InternalBuffer.java rename nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/{FileID.java => ChannelWithPosition.java} (62%) delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ReadableSpillingFile.java delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/WritableSpillingFile.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/configuration/ConfigConstants.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/configuration/ConfigConstants.java index 061afb11e6367..c053aada1ed3d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/configuration/ConfigConstants.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/configuration/ConfigConstants.java @@ -66,6 +66,19 @@ public final class ConfigConstants { * The key for the config parameter defining flag to terminate a job at job-client shutdown. */ public static final String JOBCLIENT_SHUTDOWN_TERMINATEJOB_KEY = "jobclient.shutdown.terminatejob"; + + /** + * The key for the config parameter defining how many channels spill into the same physical file. + */ + public static final String TASKMANAGER_FILECHANNEL_NUMMERGED = "taskmanager.filechannel.nummerged"; + + /** + * The key for the config parameter defining the size (in bytes) of the chunk that is contiguously + * allocated for write operations from the same channel. Only relevant when multiple channels + * write into the same file. + */ + public static final String TASKMANAGER_FILECHANNEL_EXTENDSIZE = "taskmanager.filechannel.extendsize"; + // ------------------------------------------------------------------------ // Default Values @@ -110,6 +123,16 @@ public final class ConfigConstants { * The default scheduler to be used when Nephele is started in local mode. */ public static final String DEFAULT_LOCAL_MODE_SCHEDULER = "eu.stratosphere.nephele.jobmanager.scheduler.local.LocalScheduler"; + + /** + * The default scheduler to be used when Nephele is started in local mode. + */ + public static final int DEFAULT_NUM_FILECHANNELS_MERGED = 16; + + /** + * The default size of an extend allocated for a channel in a merged file. 4 MiBytes. + */ + public static final int DEFAULT_FILECHANNEL_EXTEND_SIZE = 4 * 1024 * 1024; // ----------------------------- Instances -------------------------------- diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java index 4527cd2870adf..a93c510f03f15 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java @@ -34,14 +34,8 @@ * * @author warneke */ -public class Buffer implements ReadableByteChannel, WritableByteChannel { - - /** - * The concrete buffer implementation to which all method calls on - * this object are delegated. - */ - private final InternalBuffer internalBuffer; - +public abstract class Buffer implements ReadableByteChannel, WritableByteChannel +{ /** * Stores whether this buffer has already been recycled. */ @@ -53,77 +47,56 @@ public class Buffer implements ReadableByteChannel, WritableByteChannel { * @param internalBuffer * the concrete implementation which backs the buffer */ - Buffer(InternalBuffer internalBuffer) { - - this.internalBuffer = internalBuffer; - } + protected Buffer() + {} /** * {@inheritDoc} */ @Override - public int read(ByteBuffer arg0) throws IOException { - - return this.internalBuffer.read(arg0); - } + public abstract int read(ByteBuffer destination) throws IOException; /** * Reads data from the buffer and writes it to the * given {@link WritableByteChannel} object. * - * @param writeByteChannel - * the {@link WritableByteChannel} object to write the data to - * @return the number of bytes read from the buffer, potentially 0 or -10 or -10 - * @throws IOException - * thrown if an error occurs while writing data to the buffer + * @param source The {@link ReadableByteChannel} object to read data from. + * @return The number of bytes written to the buffer, possibly 0. + * @throws IOException Thrown if an error occurs while writing data to the buffer. */ - public int write(ReadableByteChannel readableByteChannel) throws IOException { - - return this.internalBuffer.write(readableByteChannel); - } + public abstract int write(ReadableByteChannel source) throws IOException; /** * Returns the number of bytes which can be either still written to or read from @@ -135,10 +108,7 @@ public int write(ReadableByteChannel readableByteChannel) throws IOException { * * @return the number of bytes which can be either written to or read from the buffer */ - public int remaining() { - - return this.internalBuffer.remaining(); - } + public abstract int remaining(); /** * Checks whether data can still be written to or read from the buffer. @@ -147,8 +117,7 @@ public int remaining() { * the buffer, false otherwise */ public boolean hasRemaining() { - - return (this.internalBuffer.remaining() > 0); + return remaining() > 0; } /** @@ -158,20 +127,7 @@ public boolean hasRemaining() { * * @return the size of the buffer in bytes */ - public int size() { - - return this.internalBuffer.size(); - } - - /** - * Returns the {@link InternalBuffer} object which contains - * the actual implementation of this buffer. - * - * @return the {@link InternalBuffer} object which contains the actual implementation of this buffer - */ - public InternalBuffer getInternalBuffer() { - return this.internalBuffer; - } + public abstract int size(); /** * Recycles the buffer. In case of a memory backed buffer, the internal memory buffer @@ -179,12 +135,14 @@ public InternalBuffer getInternalBuffer() { * file created for this buffer is deleted. A buffer can only be recycled once. Calling this method more than once * will therefore have no effect. */ - public void recycleBuffer() { - + public final void recycleBuffer() + { if (this.isRecycled.compareAndSet(false, true)) { - this.internalBuffer.recycleBuffer(); + recycle(); } } + + protected abstract void recycle(); /** * Switches the buffer from write mode into read mode. After being switched to read @@ -193,10 +151,7 @@ public void recycleBuffer() { * @throws IOException * throws if an error occurs while finishing writing mode */ - public void finishWritePhase() throws IOException { - - this.internalBuffer.finishWritePhase(); - } + public abstract void finishWritePhase() throws IOException; /** * Returns whether the buffer is backed by main memory or a file. @@ -204,10 +159,7 @@ public void finishWritePhase() throws IOException { * @return true if the buffer is backed by main memory * or false if it is backed by a file */ - public boolean isBackedByMemory() { - - return this.internalBuffer.isBackedByMemory(); - } + public abstract boolean isBackedByMemory(); /** * Copies the content of the buffer to the given destination buffer. The state of the source buffer is not modified @@ -218,19 +170,7 @@ public boolean isBackedByMemory() { * @throws IOException * thrown if an error occurs while copying the data */ - public void copyToBuffer(Buffer destinationBuffer) throws IOException { - - if (size() > destinationBuffer.size()) { - throw new IllegalArgumentException("Destination buffer is too small to store content of source buffer: " - + size() + " vs. " + destinationBuffer.size()); - } - - if (this.internalBuffer.isInWriteMode()) { - throw new IllegalStateException("Cannot copy buffer that is still in write mode"); - } - - this.internalBuffer.copyToBuffer(destinationBuffer); - } + public abstract void copyToBuffer(Buffer destinationBuffer) throws IOException; /** * Duplicates the buffer. This operation does not duplicate the actual @@ -239,12 +179,5 @@ public void copyToBuffer(Buffer destinationBuffer) throws IOException { * * @return the duplicated buffer */ - public Buffer duplicate() throws IOException, InterruptedException { - - if (this.internalBuffer.isInWriteMode()) { - throw new IllegalStateException("Cannot duplicate buffer that is still in write mode"); - } - - return new Buffer(this.internalBuffer.duplicate()); - } + public abstract Buffer duplicate() throws IOException, InterruptedException; } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/InternalBuffer.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/InternalBuffer.java deleted file mode 100644 index 7ae553e800e62..0000000000000 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/channels/InternalBuffer.java +++ /dev/null @@ -1,43 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.io.channels; - -import java.io.IOException; -import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; - -public interface InternalBuffer extends ReadableByteChannel, WritableByteChannel { - - int write(ReadableByteChannel readableByteChannel) throws IOException; - - int read(WritableByteChannel writableByteChannel) throws IOException; - - int remaining(); - - int size(); - - void recycleBuffer(); - - void finishWritePhase() throws IOException; - - boolean isBackedByMemory(); - - InternalBuffer duplicate() throws IOException, InterruptedException; - - boolean isInWriteMode(); - - void copyToBuffer(Buffer destinationBuffer) throws IOException; -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 5554680c1002c..149653fbc39cd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -267,7 +267,8 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro + this.task.getVertexID() + "_final").close(); // Since it is unclear whether the underlying physical file will ever be read, we force to close it. - this.fileBufferManager.forceCloseOfWritableSpillingFile(this.task.getVertexID()); + //TODO: Fix me + //this.fileBufferManager.forceCloseOfWritableSpillingFile(this.task.getVertexID()); LOG.info("Finished persistent checkpoint for vertex " + this.task.getVertexID()); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java index 1979b0388de2e..3e2618be85c99 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java @@ -15,33 +15,34 @@ package eu.stratosphere.nephele.io.channels; +import java.io.IOException; import java.nio.ByteBuffer; import eu.stratosphere.nephele.io.AbstractID; -import eu.stratosphere.nephele.io.channels.Buffer; -import eu.stratosphere.nephele.io.channels.InternalBuffer; -public abstract class BufferFactory { +public final class BufferFactory { - public static Buffer createFromFile(final int bufferSize, final AbstractID ownerID, - final FileBufferManager fileBufferManager) { + public static FileBuffer createFromFile(final int bufferSize, final AbstractID ownerID, + final FileBufferManager fileBufferManager) throws IOException { - final InternalBuffer internalBuffer = new FileBuffer(bufferSize, ownerID, fileBufferManager); - return new Buffer(internalBuffer); + return new FileBuffer(bufferSize, ownerID, fileBufferManager); } - public static Buffer createFromCheckpoint(final int bufferSize, final FileID fileID, final long offset, - final AbstractID ownerID, final FileBufferManager fileBufferManager) { + public static FileBuffer createFromCheckpoint(final int bufferSize, final long offset, + final AbstractID ownerID, final FileBufferManager fileBufferManager) throws IOException { - final InternalBuffer internalBuffer = new FileBuffer(bufferSize, fileID, offset, ownerID, fileBufferManager); - - return new Buffer(internalBuffer); + return new FileBuffer(bufferSize, offset, ownerID, fileBufferManager); } public static Buffer createFromMemory(final int bufferSize, final ByteBuffer byteBuffer, final MemoryBufferPoolConnector bufferPoolConnector) { - final InternalBuffer internalBuffer = new MemoryBuffer(bufferSize, byteBuffer, bufferPoolConnector); - return new Buffer(internalBuffer); + return new MemoryBuffer(bufferSize, byteBuffer, bufferPoolConnector); + } + + /** + * Private constructor to prevent instantiation. + */ + private BufferFactory() { } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileID.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithPosition.java similarity index 62% rename from nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileID.java rename to nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithPosition.java index 8aa53b5efa7ae..11221a0d5cc0b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileID.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithPosition.java @@ -15,13 +15,30 @@ package eu.stratosphere.nephele.io.channels; -import eu.stratosphere.nephele.io.AbstractID; +import java.nio.channels.FileChannel; /** - * A file ID represents a unique identifier for a physical file. - * - * @author warneke + * A simple encapsulation of a file channel with an offset. This object is used for purposes, where + * the channel is accessed by multiple threads and its internal position may be changed. */ -public final class FileID extends AbstractID { +public class ChannelWithPosition { + private final FileChannel channel; + + private final long offset; + + ChannelWithPosition(final FileChannel channel, final long offset) { + this.channel = channel; + this.offset = offset; + } + + public FileChannel getChannel() { + + return this.channel; + } + + public long getOffset() { + + return this.offset; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java index 925ace96d570c..4f19db76280a2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java @@ -20,142 +20,144 @@ import java.nio.channels.FileChannel; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; -import java.util.concurrent.atomic.AtomicBoolean; import eu.stratosphere.nephele.io.AbstractID; -import eu.stratosphere.nephele.io.channels.InternalBuffer; -public class FileBuffer implements InternalBuffer { - - private long bufferSize; +public final class FileBuffer extends Buffer { + + private final AbstractID ownerID; private final FileBufferManager fileBufferManager; - private final AbstractID ownerID; - - private FileID fileID; + private final FileChannel fileChannel; - private FileChannel fileChannel; + private long offset; - private final AtomicBoolean writeMode; + private int bufferSize; - private long totalBytesWritten = 0; + private int totalBytesWritten; - private long totalBytesRead = 0; + private int totalBytesRead; - private long offset; + private volatile boolean writeMode; - FileBuffer(final int bufferSize, final FileID fileID, final long offset, final AbstractID ownerID, final FileBufferManager fileBufferManager) { - this.bufferSize = bufferSize; - this.fileID = fileID; - this.offset = offset; + /** + * Creates a file buffer for a chunk of data that is already in a file. + * + * @param bufferSize + * @param offset + * @param ownerID + */ + FileBuffer(final int bufferSize, final long offset, final AbstractID ownerID, + final FileBufferManager fileBufferManager) + throws IOException + { this.ownerID = ownerID; this.fileBufferManager = fileBufferManager; - this.writeMode = new AtomicBoolean(false); - } - - FileBuffer(final int bufferSize, final AbstractID ownerID, final FileBufferManager fileBufferManager) { - + this.offset = offset; this.bufferSize = bufferSize; - this.fileID = null; - this.offset = 0L; - this.ownerID = ownerID; + this.writeMode = false; + + this.fileChannel = fileBufferManager.getChannelAndIncrementReferences(ownerID); + } + + /** + * Creates a file Buffer that will have space reserved in the file for its data. + * + * @param bufferSize + * @param ownerID + * @param fileBufferManager + * @throws IOException + */ + FileBuffer(final int bufferSize, final AbstractID ownerID, final FileBufferManager fileBufferManager) + throws IOException { + this.fileBufferManager = fileBufferManager; - this.writeMode = new AtomicBoolean(true); + this.ownerID = ownerID; + this.bufferSize = bufferSize; + this.writeMode = true; + + final ChannelWithPosition cwp = fileBufferManager.getChannelForWriteAndIncrementReferences(ownerID, bufferSize); + this.fileChannel = cwp.getChannel(); + this.offset = cwp.getOffset(); + } + + private FileBuffer(final FileBuffer toCopy) { + + this.ownerID = toCopy.ownerID; + this.fileBufferManager = toCopy.fileBufferManager; + this.fileChannel = toCopy.fileChannel; + this.offset = toCopy.offset; + this.bufferSize = toCopy.bufferSize; + this.totalBytesWritten = toCopy.totalBytesWritten; + this.totalBytesRead = toCopy.totalBytesRead; + this.writeMode = toCopy.writeMode; } + /** + * {@inheritDoc} + */ @Override - public int read(WritableByteChannel writableByteChannel) throws IOException { + public int read(final WritableByteChannel writableByteChannel) throws IOException { - if (this.writeMode.get()) { + if (this.writeMode) { throw new IOException("FileBuffer is still in write mode!"); } - - if (this.fileChannel == null) { - try { - this.fileChannel = this.fileBufferManager.getFileChannelForReading(this.ownerID, this.fileID); - if (this.fileChannel == null) { - return 0; - } - } catch (InterruptedException e) { - return -1; - } - if (this.fileChannel.position() != (this.offset + this.totalBytesRead)) { - this.fileChannel.position(this.offset + this.totalBytesRead); - } - } - if (this.totalBytesRead >= this.bufferSize) { return -1; } - final long bytesRead = this.fileChannel.transferTo(this.offset + this.totalBytesRead, this.bufferSize - - this.totalBytesRead, writableByteChannel); + final long bytesRead = this.fileChannel.transferTo(this.offset + this.totalBytesRead, + this.bufferSize - this.totalBytesRead, writableByteChannel); this.totalBytesRead += bytesRead; return (int) bytesRead; } + /** + * {@inheritDoc} + */ @Override public int read(ByteBuffer dst) throws IOException { - if (this.writeMode.get()) { + if (this.writeMode) { throw new IOException("FileBuffer is still in write mode!"); } - - if (this.fileChannel == null) { - try { - this.fileChannel = this.fileBufferManager.getFileChannelForReading(this.ownerID, this.fileID); - if (this.fileChannel == null) { - return 0; - } - } catch (InterruptedException e) { - return -1; - } - if (this.fileChannel.position() != (this.offset + this.totalBytesRead)) { - this.fileChannel.position(this.offset + this.totalBytesRead); - } - } - if (this.totalBytesRead >= this.bufferSize) { return -1; } - final int rem = remaining(); - int bytesRead; - if (dst.remaining() > rem) { - final int excess = dst.remaining() - rem; - dst.limit(dst.limit() - excess); - bytesRead = this.fileChannel.read(dst); - dst.limit(dst.limit() + excess); - } else { - bytesRead = this.fileChannel.read(dst); - } - + final int bytesRead = readInternal(dst, this.offset + this.totalBytesRead, remaining()); if (bytesRead < 0) { return -1; } this.totalBytesRead += bytesRead; - return bytesRead; } + private final int readInternal(ByteBuffer dst, long position, int numBytes) throws IOException { + if (dst.remaining() > numBytes) { + final int excess = dst.remaining() - numBytes; + dst.limit(dst.limit() - excess); + final int bytesRead = this.fileChannel.read(dst, position); + dst.limit(dst.limit() + excess); + return bytesRead >= 0 ? bytesRead : -1; + + } else { + return this.fileChannel.read(dst, position); + } + } + + /** + * {@inheritDoc} + */ @Override public int write(final ReadableByteChannel readableByteChannel) throws IOException { - if (!this.writeMode.get()) { + if (!this.writeMode) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } - - if (this.fileChannel == null) { - this.fileChannel = this.fileBufferManager.getFileChannelForWriting(this.ownerID); - if (this.fileChannel == null) { - return 0; - } - this.offset = this.fileChannel.position(); - } - if (this.totalBytesWritten >= this.bufferSize) { return 0; } @@ -167,61 +169,55 @@ public int write(final ReadableByteChannel readableByteChannel) throws IOExcepti return (int) bytesWritten; } + /** + * {@inheritDoc} + */ @Override public int write(final ByteBuffer src) throws IOException { - if (!this.writeMode.get()) { + if (!this.writeMode) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } - - if (this.fileChannel == null) { - this.fileChannel = this.fileBufferManager.getFileChannelForWriting(this.ownerID); - if (this.fileChannel == null) { - return 0; - } - } - if (this.totalBytesWritten >= this.bufferSize) { return 0; } // Make sure we do not exceed the buffer limit - long bytesWritten; + int bytesWritten; final int rem = (int) (this.bufferSize - this.totalBytesWritten); if (src.remaining() > rem) { final int excess = src.remaining() - rem; src.limit(src.limit() - excess); - bytesWritten = this.fileChannel.write(src); + bytesWritten = this.fileChannel.write(src, this.offset + this.totalBytesWritten); src.limit(src.limit() + excess); } else { - bytesWritten = this.fileChannel.write(src); + bytesWritten = this.fileChannel.write(src, this.offset + this.totalBytesWritten); } this.totalBytesWritten += bytesWritten; - return (int) bytesWritten; + return bytesWritten; } @Override public void close() throws IOException { - - System.out.println("Close"); + this.fileChannel.close(); } @Override public boolean isOpen() { - + return this.fileChannel.isOpen(); } @Override public int remaining() { - - if (this.writeMode.get()) { - return (int) (this.bufferSize - this.totalBytesWritten); + + if (this.writeMode) { + return this.bufferSize - this.totalBytesWritten; } else { - return (int) (this.bufferSize - this.totalBytesRead); + return this.bufferSize - this.totalBytesRead; } } @@ -231,77 +227,58 @@ public int size() { } @Override - public void recycleBuffer() { - - try { - if (this.fileChannel != null) { - this.fileBufferManager.releaseFileChannelForReading(this.ownerID, this.fileID); - this.fileChannel = null; - } - } catch (Exception e) { - e.printStackTrace(); - } - - this.fileBufferManager.decreaseBufferCounter(this.ownerID, this.fileID); + protected void recycle() { + this.fileBufferManager.decrementReferences(this.ownerID); } @Override public void finishWritePhase() throws IOException { - - if (this.writeMode.compareAndSet(true, false)) { - - final long currentFileSize = this.offset + this.totalBytesWritten; - // If the input channel this buffer belongs to is already canceled, fileChannel may be null - if (this.fileChannel != null) { - this.fileChannel.position(currentFileSize); - } - this.fileChannel = null; + if (this.writeMode) { this.bufferSize = this.totalBytesWritten; - // System.out.println("Buffer size: " + this.bufferSize); - this.fileID = this.fileBufferManager.reportEndOfWritePhase(this.ownerID, currentFileSize); + this.writeMode = false; } - } @Override public boolean isBackedByMemory() { - return false; } @Override - public InternalBuffer duplicate() throws IOException, InterruptedException { - - this.fileBufferManager.increaseBufferCounter(this.ownerID, this.fileID); - - final FileBuffer dup = new FileBuffer((int) this.bufferSize, this.ownerID, this.fileBufferManager); - dup.writeMode.set(this.writeMode.get()); - dup.fileID = this.fileID; - dup.offset = this.offset; + public FileBuffer duplicate() throws IOException, InterruptedException { + + if (this.writeMode) { + throw new IllegalStateException("Cannot duplicate buffer that is still in write mode"); + } - return dup; + this.fileBufferManager.incrementReferences(this.ownerID); + return new FileBuffer(this); } @Override public void copyToBuffer(final Buffer destinationBuffer) throws IOException { + + if (this.writeMode) { + throw new IllegalStateException("Cannot copy buffer that is still in write mode"); + } + if (size() > destinationBuffer.size()) { + throw new IllegalArgumentException("Destination buffer is too small to store content of source buffer: " + + size() + " vs. " + destinationBuffer.size()); + } - if (destinationBuffer.isBackedByMemory()) { - - final long tbr = this.totalBytesRead; - if (this.fileChannel != null) { - this.fileBufferManager.releaseFileChannelForReading(this.ownerID, this.fileID); - } + if (destinationBuffer.isBackedByMemory()) + { + final ByteBuffer bb = ((MemoryBuffer) destinationBuffer).getByteBuffer(); + final int tbr = this.totalBytesRead; this.totalBytesRead = 0; - while (remaining() > 0) { - destinationBuffer.write(this); + + int rem = 0; + while ((rem = remaining()) > 0) { + this.totalBytesRead += readInternal(bb, this.offset + this.totalBytesRead, rem); } + destinationBuffer.finishWritePhase(); - if (this.fileChannel != null) { - this.fileBufferManager.releaseFileChannelForReading(this.ownerID, this.fileID); - } - this.fileChannel = null; this.totalBytesRead = tbr; - return; } @@ -313,27 +290,27 @@ public void copyToBuffer(final Buffer destinationBuffer) throws IOException { */ @Override public boolean isInWriteMode() { - - return this.writeMode.get(); + + return this.writeMode; } /** - * Returns the ID of the file which backs this buffers. + * Returns the offset in bytes which marks the begin of the buffer's data in the underlying file. * - * @return the ID of the file which backs this buffer + * @return the buffer's offset in bytes */ - public FileID getFileID() { - - return this.fileID; + public long getOffset() { + + return this.offset; } /** - * Returns the offset in bytes which marks the begin of the buffer's data in the underlying file. + * Gets this file buffer's owner id. * - * @return the buffer's offset in bytes + * @return This file buffer's owner id. */ - public long getOffset() { - - return this.offset; + public AbstractID getOwnerID() { + + return this.ownerID; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java index eb35caccdfbd2..1c4dcb4b44d9f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java @@ -17,9 +17,11 @@ import java.io.File; import java.io.IOException; +import java.io.RandomAccessFile; import java.nio.channels.FileChannel; -import java.util.HashMap; -import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -29,7 +31,6 @@ import eu.stratosphere.nephele.io.AbstractID; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; -import eu.stratosphere.nephele.util.StringUtils; /** * The file buffer manager manages the physical files which may be used to store the output or input of @@ -39,257 +40,358 @@ * This class is thread-safe. * * @author warneke + * @author Stephan Ewen */ -public final class FileBufferManager { +public final class FileBufferManager +{ + /** + * The prefix with which spill files are stored. + */ + public static final String FILE_BUFFER_PREFIX = "fb_"; /** * The logging object. */ private static final Log LOG = LogFactory.getLog(FileBufferManager.class); - public static final String FILE_BUFFER_PREFIX = "fb_"; - /** - * Stores the location of the directory for temporary files. + * The singleton instance of the file buffer manager. */ - private final String tmpDir; - - private final Map writableSpillingFileMap = new HashMap(); + private static final FileBufferManager instance = new FileBufferManager(); - private final Map> readableSpillingFileMap = new HashMap>(); - - private static FileBufferManager instance = null; - - public static synchronized FileBufferManager getInstance() { - - if (instance == null) { - instance = new FileBufferManager(); - } + /** + * The minimal size for an extend allocated for a channel. + */ + private static final int MIN_EXTEND_SIZE = 64 * 1024; + /** + * Gets the singleton instance of the file buffer manager. + * + * @return The file buffer manager singleton instance. + */ + public static FileBufferManager getInstance() + { return instance; } - private FileBufferManager() { + // -------------------------------------------------------------------------------------------- - this.tmpDir = GlobalConfiguration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(":")[0]; - } + /** + * The map from owner IDs to files + */ + private final ConcurrentHashMap fileMap; - private ReadableSpillingFile getReadableSpillingFile(final AbstractID ownerID, final FileID fileID) - throws IOException, InterruptedException { + /** + * The directories for temporary files. + */ + private final String[] tmpDirs; - if (ownerID == null) { - throw new IllegalStateException("ownerID is null"); - } + /** + * The size of the extend to allocate for each channel. + */ + private final int extendSize; - if (fileID == null) { - throw new IllegalStateException("fileID is null"); + /** + * + */ + private FileBufferManager() { + + this.tmpDirs = GlobalConfiguration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(":"); + + final int extendSize = GlobalConfiguration.getConfiguration().getInteger( + ConfigConstants.TASKMANAGER_FILECHANNEL_EXTENDSIZE, + ConfigConstants.DEFAULT_FILECHANNEL_EXTEND_SIZE); + + // check extend size + if (extendSize < MIN_EXTEND_SIZE) { + LOG.error("Invalid extend size " + extendSize + ". Minimum extend size is " + MIN_EXTEND_SIZE + + ". Falling back to default extend size of " + ConfigConstants.DEFAULT_FILECHANNEL_EXTEND_SIZE); + this.extendSize = ConfigConstants.DEFAULT_FILECHANNEL_EXTEND_SIZE; + } else if ((extendSize & (extendSize - 1)) != 0) { + // not a power of two + this.extendSize = Integer.highestOneBit(extendSize); + LOG.warn("Changing extend size from " + extendSize + " to " + this.extendSize + + " to make it a power of two."); + } else { + this.extendSize = extendSize; } - Map map = null; - synchronized (this.readableSpillingFileMap) { - map = this.readableSpillingFileMap.get(ownerID); - if (map == null) { - map = new HashMap(); - this.readableSpillingFileMap.put(ownerID, map); + // check temp dirs + for (int i = 0; i < this.tmpDirs.length; i++) { + File f = new File(this.tmpDirs[i]); + if (!(f.exists() && f.isDirectory() && f.canWrite())) { + LOG.error("Temp directory '" + f.getAbsolutePath() + "' is not a writable directory. " + + "Replacing path with default temp directory: " + ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH); + this.tmpDirs[i] = ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH; } + this.tmpDirs[i] = this.tmpDirs[i] + File.separator + FILE_BUFFER_PREFIX; } - synchronized (map) { + this.fileMap = new ConcurrentHashMap(2048, 0.8f, 64); + } - while (!map.containsKey(fileID)) { + // -------------------------------------------------------------------------------------------- - synchronized (this.writableSpillingFileMap) { - WritableSpillingFile writableSpillingFile = this.writableSpillingFileMap.get(ownerID); - if (writableSpillingFile != null) { - writableSpillingFile.requestReadAccess(); + /** + * Gets the file channel to for the owner with the given id. + * + * @param id + * The id for which to retrieve the channel. + * @throws IllegalStateException + * Thrown, if the channel has not been registered or has already been removed. + */ + public FileChannel getChannel(final AbstractID id) throws IOException { - if (writableSpillingFile.isSafeToClose()) { - writableSpillingFile.close(); - this.writableSpillingFileMap.remove(ownerID); - map.put( - writableSpillingFile.getFileID(), - writableSpillingFile.toReadableSpillingFile()); - } - } - } + final ChannelWithAccessInfo info = getChannelInternal(id, false); + if (info != null) { + return info.getChannel(); + } else { + throw new IllegalStateException("No channel is registered (any more) for the given id."); + } + } - if (!map.containsKey(fileID)) { - map.wait(WritableSpillingFile.MAXIMUM_TIME_WITHOUT_WRITE_ACCESS); - } - } + /** + * Gets the file channel to for the owner with the given id and increments the references to that channel by one. + * + * @param id + * The id for which to retrieve the channel. + * @throws IllegalStateException + * Thrown, if the channel has not been registered or has already been removed. + */ + public FileChannel getChannelAndIncrementReferences(final AbstractID owner) throws IOException { - return map.get(fileID); + final ChannelWithAccessInfo info = getChannelInternal(owner, false); + if (info != null) { + return info.getAndIncrementReferences(); + } else { + throw new IllegalStateException("No channel is registered (any more) for the given id."); } } - public FileChannel getFileChannelForReading(final AbstractID ownerID, final FileID fileID) throws IOException, - InterruptedException { + /** + * Gets the file channel to for the owner with the given id and reserved the portion of the given size for + * writing. The position where the reserved space starts is contained in the return value. This method + * automatically increments the number of references to the channel by one. + *

+ * This method always returns a channel. If no channel exists (yet or any more) for the given id, one is created. + * + * @param id + * The id for which to get the channel and reserve space. + */ + public ChannelWithPosition getChannelForWriteAndIncrementReferences(final AbstractID id, final int spaceToReserve) + throws IOException { + + ChannelWithPosition c = null; + do { + // the return value may be zero, if someone asynchronously decremented the counter to zero + // and caused the disposal of the channel. falling through the loop will create a + // new channel. + c = getChannelInternal(id, true).reserveWriteSpaceAndIncrementReferences(spaceToReserve); + } while (c == null); + + return c; + } + + /** + * Increments the references to the given channel. + * + * @param id + * The channel to increment the references for. + * @throws IllegalStateException + * Thrown, if the channel has not been registered or has already been removed. + */ + public void incrementReferences(final AbstractID id) { - return getReadableSpillingFile(ownerID, fileID).lockReadableFileChannel(); + ChannelWithAccessInfo entry = this.fileMap.get(id); + if (entry == null || !entry.incrementReferences()) { + throw new IllegalStateException("No channel is registered (any more) for the given id."); + } } - public void increaseBufferCounter(final AbstractID ownerID, final FileID fileID) throws IOException, - InterruptedException { + /** + * Decrements the references to the given channel. If the channel reaches zero references, it will be removed. + * + * @param id + * The channel to decrement the references for. + * @throws IllegalStateException + * Thrown, if the channel has not been registered or has already been removed. + */ + public void decrementReferences(final AbstractID id) { - getReadableSpillingFile(ownerID, fileID).increaseNumberOfBuffers(); + ChannelWithAccessInfo entry = this.fileMap.get(id); + if (entry != null) { + if (entry.decrementReferences() <= 0) { + this.fileMap.remove(id); + } + } else { + throw new IllegalStateException("Channel is not (or no longer) registered at the file buffer manager."); + } } - public void decreaseBufferCounter(final AbstractID ownerID, final FileID fileID) { + // -------------------------------------------------------------------------------------------- - try { - Map map = null; - synchronized (this.readableSpillingFileMap) { - map = this.readableSpillingFileMap.get(ownerID); - if (map == null) { - throw new IOException("Cannot find readable spilling file queue for owner ID " + ownerID); - } + private final ChannelWithAccessInfo getChannelInternal(final AbstractID id, final boolean createIfAbsent) + throws IOException { - ReadableSpillingFile readableSpillingFile = null; - synchronized (map) { - readableSpillingFile = map.get(fileID); - if (readableSpillingFile == null) { - throw new IOException("Cannot find readable spilling file for owner ID " + ownerID); - } + ChannelWithAccessInfo cwa = this.fileMap.get(id); + if (cwa == null) { + if (createIfAbsent) { - if (readableSpillingFile.checkForEndOfFile()) { - map.remove(fileID); - if (map.isEmpty()) { - this.readableSpillingFileMap.remove(ownerID); - } - } + // Construct the filename + final int dirIndex = Math.abs(id.hashCode()) % this.tmpDirs.length; + final File file = new File(this.tmpDirs[dirIndex] + id.toString()); + + cwa = new ChannelWithAccessInfo(file); + final ChannelWithAccessInfo alreadyContained = this.fileMap.putIfAbsent(id, cwa); + if (alreadyContained != null) { + // we had a race (should be a very rare event) and have created an + // unneeded channel. dispose it and use the already contained one. + cwa.disposeSilently(); + cwa = alreadyContained; } + } else { + return null; } - } catch (IOException ioe) { - LOG.error(StringUtils.stringifyException(ioe)); } + + return cwa; } - public void releaseFileChannelForReading(final AbstractID ownerID, final FileID fileID) { + // -------------------------------------------------------------------------------------------- - try { - getReadableSpillingFile(ownerID, fileID).unlockReadableFileChannel(); - } catch (Exception e) { - LOG.error(StringUtils.stringifyException(e)); - } - } + private static final class ChannelWithAccessInfo { - /** - * Locks and returns a file channel from a {@link WritableSpillingFile}. - * - * @param ownerID - * the ID of the owner the file channel shall be locked for - * @return the file channel object if the lock could be acquired or null if the locking operation - * failed - * @throws IOException - * thrown if no spilling for the given channel ID could be allocated - */ - public FileChannel getFileChannelForWriting(final AbstractID ownerID) throws IOException { + private final File file; - synchronized (this.writableSpillingFileMap) { + private final FileChannel channel; - WritableSpillingFile writableSpillingFile = this.writableSpillingFileMap.get(ownerID); - if (writableSpillingFile == null) { - final FileID fileID = new FileID(); - final String filename = this.tmpDir + File.separator + FILE_BUFFER_PREFIX + fileID; - writableSpillingFile = new WritableSpillingFile(fileID, new File(filename)); - this.writableSpillingFileMap.put(ownerID, writableSpillingFile); - } + private final AtomicLong reservedWritePosition; - return writableSpillingFile.lockWritableFileChannel(); - } - } + private final AtomicInteger referenceCounter; - /** - * Returns the lock for a file channel of a {@link WritableSpillingFile}. - * - * @param ownerID - * the ID of the owner the lock has been acquired for - * @param currentFileSize - * the size of the file after the last write operation using the locked file channel - * @throws IOException - * thrown if the lock could not be released - */ - public FileID reportEndOfWritePhase(final AbstractID ownerID, final long currentFileSize) throws IOException { + private ChannelWithAccessInfo(final File file) throws IOException { - WritableSpillingFile writableSpillingFile = null; - boolean removed = false; - synchronized (this.writableSpillingFileMap) { + this.file = file; + this.channel = new RandomAccessFile(file, "rw").getChannel(); + this.reservedWritePosition = new AtomicLong(0); + this.referenceCounter = new AtomicInteger(0); + } - writableSpillingFile = this.writableSpillingFileMap.get(ownerID); - if (writableSpillingFile == null) { - throw new IOException("Cannot find writable spilling file for owner ID " + ownerID); - } + FileChannel getChannel() { - writableSpillingFile.unlockWritableFileChannel(currentFileSize); + return this.channel; + } + + FileChannel getAndIncrementReferences() { - if (writableSpillingFile.isReadRequested() && writableSpillingFile.isSafeToClose()) { - this.writableSpillingFileMap.remove(ownerID); - removed = true; + if (incrementReferences()) { + return this.channel; + } else { + return null; } } - if (removed) { - writableSpillingFile.close(); - Map map = null; - synchronized (this.readableSpillingFileMap) { - map = this.readableSpillingFileMap.get(ownerID); - if (map == null) { - map = new HashMap(); - this.readableSpillingFileMap.put(ownerID, map); - } - } + ChannelWithPosition reserveWriteSpaceAndIncrementReferences(final int spaceToReserve) { - synchronized (map) { - map.put(writableSpillingFile.getFileID(), - writableSpillingFile.toReadableSpillingFile()); - map.notify(); + if (incrementReferences()) { + return new ChannelWithPosition(this.channel, this.reservedWritePosition.getAndAdd(spaceToReserve)); + } else { + return null; } } - return writableSpillingFile.getFileID(); - } + /** + * Decrements the number of references to this channel. If the number of references is zero after the + * decrement, the channel is deleted. + * + * @return The number of references remaining after the decrement. + * @throws IllegalStateException + * Thrown, if the number of references is already zero or below. + */ + int decrementReferences() { + + int current = this.referenceCounter.get(); + while (true) { + if (current <= 0) { + // this is actually an error case, because the channel was deleted before + throw new IllegalStateException("The references to the file were already at zero."); + } - public void registerExternalReadableSpillingFile(final AbstractID ownerID, final FileID fileID) throws IOException { + if (current == 1) { + // this call decrements to zero, so mark it as deleted + if (this.referenceCounter.compareAndSet(current, Integer.MIN_VALUE)) { + current = 0; + break; + } + } + else if (this.referenceCounter.compareAndSet(current, current - 1)) { + current = current - 1; + break; + } + current = this.referenceCounter.get(); + } - Map map = null; - synchronized (this.readableSpillingFileMap) { - map = this.readableSpillingFileMap.get(ownerID); - if (map == null) { - map = new HashMap(); - this.readableSpillingFileMap.put(ownerID, map); + if (current > 0) { + return current; + } + else if (current == 0) { + // delete the channel + this.referenceCounter.set(Integer.MIN_VALUE); + this.reservedWritePosition.set(Long.MIN_VALUE); + try { + this.channel.close(); + } catch (IOException ioex) { + if (FileBufferManager.LOG.isErrorEnabled()) + FileBufferManager.LOG.error("Error while closing spill file for file buffers: " + + ioex.getMessage(), ioex); + } + this.file.delete(); + return current; + } + else { + throw new IllegalStateException("The references to the file were already at zero."); } } - ReadableSpillingFile readableSpillingFile = null; - synchronized (map) { - readableSpillingFile = map.get(fileID); - if (readableSpillingFile == null) { - final String filename = this.tmpDir + File.separator + FILE_BUFFER_PREFIX + fileID; - readableSpillingFile = new ReadableSpillingFile(new File(filename), 1); // Use 1 here to make sure the - // file is not immediately - // deleted - map.put(fileID, readableSpillingFile); - map.notify(); + /** + * Increments the references to this channel. Returns true, if successful, and false, + * if the channel has been disposed in the meantime. + * + * @return True, if successful, false, if the channel has been disposed. + */ + boolean incrementReferences() { + + int current = this.referenceCounter.get(); + while (true) { + // check whether it was disposed in the meantime + if (current < 0) { + return false; + } + // atomically check and increment + if (this.referenceCounter.compareAndSet(current, current + 1)) { + return true; + } + current = this.referenceCounter.get(); } } - readableSpillingFile.increaseNumberOfBuffers(); + /** + * Disposes the channel without further notice. Tries to close it (swallowing all exceptions) and tries + * to delete the file. + */ + void disposeSilently() { - } - - public void forceCloseOfWritableSpillingFile(final AbstractID ownerID) throws IOException, InterruptedException { + this.referenceCounter.set(Integer.MIN_VALUE); + this.reservedWritePosition.set(Long.MIN_VALUE); - FileID fileID = null; - synchronized (this.writableSpillingFileMap) { - final WritableSpillingFile w = this.writableSpillingFileMap.get(ownerID); - if (w != null) { - fileID = w.getFileID(); + if (this.channel.isOpen()) { + try { + this.channel.close(); + } catch (Throwable t) { + } } - } - - if (fileID != null) { - getReadableSpillingFile(ownerID, fileID); + this.file.delete(); } } } \ No newline at end of file diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java index b204488471eab..90d0a6f38f94a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java @@ -1,17 +1,17 @@ /*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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. - * - **********************************************************************************************************************/ +* +* Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) +* +* 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 eu.stratosphere.nephele.io.channels; @@ -19,52 +19,51 @@ import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; -import java.util.concurrent.atomic.AtomicBoolean; -import eu.stratosphere.nephele.io.channels.InternalBuffer; - -public class MemoryBuffer implements InternalBuffer { +public class MemoryBuffer extends Buffer +{ private final MemoryBufferRecycler bufferRecycler; - + private final ByteBuffer byteBuffer; + + private volatile boolean writeMode = true; - private final AtomicBoolean writeMode = new AtomicBoolean(true); - - MemoryBuffer(final int bufferSize, final ByteBuffer byteBuffer, final MemoryBufferPoolConnector bufferPoolConnector) { - + + + MemoryBuffer(final int bufferSize, final ByteBuffer byteBuffer, final MemoryBufferPoolConnector bufferPoolConnector) + { if (bufferSize > byteBuffer.capacity()) { throw new IllegalArgumentException("Requested buffer size is " + bufferSize + ", but provided byte buffer only has a capacity of " + byteBuffer.capacity()); } - + this.bufferRecycler = new MemoryBufferRecycler(byteBuffer, bufferPoolConnector); - + this.byteBuffer = byteBuffer; this.byteBuffer.position(0); this.byteBuffer.limit(bufferSize); } - - private MemoryBuffer(final int bufferSize, final ByteBuffer byteBuffer, final MemoryBufferRecycler bufferRecycler) { - - this.bufferRecycler = bufferRecycler; - + + private MemoryBuffer(int bufferSize, ByteBuffer byteBuffer, MemoryBufferRecycler bufferRecycler) + { + this.bufferRecycler = bufferRecycler; this.byteBuffer = byteBuffer; this.byteBuffer.position(0); this.byteBuffer.limit(bufferSize); } @Override - public int read(final ByteBuffer dst) throws IOException { - - if (this.writeMode.get()) { - throw new IOException("Buffer is still in write mode!"); + public int read(ByteBuffer dst) throws IOException + { + if (this.writeMode) { + this.writeMode = false; + this.byteBuffer.flip(); } if (!this.byteBuffer.hasRemaining()) { return -1; } - if (!dst.hasRemaining()) { return 0; } @@ -80,14 +79,13 @@ public int read(final ByteBuffer dst) throws IOException { dst.put(this.byteBuffer); } - // System.out.println("Position is " + this.byteBuffer); return (this.byteBuffer.position() - oldPosition); } @Override - public int read(final WritableByteChannel writableByteChannel) throws IOException { - - if (this.writeMode.get()) { + public int read(WritableByteChannel writableByteChannel) throws IOException + { + if (this.writeMode) { throw new IOException("Buffer is still in write mode!"); } @@ -111,29 +109,34 @@ public boolean isOpen() { } @Override - public int write(final ByteBuffer src) throws IOException { - - if (!this.writeMode.get()) { + public int write(ByteBuffer src) throws IOException + { + if (!this.writeMode) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } - final int bytesToCopy = Math.min(src.remaining(), this.byteBuffer.remaining()); - - if (bytesToCopy == 0) { - return 0; + final int sourceRemaining = src.remaining(); + final int thisRemaining = this.byteBuffer.remaining(); + final int excess = sourceRemaining - thisRemaining; + + if (excess <= 0) { + // there is enough space here for all the source data + this.byteBuffer.put(src); + return sourceRemaining; + } else { + // not enough space here, we need to limit the source + final int oldLimit = src.limit(); + src.limit(src.position() + thisRemaining); + this.byteBuffer.put(src); + src.limit(oldLimit); + return thisRemaining; } - - this.byteBuffer.put(src.array(), src.position(), bytesToCopy); - src.position(src.position() + bytesToCopy); - // this.byteBuffer.position(this.byteBuffer.position() + bytesToCopy); - - return bytesToCopy; } @Override - public int write(final ReadableByteChannel readableByteChannel) throws IOException { - - if (!this.writeMode.get()) { + public int write(ReadableByteChannel readableByteChannel) throws IOException + { + if (!this.writeMode) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } @@ -145,8 +148,8 @@ public int write(final ReadableByteChannel readableByteChannel) throws IOExcepti } @Override - public int remaining() { - + public int remaining() + { return this.byteBuffer.remaining(); } @@ -160,54 +163,63 @@ public ByteBuffer getByteBuffer() { } @Override - public void recycleBuffer() { - + protected void recycle() + { this.bufferRecycler.decreaseReferenceCounter(); } @Override - public void finishWritePhase() { - - if (!this.writeMode.compareAndSet(true, false)) { - throw new IllegalStateException("MemoryBuffer is already in read mode!"); + public void finishWritePhase() + { + if (!this.writeMode) { + throw new IllegalStateException("MemoryBuffer is already in write mode!"); } this.byteBuffer.flip(); + this.writeMode = false; } @Override - public boolean isBackedByMemory() { - + public boolean isBackedByMemory() + { return true; } @Override - public InternalBuffer duplicate() { - - final MemoryBuffer duplicatedMemoryBuffer = new MemoryBuffer(this.byteBuffer.limit(), this.byteBuffer - .duplicate(), this.bufferRecycler); - + public MemoryBuffer duplicate() + { + if (this.writeMode) { + throw new IllegalStateException("Cannot duplicate buffer that is still in write mode"); + } + + final MemoryBuffer duplicatedMemoryBuffer = new MemoryBuffer(this.byteBuffer.limit(), + this.byteBuffer.duplicate(), this.bufferRecycler); + this.bufferRecycler.increaseReferenceCounter(); - - duplicatedMemoryBuffer.byteBuffer.position(this.byteBuffer.position()); - duplicatedMemoryBuffer.byteBuffer.limit(this.byteBuffer.limit()); - duplicatedMemoryBuffer.writeMode.set(this.writeMode.get()); - + duplicatedMemoryBuffer.writeMode = this.writeMode; return duplicatedMemoryBuffer; } @Override - public void copyToBuffer(final Buffer destinationBuffer) throws IOException { - + public void copyToBuffer(Buffer destinationBuffer) throws IOException + { + if (this.writeMode) { + throw new IllegalStateException("Cannot copy buffer that is still in write mode"); + } + if (size() > destinationBuffer.size()) { + throw new IllegalArgumentException("Destination buffer is too small to store content of source buffer: " + + size() + " vs. " + destinationBuffer.size()); + } + final int oldPos = this.byteBuffer.position(); this.byteBuffer.position(0); - + while (remaining() > 0) { - destinationBuffer.write(this); + destinationBuffer.write(this.byteBuffer); } - + this.byteBuffer.position(oldPos); - + destinationBuffer.finishWritePhase(); } @@ -215,8 +227,8 @@ public void copyToBuffer(final Buffer destinationBuffer) throws IOException { * {@inheritDoc} */ @Override - public boolean isInWriteMode() { - - return this.writeMode.get(); + public boolean isInWriteMode() + { + return this.writeMode; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ReadableSpillingFile.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ReadableSpillingFile.java deleted file mode 100644 index a2c77ce6bac33..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ReadableSpillingFile.java +++ /dev/null @@ -1,88 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.io.channels; - -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.nio.channels.FileChannel; - -public final class ReadableSpillingFile { - - private boolean readableChannelLocked = false; - - private final File physicalFile; - - private final FileChannel readableFileChannel; - - private int numberOfBuffers; - - public ReadableSpillingFile(final File physicalFile, int numberOfBuffers) throws IOException { - this.physicalFile = physicalFile; - this.numberOfBuffers = numberOfBuffers; - this.readableFileChannel = new FileInputStream(this.physicalFile).getChannel(); - } - - public synchronized boolean isReadableChannelLocked() { - - return this.readableChannelLocked; - } - - public File getPhysicalFile() { - return this.physicalFile; - } - - public synchronized FileChannel lockReadableFileChannel() { - - if (this.readableChannelLocked) { - return null; - } - - this.readableChannelLocked = true; - - return this.readableFileChannel; - } - - public synchronized void unlockReadableFileChannel() throws IOException { - - if (!this.readableChannelLocked) { - return; - } - - this.readableChannelLocked = false; - this.notify(); - } - - public synchronized boolean checkForEndOfFile() throws IOException { - - --this.numberOfBuffers; - - if (this.numberOfBuffers == 0) { - // Close the file - this.readableFileChannel.close(); - - this.physicalFile.delete(); - return true; - } - - return false; - } - - public synchronized void increaseNumberOfBuffers() { - - ++this.numberOfBuffers; - } -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/WritableSpillingFile.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/WritableSpillingFile.java deleted file mode 100644 index 118ba49dc0a88..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/WritableSpillingFile.java +++ /dev/null @@ -1,225 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.io.channels; - -import java.io.Closeable; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.channels.FileChannel; -import java.nio.channels.WritableByteChannel; - -import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; - -/** - * A writable spilling file is a temporary file that is created to store incoming data for - * {@link AbstractByteBufferedInputChannel} objects. For performance reasons one writable spilling file may contain data - * from multiple input channels that are associated with the same input gate. - *

- * This class is not thread-safe. Access from different threads must be protected by an external monitor. - * - * @author warneke - */ -public final class WritableSpillingFile implements Closeable { - - /** - * The maximum period of time that is waited until the file is closed after the last write request after a read - * request has been issued (given in milliseconds). - */ - public static final long MAXIMUM_TIME_WITHOUT_WRITE_ACCESS = 500; // 500 ms - - /** - * The minimum size the spilling has to grow to before it is closed after a read request has been issued (given in - * bytes). - */ - private static final long MINIMUM_FILE_SIZE = 4L * 1024L * 1024L; // 4 MB - - /** - * Indicates whether the {@link WritableByteChannel} is currently locked. - */ - private boolean writableChannelLocked = false; - - /** - * Indicates whether a read request has been issued for this file. - */ - private boolean readRequested = false; - - private final FileID fileID; - - /** - * The physical file which backs this spilling file. - */ - private final File physicalFile; - - /** - * The file channel used to write data to the file. - */ - private final FileChannel writableFileChannel; - - /** - * Time stamp of the last file channel unlock operation. - */ - private long lastUnlockTime = 0; - - /** - * The size of the spilling file after the last unlock operation. - */ - private long currentFileSize = 0; - - /** - * The number of file buffers backed by this writable spilling file. - */ - private int numberOfBuffers = 0; - - /** - * Constructs a new writable spilling file. - * - * @param fileID - * the ID of the file, must not be null - * @param physicalFile - * the physical file which shall back this object, must not be null - * @throws IOException - * thrown if the given file cannot be opened for writing - */ - WritableSpillingFile(final FileID fileID, final File physicalFile) throws IOException { - - if (fileID == null) { - throw new IllegalArgumentException("Argument file ID must not be null"); - } - - if (physicalFile == null) { - throw new IllegalArgumentException("Argument physical file must not be null"); - } - - this.fileID = fileID; - this.physicalFile = physicalFile; - this.writableFileChannel = new FileOutputStream(this.physicalFile).getChannel(); - } - - /** - * {@inheritDoc} - */ - @Override - public void close() throws IOException { - - if (!this.readRequested) { - throw new IOException("close called but no read access has been requested"); - } - - if (!this.writableFileChannel.isOpen()) { - throw new IOException("writable file channel is already closed"); - } - - this.writableFileChannel.close(); - } - - /** - * Returns this physical file which backs this object. - * - * @return the physical file which backs this object - */ - public File getPhysicalFile() { - return this.physicalFile; - } - - /** - * Checks if it is safe to close the file. - * - * @return true if it safe to close the file, false otherwise - */ - public boolean isSafeToClose() { - - if (this.writableChannelLocked) { - return false; - } - - if (this.currentFileSize >= MINIMUM_FILE_SIZE) { - return true; - } - - if ((System.currentTimeMillis() - this.lastUnlockTime) > MAXIMUM_TIME_WITHOUT_WRITE_ACCESS) { - return true; - } - - return false; - } - - /** - * Locks and returns this spilling file's {@link WritableByteChannel} for a write operation. - * - * @return the byte channel if the lock operation has been successful, or null if the lock operation - * failed - */ - FileChannel lockWritableFileChannel() { - - if (this.writableChannelLocked) { - return null; - } - - this.writableChannelLocked = true; - - return this.writableFileChannel; - } - - /** - * Releases the lock on the spilling file's {@link WritableByteChannel}. - * - * @param currentFileSize - * the current size of the spilling file in bytes - */ - void unlockWritableFileChannel(final long currentFileSize) { - - this.writableChannelLocked = false; - - this.currentFileSize = currentFileSize; - this.lastUnlockTime = System.currentTimeMillis(); - - ++this.numberOfBuffers; - } - - /** - * Checks if a read request has been issued for this spilling file. - * - * @return true if a read request has been issued, false otherwise - */ - boolean isReadRequested() { - - return this.readRequested; - } - - /** - * Issues a read request to this spilling file. - */ - void requestReadAccess() { - - this.readRequested = true; - } - - /** - * Returns the ID of the file which backs the file buffer. - * - * @return the ID of the file which backs the file buffer - */ - FileID getFileID() { - - return this.fileID; - } - - ReadableSpillingFile toReadableSpillingFile() throws IOException { - - return new ReadableSpillingFile(this.physicalFile, this.numberOfBuffers); - } -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/compression/AbstractCompressor.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/compression/AbstractCompressor.java index e4e853873dfe0..8580603adc088 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/compression/AbstractCompressor.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/compression/AbstractCompressor.java @@ -20,7 +20,6 @@ import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.InternalBuffer; import eu.stratosphere.nephele.io.channels.MemoryBuffer; public abstract class AbstractCompressor implements Compressor { @@ -73,12 +72,11 @@ public final Buffer getUncompresssedDataBuffer() { */ private ByteBuffer getInternalByteBuffer(Buffer buffer) { - final InternalBuffer internalBuffer = buffer.getInternalBuffer(); - if (!(internalBuffer instanceof MemoryBuffer)) { + if (!(buffer instanceof MemoryBuffer)) { throw new RuntimeException("Provided buffer is not a memory buffer and cannot be used for compression"); } - final MemoryBuffer memoryBuffer = (MemoryBuffer) internalBuffer; + final MemoryBuffer memoryBuffer = (MemoryBuffer) buffer; return memoryBuffer.getByteBuffer(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/compression/AbstractDecompressor.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/compression/AbstractDecompressor.java index e4619597a2da2..3df0f4427fcb8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/compression/AbstractDecompressor.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/compression/AbstractDecompressor.java @@ -20,7 +20,6 @@ import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.InternalBuffer; import eu.stratosphere.nephele.io.channels.MemoryBuffer; import eu.stratosphere.nephele.io.compression.Decompressor; @@ -111,12 +110,11 @@ public void setUncompressedDataBuffer(Buffer buffer) { */ protected ByteBuffer getInternalByteBuffer(Buffer buffer) { - final InternalBuffer internalBuffer = buffer.getInternalBuffer(); - if (!(internalBuffer instanceof MemoryBuffer)) { + if (!(buffer instanceof MemoryBuffer)) { throw new RuntimeException("Provided buffer is not a memory buffer and cannot be used for compression"); } - final MemoryBuffer memoryBuffer = (MemoryBuffer) internalBuffer; + final MemoryBuffer memoryBuffer = (MemoryBuffer) buffer; return memoryBuffer.getByteBuffer(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java index ded5c6613e44f..f6e4eff5aae1f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java @@ -20,12 +20,9 @@ import java.nio.channels.ReadableByteChannel; import eu.stratosphere.nephele.io.AbstractID; -import eu.stratosphere.nephele.io.DefaultRecordDeserializer; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.BufferFactory; -import eu.stratosphere.nephele.io.channels.DeserializationBuffer; import eu.stratosphere.nephele.io.channels.FileBufferManager; -import eu.stratosphere.nephele.io.channels.FileID; public final class CheckpointDeserializer extends AbstractDeserializer { @@ -35,11 +32,6 @@ public final class CheckpointDeserializer extends AbstractDeserializer { private final FileBufferManager fileBufferManager; - private FileID deserializedFileID = null; - - private DeserializationBuffer fileIDDeserializationBuffer = new DeserializationBuffer( - new DefaultRecordDeserializer(FileID.class), true); - public CheckpointDeserializer(final AbstractID ownerID) { this.ownerID = ownerID; this.fileBufferManager = FileBufferManager.getInstance(); @@ -50,16 +42,6 @@ protected boolean readBufferData(final ReadableByteChannel readableByteChannel) final ByteBuffer tempBuffer = getTempBuffer(); - if (this.deserializedFileID == null) { - this.deserializedFileID = this.fileIDDeserializationBuffer.readData(null, readableByteChannel); - if (this.deserializedFileID != null) { - tempBuffer.position(0); - tempBuffer.limit(8); - } else { - return true; - } - } - readableByteChannel.read(tempBuffer); if (tempBuffer.hasRemaining()) { return true; @@ -67,15 +49,11 @@ protected boolean readBufferData(final ReadableByteChannel readableByteChannel) final long offset = byteBufferToLong(tempBuffer); - final Buffer fileBuffer = BufferFactory.createFromCheckpoint(getSizeOfBuffer(), this.deserializedFileID, - offset, this.ownerID, this.fileBufferManager); + final Buffer fileBuffer = BufferFactory.createFromCheckpoint(getSizeOfBuffer(), offset, this.ownerID, + this.fileBufferManager); setBuffer(fileBuffer); - this.fileBufferManager.registerExternalReadableSpillingFile(this.ownerID, this.deserializedFileID); - - this.deserializedFileID = null; - return false; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointSerializer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointSerializer.java index ce707b6346b4c..56362a4918d9a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointSerializer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointSerializer.java @@ -19,11 +19,8 @@ import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; -import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.FileBuffer; -import eu.stratosphere.nephele.io.channels.InternalBuffer; -import eu.stratosphere.nephele.io.channels.SerializationBuffer; /** * A checkpoint serializer is a special implementation of a transfer envelope serializer. Unlike the @@ -39,39 +36,29 @@ public class CheckpointSerializer extends AbstractSerializer { private boolean bufferDataSerializationStarted = false; - private boolean fileIDSerialized = false; - /** * {@inheritDoc} */ @Override - protected boolean writeBufferData(final WritableByteChannel writableByteChannel, final Buffer buffer) - throws IOException { + protected boolean writeBufferData(final WritableByteChannel writableByteChannel, final Buffer buffer) throws IOException { final ByteBuffer tempBuffer = getTempBuffer(); if (!this.bufferDataSerializationStarted) { - final SerializationBuffer serializationBuffer = getSerializationBuffer(); - if (buffer == null) { throw new IllegalArgumentException("Argument buffer must not be null"); } - final InternalBuffer internalBuffer = buffer.getInternalBuffer(); - - if (internalBuffer.isInWriteMode()) { + if (buffer.isInWriteMode()) { throw new IllegalStateException("Buffer to be serialized is still in write mode"); } - if (!(internalBuffer instanceof FileBuffer)) { + if (!(buffer instanceof FileBuffer)) { throw new IllegalArgumentException("Provided buffer is not a file buffer"); } - final FileBuffer fileBuffer = (FileBuffer) internalBuffer; - - serializationBuffer.clear(); - serializationBuffer.serialize(fileBuffer.getFileID()); + final FileBuffer fileBuffer = (FileBuffer) buffer; tempBuffer.clear(); longToByteBuffer(fileBuffer.getOffset(), tempBuffer); @@ -79,22 +66,10 @@ protected boolean writeBufferData(final WritableByteChannel writableByteChannel, this.bufferDataSerializationStarted = true; } - if (!this.fileIDSerialized) { - - final SerializationBuffer serializationBuffer = getSerializationBuffer(); - - if (serializationBuffer.dataLeftFromPreviousSerialization()) { - serializationBuffer.read(writableByteChannel); - } else { - this.fileIDSerialized = true; - } + if (tempBuffer.hasRemaining()) { + writableByteChannel.write(tempBuffer); } else { - - if (tempBuffer.hasRemaining()) { - writableByteChannel.write(tempBuffer); - } else { - return false; - } + return false; } return true; @@ -109,8 +84,6 @@ public void reset() { super.reset(); this.bufferDataSerializationStarted = false; - this.fileIDSerialized = false; - } private void longToByteBuffer(long longToSerialize, ByteBuffer byteBuffer) throws IOException { From 894ed5fb8853dabade94ba2903ab17a1a594b72b Mon Sep 17 00:00:00 2001 From: marrus Date: Fri, 17 Feb 2012 12:00:42 +0100 Subject: [PATCH 246/310] changed some logging for parsing --- .../main/java/eu/stratosphere/nephele/io/InputGate.java | 2 ++ .../main/java/eu/stratosphere/nephele/io/OutputGate.java | 5 +++++ .../checkpointing/CheckpointDecisionCoordinator.java | 2 ++ .../nephele/execution/ExecutionStateTransition.java | 2 +- .../eu/stratosphere/nephele/io/RuntimeInputGate.java | 9 +++++++-- .../eu/stratosphere/nephele/io/RuntimeOutputGate.java | 9 +++++++++ .../eu/stratosphere/nephele/jobmanager/JobManager.java | 2 +- .../nephele/jobmanager/scheduler/RecoveryLogic.java | 1 + .../bytebuffered/OutgoingConnectionThread.java | 2 +- stratosphere-dist/pom.xml | 5 ----- 10 files changed, 29 insertions(+), 10 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java index 2c3d2f2647f1f..9f8e37c5c781d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/InputGate.java @@ -193,4 +193,6 @@ InMemoryInputChannel createInMemoryInputChannel(InputGate inputGate, Chann */ public long getExecutionStart(); + int getNumRecords(); + } diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java index ca1c1ba49a27d..ee3864c5468d9 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/io/OutputGate.java @@ -187,4 +187,9 @@ InMemoryOutputChannel createInMemoryOutputChannel(OutputGate outputGate, C * the ID of the output channel which has forwarded its buffer */ void outputBufferSent(ChannelID channelID); + /** + * Returns the number of emitted records for this gate. + * + */ + int getNumRecords(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index 169f71ae420bf..a780b4c053784 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -99,6 +99,8 @@ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtil synchronized (decidedVertices) { if (!decidedVertices.contains(vertex.getID())) { boolean checkpointDecision = getDecision(vertex, rus); + if(checkpointDecision) + LOG.info("Creating Checkpoint for " + vertex.getName() ); final ExecutionGraph graph = vertex.getExecutionGraph(); final Map> checkpointDecisions = new HashMap>(); List checkpointDecisionList = null; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java index 486fe78e205d0..494cbeb16d193 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java @@ -40,7 +40,7 @@ public static void checkTransition(final boolean jobManager, final String taskNa final ExecutionState newState) { LOG.info((jobManager ? "JM: " : "TM: ") + "ExecutionState set from " + oldState + " to " + newState - + " for task " + taskName); + + " for task " + taskName+ " at " + System.currentTimeMillis()); boolean unexpectedStateChange = true; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java index 230431b0d0713..11bad0ea4aef5 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java @@ -108,7 +108,7 @@ public class RuntimeInputGate extends AbstractGate implemen */ private long executionstart = -1; - + private int numrecords = 0; /** * Constructs a new runtime input gate. * @@ -347,7 +347,7 @@ record = this.getInputChannel(this.channelToReadFrom).readRecord(target); this.channelToReadFrom = -1; } } - + this.numrecords++; return record; } @@ -556,4 +556,9 @@ public void notifyDataUnitConsumed(final int channelIndex) { this.channelToReadFrom = -1; } + + @Override + public int getNumRecords() { + return this.numrecords; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java index 35ed0417b6295..80e24d1c13c7e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java @@ -82,6 +82,11 @@ public class RuntimeOutputGate extends AbstractGate impleme * Stores whether all records passed to this output gate shall be transmitted through all connected output channels. */ private final boolean isBroadcast; + + /** + * Stores the number of emitted records + */ + private int numrecords = 0; /** * Constructs a new runtime output gate. @@ -367,6 +372,7 @@ public void writeRecord(final T record) throws IOException, InterruptedException } } } + this.numrecords++; } /** @@ -480,4 +486,7 @@ public void outputBufferSent(final ChannelID channelID) { // Nothing to do here } + public int getNumRecords(){ + return this.numrecords; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index d24be8283e3bd..81f76bf5b143e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -1064,7 +1064,7 @@ public void jobStatusHasChanged(final ExecutionGraph executionGraph, final Inter final String optionalMessage) { LOG.info("Status of job " + executionGraph.getJobName() + "(" + executionGraph.getJobID() + ")" - + " changed to " + newJobStatus); + + " changed to " + newJobStatus + " at " + System.currentTimeMillis()); if (newJobStatus == InternalJobStatus.FAILING) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index 2b8a59cb7649f..7fa8b92c5f077 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -127,6 +127,7 @@ public static boolean recover(final ExecutionVertex failedVertex, failedVertex.updateExecutionState(getStateToUpdate(failedVertex)); } + LOG.info("Recovery FINISHED at " + System.currentTimeMillis()); return true; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java index 1cf0db738dbfe..8da7c208328f8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java @@ -36,7 +36,7 @@ public class OutgoingConnectionThread extends Thread { /** * The minimum time a TCP connection must be idle it is closed. */ - private static final long MIN_IDLE_TIME_BEFORE_CLOSE = 60000L; // 60 seconds + private static final long MIN_IDLE_TIME_BEFORE_CLOSE = 80000L; // 80 seconds private static final Log LOG = LogFactory.getLog(OutgoingConnectionThread.class); diff --git a/stratosphere-dist/pom.xml b/stratosphere-dist/pom.xml index 856ea9a97b200..b20ee22147067 100644 --- a/stratosphere-dist/pom.xml +++ b/stratosphere-dist/pom.xml @@ -72,11 +72,6 @@ nephele-visualization ${version} - - eu.stratosphere - nephele-streaming - ${version} - eu.stratosphere pact-common From e94fd90ee8de2294b788794a8b56d4cda5e15e30 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 17 Feb 2012 22:19:23 +0100 Subject: [PATCH 247/310] Temporarily added code to measure data input/output of PACT first-order function on byte level --- .../nephele/execution/Environment.java | 4 + .../nephele/execution/ExecutionObserver.java | 4 + .../checkpointing/CheckpointEnvironment.java | 8 + .../nephele/checkpointing/ReplayTask.java | 11 ++ .../nephele/execution/RuntimeEnvironment.java | 18 ++ .../taskmanager/runtime/RuntimeTask.java | 27 +++ .../pact/common/type/PactRecord.java | 13 ++ .../hash/BuildFirstHashMatchIterator.java | 53 ++++++ .../hash/BuildSecondHashMatchIterator.java | 55 ++++++ .../runtime/sort/SortMergeMatchIterator.java | 179 +++++++++++++++++- .../pact/runtime/task/MapTask.java | 21 ++ .../pact/runtime/task/ReduceTask.java | 11 ++ .../pact/runtime/task/SelfMatchTask.java | 150 ++++++++++++++- .../runtime/task/chaining/ChainedMapTask.java | 20 ++ .../runtime/task/util/OutputCollector.java | 16 ++ .../pact/runtime/util/KeyGroupedIterator.java | 17 ++ 16 files changed, 596 insertions(+), 11 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java index 1218c69f93c6d..3477618414c27 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java @@ -92,6 +92,10 @@ public interface Environment { * the user thread which has finished */ void userThreadFinished(Thread userThread); + + // DW: Start of temporary code + void reportPACTDataStatistics(final long numberOfConsumedBytes, final long numberOfProducedBytes); + // DW: End of temporary code /** * Returns the input split provider assigned to this environment. diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionObserver.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionObserver.java index 9312d0e44ddb5..b100f3870a555 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionObserver.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/ExecutionObserver.java @@ -49,4 +49,8 @@ public interface ExecutionObserver { * @return true if the task has been canceled, false otherwise */ boolean isCanceled(); + + // DW: Start of temporary code + void reportPACTDataStatistics(long numberOfConsumedBytes, long numberOfProducedBytes); + // DW: End of temporary code } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java index 68722f0b52f4b..cbdc22628c1f7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java @@ -302,4 +302,12 @@ public ReplayThread getExecutingThread() { return this.executingThread; } } + + // DW: Start of temporary code + @Override + public void reportPACTDataStatistics(final long numberOfConsumedBytes, final long numberOfProducedBytes) { + + throw new IllegalStateException("reportPACTDataStatistics called on CheckpointEnvironment"); + } + // DW: End of temporary code } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index e9bb92dd66ce4..30cee1ad8895e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -120,6 +120,17 @@ public boolean isCanceled() { return isCanceled; } + + // DW: Start of temporary code + @Override + public void reportPACTDataStatistics(final long numberOfConsumedBytes, final long numberOfProducedBytes) { + + if (this.encapsulatedRuntimeTask != null) { + this.encapsulatedRuntimeTask.reportPACTDataStatistics(numberOfConsumedBytes, numberOfProducedBytes); + } + + } + // DW: End of temporary code } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 8b6ea40129e96..a5366b6903908 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -1148,4 +1148,22 @@ public Set getInputChannelIDsOfGate(final GateID gateID) { return Collections.unmodifiableSet(inputChannelIDs); } + + // DW: Start of temporary code + @Override + public void reportPACTDataStatistics(final long numberOfConsumedBytes, final long numberOfProducedBytes) { + + if(numberOfConsumedBytes < 0L) { + return; + } + + if(numberOfProducedBytes < 0L) { + return; + } + + if (this.executionObserver != null) { + this.executionObserver.reportPACTDataStatistics(numberOfConsumedBytes, numberOfProducedBytes); + } + } + // DW: End of temporary code } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index d94c9a3f0071d..bbddb38a6b7b7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -87,6 +87,12 @@ public final class RuntimeTask implements Task, ExecutionObserver { private long startTime; + // DW: Start of temporary code + private double pactInputOutputRatioSum = 0.0; + + private int numberOfPactInputOutputRatioEntries = 0; + // DW: End of temporay code + public RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, final TaskManager taskManager) { @@ -298,6 +304,8 @@ public boolean isCanceled() { */ public void initialExecutionResourcesExhausted() { + System.out.println("PACT input/output for task " + this.environment.getTaskNameWithIndex() + ": " + getPACTInputOutputRatio()); + // if (this.environment.getExecutingThread() != Thread.currentThread()) { // throw new ConcurrentModificationException( // "initialExecutionResourcesExhausted must be called from the task that executes the user code"); @@ -514,4 +522,23 @@ public ExecutionState getExecutionState() { return this.executionState; } + + // DW: Start of temporary code + @Override + public void reportPACTDataStatistics(final long numberOfConsumedBytes, final long numberOfProducedBytes) { + + this.pactInputOutputRatioSum += ((double) numberOfProducedBytes / (double) numberOfConsumedBytes); + ++this.numberOfPactInputOutputRatioEntries; + } + + private double getPACTInputOutputRatio() { + + if(this.numberOfPactInputOutputRatioEntries == 0) { + return -1.0; + } + + return (this.pactInputOutputRatioSum / (double) this.numberOfPactInputOutputRatioEntries); + } + // DW: End of temporary code + } diff --git a/pact/pact-common/src/main/java/eu/stratosphere/pact/common/type/PactRecord.java b/pact/pact-common/src/main/java/eu/stratosphere/pact/common/type/PactRecord.java index e4c345ef9982c..5f29eccc374b1 100644 --- a/pact/pact-common/src/main/java/eu/stratosphere/pact/common/type/PactRecord.java +++ b/pact/pact-common/src/main/java/eu/stratosphere/pact/common/type/PactRecord.java @@ -1356,6 +1356,19 @@ public void deserialize(List sources, int segmentNum, int segment initFields(this.binaryData, 0, this.binaryLen); } + // DW: Start of temporary code + /** + * Returns the length of this record's binary representation in bytes. + * + * @return the length of this record's binary representation in bytes + */ + public int getBinaryLength() + { + updateBinaryRepresenation(); + return this.binaryLen; + } + // DW: End of temporary code + // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java index 67d2fc4405158..82d5d2a3bae7f 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java @@ -18,6 +18,7 @@ import java.io.IOException; import java.util.List; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryAllocationException; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; @@ -29,6 +30,7 @@ import eu.stratosphere.pact.common.type.PactRecord; import eu.stratosphere.pact.common.util.MutableObjectIterator; import eu.stratosphere.pact.runtime.task.util.MatchTaskIterator; +import eu.stratosphere.pact.runtime.task.util.OutputCollector; /** @@ -48,6 +50,10 @@ public final class BuildFirstHashMatchIterator implements MatchTaskIterator private final MemoryManager memManager; + // DW: Start of temporary code + private final Environment environment; + // DW: End of temporary code + private final HashJoin hashJoin; private PactRecord nextBuildSideObject; @@ -68,6 +74,10 @@ public BuildFirstHashMatchIterator(MutableObjectIterator firstInput, this.hashJoin = getHashJoin(firstInput, secondInput, buildSideKeyFields, probeSideKeyFields, keyClasses, memManager, ioManager, ownerTask, totalMemory); + + // DW: Start of temporary code + this.environment = ownerTask.getEnvironment(); + // DW: End of temporary code } // -------------------------------------------------------------------------------------------- @@ -98,6 +108,10 @@ public void close() public boolean callWithNextKey(MatchStub matchFunction, Collector collector) throws Exception { + // DW: Start of temporary code + final OutputCollector oc = (OutputCollector) collector; + // DW: End of temporary code + if (this.hashJoin.nextRecord()) { // we have a next record, get the iterators to the probe and build side values @@ -114,20 +128,49 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) // more than one build-side value --> copy the probe side probeRecord.copyTo(this.probeCopy); + // DW : Start of temporary code + long r1 = nextBuildSidePair.getBinaryLength(); + long r2 = probeRecord.getBinaryLength(); + // DW: End of temporary code + // call match on the first pair matchFunction.match(nextBuildSidePair, probeRecord, collector); + // DW: Start of temporary code + this.environment.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code // call match on the second pair probeRecord = new PactRecord(); this.probeCopy.copyTo(probeRecord); + + // DW : Start of temporary code + r1 = tmpPair.getBinaryLength(); + r2 = probeRecord.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(tmpPair, probeRecord, collector); + // DW: Start of temporary code + this.environment.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code tmpPair = new PactRecord(); while (this.running && buildSideIterator.next(tmpPair)) { // call match on the next pair probeRecord = new PactRecord(); this.probeCopy.copyTo(probeRecord); + + // DW : Start of temporary code + r1 = tmpPair.getBinaryLength(); + r2 = probeRecord.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(tmpPair, probeRecord, collector); + // DW: Start of temporary code + this.environment.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code tmpPair = new PactRecord(); } this.nextBuildSideObject = tmpPair; @@ -135,7 +178,17 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) else { // only single pair matches this.nextBuildSideObject = tmpPair; + + // DW : Start of temporary code + final long r1 = nextBuildSidePair.getBinaryLength(); + final long r2 = probeRecord.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(nextBuildSidePair, probeRecord, collector); + // DW: Start of temporary code + this.environment.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code } } return true; diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java index 894dc960547d3..572c2b2e04a64 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java @@ -18,6 +18,7 @@ import java.io.IOException; import java.util.List; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryAllocationException; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; @@ -29,6 +30,7 @@ import eu.stratosphere.pact.common.type.PactRecord; import eu.stratosphere.pact.common.util.MutableObjectIterator; import eu.stratosphere.pact.runtime.task.util.MatchTaskIterator; +import eu.stratosphere.pact.runtime.task.util.OutputCollector; /** @@ -47,6 +49,10 @@ public final class BuildSecondHashMatchIterator implements MatchTaskIterator private PactRecord probeCopy = new PactRecord(); + // DW: Start of temporary code + private final Environment environment; + // DW: End of temporary code + private volatile boolean running = true; // -------------------------------------------------------------------------------------------- @@ -61,6 +67,10 @@ public BuildSecondHashMatchIterator(MutableObjectIterator firstInput this.hashJoin = BuildFirstHashMatchIterator.getHashJoin(secondInput, firstInput, buildSideKeyFields, probeSideKeyFields, keyClasses, memManager, ioManager, ownerTask, totalMemory); + + // DW: Start of temporary code + this.environment = ownerTask.getEnvironment(); + // DW: End of temporary code } // -------------------------------------------------------------------------------------------- @@ -91,6 +101,12 @@ public void close() public boolean callWithNextKey(MatchStub matchFunction, Collector collector) throws Exception { + // DW: Start of temporary code + final Environment env = this.environment; + final OutputCollector oc = (OutputCollector) collector; + // DW: End of temporary code + + if (this.hashJoin.nextRecord()) { // we have a next record, get the iterators to the probe and build side values @@ -107,20 +123,49 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) // more than one build-side value --> copy the probe side probeRecord.copyTo(this.probeCopy); + // DW : Start of temporary code + long r1 = probeRecord.getBinaryLength(); + long r2 = nextBuildSidePair.getBinaryLength(); + // DW: End of temporary code + // call match on the first pair matchFunction.match(probeRecord, nextBuildSidePair, collector); + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code // call match on the second pair probeRecord = new PactRecord(); this.probeCopy.copyTo(probeRecord); + + // DW : Start of temporary code + r1 = probeRecord.getBinaryLength(); + r2 = tmpPair.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(probeRecord, tmpPair, collector); + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code tmpPair = new PactRecord(); while (this.running && buildSideIterator.next(tmpPair)) { // call match on the next pair probeRecord = new PactRecord(); this.probeCopy.copyTo(probeRecord); + + // DW : Start of temporary code + r1 = probeRecord.getBinaryLength(); + r2 = tmpPair.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(probeRecord, tmpPair, collector); + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code tmpPair = new PactRecord(); } this.nextBuildSideObject = tmpPair; @@ -128,7 +173,17 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) else { // only single pair matches this.nextBuildSideObject = tmpPair; + + // DW : Start of temporary code + final long r1 = probeRecord.getBinaryLength(); + final long r2 = nextBuildSidePair.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(probeRecord, nextBuildSidePair, collector); + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code } } return true; diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIterator.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIterator.java index fa03651511a9c..6cf08c93df76a 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIterator.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/SortMergeMatchIterator.java @@ -22,6 +22,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryAllocationException; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; @@ -35,6 +36,7 @@ import eu.stratosphere.pact.runtime.resettable.BlockResettableMutableObjectIterator; import eu.stratosphere.pact.runtime.resettable.SpillingResettableIterator; import eu.stratosphere.pact.runtime.task.util.MatchTaskIterator; +import eu.stratosphere.pact.runtime.task.util.OutputCollector; import eu.stratosphere.pact.runtime.task.util.TaskConfig.LocalStrategy; import eu.stratosphere.pact.runtime.util.KeyComparator; import eu.stratosphere.pact.runtime.util.KeyGroupedIterator; @@ -96,6 +98,9 @@ public class SortMergeMatchIterator implements MatchTaskIterator private final float spillingThreshold; + // DW: Start of temporary code + private final Environment environment; + // DW: End of temporary code private SortMerger sortMerger1; @@ -147,6 +152,10 @@ public SortMergeMatchIterator(MemoryManager memoryManager, IOManager ioManager, this.blockIt = new BlockResettableIterator(this.memoryManager, memoryForBlockNestedLoops - SpillingResettableIterator.MIN_TOTAL_MEMORY, 1, parentTask); + + // DW: Start of temporary code + this.environment = parentTask.getEnvironment(); + // DW: End of temporary code } /* (non-Javadoc) @@ -275,6 +284,11 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) Key[] keys1 = this.iterator1.getKeys(); Key[] keys2 = this.iterator2.getKeys(); + // DW: Start of temporary code + final Environment env = this.environment; + final OutputCollector oc = (OutputCollector) collector; + // DW: End of temporary code + // zig zag while (true) { // determine the relation between the (possibly composite) keys @@ -324,7 +338,17 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) // then we can derive the local strategy (with build side). if (!v1HasNext && !v2HasNext) { // both sides contain only one value + + // DW : Start of temporary code + final long r1 = firstV1.getBinaryLength(); + final long r2 = firstV2.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(firstV1, firstV2, collector); + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code } else if (!v1HasNext) { crossFirst1withNValues(firstV1, firstV2, values2, matchFunction, collector); @@ -364,9 +388,25 @@ private void crossFirst1withNValues(PactRecord val1, PactRecord firstValN, Iterator valsN, MatchStub matchFunction, Collector collector) throws Exception { + // DW: Start of temporary code + final Environment env = this.environment; + final OutputCollector oc = (OutputCollector) collector; + // DW: End of temporary code + val1.copyTo(this.copy1); + + // DW : Start of temporary code + long r1 = val1.getBinaryLength(); + long r2 = firstValN.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(val1, firstValN, collector); + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code + // set copy and match first element boolean more = true; do { @@ -374,9 +414,33 @@ private void crossFirst1withNValues(PactRecord val1, PactRecord firstValN, if (valsN.hasNext()) { this.copy1.copyToIfModified(val1); + + // DW : Start of temporary code + r1 = val1.getBinaryLength(); + r2 = nRec.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(val1, nRec, collector); + + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code + } else { + + // DW : Start of temporary code + r1 = this.copy1.getBinaryLength(); + r2 = nRec.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(this.copy1, nRec, collector); + + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code + more = false; } } @@ -398,8 +462,24 @@ private void crossSecond1withNValues(PactRecord val1, PactRecord firstValN, throws Exception { val1.copyTo(this.copy1); + + // DW: Start of temporary code + final Environment env = this.environment; + final OutputCollector oc = (OutputCollector) collector; + // DW: End of temporary code + + // DW : Start of temporary code + long r1 = firstValN.getBinaryLength(); + long r2 = val1.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(firstValN, val1, collector); + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code + // set copy and match first element boolean more = true; do { @@ -407,9 +487,33 @@ private void crossSecond1withNValues(PactRecord val1, PactRecord firstValN, if (valsN.hasNext()) { this.copy1.copyToIfModified(val1); + + // DW : Start of temporary code + r1 = nRec.getBinaryLength(); + r2 = val1.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(nRec, val1, collector); + + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code + } else { + + // DW : Start of temporary code + r1 = nRec.getBinaryLength(); + r2 = this.copy1.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(nRec, this.copy1, collector); + + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code + more = false; } } @@ -447,9 +551,24 @@ private void crossMwithNValues(final PactRecord firstV1, Iterator sp firstV1.copyTo(this.copy1); firstV2.copyTo(this.copy2); + // DW: Start of temporary code + final Environment env = this.environment; + final OutputCollector oc = (OutputCollector) collector; + // DW: End of temporary code + + // DW: Start of temporary code + long r1 = firstV1.getBinaryLength(); + long r2 = firstV2.getBinaryLength(); + // DW: End of temporary code + // --------------- 1) Cross the heads ------------------- matchFunction.match(firstV1, firstV2, collector); + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code + // for the remaining values, we do a block-nested-loops join SpillingResettableIterator spillIt = null; @@ -461,7 +580,18 @@ private void crossMwithNValues(final PactRecord firstV1, Iterator sp while (blockIt.hasNext()) { PactRecord nextBlockRec = blockIt.next(); this.copy1.copyTo(this.instance); + + // DW: Start of temporary code + r1 = this.instance.getBinaryLength(); + r2 = nextBlockRec.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(this.instance, nextBlockRec, collector); + + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code } blockIt.reset(); @@ -489,15 +619,37 @@ private void crossMwithNValues(final PactRecord firstV1, Iterator sp // -------- 3) cross the iterator of the spilling side with the head of the block side -------- this.copy2.copyTo(this.instance); + + // DW: Start of temporary code + r1 = nextSpillVal.getBinaryLength(); + r2 = this.instance.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(nextSpillVal, this.instance, collector); + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code + // -------- 4) cross the iterator of the spilling side with the first block -------- while (blockIt.hasNext()) { PactRecord nextBlockRec = blockIt.next(); // get instances of key and block value nextSpillVal = repeatableIter.repeatLast(); - matchFunction.match(nextSpillVal, nextBlockRec, collector); + + // DW: Start of temporary code + r1 = nextSpillVal.getBinaryLength(); + r2 = nextBlockRec.getBinaryLength(); + // DW: End of temporary code + + matchFunction.match(nextSpillVal, nextBlockRec, collector); + + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code } // reset block iterator blockIt.reset(); @@ -520,7 +672,18 @@ private void crossMwithNValues(final PactRecord firstV1, Iterator sp while (blockIt.hasNext()) { this.copy1.copyTo(this.instance); final PactRecord nextBlockVal = blockIt.next(); + + // DW: Start of temporary code + r1 = this.instance.getBinaryLength(); + r2 = nextBlockVal.getBinaryLength(); + // DW: End of temporary code + matchFunction.match(this.instance, nextBlockVal, collector); + + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code } blockIt.reset(); @@ -533,7 +696,19 @@ private void crossMwithNValues(final PactRecord firstV1, Iterator sp while (blockIt.hasNext()) { // get instances of key and block value final PactRecord nextBlockVal = blockIt.next(); - matchFunction.match(nextSpillVal, nextBlockVal, collector); + + // DW: Start of temporary code + r1 = nextSpillVal.getBinaryLength(); + r2 = nextBlockVal.getBinaryLength(); + // DW: End of temporary code + + matchFunction.match(nextSpillVal, nextBlockVal, collector); + + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code + // get new instance of resettable value if (blockIt.hasNext()) nextSpillVal = spillIt.repeatLast(); diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MapTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MapTask.java index 80a2c59e47ed0..209a448c7e42b 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MapTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MapTask.java @@ -15,10 +15,12 @@ package eu.stratosphere.pact.runtime.task; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.pact.common.stubs.Collector; import eu.stratosphere.pact.common.stubs.MapStub; import eu.stratosphere.pact.common.type.PactRecord; import eu.stratosphere.pact.common.util.MutableObjectIterator; +import eu.stratosphere.pact.runtime.task.util.OutputCollector; /** * Map task which is executed by a Nephele task manager. The task has a single @@ -72,8 +74,27 @@ public void run() throws Exception final PactRecord record = new PactRecord(); + // DW: Start to temporary code + int count = 0; + long consumedPactRecordsInBytes = 0L; + final Environment env = getEnvironment(); + final OutputCollector oc = (OutputCollector) output; + // DW: End of temporary code + while (this.running && input.next(record)) { + // DW: Start to temporary code + consumedPactRecordsInBytes =+ record.getBinaryLength(); + // DW: End of temporary code stub.map(record, output); + + // DW: Start to temporary code + if(++count == 10) { + env.reportPACTDataStatistics(consumedPactRecordsInBytes, + oc.getCollectedPactRecordsInBytes()); + consumedPactRecordsInBytes = 0L; + count = 0; + } + // DW: End of temporary code } } diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/ReduceTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/ReduceTask.java index 6e314480d68a8..2c2eba3854bac 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/ReduceTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/ReduceTask.java @@ -17,6 +17,7 @@ import java.util.Comparator; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; import eu.stratosphere.pact.common.stubs.Collector; @@ -26,6 +27,7 @@ import eu.stratosphere.pact.runtime.sort.CombiningUnilateralSortMerger; import eu.stratosphere.pact.runtime.sort.UnilateralSortMerger; import eu.stratosphere.pact.runtime.task.util.CloseableInputProvider; +import eu.stratosphere.pact.runtime.task.util.OutputCollector; import eu.stratosphere.pact.runtime.task.util.SimpleCloseableInputProvider; import eu.stratosphere.pact.runtime.task.util.TaskConfig.LocalStrategy; import eu.stratosphere.pact.runtime.util.KeyComparator; @@ -74,10 +76,19 @@ public void run() throws Exception final ReduceStub stub = this.stub; final Collector output = this.output; + // DW: Start of temporary code + final OutputCollector oc = (OutputCollector) output; + final Environment env = getEnvironment(); + // DW: End of temporary code + // run stub implementation while (this.running && iter.nextKey()) { stub.reduce(iter.getValues(), output); + // DW: Start of temporary code + env.reportPACTDataStatistics(iter.getConsumedPactRecordsInBytes(), + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code } } diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/SelfMatchTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/SelfMatchTask.java index cf3137edb7896..14b49dcc5bb47 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/SelfMatchTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/SelfMatchTask.java @@ -19,6 +19,7 @@ import java.util.Comparator; import java.util.Iterator; +import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager; import eu.stratosphere.pact.common.stubs.Collector; @@ -29,6 +30,7 @@ import eu.stratosphere.pact.runtime.resettable.SpillingResettableMutableObjectIterator; import eu.stratosphere.pact.runtime.sort.UnilateralSortMerger; import eu.stratosphere.pact.runtime.task.util.CloseableInputProvider; +import eu.stratosphere.pact.runtime.task.util.OutputCollector; import eu.stratosphere.pact.runtime.task.util.SimpleCloseableInputProvider; import eu.stratosphere.pact.runtime.task.util.TaskConfig.LocalStrategy; import eu.stratosphere.pact.runtime.util.KeyComparator; @@ -270,6 +272,11 @@ private final void fullCross(final Iterator values, final Collector // allocate buffer final PactRecord[] valBuffer = new PactRecord[VALUE_BUFFER_SIZE]; + // DW: Start of temporary code + final Environment env = getEnvironment(); + final OutputCollector oc = (OutputCollector) out; + // DW: End of temporary code + // fill value buffer for the first time int bufferValCnt; for(bufferValCnt = 0; bufferValCnt < VALUE_BUFFER_SIZE; bufferValCnt++) { @@ -292,7 +299,20 @@ private final void fullCross(final Iterator values, final Collector if (!this.running) return; // match - stub.match(valBuffer[i].createCopy(), valBuffer[j].createCopy(), out); + + // DW : Start of temporary code + final PactRecord copy1 = valBuffer[i].createCopy(); + final PactRecord copy2 = valBuffer[j].createCopy(); + final long r1 = copy1.getBinaryLength(); + final long r2 = copy2.getBinaryLength(); + // DW: End of temporary code + + stub.match(copy1, copy2, out); + + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code } } @@ -311,7 +331,21 @@ public boolean next(PactRecord target) throws IOException { for(int i=0;i values, fina // allocate buffer final PactRecord[] valBuffer = new PactRecord[VALUE_BUFFER_SIZE]; + // DW: Start of temporary code + final Environment env = getEnvironment(); + final OutputCollector oc = (OutputCollector) out; + // DW: End of temporary code + // fill value buffer for the first time int bufferValCnt; for(bufferValCnt = 0; bufferValCnt < VALUE_BUFFER_SIZE; bufferValCnt++) { @@ -426,7 +476,20 @@ private final void diagInclTriangleCross(final Iterator values, fina if (!this.running) return; // match - stub.match(valBuffer[i].createCopy(), valBuffer[j].createCopy(), out); + + // DW : Start of temporary code + final PactRecord copy1 = valBuffer[i].createCopy(); + final PactRecord copy2 = valBuffer[j].createCopy(); + final long r1 = copy1.getBinaryLength(); + final long r2 = copy2.getBinaryLength(); + // DW: End of temporary code + + stub.match(copy1, copy2, out); + + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code } } @@ -445,7 +508,21 @@ public boolean next(PactRecord target) throws IOException { for(int i=0;i values, fina // allocate buffer final PactRecord[] valBuffer = new PactRecord[VALUE_BUFFER_SIZE]; + // DW: Start of temporary code + final Environment env = getEnvironment(); + final OutputCollector oc = (OutputCollector) out; + // DW: End of temporary code + // fill value buffer for the first time int bufferValCnt; for(bufferValCnt = 0; bufferValCnt < VALUE_BUFFER_SIZE; bufferValCnt++) { @@ -567,7 +659,22 @@ private final void diagExclTriangleCross(final Iterator values, fina if (!this.running) return; // match - stub.match(valBuffer[i].createCopy(), valBuffer[j].createCopy(), out); + + // DW : Start of temporary code + final PactRecord copy1 = valBuffer[i].createCopy(); + final PactRecord copy2 = valBuffer[j].createCopy(); + final long r1 = copy1.getBinaryLength(); + final long r2 = copy2.getBinaryLength(); + // DW: End of temporary code + + stub.match(copy1, copy2, out); + + // DW: Start of temporary code + env.reportPACTDataStatistics(r1 + r2, + oc.getCollectedPactRecordsInBytes()); + // DW: End of temporary code + + } } @@ -586,7 +693,21 @@ public boolean next(PactRecord target) throws IOException { for(int i=0;i writer) } } + // DW: Start of temporary code + private long collectedPactRecordsInBytes = 0L; + + public long getCollectedPactRecordsInBytes() { + + final long retVal = this.collectedPactRecordsInBytes; + this.collectedPactRecordsInBytes = 0L; + + return retVal; + } + // DW: End of temporary code + /** * Collects a {@link PactRecord}, and emits it to all writers. * Writers which require a deep-copy are fed with a copy. @@ -81,6 +93,10 @@ public void addWriter(RecordWriter writer) @Override public void collect(PactRecord record) { + // DW: Start of temporary code + this.collectedPactRecordsInBytes += record.getBinaryLength(); + // DW: End of temporary code + try { for (int i = 0; i < writers.length; i++) { this.writers[i].emit(record); diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/util/KeyGroupedIterator.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/util/KeyGroupedIterator.java index 0dd6c76a69370..51a014ff3e879 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/util/KeyGroupedIterator.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/util/KeyGroupedIterator.java @@ -74,6 +74,18 @@ public KeyGroupedIterator(MutableObjectIterator iterator, int[] keyP } } + // DW: Start of temporary code + private long consumedPactRecordsInBytes = 0L; + + public long getConsumedPactRecordsInBytes() { + + final long retVal = this.consumedPactRecordsInBytes; + this.consumedPactRecordsInBytes = 0L; + + return retVal; + } + // DW: End of temporary code + /** * Moves the iterator to the next key. This method may skip any values that have not yet been returned by the * iterator created by the {@link #getValues()} method. Hence, if called multiple times it "removes" pairs. @@ -216,6 +228,11 @@ public boolean hasNext() public PactRecord next() { if (this.nextIsUnconsumed || hasNext()) { this.nextIsUnconsumed = false; + + // DW: Start of temporary code + KeyGroupedIterator.this.consumedPactRecordsInBytes += KeyGroupedIterator.this.next.getBinaryLength(); + // DW: End of temporary code + return KeyGroupedIterator.this.next; } else { throw new NoSuchElementException(); From c2b45d7d661d4a2694e7cb15999eba4aa399c5da Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 19 Feb 2012 16:44:07 +0100 Subject: [PATCH 248/310] Improved logging --- .../ReplayInputChannelContext.java | 24 +++---- .../ByteBufferedChannelManager.java | 6 +- .../bytebuffered/InputChannelContext.java | 4 +- .../runtime/RuntimeInputChannelContext.java | 63 +++++++++++++------ .../runtime/RuntimeInputGateContext.java | 10 +++ 5 files changed, 62 insertions(+), 45 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java index 7435b3ac1b524..8cabb4b2f7285 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java @@ -31,7 +31,7 @@ public boolean isInputChannel() { */ @Override public JobID getJobID() { - + return this.encapsulatedContext.getJobID(); } @@ -40,7 +40,7 @@ public JobID getJobID() { */ @Override public ChannelID getChannelID() { - + return this.encapsulatedContext.getChannelID(); } @@ -49,7 +49,7 @@ public ChannelID getChannelID() { */ @Override public ChannelID getConnectedChannelID() { - + return this.encapsulatedContext.getConnectedChannelID(); } @@ -120,27 +120,17 @@ public void reportAsynchronousEvent() { * {@inheritDoc} */ @Override - public int getNumberOfQueuedEnvelopes() { + public void destroy() { - return this.encapsulatedContext.getNumberOfQueuedEnvelopes(); + this.encapsulatedContext.destroy(); } /** * {@inheritDoc} */ @Override - public int getNumberOfQueuedMemoryBuffers() { - - return this.encapsulatedContext.getNumberOfQueuedMemoryBuffers(); - } + public void logQueuedEnvelopes() { - /** - * {@inheritDoc} - */ - @Override - public void destroy() { - - this.encapsulatedContext.destroy(); + this.encapsulatedContext.logQueuedEnvelopes(); } - } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 4f020416e0922..fd5afb934289c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -586,11 +586,7 @@ public void logBufferUtilization() { if (context.isInputChannel()) { final InputChannelContext inputChannelContext = (InputChannelContext) context; - final int numberOfQueuedEnvelopes = inputChannelContext.getNumberOfQueuedEnvelopes(); - final int numberOfQueuedMemoryBuffers = inputChannelContext.getNumberOfQueuedMemoryBuffers(); - - System.out.println("\t\t" + entry.getKey() + ": " + numberOfQueuedMemoryBuffers + " (" - + numberOfQueuedEnvelopes + ")"); + inputChannelContext.logQueuedEnvelopes(); } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java index 2b1a584b41531..2376dfe0fe7ec 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java @@ -19,7 +19,5 @@ public interface InputChannelContext extends ChannelContext, BufferProvider { - int getNumberOfQueuedEnvelopes(); - - int getNumberOfQueuedMemoryBuffers(); + void logQueuedEnvelopes(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java index 5554d3e89d605..d453fb0009566 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java @@ -218,7 +218,7 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { } } - LOG.warn("Input channel " + getChannelID() + " expected envelope " + expectedSequenceNumber + LOG.warn("Input channel " + getChannelName() + " expected envelope " + expectedSequenceNumber + " but received " + sequenceNumber); final Buffer buffer = transferEnvelope.getBuffer(); @@ -229,7 +229,7 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { this.queuedEnvelopes.add(transferEnvelope); this.lastReceivedEnvelope = sequenceNumber; - + // Notify the channel about the new data this.envelopeConsumptionLog.reportEnvelopeAvailability(this.byteBufferedInputChannel); } @@ -265,7 +265,6 @@ private AbstractEvent lookForCloseEvent(final TransferEnvelope envelope) { final AbstractEvent event = it.next(); if (event instanceof ByteBufferedChannelCloseEvent) { - LOG.info("Found close event in unexpected envelope"); return event; } } @@ -322,35 +321,39 @@ public void destroy() { } } + /** + * {@inheritDoc} + */ @Override - public int getNumberOfQueuedEnvelopes() { - - synchronized (this.queuedEnvelopes) { - - return this.queuedEnvelopes.size(); - } - } + public void logQueuedEnvelopes() { - @Override - public int getNumberOfQueuedMemoryBuffers() { + int numberOfQueuedEnvelopes = 0; + int numberOfQueuedEnvelopesWithMemoryBuffers = 0; + int numberOfQueuedEnvelopesWithFileBuffers = 0; synchronized (this.queuedEnvelopes) { - int count = 0; - final Iterator it = this.queuedEnvelopes.iterator(); while (it.hasNext()) { final TransferEnvelope envelope = it.next(); - if (envelope.getBuffer() != null) { - if (envelope.getBuffer().isBackedByMemory()) { - ++count; - } + ++numberOfQueuedEnvelopes; + final Buffer buffer = envelope.getBuffer(); + if (buffer == null) { + continue; } - } - return count; + if (buffer.isBackedByMemory()) { + ++numberOfQueuedEnvelopesWithMemoryBuffers; + } else { + ++numberOfQueuedEnvelopesWithFileBuffers; + } + } } + + System.out.println("\t\t" + getChannelName() + ": " + numberOfQueuedEnvelopes + " (" + + numberOfQueuedEnvelopesWithMemoryBuffers + ", " + numberOfQueuedEnvelopesWithFileBuffers + ")"); + } /** @@ -406,4 +409,24 @@ public ChannelType getType() { return this.byteBufferedInputChannel.getType(); } + + /** + * Constructs and returns a human-readable name of this channel used for debugging. + * + * @return a human-readable name of this channel used for debugging + */ + private String getChannelName() { + + final StringBuilder sb = new StringBuilder(this.inputGateContext.getTaskName()); + + sb.append(' '); + sb.append('('); + sb.append(this.byteBufferedInputChannel.getChannelIndex()); + sb.append(','); + sb.append(' '); + sb.append(this.byteBufferedInputChannel.getID()); + sb.append(')'); + + return sb.toString(); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index 06a427c00330b..900ad154756db 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -200,4 +200,14 @@ public LocalBufferPoolOwner getLocalBufferPoolOwner() { return this; } + + /** + * Returns the name of the task this gate belongs to. + * + * @return the name of the task this gate belongs to + */ + String getTaskName() { + + return this.taskName; + } } From fb8b77d4f39eccec8d06d366b4d95a6b931560c2 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 19 Feb 2012 18:43:02 +0100 Subject: [PATCH 249/310] Fix to make checkpoint deserializer work with new file buffer manager --- .../transferenvelope/CheckpointDeserializer.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java index f6e4eff5aae1f..5855274de9dcd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java @@ -32,6 +32,8 @@ public final class CheckpointDeserializer extends AbstractDeserializer { private final FileBufferManager fileBufferManager; + private boolean bufferDataSerializationStarted = false; + public CheckpointDeserializer(final AbstractID ownerID) { this.ownerID = ownerID; this.fileBufferManager = FileBufferManager.getInstance(); @@ -42,6 +44,11 @@ protected boolean readBufferData(final ReadableByteChannel readableByteChannel) final ByteBuffer tempBuffer = getTempBuffer(); + if (!this.bufferDataSerializationStarted) { + tempBuffer.clear(); + this.bufferDataSerializationStarted = true; + } + readableByteChannel.read(tempBuffer); if (tempBuffer.hasRemaining()) { return true; @@ -54,6 +61,7 @@ protected boolean readBufferData(final ReadableByteChannel readableByteChannel) setBuffer(fileBuffer); + this.bufferDataSerializationStarted = false; return false; } From 295ce85a2d3bd40265d54ef5b2afbfb1ff6a3241 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 19 Feb 2012 18:44:43 +0100 Subject: [PATCH 250/310] Improved robustness of buffer implementation --- .../nephele/io/channels/FileBuffer.java | 85 ++++++--- .../nephele/io/channels/MemoryBuffer.java | 174 +++++++++++------- 2 files changed, 159 insertions(+), 100 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java index 4f19db76280a2..75563c4f8b030 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java @@ -20,11 +20,12 @@ import java.nio.channels.FileChannel; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; +import java.util.concurrent.atomic.AtomicBoolean; import eu.stratosphere.nephele.io.AbstractID; public final class FileBuffer extends Buffer { - + private final AbstractID ownerID; private final FileBufferManager fileBufferManager; @@ -39,7 +40,7 @@ public final class FileBuffer extends Buffer { private int totalBytesRead; - private volatile boolean writeMode; + private final AtomicBoolean writeMode = new AtomicBoolean(true); /** * Creates a file buffer for a chunk of data that is already in a file. @@ -49,14 +50,13 @@ public final class FileBuffer extends Buffer { * @param ownerID */ FileBuffer(final int bufferSize, final long offset, final AbstractID ownerID, - final FileBufferManager fileBufferManager) - throws IOException - { + final FileBufferManager fileBufferManager) throws IOException { + this.ownerID = ownerID; this.fileBufferManager = fileBufferManager; this.offset = offset; this.bufferSize = bufferSize; - this.writeMode = false; + this.writeMode.set(false); this.fileChannel = fileBufferManager.getChannelAndIncrementReferences(ownerID); } @@ -75,7 +75,7 @@ public final class FileBuffer extends Buffer { this.fileBufferManager = fileBufferManager; this.ownerID = ownerID; this.bufferSize = bufferSize; - this.writeMode = true; + this.writeMode.set(true); final ChannelWithPosition cwp = fileBufferManager.getChannelForWriteAndIncrementReferences(ownerID, bufferSize); this.fileChannel = cwp.getChannel(); @@ -91,7 +91,7 @@ private FileBuffer(final FileBuffer toCopy) { this.bufferSize = toCopy.bufferSize; this.totalBytesWritten = toCopy.totalBytesWritten; this.totalBytesRead = toCopy.totalBytesRead; - this.writeMode = toCopy.writeMode; + this.writeMode.set(toCopy.writeMode.get()); } /** @@ -100,9 +100,10 @@ private FileBuffer(final FileBuffer toCopy) { @Override public int read(final WritableByteChannel writableByteChannel) throws IOException { - if (this.writeMode) { + if (this.writeMode.get()) { throw new IOException("FileBuffer is still in write mode!"); } + if (this.totalBytesRead >= this.bufferSize) { return -1; } @@ -118,11 +119,12 @@ public int read(final WritableByteChannel writableByteChannel) throws IOExceptio * {@inheritDoc} */ @Override - public int read(ByteBuffer dst) throws IOException { + public int read(final ByteBuffer dst) throws IOException { - if (this.writeMode) { + if (this.writeMode.get()) { throw new IOException("FileBuffer is still in write mode!"); } + if (this.totalBytesRead >= this.bufferSize) { return -1; } @@ -136,7 +138,8 @@ public int read(ByteBuffer dst) throws IOException { return bytesRead; } - private final int readInternal(ByteBuffer dst, long position, int numBytes) throws IOException { + private final int readInternal(final ByteBuffer dst, final long position, final int numBytes) throws IOException { + if (dst.remaining() > numBytes) { final int excess = dst.remaining() - numBytes; dst.limit(dst.limit() - excess); @@ -155,9 +158,10 @@ private final int readInternal(ByteBuffer dst, long position, int numBytes) thro @Override public int write(final ReadableByteChannel readableByteChannel) throws IOException { - if (!this.writeMode) { + if (!this.writeMode.get()) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } + if (this.totalBytesWritten >= this.bufferSize) { return 0; } @@ -175,9 +179,10 @@ public int write(final ReadableByteChannel readableByteChannel) throws IOExcepti @Override public int write(final ByteBuffer src) throws IOException { - if (!this.writeMode) { + if (!this.writeMode.get()) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } + if (this.totalBytesWritten >= this.bufferSize) { return 0; } @@ -201,53 +206,70 @@ public int write(final ByteBuffer src) throws IOException { @Override public void close() throws IOException { - + this.fileChannel.close(); } @Override public boolean isOpen() { - + return this.fileChannel.isOpen(); } @Override public int remaining() { - - if (this.writeMode) { + + if (this.writeMode.get()) { return this.bufferSize - this.totalBytesWritten; } else { return this.bufferSize - this.totalBytesRead; } } + /** + * {@inheritDoc} + */ @Override public int size() { + return (int) this.bufferSize; } + /** + * {@inheritDoc} + */ @Override protected void recycle() { + this.fileBufferManager.decrementReferences(this.ownerID); } + /** + * {@inheritDoc} + */ @Override public void finishWritePhase() throws IOException { - if (this.writeMode) { + + if (this.writeMode.compareAndSet(true, false)) { this.bufferSize = this.totalBytesWritten; - this.writeMode = false; } } + /** + * {@inheritDoc} + */ @Override public boolean isBackedByMemory() { return false; } + /** + * {@inheritDoc} + */ @Override public FileBuffer duplicate() throws IOException, InterruptedException { - - if (this.writeMode) { + + if (this.writeMode.get()) { throw new IllegalStateException("Cannot duplicate buffer that is still in write mode"); } @@ -255,19 +277,22 @@ public FileBuffer duplicate() throws IOException, InterruptedException { return new FileBuffer(this); } + /** + * {@inheritDoc} + */ @Override public void copyToBuffer(final Buffer destinationBuffer) throws IOException { - - if (this.writeMode) { + + if (this.writeMode.get()) { throw new IllegalStateException("Cannot copy buffer that is still in write mode"); } + if (size() > destinationBuffer.size()) { throw new IllegalArgumentException("Destination buffer is too small to store content of source buffer: " + size() + " vs. " + destinationBuffer.size()); } - if (destinationBuffer.isBackedByMemory()) - { + if (destinationBuffer.isBackedByMemory()) { final ByteBuffer bb = ((MemoryBuffer) destinationBuffer).getByteBuffer(); final int tbr = this.totalBytesRead; this.totalBytesRead = 0; @@ -290,8 +315,8 @@ public void copyToBuffer(final Buffer destinationBuffer) throws IOException { */ @Override public boolean isInWriteMode() { - - return this.writeMode; + + return this.writeMode.get(); } /** @@ -300,7 +325,7 @@ public boolean isInWriteMode() { * @return the buffer's offset in bytes */ public long getOffset() { - + return this.offset; } @@ -310,7 +335,7 @@ public long getOffset() { * @return This file buffer's owner id. */ public AbstractID getOwnerID() { - + return this.ownerID; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java index 90d0a6f38f94a..7930cdee076ec 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java @@ -1,17 +1,17 @@ /*********************************************************************************************************************** -* -* Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) -* -* 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. -* -**********************************************************************************************************************/ + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.io.channels; @@ -19,46 +19,46 @@ import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; +import java.util.concurrent.atomic.AtomicBoolean; +public final class MemoryBuffer extends Buffer { -public class MemoryBuffer extends Buffer -{ private final MemoryBufferRecycler bufferRecycler; - + private final ByteBuffer byteBuffer; - - private volatile boolean writeMode = true; - - - MemoryBuffer(final int bufferSize, final ByteBuffer byteBuffer, final MemoryBufferPoolConnector bufferPoolConnector) - { + private final AtomicBoolean writeMode = new AtomicBoolean(true); + + MemoryBuffer(final int bufferSize, final ByteBuffer byteBuffer, final MemoryBufferPoolConnector bufferPoolConnector) { + if (bufferSize > byteBuffer.capacity()) { throw new IllegalArgumentException("Requested buffer size is " + bufferSize + ", but provided byte buffer only has a capacity of " + byteBuffer.capacity()); } - + this.bufferRecycler = new MemoryBufferRecycler(byteBuffer, bufferPoolConnector); - + this.byteBuffer = byteBuffer; this.byteBuffer.position(0); this.byteBuffer.limit(bufferSize); } - - private MemoryBuffer(int bufferSize, ByteBuffer byteBuffer, MemoryBufferRecycler bufferRecycler) - { - this.bufferRecycler = bufferRecycler; + + private MemoryBuffer(final int bufferSize, final ByteBuffer byteBuffer, final MemoryBufferRecycler bufferRecycler) { + + this.bufferRecycler = bufferRecycler; this.byteBuffer = byteBuffer; this.byteBuffer.position(0); this.byteBuffer.limit(bufferSize); } + /** + * {@inheritDoc} + */ @Override - public int read(ByteBuffer dst) throws IOException - { - if (this.writeMode) { - this.writeMode = false; - this.byteBuffer.flip(); + public int read(final ByteBuffer dst) throws IOException { + + if (this.writeMode.get()) { + throw new IOException("Buffer is still in write mode!"); } if (!this.byteBuffer.hasRemaining()) { @@ -82,10 +82,13 @@ public int read(ByteBuffer dst) throws IOException return (this.byteBuffer.position() - oldPosition); } + /** + * {@inheritDoc} + */ @Override - public int read(WritableByteChannel writableByteChannel) throws IOException - { - if (this.writeMode) { + public int read(final WritableByteChannel writableByteChannel) throws IOException { + + if (this.writeMode.get()) { throw new IOException("Buffer is still in write mode!"); } @@ -96,29 +99,38 @@ public int read(WritableByteChannel writableByteChannel) throws IOException return writableByteChannel.write(this.byteBuffer); } + /** + * {@inheritDoc} + */ @Override public void close() throws IOException { this.byteBuffer.position(this.byteBuffer.limit()); } + /** + * {@inheritDoc} + */ @Override public boolean isOpen() { return this.byteBuffer.hasRemaining(); } + /** + * {@inheritDoc} + */ @Override - public int write(ByteBuffer src) throws IOException - { - if (!this.writeMode) { + public int write(final ByteBuffer src) throws IOException { + + if (!this.writeMode.get()) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } final int sourceRemaining = src.remaining(); final int thisRemaining = this.byteBuffer.remaining(); final int excess = sourceRemaining - thisRemaining; - + if (excess <= 0) { // there is enough space here for all the source data this.byteBuffer.put(src); @@ -133,10 +145,13 @@ public int write(ByteBuffer src) throws IOException } } + /** + * {@inheritDoc} + */ @Override - public int write(ReadableByteChannel readableByteChannel) throws IOException - { - if (!this.writeMode) { + public int write(final ReadableByteChannel readableByteChannel) throws IOException { + + if (!this.writeMode.get()) { throw new IOException("Cannot write to buffer, buffer already switched to read mode"); } @@ -147,12 +162,17 @@ public int write(ReadableByteChannel readableByteChannel) throws IOException return readableByteChannel.read(this.byteBuffer); } + /** + * {@inheritDoc} + */ @Override - public int remaining() - { + public int remaining() { return this.byteBuffer.remaining(); } + /** + * {@inheritDoc} + */ @Override public int size() { return this.byteBuffer.limit(); @@ -162,64 +182,78 @@ public ByteBuffer getByteBuffer() { return this.byteBuffer; } + /** + * {@inheritDoc} + */ @Override - protected void recycle() - { + protected void recycle() { + this.bufferRecycler.decreaseReferenceCounter(); } + /** + * {@inheritDoc} + */ @Override - public void finishWritePhase() - { - if (!this.writeMode) { - throw new IllegalStateException("MemoryBuffer is already in write mode!"); + public void finishWritePhase() { + + if (!this.writeMode.compareAndSet(true, false)) { + throw new IllegalStateException("MemoryBuffer is already in read mode!"); } this.byteBuffer.flip(); - this.writeMode = false; } + /** + * {@inheritDoc} + */ @Override - public boolean isBackedByMemory() - { + public boolean isBackedByMemory() { + return true; } + /** + * {@inheritDoc} + */ @Override - public MemoryBuffer duplicate() - { - if (this.writeMode) { + public MemoryBuffer duplicate() { + + if (this.writeMode.get()) { throw new IllegalStateException("Cannot duplicate buffer that is still in write mode"); } - + final MemoryBuffer duplicatedMemoryBuffer = new MemoryBuffer(this.byteBuffer.limit(), this.byteBuffer.duplicate(), this.bufferRecycler); - + this.bufferRecycler.increaseReferenceCounter(); - duplicatedMemoryBuffer.writeMode = this.writeMode; + duplicatedMemoryBuffer.writeMode.set(this.writeMode.get()); return duplicatedMemoryBuffer; } + /** + * {@inheritDoc} + */ @Override - public void copyToBuffer(Buffer destinationBuffer) throws IOException - { - if (this.writeMode) { + public void copyToBuffer(final Buffer destinationBuffer) throws IOException { + + if (this.writeMode.get()) { throw new IllegalStateException("Cannot copy buffer that is still in write mode"); } if (size() > destinationBuffer.size()) { throw new IllegalArgumentException("Destination buffer is too small to store content of source buffer: " + size() + " vs. " + destinationBuffer.size()); } - + final int oldPos = this.byteBuffer.position(); this.byteBuffer.position(0); - + while (remaining() > 0) { destinationBuffer.write(this.byteBuffer); } - + this.byteBuffer.position(oldPos); - + destinationBuffer.finishWritePhase(); } @@ -227,8 +261,8 @@ public void copyToBuffer(Buffer destinationBuffer) throws IOException * {@inheritDoc} */ @Override - public boolean isInWriteMode() - { - return this.writeMode; + public boolean isInWriteMode() { + + return this.writeMode.get(); } } From f9a690483636a5137e2b38a77ca59ad56d8c474c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 19 Feb 2012 18:46:31 +0100 Subject: [PATCH 251/310] Small fix to avoid deadlocks after recovery --- .../CheckpointDecisionCoordinator.java | 162 ++++++++++-------- 1 file changed, 89 insertions(+), 73 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index 169f71ae420bf..7f79e850f98eb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -15,16 +15,19 @@ package eu.stratosphere.nephele.checkpointing; -import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; +import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; @@ -54,10 +57,8 @@ public final class CheckpointDecisionCoordinator { * The object in charge of propagating checkpoint decisions to the respective task managers. */ private final CheckpointDecisionPropagator decisionPropagator; - - - private List decidedVertices = new ArrayList(); + private final Set decidedVertices; /** * Constructs a new checkpoint decision coordinator. @@ -67,6 +68,8 @@ public final class CheckpointDecisionCoordinator { */ public CheckpointDecisionCoordinator(final CheckpointDecisionPropagator decisionPropagator) { this.decisionPropagator = decisionPropagator; + + this.decidedVertices = Collections.newSetFromMap(new ConcurrentHashMap()); } /** @@ -94,96 +97,109 @@ public void registerJob(final ExecutionGraph executionGraph) { * the current resource utilization of the vertex */ void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { + LOG.info("Checkpoint decision for vertex " + vertex + " required"); - synchronized (decidedVertices) { - if (!decidedVertices.contains(vertex.getID())) { - boolean checkpointDecision = getDecision(vertex, rus); - final ExecutionGraph graph = vertex.getExecutionGraph(); - final Map> checkpointDecisions = new HashMap>(); - List checkpointDecisionList = null; - - - synchronized (graph) { - ExecutionGroupVertex groupVertex = vertex.getGroupVertex(); - LOG.info("Forcing decision to " + checkpointDecision + " for all of " + groupVertex.getName()); - //force decision to all groupVertex members - for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); i++) { - ExecutionVertex member = groupVertex.getGroupMember(i); - AbstractInstance instance = member.getAllocatedResource().getInstance(); - if(checkpointDecisions.containsKey(instance)){ - //if instance already in list append new decision - checkpointDecisionList = checkpointDecisions.get(instance); - }else{ - //make an new list for each instance - checkpointDecisionList = new SerializableArrayList(); - } - checkpointDecisionList.add(new CheckpointDecision(member.getID(), checkpointDecision)); - checkpointDecisions.put(instance, checkpointDecisionList); - - this.decidedVertices.add(member.getID()); + if (!this.decidedVertices.contains(vertex.getID())) { + boolean checkpointDecision = getDecision(vertex, rus); + final ExecutionGraph graph = vertex.getExecutionGraph(); + final Map> checkpointDecisions = new HashMap>(); + List checkpointDecisionList = null; + + synchronized (graph) { + ExecutionGroupVertex groupVertex = vertex.getGroupVertex(); + LOG.info("Forcing decision to " + checkpointDecision + " for all of " + groupVertex.getName()); + // force decision to all groupVertex members + for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); i++) { + ExecutionVertex member = groupVertex.getGroupMember(i); + AbstractInstance instance = member.getAllocatedResource().getInstance(); + if (checkpointDecisions.containsKey(instance)) { + // if instance already in list append new decision + checkpointDecisionList = checkpointDecisions.get(instance); + } else { + // make an new list for each instance + checkpointDecisionList = new SerializableArrayList(); } - } + checkpointDecisionList.add(new CheckpointDecision(member.getID(), checkpointDecision)); + checkpointDecisions.put(instance, checkpointDecisionList); - // Propagate checkpoint decisions - this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); + this.decidedVertices.add(member.getID()); + } } + + // Propagate checkpoint decisions + this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); + + } else { + + // Propagate checkpoint decisions + final List checkpointDecisionList = new SerializableArrayList(); + final Map> checkpointDecisions = new HashMap>(); + checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), vertex.getCheckpointState() != CheckpointState.NONE)); + checkpointDecisions.put(vertex.getAllocatedResource().getInstance(), checkpointDecisionList); + this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); + } -// LOG.info("Checkpoint decision for vertex " + vertex + " required"); -// -// // TODO: Provide sensible implementation here -// boolean checkpointDecision = getDecision(vertex, rus); -// final ExecutionGraph graph = vertex.getExecutionGraph(); -// final Map> checkpointDecisions = new HashMap>(); -// final List checkpointDecisionList = new SerializableArrayList(); -// -// synchronized (graph) { -// checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), checkpointDecision)); -// checkpointDecisions.put(vertex.getAllocatedResource().getInstance(), checkpointDecisionList); -// } -// -// // Propagate checkpoint decisions -// this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); + + // LOG.info("Checkpoint decision for vertex " + vertex + " required"); + // + // // TODO: Provide sensible implementation here + // boolean checkpointDecision = getDecision(vertex, rus); + // final ExecutionGraph graph = vertex.getExecutionGraph(); + // final Map> checkpointDecisions = new HashMap>(); + // final List checkpointDecisionList = new SerializableArrayList(); + // + // synchronized (graph) { + // checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), checkpointDecision)); + // checkpointDecisions.put(vertex.getAllocatedResource().getInstance(), checkpointDecisionList); + // } + // + // // Propagate checkpoint decisions + // this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); } private boolean getDecision(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { // This implementation always creates the checkpoint - if(rus.getForced() == null){ - if(rus.getTotalInputAmount() != 0 ){ - LOG.info("selektivity is " + (double)rus.getTotalOutputAmount() / rus.getTotalInputAmount()); - LOG.info("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); + if (rus.getForced() == null) { + if (rus.getTotalInputAmount() != 0) { + LOG.info("selektivity is " + (double) rus.getTotalOutputAmount() / rus.getTotalInputAmount()); + LOG.info("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); } - if(rus.getTotalInputAmount() != 0 && ((double)rus.getTotalOutputAmount() / rus.getTotalInputAmount() > 2.0)){ - //estimated size of checkpoint - //TODO progress estimation would make sense here - LOG.info(vertex.getEnvironment().getTaskName() + "Chechpoint to large selektivity " + ((double)rus.getTotalOutputAmount()/ rus.getTotalInputAmount() > 2.0)); + if (rus.getTotalInputAmount() != 0 + && ((double) rus.getTotalOutputAmount() / rus.getTotalInputAmount() > 2.0)) { + // estimated size of checkpoint + // TODO progress estimation would make sense here + LOG.info(vertex.getEnvironment().getTaskName() + "Chechpoint to large selektivity " + + ((double) rus.getTotalOutputAmount() / rus.getTotalInputAmount() > 2.0)); return false; - + } - if (rus.getUserCPU() >= 90) { + if (rus.getUserCPU() >= 90) { LOG.info(vertex.getEnvironment().getTaskName() + "CPU-Bottleneck"); - //CPU bottleneck + // CPU bottleneck return true; - } - - if ( vertex.getNumberOfSuccessors() != 0 - && vertex.getNumberOfPredecessors() * 1.0 / vertex.getNumberOfSuccessors() > 1.5) { - - LOG.info(vertex.getEnvironment().getTaskName() + " vertex.getNumberOfPredecessors() " + vertex.getNumberOfPredecessors() +" / vertex.getNumberOfSuccessors() " + vertex.getNumberOfSuccessors() +" > 1.5"); - //less output-channels than input-channels - //checkpoint at this position probably saves network-traffic + } + + if (vertex.getNumberOfSuccessors() != 0 + && vertex.getNumberOfPredecessors() * 1.0 / vertex.getNumberOfSuccessors() > 1.5) { + + LOG.info(vertex.getEnvironment().getTaskName() + " vertex.getNumberOfPredecessors() " + + vertex.getNumberOfPredecessors() + " / vertex.getNumberOfSuccessors() " + + vertex.getNumberOfSuccessors() + " > 1.5"); + // less output-channels than input-channels + // checkpoint at this position probably saves network-traffic return true; - } - - }else{ + } + + } else { LOG.info("Checkpoint decision was forced"); - //checkpoint decision was forced by the user + // checkpoint decision was forced by the user return rus.getForced(); } LOG.info("always create Checkpoint for testing"); - //FIXME always create checkpoint for testing + // FIXME always create checkpoint for testing return true; } } - From d90d06e851f9fb6882b05d830b47d699f4d8d077 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 19 Feb 2012 19:56:33 +0100 Subject: [PATCH 252/310] Fixed minor bug in JobManager --- .../main/java/eu/stratosphere/nephele/jobmanager/JobManager.java | 1 + 1 file changed, 1 insertion(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index d24be8283e3bd..f1d1a5eea72cf 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -954,6 +954,7 @@ public void killInstance(final StringRecord instanceName) throws IOException { final AbstractInstance instance = this.instanceManager.getInstanceByName(instanceName.toString()); if (instance == null) { LOG.error("Cannot find instance with name " + instanceName + " to kill it"); + return; } LOG.info("Killing task manager on instance " + instance); From 3641a09a275a31f3caee6e2a65e2717249e2738a Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 20 Feb 2012 11:39:49 +0100 Subject: [PATCH 253/310] Modified recovery logic to also handle node failures --- .../instance/cluster/ClusterManager.java | 8 +- .../executiongraph/ExecutionVertex.java | 4 + .../scheduler/AbstractExecutionListener.java | 104 +--------------- .../scheduler/AbstractScheduler.java | 117 +++++++++++------- .../jobmanager/scheduler/RecoveryLogic.java | 47 ++++--- 5 files changed, 115 insertions(+), 165 deletions(-) diff --git a/nephele/nephele-clustermanager/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java b/nephele/nephele-clustermanager/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java index 91fc359b30ac4..6d62d31dd0de2 100644 --- a/nephele/nephele-clustermanager/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java +++ b/nephele/nephele-clustermanager/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java @@ -252,6 +252,10 @@ public void run() { } } + registeredHosts.entrySet().removeAll(hostsToRemove); + + updateInstaceTypeDescriptionMap(); + final Iterator>> it = staleResources.entrySet().iterator(); while (it.hasNext()) { final Map.Entry> entry = it.next(); @@ -259,10 +263,6 @@ public void run() { instanceListener.allocatedResourcesDied(entry.getKey(), entry.getValue()); } } - - registeredHosts.entrySet().removeAll(hostsToRemove); - - updateInstaceTypeDescriptionMap(); } } }; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index dd1ad91205b53..76fac4a369eae 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -659,6 +659,10 @@ public TaskCancelResult cancelTask() { final ExecutionState previousState = this.executionState.get(); + if(previousState == ExecutionState.CANCELED) { + return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); + } + if (updateExecutionState(ExecutionState.CANCELING) != ExecutionState.CANCELING) { if (this.groupVertex.getStageNumber() != this.executionGraph.getIndexOfCurrentExecutionStage()) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java index d063363328aad..8b7de50b08a2c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java @@ -23,7 +23,6 @@ import eu.stratosphere.nephele.executiongraph.ExecutionPipeline; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.instance.InstanceException; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobmanager.scheduler.local.LocalScheduler; @@ -112,15 +111,11 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver if (RecoveryLogic.recover(this.executionVertex, this.scheduler.getVerticesToBeRestarted())) { - // Run through the deployment procedure - this.scheduler.deployAssignedVertices(eg); - - try { - this.scheduler.requestInstances(this.executionVertex.getGroupVertex().getExecutionStage()); - } catch (InstanceException e) { - e.printStackTrace(); - // TODO: Cancel the entire job in this case + if (RecoveryLogic.hasInstanceAssigned(this.executionVertex)) { + // Run through the deployment procedure + this.scheduler.deployAssignedVertices(eg); } + } else { // TODO: Cancel the entire job in this case } @@ -151,97 +146,6 @@ public void userThreadStarted(final JobID jobID, final ExecutionVertexID vertexI @Override public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID vertexID, final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { - - /* - * final ExecutionGraph executionGraph = this.executionVertex.getExecutionGraph(); - * System.out.println(this.executionVertex + " has run out of execution resources"); - * final Map targetVertices = new HashMap(); - * final Map availableResources = new HashMap(); - * final Environment ee = this.executionVertex.getEnvironment(); - * for (int i = 0; i < ee.getNumberOfOutputGates(); ++i) { - * final OutputGate outputGate = ee.getOutputGate(i); - * for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { - * final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); - * final long transmittedData = resourceUtilizationSnapshot.getAmountOfDataTransmitted(outputChannel - * .getID()); - * final ExecutionVertex connectedVertex = executionGraph.getVertexByChannelID(outputChannel - * .getConnectedChannelID()); - * final ExecutionState state = connectedVertex.getExecutionState(); - * if (state == ExecutionState.SCHEDULED || state == ExecutionState.ASSIGNED) { - * targetVertices.put(connectedVertex, Long.valueOf(transmittedData)); - * final AllocatedResource allocatedResource = connectedVertex.getAllocatedResource(); - * if (!(allocatedResource.getInstance() instanceof DummyInstance)) { - * availableResources.put(allocatedResource, Long.valueOf(0L)); - * } - * } - * } - * if (targetVertices.isEmpty()) { - * return; - * } - * final Queue vertexQueue = new PriorityQueue(targetVertices.size(), - * new Comparator() { - * @Override - * public int compare(final ExecutionVertex arg0, final ExecutionVertex arg1) { - * final Long l0 = targetVertices.get(arg0); - * final Long l1 = targetVertices.get(arg1); - * if (l0.longValue() == l1.longValue()) { - * return 0; - * } - * if (l0.longValue() < l1.longValue()) { - * return 1; - * } - * return -1; - * } - * }); - * final Queue resourceQueue = new PriorityQueue( - * availableResources.size(), new Comparator() { - * @Override - * public int compare(final AllocatedResource arg0, final AllocatedResource arg1) { - * final Long l0 = availableResources.get(arg0); - * final Long l1 = availableResources.get(arg1); - * if (l0.longValue() == l1.longValue()) { - * return 0; - * } - * if (l0.longValue() < l1.longValue()) { - * return -1; - * } - * return 1; - * } - * }); - * Iterator vertexIt = targetVertices.keySet().iterator(); - * while (vertexIt.hasNext()) { - * vertexQueue.add(vertexIt.next()); - * } - * final Iterator resourceIt = availableResources.keySet().iterator(); - * while (resourceIt.hasNext()) { - * resourceQueue.add(resourceIt.next()); - * } - * while (!vertexQueue.isEmpty()) { - * final ExecutionVertex v = vertexQueue.poll(); - * final long vertexLoad = targetVertices.get(v); - * System.out.println(v + ": " + vertexLoad); - * final AllocatedResource ar = resourceQueue.poll(); - * final long resourceLoad = availableResources.get(ar).longValue(); - * System.out.println(ar + ": " + resourceLoad); - * availableResources.put(ar, Long.valueOf(vertexLoad + resourceLoad)); - * resourceQueue.add(ar); - * reassignGraphFragment(v, v.getAllocatedResource(), ar); - * } - * final Map> verticesToBeDeployed = new HashMap>(); - * vertexIt = targetVertices.keySet().iterator(); - * while (vertexIt.hasNext()) { - * this.scheduler.findVerticesToBeDeployed(vertexIt.next(), verticesToBeDeployed); - * } - * final Iterator>> deploymentIt = verticesToBeDeployed - * .entrySet().iterator(); - * while (deploymentIt.hasNext()) { - * final Map.Entry> entry = deploymentIt.next(); - * this.scheduler.getDeploymentManager().deploy(executionGraph.getJobID(), entry.getKey(), - * entry.getValue()); - * } - * } - */ } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index a11441e7d7cb0..db66461e240d5 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -39,6 +39,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionStage; import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.executiongraph.InternalJobStatus; import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.instance.AllocatedResource; import eu.stratosphere.nephele.instance.AllocationID; @@ -204,10 +205,10 @@ void findVerticesToBeDeployed(final ExecutionVertex vertex, final Map> verticesToBeDeployed, final Set alreadyVisited) { - if(!alreadyVisited.add(vertex)) { + if (!alreadyVisited.add(vertex)) { return; } - + if (vertex.compareAndUpdateExecutionState(ExecutionState.ASSIGNED, ExecutionState.READY)) { final AbstractInstance instance = vertex.getAllocatedResource().getInstance(); @@ -446,19 +447,6 @@ public void checkAndReleaseAllocatedResource(final ExecutionGraph executionGraph if (resourceCanBeReleased) { - final DummyInstance dummyInstance = DummyInstance.createDummyInstance(allocatedResource.getInstance() - .getType()); - final AllocatedResource dummyResource = new AllocatedResource(dummyInstance, - allocatedResource.getInstanceType(), new AllocationID()); - - // Assign vertices back to a dummy resource in case we need the resource information once more for another - // execution. - it = assignedVertices.iterator(); - while (it.hasNext()) { - final ExecutionVertex vertex = it.next(); - vertex.setAllocatedResource(dummyResource); - } - LOG.info("Releasing instance " + allocatedResource.getInstance()); try { getInstanceManager().releaseAllocatedResource(executionGraph.getJobID(), executionGraph @@ -503,44 +491,89 @@ Map getVerticesToBeRestarted() { @Override public void allocatedResourcesDied(final JobID jobID, final List allocatedResources) { - // TODO: Don't forget to synchronize on stage here + final ExecutionGraph eg = getExecutionGraphByID(jobID); - for (final AllocatedResource allocatedResource : allocatedResources) { + if (eg == null) { + LOG.error("Cannot find execution graph for job with ID " + jobID); + return; + } - LOG.info("Resource on " + allocatedResource.getInstance().getName() + " for Job " + jobID + " died."); - // TODO (marrus) + synchronized (eg) { - final ExecutionGraph executionGraph = getExecutionGraphByID(jobID); + for (final AllocatedResource allocatedResource : allocatedResources) { - if (executionGraph == null) { - LOG.error("Cannot find execution graph for job " + jobID); - return; - } + LOG.info("Resource " + allocatedResource.getInstance().getName() + " for Job " + jobID + " died."); + + final ExecutionGraph executionGraph = getExecutionGraphByID(jobID); + + if (executionGraph == null) { + LOG.error("Cannot find execution graph for job " + jobID); + return; + } - final List vertices = executionGraph.getVerticesAssignedToResource(allocatedResource); - final Iterator vertexIter = vertices.iterator(); - while (vertexIter.hasNext()) { - final ExecutionVertex vertex = vertexIter.next(); + final List vertices = executionGraph.getVerticesAssignedToResource(allocatedResource); + Iterator vertexIter = vertices.iterator(); - // Even if the vertex had a checkpoint before, it is now gone - vertex.updateCheckpointState(CheckpointState.NONE); + // Assign vertices back to a dummy resource. + final DummyInstance dummyInstance = DummyInstance.createDummyInstance(allocatedResource.getInstance() + .getType()); + final AllocatedResource dummyResource = new AllocatedResource(dummyInstance, + allocatedResource.getInstanceType(), new AllocationID()); - final ExecutionState state = vertex.getExecutionState(); + while (vertexIter.hasNext()) { + final ExecutionVertex vertex = vertexIter.next(); + vertex.setAllocatedResource(dummyResource); + } + + final String failureMessage = allocatedResource.getInstance().getName() + " died"; + + vertexIter = vertices.iterator(); + + while (vertexIter.hasNext()) { + final ExecutionVertex vertex = vertexIter.next(); + + // Even if the vertex had a checkpoint before, it is now gone + vertex.updateCheckpointState(CheckpointState.NONE); - switch (state) { - case ASSIGNED: - case READY: - case STARTING: - case RUNNING: - case FINISHING: + final ExecutionState state = vertex.getExecutionState(); - vertex.updateExecutionState(ExecutionState.FAILED, "The resource " - + allocatedResource.getInstance().getName() + " the vertex " - + vertex.getEnvironment().getTaskName() + " was assigned to died"); + switch (state) { + case ASSIGNED: + case READY: + case STARTING: + case RUNNING: + case FINISHING: - break; - default: + vertex.updateExecutionState(ExecutionState.FAILED, failureMessage); + + break; + default: + } } + + /* + * try { + * requestInstances(this.executionVertex.getGroupVertex().getExecutionStage()); + * } catch (InstanceException e) { + * e.printStackTrace(); + * // TODO: Cancel the entire job in this case + * } + */ + } + } + + final InternalJobStatus js = eg.getJobStatus(); + if (js != InternalJobStatus.FAILING && js != InternalJobStatus.FAILED) { + + deployAssignedVertices(eg); + + final ExecutionStage stage = eg.getCurrentExecutionStage(); + + try { + requestInstances(stage); + } catch (InstanceException e) { + e.printStackTrace(); + // TODO: Cancel the entire job in this case } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index 2b8a59cb7649f..5276247462938 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -130,13 +130,18 @@ public static boolean recover(final ExecutionVertex failedVertex, return true; } + static boolean hasInstanceAssigned(final ExecutionVertex vertex) { + + return !(vertex.getAllocatedResource().getInstance() instanceof DummyInstance); + } + private static ExecutionState getStateToUpdate(final ExecutionVertex vertex) { - if (vertex.getAllocatedResource().getInstance() instanceof DummyInstance) { - return ExecutionState.CREATED; + if (hasInstanceAssigned(vertex)) { + return ExecutionState.ASSIGNED; } - return ExecutionState.ASSIGNED; + return ExecutionState.CREATED; } private static void findVerticesToRestart(final ExecutionVertex failedVertex, @@ -147,30 +152,26 @@ private static void findVerticesToRestart(final ExecutionVertex failedVertex, final Set visited = new HashSet(); verticesToTest.add(failedVertex); - System.out.println("++++" + failedVertex + " failed"); - while (!verticesToTest.isEmpty()) { final ExecutionVertex vertex = verticesToTest.poll(); - if (!vertex.getID().equals(failedVertex.getID())) { - verticesToBeCanceled.add(vertex); - } - // Predecessors must be either checkpoints or need to be restarted, too for (int j = 0; j < vertex.getNumberOfPredecessors(); j++) { final ExecutionVertex predecessor = vertex.getPredecessor(j); - System.out.println("++++ Predecessor " + predecessor + " has checkpoint state " - + predecessor.getCheckpointState()); - if (predecessor.getCheckpointState() != CheckpointState.PARTIAL - && predecessor.getCheckpointState() != CheckpointState.COMPLETE) { - - verticesToBeCanceled.add(predecessor); - if (!visited.contains(predecessor)) { - verticesToTest.add(predecessor); + + if (hasInstanceAssigned(predecessor)) { + + if (predecessor.getCheckpointState() == CheckpointState.NONE) { + verticesToBeCanceled.add(predecessor); + } else { + checkpointsToBeReplayed.add(predecessor); + continue; } - } else { - checkpointsToBeReplayed.add(predecessor); + } + + if (!visited.contains(predecessor)) { + verticesToTest.add(predecessor); } } visited.add(vertex); @@ -230,6 +231,10 @@ private static void collectCacheEntriesToInvalidate(final ExecutionVertex vertex } final AbstractInstance instance = connectedVertex.getAllocatedResource().getInstance(); + if (instance instanceof DummyInstance) { + continue; + } + Set channelIDs = entriesToInvalidate.get(instance); if (channelIDs == null) { channelIDs = new SerializableHashSet(); @@ -259,6 +264,10 @@ private static void collectCacheEntriesToInvalidate(final ExecutionVertex vertex } final AbstractInstance instance = connectedVertex.getAllocatedResource().getInstance(); + if (instance instanceof DummyInstance) { + continue; + } + Set channelIDs = entriesToInvalidate.get(instance); if (channelIDs == null) { channelIDs = new SerializableHashSet(); From 88833e4131a3fa7dc12e414d5313ba24713a5fda Mon Sep 17 00:00:00 2001 From: marrus Date: Mon, 20 Feb 2012 12:14:26 +0100 Subject: [PATCH 254/310] changes in DecisionCoordinator --- .../CheckpointDecisionCoordinator.java | 68 +++++-------------- 1 file changed, 17 insertions(+), 51 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java index a780b4c053784..8454fa4dfc1e8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java @@ -96,60 +96,27 @@ public void registerJob(final ExecutionGraph executionGraph) { void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { LOG.info("Checkpoint decision for vertex " + vertex + " required"); - synchronized (decidedVertices) { - if (!decidedVertices.contains(vertex.getID())) { boolean checkpointDecision = getDecision(vertex, rus); if(checkpointDecision) - LOG.info("Creating Checkpoint for " + vertex.getName() ); - final ExecutionGraph graph = vertex.getExecutionGraph(); - final Map> checkpointDecisions = new HashMap>(); - List checkpointDecisionList = null; - - - synchronized (graph) { - ExecutionGroupVertex groupVertex = vertex.getGroupVertex(); - LOG.info("Forcing decision to " + checkpointDecision + " for all of " + groupVertex.getName()); - //force decision to all groupVertex members - for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); i++) { - ExecutionVertex member = groupVertex.getGroupMember(i); - AbstractInstance instance = member.getAllocatedResource().getInstance(); - if(checkpointDecisions.containsKey(instance)){ - //if instance already in list append new decision - checkpointDecisionList = checkpointDecisions.get(instance); - }else{ - //make an new list for each instance - checkpointDecisionList = new SerializableArrayList(); - } - checkpointDecisionList.add(new CheckpointDecision(member.getID(), checkpointDecision)); - checkpointDecisions.put(instance, checkpointDecisionList); - - this.decidedVertices.add(member.getID()); - } - } + LOG.info("Creating Checkpoint for " + vertex.getEnvironment().getTaskNameWithIndex() ); + + + final ExecutionGraph graph = vertex.getExecutionGraph(); + final Map> checkpointDecisions = new HashMap>(); + final List checkpointDecisionList = new SerializableArrayList(); + + synchronized (graph) { + checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), checkpointDecision)); + checkpointDecisions.put(vertex.getAllocatedResource().getInstance(), checkpointDecisionList); + } + // Propagate checkpoint decisions - this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); - } - } -// LOG.info("Checkpoint decision for vertex " + vertex + " required"); -// -// // TODO: Provide sensible implementation here -// boolean checkpointDecision = getDecision(vertex, rus); -// final ExecutionGraph graph = vertex.getExecutionGraph(); -// final Map> checkpointDecisions = new HashMap>(); -// final List checkpointDecisionList = new SerializableArrayList(); -// -// synchronized (graph) { -// checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), checkpointDecision)); -// checkpointDecisions.put(vertex.getAllocatedResource().getInstance(), checkpointDecisionList); -// } -// -// // Propagate checkpoint decisions -// this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); + this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); } private boolean getDecision(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { - // This implementation always creates the checkpoint + if(rus.getForced() == null){ if(rus.getTotalInputAmount() != 0 ){ LOG.info("selektivity is " + (double)rus.getTotalOutputAmount() / rus.getTotalInputAmount()); @@ -158,7 +125,7 @@ private boolean getDecision(final ExecutionVertex vertex, final ResourceUtilizat if(rus.getTotalInputAmount() != 0 && ((double)rus.getTotalOutputAmount() / rus.getTotalInputAmount() > 2.0)){ //estimated size of checkpoint //TODO progress estimation would make sense here - LOG.info(vertex.getEnvironment().getTaskName() + "Chechpoint to large selektivity " + ((double)rus.getTotalOutputAmount()/ rus.getTotalInputAmount() > 2.0)); + LOG.info(vertex.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double)rus.getTotalOutputAmount()/ rus.getTotalInputAmount() > 2.0)); return false; } @@ -182,9 +149,8 @@ private boolean getDecision(final ExecutionVertex vertex, final ResourceUtilizat //checkpoint decision was forced by the user return rus.getForced(); } - LOG.info("always create Checkpoint for testing"); - //FIXME always create checkpoint for testing - return true; + + return false; } } From bc1256b4cbe9636db91ed4579d09ecde7767fe5d Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 20 Feb 2012 14:07:07 +0100 Subject: [PATCH 255/310] Clean-up of code --- .../configuration/ConfigConstants.java | 23 --------- .../io/channels/FileBufferManager.java | 47 +++---------------- 2 files changed, 7 insertions(+), 63 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/configuration/ConfigConstants.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/configuration/ConfigConstants.java index c053aada1ed3d..061afb11e6367 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/configuration/ConfigConstants.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/configuration/ConfigConstants.java @@ -66,19 +66,6 @@ public final class ConfigConstants { * The key for the config parameter defining flag to terminate a job at job-client shutdown. */ public static final String JOBCLIENT_SHUTDOWN_TERMINATEJOB_KEY = "jobclient.shutdown.terminatejob"; - - /** - * The key for the config parameter defining how many channels spill into the same physical file. - */ - public static final String TASKMANAGER_FILECHANNEL_NUMMERGED = "taskmanager.filechannel.nummerged"; - - /** - * The key for the config parameter defining the size (in bytes) of the chunk that is contiguously - * allocated for write operations from the same channel. Only relevant when multiple channels - * write into the same file. - */ - public static final String TASKMANAGER_FILECHANNEL_EXTENDSIZE = "taskmanager.filechannel.extendsize"; - // ------------------------------------------------------------------------ // Default Values @@ -123,16 +110,6 @@ public final class ConfigConstants { * The default scheduler to be used when Nephele is started in local mode. */ public static final String DEFAULT_LOCAL_MODE_SCHEDULER = "eu.stratosphere.nephele.jobmanager.scheduler.local.LocalScheduler"; - - /** - * The default scheduler to be used when Nephele is started in local mode. - */ - public static final int DEFAULT_NUM_FILECHANNELS_MERGED = 16; - - /** - * The default size of an extend allocated for a channel in a merged file. 4 MiBytes. - */ - public static final int DEFAULT_FILECHANNEL_EXTEND_SIZE = 4 * 1024 * 1024; // ----------------------------- Instances -------------------------------- diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java index 1c4dcb4b44d9f..3e84ec5397229 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java @@ -42,8 +42,7 @@ * @author warneke * @author Stephan Ewen */ -public final class FileBufferManager -{ +public final class FileBufferManager { /** * The prefix with which spill files are stored. */ @@ -59,18 +58,12 @@ public final class FileBufferManager */ private static final FileBufferManager instance = new FileBufferManager(); - /** - * The minimal size for an extend allocated for a channel. - */ - private static final int MIN_EXTEND_SIZE = 64 * 1024; - /** * Gets the singleton instance of the file buffer manager. * - * @return The file buffer manager singleton instance. + * @return the file buffer manager singleton instance */ - public static FileBufferManager getInstance() - { + public static FileBufferManager getInstance() { return instance; } @@ -87,36 +80,13 @@ public static FileBufferManager getInstance() private final String[] tmpDirs; /** - * The size of the extend to allocate for each channel. - */ - private final int extendSize; - - /** - * + * Constructs a new file buffer manager object. */ private FileBufferManager() { this.tmpDirs = GlobalConfiguration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(":"); - final int extendSize = GlobalConfiguration.getConfiguration().getInteger( - ConfigConstants.TASKMANAGER_FILECHANNEL_EXTENDSIZE, - ConfigConstants.DEFAULT_FILECHANNEL_EXTEND_SIZE); - - // check extend size - if (extendSize < MIN_EXTEND_SIZE) { - LOG.error("Invalid extend size " + extendSize + ". Minimum extend size is " + MIN_EXTEND_SIZE + - ". Falling back to default extend size of " + ConfigConstants.DEFAULT_FILECHANNEL_EXTEND_SIZE); - this.extendSize = ConfigConstants.DEFAULT_FILECHANNEL_EXTEND_SIZE; - } else if ((extendSize & (extendSize - 1)) != 0) { - // not a power of two - this.extendSize = Integer.highestOneBit(extendSize); - LOG.warn("Changing extend size from " + extendSize + " to " + this.extendSize - + " to make it a power of two."); - } else { - this.extendSize = extendSize; - } - // check temp dirs for (int i = 0; i < this.tmpDirs.length; i++) { File f = new File(this.tmpDirs[i]); @@ -324,8 +294,7 @@ int decrementReferences() { current = 0; break; } - } - else if (this.referenceCounter.compareAndSet(current, current - 1)) { + } else if (this.referenceCounter.compareAndSet(current, current - 1)) { current = current - 1; break; } @@ -334,8 +303,7 @@ else if (this.referenceCounter.compareAndSet(current, current - 1)) { if (current > 0) { return current; - } - else if (current == 0) { + } else if (current == 0) { // delete the channel this.referenceCounter.set(Integer.MIN_VALUE); this.reservedWritePosition.set(Long.MIN_VALUE); @@ -348,8 +316,7 @@ else if (current == 0) { } this.file.delete(); return current; - } - else { + } else { throw new IllegalStateException("The references to the file were already at zero."); } } From cf7ad57a4b45492155b0e997d040890facb67a05 Mon Sep 17 00:00:00 2001 From: marrus Date: Mon, 20 Feb 2012 15:59:11 +0100 Subject: [PATCH 256/310] changes in decision logic an locality --- .../ResourceUtilizationSnapshot.java | 18 ++- .../nephele/jobmanager/JobManager.java | 2 + .../nephele/taskmanager/TaskManager.java | 109 ++++++++++++++++-- .../taskmanager/runtime/RuntimeTask.java | 4 +- 4 files changed, 122 insertions(+), 11 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java index a23c8ccd708c8..1d2bbc6eae409 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java @@ -65,8 +65,12 @@ public final class ResourceUtilizationSnapshot implements IOReadableWritable { private long averageInputRecordSize; + private double pactRatio; - public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU) { + private boolean isDam; + + + public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU, Boolean force, long totalInputAmount2, long totalOutputAmount2, long averageOutputRecordSize2, long averageInputRecordSize2, double pactRatio, boolean isDam) { if (timestamp <= 0L) { throw new IllegalArgumentException("Argument timestamp must be larger than zero"); @@ -79,6 +83,12 @@ public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU, final Boolean forced, final long totalInputAmount, final long totalOutputAmount) { @@ -247,4 +257,10 @@ public long getAverageInputRecordSize() { return averageInputRecordSize; } + public double getPactRatio(){ + return this.pactRatio; + } + public boolean isDam(){ + return this.isDam; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index fe77cf351f8f3..7c14eaa87c0fd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -1262,6 +1262,7 @@ public InputSplitWrapper requestNextInputSplit(final JobID jobID, final Executio * {@inheritDoc} */ @Override + @Deprecated public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID vertexID, final ResourceUtilizationSnapshot resourceUtilizationSnapshot) throws IOException { @@ -1329,6 +1330,7 @@ public void run() { * {@inheritDoc} */ @Override + @Deprecated public void propagateCheckpointDecisions(final Map> checkpointDecisions) { final Iterator>> it = checkpointDecisions.entrySet() diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 4566e024cbddd..efc7d7f4d76a6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -20,6 +20,7 @@ import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -55,7 +56,10 @@ import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate; import eu.stratosphere.nephele.executiongraph.CheckpointState; +import eu.stratosphere.nephele.executiongraph.ExecutionGraph; +import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.instance.HardwareDescription; import eu.stratosphere.nephele.instance.HardwareDescriptionFactory; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; @@ -333,7 +337,7 @@ public TaskManager(String configDir) throws Exception { public static void main(String[] args) { Option configDirOpt = OptionBuilder.withArgName("config directory").hasArg().withDescription( - "Specify configuration directory.").create("configDir"); + "Specify configuration directory.").create("configDir"); Options options = new Options(); options.addOption(configDirOpt); @@ -683,7 +687,7 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final ExecutionState newExecutionState, final String optionalDescription) { if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED - || newExecutionState == ExecutionState.FAILED) { + || newExecutionState == ExecutionState.FAILED) { // Unregister the task (free all buffers, remove all channels, task-specific class loaders, etc...) unregisterTask(id); @@ -712,14 +716,103 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID id, */ public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID id, final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { + Task task = this.runningTasks.get(id); + LOG.info("Checkpoint decision for vertex " + task.getEnvironment().getTaskName() + " required"); + boolean checkpointDecision = getDecision(task, resourceUtilizationSnapshot); + if(checkpointDecision) + LOG.info("Creating Checkpoint for " + task.getEnvironment().getTaskName() + " (" +task.getEnvironment().getIndexInSubtaskGroup() +"/" +task.getEnvironment().getCurrentNumberOfSubtasks() + " )"); - synchronized (this.jobManager) { - try { - this.jobManager.initialExecutionResourcesExhausted(jobID, id, resourceUtilizationSnapshot); - } catch (IOException e) { - LOG.error(StringUtils.stringifyException(e)); + final List checkpointDecisionList = new SerializableArrayList(); + + checkpointDecisionList.add(new CheckpointDecision(id, checkpointDecision)); + + // Propagate checkpoint decisions + try { + propagateCheckpointDecisions(checkpointDecisionList); + } catch (IOException e) { + e.printStackTrace(); + } + + } + private boolean getDecision(final Task task, final ResourceUtilizationSnapshot rus) { + + if(rus.getForced() == null){ + + + if(rus.getPactRatio() != -1){ + System.out.println("Ratio = " + rus.getPactRatio()); + if(rus.getPactRatio()>=5.0){ + //amount of data is small so we checkpoint + return true; + } + if(rus.getPactRatio()<=0.6){ + //amount of data is too big + return false; + } + }else{ + //no info from upper layer so use average sizes + if(rus.isDam()){ + System.out.println("is Dam"); + + if(rus.getAverageInputRecordSize() != 0){ + System.out.println( "avg ratio" + rus.getAverageOutputRecordSize()/rus.getAverageInputRecordSize()); + } + + if(rus.getAverageInputRecordSize() != 0 && + rus.getAverageOutputRecordSize()/rus.getAverageInputRecordSize() < 0.6){ + return true; + } + + if(rus.getAverageInputRecordSize() != 0 && + rus.getAverageOutputRecordSize()/rus.getAverageInputRecordSize() > 2.0){ + return false; + } + }else{ + + + + // we have no data dam so we can estimate the input/output-ratio + System.out.println("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); + if(rus.getTotalInputAmount() != 0 ){ + System.out.println("selektivity is " + (double)rus.getTotalOutputAmount() / rus.getTotalInputAmount()); + + } + if(rus.getTotalInputAmount() != 0 && ((double)rus.getTotalOutputAmount() / rus.getTotalInputAmount() > 2.0)){ + //size off checkpoint would be to large: do not checkpoint + //TODO progress estimation would make sense here + LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double)rus.getTotalOutputAmount()/ rus.getTotalInputAmount() > 2.0)); + return false; + + } + if(rus.getTotalInputAmount() != 0 && ((double)rus.getTotalOutputAmount() / rus.getTotalInputAmount() < 0.6)){ + //size of checkpoint will be small enough: checkpoint + //TODO progress estimation would make sense here + LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double)rus.getTotalOutputAmount()/ rus.getTotalInputAmount() > 2.0)); + return true; + + } + + + } } + //between thresholds check CPU Usage. + if (rus.getUserCPU() >= 90) { + System.out.println(task.getEnvironment().getTaskName() + "CPU-Bottleneck"); + // CPU bottleneck + return true; + } + + + + } else { + System.out.println("Checkpoint decision was forced"); + // checkpoint decision was forced by the user + return rus.getForced(); } + + //in case of doubt do not checkpoint + return false; + } public void checkpointStateChanged(final JobID jobID, final ExecutionVertexID id, @@ -894,7 +987,7 @@ public void sendDataToJobManager(final PluginID pluginID, final IOReadableWritab * thrown if an I/O error occurs during the RPC call */ public IOReadableWritable requestDataFromJobManager(final PluginID pluginID, final IOReadableWritable data) - throws IOException { + throws IOException { synchronized (this.pluginCommunicationService) { return this.pluginCommunicationService.requestData(pluginID, data); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index 84144eeca7fdf..818235dabb7ad 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -351,7 +351,7 @@ public void initialExecutionResourcesExhausted() { long averageInputRecordSize = 0; for (int i = 0; i < this.environment.getNumberOfInputGates(); ++i) { final InputGate inputGate = this.environment.getInputGate(i); - numrec += inputGate.getNumRecords(); + numinrec += inputGate.getNumRecords(); for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); channelUtilization.put(inputChannel.getID(), @@ -387,7 +387,7 @@ public void initialExecutionResourcesExhausted() { } final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, channelUtilization, userCPU, - force, totalInputAmount, totalOutputAmount, averageOutputRecordSize, averageInputRecordSize); + force, totalInputAmount, totalOutputAmount, averageOutputRecordSize, averageInputRecordSize, getPACTInputOutputRatio(), allClosed); // Notify the listener objects final Iterator it = this.registeredListeners.iterator(); From 9f5a8a63f6ae1baa7b6ece839112364e2e5d2455 Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 21 Feb 2012 15:39:42 +0100 Subject: [PATCH 257/310] somewhat dirty fix for checkpoint annotation from pact layer --- .../stratosphere/nephele/execution/Environment.java | 4 ++++ .../nephele/checkpointing/CheckpointDecision.java | 2 +- .../nephele/checkpointing/CheckpointEnvironment.java | 9 +++++++++ .../execution/ResourceUtilizationSnapshot.java | 1 + .../nephele/execution/RuntimeEnvironment.java | 12 ++++++++++++ .../nephele/taskmanager/TaskManager.java | 4 ++-- .../nephele/taskmanager/runtime/RuntimeTask.java | 9 +++++++++ .../stratosphere/pact/runtime/task/CoGroupTask.java | 5 ++++- .../stratosphere/pact/runtime/task/CombineTask.java | 5 ++++- .../pact/runtime/task/DataSourceTask.java | 4 ++++ .../eu/stratosphere/pact/runtime/task/MapTask.java | 5 ++++- .../eu/stratosphere/pact/runtime/task/MatchTask.java | 5 ++++- .../stratosphere/pact/runtime/task/ReduceTask.java | 5 ++++- 13 files changed, 62 insertions(+), 8 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java index 3477618414c27..721c4ad90fbd0 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Environment.java @@ -237,4 +237,8 @@ InputGate createInputGate(GateID gateID, RecordDeserializer getInputChannelIDsOfGate(GateID gateID); + + void isForced(boolean force); + + Boolean getForced(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java index 465a25cf05543..c0a18234f5b28 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java @@ -42,7 +42,7 @@ public final class CheckpointDecision implements IOReadableWritable { * @param checkpointRequired * true to indicate the checkpoint shall be materialized, false to discard it */ - CheckpointDecision(final ExecutionVertexID vertexID, final boolean checkpointRequired) { + public CheckpointDecision(final ExecutionVertexID vertexID, final boolean checkpointRequired) { this.vertexID = vertexID; this.checkpointRequired = checkpointRequired; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java index cbdc22628c1f7..863260ff43b15 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java @@ -310,4 +310,13 @@ public void reportPACTDataStatistics(final long numberOfConsumedBytes, final lon throw new IllegalStateException("reportPACTDataStatistics called on CheckpointEnvironment"); } // DW: End of temporary code + + @Override + public void isForced(boolean force) { + } + + @Override + public Boolean getForced() { + return null; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java index 1d2bbc6eae409..6893c60faae88 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java @@ -83,6 +83,7 @@ public ResourceUtilizationSnapshot(final long timestamp, final Map Date: Tue, 21 Feb 2012 17:16:15 +0100 Subject: [PATCH 258/310] made lower and upper bound configurable --- .../nephele/taskmanager/TaskManager.java | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index d011bfa54243d..060102b1a2db9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -148,6 +148,10 @@ public class TaskManager implements TaskOperationProtocol, PluginCommunicationPr */ private boolean isShutDown = false; + private final double CPupper; + + private final double CPlower; + /** * Constructs a new task manager, starts its IPC service and attempts to discover the job manager to * receive an initial configuration. @@ -325,6 +329,9 @@ public TaskManager(String configDir) throws Exception { // Add shutdown hook for clean up tasks Runtime.getRuntime().addShutdownHook(new TaskManagerCleanUp(this)); + + this.CPupper = Double.parseDouble(GlobalConfiguration.getString("checkpoint.upperbound","0.9")); + this.CPlower = Double.parseDouble(GlobalConfiguration.getString("checkpoint.lowerbound","1.5")); } /** @@ -741,11 +748,11 @@ private boolean getDecision(final Task task, final ResourceUtilizationSnapshot r if(rus.getPactRatio() != -1){ System.out.println("Ratio = " + rus.getPactRatio()); - if(rus.getPactRatio()>=5.0){ + if(rus.getPactRatio()>=this.CPlower){ //amount of data is small so we checkpoint return true; } - if(rus.getPactRatio()<=0.6){ + if(rus.getPactRatio()<=this.CPupper){ //amount of data is too big return false; } @@ -754,17 +761,17 @@ private boolean getDecision(final Task task, final ResourceUtilizationSnapshot r if(rus.isDam()){ System.out.println("is Dam"); - if(rus.getAverageInputRecordSize() != 0){ - System.out.println( "avg ratio" + rus.getAverageOutputRecordSize()/rus.getAverageInputRecordSize()); + if(rus.getAverageOutputRecordSize() != 0){ + System.out.println( "avg ratio" + rus.getAverageInputRecordSize()/rus.getAverageOutputRecordSize()); } - if(rus.getAverageInputRecordSize() != 0 && - rus.getAverageOutputRecordSize()/rus.getAverageInputRecordSize() < 0.6){ + if(rus.getAverageOutputRecordSize() != 0 && + rus.getAverageInputRecordSize()/rus.getAverageOutputRecordSize() >=this.CPlower){ return true; } - if(rus.getAverageInputRecordSize() != 0 && - rus.getAverageOutputRecordSize()/rus.getAverageInputRecordSize() > 2.0){ + if(rus.getAverageOutputRecordSize() != 0 && + rus.getAverageInputRecordSize()/rus.getAverageOutputRecordSize() <=this.CPupper){ return false; } }else{ @@ -777,17 +784,17 @@ private boolean getDecision(final Task task, final ResourceUtilizationSnapshot r System.out.println("selektivity is " + (double)rus.getTotalOutputAmount() / rus.getTotalInputAmount()); } - if(rus.getTotalInputAmount() != 0 && ((double)rus.getTotalOutputAmount() / rus.getTotalInputAmount() > 2.0)){ + if(rus.getTotalOutputAmount() != 0 && ((double)rus.getTotalInputAmount() / rus.getTotalOutputAmount() >=this.CPlower)){ //size off checkpoint would be to large: do not checkpoint //TODO progress estimation would make sense here - LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double)rus.getTotalOutputAmount()/ rus.getTotalInputAmount() > 2.0)); + LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double)rus.getTotalInputAmount()/ rus.getTotalOutputAmount() > 2.0)); return false; } - if(rus.getTotalInputAmount() != 0 && ((double)rus.getTotalOutputAmount() / rus.getTotalInputAmount() < 0.6)){ + if(rus.getTotalOutputAmount() != 0 && ((double)rus.getTotalInputAmount() / rus.getTotalOutputAmount() <=this.CPupper)){ //size of checkpoint will be small enough: checkpoint //TODO progress estimation would make sense here - LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double)rus.getTotalOutputAmount()/ rus.getTotalInputAmount() > 2.0)); + LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double)rus.getTotalInputAmount()/ rus.getTotalOutputAmount() > 2.0)); return true; } From c8de5b9ad11c54e5918d74dfe797e09600c81274 Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 21 Feb 2012 17:27:35 +0100 Subject: [PATCH 259/310] removed some debugg syso --- .../java/eu/stratosphere/nephele/taskmanager/TaskManager.java | 1 - .../stratosphere/nephele/taskmanager/runtime/RuntimeTask.java | 2 -- 2 files changed, 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 060102b1a2db9..2f4ad9d0feb19 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -742,7 +742,6 @@ public void initialExecutionResourcesExhausted(final JobID jobID, final Executio } private boolean getDecision(final Task task, final ResourceUtilizationSnapshot rus) { - System.out.println("rus force " + rus.getForced()); if(rus.getForced() == null){ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index 6f383ddb4ebea..067f212a4b9f7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -379,10 +379,8 @@ public void initialExecutionResourcesExhausted() { // Don't checkpoint stateful tasks force = false; } else { - System.out.println(this.environment.getInvokable().getClass().getAnnotation(ForceCheckpoint.class)); if(this.environment.getForced() != null){ force = this.environment.getForced(); - System.out.println("force " + force); }else{ // look for a forced decision from the user ForceCheckpoint forced = this.environment.getInvokable().getClass().getAnnotation(ForceCheckpoint.class); From 11359905e84ee48fda9541af1ae0101a1d747c32 Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 21 Feb 2012 21:30:53 +0100 Subject: [PATCH 260/310] added some checkpoint configuration --- .../nephele/taskmanager/TaskManager.java | 24 +++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 2f4ad9d0feb19..2e0b1d90273f1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -151,6 +151,12 @@ public class TaskManager implements TaskOperationProtocol, PluginCommunicationPr private final double CPupper; private final double CPlower; + + private final boolean usePACT; + + private final boolean useAVG; + + private final boolean noCheckpointing; /** * Constructs a new task manager, starts its IPC service and attempts to discover the job manager to @@ -332,6 +338,14 @@ public TaskManager(String configDir) throws Exception { this.CPupper = Double.parseDouble(GlobalConfiguration.getString("checkpoint.upperbound","0.9")); this.CPlower = Double.parseDouble(GlobalConfiguration.getString("checkpoint.lowerbound","1.5")); + this.usePACT = GlobalConfiguration.getBoolean("checkpoint.usepact",false); + this.useAVG = GlobalConfiguration.getBoolean("checkpoint.useavg",false); + this.noCheckpointing = GlobalConfiguration.getBoolean("checkpoint.no",false); + LOG.info("Checkpointing Summary: UpperBound=" + this.CPupper + " LowerBound=" + this.CPlower + + " ForcedValues: usePACT=" + this.usePACT + " useAVG=" + this.useAVG + + " NOCheckpoting="+this.noCheckpointing); + + } /** @@ -742,10 +756,12 @@ public void initialExecutionResourcesExhausted(final JobID jobID, final Executio } private boolean getDecision(final Task task, final ResourceUtilizationSnapshot rus) { + if(this.noCheckpointing){ + return false; + } + if(rus.getForced() == null){ - - - if(rus.getPactRatio() != -1){ + if(rus.getPactRatio() != -1 || this.usePACT){ System.out.println("Ratio = " + rus.getPactRatio()); if(rus.getPactRatio()>=this.CPlower){ //amount of data is small so we checkpoint @@ -757,7 +773,7 @@ private boolean getDecision(final Task task, final ResourceUtilizationSnapshot r } }else{ //no info from upper layer so use average sizes - if(rus.isDam()){ + if(rus.isDam() || this.useAVG){ System.out.println("is Dam"); if(rus.getAverageOutputRecordSize() != 0){ From b7daec08868f0cdf58cd6d7f07eb783029884c9f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 21 Feb 2012 22:07:00 +0100 Subject: [PATCH 261/310] Fixed problem with data collection in PACT runtime --- .../pact/common/stubs/Collector.java | 2 ++ .../hash/BuildFirstHashMatchIterator.java | 12 +++---- .../hash/BuildSecondHashMatchIterator.java | 10 +++--- .../sort/CombiningUnilateralSortMerger.java | 35 +++++++++++++++++++ .../runtime/sort/UnilateralSortMerger.java | 17 +++++++++ .../task/chaining/ChainedCombineTask.java | 17 +++++++++ .../runtime/task/chaining/ChainedMapTask.java | 20 ++++++++++- .../test/util/DiscardingOutputCollector.java | 8 +++++ 8 files changed, 106 insertions(+), 15 deletions(-) diff --git a/pact/pact-common/src/main/java/eu/stratosphere/pact/common/stubs/Collector.java b/pact/pact-common/src/main/java/eu/stratosphere/pact/common/stubs/Collector.java index e438f6254fac4..c64905294ede8 100644 --- a/pact/pact-common/src/main/java/eu/stratosphere/pact/common/stubs/Collector.java +++ b/pact/pact-common/src/main/java/eu/stratosphere/pact/common/stubs/Collector.java @@ -37,4 +37,6 @@ public interface Collector * Closes the collector. */ void close(); + + long getCollectedPactRecordsInBytes(); } diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java index 82d5d2a3bae7f..c466d948a3ee3 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildFirstHashMatchIterator.java @@ -30,7 +30,6 @@ import eu.stratosphere.pact.common.type.PactRecord; import eu.stratosphere.pact.common.util.MutableObjectIterator; import eu.stratosphere.pact.runtime.task.util.MatchTaskIterator; -import eu.stratosphere.pact.runtime.task.util.OutputCollector; /** @@ -108,9 +107,6 @@ public void close() public boolean callWithNextKey(MatchStub matchFunction, Collector collector) throws Exception { - // DW: Start of temporary code - final OutputCollector oc = (OutputCollector) collector; - // DW: End of temporary code if (this.hashJoin.nextRecord()) { @@ -137,7 +133,7 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) matchFunction.match(nextBuildSidePair, probeRecord, collector); // DW: Start of temporary code this.environment.reportPACTDataStatistics(r1 + r2, - oc.getCollectedPactRecordsInBytes()); + collector.getCollectedPactRecordsInBytes()); // DW: End of temporary code // call match on the second pair @@ -152,7 +148,7 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) matchFunction.match(tmpPair, probeRecord, collector); // DW: Start of temporary code this.environment.reportPACTDataStatistics(r1 + r2, - oc.getCollectedPactRecordsInBytes()); + collector.getCollectedPactRecordsInBytes()); // DW: End of temporary code tmpPair = new PactRecord(); @@ -169,7 +165,7 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) matchFunction.match(tmpPair, probeRecord, collector); // DW: Start of temporary code this.environment.reportPACTDataStatistics(r1 + r2, - oc.getCollectedPactRecordsInBytes()); + collector.getCollectedPactRecordsInBytes()); // DW: End of temporary code tmpPair = new PactRecord(); } @@ -187,7 +183,7 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) matchFunction.match(nextBuildSidePair, probeRecord, collector); // DW: Start of temporary code this.environment.reportPACTDataStatistics(r1 + r2, - oc.getCollectedPactRecordsInBytes()); + collector.getCollectedPactRecordsInBytes()); // DW: End of temporary code } } diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java index 572c2b2e04a64..243a068fe6312 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/hash/BuildSecondHashMatchIterator.java @@ -30,7 +30,6 @@ import eu.stratosphere.pact.common.type.PactRecord; import eu.stratosphere.pact.common.util.MutableObjectIterator; import eu.stratosphere.pact.runtime.task.util.MatchTaskIterator; -import eu.stratosphere.pact.runtime.task.util.OutputCollector; /** @@ -103,7 +102,6 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) { // DW: Start of temporary code final Environment env = this.environment; - final OutputCollector oc = (OutputCollector) collector; // DW: End of temporary code @@ -132,7 +130,7 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) matchFunction.match(probeRecord, nextBuildSidePair, collector); // DW: Start of temporary code env.reportPACTDataStatistics(r1 + r2, - oc.getCollectedPactRecordsInBytes()); + collector.getCollectedPactRecordsInBytes()); // DW: End of temporary code // call match on the second pair @@ -147,7 +145,7 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) matchFunction.match(probeRecord, tmpPair, collector); // DW: Start of temporary code env.reportPACTDataStatistics(r1 + r2, - oc.getCollectedPactRecordsInBytes()); + collector.getCollectedPactRecordsInBytes()); // DW: End of temporary code tmpPair = new PactRecord(); @@ -164,7 +162,7 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) matchFunction.match(probeRecord, tmpPair, collector); // DW: Start of temporary code env.reportPACTDataStatistics(r1 + r2, - oc.getCollectedPactRecordsInBytes()); + collector.getCollectedPactRecordsInBytes()); // DW: End of temporary code tmpPair = new PactRecord(); } @@ -182,7 +180,7 @@ public boolean callWithNextKey(MatchStub matchFunction, Collector collector) matchFunction.match(probeRecord, nextBuildSidePair, collector); // DW: Start of temporary code env.reportPACTDataStatistics(r1 + r2, - oc.getCollectedPactRecordsInBytes()); + collector.getCollectedPactRecordsInBytes()); // DW: End of temporary code } } diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMerger.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMerger.java index 6edb27bd70c10..20f501953a633 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMerger.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/CombiningUnilateralSortMerger.java @@ -612,6 +612,11 @@ private WriterCollector(ChannelWriterOutputView output) { */ @Override public void collect(PactRecord record) { + + // DW: Start of temporary code + this.collectedPactRecordsInBytes += record.getBinaryLength(); + // DW: End of temporary code + try { record.write(this.output); } @@ -627,6 +632,18 @@ public void collect(PactRecord record) { @Override public void close() {} + // DW: Start of temporary code + private long collectedPactRecordsInBytes = 0L; + + @Override + public long getCollectedPactRecordsInBytes() { + + final long retVal = this.collectedPactRecordsInBytes; + this.collectedPactRecordsInBytes = 0L; + + return retVal; + } + // DW: End of temporary code } // ------------------------------------------------------------------------ @@ -653,6 +670,11 @@ private ListCollector(ArrayList list) { */ @Override public void collect(PactRecord record) { + + // DW: Start of temporary code + this.collectedPactRecordsInBytes += record.getBinaryLength(); + // DW: End of temporary code + this.list.add(record.createCopy()); } @@ -665,6 +687,19 @@ public void collect(PactRecord record) { public void close() { // does nothing } + + // DW: Start of temporary code + private long collectedPactRecordsInBytes = 0L; + + @Override + public long getCollectedPactRecordsInBytes() { + + final long retVal = this.collectedPactRecordsInBytes; + this.collectedPactRecordsInBytes = 0L; + + return retVal; + } + // DW: End of temporary code } // ------------------------------------------------------------------------ diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/UnilateralSortMerger.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/UnilateralSortMerger.java index 0250f2b01f68c..02735439839f0 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/UnilateralSortMerger.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/sort/UnilateralSortMerger.java @@ -1706,6 +1706,10 @@ private void grabBuffer() @Override public void collect(PactRecord record) { + // DW: Start of temporary code + this.collectedPactRecordsInBytes += record.getBinaryLength(); + // DW: End of temporary code + try { if (this.spillingInThisBuffer) { if (this.currentBuffer.write(record)) { @@ -1803,5 +1807,18 @@ public void close() this.queues.sort.add(SENTINEL); } } + + // DW: Start of temporary code + private long collectedPactRecordsInBytes = 0L; + + @Override + public long getCollectedPactRecordsInBytes() { + + final long retVal = this.collectedPactRecordsInBytes; + this.collectedPactRecordsInBytes = 0L; + + return retVal; + } + // DW: End of temporary code } } \ No newline at end of file diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ChainedCombineTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ChainedCombineTask.java index 6385b3011da47..1e979f867b4fe 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ChainedCombineTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ChainedCombineTask.java @@ -223,6 +223,10 @@ public String getTaskName() { @Override public void collect(PactRecord record) { + // DW: Start of temporary code + this.collectedPactRecordsInBytes += record.getBinaryLength(); + // DW: End of temporary code + if (this.exception != null) throw new RuntimeException("The combiner failed due to an exception.", this.exception.getCause() == null ? this.exception : this.exception.getCause()); @@ -305,4 +309,17 @@ public void cancel() { this.running = false; } } + + // DW: Start of temporary code + private long collectedPactRecordsInBytes = 0L; + + @Override + public long getCollectedPactRecordsInBytes() { + + final long retVal = this.collectedPactRecordsInBytes; + this.collectedPactRecordsInBytes = 0L; + + return retVal; + } + // DW: End of temporary code } \ No newline at end of file diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ChainedMapTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ChainedMapTask.java index 8e85a0fd7eca2..4583bbfd92263 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ChainedMapTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ChainedMapTask.java @@ -122,9 +122,14 @@ public String getTaskName() { @Override public void collect(PactRecord record) { + // DW: Start of temporary code + final int recordLength = record.getBinaryLength(); + this.collectedPactRecordsInBytes += recordLength; + // DW: End of temporary code + try { // DW: Start of temporary code - this.consumedPactRecordsInBytes += record.getBinaryLength(); + this.consumedPactRecordsInBytes += recordLength; // DW: End of temporary code this.mapper.map(record, this.collector); // DW: Start of temporary code @@ -151,4 +156,17 @@ public void close() { this.collector.close(); } + + // DW: Start of temporary code + private long collectedPactRecordsInBytes = 0L; + + @Override + public long getCollectedPactRecordsInBytes() { + + final long retVal = this.collectedPactRecordsInBytes; + this.collectedPactRecordsInBytes = 0L; + + return retVal; + } + // DW: End of temporary code } diff --git a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DiscardingOutputCollector.java b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DiscardingOutputCollector.java index f1e7ac2a8525d..8ebc4c87171c9 100644 --- a/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DiscardingOutputCollector.java +++ b/pact/pact-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/DiscardingOutputCollector.java @@ -26,4 +26,12 @@ public void collect(PactRecord rec) {} @Override public void close() {} + + // DW: Start of temporary code + @Override + public long getCollectedPactRecordsInBytes() { + // TODO Auto-generated method stub + return 0; + } + // DW: End of temporary code } From 4342adc4c557a03095a966a29063be46e4908292 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 21 Feb 2012 22:14:43 +0100 Subject: [PATCH 262/310] Marked temporary code properly --- .../main/java/eu/stratosphere/pact/common/stubs/Collector.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pact/pact-common/src/main/java/eu/stratosphere/pact/common/stubs/Collector.java b/pact/pact-common/src/main/java/eu/stratosphere/pact/common/stubs/Collector.java index c64905294ede8..1bc9f494df283 100644 --- a/pact/pact-common/src/main/java/eu/stratosphere/pact/common/stubs/Collector.java +++ b/pact/pact-common/src/main/java/eu/stratosphere/pact/common/stubs/Collector.java @@ -38,5 +38,7 @@ public interface Collector */ void close(); + // DW: Start of temporary code long getCollectedPactRecordsInBytes(); + // DW: End of temporary code } From 7956ed21a29a08d0fd8c0129880316508760e700 Mon Sep 17 00:00:00 2001 From: marrus Date: Tue, 21 Feb 2012 22:54:14 +0100 Subject: [PATCH 263/310] changed configuration of checkpoints --- .../java/eu/stratosphere/nephele/taskmanager/TaskManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 2e0b1d90273f1..62a92ce8f1a43 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -761,7 +761,7 @@ private boolean getDecision(final Task task, final ResourceUtilizationSnapshot r } if(rus.getForced() == null){ - if(rus.getPactRatio() != -1 || this.usePACT){ + if((rus.getPactRatio() != -1 || this.usePACT)&& !this.useAVG){ System.out.println("Ratio = " + rus.getPactRatio()); if(rus.getPactRatio()>=this.CPlower){ //amount of data is small so we checkpoint From f5417d735cd7589e6848c268407495b685050a7c Mon Sep 17 00:00:00 2001 From: marrus Date: Wed, 22 Feb 2012 10:29:21 +0100 Subject: [PATCH 264/310] added always checkpointing flag --- .../eu/stratosphere/nephele/taskmanager/TaskManager.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 62a92ce8f1a43..c4b5e273e90a4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -158,6 +158,8 @@ public class TaskManager implements TaskOperationProtocol, PluginCommunicationPr private final boolean noCheckpointing; + private boolean alwaysCheckpointing; + /** * Constructs a new task manager, starts its IPC service and attempts to discover the job manager to * receive an initial configuration. @@ -341,9 +343,10 @@ public TaskManager(String configDir) throws Exception { this.usePACT = GlobalConfiguration.getBoolean("checkpoint.usepact",false); this.useAVG = GlobalConfiguration.getBoolean("checkpoint.useavg",false); this.noCheckpointing = GlobalConfiguration.getBoolean("checkpoint.no",false); + this.alwaysCheckpointing = GlobalConfiguration.getBoolean("checkpoint.always",false); LOG.info("Checkpointing Summary: UpperBound=" + this.CPupper + " LowerBound=" + this.CPlower + " ForcedValues: usePACT=" + this.usePACT + " useAVG=" + this.useAVG - + " NOCheckpoting="+this.noCheckpointing); + + " NOCheckpoting="+this.noCheckpointing + " AlwaysCheckpointing=" + this.alwaysCheckpointing); } @@ -759,7 +762,9 @@ private boolean getDecision(final Task task, final ResourceUtilizationSnapshot r if(this.noCheckpointing){ return false; } - + if(this.alwaysCheckpointing){ + return true; + } if(rus.getForced() == null){ if((rus.getPactRatio() != -1 || this.usePACT)&& !this.useAVG){ System.out.println("Ratio = " + rus.getPactRatio()); From f35feb9db91a7db3598591a03f88d04cddf6746c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 22 Feb 2012 13:37:36 +0100 Subject: [PATCH 265/310] Refactored code for checkpoint decision logic --- .../checkpointing/CheckpointDecision.java | 171 ++++++++++-------- .../CheckpointDecisionCoordinator.java | 162 ----------------- .../CheckpointDecisionPropagator.java | 38 ---- .../CheckpointExecutionListener.java | 114 ------------ .../checkpointing/CheckpointUtils.java | 44 +++++ .../nephele/execution/ExecutionListener.java | 14 -- .../executiongraph/ExecutionGraph.java | 11 -- .../executiongraph/ExecutionVertex.java | 12 -- .../nephele/instance/AbstractInstance.java | 7 - .../nephele/jobmanager/EventCollector.java | 10 - .../nephele/jobmanager/JobManager.java | 85 +-------- .../scheduler/AbstractExecutionListener.java | 9 - .../nephele/protocols/JobManagerProtocol.java | 18 -- .../protocols/TaskOperationProtocol.java | 3 - .../nephele/taskmanager/TaskManager.java | 153 +--------------- .../ByteBufferedChannelManager.java | 17 -- .../runtime/EnvelopeConsumptionLog.java | 9 +- .../taskmanager/runtime/RuntimeTask.java | 131 +------------- .../runtime/RuntimeTaskContext.java | 134 +++++++++++++- 19 files changed, 283 insertions(+), 859 deletions(-) delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionPropagator.java delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointExecutionListener.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java index c0a18234f5b28..ad365a29f223c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java @@ -15,82 +15,101 @@ package eu.stratosphere.nephele.checkpointing; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.io.IOReadableWritable; - -public final class CheckpointDecision implements IOReadableWritable { - - /** - * The ID of the vertex the checkpoint decision applies to. - */ - private final ExecutionVertexID vertexID; - - /** - * The checkpoint decision itself. - */ - private boolean checkpointRequired = false; - - /** - * Constructs a new checkpoint decision object. - * - * @param vertexID - * the ID of the vertex the checkpoint decision applies to - * @param checkpointRequired - * true to indicate the checkpoint shall be materialized, false to discard it - */ - public CheckpointDecision(final ExecutionVertexID vertexID, final boolean checkpointRequired) { - this.vertexID = vertexID; - this.checkpointRequired = checkpointRequired; - } - - /** - * Default constructor required for serialized/deserialization. - */ - public CheckpointDecision() { - this.vertexID = new ExecutionVertexID(); - } - - /** - * {@inheritDoc} - */ - @Override - public void write(final DataOutput out) throws IOException { - - this.vertexID.write(out); - out.writeBoolean(this.checkpointRequired); - } - - /** - * {@inheritDoc} - */ - @Override - public void read(final DataInput in) throws IOException { - - this.vertexID.read(in); - this.checkpointRequired = in.readBoolean(); - } - - /** - * Returns the ID of the vertex the checkpoint decision applies to. - * - * @return the ID of the vertex the checkpoint decision applies to - */ - public ExecutionVertexID getVertexID() { - - return this.vertexID; - } - - /** - * Returns the checkpoint decision itself. - * - * @return true to indicate that the checkpoint shall be materialized, false to discard it - */ - public boolean getCheckpointDecision() { +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; +import eu.stratosphere.nephele.taskmanager.Task; + +public final class CheckpointDecision { + + private static final Log LOG = LogFactory.getLog(CheckpointDecision.class); + + public static boolean getDecision(final Task task, final ResourceUtilizationSnapshot rus) { + + if (CheckpointUtils.isCheckpointingDisabled()) { + return false; + } + + final double CPlower = CheckpointUtils.getCPLower(); + + final double CPupper = CheckpointUtils.getCPUpper(); + + if (rus.getForced() == null) { + if (rus.getPactRatio() != -1 && CheckpointUtils.usePACT()) { + System.out.println("Ratio = " + rus.getPactRatio()); + if (rus.getPactRatio() >= CPlower) { + // amount of data is small so we checkpoint + return true; + } + if (rus.getPactRatio() <= CPupper) { + // amount of data is too big + return false; + } + } else { + // no info from upper layer so use average sizes + if (rus.isDam() && CheckpointUtils.useAVG()) { + System.out.println("is Dam"); + + if (rus.getAverageOutputRecordSize() != 0) { + System.out.println("avg ratio" + rus.getAverageInputRecordSize() + / rus.getAverageOutputRecordSize()); + } + + if (rus.getAverageOutputRecordSize() != 0 && + rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize() >= CPlower) { + return true; + } + + if (rus.getAverageOutputRecordSize() != 0 && + rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize() <= CPupper) { + return false; + } + } else { + + // we have no data dam so we can estimate the input/output-ratio + System.out.println("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); + if (rus.getTotalInputAmount() != 0) { + System.out.println("selektivity is " + (double) rus.getTotalOutputAmount() + / rus.getTotalInputAmount()); + + } + if (rus.getTotalOutputAmount() != 0 + && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() >= CPlower)) { + // size off checkpoint would be to large: do not checkpoint + // TODO progress estimation would make sense here + LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() > 2.0)); + return false; + + } + if (rus.getTotalOutputAmount() != 0 + && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() <= CPupper)) { + // size of checkpoint will be small enough: checkpoint + // TODO progress estimation would make sense here + LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() > 2.0)); + return true; + + } + + } + } + // between thresholds check CPU Usage. + if (rus.getUserCPU() >= 90) { + System.out.println(task.getEnvironment().getTaskName() + "CPU-Bottleneck"); + // CPU bottleneck + return true; + } + + } else { + System.out.println("Checkpoint decision was forced too " + rus.getForced()); + // checkpoint decision was forced by the user + return rus.getForced(); + } + + // in case of doubt do not checkpoint + return false; - return this.checkpointRequired; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java deleted file mode 100644 index 5fe18bca65db9..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionCoordinator.java +++ /dev/null @@ -1,162 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.checkpointing; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; -import eu.stratosphere.nephele.executiongraph.CheckpointState; -import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; -import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.instance.AbstractInstance; -import eu.stratosphere.nephele.util.SerializableArrayList; - -/** - * The checkpoint decision coordinator is the central instance to make a coordinated checkpoint decision. Incoming jobs - * are registered with this decision coordinator. As a result, the decision coordinator follows the resource exhaustion - * of the tasks included in the registered job and eventually decides if it is beneficial to materialized - * {@link EphemeralCheckpoint} objects or to discard them. - *

- * This class is thread-safe. - * - * @author warneke - */ -public final class CheckpointDecisionCoordinator { - - /** - * The log object used to report errors and information in general. - */ - private static final Log LOG = LogFactory.getLog(CheckpointDecisionCoordinator.class); - - /** - * The object in charge of propagating checkpoint decisions to the respective task managers. - */ - private final CheckpointDecisionPropagator decisionPropagator; - - private final Set decidedVertices; - - /** - * Constructs a new checkpoint decision coordinator. - * - * @param decisionPropagator - * the object in charge of propagating the checkpoint decisions to the respective task managers. - */ - public CheckpointDecisionCoordinator(final CheckpointDecisionPropagator decisionPropagator) { - this.decisionPropagator = decisionPropagator; - - this.decidedVertices = Collections.newSetFromMap(new ConcurrentHashMap()); - } - - /** - * Registers a new {@link ExecutionGraph} with the checkpoint decision coordinator. As a result of this operation, - * the checkpoint decision coordinator will receive events about resource exhaustion of the tasks included in this - * job. - * - * @param executionGraph - * the job to register - */ - public void registerJob(final ExecutionGraph executionGraph) { - final Iterator it = new ExecutionGraphIterator(executionGraph, true); - while (it.hasNext()) { - final ExecutionVertex vertex = it.next(); - vertex.registerExecutionListener(new CheckpointExecutionListener(this, vertex)); - } - } - - /** - * Computes a checkpoint decision for the given {@link ExecutionVertex}. - * - * @param vertex - * the vertex which requires a checkpoint decision - * @param rus - * the current resource utilization of the vertex - */ - void checkpointDecisionRequired(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { - - LOG.info("Checkpoint decision for vertex " + vertex + " required"); - boolean checkpointDecision = getDecision(vertex, rus); - if(checkpointDecision) - LOG.info("Creating Checkpoint for " + vertex.getEnvironment().getTaskNameWithIndex() ); - - - - final ExecutionGraph graph = vertex.getExecutionGraph(); - final Map> checkpointDecisions = new HashMap>(); - final List checkpointDecisionList = new SerializableArrayList(); - - synchronized (graph) { - checkpointDecisionList.add(new CheckpointDecision(vertex.getID(), checkpointDecision)); - checkpointDecisions.put(vertex.getAllocatedResource().getInstance(), checkpointDecisionList); - } - - // Propagate checkpoint decisions - this.decisionPropagator.propagateCheckpointDecisions(checkpointDecisions); - } - - private boolean getDecision(final ExecutionVertex vertex, final ResourceUtilizationSnapshot rus) { - - if(rus.getForced() == null){ - if(rus.getTotalInputAmount() != 0 ){ - LOG.info("selektivity is " + (double)rus.getTotalOutputAmount() / rus.getTotalInputAmount()); - LOG.info("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); - } - if(rus.getTotalInputAmount() != 0 && ((double)rus.getTotalOutputAmount() / rus.getTotalInputAmount() > 2.0)){ - //estimated size of checkpoint - //TODO progress estimation would make sense here - LOG.info(vertex.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double)rus.getTotalOutputAmount()/ rus.getTotalInputAmount() > 2.0)); - return false; - - } - if (rus.getUserCPU() >= 90) { - LOG.info(vertex.getEnvironment().getTaskName() + "CPU-Bottleneck"); - // CPU bottleneck - return true; - } - - if (vertex.getNumberOfSuccessors() != 0 - && vertex.getNumberOfPredecessors() * 1.0 / vertex.getNumberOfSuccessors() > 1.5) { - - LOG.info(vertex.getEnvironment().getTaskName() + " vertex.getNumberOfPredecessors() " - + vertex.getNumberOfPredecessors() + " / vertex.getNumberOfSuccessors() " - + vertex.getNumberOfSuccessors() + " > 1.5"); - // less output-channels than input-channels - // checkpoint at this position probably saves network-traffic - return true; - } - - } else { - LOG.info("Checkpoint decision was forced"); - // checkpoint decision was forced by the user - return rus.getForced(); - } - - return false; - - } - -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionPropagator.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionPropagator.java deleted file mode 100644 index 3c18652614a9c..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionPropagator.java +++ /dev/null @@ -1,38 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.checkpointing; - -import java.util.List; -import java.util.Map; - -import eu.stratosphere.nephele.instance.AbstractInstance; - -/** - * A checkpoint decision propagator is responsible for propagating a coordinated checkpoint decision to the individual - * {@link Task} objects running on the different {@link TaskManager} instances. - * - * @author warneke - */ -public interface CheckpointDecisionPropagator { - - /** - * Propagates a list of checkpoint decisions to the respective {@link TaskManager} instances. - * - * @param checkpointDecisions - * a map with checkpoint decisions divided by task managers - */ - void propagateCheckpointDecisions(Map> checkpointDecisions); -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointExecutionListener.java deleted file mode 100644 index f8f707f159b5e..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointExecutionListener.java +++ /dev/null @@ -1,114 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) - * - * 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 eu.stratosphere.nephele.checkpointing; - -import eu.stratosphere.nephele.execution.ExecutionListener; -import eu.stratosphere.nephele.execution.ExecutionState; -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.jobgraph.JobID; - -/** - * This class implements a {@link ExecutionListener} designed to receive notifications about the initial resource - * exhaustion of a task. The notification is then forwarded to the {@link CheckpointDecisionCoordinator}. - *

- * This class is thread-safe. - * - * @author warneke - */ -public final class CheckpointExecutionListener implements ExecutionListener { - - /** - * The decision coordinator that will receive notifications about a vertex's resource exhaustion. - */ - private final CheckpointDecisionCoordinator decisionCoordinator; - - /** - * The execution vertex this listener has been created for. - */ - private final ExecutionVertex executionVertex; - - /** - * Constructs a new checkpoint execution listener. - * - * @param decisionCoordinator - * the decision coordinator - * @param executionVertex - * the execution vertex this listener is created for - */ - CheckpointExecutionListener(final CheckpointDecisionCoordinator decisionCoordinator, - final ExecutionVertex executionVertex) { - - if (decisionCoordinator == null) { - throw new IllegalArgumentException("Argument decisionCoordinator must not be null"); - } - - if (executionVertex == null) { - throw new IllegalArgumentException("Argument executionVertex must not be null"); - } - - this.decisionCoordinator = decisionCoordinator; - this.executionVertex = executionVertex; - } - - /** - * {@inheritDoc} - */ - @Override - public void executionStateChanged(final JobID jobID, final ExecutionVertexID vertexID, - final ExecutionState newExecutionState, final String optionalMessage) { - // Nothing to do here - } - - /** - * {@inheritDoc} - */ - @Override - public void userThreadStarted(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) { - // Nothing to do here - - } - - /** - * {@inheritDoc} - */ - @Override - public void userThreadFinished(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) { - // Nothing to do here - } - - /** - * {@inheritDoc} - */ - @Override - public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID vertexID, - final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { - - // Trigger checkpoint decision - this.decisionCoordinator.checkpointDecisionRequired(this.executionVertex, resourceUtilizationSnapshot); - } - - /** - * {@inheritDoc} - */ - @Override - public int getPriority() { - - return 2; - } - -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java index 72196ca1bbdd8..fb61950abfe5d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java @@ -33,6 +33,10 @@ public final class CheckpointUtils { private static String CHECKPOINT_DIRECTORY = null; + private static double CP_UPPER = -1.0; + + private static double CP_LOWER = -1.0; + private CheckpointUtils() { } @@ -98,4 +102,44 @@ public static void removeCheckpoint(final ExecutionVertexID vertexID) { file.delete(); } } + + public static boolean isCheckpointingDisabled() { + + return GlobalConfiguration.getBoolean("checkpoint.no", false); + } + + public static double getCPLower() { + + if (CP_LOWER < 0.0f) { + CP_LOWER = Double.parseDouble(GlobalConfiguration.getString("checkpoint.lowerbound", "0.9")); + } + + return CP_LOWER; + } + + public static double getCPUpper() { + + if (CP_UPPER < 0.0f) { + CP_UPPER = Double.parseDouble(GlobalConfiguration.getString("checkpoint.upperbound", "0.9")); + } + + return CP_UPPER; + } + + public static boolean usePACT() { + + return GlobalConfiguration.getBoolean("checkpoint.usepact", false); + } + + public static boolean useAVG() { + + return GlobalConfiguration.getBoolean("checkpoint.useavg", false); + } + + public static String getSummary() { + + return "Checkpointing Summary: UpperBound=" + getCPUpper() + " LowerBound=" + getCPLower() + + " ForcedValues: usePACT=" + usePACT() + " useAVG=" + useAVG() + + " NOCheckpoting=" + isCheckpointingDisabled(); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java index fda61432d5eee..0db0da74638fb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionListener.java @@ -77,18 +77,4 @@ void executionStateChanged(JobID jobID, ExecutionVertexID vertexID, ExecutionSta * the user thread which has finished */ void userThreadFinished(JobID jobID, ExecutionVertexID vertexID, Thread userThread); - - /** - * Called when the task has exhausted its initial execution resources (for example its initial memory buffers to - * transmit output records) and requires a decision how to proceed. - * - * @param jobID - * the ID of the job the task belongs to - * @param vertexID - * the ID of the task that ran out of its initial execution resources - * @param resourceUtilizationSnapshot - * a snapshot of the task's current resource utilization - */ - void initialExecutionResourcesExhausted(JobID jobID, ExecutionVertexID vertexID, - ResourceUtilizationSnapshot resourceUtilizationSnapshot); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index 22f98118de0f0..76dc588cf5174 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -33,7 +33,6 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionState; -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.instance.AllocatedResource; import eu.stratosphere.nephele.instance.DummyInstance; import eu.stratosphere.nephele.instance.InstanceManager; @@ -1480,16 +1479,6 @@ public void userThreadFinished(final JobID jobID, final ExecutionVertexID vertex } - /** - * {@inheritDoc} - */ - @Override - public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID vertexID, - final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { - - // Nothing to do here - } - /** * Reconstructs the execution pipelines for the entire execution graph. */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 76fac4a369eae..efe4a3e5cea0c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -32,7 +32,6 @@ import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.execution.ExecutionStateTransition; -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.instance.AllocatedResource; import eu.stratosphere.nephele.instance.AllocationID; @@ -379,17 +378,6 @@ public void updateCheckpointState(final CheckpointState newCheckpointState) { } } - public void initialExecutionResourcesExhausted( - final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { - - // Notify the listener objects - final Iterator it = this.executionListeners.values().iterator(); - while (it.hasNext()) { - it.next().initialExecutionResourcesExhausted(this.environment.getJobID(), this.vertexID, - resourceUtilizationSnapshot); - } - } - /** * Assigns the execution vertex with an {@link AllocatedResource}. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index cd18f7de20169..6015cf5d9d85f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Set; -import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse; @@ -211,12 +210,6 @@ public synchronized List submitTasks(final List checkpointDecisions) - throws IOException { - - getTaskManager().propagateCheckpointDecisions(checkpointDecisions); - } - /** * Cancels the task identified by the given ID at the instance's * {@link eu.stratosphere.nephele.taskmanager.TaskManager}. diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java index e86a85d863e02..fee4f1ec29141 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/EventCollector.java @@ -34,7 +34,6 @@ import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionState; -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.CheckpointStateListener; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; @@ -161,15 +160,6 @@ public void userThreadFinished(final JobID jobID, final ExecutionVertexID vertex // Nothing to do here } - /** - * {@inheritDoc} - */ - @Override - public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID vertexID, - final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { - // Nothing to do here - } - /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 7c14eaa87c0fd..55230f0ee3f86 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -56,9 +56,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.checkpointing.CheckpointDecision; -import eu.stratosphere.nephele.checkpointing.CheckpointDecisionCoordinator; -import eu.stratosphere.nephele.checkpointing.CheckpointDecisionPropagator; import eu.stratosphere.nephele.client.AbstractJobResult; import eu.stratosphere.nephele.client.JobCancelResult; import eu.stratosphere.nephele.client.JobProgressResult; @@ -71,7 +68,6 @@ import eu.stratosphere.nephele.event.job.AbstractEvent; import eu.stratosphere.nephele.event.job.RecentJobEvent; import eu.stratosphere.nephele.execution.ExecutionState; -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGraphIterator; @@ -141,8 +137,7 @@ * @author warneke */ public class JobManager implements DeploymentManager, ExtendedManagementProtocol, InputSplitProviderProtocol, - JobManagerProtocol, ChannelLookupProtocol, JobStatusListener, CheckpointDecisionPropagator, - PluginCommunicationProtocol { + JobManagerProtocol, ChannelLookupProtocol, JobStatusListener, PluginCommunicationProtocol { private static final Log LOG = LogFactory.getLog(JobManager.class); @@ -160,8 +155,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol private InstanceManager instanceManager; - private final CheckpointDecisionCoordinator checkpointDecisionCoordinator; - private final Map jobManagerPlugins; private final int recommendedClientPollingInterval; @@ -216,9 +209,6 @@ public JobManager(final String configDir, final String executionMode) { // Load the input split manager this.inputSplitManager = new InputSplitManager(); - // Load the checkpoint decision coordinator - this.checkpointDecisionCoordinator = new CheckpointDecisionCoordinator(this); - // Determine own RPC address final InetSocketAddress rpcServerAddress = new InetSocketAddress(ipcAddress, ipcPort); @@ -554,9 +544,6 @@ public JobSubmissionResult submitJob(JobGraph job) throws IOException { // Register job with the dynamic input split assigner this.inputSplitManager.registerJob(eg); - // Register with the checkpoint decision coordinator - this.checkpointDecisionCoordinator.registerJob(eg); - // Register for updates on the job status eg.registerJobStatusListener(this); @@ -1258,40 +1245,6 @@ public InputSplitWrapper requestNextInputSplit(final JobID jobID, final Executio return new InputSplitWrapper(jobID, this.inputSplitManager.getNextInputSplit(vertex, sequenceNumber.getValue())); } - /** - * {@inheritDoc} - */ - @Override - @Deprecated - public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID vertexID, - final ResourceUtilizationSnapshot resourceUtilizationSnapshot) throws IOException { - - final ExecutionGraph graph = this.scheduler.getExecutionGraphByID(jobID); - if (graph == null) { - LOG.error("Cannot find execution graph to job ID " + jobID); - return; - } - - final ExecutionVertex vertex = graph.getVertexByID(vertexID); - if (vertex == null) { - LOG.error("Cannot find execution vertex with ID " + vertexID); - return; - } - - final Runnable taskStateChangeRunnable = new Runnable() { - - @Override - public void run() { - - // The registered listeners of the vertex will make sure the appropriate actions are taken - vertex.initialExecutionResourcesExhausted(resourceUtilizationSnapshot); - } - }; - - // Hand over to the executor service, as this may result in a longer operation with several IPC operations - this.executorService.execute(taskStateChangeRunnable); - } - /** * {@inheritDoc} */ @@ -1326,42 +1279,6 @@ public void run() { this.executorService.execute(taskStateChangeRunnable); } - /** - * {@inheritDoc} - */ - @Override - @Deprecated - public void propagateCheckpointDecisions(final Map> checkpointDecisions) { - - final Iterator>> it = checkpointDecisions.entrySet() - .iterator(); - while (it.hasNext()) { - - final Map.Entry> entry = it.next(); - final AbstractInstance instance = entry.getKey(); - final List decisions = entry.getValue(); - - final Runnable runnable = new Runnable() { - - /** - * {@inheritDoc} - */ - @Override - public void run() { - - try { - instance.propagateCheckpointDecisions(decisions); - } catch (IOException ioe) { - LOG.error(StringUtils.stringifyException(ioe)); - } - } - }; - - this.executorService.execute(runnable); - } - - } - /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java index 8b7de50b08a2c..9b078cd1a7015 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java @@ -17,7 +17,6 @@ import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionState; -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; import eu.stratosphere.nephele.executiongraph.ExecutionGroupVertex; import eu.stratosphere.nephele.executiongraph.ExecutionPipeline; @@ -140,14 +139,6 @@ public void userThreadStarted(final JobID jobID, final ExecutionVertexID vertexI // Nothing to do here } - /** - * {@inheritDoc} - */ - @Override - public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID vertexID, - final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { - } - /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java index 40b5ec2a8e946..6366799705863 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/JobManagerProtocol.java @@ -17,11 +17,8 @@ import java.io.IOException; -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.instance.HardwareDescription; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; -import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.protocols.VersionedProtocol; import eu.stratosphere.nephele.taskmanager.TaskCheckpointState; import eu.stratosphere.nephele.taskmanager.TaskExecutionState; @@ -67,19 +64,4 @@ void sendHeartbeat(InstanceConnectionInfo instanceConnectionInfo, HardwareDescri * thrown if an error occurs during this remote procedure call */ void updateCheckpointState(TaskCheckpointState taskCheckpointState) throws IOException; - - /** - * Reports that a task has exhausted its initial execution resources. - * - * @param jobID - * the ID of the job the task belongs to - * @param vertexID - * the ID of the vertex representing the task - * @param resourceUtilizationSnapshot - * snapshot of the task's resource utilization taken at the point in time when the exhaustion occurred - * @throws IOException - * thrown if an error occurs during this remote procedure call - */ - void initialExecutionResourcesExhausted(JobID jobID, ExecutionVertexID vertexID, - ResourceUtilizationSnapshot resourceUtilizationSnapshot) throws IOException; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index 2654c20cae1c0..7462b965ba111 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -19,7 +19,6 @@ import java.util.List; import java.util.Set; -import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate; @@ -97,8 +96,6 @@ List submitTasks(List tasks) */ void updateLibraryCache(LibraryCacheUpdate update) throws IOException; - void propagateCheckpointDecisions(List checkpointDecisions) throws IOException; - /** * Removes the checkpoints which are identified by the provided list of vertex IDs. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 2e0b1d90273f1..ecbab8cee0992 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -20,7 +20,6 @@ import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.ArrayList; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -39,7 +38,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.checkpointing.ReplayTask; import eu.stratosphere.nephele.checkpointing.CheckpointUtils; import eu.stratosphere.nephele.configuration.ConfigConstants; @@ -49,17 +47,13 @@ import eu.stratosphere.nephele.discovery.DiscoveryService; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate; import eu.stratosphere.nephele.executiongraph.CheckpointState; -import eu.stratosphere.nephele.executiongraph.ExecutionGraph; -import eu.stratosphere.nephele.executiongraph.ExecutionVertex; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; -import eu.stratosphere.nephele.instance.AbstractInstance; import eu.stratosphere.nephele.instance.HardwareDescription; import eu.stratosphere.nephele.instance.HardwareDescriptionFactory; import eu.stratosphere.nephele.instance.InstanceConnectionInfo; @@ -148,16 +142,6 @@ public class TaskManager implements TaskOperationProtocol, PluginCommunicationPr */ private boolean isShutDown = false; - private final double CPupper; - - private final double CPlower; - - private final boolean usePACT; - - private final boolean useAVG; - - private final boolean noCheckpointing; - /** * Constructs a new task manager, starts its IPC service and attempts to discover the job manager to * receive an initial configuration. @@ -335,17 +319,8 @@ public TaskManager(String configDir) throws Exception { // Add shutdown hook for clean up tasks Runtime.getRuntime().addShutdownHook(new TaskManagerCleanUp(this)); - - this.CPupper = Double.parseDouble(GlobalConfiguration.getString("checkpoint.upperbound","0.9")); - this.CPlower = Double.parseDouble(GlobalConfiguration.getString("checkpoint.lowerbound","1.5")); - this.usePACT = GlobalConfiguration.getBoolean("checkpoint.usepact",false); - this.useAVG = GlobalConfiguration.getBoolean("checkpoint.useavg",false); - this.noCheckpointing = GlobalConfiguration.getBoolean("checkpoint.no",false); - LOG.info("Checkpointing Summary: UpperBound=" + this.CPupper + " LowerBound=" + this.CPlower - + " ForcedValues: usePACT=" + this.usePACT + " useAVG=" + this.useAVG - + " NOCheckpoting="+this.noCheckpointing); - + LOG.info(CheckpointUtils.getSummary()); } /** @@ -358,7 +333,7 @@ public TaskManager(String configDir) throws Exception { public static void main(String[] args) { Option configDirOpt = OptionBuilder.withArgName("config directory").hasArg().withDescription( - "Specify configuration directory.").create("configDir"); + "Specify configuration directory.").create("configDir"); Options options = new Options(); options.addOption(configDirOpt); @@ -724,119 +699,6 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID id, } } - /** - * Called by an {@link EnvironmentWrapper} object to indicate that a task has exhausted its initial execution - * resources. - * - * @param jobID - * the ID of the job the task belongs to - * @param id - * the ID of the vertex representing the task - * @param resourceUtilizationSnapshot - * snapshot of the task's resource utilization taken at the point in time when the exhaustion occurred - */ - public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID id, - final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { - Task task = this.runningTasks.get(id); - LOG.info("Checkpoint decision for vertex " + task.getEnvironment().getTaskName() + " required"); - boolean checkpointDecision = getDecision(task, resourceUtilizationSnapshot); - if(checkpointDecision) - LOG.info("Creating Checkpoint for " + task.getEnvironment().getTaskName() + " (" +task.getEnvironment().getIndexInSubtaskGroup() +"/" +task.getEnvironment().getCurrentNumberOfSubtasks() + " )"); - - final List checkpointDecisionList = new SerializableArrayList(); - - checkpointDecisionList.add(new CheckpointDecision(id, checkpointDecision)); - - // Propagate checkpoint decisions - try { - propagateCheckpointDecisions(checkpointDecisionList); - } catch (IOException e) { - e.printStackTrace(); - } - - } - private boolean getDecision(final Task task, final ResourceUtilizationSnapshot rus) { - if(this.noCheckpointing){ - return false; - } - - if(rus.getForced() == null){ - if(rus.getPactRatio() != -1 || this.usePACT){ - System.out.println("Ratio = " + rus.getPactRatio()); - if(rus.getPactRatio()>=this.CPlower){ - //amount of data is small so we checkpoint - return true; - } - if(rus.getPactRatio()<=this.CPupper){ - //amount of data is too big - return false; - } - }else{ - //no info from upper layer so use average sizes - if(rus.isDam() || this.useAVG){ - System.out.println("is Dam"); - - if(rus.getAverageOutputRecordSize() != 0){ - System.out.println( "avg ratio" + rus.getAverageInputRecordSize()/rus.getAverageOutputRecordSize()); - } - - if(rus.getAverageOutputRecordSize() != 0 && - rus.getAverageInputRecordSize()/rus.getAverageOutputRecordSize() >=this.CPlower){ - return true; - } - - if(rus.getAverageOutputRecordSize() != 0 && - rus.getAverageInputRecordSize()/rus.getAverageOutputRecordSize() <=this.CPupper){ - return false; - } - }else{ - - - - // we have no data dam so we can estimate the input/output-ratio - System.out.println("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); - if(rus.getTotalInputAmount() != 0 ){ - System.out.println("selektivity is " + (double)rus.getTotalOutputAmount() / rus.getTotalInputAmount()); - - } - if(rus.getTotalOutputAmount() != 0 && ((double)rus.getTotalInputAmount() / rus.getTotalOutputAmount() >=this.CPlower)){ - //size off checkpoint would be to large: do not checkpoint - //TODO progress estimation would make sense here - LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double)rus.getTotalInputAmount()/ rus.getTotalOutputAmount() > 2.0)); - return false; - - } - if(rus.getTotalOutputAmount() != 0 && ((double)rus.getTotalInputAmount() / rus.getTotalOutputAmount() <=this.CPupper)){ - //size of checkpoint will be small enough: checkpoint - //TODO progress estimation would make sense here - LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double)rus.getTotalInputAmount()/ rus.getTotalOutputAmount() > 2.0)); - return true; - - } - - - } - } - //between thresholds check CPU Usage. - if (rus.getUserCPU() >= 90) { - System.out.println(task.getEnvironment().getTaskName() + "CPU-Bottleneck"); - // CPU bottleneck - return true; - } - - - - } else { - System.out.println("Checkpoint decision was forced too " + rus.getForced()); - // checkpoint decision was forced by the user - return rus.getForced(); - } - - //in case of doubt do not checkpoint - return false; - - } - public void checkpointStateChanged(final JobID jobID, final ExecutionVertexID id, final CheckpointState newCheckpointState) { @@ -941,15 +803,6 @@ public void logBufferUtilization() throws IOException { this.byteBufferedChannelManager.logBufferUtilization(); } - /** - * {@inheritDoc} - */ - @Override - public void propagateCheckpointDecisions(final List checkpointDecisions) throws IOException { - - this.byteBufferedChannelManager.reportCheckpointDecisions(checkpointDecisions); - } - /** * {@inheritDoc} */ @@ -1009,7 +862,7 @@ public void sendDataToJobManager(final PluginID pluginID, final IOReadableWritab * thrown if an I/O error occurs during the RPC call */ public IOReadableWritable requestDataFromJobManager(final PluginID pluginID, final IOReadableWritable data) - throws IOException { + throws IOException { synchronized (this.pluginCommunicationService) { return this.pluginCommunicationService.requestData(pluginID, data); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index fd5afb934289c..bf314d0192301 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -26,7 +26,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; @@ -663,22 +662,6 @@ private void redistributeGlobalBuffers() { } } - public void reportCheckpointDecisions(final List checkpointDecisions) { - - for (final CheckpointDecision cd : checkpointDecisions) { - - final RuntimeTaskContext taskContext = this.tasksWithUndecidedCheckpoints.remove(cd.getVertexID()); - - if (taskContext == null) { - LOG.error("Cannot report checkpoint decision for vertex " + cd.getVertexID()); - continue; - } - LOG.info("reporting checkpoint decision for " + cd.getVertexID()); - taskContext.setCheckpointDecisionAsynchronously(cd.getCheckpointDecision()); - taskContext.reportAsynchronousEvent(); - } - } - /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java index 06b0a6128904f..6617edc1e7ae7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java @@ -24,7 +24,7 @@ final class EnvelopeConsumptionLog { private static final Log LOG = LogFactory.getLog(EnvelopeConsumptionLog.class); - private static final int LOG_WINDOW_SIZE = 262144; + private static final int LOG_WINDOW_SIZE = 64 * 1024; private static final int SIZE_OF_INTEGER = 4; @@ -223,7 +223,7 @@ private void addOutstandingEnvelope(final AbstractByteBufferedInputChannel registeredListeners = new ConcurrentLinkedQueue(); - - private long startTime; - + // DW: Start of temporary code private double pactInputOutputRatioSum = 0.0; @@ -227,7 +209,7 @@ public void killExecution() { * Cancels or kills the task. * * @param cancel - * true/code> if the task shall be cancelled, false if it shall be killed + * true/code> if the task shall be canceled, false if it shall be killed */ private void cancelOrKillExecution(final boolean cancel) { @@ -288,7 +270,6 @@ public void startExecution() { final Thread thread = this.environment.getExecutingThread(); thread.start(); - this.startTime = System.currentTimeMillis(); } /** @@ -300,112 +281,6 @@ public boolean isCanceled() { return this.isCanceled; } - /** - * Triggers the notification that the task has run out of its initial execution resources. - */ - public void initialExecutionResourcesExhausted() { - - System.out.println("PACT input/output for task " + this.environment.getTaskNameWithIndex() + ": " + getPACTInputOutputRatio()); - - // if (this.environment.getExecutingThread() != Thread.currentThread()) { - // throw new ConcurrentModificationException( - // "initialExecutionResourcesExhausted must be called from the task that executes the user code"); - // } - - // Construct a resource utilization snapshot - final long timestamp = System.currentTimeMillis(); - if (this.environment.getInputGate(0) != null - && this.environment.getInputGate(0).getExecutionStart() < timestamp) { - this.startTime = this.environment.getInputGate(0).getExecutionStart(); - } - LOG.info("Task " + this.getTaskName() + " started " + this.startTime); - // Get CPU-Usertime in percent - ThreadMXBean threadBean = ManagementFactory.getThreadMXBean(); - long userCPU = (threadBean.getCurrentThreadUserTime() / NANO_TO_MILLISECONDS) * 100 - / (timestamp - this.startTime); - LOG.info("USER CPU for " + this.getTaskName() + " : " + userCPU); - // collect outputChannelUtilization - final Map channelUtilization = new HashMap(); - long totalOutputAmount = 0; - int numrec = 0; - long averageOutputRecordSize= 0; - for (int i = 0; i < this.environment.getNumberOfOutputGates(); ++i) { - final OutputGate outputGate = this.environment.getOutputGate(i); - numrec += outputGate.getNumRecords(); - for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { - final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); - channelUtilization.put(outputChannel.getID(), - Long.valueOf(outputChannel.getAmountOfDataTransmitted())); - totalOutputAmount += outputChannel.getAmountOfDataTransmitted(); - } - } - - if(numrec != 0){ - averageOutputRecordSize = totalOutputAmount/numrec; - } - //FIXME (marrus) it is not about what we received but what we processed yet - boolean allClosed = true; - int numinrec = 0; - - long totalInputAmount = 0; - long averageInputRecordSize = 0; - for (int i = 0; i < this.environment.getNumberOfInputGates(); ++i) { - final InputGate inputGate = this.environment.getInputGate(i); - numinrec += inputGate.getNumRecords(); - for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { - final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); - channelUtilization.put(inputChannel.getID(), - Long.valueOf(inputChannel.getAmountOfDataTransmitted())); - totalInputAmount += inputChannel.getAmountOfDataTransmitted(); - try { - if(!inputChannel.isClosed()){ - allClosed = false; - } - } catch (IOException e) { - e.printStackTrace(); - } catch (InterruptedException e) { - e.printStackTrace(); - } - - } - } - if(numinrec != 0){ - averageInputRecordSize = totalInputAmount/numinrec; - } - Boolean force = null; - Boolean stateful = false; - if (this.environment.getInvokable().getClass().isAnnotationPresent(Stateful.class) - && !this.environment.getInvokable().getClass().isAnnotationPresent(Stateless.class)) { - // Don't checkpoint stateful tasks - force = false; - } else { - if(this.environment.getForced() != null){ - force = this.environment.getForced(); - }else{ - // look for a forced decision from the user - ForceCheckpoint forced = this.environment.getInvokable().getClass().getAnnotation(ForceCheckpoint.class); - - //this.environment.getInvokable().getTaskConfiguration().getBoolean("forced_checkpoint", false) - - if (forced != null) { - force = forced.checkpoint(); - } - } - } - - final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, channelUtilization, userCPU, - force, totalInputAmount, totalOutputAmount, averageOutputRecordSize, averageInputRecordSize, getPACTInputOutputRatio(), allClosed); - - // Notify the listener objects - final Iterator it = this.registeredListeners.iterator(); - while (it.hasNext()) { - it.next().initialExecutionResourcesExhausted(this.environment.getJobID(), this.vertexID, rus); - } - - // Finally, propagate event to the job manager - this.taskManager.initialExecutionResourcesExhausted(this.environment.getJobID(), this.vertexID, rus); - } - public void checkpointStateChanged(final CheckpointState newCheckpointState) { // Propagate event to the job manager @@ -565,7 +440,7 @@ public void reportPACTDataStatistics(final long numberOfConsumedBytes, final lon ++this.numberOfPactInputOutputRatioEntries; } - private double getPACTInputOutputRatio() { + double getPACTInputOutputRatio() { if(this.numberOfPactInputOutputRatioEntries == 0) { return -1.0; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 8b4877c283f41..7ff4385ac397f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -16,16 +16,29 @@ package eu.stratosphere.nephele.taskmanager.runtime; import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.lang.management.ThreadMXBean; +import java.util.HashMap; import java.util.Map; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.annotations.ForceCheckpoint; +import eu.stratosphere.nephele.annotations.Stateful; +import eu.stratosphere.nephele.annotations.Stateless; import eu.stratosphere.nephele.checkpointing.EphemeralCheckpoint; +import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.AbstractID; import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.InputGate; import eu.stratosphere.nephele.io.OutputGate; +import eu.stratosphere.nephele.io.channels.AbstractInputChannel; +import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.taskmanager.bufferprovider.AsynchronousEventListener; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; @@ -38,6 +51,13 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEventListener, TaskContext { + /** + * The log object used for debugging. + */ + private static final Log LOG = LogFactory.getLog(RuntimeTaskContext.class); + + private static final long NANO_TO_MILLISECONDS = 1000 * 1000; + private final LocalBufferPool localBufferPool; private final RuntimeTask task; @@ -52,6 +72,8 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve private final EnvelopeConsumptionLog envelopeConsumptionLog; + private final long startTime; + /** * Stores whether the initial exhaustion of memory buffers has already been reported */ @@ -86,6 +108,8 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve this.runtimeDispatcher = new RuntimeDispatcher(transferEnvelopeDispatcher); this.envelopeConsumptionLog = new EnvelopeConsumptionLog(task.getVertexID(), environment); + + this.startTime = System.currentTimeMillis(); } RuntimeDispatcher getRuntimeDispatcher() { @@ -162,6 +186,10 @@ public void logBufferUtilization() { System.out.println("\t\t" + environment.getTaskNameWithIndex() + ": " + ava + " available, " + req + " requested, " + des + " designated"); + + if (this.envelopeConsumptionLog.followsLog()) { + this.envelopeConsumptionLog.showOustandingEnvelopeLog(); + } } /** @@ -171,8 +199,108 @@ void reportExhaustionOfMemoryBuffers() { if (!this.initialExhaustionOfMemoryBuffersReported) { - this.task.initialExecutionResourcesExhausted(); this.initialExhaustionOfMemoryBuffersReported = true; + + final RuntimeEnvironment environment = this.task.getRuntimeEnvironment(); + + System.out.println("PACT input/output for task " + environment.getTaskNameWithIndex() + ": " + + this.task.getPACTInputOutputRatio()); + + // if (this.environment.getExecutingThread() != Thread.currentThread()) { + // throw new ConcurrentModificationException( + // "initialExecutionResourcesExhausted must be called from the task that executes the user code"); + // } + + // Construct a resource utilization snapshot + final long timestamp = System.currentTimeMillis(); + if (environment.getInputGate(0) != null + && environment.getInputGate(0).getExecutionStart() < timestamp) { + this.startTime = environment.getInputGate(0).getExecutionStart(); + } + LOG.info("Task " + environment.getTaskNameWithIndex() + " started " + this.startTime); + // Get CPU-Usertime in percent + ThreadMXBean threadBean = ManagementFactory.getThreadMXBean(); + long userCPU = (threadBean.getCurrentThreadUserTime() / NANO_TO_MILLISECONDS) * 100 + / (timestamp - this.startTime); + LOG.info("USER CPU for " + environment.getTaskNameWithIndex() + " : " + userCPU); + // collect outputChannelUtilization + final Map channelUtilization = new HashMap(); + long totalOutputAmount = 0; + int numrec = 0; + long averageOutputRecordSize = 0; + for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { + final OutputGate outputGate = environment.getOutputGate(i); + numrec += outputGate.getNumRecords(); + for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { + final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); + channelUtilization.put(outputChannel.getID(), + Long.valueOf(outputChannel.getAmountOfDataTransmitted())); + totalOutputAmount += outputChannel.getAmountOfDataTransmitted(); + } + } + + if (numrec != 0) { + averageOutputRecordSize = totalOutputAmount / numrec; + } + // FIXME (marrus) it is not about what we received but what we processed yet + boolean allClosed = true; + int numinrec = 0; + + long totalInputAmount = 0; + long averageInputRecordSize = 0; + for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { + final InputGate inputGate = environment.getInputGate(i); + numinrec += inputGate.getNumRecords(); + for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { + final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); + channelUtilization.put(inputChannel.getID(), + Long.valueOf(inputChannel.getAmountOfDataTransmitted())); + totalInputAmount += inputChannel.getAmountOfDataTransmitted(); + try { + if (!inputChannel.isClosed()) { + allClosed = false; + } + } catch (IOException e) { + e.printStackTrace(); + } catch (InterruptedException e) { + e.printStackTrace(); + } + + } + } + if (numinrec != 0) { + averageInputRecordSize = totalInputAmount / numinrec; + } + Boolean force = null; + Boolean stateful = false; + if (environment.getInvokable().getClass().isAnnotationPresent(Stateful.class) + && !environment.getInvokable().getClass().isAnnotationPresent(Stateless.class)) { + // Don't checkpoint stateful tasks + force = false; + } else { + if (environment.getForced() != null) { + force = environment.getForced(); + } else { + // look for a forced decision from the user + ForceCheckpoint forced = environment.getInvokable().getClass().getAnnotation(ForceCheckpoint.class); + + // this.environment.getInvokable().getTaskConfiguration().getBoolean("forced_checkpoint", false) + + if (forced != null) { + force = forced.checkpoint(); + } + } + } + + final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, channelUtilization, + userCPU, + force, totalInputAmount, totalOutputAmount, averageOutputRecordSize, averageInputRecordSize, + this.task.getPACTInputOutputRatio(), allClosed); + + + // Finally, propagate event to the job manager + this.taskManager.initialExecutionResourcesExhausted(this.environment.getJobID(), this.vertexID, rus); + } } @@ -278,9 +406,9 @@ public InputGateContext createInputGateContext(final GateID gateID) { return new RuntimeInputGateContext(re.getTaskNameWithIndex(), this.transferEnvelopeDispatcher, inputGate, this.envelopeConsumptionLog); } - + public LocalBufferPool getLocalBufferPool() { - + return this.localBufferPool; } } From bd41a87c7167b5c75ead62217ed74d1c99d57367 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 22 Feb 2012 15:29:40 +0100 Subject: [PATCH 266/310] Checkpoint decision is now made by the task thread itself --- .../impl/EnvironmentListenerImpl.java | 10 ------ .../checkpointing/EphemeralCheckpoint.java | 33 +++++-------------- .../runtime/RuntimeTaskContext.java | 18 +++------- 3 files changed, 13 insertions(+), 48 deletions(-) diff --git a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java index 9df785356ed3d..1d6b41d304759 100644 --- a/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java +++ b/nephele/nephele-profiling/src/main/java/eu/stratosphere/nephele/profiling/impl/EnvironmentListenerImpl.java @@ -20,7 +20,6 @@ import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionState; -import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; @@ -94,15 +93,6 @@ public void userThreadStarted(final JobID jobID, final ExecutionVertexID vertexI this.taskManagerProfiler.registerUserThreadForCPUProfiling(this.environment, userThread); } - /** - * {@inheritDoc} - */ - @Override - public void initialExecutionResourcesExhausted(final JobID jobID, final ExecutionVertexID vertexID, - final ResourceUtilizationSnapshot resourceUtilizationSnapshot) { - // Nothing to do here - } - /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 146687bc9faaf..dfdc39f3ad936 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -125,8 +125,6 @@ private enum CheckpointingDecisionState { */ private CheckpointingDecisionState checkpointingDecision; - private volatile CheckpointingDecisionState asynchronousCheckpointingDecision; - public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { this.task = task; @@ -141,7 +139,6 @@ public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { this.checkpointingDecision = (ephemeral ? CheckpointingDecisionState.UNDECIDED : CheckpointingDecisionState.CHECKPOINTING); - this.asynchronousCheckpointingDecision = this.checkpointingDecision; this.fileBufferManager = FileBufferManager.getInstance(); @@ -276,37 +273,23 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro this.task.checkpointStateChanged(CheckpointState.COMPLETE); } } - - public void setCheckpointDecisionAsynchronously(final boolean checkpointDecision) { - - if (checkpointDecision) { - this.asynchronousCheckpointingDecision = CheckpointingDecisionState.CHECKPOINTING; - } else { - this.asynchronousCheckpointingDecision = CheckpointingDecisionState.NO_CHECKPOINTING; - } - } - - public void checkAsynchronousCheckpointDecision() throws IOException, InterruptedException { - - if (this.asynchronousCheckpointingDecision == this.checkpointingDecision) { - return; - } - - if (this.asynchronousCheckpointingDecision == CheckpointingDecisionState.UNDECIDED) { - LOG.error("Asynchronous checkpoint decision is UNDECIDED"); + + public void setCheckpointDecisionSynchronously(final boolean checkpointDecision) throws IOException, InterruptedException { + + if(this.checkpointingDecision != CheckpointingDecisionState.UNDECIDED) { return; } - - if (this.asynchronousCheckpointingDecision == CheckpointingDecisionState.CHECKPOINTING) { + + if(checkpointDecision) { + this.checkpointingDecision = CheckpointingDecisionState.CHECKPOINTING; // Write the data which has been queued so far and update checkpoint state write(); this.task.checkpointStateChanged(CheckpointState.PARTIAL); } else { + this.checkpointingDecision = CheckpointingDecisionState.NO_CHECKPOINTING; // Simply destroy the checkpoint destroy(); } - - this.checkpointingDecision = this.asynchronousCheckpointingDecision; } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 7ff4385ac397f..f62a892d7f4f1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -27,6 +27,7 @@ import eu.stratosphere.nephele.annotations.ForceCheckpoint; import eu.stratosphere.nephele.annotations.Stateful; import eu.stratosphere.nephele.annotations.Stateless; +import eu.stratosphere.nephele.checkpointing.CheckpointDecision; import eu.stratosphere.nephele.checkpointing.EphemeralCheckpoint; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.execution.RuntimeEnvironment; @@ -72,7 +73,7 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve private final EnvelopeConsumptionLog envelopeConsumptionLog; - private final long startTime; + private long startTime; /** * Stores whether the initial exhaustion of memory buffers has already been reported @@ -195,7 +196,7 @@ public void logBufferUtilization() { /** * Called by an {@link OutputGateContext} to indicate that the task has temporarily run out of memory buffers. */ - void reportExhaustionOfMemoryBuffers() { + void reportExhaustionOfMemoryBuffers() throws IOException, InterruptedException { if (!this.initialExhaustionOfMemoryBuffersReported) { @@ -297,10 +298,9 @@ void reportExhaustionOfMemoryBuffers() { force, totalInputAmount, totalOutputAmount, averageOutputRecordSize, averageInputRecordSize, this.task.getPACTInputOutputRatio(), allClosed); - - // Finally, propagate event to the job manager - this.taskManager.initialExecutionResourcesExhausted(this.environment.getJobID(), this.vertexID, rus); + final boolean checkpointDecision = CheckpointDecision.getDecision(this.task, rus); + this.ephemeralCheckpoint.setCheckpointDecisionSynchronously(checkpointDecision); } } @@ -319,8 +319,6 @@ public void reportAsynchronousEvent() { @Override public void asynchronousEventOccurred() throws IOException, InterruptedException { - // Check if the checkpoint decision changed - this.ephemeralCheckpoint.checkAsynchronousCheckpointDecision(); } /** @@ -346,12 +344,6 @@ AbstractID getFileOwnerID() { return this.task.getVertexID(); } - public void setCheckpointDecisionAsynchronously(final boolean checkpointDecision) { - - // Simply delegate call - this.ephemeralCheckpoint.setCheckpointDecisionAsynchronously(checkpointDecision); - } - /** * {@inheritDoc} */ From 9c4b0cba84ae091538cd079d26c246b5fe844bc8 Mon Sep 17 00:00:00 2001 From: marrus Date: Wed, 22 Feb 2012 17:22:47 +0100 Subject: [PATCH 267/310] fixed CPdecision --- .../checkpointing/CheckpointDecision.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java index ad365a29f223c..84da235d430b6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java @@ -36,7 +36,7 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap final double CPupper = CheckpointUtils.getCPUpper(); if (rus.getForced() == null) { - if (rus.getPactRatio() != -1 && CheckpointUtils.usePACT()) { + if (rus.getPactRatio() != -1 && !CheckpointUtils.useAVG()) { System.out.println("Ratio = " + rus.getPactRatio()); if (rus.getPactRatio() >= CPlower) { // amount of data is small so we checkpoint @@ -48,11 +48,11 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap } } else { // no info from upper layer so use average sizes - if (rus.isDam() && CheckpointUtils.useAVG()) { + if (rus.isDam()) { System.out.println("is Dam"); if (rus.getAverageOutputRecordSize() != 0) { - System.out.println("avg ratio" + rus.getAverageInputRecordSize() + System.out.println("avg ratio " + rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize()); } @@ -78,17 +78,17 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() >= CPlower)) { // size off checkpoint would be to large: do not checkpoint // TODO progress estimation would make sense here - LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " - + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() > 2.0)); + System.out.println(task.getEnvironment().getTaskName() + " Checkpoint to large selektivity " + + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount())); return false; } if (rus.getTotalOutputAmount() != 0 && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() <= CPupper)) { // size of checkpoint will be small enough: checkpoint - // TODO progress estimation would make sense here - LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " - + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() > 2.0)); + // TODO progress estimation would make sense here + System.out.println(task.getEnvironment().getTaskName() + " Checkpoint small selektivity " + + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount())); return true; } @@ -107,7 +107,7 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap // checkpoint decision was forced by the user return rus.getForced(); } - + System.out.println("Checkpoint decision false by default"); // in case of doubt do not checkpoint return false; From 411ca70a8bbfe8c08b261cc9ba26f7413f2a719e Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 22 Feb 2012 17:26:04 +0100 Subject: [PATCH 268/310] Added some debug output --- .../checkpointing/CheckpointDecision.java | 109 ++++++------ .../checkpointing/EphemeralCheckpoint.java | 5 + .../nephele/jobmanager/JobManager.java | 4 + .../runtime/RuntimeTaskContext.java | 157 +++++++++--------- 4 files changed, 142 insertions(+), 133 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java index ad365a29f223c..9ab6538f27c1a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java @@ -31,81 +31,80 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap return false; } + if (rus.getForced() != null) { + System.out.println("Checkpoint decision was forced to " + rus.getForced()); + // checkpoint decision was forced by the user + return rus.getForced(); + } + final double CPlower = CheckpointUtils.getCPLower(); final double CPupper = CheckpointUtils.getCPUpper(); - if (rus.getForced() == null) { - if (rus.getPactRatio() != -1 && CheckpointUtils.usePACT()) { - System.out.println("Ratio = " + rus.getPactRatio()); - if (rus.getPactRatio() >= CPlower) { - // amount of data is small so we checkpoint - return true; - } - if (rus.getPactRatio() <= CPupper) { - // amount of data is too big - return false; - } - } else { - // no info from upper layer so use average sizes - if (rus.isDam() && CheckpointUtils.useAVG()) { - System.out.println("is Dam"); + if (rus.getPactRatio() >= 0 && CheckpointUtils.usePACT()) { + System.out.println("Ratio = " + rus.getPactRatio()); + if (rus.getPactRatio() >= CPlower) { + // amount of data is small so we checkpoint + return true; + } + if (rus.getPactRatio() <= CPupper) { + // amount of data is too big + return false; + } + } else { + // no info from upper layer so use average sizes + if (rus.isDam() && CheckpointUtils.useAVG()) { + System.out.println("is Dam"); - if (rus.getAverageOutputRecordSize() != 0) { - System.out.println("avg ratio" + rus.getAverageInputRecordSize() + if (rus.getAverageOutputRecordSize() != 0) { + System.out.println("avg ratio" + rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize()); - } + } - if (rus.getAverageOutputRecordSize() != 0 && + if (rus.getAverageOutputRecordSize() != 0 && rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize() >= CPlower) { - return true; - } + return true; + } - if (rus.getAverageOutputRecordSize() != 0 && + if (rus.getAverageOutputRecordSize() != 0 && rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize() <= CPupper) { - return false; - } - } else { - - // we have no data dam so we can estimate the input/output-ratio - System.out.println("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); - if (rus.getTotalInputAmount() != 0) { - System.out.println("selektivity is " + (double) rus.getTotalOutputAmount() + return false; + } + } else { + + // we have no data dam so we can estimate the input/output-ratio + System.out.println("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); + if (rus.getTotalInputAmount() != 0) { + System.out.println("selektivity is " + (double) rus.getTotalOutputAmount() / rus.getTotalInputAmount()); - } - if (rus.getTotalOutputAmount() != 0 + } + if (rus.getTotalOutputAmount() != 0 && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() >= CPlower)) { - // size off checkpoint would be to large: do not checkpoint - // TODO progress estimation would make sense here - LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + // size off checkpoint would be to large: do not checkpoint + // TODO progress estimation would make sense here + System.out.println(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() > 2.0)); - return false; + return false; - } - if (rus.getTotalOutputAmount() != 0 + } + if (rus.getTotalOutputAmount() != 0 && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() <= CPupper)) { - // size of checkpoint will be small enough: checkpoint - // TODO progress estimation would make sense here - LOG.info(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + // size of checkpoint will be small enough: checkpoint + // TODO progress estimation would make sense here + System.out.println(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() > 2.0)); - return true; - - } + return true; } - } - // between thresholds check CPU Usage. - if (rus.getUserCPU() >= 90) { - System.out.println(task.getEnvironment().getTaskName() + "CPU-Bottleneck"); - // CPU bottleneck - return true; - } - } else { - System.out.println("Checkpoint decision was forced too " + rus.getForced()); - // checkpoint decision was forced by the user - return rus.getForced(); + } + } + // between thresholds check CPU Usage. + if (rus.getUserCPU() >= 90) { + System.out.println(task.getEnvironment().getTaskName() + "CPU-Bottleneck"); + // CPU bottleneck + return true; } // in case of doubt do not checkpoint diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index dfdc39f3ad936..6c0fe62110bbe 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -313,6 +313,11 @@ public boolean forward(final TransferEnvelope transferEnvelope) throws IOExcepti return true; } + public boolean isUndecided() { + + return (this.checkpointingDecision == CheckpointingDecisionState.UNDECIDED); + } + /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 55230f0ee3f86..5f0cec76c879f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -757,6 +757,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio if (assignedInstance == null) { LOG.error("Cannot resolve lookup: vertex found for channel ID " + connectedChannelID + " but no instance assigned"); + LOG.info("Created receiverNotReady for " + connectedVertex + " 1"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } @@ -768,6 +769,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.REPLAYING && executionState != ExecutionState.FINISHING) { + LOG.info("Created receiverNotReady for " + connectedVertex + " in state " + executionState + " 2"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } @@ -799,6 +801,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.REPLAYING && executionState != ExecutionState.FINISHING && executionState != ExecutionState.FINISHED) { + LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } @@ -807,6 +810,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio LOG.error("Cannot resolve lookup: vertex found for channel ID " + outputChannel.getConnectedChannelID() + " but no instance assigned"); + LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index f62a892d7f4f1..d886148867e49 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -198,110 +198,111 @@ public void logBufferUtilization() { */ void reportExhaustionOfMemoryBuffers() throws IOException, InterruptedException { - if (!this.initialExhaustionOfMemoryBuffersReported) { - - this.initialExhaustionOfMemoryBuffersReported = true; + if (!this.ephemeralCheckpoint.isUndecided()) { + return; + } - final RuntimeEnvironment environment = this.task.getRuntimeEnvironment(); + final RuntimeEnvironment environment = this.task.getRuntimeEnvironment(); - System.out.println("PACT input/output for task " + environment.getTaskNameWithIndex() + ": " + System.out.println("PACT input/output for task " + environment.getTaskNameWithIndex() + ": " + this.task.getPACTInputOutputRatio()); - // if (this.environment.getExecutingThread() != Thread.currentThread()) { - // throw new ConcurrentModificationException( - // "initialExecutionResourcesExhausted must be called from the task that executes the user code"); - // } + // if (this.environment.getExecutingThread() != Thread.currentThread()) { + // throw new ConcurrentModificationException( + // "initialExecutionResourcesExhausted must be called from the task that executes the user code"); + // } - // Construct a resource utilization snapshot - final long timestamp = System.currentTimeMillis(); - if (environment.getInputGate(0) != null + // Construct a resource utilization snapshot + final long timestamp = System.currentTimeMillis(); + if (environment.getInputGate(0) != null && environment.getInputGate(0).getExecutionStart() < timestamp) { - this.startTime = environment.getInputGate(0).getExecutionStart(); - } - LOG.info("Task " + environment.getTaskNameWithIndex() + " started " + this.startTime); - // Get CPU-Usertime in percent - ThreadMXBean threadBean = ManagementFactory.getThreadMXBean(); - long userCPU = (threadBean.getCurrentThreadUserTime() / NANO_TO_MILLISECONDS) * 100 + this.startTime = environment.getInputGate(0).getExecutionStart(); + } + LOG.info("Task " + environment.getTaskNameWithIndex() + " started " + this.startTime); + // Get CPU-Usertime in percent + ThreadMXBean threadBean = ManagementFactory.getThreadMXBean(); + long userCPU = (threadBean.getCurrentThreadUserTime() / NANO_TO_MILLISECONDS) * 100 / (timestamp - this.startTime); - LOG.info("USER CPU for " + environment.getTaskNameWithIndex() + " : " + userCPU); - // collect outputChannelUtilization - final Map channelUtilization = new HashMap(); - long totalOutputAmount = 0; - int numrec = 0; - long averageOutputRecordSize = 0; - for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - final OutputGate outputGate = environment.getOutputGate(i); - numrec += outputGate.getNumRecords(); - for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { - final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); - channelUtilization.put(outputChannel.getID(), + LOG.info("USER CPU for " + environment.getTaskNameWithIndex() + " : " + userCPU); + // collect outputChannelUtilization + final Map channelUtilization = new HashMap(); + long totalOutputAmount = 0; + int numrec = 0; + long averageOutputRecordSize = 0; + for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { + final OutputGate outputGate = environment.getOutputGate(i); + numrec += outputGate.getNumRecords(); + for (int j = 0; j < outputGate.getNumberOfOutputChannels(); ++j) { + final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); + channelUtilization.put(outputChannel.getID(), Long.valueOf(outputChannel.getAmountOfDataTransmitted())); - totalOutputAmount += outputChannel.getAmountOfDataTransmitted(); - } + totalOutputAmount += outputChannel.getAmountOfDataTransmitted(); } + } - if (numrec != 0) { - averageOutputRecordSize = totalOutputAmount / numrec; - } - // FIXME (marrus) it is not about what we received but what we processed yet - boolean allClosed = true; - int numinrec = 0; - - long totalInputAmount = 0; - long averageInputRecordSize = 0; - for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { - final InputGate inputGate = environment.getInputGate(i); - numinrec += inputGate.getNumRecords(); - for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { - final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); - channelUtilization.put(inputChannel.getID(), + if (numrec != 0) { + averageOutputRecordSize = totalOutputAmount / numrec; + } + // FIXME (marrus) it is not about what we received but what we processed yet + boolean allClosed = true; + int numinrec = 0; + + long totalInputAmount = 0; + long averageInputRecordSize = 0; + for (int i = 0; i < environment.getNumberOfInputGates(); ++i) { + final InputGate inputGate = environment.getInputGate(i); + numinrec += inputGate.getNumRecords(); + for (int j = 0; j < inputGate.getNumberOfInputChannels(); ++j) { + final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); + channelUtilization.put(inputChannel.getID(), Long.valueOf(inputChannel.getAmountOfDataTransmitted())); - totalInputAmount += inputChannel.getAmountOfDataTransmitted(); - try { - if (!inputChannel.isClosed()) { - allClosed = false; - } - } catch (IOException e) { - e.printStackTrace(); - } catch (InterruptedException e) { - e.printStackTrace(); + totalInputAmount += inputChannel.getAmountOfDataTransmitted(); + try { + if (!inputChannel.isClosed()) { + allClosed = false; } - + } catch (IOException e) { + e.printStackTrace(); + } catch (InterruptedException e) { + e.printStackTrace(); } + } - if (numinrec != 0) { - averageInputRecordSize = totalInputAmount / numinrec; - } - Boolean force = null; - Boolean stateful = false; - if (environment.getInvokable().getClass().isAnnotationPresent(Stateful.class) + } + if (numinrec != 0) { + averageInputRecordSize = totalInputAmount / numinrec; + } + Boolean force = null; + if (environment.getInvokable().getClass().isAnnotationPresent(Stateful.class) && !environment.getInvokable().getClass().isAnnotationPresent(Stateless.class)) { - // Don't checkpoint stateful tasks - force = false; + // Don't checkpoint stateful tasks + force = false; + } else { + if (environment.getForced() != null) { + force = environment.getForced(); } else { - if (environment.getForced() != null) { - force = environment.getForced(); - } else { - // look for a forced decision from the user - ForceCheckpoint forced = environment.getInvokable().getClass().getAnnotation(ForceCheckpoint.class); + // look for a forced decision from the user + ForceCheckpoint forced = environment.getInvokable().getClass().getAnnotation(ForceCheckpoint.class); - // this.environment.getInvokable().getTaskConfiguration().getBoolean("forced_checkpoint", false) + // this.environment.getInvokable().getTaskConfiguration().getBoolean("forced_checkpoint", false) - if (forced != null) { - force = forced.checkpoint(); - } + if (forced != null) { + force = forced.checkpoint(); } } + } - final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, channelUtilization, + final ResourceUtilizationSnapshot rus = new ResourceUtilizationSnapshot(timestamp, channelUtilization, userCPU, force, totalInputAmount, totalOutputAmount, averageOutputRecordSize, averageInputRecordSize, this.task.getPACTInputOutputRatio(), allClosed); + System.out.println("Making checkpoint decision for " + environment.getTaskNameWithIndex()); + final boolean checkpointDecision = CheckpointDecision.getDecision(this.task, rus); + System.out.println("Checkpoint decision for " + environment.getTaskNameWithIndex() + " is " + + checkpointDecision); + this.ephemeralCheckpoint.setCheckpointDecisionSynchronously(checkpointDecision); - final boolean checkpointDecision = CheckpointDecision.getDecision(this.task, rus); - this.ephemeralCheckpoint.setCheckpointDecisionSynchronously(checkpointDecision); - } } /** From b7c2ca90eeff1274ccc95ed8f40386e85061715c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 22 Feb 2012 18:41:33 +0100 Subject: [PATCH 269/310] Resolved conflict after merge --- .../checkpointing/CheckpointDecision.java | 68 ++++--------------- 1 file changed, 15 insertions(+), 53 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java index 989902fa83ec0..ef177503e71f2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java @@ -41,8 +41,7 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap final double CPupper = CheckpointUtils.getCPUpper(); -<<<<<<< HEAD - if (rus.getPactRatio() >= 0 && CheckpointUtils.usePACT()) { + if (rus.getPactRatio() >= 0.0 && !CheckpointUtils.useAVG()) { System.out.println("Ratio = " + rus.getPactRatio()); if (rus.getPactRatio() >= CPlower) { // amount of data is small so we checkpoint @@ -54,41 +53,21 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap } } else { // no info from upper layer so use average sizes - if (rus.isDam() && CheckpointUtils.useAVG()) { + if (rus.isDam()) { System.out.println("is Dam"); if (rus.getAverageOutputRecordSize() != 0) { - System.out.println("avg ratio" + rus.getAverageInputRecordSize() -======= - if (rus.getForced() == null) { - if (rus.getPactRatio() != -1 && !CheckpointUtils.useAVG()) { - System.out.println("Ratio = " + rus.getPactRatio()); - if (rus.getPactRatio() >= CPlower) { - // amount of data is small so we checkpoint - return true; - } - if (rus.getPactRatio() <= CPupper) { - // amount of data is too big - return false; - } - } else { - // no info from upper layer so use average sizes - if (rus.isDam()) { - System.out.println("is Dam"); - - if (rus.getAverageOutputRecordSize() != 0) { - System.out.println("avg ratio " + rus.getAverageInputRecordSize() ->>>>>>> marrus_checkpointing - / rus.getAverageOutputRecordSize()); + System.out.println("avg ratio " + rus.getAverageInputRecordSize() + / rus.getAverageOutputRecordSize()); } if (rus.getAverageOutputRecordSize() != 0 && - rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize() >= CPlower) { + rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize() >= CPlower) { return true; } if (rus.getAverageOutputRecordSize() != 0 && - rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize() <= CPupper) { + rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize() <= CPupper) { return false; } } else { @@ -97,43 +76,25 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap System.out.println("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); if (rus.getTotalInputAmount() != 0) { System.out.println("selektivity is " + (double) rus.getTotalOutputAmount() - / rus.getTotalInputAmount()); + / rus.getTotalInputAmount()); } if (rus.getTotalOutputAmount() != 0 - && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() >= CPlower)) { -<<<<<<< HEAD + && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() >= CPlower)) { // size off checkpoint would be to large: do not checkpoint // TODO progress estimation would make sense here - System.out.println(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " - + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() > 2.0)); + System.out.println(task.getEnvironment().getTaskName() + " Checkpoint to large selektivity " + + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount())); return false; -======= - // size off checkpoint would be to large: do not checkpoint - // TODO progress estimation would make sense here - System.out.println(task.getEnvironment().getTaskName() + " Checkpoint to large selektivity " - + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount())); - return false; ->>>>>>> marrus_checkpointing } if (rus.getTotalOutputAmount() != 0 - && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() <= CPupper)) { -<<<<<<< HEAD + && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() <= CPupper)) { // size of checkpoint will be small enough: checkpoint - // TODO progress estimation would make sense here - System.out.println(task.getEnvironment().getTaskName() + "Checkpoint to large selektivity " - + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() > 2.0)); + // TODO progress estimation would make sense here + System.out.println(task.getEnvironment().getTaskName() + " Checkpoint small selektivity " + + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount())); return true; -======= - // size of checkpoint will be small enough: checkpoint - // TODO progress estimation would make sense here - System.out.println(task.getEnvironment().getTaskName() + " Checkpoint small selektivity " - + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount())); - return true; - - } ->>>>>>> marrus_checkpointing } @@ -145,6 +106,7 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap // CPU bottleneck return true; } + System.out.println("Checkpoint decision false by default"); // in case of doubt do not checkpoint return false; From dffd3c98325600d123be4aa86604cd4b3e0b9e2e Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 22 Feb 2012 20:04:55 +0100 Subject: [PATCH 270/310] Refined configuration of checkpoints --- .../checkpointing/CheckpointDecision.java | 53 +++++++++++++------ .../nephele/checkpointing/CheckpointMode.java | 44 +++++++++++++++ .../checkpointing/CheckpointUtils.java | 22 ++++++-- 3 files changed, 100 insertions(+), 19 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointMode.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java index ef177503e71f2..e5d2e15e2e06f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java @@ -19,20 +19,27 @@ import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; -import eu.stratosphere.nephele.taskmanager.Task; +import eu.stratosphere.nephele.execution.RuntimeEnvironment; +import eu.stratosphere.nephele.io.channels.ChannelType; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; public final class CheckpointDecision { private static final Log LOG = LogFactory.getLog(CheckpointDecision.class); - public static boolean getDecision(final Task task, final ResourceUtilizationSnapshot rus) { + public static boolean getDecision(final RuntimeTask task, final ResourceUtilizationSnapshot rus) { - if (CheckpointUtils.isCheckpointingDisabled()) { + switch (CheckpointUtils.getCheckpointMode()) { + case NEVER: return false; + case ALWAYS: + return true; + case NETWORK: + return isNetworkTask(task); } if (rus.getForced() != null) { - System.out.println("Checkpoint decision was forced to " + rus.getForced()); + LOG.info("Checkpoint decision was forced to " + rus.getForced()); // checkpoint decision was forced by the user return rus.getForced(); } @@ -41,8 +48,8 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap final double CPupper = CheckpointUtils.getCPUpper(); - if (rus.getPactRatio() >= 0.0 && !CheckpointUtils.useAVG()) { - System.out.println("Ratio = " + rus.getPactRatio()); + if (rus.getPactRatio() >= 0.0 && CheckpointUtils.usePACT()) { + LOG.info("Ratio = " + rus.getPactRatio()); if (rus.getPactRatio() >= CPlower) { // amount of data is small so we checkpoint return true; @@ -54,10 +61,10 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap } else { // no info from upper layer so use average sizes if (rus.isDam()) { - System.out.println("is Dam"); + LOG.info("is Dam"); if (rus.getAverageOutputRecordSize() != 0) { - System.out.println("avg ratio " + rus.getAverageInputRecordSize() + LOG.info("avg ratio " + rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize()); } @@ -73,9 +80,9 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap } else { // we have no data dam so we can estimate the input/output-ratio - System.out.println("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); + LOG.info("out " + rus.getTotalOutputAmount() + " in " + rus.getTotalInputAmount()); if (rus.getTotalInputAmount() != 0) { - System.out.println("selektivity is " + (double) rus.getTotalOutputAmount() + LOG.info("Selectivity is " + (double) rus.getTotalOutputAmount() / rus.getTotalInputAmount()); } @@ -83,7 +90,7 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() >= CPlower)) { // size off checkpoint would be to large: do not checkpoint // TODO progress estimation would make sense here - System.out.println(task.getEnvironment().getTaskName() + " Checkpoint to large selektivity " + LOG.info(task.getEnvironment().getTaskName() + " Checkpoint too large selectivity " + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount())); return false; @@ -91,8 +98,8 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap if (rus.getTotalOutputAmount() != 0 && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() <= CPupper)) { // size of checkpoint will be small enough: checkpoint - // TODO progress estimation would make sense here - System.out.println(task.getEnvironment().getTaskName() + " Checkpoint small selektivity " + // TODO progress estimation would make sense here + LOG.info(task.getEnvironment().getTaskName() + " Checkpoint small selectivity " + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount())); return true; @@ -102,14 +109,28 @@ public static boolean getDecision(final Task task, final ResourceUtilizationSnap } // between thresholds check CPU Usage. if (rus.getUserCPU() >= 90) { - System.out.println(task.getEnvironment().getTaskName() + "CPU-Bottleneck"); + LOG.info(task.getEnvironment().getTaskName() + "CPU-Bottleneck"); // CPU bottleneck return true; } - - System.out.println("Checkpoint decision false by default"); + + LOG.info("Checkpoint decision false by default"); // in case of doubt do not checkpoint return false; } + + private static boolean isNetworkTask(final RuntimeTask task) { + + final RuntimeEnvironment environment = task.getRuntimeEnvironment(); + + for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { + + if (environment.getOutputGate(i).getChannelType() == ChannelType.NETWORK) { + return true; + } + } + + return false; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointMode.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointMode.java new file mode 100644 index 0000000000000..c30c4d81e9499 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointMode.java @@ -0,0 +1,44 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; + +/** + * This enumeration describes the different modes in which Nephele's checkpoints can be configured. + * + * @author warneke + */ +public enum CheckpointMode { + + /** + * Never create any checkpoint. + */ + NEVER, + + /** + * Create a checkpoint whenever possible. + */ + ALWAYS, + + /** + * Create a checkpoint if at least one output of the task is transfered over the network. + */ + NETWORK, + + /** + * Creates checkpoints according to Nephele's internal strategies. + */ + DYNAMIC; +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java index fb61950abfe5d..ab2d2ec2a1d38 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java @@ -37,6 +37,8 @@ public final class CheckpointUtils { private static double CP_LOWER = -1.0; + private static CheckpointMode CHECKPOINT_MODE = null; + private CheckpointUtils() { } @@ -103,9 +105,23 @@ public static void removeCheckpoint(final ExecutionVertexID vertexID) { } } - public static boolean isCheckpointingDisabled() { + public static CheckpointMode getCheckpointMode() { + + if (CHECKPOINT_MODE == null) { + + final String mode = GlobalConfiguration.getString("checkpoint.mode", "never").toLowerCase(); + if ("always".equals(mode)) { + CHECKPOINT_MODE = CheckpointMode.ALWAYS; + } else if ("netork".equals(mode)) { + CHECKPOINT_MODE = CheckpointMode.NETWORK; + } else if ("dynamic".equals(mode)) { + CHECKPOINT_MODE = CheckpointMode.DYNAMIC; + } else { + CHECKPOINT_MODE = CheckpointMode.NETWORK; + } + } - return GlobalConfiguration.getBoolean("checkpoint.no", false); + return CHECKPOINT_MODE; } public static double getCPLower() { @@ -140,6 +156,6 @@ public static String getSummary() { return "Checkpointing Summary: UpperBound=" + getCPUpper() + " LowerBound=" + getCPLower() + " ForcedValues: usePACT=" + usePACT() + " useAVG=" + useAVG() - + " NOCheckpoting=" + isCheckpointingDisabled(); + + " Mode=" + getCheckpointMode(); } } From 6eea6e4ae0ed77cba7c2043248308756b2fff83b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 22 Feb 2012 21:42:33 +0100 Subject: [PATCH 271/310] Fixed two bugs related to checkpoint configuration --- .../nephele/checkpointing/CheckpointDecision.java | 1 + .../stratosphere/nephele/checkpointing/CheckpointUtils.java | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java index e5d2e15e2e06f..be415383cfa1f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java @@ -127,6 +127,7 @@ private static boolean isNetworkTask(final RuntimeTask task) { for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { if (environment.getOutputGate(i).getChannelType() == ChannelType.NETWORK) { + LOG.info(environment.getTaskNameWithIndex() + " is a network task"); return true; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java index ab2d2ec2a1d38..e2cabf19d5bf5 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java @@ -112,12 +112,12 @@ public static CheckpointMode getCheckpointMode() { final String mode = GlobalConfiguration.getString("checkpoint.mode", "never").toLowerCase(); if ("always".equals(mode)) { CHECKPOINT_MODE = CheckpointMode.ALWAYS; - } else if ("netork".equals(mode)) { + } else if ("network".equals(mode)) { CHECKPOINT_MODE = CheckpointMode.NETWORK; } else if ("dynamic".equals(mode)) { CHECKPOINT_MODE = CheckpointMode.DYNAMIC; } else { - CHECKPOINT_MODE = CheckpointMode.NETWORK; + CHECKPOINT_MODE = CheckpointMode.NEVER; } } @@ -156,6 +156,6 @@ public static String getSummary() { return "Checkpointing Summary: UpperBound=" + getCPUpper() + " LowerBound=" + getCPLower() + " ForcedValues: usePACT=" + usePACT() + " useAVG=" + useAVG() - + " Mode=" + getCheckpointMode(); + + " mode=" + getCheckpointMode(); } } From 9cc2318d100337952cf080380e52b4ce3a619595 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 22 Feb 2012 21:43:58 +0100 Subject: [PATCH 272/310] Fixed wrong return value of getChannelType method in AbstractGate on task managers --- .../nephele/execution/RuntimeEnvironment.java | 27 +++++++++++++------ 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java index 6299ed0fb8c1e..e44a1d312c00a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java @@ -560,12 +560,15 @@ public void read(final DataInput in) throws IOException { for (int i = 0; i < numOuputGates; ++i) { final OutputGate outputGate = this.outputGates.get(i); final int numberOfOutputChannels = in.readInt(); + ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class); + outputGate.setChannelType(channelType); + for (int j = 0; j < numberOfOutputChannels; ++j) { final ChannelID channelID = new ChannelID(); channelID.read(in); final ChannelID connectedChannelID = new ChannelID(); connectedChannelID.read(in); - final ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class); + channelType = EnumUtils.readEnum(in, ChannelType.class); final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); AbstractOutputChannel outputChannel = null; @@ -597,12 +600,15 @@ public void read(final DataInput in) throws IOException { for (int i = 0; i < numInputGates; ++i) { final InputGate inputGate = this.inputGates.get(i); final int numberOfInputChannels = in.readInt(); + ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class); + inputGate.setChannelType(channelType); + for (int j = 0; j < numberOfInputChannels; ++j) { final ChannelID channelID = new ChannelID(); channelID.read(in); final ChannelID connectedChannelID = new ChannelID(); connectedChannelID.read(in); - final ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class); + channelType = EnumUtils.readEnum(in, ChannelType.class); final CompressionLevel compressionLevel = EnumUtils.readEnum(in, CompressionLevel.class); AbstractInputChannel inputChannel = null; @@ -691,6 +697,8 @@ public void write(final DataOutput out) throws IOException { final OutputGate outputGate = this.outputGates.get(i); final int numberOfOutputChannels = outputGate.getNumberOfOutputChannels(); out.writeInt(numberOfOutputChannels); + EnumUtils.writeEnum(out, outputGate.getChannelType()); + for (int j = 0; j < numberOfOutputChannels; ++j) { final AbstractOutputChannel outputChannel = outputGate.getOutputChannel(j); outputChannel.getID().write(out); @@ -705,6 +713,8 @@ public void write(final DataOutput out) throws IOException { final InputGate inputGate = this.inputGates.get(i); final int numberOfInputChannels = inputGate.getNumberOfInputChannels(); out.writeInt(numberOfInputChannels); + EnumUtils.writeEnum(out, inputGate.getChannelType()); + for (int j = 0; j < numberOfInputChannels; ++j) { final AbstractInputChannel inputChannel = inputGate.getInputChannel(j); inputChannel.getID().write(out); @@ -1150,23 +1160,24 @@ public Set getInputChannelIDsOfGate(final GateID gateID) { return Collections.unmodifiableSet(inputChannelIDs); } - + // DW: Start of temporary code @Override public void reportPACTDataStatistics(final long numberOfConsumedBytes, final long numberOfProducedBytes) { - - if(numberOfConsumedBytes < 0L) { + + if (numberOfConsumedBytes < 0L) { return; } - - if(numberOfProducedBytes < 0L) { + + if (numberOfProducedBytes < 0L) { return; } - + if (this.executionObserver != null) { this.executionObserver.reportPACTDataStatistics(numberOfConsumedBytes, numberOfProducedBytes); } } + // DW: End of temporary code @Override From 8fb9db26693f271429039fd46b54aa3765c93ae8 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 23 Feb 2012 22:02:46 +0100 Subject: [PATCH 273/310] Fixed bug in collection of PACT data statistics --- .../main/java/eu/stratosphere/pact/runtime/task/MapTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MapTask.java b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MapTask.java index abab62e451a6c..23d487fac1cf0 100644 --- a/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MapTask.java +++ b/pact/pact-runtime/src/main/java/eu/stratosphere/pact/runtime/task/MapTask.java @@ -86,7 +86,7 @@ public void run() throws Exception } while (this.running && input.next(record)) { // DW: Start to temporary code - consumedPactRecordsInBytes =+ record.getBinaryLength(); + consumedPactRecordsInBytes += record.getBinaryLength(); // DW: End of temporary code stub.map(record, output); From 1c53d5aa7da8730bdc610ef4e9482ef80cf51712 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 24 Feb 2012 16:48:50 +0100 Subject: [PATCH 274/310] Started to implemented distributed checkpoints --- .../checkpointing/CheckpointDecision.java | 44 ++-- .../checkpointing/CheckpointUtils.java | 5 + .../checkpointing/EphemeralCheckpoint.java | 27 ++- .../nephele/io/channels/BufferFactory.java | 9 +- .../io/channels/ChannelWithAccessInfo.java | 36 +++ .../DistributedChannelWithAccessInfo.java | 155 +++++++++++++ .../io/channels/DistributedFileChannel.java | 202 +++++++++++++++++ .../nephele/io/channels/FileBuffer.java | 11 +- .../io/channels/FileBufferManager.java | 208 +++++------------- .../channels/LocalChannelWithAccessInfo.java | 150 +++++++++++++ .../nephele/jobmanager/JobManager.java | 8 +- .../runtime/RuntimeInputGateContext.java | 3 +- .../CheckpointDeserializer.java | 2 +- .../SpillingQueueElement.java | 4 +- 14 files changed, 666 insertions(+), 198 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedFileChannel.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java index be415383cfa1f..8394f049c8cb4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java @@ -48,13 +48,13 @@ public static boolean getDecision(final RuntimeTask task, final ResourceUtilizat final double CPupper = CheckpointUtils.getCPUpper(); - if (rus.getPactRatio() >= 0.0 && CheckpointUtils.usePACT()) { + if (rus.getPactRatio() >= 0.0 && !CheckpointUtils.useAVG()) { LOG.info("Ratio = " + rus.getPactRatio()); - if (rus.getPactRatio() >= CPlower) { + if (rus.getPactRatio() <= CPlower) { // amount of data is small so we checkpoint return true; } - if (rus.getPactRatio() <= CPupper) { + if (rus.getPactRatio() >= CPupper) { // amount of data is too big return false; } @@ -63,18 +63,18 @@ public static boolean getDecision(final RuntimeTask task, final ResourceUtilizat if (rus.isDam()) { LOG.info("is Dam"); - if (rus.getAverageOutputRecordSize() != 0) { - LOG.info("avg ratio " + rus.getAverageInputRecordSize() - / rus.getAverageOutputRecordSize()); + if (rus.getAverageInputRecordSize() != 0) { + LOG.info("avg ratio " + rus.getAverageOutputRecordSize() + / rus.getAverageInputRecordSize()); } - if (rus.getAverageOutputRecordSize() != 0 && - rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize() >= CPlower) { + if (rus.getAverageInputRecordSize() != 0 && + rus.getAverageOutputRecordSize() / rus.getAverageInputRecordSize() <= CPlower) { return true; } - if (rus.getAverageOutputRecordSize() != 0 && - rus.getAverageInputRecordSize() / rus.getAverageOutputRecordSize() <= CPupper) { + if (rus.getAverageInputRecordSize() != 0 && + rus.getAverageOutputRecordSize() / rus.getAverageInputRecordSize() >= CPupper) { return false; } } else { @@ -86,24 +86,24 @@ public static boolean getDecision(final RuntimeTask task, final ResourceUtilizat / rus.getTotalInputAmount()); } - if (rus.getTotalOutputAmount() != 0 - && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() >= CPlower)) { - // size off checkpoint would be to large: do not checkpoint - // TODO progress estimation would make sense here - LOG.info(task.getEnvironment().getTaskName() + " Checkpoint too large selectivity " - + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount())); - return false; - - } - if (rus.getTotalOutputAmount() != 0 - && ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount() <= CPupper)) { + if (rus.getTotalInputAmount() != 0 + && ((double) rus.getTotalOutputAmount() / rus.getTotalInputAmount() <= CPlower)) { // size of checkpoint will be small enough: checkpoint // TODO progress estimation would make sense here LOG.info(task.getEnvironment().getTaskName() + " Checkpoint small selectivity " - + ((double) rus.getTotalInputAmount() / rus.getTotalOutputAmount())); + + ((double) rus.getTotalOutputAmount() / rus.getTotalInputAmount())); return true; } + if (rus.getTotalInputAmount() != 0 + && ((double) rus.getTotalOutputAmount() / rus.getTotalInputAmount() >= CPupper)) { + // size off checkpoint would be to large: do not checkpoint + // TODO progress estimation would make sense here + LOG.info(task.getEnvironment().getTaskName() + " Checkpoint to large selectivity " + + ((double) rus.getTotalOutputAmount() / rus.getTotalInputAmount())); + return false; + + } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java index e2cabf19d5bf5..cc4129a89f7af 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java @@ -152,6 +152,11 @@ public static boolean useAVG() { return GlobalConfiguration.getBoolean("checkpoint.useavg", false); } + public static boolean createDistributedCheckpoint() { + + return GlobalConfiguration.getBoolean("checkpoint.distributed", true); + } + public static String getSummary() { return "Checkpointing Summary: UpperBound=" + getCPUpper() + " LowerBound=" + getCPLower() diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 6c0fe62110bbe..e397870be3fdb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -85,6 +85,8 @@ public class EphemeralCheckpoint implements OutputChannelForwarder { */ private final int numberOfConnectedChannels; + private final boolean distributed; + /** * The number of channels which can confirmed not to send any further data. */ @@ -137,6 +139,8 @@ public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { } this.numberOfConnectedChannels = nooc; + this.distributed = CheckpointUtils.createDistributedCheckpoint(); + this.checkpointingDecision = (ephemeral ? CheckpointingDecisionState.UNDECIDED : CheckpointingDecisionState.CHECKPOINTING); @@ -200,7 +204,7 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro // Make sure we transfer the encapsulated buffer to a file and release the memory buffer again final Buffer fileBuffer = BufferFactory.createFromFile(buffer.size(), this.task.getVertexID(), - this.fileBufferManager); + this.fileBufferManager, this.distributed); buffer.copyToBuffer(fileBuffer); transferEnvelope.setBuffer(fileBuffer); buffer.recycleBuffer(); @@ -264,8 +268,8 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro + this.task.getVertexID() + "_final").close(); // Since it is unclear whether the underlying physical file will ever be read, we force to close it. - //TODO: Fix me - //this.fileBufferManager.forceCloseOfWritableSpillingFile(this.task.getVertexID()); + // TODO: Fix me + // this.fileBufferManager.forceCloseOfWritableSpillingFile(this.task.getVertexID()); LOG.info("Finished persistent checkpoint for vertex " + this.task.getVertexID()); @@ -273,14 +277,15 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro this.task.checkpointStateChanged(CheckpointState.COMPLETE); } } - - public void setCheckpointDecisionSynchronously(final boolean checkpointDecision) throws IOException, InterruptedException { - - if(this.checkpointingDecision != CheckpointingDecisionState.UNDECIDED) { + + public void setCheckpointDecisionSynchronously(final boolean checkpointDecision) throws IOException, + InterruptedException { + + if (this.checkpointingDecision != CheckpointingDecisionState.UNDECIDED) { return; } - - if(checkpointDecision) { + + if (checkpointDecision) { this.checkpointingDecision = CheckpointingDecisionState.CHECKPOINTING; // Write the data which has been queued so far and update checkpoint state write(); @@ -314,10 +319,10 @@ public boolean forward(final TransferEnvelope transferEnvelope) throws IOExcepti } public boolean isUndecided() { - + return (this.checkpointingDecision == CheckpointingDecisionState.UNDECIDED); } - + /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java index 3e2618be85c99..ce3b5e1243caf 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java @@ -23,15 +23,16 @@ public final class BufferFactory { public static FileBuffer createFromFile(final int bufferSize, final AbstractID ownerID, - final FileBufferManager fileBufferManager) throws IOException { + final FileBufferManager fileBufferManager, final boolean distributed) throws IOException { - return new FileBuffer(bufferSize, ownerID, fileBufferManager); + return new FileBuffer(bufferSize, ownerID, fileBufferManager, distributed); } public static FileBuffer createFromCheckpoint(final int bufferSize, final long offset, - final AbstractID ownerID, final FileBufferManager fileBufferManager) throws IOException { + final AbstractID ownerID, final FileBufferManager fileBufferManager, final boolean distributed) + throws IOException { - return new FileBuffer(bufferSize, offset, ownerID, fileBufferManager); + return new FileBuffer(bufferSize, offset, ownerID, fileBufferManager, distributed); } public static Buffer createFromMemory(final int bufferSize, final ByteBuffer byteBuffer, diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java new file mode 100644 index 0000000000000..935eb4ebd696f --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java @@ -0,0 +1,36 @@ +package eu.stratosphere.nephele.io.channels; + +import java.nio.channels.FileChannel; + +interface ChannelWithAccessInfo { + + FileChannel getChannel(); + + FileChannel getAndIncrementReferences(); + + /** + * Increments the references to this channel. Returns true, if successful, and false, + * if the channel has been disposed in the meantime. + * + * @return True, if successful, false, if the channel has been disposed. + */ + boolean incrementReferences(); + + ChannelWithPosition reserveWriteSpaceAndIncrementReferences(int spaceToReserve); + + /** + * Decrements the number of references to this channel. If the number of references is zero after the + * decrement, the channel is deleted. + * + * @return The number of references remaining after the decrement. + * @throws IllegalStateException + * Thrown, if the number of references is already zero or below. + */ + int decrementReferences(); + + /** + * Disposes the channel without further notice. Tries to close it (swallowing all exceptions) and tries + * to delete the file. + */ + void disposeSilently(); +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java new file mode 100644 index 0000000000000..98f21199f1686 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java @@ -0,0 +1,155 @@ +package eu.stratosphere.nephele.io.channels; + +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.fs.FileSystem; +import eu.stratosphere.nephele.fs.Path; + +final class DistributedChannelWithAccessInfo implements ChannelWithAccessInfo { + + /** + * The logging object. + */ + private static final Log LOG = LogFactory.getLog(DistributedChannelWithAccessInfo.class); + + private final FileSystem fs; + + private final Path checkpointFile; + + private final DistributedFileChannel channel; + + private final AtomicLong reservedWritePosition; + + private final AtomicInteger referenceCounter; + + DistributedChannelWithAccessInfo(final FileSystem fs, final Path checkpointFile, final int bufferSize) + throws IOException { + + this.fs = fs; + this.checkpointFile = checkpointFile; + this.channel = new DistributedFileChannel(fs, checkpointFile, bufferSize); + this.reservedWritePosition = new AtomicLong(0); + this.referenceCounter = new AtomicInteger(0); + } + + /** + * {@inheritDoc} + */ + @Override + public FileChannel getChannel() { + + return this.channel; + } + + /** + * {@inheritDoc} + */ + @Override + public FileChannel getAndIncrementReferences() { + + if (incrementReferences()) { + return this.channel; + } else { + return null; + } + } + + @Override + public ChannelWithPosition reserveWriteSpaceAndIncrementReferences(final int spaceToReserve) { + + if (incrementReferences()) { + return new ChannelWithPosition(this.channel, this.reservedWritePosition.getAndAdd(spaceToReserve)); + } else { + return null; + } + } + + /** + * {@inheritDoc} + */ + @Override + public int decrementReferences() { + + int current = this.referenceCounter.get(); + while (true) { + if (current <= 0) { + // this is actually an error case, because the channel was deleted before + throw new IllegalStateException("The references to the file were already at zero."); + } + + if (current == 1) { + // this call decrements to zero, so mark it as deleted + if (this.referenceCounter.compareAndSet(current, Integer.MIN_VALUE)) { + current = 0; + break; + } + } else if (this.referenceCounter.compareAndSet(current, current - 1)) { + current = current - 1; + break; + } + current = this.referenceCounter.get(); + } + + if (current > 0) { + return current; + } else if (current == 0) { + // delete the channel + this.referenceCounter.set(Integer.MIN_VALUE); + this.reservedWritePosition.set(Long.MIN_VALUE); + try { + this.channel.close(); + this.fs.delete(this.checkpointFile, false); + } catch (IOException ioex) { + if (LOG.isErrorEnabled()) + LOG.error("Error while closing spill file for file buffers: " + ioex.getMessage(), ioex); + } + return current; + } else { + throw new IllegalStateException("The references to the file were already at zero."); + } + } + + /** + * {@inheritDoc} + */ + @Override + public boolean incrementReferences() { + + int current = this.referenceCounter.get(); + while (true) { + // check whether it was disposed in the meantime + if (current < 0) { + return false; + } + // atomically check and increment + if (this.referenceCounter.compareAndSet(current, current + 1)) { + return true; + } + current = this.referenceCounter.get(); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void disposeSilently() { + + this.referenceCounter.set(Integer.MIN_VALUE); + this.reservedWritePosition.set(Long.MIN_VALUE); + + if (this.channel.isOpen()) { + try { + this.channel.close(); + this.fs.delete(this.checkpointFile, false); + } catch (Throwable t) { + } + } + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedFileChannel.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedFileChannel.java new file mode 100644 index 0000000000000..d141d3c5d51bf --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedFileChannel.java @@ -0,0 +1,202 @@ +package eu.stratosphere.nephele.io.channels; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.MappedByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; + +import eu.stratosphere.nephele.fs.FSDataOutputStream; +import eu.stratosphere.nephele.fs.FileSystem; +import eu.stratosphere.nephele.fs.Path; + +final class DistributedFileChannel extends FileChannel { + + private static final short REPLICATION = 2; + + private final FileSystem fs; + + private final Path checkpointFile; + + private final byte[] buf; + + private FSDataOutputStream outputStream = null; + + private long nextExpectedWritePosition = 0L; + + DistributedFileChannel(final FileSystem fs, final Path checkpointFile, final int bufferSize) { + + this.fs = fs; + this.checkpointFile = checkpointFile; + this.buf = new byte[bufferSize]; + } + + @Override + public void force(boolean metaData) throws IOException { + // TODO Auto-generated method stub + System.out.println("force called"); + } + + @Override + public FileLock lock(long position, long size, boolean shared) throws IOException { + // TODO Auto-generated method stub + + System.out.println("lock called"); + + return null; + } + + @Override + public MappedByteBuffer map(MapMode mode, long position, long size) throws IOException { + // TODO Auto-generated method stub + + System.out.println("map called"); + + return null; + } + + @Override + public long position() throws IOException { + // TODO Auto-generated method stub + + System.out.println("position called"); + + return 0; + } + + @Override + public FileChannel position(long newPosition) throws IOException { + // TODO Auto-generated method stub + + System.out.println("position2 called"); + + return null; + } + + @Override + public int read(ByteBuffer dst) throws IOException { + // TODO Auto-generated method stub + + System.out.println("read called"); + + return 0; + } + + @Override + public int read(ByteBuffer dst, long position) throws IOException { + // TODO Auto-generated method stub + + System.out.println("read2 called"); + + return 0; + } + + @Override + public long read(ByteBuffer[] dsts, int offset, int length) throws IOException { + // TODO Auto-generated method stub + + System.out.println("read3 called"); + + return 0; + } + + @Override + public long size() throws IOException { + // TODO Auto-generated method stub + + System.out.println("size called"); + + return 0; + } + + @Override + public long transferFrom(ReadableByteChannel src, long position, long count) throws IOException { + + System.out.println("transferFrom called"); + + // TODO Auto-generated method stub + return 0; + } + + @Override + public long transferTo(long position, long count, WritableByteChannel target) throws IOException { + + System.out.println("transferTo called"); + + // TODO Auto-generated method stub + return 0; + } + + @Override + public FileChannel truncate(long size) throws IOException { + + System.out.println("truncate called"); + + // TODO Auto-generated method stub + return null; + } + + @Override + public FileLock tryLock(long position, long size, boolean shared) throws IOException { + + System.out.println("tryLock called"); + + // TODO Auto-generated method stub + return null; + } + + @Override + public int write(ByteBuffer src) throws IOException { + + System.out.println("write called"); + + // TODO Auto-generated method stub + return 0; + } + + @Override + public int write(final ByteBuffer src, final long position) throws IOException { + + if (position != this.nextExpectedWritePosition) { + throw new IOException("Next expected write position is " + this.nextExpectedWritePosition); + } + + if (this.outputStream == null) { + this.outputStream = this.fs.create(this.checkpointFile, false, this.buf.length, REPLICATION, + this.fs.getDefaultBlockSize()); + } + + int totalBytesWritten = 0; + + while (src.hasRemaining()) { + + final int length = Math.min(this.buf.length, src.remaining()); + src.get(this.buf, 0, length); + this.outputStream.write(this.buf, 0, length); + totalBytesWritten += length; + } + + this.nextExpectedWritePosition += totalBytesWritten; + + return totalBytesWritten; + } + + @Override + public long write(ByteBuffer[] srcs, int offset, int length) throws IOException { + + System.out.println("write3 called"); + + // TODO Auto-generated method stub + return 0; + } + + @Override + protected void implCloseChannel() throws IOException { + // TODO Auto-generated method stub + + System.out.println("implCloseChannel called"); + } + +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java index 75563c4f8b030..e514baad38010 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java @@ -50,7 +50,7 @@ public final class FileBuffer extends Buffer { * @param ownerID */ FileBuffer(final int bufferSize, final long offset, final AbstractID ownerID, - final FileBufferManager fileBufferManager) throws IOException { + final FileBufferManager fileBufferManager, final boolean distributed) throws IOException { this.ownerID = ownerID; this.fileBufferManager = fileBufferManager; @@ -58,7 +58,7 @@ public final class FileBuffer extends Buffer { this.bufferSize = bufferSize; this.writeMode.set(false); - this.fileChannel = fileBufferManager.getChannelAndIncrementReferences(ownerID); + this.fileChannel = fileBufferManager.getChannelAndIncrementReferences(ownerID, distributed); } /** @@ -69,15 +69,16 @@ public final class FileBuffer extends Buffer { * @param fileBufferManager * @throws IOException */ - FileBuffer(final int bufferSize, final AbstractID ownerID, final FileBufferManager fileBufferManager) - throws IOException { + FileBuffer(final int bufferSize, final AbstractID ownerID, final FileBufferManager fileBufferManager, + final boolean distributed) throws IOException { this.fileBufferManager = fileBufferManager; this.ownerID = ownerID; this.bufferSize = bufferSize; this.writeMode.set(true); - final ChannelWithPosition cwp = fileBufferManager.getChannelForWriteAndIncrementReferences(ownerID, bufferSize); + final ChannelWithPosition cwp = fileBufferManager.getChannelForWriteAndIncrementReferences(ownerID, bufferSize, + distributed); this.fileChannel = cwp.getChannel(); this.offset = cwp.getOffset(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java index 3e84ec5397229..bdc3b45b7b499 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java @@ -17,20 +17,20 @@ import java.io.File; import java.io.IOException; -import java.io.RandomAccessFile; import java.nio.channels.FileChannel; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.configuration.ConfigConstants; import eu.stratosphere.nephele.configuration.GlobalConfiguration; +import eu.stratosphere.nephele.fs.FileSystem; +import eu.stratosphere.nephele.fs.Path; import eu.stratosphere.nephele.io.AbstractID; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; +import eu.stratosphere.nephele.util.StringUtils; /** * The file buffer manager manages the physical files which may be used to store the output or input of @@ -79,6 +79,12 @@ public static FileBufferManager getInstance() { */ private final String[] tmpDirs; + private final int bufferSize; + + private final String distributedTempPath; + + private final FileSystem distributedFileSystem; + /** * Constructs a new file buffer manager object. */ @@ -98,7 +104,32 @@ private FileBufferManager() { this.tmpDirs[i] = this.tmpDirs[i] + File.separator + FILE_BUFFER_PREFIX; } + this.bufferSize = GlobalConfiguration.getInteger("channel.network.bufferSizeInBytes", 64 * 1024); // TODO: Use + // config + // constants + // here + this.fileMap = new ConcurrentHashMap(2048, 0.8f, 64); + + this.distributedTempPath = GlobalConfiguration.getString("taskmanager.tmp.distdir", + "hdfs://master:9000/checkpoints"); + FileSystem distFS = null; + if (this.distributedTempPath != null) { + + try { + + final Path p = new Path(this.distributedTempPath); + distFS = FileSystem.get(p.toUri()); + if (!distFS.exists(p)) { + distFS.mkdirs(p); + } + + } catch (IOException e) { + LOG.error(StringUtils.stringifyException(e)); + } + } + + this.distributedFileSystem = distFS; } // -------------------------------------------------------------------------------------------- @@ -111,9 +142,9 @@ private FileBufferManager() { * @throws IllegalStateException * Thrown, if the channel has not been registered or has already been removed. */ - public FileChannel getChannel(final AbstractID id) throws IOException { + public FileChannel getChannel(final AbstractID id, final boolean distributed) throws IOException { - final ChannelWithAccessInfo info = getChannelInternal(id, false); + final ChannelWithAccessInfo info = getChannelInternal(id, false, distributed); if (info != null) { return info.getChannel(); } else { @@ -129,9 +160,10 @@ public FileChannel getChannel(final AbstractID id) throws IOException { * @throws IllegalStateException * Thrown, if the channel has not been registered or has already been removed. */ - public FileChannel getChannelAndIncrementReferences(final AbstractID owner) throws IOException { + public FileChannel getChannelAndIncrementReferences(final AbstractID owner, final boolean distributed) + throws IOException { - final ChannelWithAccessInfo info = getChannelInternal(owner, false); + final ChannelWithAccessInfo info = getChannelInternal(owner, false, distributed); if (info != null) { return info.getAndIncrementReferences(); } else { @@ -149,15 +181,15 @@ public FileChannel getChannelAndIncrementReferences(final AbstractID owner) thro * @param id * The id for which to get the channel and reserve space. */ - public ChannelWithPosition getChannelForWriteAndIncrementReferences(final AbstractID id, final int spaceToReserve) - throws IOException { + public ChannelWithPosition getChannelForWriteAndIncrementReferences(final AbstractID id, final int spaceToReserve, + final boolean distributed) throws IOException { ChannelWithPosition c = null; do { // the return value may be zero, if someone asynchronously decremented the counter to zero // and caused the disposal of the channel. falling through the loop will create a // new channel. - c = getChannelInternal(id, true).reserveWriteSpaceAndIncrementReferences(spaceToReserve); + c = getChannelInternal(id, true, distributed).reserveWriteSpaceAndIncrementReferences(spaceToReserve); } while (c == null); return c; @@ -201,18 +233,29 @@ public void decrementReferences(final AbstractID id) { // -------------------------------------------------------------------------------------------- - private final ChannelWithAccessInfo getChannelInternal(final AbstractID id, final boolean createIfAbsent) - throws IOException { + private final ChannelWithAccessInfo getChannelInternal(final AbstractID id, final boolean createIfAbsent, + final boolean distributed) throws IOException { ChannelWithAccessInfo cwa = this.fileMap.get(id); if (cwa == null) { if (createIfAbsent) { - // Construct the filename - final int dirIndex = Math.abs(id.hashCode()) % this.tmpDirs.length; - final File file = new File(this.tmpDirs[dirIndex] + id.toString()); + if (distributed && this.distributedFileSystem != null) { + + final String checkpointFile = this.distributedTempPath + File.separator + FILE_BUFFER_PREFIX + + id.toString(); + final Path p = new Path(checkpointFile); + cwa = new DistributedChannelWithAccessInfo(this.distributedFileSystem, p, this.bufferSize); + + } else { + + // Construct the filename + final int dirIndex = Math.abs(id.hashCode()) % this.tmpDirs.length; + final File file = new File(this.tmpDirs[dirIndex] + id.toString()); + + cwa = new LocalChannelWithAccessInfo(file); + } - cwa = new ChannelWithAccessInfo(file); final ChannelWithAccessInfo alreadyContained = this.fileMap.putIfAbsent(id, cwa); if (alreadyContained != null) { // we had a race (should be a very rare event) and have created an @@ -220,6 +263,7 @@ private final ChannelWithAccessInfo getChannelInternal(final AbstractID id, fina cwa.disposeSilently(); cwa = alreadyContained; } + } else { return null; } @@ -227,138 +271,6 @@ private final ChannelWithAccessInfo getChannelInternal(final AbstractID id, fina return cwa; } - // -------------------------------------------------------------------------------------------- - private static final class ChannelWithAccessInfo { - - private final File file; - - private final FileChannel channel; - - private final AtomicLong reservedWritePosition; - - private final AtomicInteger referenceCounter; - - private ChannelWithAccessInfo(final File file) throws IOException { - - this.file = file; - this.channel = new RandomAccessFile(file, "rw").getChannel(); - this.reservedWritePosition = new AtomicLong(0); - this.referenceCounter = new AtomicInteger(0); - } - - FileChannel getChannel() { - - return this.channel; - } - - FileChannel getAndIncrementReferences() { - - if (incrementReferences()) { - return this.channel; - } else { - return null; - } - } - - ChannelWithPosition reserveWriteSpaceAndIncrementReferences(final int spaceToReserve) { - - if (incrementReferences()) { - return new ChannelWithPosition(this.channel, this.reservedWritePosition.getAndAdd(spaceToReserve)); - } else { - return null; - } - } - - /** - * Decrements the number of references to this channel. If the number of references is zero after the - * decrement, the channel is deleted. - * - * @return The number of references remaining after the decrement. - * @throws IllegalStateException - * Thrown, if the number of references is already zero or below. - */ - int decrementReferences() { - - int current = this.referenceCounter.get(); - while (true) { - if (current <= 0) { - // this is actually an error case, because the channel was deleted before - throw new IllegalStateException("The references to the file were already at zero."); - } - - if (current == 1) { - // this call decrements to zero, so mark it as deleted - if (this.referenceCounter.compareAndSet(current, Integer.MIN_VALUE)) { - current = 0; - break; - } - } else if (this.referenceCounter.compareAndSet(current, current - 1)) { - current = current - 1; - break; - } - current = this.referenceCounter.get(); - } - - if (current > 0) { - return current; - } else if (current == 0) { - // delete the channel - this.referenceCounter.set(Integer.MIN_VALUE); - this.reservedWritePosition.set(Long.MIN_VALUE); - try { - this.channel.close(); - } catch (IOException ioex) { - if (FileBufferManager.LOG.isErrorEnabled()) - FileBufferManager.LOG.error("Error while closing spill file for file buffers: " + - ioex.getMessage(), ioex); - } - this.file.delete(); - return current; - } else { - throw new IllegalStateException("The references to the file were already at zero."); - } - } - - /** - * Increments the references to this channel. Returns true, if successful, and false, - * if the channel has been disposed in the meantime. - * - * @return True, if successful, false, if the channel has been disposed. - */ - boolean incrementReferences() { - - int current = this.referenceCounter.get(); - while (true) { - // check whether it was disposed in the meantime - if (current < 0) { - return false; - } - // atomically check and increment - if (this.referenceCounter.compareAndSet(current, current + 1)) { - return true; - } - current = this.referenceCounter.get(); - } - } - - /** - * Disposes the channel without further notice. Tries to close it (swallowing all exceptions) and tries - * to delete the file. - */ - void disposeSilently() { - - this.referenceCounter.set(Integer.MIN_VALUE); - this.reservedWritePosition.set(Long.MIN_VALUE); - - if (this.channel.isOpen()) { - try { - this.channel.close(); - } catch (Throwable t) { - } - } - this.file.delete(); - } - } } \ No newline at end of file diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java new file mode 100644 index 0000000000000..b388b7d1a13cd --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java @@ -0,0 +1,150 @@ +package eu.stratosphere.nephele.io.channels; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.channels.FileChannel; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +final class LocalChannelWithAccessInfo implements ChannelWithAccessInfo { + + /** + * The logging object. + */ + private static final Log LOG = LogFactory.getLog(LocalChannelWithAccessInfo.class); + + private final File file; + + private final FileChannel channel; + + private final AtomicLong reservedWritePosition; + + private final AtomicInteger referenceCounter; + + LocalChannelWithAccessInfo(final File file) throws IOException { + + this.file = file; + this.channel = new RandomAccessFile(file, "rw").getChannel(); + this.reservedWritePosition = new AtomicLong(0); + this.referenceCounter = new AtomicInteger(0); + } + + /** + * {@inheritDoc} + */ + @Override + public FileChannel getChannel() { + + return this.channel; + } + + /** + * {@inheritDoc} + */ + @Override + public FileChannel getAndIncrementReferences() { + + if (incrementReferences()) { + return this.channel; + } else { + return null; + } + } + + @Override + public ChannelWithPosition reserveWriteSpaceAndIncrementReferences(final int spaceToReserve) { + + if (incrementReferences()) { + return new ChannelWithPosition(this.channel, this.reservedWritePosition.getAndAdd(spaceToReserve)); + } else { + return null; + } + } + + /** + * {@inheritDoc} + */ + @Override + public int decrementReferences() { + + int current = this.referenceCounter.get(); + while (true) { + if (current <= 0) { + // this is actually an error case, because the channel was deleted before + throw new IllegalStateException("The references to the file were already at zero."); + } + + if (current == 1) { + // this call decrements to zero, so mark it as deleted + if (this.referenceCounter.compareAndSet(current, Integer.MIN_VALUE)) { + current = 0; + break; + } + } else if (this.referenceCounter.compareAndSet(current, current - 1)) { + current = current - 1; + break; + } + current = this.referenceCounter.get(); + } + + if (current > 0) { + return current; + } else if (current == 0) { + // delete the channel + this.referenceCounter.set(Integer.MIN_VALUE); + this.reservedWritePosition.set(Long.MIN_VALUE); + try { + this.channel.close(); + } catch (IOException ioex) { + if (LOG.isErrorEnabled()) + LOG.error("Error while closing spill file for file buffers: " + ioex.getMessage(), ioex); + } + this.file.delete(); + return current; + } else { + throw new IllegalStateException("The references to the file were already at zero."); + } + } + + /** + * {@inheritDoc} + */ + @Override + public boolean incrementReferences() { + + int current = this.referenceCounter.get(); + while (true) { + // check whether it was disposed in the meantime + if (current < 0) { + return false; + } + // atomically check and increment + if (this.referenceCounter.compareAndSet(current, current + 1)) { + return true; + } + current = this.referenceCounter.get(); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void disposeSilently() { + + this.referenceCounter.set(Integer.MIN_VALUE); + this.reservedWritePosition.set(Long.MIN_VALUE); + + if (this.channel.isOpen()) { + try { + this.channel.close(); + } catch (Throwable t) { + } + } + this.file.delete(); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 5f0cec76c879f..724301836de4c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -757,7 +757,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio if (assignedInstance == null) { LOG.error("Cannot resolve lookup: vertex found for channel ID " + connectedChannelID + " but no instance assigned"); - LOG.info("Created receiverNotReady for " + connectedVertex + " 1"); + //LOG.info("Created receiverNotReady for " + connectedVertex + " 1"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } @@ -769,7 +769,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.REPLAYING && executionState != ExecutionState.FINISHING) { - LOG.info("Created receiverNotReady for " + connectedVertex + " in state " + executionState + " 2"); + //LOG.info("Created receiverNotReady for " + connectedVertex + " in state " + executionState + " 2"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } @@ -801,7 +801,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.REPLAYING && executionState != ExecutionState.FINISHING && executionState != ExecutionState.FINISHED) { - LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3"); + //LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } @@ -810,7 +810,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio LOG.error("Cannot resolve lookup: vertex found for channel ID " + outputChannel.getConnectedChannelID() + " but no instance assigned"); - LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4"); + //LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index 900ad154756db..730cbfce06caf 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -82,7 +82,8 @@ public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws I } if (this.envelopeConsumptionLog.followsLog() || this.inputGate.getChannelType() == ChannelType.NETWORK) { - return BufferFactory.createFromFile(minimumSizeOfBuffer, this.inputGate.getGateID(), fileBufferManager); + return BufferFactory.createFromFile(minimumSizeOfBuffer, this.inputGate.getGateID(), + this.fileBufferManager, false); } return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java index 5855274de9dcd..577b6dbe9c6ba 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java @@ -57,7 +57,7 @@ protected boolean readBufferData(final ReadableByteChannel readableByteChannel) final long offset = byteBufferToLong(tempBuffer); final Buffer fileBuffer = BufferFactory.createFromCheckpoint(getSizeOfBuffer(), offset, this.ownerID, - this.fileBufferManager); + this.fileBufferManager, true); setBuffer(fileBuffer); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElement.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElement.java index c498393f3f97e..c83b9d697935f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElement.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElement.java @@ -360,7 +360,7 @@ int spill(final AbstractID ownerID, final FileBufferManager fileBufferManager) t } final int size = buffer.size(); - final Buffer fileBuffer = BufferFactory.createFromFile(size, ownerID, fileBufferManager); + final Buffer fileBuffer = BufferFactory.createFromFile(size, ownerID, fileBufferManager, false); buffer.copyToBuffer(fileBuffer); this.bufferRef = fileBuffer; buffer.recycleBuffer(); @@ -389,7 +389,7 @@ int spill(final AbstractID ownerID, final FileBufferManager fileBufferManager) t } final int size = buffer.size(); - final Buffer fileBuffer = BufferFactory.createFromFile(size, ownerID, fileBufferManager); + final Buffer fileBuffer = BufferFactory.createFromFile(size, ownerID, fileBufferManager, false); buffer.copyToBuffer(fileBuffer); bufferQueue.offer(fileBuffer); buffer.recycleBuffer(); From 8dda08702eb3dd8b6c472ddf32baba1ffa64bb9f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 24 Feb 2012 18:12:32 +0100 Subject: [PATCH 275/310] Extended file system API to support renaming of files --- .../java/eu/stratosphere/nephele/fs/FileSystem.java | 12 ++++++++++++ .../nephele/fs/file/LocalFileSystem.java | 12 ++++++++++++ .../nephele/fs/hdfs/DistributedFileSystem.java | 10 ++++++++++ .../eu/stratosphere/nephele/fs/s3/S3FileSystem.java | 9 +++++++++ 4 files changed, 43 insertions(+) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileSystem.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileSystem.java index 9310f007c8e04..dd429bbcf591d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileSystem.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileSystem.java @@ -377,6 +377,18 @@ public abstract FSDataOutputStream create(Path f, boolean overwrite, int bufferS */ public abstract FSDataOutputStream create(Path f, boolean overwrite) throws IOException; + /** + * Renames the file/directory src to dst. + * + * @param src + * the file/directory to rename + * @param dst + * the new name of the file/directory + * @return true if the renaming was successful, false otherwise + * @throws IOException + */ + public abstract boolean rename(Path src, Path dst) throws IOException; + /** * Returns the number of blocks this file/directory consists of * assuming the file system's standard block size. diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalFileSystem.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalFileSystem.java index 4984e1a981da6..d84e156ec7f06 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalFileSystem.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/file/LocalFileSystem.java @@ -282,4 +282,16 @@ public FSDataOutputStream create(final Path f, final boolean overwrite) throws I return create(f, overwrite, 0, (short) 0, 0); } + + /** + * {@inheritDoc} + */ + @Override + public boolean rename(final Path src, final Path dst) throws IOException { + + final File srcFile = pathToFile(src); + final File dstFile = pathToFile(dst); + + return srcFile.renameTo(dstFile); + } } diff --git a/nephele/nephele-hdfs/src/main/java/eu/stratosphere/nephele/fs/hdfs/DistributedFileSystem.java b/nephele/nephele-hdfs/src/main/java/eu/stratosphere/nephele/fs/hdfs/DistributedFileSystem.java index 224932ea2f562..e8e6bd74b6e64 100644 --- a/nephele/nephele-hdfs/src/main/java/eu/stratosphere/nephele/fs/hdfs/DistributedFileSystem.java +++ b/nephele/nephele-hdfs/src/main/java/eu/stratosphere/nephele/fs/hdfs/DistributedFileSystem.java @@ -239,4 +239,14 @@ public boolean mkdirs(final Path f) throws IOException { return this.fs.mkdirs(new org.apache.hadoop.fs.Path(f.toString())); } + /** + * {@inheritDoc} + */ + @Override + public boolean rename(final Path src, final Path dst) throws IOException { + + return this.fs.rename(new org.apache.hadoop.fs.Path(src.toString()), + new org.apache.hadoop.fs.Path(dst.toString())); + } + } diff --git a/nephele/nephele-s3/src/main/java/eu/stratosphere/nephele/fs/s3/S3FileSystem.java b/nephele/nephele-s3/src/main/java/eu/stratosphere/nephele/fs/s3/S3FileSystem.java index b704b479cef3f..1fc8d21abb603 100644 --- a/nephele/nephele-s3/src/main/java/eu/stratosphere/nephele/fs/s3/S3FileSystem.java +++ b/nephele/nephele-s3/src/main/java/eu/stratosphere/nephele/fs/s3/S3FileSystem.java @@ -777,4 +777,13 @@ static Path extendPath(final Path parent, final String extension) throws IOExcep throw new IOException(StringUtils.stringifyException(e)); } } + + /** + * {@inheritDoc} + */ + @Override + public boolean rename(final Path src, final Path dst) throws IOException { + + throw new UnsupportedOperationException("This method is not yet implemented"); + } } From c7d4cfa7173eca0825cdd138965bcc9cd4344527 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 24 Feb 2012 19:53:38 +0100 Subject: [PATCH 276/310] Continued to work on implementation of distributed checkpoints --- .../nephele/fs/FileChannelWrapper.java} | 51 ++++---- .../checkpointing/CheckpointUtils.java | 115 ++++++++++++------ .../checkpointing/EphemeralCheckpoint.java | 96 +++++++++++---- .../nephele/checkpointing/ReplayTask.java | 15 ++- .../DistributedChannelWithAccessInfo.java | 5 +- .../io/channels/FileBufferManager.java | 13 +- .../nephele/taskmanager/TaskManager.java | 2 +- 7 files changed, 204 insertions(+), 93 deletions(-) rename nephele/{nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedFileChannel.java => nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java} (84%) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedFileChannel.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java similarity index 84% rename from nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedFileChannel.java rename to nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java index d141d3c5d51bf..a5d5c6aaa28ff 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedFileChannel.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java @@ -1,4 +1,4 @@ -package eu.stratosphere.nephele.io.channels; +package eu.stratosphere.nephele.fs; import java.io.IOException; import java.nio.ByteBuffer; @@ -8,13 +8,7 @@ import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; -import eu.stratosphere.nephele.fs.FSDataOutputStream; -import eu.stratosphere.nephele.fs.FileSystem; -import eu.stratosphere.nephele.fs.Path; - -final class DistributedFileChannel extends FileChannel { - - private static final short REPLICATION = 2; +public final class FileChannelWrapper extends FileChannel { private final FileSystem fs; @@ -22,15 +16,19 @@ final class DistributedFileChannel extends FileChannel { private final byte[] buf; + private final short replication; + private FSDataOutputStream outputStream = null; private long nextExpectedWritePosition = 0L; - DistributedFileChannel(final FileSystem fs, final Path checkpointFile, final int bufferSize) { + public FileChannelWrapper(final FileSystem fs, final Path checkpointFile, final int bufferSize, + final short replication) { this.fs = fs; this.checkpointFile = checkpointFile; this.buf = new byte[bufferSize]; + this.replication = replication; } @Override @@ -147,15 +145,18 @@ public FileLock tryLock(long position, long size, boolean shared) throws IOExcep return null; } + /** + * {@inheritDoc} + */ @Override public int write(ByteBuffer src) throws IOException { - System.out.println("write called"); - - // TODO Auto-generated method stub - return 0; + return write(src, this.nextExpectedWritePosition); } + /** + * {@inheritDoc} + */ @Override public int write(final ByteBuffer src, final long position) throws IOException { @@ -163,10 +164,7 @@ public int write(final ByteBuffer src, final long position) throws IOException { throw new IOException("Next expected write position is " + this.nextExpectedWritePosition); } - if (this.outputStream == null) { - this.outputStream = this.fs.create(this.checkpointFile, false, this.buf.length, REPLICATION, - this.fs.getDefaultBlockSize()); - } + final FSDataOutputStream outputStream = getOutputStream(); int totalBytesWritten = 0; @@ -174,7 +172,7 @@ public int write(final ByteBuffer src, final long position) throws IOException { final int length = Math.min(this.buf.length, src.remaining()); src.get(this.buf, 0, length); - this.outputStream.write(this.buf, 0, length); + outputStream.write(this.buf, 0, length); totalBytesWritten += length; } @@ -192,11 +190,22 @@ public long write(ByteBuffer[] srcs, int offset, int length) throws IOException return 0; } + private FSDataOutputStream getOutputStream() throws IOException { + + if (this.outputStream == null) { + this.outputStream = this.fs.create(this.checkpointFile, false, this.buf.length, this.replication, + this.fs.getDefaultBlockSize()); + } + + return this.outputStream; + } + + /** + * {@inheritDoc} + */ @Override protected void implCloseChannel() throws IOException { - // TODO Auto-generated method stub - System.out.println("implCloseChannel called"); + getOutputStream().close(); } - } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java index cc4129a89f7af..9f8ad9760d161 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java @@ -15,10 +15,12 @@ package eu.stratosphere.nephele.checkpointing; -import java.io.File; +import java.io.IOException; import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.fs.FileSystem; +import eu.stratosphere.nephele.fs.Path; public final class CheckpointUtils { @@ -27,11 +29,17 @@ public final class CheckpointUtils { */ public static final String METADATA_PREFIX = "checkpoint"; - public static final String CHECKPOINT_DIRECTORY_KEY = "channel.checkpoint.directory"; + public static final String LOCAL_CHECKPOINT_PATH_KEY = "checkpoint.local.path"; - public static final String DEFAULT_CHECKPOINT_DIRECTORY = "/tmp"; + public static final String DISTRIBUTED_CHECKPOINT_PATH_KEY = "checkpoint.distributed.path"; - private static String CHECKPOINT_DIRECTORY = null; + public static final String DEFAULT_LOCAL_CHECKPOINT_PATH = "file:///tmp"; + + public static final String COMPLETED_CHECKPOINT_SUFFIX = "_final"; + + private static Path LOCAL_CHECKPOINT_PATH = null; + + private static Path DISTRIBUTED_CHECKPOINT_PATH = null; private static double CP_UPPER = -1.0; @@ -42,40 +50,64 @@ public final class CheckpointUtils { private CheckpointUtils() { } - static String getCheckpointDirectory() { + public static Path getLocalCheckpointPath() { - if (CHECKPOINT_DIRECTORY == null) { - CHECKPOINT_DIRECTORY = GlobalConfiguration - .getString(CHECKPOINT_DIRECTORY_KEY, DEFAULT_CHECKPOINT_DIRECTORY); + if (LOCAL_CHECKPOINT_PATH == null) { + LOCAL_CHECKPOINT_PATH = new Path(GlobalConfiguration.getString(LOCAL_CHECKPOINT_PATH_KEY, + DEFAULT_LOCAL_CHECKPOINT_PATH)); } - return CHECKPOINT_DIRECTORY; + return LOCAL_CHECKPOINT_PATH; } - public static boolean hasCompleteCheckpointAvailable(final ExecutionVertexID vertexID) { + public static Path getDistributedCheckpointPath() { - final File file = new File(getCheckpointDirectory() + File.separator + METADATA_PREFIX + "_" + vertexID - + "_final"); - if (file.exists()) { - return true; + if (DISTRIBUTED_CHECKPOINT_PATH == null) { + + final String path = GlobalConfiguration.getString(DISTRIBUTED_CHECKPOINT_PATH_KEY, null); + if (path == null) { + return null; + } + + DISTRIBUTED_CHECKPOINT_PATH = new Path(path); } - return false; + return DISTRIBUTED_CHECKPOINT_PATH; + } + + public static boolean hasCompleteCheckpointAvailable(final ExecutionVertexID vertexID) throws IOException { + + return checkForCheckpoint(vertexID, COMPLETED_CHECKPOINT_SUFFIX); + } + + public static boolean hasPartialCheckpointAvailable(final ExecutionVertexID vertexID) throws IOException { + + return checkForCheckpoint(vertexID, "_0"); } - public static boolean hasPartialCheckpointAvailable(final ExecutionVertexID vertexID) { + private static boolean checkForCheckpoint(final ExecutionVertexID vertexID, final String suffix) throws IOException { - File file = new File(getCheckpointDirectory() + File.separator + METADATA_PREFIX + "_" + vertexID + "_0"); - if (file.exists()) { + final Path local = new Path(getLocalCheckpointPath() + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID + + COMPLETED_CHECKPOINT_SUFFIX); + + final FileSystem localFs = local.getFileSystem(); + + if (localFs.exists(local)) { return true; } - file = new File(getCheckpointDirectory() + File.separator + METADATA_PREFIX + "_" + vertexID + "_part"); - if (file.exists()) { - return true; + final Path distributedCheckpointPath = getDistributedCheckpointPath(); + if (distributedCheckpointPath == null) { + return false; } - return false; + final Path distributed = new Path(distributedCheckpointPath + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID + + COMPLETED_CHECKPOINT_SUFFIX); + + final FileSystem distFs = distributed.getFileSystem(); + + return distFs.exists(distributed); + } /** @@ -86,22 +118,37 @@ public static boolean hasPartialCheckpointAvailable(final ExecutionVertexID vert */ public static void removeCheckpoint(final ExecutionVertexID vertexID) { - final String checkpointDirectory = getCheckpointDirectory(); + final Path localChPath = getLocalCheckpointPath(); + + try { + removeCheckpoint(new Path(localChPath + Path.SEPARATOR + METADATA_PREFIX)); - File file = new File(checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID - + "_final"); - if (file.exists()) { - file.delete(); + final Path distributedChPath = getDistributedCheckpointPath(); + if (distributedChPath != null) { + removeCheckpoint(new Path(distributedChPath + Path.SEPARATOR + METADATA_PREFIX)); + } + } catch (IOException e) { + } + } + + private static void removeCheckpoint(final Path pathPrefix) throws IOException { + + + Path p = pathPrefix.suffix(COMPLETED_CHECKPOINT_SUFFIX); + FileSystem fs = p.getFileSystem(); + if(fs.exists(p)) { + fs.delete(p, false); return; } - file = new File(checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_0"); - if (file.exists()) { - file.delete(); + + p = pathPrefix.suffix("_0"); + if(fs.exists(p)) { + fs.delete(p, false); } - - file = new File(checkpointDirectory + File.separator + METADATA_PREFIX + "_" + vertexID + "_part"); - if (file.exists()) { - file.delete(); + + p = pathPrefix.suffix("_part"); + if(fs.exists(p)) { + fs.delete(p, false); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index e397870be3fdb..2e5014bc99068 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -15,7 +15,6 @@ package eu.stratosphere.nephele.checkpointing; -import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.nio.channels.FileChannel; @@ -30,12 +29,14 @@ import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.taskmanager.transferenvelope.CheckpointSerializer; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.event.task.EventList; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; +import eu.stratosphere.nephele.fs.FileChannelWrapper; +import eu.stratosphere.nephele.fs.FileSystem; +import eu.stratosphere.nephele.fs.Path; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.BufferFactory; import eu.stratosphere.nephele.io.channels.FileBufferManager; @@ -65,6 +66,11 @@ public class EphemeralCheckpoint implements OutputChannelForwarder { */ private static final int ENVELOPES_PER_META_DATA_FILE = 10000; + /** + * The buffer size in bytes to use for the meta data file channel. + */ + private static final int BUFFER_SIZE = 4096; + /** * The enveloped which are currently queued until the state of the checkpoint is decided. */ @@ -102,6 +108,16 @@ public class EphemeralCheckpoint implements OutputChannelForwarder { */ private final FileBufferManager fileBufferManager; + /** + * The path to which the checkpoint meta data shall be written to. + */ + private final Path checkpointPath; + + /** + * The file system to write the checkpoint's to. + */ + private FileSystem fileSystem; + /** * The file channel to write the checkpoint's meta data. */ @@ -139,7 +155,7 @@ public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { } this.numberOfConnectedChannels = nooc; - this.distributed = CheckpointUtils.createDistributedCheckpoint(); + final boolean dist = CheckpointUtils.createDistributedCheckpoint(); this.checkpointingDecision = (ephemeral ? CheckpointingDecisionState.UNDECIDED : CheckpointingDecisionState.CHECKPOINTING); @@ -152,6 +168,22 @@ public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { if (this.checkpointingDecision == CheckpointingDecisionState.CHECKPOINTING) { this.task.checkpointStateChanged(CheckpointState.PARTIAL); } + + if (dist) { + final Path p = CheckpointUtils.getDistributedCheckpointPath(); + System.out.println("Distributed checkpoint path is " + p); + if (p == null) { + LOG.error("No distributed checkpoint path configured, writing local checkpoints instead"); + this.checkpointPath = CheckpointUtils.getLocalCheckpointPath(); + this.distributed = false; + } else { + this.checkpointPath = p; + this.distributed = true; + } + } else { + this.checkpointPath = CheckpointUtils.getLocalCheckpointPath(); + this.distributed = false; + } } private void destroy() { @@ -173,18 +205,16 @@ private void write() throws IOException, InterruptedException { } } - private boolean renameCheckpointPart(final String checkpointDir) { + private boolean renameCheckpointPart() throws IOException { - final File oldFile = new File(checkpointDir + File.separator - + CheckpointUtils.METADATA_PREFIX + "_" + final Path oldFile = this.checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + "_" + this.task.getVertexID() + "_part"); - final File newFile = new File(checkpointDir + File.separator - + CheckpointUtils.METADATA_PREFIX + "_" + final Path newFile = this.checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + "_" + this.task.getVertexID() + "_" + this.metaDataSuffix); - if (!oldFile.renameTo(newFile)) { - LOG.error("Unable to rename " + oldFile.getAbsoluteFile() + " to " + newFile.getAbsoluteFile()); + if (!this.fileSystem.rename(oldFile, newFile)) { + LOG.error("Unable to rename " + oldFile + " to " + newFile); return false; } @@ -194,10 +224,6 @@ private boolean renameCheckpointPart(final String checkpointDir) { private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - final String checkpointDir = GlobalConfiguration.getString( - CheckpointUtils.CHECKPOINT_DIRECTORY_KEY, - CheckpointUtils.DEFAULT_CHECKPOINT_DIRECTORY); - final Buffer buffer = transferEnvelope.getBuffer(); if (buffer != null) { if (buffer.isBackedByMemory()) { @@ -214,12 +240,16 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro // Write the meta data of the transfer envelope to disk if (this.numberOfSerializedTransferEnvelopes % ENVELOPES_PER_META_DATA_FILE == 0) { + if (this.fileSystem == null) { + this.fileSystem = this.checkpointPath.getFileSystem(); + } + if (this.metaDataFileChannel != null) { this.metaDataFileChannel.close(); this.metaDataFileChannel = null; // Rename file - renameCheckpointPart(checkpointDir); + renameCheckpointPart(); // Increase the meta data suffix ++this.metaDataSuffix; @@ -227,14 +257,7 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro } if (this.metaDataFileChannel == null) { - - if (LOG.isDebugEnabled()) { - LOG.debug("Writing checkpointing meta data to directory " + checkpointDir); - } - final FileOutputStream fos = new FileOutputStream(checkpointDir + File.separator - + CheckpointUtils.METADATA_PREFIX - + "_" + this.task.getVertexID() + "_part"); - this.metaDataFileChannel = fos.getChannel(); + this.metaDataFileChannel = getMetaDataFileChannel("_part"); } this.transferEnvelopeSerializer.setTransferEnvelope(transferEnvelope); @@ -261,11 +284,10 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro this.metaDataFileChannel.close(); // Rename file - renameCheckpointPart(checkpointDir); + renameCheckpointPart(); } - new FileOutputStream(checkpointDir + File.separator + CheckpointUtils.METADATA_PREFIX + "_" - + this.task.getVertexID() + "_final").close(); + getMetaDataFileChannel(CheckpointUtils.COMPLETED_CHECKPOINT_SUFFIX).close(); // Since it is unclear whether the underlying physical file will ever be read, we force to close it. // TODO: Fix me @@ -278,6 +300,28 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro } } + private FileChannel getMetaDataFileChannel(final String suffix) throws IOException { + + if (LOG.isDebugEnabled()) { + LOG.debug("Writing checkpointing meta data to directory " + this.checkpointPath); + } + + // Bypass FileSystem API for local checkpoints + if (!this.distributed) { + + final FileOutputStream fos = new FileOutputStream(this.checkpointPath.toUri().getPath() + + Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + "_" + this.task.getVertexID() + suffix); + + return fos.getChannel(); + } + + return new FileChannelWrapper(this.fileSystem, this.checkpointPath.suffix(Path.SEPARATOR + + CheckpointUtils.METADATA_PREFIX + "_" + this.task.getVertexID() + suffix), BUFFER_SIZE, (short) 2); // TODO: + // Make + // replication + // configurable + } + public void setCheckpointDecisionSynchronously(final boolean checkpointDecision) throws IOException, InterruptedException { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index 30cee1ad8895e..916a07579aa80 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -15,6 +15,7 @@ package eu.stratosphere.nephele.checkpointing; +import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; @@ -167,7 +168,7 @@ public ReplayTask(final ExecutionVertexID vertexID, final Environment environmen this.vertexID = vertexID; this.environment = new CheckpointEnvironment(this.vertexID, environment, - CheckpointUtils.hasCompleteCheckpointAvailable(vertexID), this.outputBrokerMap); + hasCompleteCheckpointAvailable(this.vertexID), this.outputBrokerMap); this.environment.setExecutionObserver(new ReplayTaskExecutionObserver(null)); this.encapsulatedTask = null; @@ -178,7 +179,7 @@ public ReplayTask(final RuntimeTask encapsulatedTask, final TaskManager taskMana this.vertexID = encapsulatedTask.getVertexID(); this.environment = new CheckpointEnvironment(this.vertexID, encapsulatedTask.getEnvironment(), - CheckpointUtils.hasCompleteCheckpointAvailable(vertexID), this.outputBrokerMap); + hasCompleteCheckpointAvailable(this.vertexID), this.outputBrokerMap); this.environment.setExecutionObserver(new ReplayTaskExecutionObserver(null)); this.encapsulatedTask = encapsulatedTask; @@ -189,6 +190,16 @@ public ReplayTask(final RuntimeTask encapsulatedTask, final TaskManager taskMana this.taskManager = taskManager; } + private static boolean hasCompleteCheckpointAvailable(final ExecutionVertexID vertexID) { + + try { + return CheckpointUtils.hasCompleteCheckpointAvailable(vertexID); + } catch (IOException ioe) { + } + + return false; + } + /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java index 98f21199f1686..5141dd2548244 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java @@ -8,6 +8,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.stratosphere.nephele.fs.FileChannelWrapper; import eu.stratosphere.nephele.fs.FileSystem; import eu.stratosphere.nephele.fs.Path; @@ -22,7 +23,7 @@ final class DistributedChannelWithAccessInfo implements ChannelWithAccessInfo { private final Path checkpointFile; - private final DistributedFileChannel channel; + private final FileChannelWrapper channel; private final AtomicLong reservedWritePosition; @@ -33,7 +34,7 @@ final class DistributedChannelWithAccessInfo implements ChannelWithAccessInfo { this.fs = fs; this.checkpointFile = checkpointFile; - this.channel = new DistributedFileChannel(fs, checkpointFile, bufferSize); + this.channel = new FileChannelWrapper(fs, checkpointFile, bufferSize, (short) 2); this.reservedWritePosition = new AtomicLong(0); this.referenceCounter = new AtomicInteger(0); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java index bdc3b45b7b499..bfccf20ea4287 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java @@ -23,6 +23,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.stratosphere.nephele.checkpointing.CheckpointUtils; import eu.stratosphere.nephele.configuration.ConfigConstants; import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.fs.FileSystem; @@ -81,7 +82,7 @@ public static FileBufferManager getInstance() { private final int bufferSize; - private final String distributedTempPath; + private final Path distributedTempPath; private final FileSystem distributedFileSystem; @@ -111,17 +112,15 @@ private FileBufferManager() { this.fileMap = new ConcurrentHashMap(2048, 0.8f, 64); - this.distributedTempPath = GlobalConfiguration.getString("taskmanager.tmp.distdir", - "hdfs://master:9000/checkpoints"); + this.distributedTempPath = CheckpointUtils.getDistributedCheckpointPath(); FileSystem distFS = null; if (this.distributedTempPath != null) { try { - final Path p = new Path(this.distributedTempPath); - distFS = FileSystem.get(p.toUri()); - if (!distFS.exists(p)) { - distFS.mkdirs(p); + distFS = this.distributedTempPath.getFileSystem(); + if (!distFS.exists(this.distributedTempPath)) { + distFS.mkdirs(this.distributedTempPath); } } catch (IOException e) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index e73a5d878e57c..632c8d4e7671b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -516,7 +516,7 @@ public List submitTasks(final List * @return the task to be started or null if a task with the same ID was already running */ private Task createAndRegisterTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final RuntimeEnvironment environment, final Set activeOutputChannels) { + final RuntimeEnvironment environment, final Set activeOutputChannels) throws IOException { if (id == null) { throw new IllegalArgumentException("Argument id is null"); From e083ce8fa478fcb100154674aca7b32b8161e146 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 24 Feb 2012 21:13:04 +0100 Subject: [PATCH 277/310] Modified ReplayThread so it can handle distributed checkpoints --- .../checkpointing/CheckpointEnvironment.java | 10 ++- .../checkpointing/CheckpointUtils.java | 38 +++++++--- .../nephele/checkpointing/ReplayTask.java | 20 ++++- .../nephele/checkpointing/ReplayThread.java | 75 ++++++++++++++----- 4 files changed, 109 insertions(+), 34 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java index 863260ff43b15..39ec8dbf69bc0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java @@ -26,6 +26,8 @@ final class CheckpointEnvironment implements Environment { private final Environment environment; + private final boolean hasLocalCheckpoint; + private final boolean hasCompleteCheckpoint; private final Map outputBrokerMap; @@ -38,11 +40,12 @@ final class CheckpointEnvironment implements Environment { private volatile ReplayThread executingThread = null; CheckpointEnvironment(final ExecutionVertexID vertexID, final Environment environment, - final boolean hasCompleteCheckpoint, + final boolean hasLocalCheckpoint, final boolean hasCompleteCheckpoint, final Map outputBrokerMap) { this.vertexID = vertexID; this.environment = environment; + this.hasLocalCheckpoint = hasLocalCheckpoint; this.hasCompleteCheckpoint = hasCompleteCheckpoint; this.outputBrokerMap = outputBrokerMap; } @@ -296,7 +299,7 @@ public ReplayThread getExecutingThread() { if (this.executingThread == null) { this.executingThread = new ReplayThread(this.vertexID, this.executionObserver, getTaskName(), - this.hasCompleteCheckpoint, this.outputBrokerMap); + this.hasLocalCheckpoint, this.hasCompleteCheckpoint, this.outputBrokerMap); } return this.executingThread; @@ -306,9 +309,10 @@ public ReplayThread getExecutingThread() { // DW: Start of temporary code @Override public void reportPACTDataStatistics(final long numberOfConsumedBytes, final long numberOfProducedBytes) { - + throw new IllegalStateException("reportPACTDataStatistics called on CheckpointEnvironment"); } + // DW: End of temporary code @Override diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java index 9f8ad9760d161..775a030b4b838 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java @@ -82,13 +82,34 @@ public static boolean hasCompleteCheckpointAvailable(final ExecutionVertexID ver public static boolean hasPartialCheckpointAvailable(final ExecutionVertexID vertexID) throws IOException { - return checkForCheckpoint(vertexID, "_0"); + if (checkForCheckpoint(vertexID, "_0")) { + return true; + } + + return checkForCheckpoint(vertexID, "_part"); + } + + public static boolean hasLocalCheckpointAvailable(final ExecutionVertexID vertexID) throws IOException { + + Path local = new Path(getLocalCheckpointPath() + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID + + "_0"); + + final FileSystem localFs = local.getFileSystem(); + + if (localFs.exists(local)) { + return true; + } + + local = new Path(getLocalCheckpointPath() + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID + + "_part"); + + return localFs.exists(local); } private static boolean checkForCheckpoint(final ExecutionVertexID vertexID, final String suffix) throws IOException { final Path local = new Path(getLocalCheckpointPath() + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID - + COMPLETED_CHECKPOINT_SUFFIX); + + suffix); final FileSystem localFs = local.getFileSystem(); @@ -102,7 +123,7 @@ private static boolean checkForCheckpoint(final ExecutionVertexID vertexID, fina } final Path distributed = new Path(distributedCheckpointPath + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID - + COMPLETED_CHECKPOINT_SUFFIX); + + suffix); final FileSystem distFs = distributed.getFileSystem(); @@ -133,21 +154,20 @@ public static void removeCheckpoint(final ExecutionVertexID vertexID) { private static void removeCheckpoint(final Path pathPrefix) throws IOException { - Path p = pathPrefix.suffix(COMPLETED_CHECKPOINT_SUFFIX); FileSystem fs = p.getFileSystem(); - if(fs.exists(p)) { + if (fs.exists(p)) { fs.delete(p, false); return; } - + p = pathPrefix.suffix("_0"); - if(fs.exists(p)) { + if (fs.exists(p)) { fs.delete(p, false); } - + p = pathPrefix.suffix("_part"); - if(fs.exists(p)) { + if (fs.exists(p)) { fs.delete(p, false); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index 916a07579aa80..df8f994ce7cfa 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -167,8 +167,11 @@ public ReplayTask(final ExecutionVertexID vertexID, final Environment environmen final TaskManager taskManager) { this.vertexID = vertexID; + this.environment = new CheckpointEnvironment(this.vertexID, environment, - hasCompleteCheckpointAvailable(this.vertexID), this.outputBrokerMap); + hasLocalCheckpointAvailable(this.vertexID), hasCompleteCheckpointAvailable(this.vertexID), + this.outputBrokerMap); + this.environment.setExecutionObserver(new ReplayTaskExecutionObserver(null)); this.encapsulatedTask = null; @@ -178,8 +181,11 @@ public ReplayTask(final ExecutionVertexID vertexID, final Environment environmen public ReplayTask(final RuntimeTask encapsulatedTask, final TaskManager taskManager) { this.vertexID = encapsulatedTask.getVertexID(); + this.environment = new CheckpointEnvironment(this.vertexID, encapsulatedTask.getEnvironment(), - hasCompleteCheckpointAvailable(this.vertexID), this.outputBrokerMap); + hasLocalCheckpointAvailable(this.vertexID), hasCompleteCheckpointAvailable(this.vertexID), + this.outputBrokerMap); + this.environment.setExecutionObserver(new ReplayTaskExecutionObserver(null)); this.encapsulatedTask = encapsulatedTask; @@ -200,6 +206,16 @@ private static boolean hasCompleteCheckpointAvailable(final ExecutionVertexID ve return false; } + private static boolean hasLocalCheckpointAvailable(final ExecutionVertexID vertexID) { + + try { + return CheckpointUtils.hasLocalCheckpointAvailable(vertexID); + } catch (IOException ioe) { + } + + return false; + } + /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index d93f1deeaffb3..e5e7fcba68495 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -1,10 +1,10 @@ package eu.stratosphere.nephele.checkpointing; import java.io.EOFException; -import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; +import java.net.URI; import java.nio.channels.FileChannel; import java.util.Iterator; import java.util.Map; @@ -13,6 +13,9 @@ import eu.stratosphere.nephele.execution.ExecutionObserver; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.fs.FileChannelWrapper; +import eu.stratosphere.nephele.fs.FileSystem; +import eu.stratosphere.nephele.fs.Path; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.taskmanager.transferenvelope.CheckpointDeserializer; @@ -23,27 +26,36 @@ final class ReplayThread extends Thread { private static final String REPLAY_SUFFIX = " (Replay)"; + /** + * The interval to sleep in case a communication channel is not yet entirely set up (in milliseconds). + */ + private static final int SLEEPINTERVAL = 100; + + /** + * The buffer size in bytes to use for the meta data file channel. + */ + private static final int BUFFER_SIZE = 4096; + private final ExecutionVertexID vertexID; private final ExecutionObserver executionObserver; + private final boolean isCheckpointLocal; + private final boolean isCheckpointComplete; private final Map outputBrokerMap; - /** - * The interval to sleep in case a communication channel is not yet entirely set up (in milliseconds). - */ - private static final int SLEEPINTERVAL = 100; - private final AtomicBoolean restartRequested = new AtomicBoolean(false); ReplayThread(final ExecutionVertexID vertexID, final ExecutionObserver executionObserver, final String taskName, - final boolean isCheckpointComplete, final Map outputBrokerMap) { + final boolean isCheckpointLocal, final boolean isCheckpointComplete, + final Map outputBrokerMap) { super((taskName == null ? "Unkown" : taskName) + REPLAY_SUFFIX); this.vertexID = vertexID; this.executionObserver = executionObserver; + this.isCheckpointLocal = isCheckpointLocal; this.isCheckpointComplete = isCheckpointComplete; this.outputBrokerMap = outputBrokerMap; } @@ -138,8 +150,20 @@ void restart() { private void replayCheckpoint() throws Exception { + System.out.println("Replaying checkpoint for vertex " + this.vertexID); + final CheckpointDeserializer deserializer = new CheckpointDeserializer(this.vertexID); + final Path checkpointPath = this.isCheckpointLocal ? CheckpointUtils.getLocalCheckpointPath() : CheckpointUtils + .getDistributedCheckpointPath(); + + if (checkpointPath == null) { + throw new IOException("Cannot determine checkpoint path for vertex " + this.vertexID); + } + + // The file system the checkpoint's meta data is stored on + final FileSystem fileSystem = checkpointPath.getFileSystem(); + int metaDataIndex = 0; while (true) { @@ -149,16 +173,16 @@ private void replayCheckpoint() throws Exception { } // Try to locate the meta data file - final File metaDataFile = new File(CheckpointUtils.getCheckpointDirectory() + File.separator - + CheckpointUtils.METADATA_PREFIX + "_" + this.vertexID + "_" + metaDataIndex); + final Path metaDataFile = checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + + "_" + this.vertexID + "_" + metaDataIndex); - while (!metaDataFile.exists()) { + while (!fileSystem.exists(metaDataFile)) { // Try to locate the final meta data file - final File finalMetaDataFile = new File(CheckpointUtils.getCheckpointDirectory() + File.separator - + CheckpointUtils.METADATA_PREFIX + "_" + this.vertexID + "_final"); + final Path finalMetaDataFile = checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + + "_" + this.vertexID + "_final"); - if (finalMetaDataFile.exists()) { + if (fileSystem.exists(finalMetaDataFile)) { return; } @@ -168,16 +192,15 @@ private void replayCheckpoint() throws Exception { } // Wait for the file to be created - Thread.sleep(10); + Thread.sleep(100); } - FileInputStream fis = null; + FileChannel fileChannel = null; try { - fis = new FileInputStream(metaDataFile); - final FileChannel fileChannel = fis.getChannel(); + fileChannel = getFileChannel(fileSystem, metaDataFile); while (true) { try { @@ -197,7 +220,7 @@ private void replayCheckpoint() throws Exception { final Buffer destBuffer = broker.requestEmptyBufferBlocking(srcBuffer.size()); srcBuffer.copyToBuffer(destBuffer); transferEnvelope.setBuffer(destBuffer); - srcBuffer.recycleBuffer(); + srcBuffer.recycleBuffer(); } broker.outputEnvelope(transferEnvelope); @@ -211,10 +234,22 @@ private void replayCheckpoint() throws Exception { } } } finally { - if (fis != null) { - fis.close(); + if (fileChannel != null) { + fileChannel.close(); } } } } + + private FileChannel getFileChannel(final FileSystem fs, final Path p) throws IOException { + + // Bypass FileSystem API for local checkpoints + if (this.isCheckpointLocal) { + + final URI uri = p.toUri(); + return new FileInputStream(uri.getPath()).getChannel(); + } + + return new FileChannelWrapper(fs, p, BUFFER_SIZE, (short) -1); + } } From 52a19f84ab83c2523210101ef97e724879d17ffa Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Fri, 24 Feb 2012 23:06:46 +0100 Subject: [PATCH 278/310] Fixed some bugs related to writing checkpoints to a distributed file system --- .../nephele/fs/FileChannelWrapper.java | 55 ++++++++++++++++--- .../checkpointing/EphemeralCheckpoint.java | 25 +++++++-- .../nephele/checkpointing/ReplayThread.java | 2 - 3 files changed, 66 insertions(+), 16 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java index a5d5c6aaa28ff..553f80182e382 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java @@ -20,8 +20,12 @@ public final class FileChannelWrapper extends FileChannel { private FSDataOutputStream outputStream = null; + private FSDataInputStream inputStream = null; + private long nextExpectedWritePosition = 0L; + private long nextExpectedReadPosition = 0L; + public FileChannelWrapper(final FileSystem fs, final Path checkpointFile, final int bufferSize, final short replication) { @@ -73,22 +77,41 @@ public FileChannel position(long newPosition) throws IOException { return null; } + /** + * {@inheritDoc} + */ @Override - public int read(ByteBuffer dst) throws IOException { - // TODO Auto-generated method stub - - System.out.println("read called"); + public int read(final ByteBuffer dst) throws IOException { - return 0; + return read(dst, this.nextExpectedReadPosition); } + /** + * {@inheritDoc} + */ @Override - public int read(ByteBuffer dst, long position) throws IOException { - // TODO Auto-generated method stub + public int read(final ByteBuffer dst, final long position) throws IOException { - System.out.println("read2 called"); + final int length = Math.min(this.buf.length, dst.remaining()); - return 0; + final FSDataInputStream inputStream = getInputStream(); + if (position != this.nextExpectedReadPosition) { + System.out.println("Next expected position is " + this.nextExpectedReadPosition + ", seeking to " + + position); + inputStream.seek(position); + this.nextExpectedReadPosition = position; + } + + final int bytesRead = inputStream.read(this.buf, 0, length); + if (bytesRead == -1) { + return -1; + } + + dst.put(this.buf, 0, length); + + this.nextExpectedReadPosition += bytesRead; + + return bytesRead; } @Override @@ -200,6 +223,15 @@ private FSDataOutputStream getOutputStream() throws IOException { return this.outputStream; } + private FSDataInputStream getInputStream() throws IOException { + + if (this.inputStream == null) { + this.inputStream = this.fs.open(this.checkpointFile, this.buf.length); + } + + return this.inputStream; + } + /** * {@inheritDoc} */ @@ -207,5 +239,10 @@ private FSDataOutputStream getOutputStream() throws IOException { protected void implCloseChannel() throws IOException { getOutputStream().close(); + + if (this.inputStream != null) { + this.inputStream.close(); + this.inputStream = null; + } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 2e5014bc99068..270d66cac7a84 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -128,6 +128,8 @@ public class EphemeralCheckpoint implements OutputChannelForwarder { */ private int numberOfSerializedTransferEnvelopes = 0; + private Buffer firstSerializedFileBuffer = null; + /** * This enumeration reflects the possible states an ephemeral * checkpoint can be in. @@ -224,7 +226,7 @@ private boolean renameCheckpointPart() throws IOException { private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - final Buffer buffer = transferEnvelope.getBuffer(); + Buffer buffer = transferEnvelope.getBuffer(); if (buffer != null) { if (buffer.isBackedByMemory()) { @@ -264,6 +266,16 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro while (this.transferEnvelopeSerializer.write(this.metaDataFileChannel)) { } + // The following code will prevent the underlying file from being closed + buffer = transferEnvelope.getBuffer(); + if (buffer != null) { + if (this.firstSerializedFileBuffer == null) { + this.firstSerializedFileBuffer = buffer; + } else { + buffer.recycleBuffer(); + } + } + // Look for close event final EventList eventList = transferEnvelope.getEventList(); if (eventList != null) { @@ -280,6 +292,12 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro if (this.numberOfClosedChannels == this.numberOfConnectedChannels) { + // Finally, close the underlying file + if (this.firstSerializedFileBuffer != null) { + this.firstSerializedFileBuffer.recycleBuffer(); + } + + // Finish meta data file if (this.metaDataFileChannel != null) { this.metaDataFileChannel.close(); @@ -287,12 +305,9 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro renameCheckpointPart(); } + // Write the meta data file to indicate the checkpoint is complete getMetaDataFileChannel(CheckpointUtils.COMPLETED_CHECKPOINT_SUFFIX).close(); - // Since it is unclear whether the underlying physical file will ever be read, we force to close it. - // TODO: Fix me - // this.fileBufferManager.forceCloseOfWritableSpillingFile(this.task.getVertexID()); - LOG.info("Finished persistent checkpoint for vertex " + this.task.getVertexID()); // Send notification that checkpoint is completed diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index e5e7fcba68495..2792fd8f76996 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -150,8 +150,6 @@ void restart() { private void replayCheckpoint() throws Exception { - System.out.println("Replaying checkpoint for vertex " + this.vertexID); - final CheckpointDeserializer deserializer = new CheckpointDeserializer(this.vertexID); final Path checkpointPath = this.isCheckpointLocal ? CheckpointUtils.getLocalCheckpointPath() : CheckpointUtils From 1d6f64dbe4f834bf04b8b680604f1e902f901604 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 25 Feb 2012 16:50:43 +0100 Subject: [PATCH 279/310] Finished implementation of recovery from distributed checkpoints --- .../nephele/fs/FileChannelWrapper.java | 5 +- .../checkpointing/EphemeralCheckpoint.java | 7 +- .../nephele/checkpointing/ReplayThread.java | 77 +++++++++++-------- .../nephele/io/channels/BufferFactory.java | 7 +- .../io/channels/ChannelWithAccessInfo.java | 17 +++- .../DistributedChannelWithAccessInfo.java | 28 +++++-- .../nephele/io/channels/FileBuffer.java | 10 ++- .../io/channels/FileBufferManager.java | 68 ++++++++++++---- .../channels/LocalChannelWithAccessInfo.java | 26 ++++++- .../runtime/RuntimeInputGateContext.java | 2 +- .../SpillingQueueElement.java | 4 +- 11 files changed, 178 insertions(+), 73 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java index 553f80182e382..78177bf86c332 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java @@ -238,7 +238,10 @@ private FSDataInputStream getInputStream() throws IOException { @Override protected void implCloseChannel() throws IOException { - getOutputStream().close(); + if(this.outputStream != null) { + this.outputStream.close(); + this.outputStream = null; + } if (this.inputStream != null) { this.inputStream.close(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 270d66cac7a84..f7cdc65448c9b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -17,6 +17,7 @@ import java.io.FileOutputStream; import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.util.ArrayDeque; import java.util.Iterator; @@ -232,7 +233,7 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro // Make sure we transfer the encapsulated buffer to a file and release the memory buffer again final Buffer fileBuffer = BufferFactory.createFromFile(buffer.size(), this.task.getVertexID(), - this.fileBufferManager, this.distributed); + this.fileBufferManager, this.distributed, false); buffer.copyToBuffer(fileBuffer); transferEnvelope.setBuffer(fileBuffer); buffer.recycleBuffer(); @@ -306,7 +307,9 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro } // Write the meta data file to indicate the checkpoint is complete - getMetaDataFileChannel(CheckpointUtils.COMPLETED_CHECKPOINT_SUFFIX).close(); + this.metaDataFileChannel = getMetaDataFileChannel(CheckpointUtils.COMPLETED_CHECKPOINT_SUFFIX); + this.metaDataFileChannel.write(ByteBuffer.allocate(0)); + this.metaDataFileChannel.close(); LOG.info("Finished persistent checkpoint for vertex " + this.task.getVertexID()); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index 2792fd8f76996..f39f4b09c9948 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -164,39 +164,41 @@ private void replayCheckpoint() throws Exception { int metaDataIndex = 0; - while (true) { + Buffer firstDeserializedFileBuffer = null; + FileChannel fileChannel = null; - if (this.restartRequested.compareAndSet(true, false)) { - metaDataIndex = 0; - } + try { - // Try to locate the meta data file - final Path metaDataFile = checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX - + "_" + this.vertexID + "_" + metaDataIndex); + while (true) { - while (!fileSystem.exists(metaDataFile)) { + if (this.restartRequested.compareAndSet(true, false)) { + metaDataIndex = 0; + } - // Try to locate the final meta data file - final Path finalMetaDataFile = checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX - + "_" + this.vertexID + "_final"); + // Try to locate the meta data file + final Path metaDataFile = checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + + "_" + this.vertexID + "_" + metaDataIndex); - if (fileSystem.exists(finalMetaDataFile)) { - return; - } + while (!fileSystem.exists(metaDataFile)) { - if (this.isCheckpointComplete) { - throw new FileNotFoundException("Cannot find meta data file " + metaDataIndex - + " for checkpoint of vertex " + this.vertexID); - } + // Try to locate the final meta data file + final Path finalMetaDataFile = checkpointPath.suffix(Path.SEPARATOR + + CheckpointUtils.METADATA_PREFIX + + "_" + this.vertexID + "_final"); - // Wait for the file to be created - Thread.sleep(100); + if (fileSystem.exists(finalMetaDataFile)) { + return; + } - } + if (this.isCheckpointComplete) { + throw new FileNotFoundException("Cannot find meta data file " + metaDataIndex + + " for checkpoint of vertex " + this.vertexID); + } - FileChannel fileChannel = null; + // Wait for the file to be created + Thread.sleep(100); - try { + } fileChannel = getFileChannel(fileSystem, metaDataFile); @@ -204,17 +206,25 @@ private void replayCheckpoint() throws Exception { try { deserializer.read(fileChannel); - final TransferEnvelope transferEnvelope = deserializer.getFullyDeserializedTransferEnvelope(); + final TransferEnvelope transferEnvelope = deserializer + .getFullyDeserializedTransferEnvelope(); if (transferEnvelope != null) { - final ReplayOutputBroker broker = this.outputBrokerMap.get(transferEnvelope.getSource()); + final ReplayOutputBroker broker = this.outputBrokerMap + .get(transferEnvelope.getSource()); if (broker == null) { throw new IOException("Cannot find output broker for channel " - + transferEnvelope.getSource()); + + transferEnvelope.getSource()); } final Buffer srcBuffer = transferEnvelope.getBuffer(); if (srcBuffer != null) { + + // Prevent underlying file from being closed + if (firstDeserializedFileBuffer == null) { + firstDeserializedFileBuffer = srcBuffer.duplicate(); + } + final Buffer destBuffer = broker.requestEmptyBufferBlocking(srcBuffer.size()); srcBuffer.copyToBuffer(destBuffer); transferEnvelope.setBuffer(destBuffer); @@ -226,15 +236,22 @@ private void replayCheckpoint() throws Exception { } catch (EOFException eof) { // Close the file channel fileChannel.close(); + fileChannel = null; // Increase the index of the meta data file ++metaDataIndex; break; } } - } finally { - if (fileChannel != null) { - fileChannel.close(); - } + } + + } finally { + if (firstDeserializedFileBuffer != null) { + firstDeserializedFileBuffer.recycleBuffer(); + firstDeserializedFileBuffer = null; + } + if (fileChannel != null) { + fileChannel.close(); + fileChannel = null; } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java index ce3b5e1243caf..a1d6cdb1aa520 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java @@ -23,16 +23,17 @@ public final class BufferFactory { public static FileBuffer createFromFile(final int bufferSize, final AbstractID ownerID, - final FileBufferManager fileBufferManager, final boolean distributed) throws IOException { + final FileBufferManager fileBufferManager, final boolean distributed, final boolean deleteOnClose) + throws IOException { - return new FileBuffer(bufferSize, ownerID, fileBufferManager, distributed); + return new FileBuffer(bufferSize, ownerID, fileBufferManager, distributed, deleteOnClose); } public static FileBuffer createFromCheckpoint(final int bufferSize, final long offset, final AbstractID ownerID, final FileBufferManager fileBufferManager, final boolean distributed) throws IOException { - return new FileBuffer(bufferSize, offset, ownerID, fileBufferManager, distributed); + return new FileBuffer(bufferSize, offset, ownerID, fileBufferManager, distributed, false); } public static Buffer createFromMemory(final int bufferSize, final ByteBuffer byteBuffer, diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java index 935eb4ebd696f..a308893429671 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java @@ -7,7 +7,7 @@ interface ChannelWithAccessInfo { FileChannel getChannel(); FileChannel getAndIncrementReferences(); - + /** * Increments the references to this channel. Returns true, if successful, and false, * if the channel has been disposed in the meantime. @@ -15,9 +15,9 @@ interface ChannelWithAccessInfo { * @return True, if successful, false, if the channel has been disposed. */ boolean incrementReferences(); - + ChannelWithPosition reserveWriteSpaceAndIncrementReferences(int spaceToReserve); - + /** * Decrements the number of references to this channel. If the number of references is zero after the * decrement, the channel is deleted. @@ -27,10 +27,19 @@ interface ChannelWithAccessInfo { * Thrown, if the number of references is already zero or below. */ int decrementReferences(); - + /** * Disposes the channel without further notice. Tries to close it (swallowing all exceptions) and tries * to delete the file. */ void disposeSilently(); + + /** + * Updates the flag which indicates whether the underlying physical file shall be deleted when it is closed. Once + * the flag was updated to false it cannot be set to true again. + * + * @param deleteOnClose + * true to indicate the file shall be deleted when closed, false otherwise + */ + void updateDeleteOnCloseFlag(final boolean deleteOnClose); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java index 5141dd2548244..a15ddab5daf86 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java @@ -2,6 +2,7 @@ import java.io.IOException; import java.nio.channels.FileChannel; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -29,14 +30,17 @@ final class DistributedChannelWithAccessInfo implements ChannelWithAccessInfo { private final AtomicInteger referenceCounter; - DistributedChannelWithAccessInfo(final FileSystem fs, final Path checkpointFile, final int bufferSize) - throws IOException { + private final AtomicBoolean deleteOnClose; + + DistributedChannelWithAccessInfo(final FileSystem fs, final Path checkpointFile, final int bufferSize, + final boolean deleteOnClose) throws IOException { this.fs = fs; this.checkpointFile = checkpointFile; this.channel = new FileChannelWrapper(fs, checkpointFile, bufferSize, (short) 2); - this.reservedWritePosition = new AtomicLong(0); + this.reservedWritePosition = new AtomicLong(0L); this.referenceCounter = new AtomicInteger(0); + this.deleteOnClose = new AtomicBoolean(deleteOnClose); } /** @@ -105,7 +109,10 @@ public int decrementReferences() { this.reservedWritePosition.set(Long.MIN_VALUE); try { this.channel.close(); - this.fs.delete(this.checkpointFile, false); + if (this.deleteOnClose.get()) { + this.fs.delete(this.checkpointFile, false); + } + } catch (IOException ioex) { if (LOG.isErrorEnabled()) LOG.error("Error while closing spill file for file buffers: " + ioex.getMessage(), ioex); @@ -148,9 +155,20 @@ public void disposeSilently() { if (this.channel.isOpen()) { try { this.channel.close(); - this.fs.delete(this.checkpointFile, false); + if (this.deleteOnClose.get()) { + this.fs.delete(this.checkpointFile, false); + } } catch (Throwable t) { } } } + + /** + * {@inheritDoc} + */ + @Override + public void updateDeleteOnCloseFlag(final boolean deleteOnClose) { + + this.deleteOnClose.compareAndSet(true, deleteOnClose); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java index e514baad38010..e6ad2195b47f4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBuffer.java @@ -50,7 +50,8 @@ public final class FileBuffer extends Buffer { * @param ownerID */ FileBuffer(final int bufferSize, final long offset, final AbstractID ownerID, - final FileBufferManager fileBufferManager, final boolean distributed) throws IOException { + final FileBufferManager fileBufferManager, final boolean distributed, final boolean deleteOnClose) + throws IOException { this.ownerID = ownerID; this.fileBufferManager = fileBufferManager; @@ -58,7 +59,7 @@ public final class FileBuffer extends Buffer { this.bufferSize = bufferSize; this.writeMode.set(false); - this.fileChannel = fileBufferManager.getChannelAndIncrementReferences(ownerID, distributed); + this.fileChannel = fileBufferManager.getChannelAndIncrementReferences(ownerID, distributed, deleteOnClose); } /** @@ -70,7 +71,7 @@ public final class FileBuffer extends Buffer { * @throws IOException */ FileBuffer(final int bufferSize, final AbstractID ownerID, final FileBufferManager fileBufferManager, - final boolean distributed) throws IOException { + final boolean distributed, final boolean deleteOnClose) throws IOException { this.fileBufferManager = fileBufferManager; this.ownerID = ownerID; @@ -78,7 +79,8 @@ public final class FileBuffer extends Buffer { this.writeMode.set(true); final ChannelWithPosition cwp = fileBufferManager.getChannelForWriteAndIncrementReferences(ownerID, bufferSize, - distributed); + distributed, deleteOnClose); + this.fileChannel = cwp.getChannel(); this.offset = cwp.getOffset(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java index bfccf20ea4287..3654cb39d37ba 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/FileBufferManager.java @@ -143,7 +143,7 @@ private FileBufferManager() { */ public FileChannel getChannel(final AbstractID id, final boolean distributed) throws IOException { - final ChannelWithAccessInfo info = getChannelInternal(id, false, distributed); + final ChannelWithAccessInfo info = getChannelInternal(id, false, distributed, false); if (info != null) { return info.getChannel(); } else { @@ -159,10 +159,10 @@ public FileChannel getChannel(final AbstractID id, final boolean distributed) th * @throws IllegalStateException * Thrown, if the channel has not been registered or has already been removed. */ - public FileChannel getChannelAndIncrementReferences(final AbstractID owner, final boolean distributed) - throws IOException { + public FileChannel getChannelAndIncrementReferences(final AbstractID owner, final boolean distributed, + final boolean deleteOnClose) throws IOException { - final ChannelWithAccessInfo info = getChannelInternal(owner, false, distributed); + final ChannelWithAccessInfo info = getChannelInternal(owner, false, distributed, deleteOnClose); if (info != null) { return info.getAndIncrementReferences(); } else { @@ -181,14 +181,16 @@ public FileChannel getChannelAndIncrementReferences(final AbstractID owner, fina * The id for which to get the channel and reserve space. */ public ChannelWithPosition getChannelForWriteAndIncrementReferences(final AbstractID id, final int spaceToReserve, - final boolean distributed) throws IOException { + final boolean distributed, final boolean deleteOnClose) throws IOException { ChannelWithPosition c = null; do { // the return value may be zero, if someone asynchronously decremented the counter to zero // and caused the disposal of the channel. falling through the loop will create a // new channel. - c = getChannelInternal(id, true, distributed).reserveWriteSpaceAndIncrementReferences(spaceToReserve); + c = getChannelInternal(id, true, distributed, deleteOnClose).reserveWriteSpaceAndIncrementReferences( + spaceToReserve); + } while (c == null); return c; @@ -232,29 +234,60 @@ public void decrementReferences(final AbstractID id) { // -------------------------------------------------------------------------------------------- + private final Path constructDistributedPath(final AbstractID ownerID) { + + return this.distributedTempPath.suffix(Path.SEPARATOR + FILE_BUFFER_PREFIX + ownerID.toString()); + } + + private final File constructLocalFile(final AbstractID ownerID) { + + final int dirIndex = Math.abs(ownerID.hashCode()) % this.tmpDirs.length; + + return new File(this.tmpDirs[dirIndex] + ownerID.toString()); + } + private final ChannelWithAccessInfo getChannelInternal(final AbstractID id, final boolean createIfAbsent, - final boolean distributed) throws IOException { + final boolean distributed, final boolean deleteOnClose) throws IOException { ChannelWithAccessInfo cwa = this.fileMap.get(id); if (cwa == null) { - if (createIfAbsent) { + + // Check if file exists + if (distributed && this.distributedFileSystem != null) { + + final Path p = constructDistributedPath(id); + + if (this.distributedFileSystem.exists(p)) { + cwa = new DistributedChannelWithAccessInfo(this.distributedFileSystem, p, this.bufferSize, + deleteOnClose); + } + + } else { + + final File f = constructLocalFile(id); + if (f.exists()) { + cwa = new LocalChannelWithAccessInfo(f, deleteOnClose); + } + } + + // If file does not exist, check if we are allowed to create it + if (createIfAbsent && cwa == null) { if (distributed && this.distributedFileSystem != null) { - final String checkpointFile = this.distributedTempPath + File.separator + FILE_BUFFER_PREFIX - + id.toString(); - final Path p = new Path(checkpointFile); - cwa = new DistributedChannelWithAccessInfo(this.distributedFileSystem, p, this.bufferSize); + final Path p = constructDistributedPath(id); + cwa = new DistributedChannelWithAccessInfo(this.distributedFileSystem, p, this.bufferSize, + deleteOnClose); } else { // Construct the filename - final int dirIndex = Math.abs(id.hashCode()) % this.tmpDirs.length; - final File file = new File(this.tmpDirs[dirIndex] + id.toString()); - - cwa = new LocalChannelWithAccessInfo(file); + final File f = constructLocalFile(id); + cwa = new LocalChannelWithAccessInfo(f, deleteOnClose); } + } + if (cwa != null) { final ChannelWithAccessInfo alreadyContained = this.fileMap.putIfAbsent(id, cwa); if (alreadyContained != null) { // we had a race (should be a very rare event) and have created an @@ -262,12 +295,13 @@ private final ChannelWithAccessInfo getChannelInternal(final AbstractID id, fina cwa.disposeSilently(); cwa = alreadyContained; } - } else { return null; } } + cwa.updateDeleteOnCloseFlag(deleteOnClose); + return cwa; } // -------------------------------------------------------------------------------------------- diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java index b388b7d1a13cd..8ac08ec15b4f8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java @@ -4,6 +4,7 @@ import java.io.IOException; import java.io.RandomAccessFile; import java.nio.channels.FileChannel; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -25,12 +26,15 @@ final class LocalChannelWithAccessInfo implements ChannelWithAccessInfo { private final AtomicInteger referenceCounter; - LocalChannelWithAccessInfo(final File file) throws IOException { + private final AtomicBoolean deleteOnClose; + + LocalChannelWithAccessInfo(final File file, final boolean deleteOnClose) throws IOException { this.file = file; this.channel = new RandomAccessFile(file, "rw").getChannel(); - this.reservedWritePosition = new AtomicLong(0); + this.reservedWritePosition = new AtomicLong(0L); this.referenceCounter = new AtomicInteger(0); + this.deleteOnClose = new AtomicBoolean(deleteOnClose); } /** @@ -103,7 +107,9 @@ public int decrementReferences() { if (LOG.isErrorEnabled()) LOG.error("Error while closing spill file for file buffers: " + ioex.getMessage(), ioex); } - this.file.delete(); + if (this.deleteOnClose.get()) { + this.file.delete(); + } return current; } else { throw new IllegalStateException("The references to the file were already at zero."); @@ -145,6 +151,18 @@ public void disposeSilently() { } catch (Throwable t) { } } - this.file.delete(); + + if (this.deleteOnClose.get()) { + this.file.delete(); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void updateDeleteOnCloseFlag(final boolean deleteOnClose) { + + this.deleteOnClose.compareAndSet(true, deleteOnClose); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index 730cbfce06caf..79112bf58f25a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -83,7 +83,7 @@ public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws I if (this.envelopeConsumptionLog.followsLog() || this.inputGate.getChannelType() == ChannelType.NETWORK) { return BufferFactory.createFromFile(minimumSizeOfBuffer, this.inputGate.getGateID(), - this.fileBufferManager, false); + this.fileBufferManager, false, true); } return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElement.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElement.java index c83b9d697935f..666cb703bdba2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElement.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElement.java @@ -360,7 +360,7 @@ int spill(final AbstractID ownerID, final FileBufferManager fileBufferManager) t } final int size = buffer.size(); - final Buffer fileBuffer = BufferFactory.createFromFile(size, ownerID, fileBufferManager, false); + final Buffer fileBuffer = BufferFactory.createFromFile(size, ownerID, fileBufferManager, false, true); buffer.copyToBuffer(fileBuffer); this.bufferRef = fileBuffer; buffer.recycleBuffer(); @@ -389,7 +389,7 @@ int spill(final AbstractID ownerID, final FileBufferManager fileBufferManager) t } final int size = buffer.size(); - final Buffer fileBuffer = BufferFactory.createFromFile(size, ownerID, fileBufferManager, false); + final Buffer fileBuffer = BufferFactory.createFromFile(size, ownerID, fileBufferManager, false, true); buffer.copyToBuffer(fileBuffer); bufferQueue.offer(fileBuffer); buffer.recycleBuffer(); From 032d5afb588a661bd217cc10a891b0ab2daf7c00 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 25 Feb 2012 19:23:22 +0100 Subject: [PATCH 280/310] Fixed bug in ReplayInputChannelContext --- .../nephele/checkpointing/ReplayInputChannelContext.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java index 8cabb4b2f7285..015ef80dc834a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java @@ -59,7 +59,7 @@ public ChannelID getConnectedChannelID() { @Override public ChannelType getType() { - return this.encapsulatedContext.getType(); + return null; } /** From 7612fc74577a09110e65e15924c3e9f56c99ecdc Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 25 Feb 2012 21:12:01 +0100 Subject: [PATCH 281/310] Disabled receiver-side spilling --- .../nephele/taskmanager/runtime/RuntimeInputGateContext.java | 3 +-- .../nephele/taskmanager/runtime/RuntimeTaskContext.java | 5 ----- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java index 79112bf58f25a..17cc874993033 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java @@ -23,7 +23,6 @@ import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.BufferFactory; import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.io.channels.FileBufferManager; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; @@ -81,7 +80,7 @@ public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws I return buffer; } - if (this.envelopeConsumptionLog.followsLog() || this.inputGate.getChannelType() == ChannelType.NETWORK) { + if (this.envelopeConsumptionLog.followsLog()) { return BufferFactory.createFromFile(minimumSizeOfBuffer, this.inputGate.getGateID(), this.fileBufferManager, false, true); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index d886148867e49..1545ff5ac33cc 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -75,11 +75,6 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve private long startTime; - /** - * Stores whether the initial exhaustion of memory buffers has already been reported - */ - private boolean initialExhaustionOfMemoryBuffersReported = false; - RuntimeTaskContext(final RuntimeTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final Map tasksWithUndecidedCheckpoints) { From 1cc94d23533a020bf91118ef21ffef54143744a4 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sat, 25 Feb 2012 22:13:36 +0100 Subject: [PATCH 282/310] Changed calculation of meta data file size to enable faster recovery --- .../checkpointing/EphemeralCheckpoint.java | 40 +++++++++++-------- 1 file changed, 23 insertions(+), 17 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index f7cdc65448c9b..c3aad21d16c4a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -62,11 +62,6 @@ public class EphemeralCheckpoint implements OutputChannelForwarder { */ private static final Log LOG = LogFactory.getLog(EphemeralCheckpoint.class); - /** - * The number of envelopes to be stored in a single meta data file. - */ - private static final int ENVELOPES_PER_META_DATA_FILE = 10000; - /** * The buffer size in bytes to use for the meta data file channel. */ @@ -115,19 +110,24 @@ public class EphemeralCheckpoint implements OutputChannelForwarder { private final Path checkpointPath; /** - * The file system to write the checkpoint's to. + * The file system to write the checkpoints to. */ private FileSystem fileSystem; + /** + * The default block size of the file system to write the checkpoints to + */ + private long defaultBlockSize = -1L; + /** * The file channel to write the checkpoint's meta data. */ private FileChannel metaDataFileChannel = null; /** - * A counter for the number of serialized transfer envelopes. + * A counter for the number of bytes in the checkpoint per meta data file. */ - private int numberOfSerializedTransferEnvelopes = 0; + private long numberOfBytesPerMetaDataFile = 0; private Buffer firstSerializedFileBuffer = null; @@ -174,7 +174,6 @@ public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { if (dist) { final Path p = CheckpointUtils.getDistributedCheckpointPath(); - System.out.println("Distributed checkpoint path is " + p); if (p == null) { LOG.error("No distributed checkpoint path configured, writing local checkpoints instead"); this.checkpointPath = CheckpointUtils.getLocalCheckpointPath(); @@ -240,12 +239,16 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro } } - // Write the meta data of the transfer envelope to disk - if (this.numberOfSerializedTransferEnvelopes % ENVELOPES_PER_META_DATA_FILE == 0) { + if (this.fileSystem == null) { + this.fileSystem = this.checkpointPath.getFileSystem(); + } - if (this.fileSystem == null) { - this.fileSystem = this.checkpointPath.getFileSystem(); - } + if (this.defaultBlockSize < 0L) { + this.defaultBlockSize = this.fileSystem.getDefaultBlockSize(); + } + + // Finish meta data file when the corresponding checkpoint fraction is 10 times the file system's block size + if (this.numberOfBytesPerMetaDataFile > 10L * this.defaultBlockSize) { if (this.metaDataFileChannel != null) { this.metaDataFileChannel.close(); @@ -257,6 +260,9 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro // Increase the meta data suffix ++this.metaDataSuffix; } + + // Reset counter + this.numberOfBytesPerMetaDataFile = 0L; } if (this.metaDataFileChannel == null) { @@ -275,6 +281,9 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro } else { buffer.recycleBuffer(); } + + // Increase the number of serialized transfer envelopes + this.numberOfBytesPerMetaDataFile += buffer.size(); } // Look for close event @@ -288,9 +297,6 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro } } - // Increase the number of serialized transfer envelopes - ++this.numberOfSerializedTransferEnvelopes; - if (this.numberOfClosedChannels == this.numberOfConnectedChannels) { // Finally, close the underlying file From c1241753eb74b0fbb2b84ce76b6ae5be30477118 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 26 Feb 2012 16:56:37 +0100 Subject: [PATCH 283/310] Fixed problem with default block size in HDFS binding --- .../main/java/eu/stratosphere/nephele/fs/FileSystem.java | 5 +++++ .../nephele/fs/hdfs/DistributedFileSystem.java | 8 ++++++++ 2 files changed, 13 insertions(+) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileSystem.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileSystem.java index dd429bbcf591d..f4474e181e13d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileSystem.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileSystem.java @@ -292,6 +292,11 @@ public static FileSystem get(final URI uri) throws IOException { */ public abstract FSDataInputStream open(Path f) throws IOException; + /** + * Return the number of bytes that large input files should be optimally be split into to minimize I/O time. + * + * @return the number of bytes that large input files should be optimally be split into to minimize I/O time + */ public long getDefaultBlockSize() { return 32 * 1024 * 1024; // 32 MB; } diff --git a/nephele/nephele-hdfs/src/main/java/eu/stratosphere/nephele/fs/hdfs/DistributedFileSystem.java b/nephele/nephele-hdfs/src/main/java/eu/stratosphere/nephele/fs/hdfs/DistributedFileSystem.java index e8e6bd74b6e64..cc3baa9f0f12e 100644 --- a/nephele/nephele-hdfs/src/main/java/eu/stratosphere/nephele/fs/hdfs/DistributedFileSystem.java +++ b/nephele/nephele-hdfs/src/main/java/eu/stratosphere/nephele/fs/hdfs/DistributedFileSystem.java @@ -249,4 +249,12 @@ public boolean rename(final Path src, final Path dst) throws IOException { new org.apache.hadoop.fs.Path(dst.toString())); } + /** + * {@inheritDoc} + */ + @Override + public long getDefaultBlockSize() { + + return this.fs.getDefaultBlockSize(); + } } From 7e029eb52a4eda0ae20bd73a81e61959297b181e Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Sun, 26 Feb 2012 18:36:46 +0100 Subject: [PATCH 284/310] Improved strategy to read checkpoints as part of recovery --- .../checkpointing/EphemeralCheckpoint.java | 2 +- .../checkpointing/ReplayOutputBroker.java | 17 +++++++++++++++++ .../nephele/checkpointing/ReplayThread.java | 15 ++++++++++----- 3 files changed, 28 insertions(+), 6 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index c3aad21d16c4a..f190ca4b0fe2e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -248,7 +248,7 @@ private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) thro } // Finish meta data file when the corresponding checkpoint fraction is 10 times the file system's block size - if (this.numberOfBytesPerMetaDataFile > 10L * this.defaultBlockSize) { + if (this.numberOfBytesPerMetaDataFile > 10L * this.defaultBlockSize && !this.distributed) { if (this.metaDataFileChannel != null) { this.metaDataFileChannel.close(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java index cabd0512cfa0a..e7013c326faf4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java @@ -12,6 +12,7 @@ import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; +import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; final class ReplayOutputBroker implements OutputChannelForwarder, BufferProvider { @@ -27,6 +28,8 @@ final class ReplayOutputBroker implements OutputChannelForwarder, BufferProvider private final IncomingEventQueue incomingEventQueue; + private int nextEnvelopeToSend = 0; + ReplayOutputBroker(final BufferProvider bufferProvider, final OutputChannelForwardingChain forwardingChain, final IncomingEventQueue incomingEventQueue) { @@ -66,6 +69,11 @@ public void processEvent(final AbstractEvent event) { if (event instanceof ByteBufferedChannelCloseEvent) { LOG.info("Replay output broker received event to close channel"); + } else if (event instanceof UnexpectedEnvelopeEvent) { + final UnexpectedEnvelopeEvent uee = (UnexpectedEnvelopeEvent) event; + if (uee.getExpectedSequenceNumber() > this.nextEnvelopeToSend) { + this.nextEnvelopeToSend = uee.getExpectedSequenceNumber(); + } } else { LOG.warn("Received unknown event: " + event); } @@ -75,9 +83,18 @@ void outputEnvelope(final TransferEnvelope transferEnvelope) throws IOException, this.incomingEventQueue.processQueuedEvents(); + if (transferEnvelope.getSequenceNumber() == this.nextEnvelopeToSend) { + ++this.nextEnvelopeToSend; + } + this.forwardingChain.forwardEnvelope(transferEnvelope); } + int getNextEnvelopeToSend() { + + return this.nextEnvelopeToSend; + } + boolean hasFinished() { // Check for events diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index f39f4b09c9948..decd242882074 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -86,6 +86,8 @@ public void run() { } catch (Exception e) { + e.printStackTrace(); + if (this.executionObserver.isCanceled()) { changeExecutionState(ExecutionState.CANCELED, null); } else { @@ -196,7 +198,7 @@ private void replayCheckpoint() throws Exception { } // Wait for the file to be created - Thread.sleep(100); + Thread.sleep(1000); } @@ -206,12 +208,10 @@ private void replayCheckpoint() throws Exception { try { deserializer.read(fileChannel); - final TransferEnvelope transferEnvelope = deserializer - .getFullyDeserializedTransferEnvelope(); + final TransferEnvelope transferEnvelope = deserializer.getFullyDeserializedTransferEnvelope(); if (transferEnvelope != null) { - final ReplayOutputBroker broker = this.outputBrokerMap - .get(transferEnvelope.getSource()); + final ReplayOutputBroker broker = this.outputBrokerMap.get(transferEnvelope.getSource()); if (broker == null) { throw new IOException("Cannot find output broker for channel " + transferEnvelope.getSource()); @@ -225,6 +225,11 @@ private void replayCheckpoint() throws Exception { firstDeserializedFileBuffer = srcBuffer.duplicate(); } + if (transferEnvelope.getSequenceNumber() < broker.getNextEnvelopeToSend()) { + srcBuffer.recycleBuffer(); + continue; + } + final Buffer destBuffer = broker.requestEmptyBufferBlocking(srcBuffer.size()); srcBuffer.copyToBuffer(destBuffer); transferEnvelope.setBuffer(destBuffer); From ca08872d28ccf266b89ce0e228cb5260927a1121 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 27 Feb 2012 21:04:53 +0100 Subject: [PATCH 285/310] Increased size of envelope consumption log --- .../nephele/taskmanager/runtime/EnvelopeConsumptionLog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java index 6617edc1e7ae7..ec357aa0b63ea 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java @@ -24,7 +24,7 @@ final class EnvelopeConsumptionLog { private static final Log LOG = LogFactory.getLog(EnvelopeConsumptionLog.class); - private static final int LOG_WINDOW_SIZE = 64 * 1024; + private static final int LOG_WINDOW_SIZE = 256 * 1024; private static final int SIZE_OF_INTEGER = 4; From 078e87f3a705c896d5196adc9460fdb3af427edc Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 27 Feb 2012 21:42:48 +0100 Subject: [PATCH 286/310] Checkpoints are now written asynchronously --- .../checkpointing/EphemeralCheckpoint.java | 249 ++----------- .../checkpointing/ReplayOutputBroker.java | 9 + .../nephele/checkpointing/WriteThread.java | 328 ++++++++++++++++++ .../bytebuffered/OutputChannelForwarder.java | 2 + .../runtime/ForwardingBarrier.java | 9 + .../runtime/RuntimeDispatcher.java | 13 +- .../runtime/RuntimeOutputChannelBroker.java | 9 + .../taskmanager/runtime/SpillingBarrier.java | 21 +- 8 files changed, 409 insertions(+), 231 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index f190ca4b0fe2e..972bb72e02bdc 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -15,12 +15,8 @@ package eu.stratosphere.nephele.checkpointing; -import java.io.FileOutputStream; import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; import java.util.ArrayDeque; -import java.util.Iterator; import java.util.Queue; import org.apache.commons.logging.Log; @@ -28,20 +24,13 @@ import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; -import eu.stratosphere.nephele.taskmanager.transferenvelope.CheckpointSerializer; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.event.task.AbstractEvent; -import eu.stratosphere.nephele.event.task.EventList; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; -import eu.stratosphere.nephele.fs.FileChannelWrapper; -import eu.stratosphere.nephele.fs.FileSystem; -import eu.stratosphere.nephele.fs.Path; import eu.stratosphere.nephele.io.channels.Buffer; -import eu.stratosphere.nephele.io.channels.BufferFactory; import eu.stratosphere.nephele.io.channels.FileBufferManager; -import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; /** * An ephemeral checkpoint is a checkpoint that can be used to recover from @@ -51,7 +40,7 @@ * For network channels the ephemeral checkpoint is held into main memory until a checkpoint * decision is made. Based on this decision the checkpoint is either made permanent or discarded. *

- * This class is thread-safe. + * This class is not thread-safe. * * @author warneke */ @@ -62,21 +51,11 @@ public class EphemeralCheckpoint implements OutputChannelForwarder { */ private static final Log LOG = LogFactory.getLog(EphemeralCheckpoint.class); - /** - * The buffer size in bytes to use for the meta data file channel. - */ - private static final int BUFFER_SIZE = 4096; - /** * The enveloped which are currently queued until the state of the checkpoint is decided. */ private final Queue queuedEnvelopes = new ArrayDeque(); - /** - * The serializer to convert a transfer envelope into a byte stream. - */ - private final CheckpointSerializer transferEnvelopeSerializer = new CheckpointSerializer(); - /** * The task this checkpoint is created for. */ @@ -87,49 +66,10 @@ public class EphemeralCheckpoint implements OutputChannelForwarder { */ private final int numberOfConnectedChannels; - private final boolean distributed; - - /** - * The number of channels which can confirmed not to send any further data. - */ - private int numberOfClosedChannels = 0; - - /** - * The current suffix for the name of the file containing the meta data. - */ - private int metaDataSuffix = 0; - /** - * The file buffer manager used to allocate file buffers. + * Reference to a write thread that may be spawned to write the checkpoint data asynchronously */ - private final FileBufferManager fileBufferManager; - - /** - * The path to which the checkpoint meta data shall be written to. - */ - private final Path checkpointPath; - - /** - * The file system to write the checkpoints to. - */ - private FileSystem fileSystem; - - /** - * The default block size of the file system to write the checkpoints to - */ - private long defaultBlockSize = -1L; - - /** - * The file channel to write the checkpoint's meta data. - */ - private FileChannel metaDataFileChannel = null; - - /** - * A counter for the number of bytes in the checkpoint per meta data file. - */ - private long numberOfBytesPerMetaDataFile = 0; - - private Buffer firstSerializedFileBuffer = null; + private WriteThread writeThread = null; /** * This enumeration reflects the possible states an ephemeral @@ -156,39 +96,28 @@ public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { nooc += environment.getOutputGate(i).getNumberOfOutputChannels(); } - this.numberOfConnectedChannels = nooc; - final boolean dist = CheckpointUtils.createDistributedCheckpoint(); + this.numberOfConnectedChannels = nooc; this.checkpointingDecision = (ephemeral ? CheckpointingDecisionState.UNDECIDED : CheckpointingDecisionState.CHECKPOINTING); - this.fileBufferManager = FileBufferManager.getInstance(); - - if (LOG.isDebugEnabled()) + if (LOG.isDebugEnabled()) { LOG.debug("Created checkpoint for vertex " + task.getVertexID() + ", state " + this.checkpointingDecision); + } if (this.checkpointingDecision == CheckpointingDecisionState.CHECKPOINTING) { this.task.checkpointStateChanged(CheckpointState.PARTIAL); - } - - if (dist) { - final Path p = CheckpointUtils.getDistributedCheckpointPath(); - if (p == null) { - LOG.error("No distributed checkpoint path configured, writing local checkpoints instead"); - this.checkpointPath = CheckpointUtils.getLocalCheckpointPath(); - this.distributed = false; - } else { - this.checkpointPath = p; - this.distributed = true; - } - } else { - this.checkpointPath = CheckpointUtils.getLocalCheckpointPath(); - this.distributed = false; + this.writeThread = new WriteThread(FileBufferManager.getInstance(), this.task.getVertexID(), + this.numberOfConnectedChannels); } } - private void destroy() { + /** + * {@inheritDoc} + */ + @Override + public void destroy() { while (!this.queuedEnvelopes.isEmpty()) { @@ -198,152 +127,24 @@ private void destroy() { buffer.recycleBuffer(); } } - } - - private void write() throws IOException, InterruptedException { - while (!this.queuedEnvelopes.isEmpty()) { - writeTransferEnvelope(this.queuedEnvelopes.poll()); + if (this.writeThread != null) { + this.writeThread.cancelAndDestroy(); + this.writeThread = null; } } - private boolean renameCheckpointPart() throws IOException { - - final Path oldFile = this.checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + "_" - + this.task.getVertexID() + "_part"); - - final Path newFile = this.checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + "_" - + this.task.getVertexID() + "_" + this.metaDataSuffix); - - if (!this.fileSystem.rename(oldFile, newFile)) { - LOG.error("Unable to rename " + oldFile + " to " + newFile); - return false; - } - - return true; - } - - private void writeTransferEnvelope(final TransferEnvelope transferEnvelope) throws IOException, - InterruptedException { - - Buffer buffer = transferEnvelope.getBuffer(); - if (buffer != null) { - if (buffer.isBackedByMemory()) { - - // Make sure we transfer the encapsulated buffer to a file and release the memory buffer again - final Buffer fileBuffer = BufferFactory.createFromFile(buffer.size(), this.task.getVertexID(), - this.fileBufferManager, this.distributed, false); - buffer.copyToBuffer(fileBuffer); - transferEnvelope.setBuffer(fileBuffer); - buffer.recycleBuffer(); - } - } - - if (this.fileSystem == null) { - this.fileSystem = this.checkpointPath.getFileSystem(); - } - - if (this.defaultBlockSize < 0L) { - this.defaultBlockSize = this.fileSystem.getDefaultBlockSize(); - } - - // Finish meta data file when the corresponding checkpoint fraction is 10 times the file system's block size - if (this.numberOfBytesPerMetaDataFile > 10L * this.defaultBlockSize && !this.distributed) { - - if (this.metaDataFileChannel != null) { - this.metaDataFileChannel.close(); - this.metaDataFileChannel = null; - - // Rename file - renameCheckpointPart(); - - // Increase the meta data suffix - ++this.metaDataSuffix; - } - - // Reset counter - this.numberOfBytesPerMetaDataFile = 0L; - } - - if (this.metaDataFileChannel == null) { - this.metaDataFileChannel = getMetaDataFileChannel("_part"); - } - - this.transferEnvelopeSerializer.setTransferEnvelope(transferEnvelope); - while (this.transferEnvelopeSerializer.write(this.metaDataFileChannel)) { - } - - // The following code will prevent the underlying file from being closed - buffer = transferEnvelope.getBuffer(); - if (buffer != null) { - if (this.firstSerializedFileBuffer == null) { - this.firstSerializedFileBuffer = buffer; - } else { - buffer.recycleBuffer(); - } - - // Increase the number of serialized transfer envelopes - this.numberOfBytesPerMetaDataFile += buffer.size(); - } - - // Look for close event - final EventList eventList = transferEnvelope.getEventList(); - if (eventList != null) { - final Iterator it = eventList.iterator(); - while (it.hasNext()) { - if (it.next() instanceof ByteBufferedChannelCloseEvent) { - ++this.numberOfClosedChannels; - } - } - } - - if (this.numberOfClosedChannels == this.numberOfConnectedChannels) { - - // Finally, close the underlying file - if (this.firstSerializedFileBuffer != null) { - this.firstSerializedFileBuffer.recycleBuffer(); - } - - // Finish meta data file - if (this.metaDataFileChannel != null) { - this.metaDataFileChannel.close(); - - // Rename file - renameCheckpointPart(); - } - - // Write the meta data file to indicate the checkpoint is complete - this.metaDataFileChannel = getMetaDataFileChannel(CheckpointUtils.COMPLETED_CHECKPOINT_SUFFIX); - this.metaDataFileChannel.write(ByteBuffer.allocate(0)); - this.metaDataFileChannel.close(); - - LOG.info("Finished persistent checkpoint for vertex " + this.task.getVertexID()); - - // Send notification that checkpoint is completed - this.task.checkpointStateChanged(CheckpointState.COMPLETE); - } - } - - private FileChannel getMetaDataFileChannel(final String suffix) throws IOException { + private void write() throws IOException, InterruptedException { - if (LOG.isDebugEnabled()) { - LOG.debug("Writing checkpointing meta data to directory " + this.checkpointPath); + if (this.writeThread == null) { + this.writeThread = new WriteThread(FileBufferManager.getInstance(), task.getVertexID(), + this.numberOfConnectedChannels); + this.writeThread.start(); } - // Bypass FileSystem API for local checkpoints - if (!this.distributed) { - - final FileOutputStream fos = new FileOutputStream(this.checkpointPath.toUri().getPath() - + Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + "_" + this.task.getVertexID() + suffix); - - return fos.getChannel(); + while (!this.queuedEnvelopes.isEmpty()) { + this.writeThread.write(this.queuedEnvelopes.poll()); } - - return new FileChannelWrapper(this.fileSystem, this.checkpointPath.suffix(Path.SEPARATOR - + CheckpointUtils.METADATA_PREFIX + "_" + this.task.getVertexID() + suffix), BUFFER_SIZE, (short) 2); // TODO: - // Make - // replication - // configurable } public void setCheckpointDecisionSynchronously(final boolean checkpointDecision) throws IOException, @@ -380,7 +181,7 @@ public boolean forward(final TransferEnvelope transferEnvelope) throws IOExcepti if (this.checkpointingDecision == CheckpointingDecisionState.UNDECIDED) { this.queuedEnvelopes.add(dup); } else { - writeTransferEnvelope(dup); + this.writeThread.write(dup); } return true; @@ -397,7 +198,7 @@ public boolean isUndecided() { @Override public boolean hasDataLeft() { - return false; + return this.writeThread.hasDataLeft(); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java index e7013c326faf4..554ec72c9d2d9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java @@ -147,4 +147,13 @@ public void reportAsynchronousEvent() { this.bufferProvider.reportAsynchronousEvent(); } + + /** + * {@inheritDoc} + */ + @Override + public void destroy() { + + // Nothing to do here + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java new file mode 100644 index 0000000000000..585b46b62707b --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java @@ -0,0 +1,328 @@ +package eu.stratosphere.nephele.checkpointing; + +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.util.Iterator; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.event.task.EventList; +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; +import eu.stratosphere.nephele.fs.FileChannelWrapper; +import eu.stratosphere.nephele.fs.FileSystem; +import eu.stratosphere.nephele.fs.Path; +import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.BufferFactory; +import eu.stratosphere.nephele.io.channels.FileBufferManager; +import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; +import eu.stratosphere.nephele.taskmanager.transferenvelope.CheckpointSerializer; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +final class WriteThread extends Thread { + + /** + * The log object used to report problems. + */ + private static final Log LOG = LogFactory.getLog(WriteThread.class); + + /** + * The buffer size in bytes to use for the meta data file channel. + */ + private static final int BUFFER_SIZE = 4096; + + private final BlockingQueue queuedEnvelopes; + + /** + * The serializer to convert a transfer envelope into a byte stream. + */ + private final CheckpointSerializer transferEnvelopeSerializer = new CheckpointSerializer(); + + /** + * The current suffix for the name of the file containing the meta data. + */ + private int metaDataSuffix = 0; + + /** + * The file buffer manager used to allocate file buffers. + */ + private final FileBufferManager fileBufferManager; + + /** + * The number of channels connected to this checkpoint. + */ + private final int numberOfConnectedChannels; + + private final boolean distributed; + + /** + * The number of channels which can confirmed not to send any further data. + */ + private int numberOfClosedChannels = 0; + + /** + * The path to which the checkpoint meta data shall be written to. + */ + private final Path checkpointPath; + + private final ExecutionVertexID vertexID; + + /** + * The file system to write the checkpoints to. + */ + private FileSystem fileSystem; + + /** + * The default block size of the file system to write the checkpoints to + */ + private long defaultBlockSize = -1L; + + /** + * The file channel to write the checkpoint's meta data. + */ + private FileChannel metaDataFileChannel = null; + + /** + * A counter for the number of bytes in the checkpoint per meta data file. + */ + private long numberOfBytesPerMetaDataFile = 0; + + private Buffer firstSerializedFileBuffer = null; + + private volatile boolean hasDataLeft = false; + + private volatile boolean isCanceled = false; + + WriteThread(final FileBufferManager fileBufferManager, final ExecutionVertexID vertexID, + final int numberOfConnectedChannels) { + + this.fileBufferManager = fileBufferManager; + this.vertexID = vertexID; + this.numberOfConnectedChannels = numberOfConnectedChannels; + this.queuedEnvelopes = new ArrayBlockingQueue(256); + + final boolean dist = CheckpointUtils.createDistributedCheckpoint(); + + if (dist) { + final Path p = CheckpointUtils.getDistributedCheckpointPath(); + if (p == null) { + LOG.error("No distributed checkpoint path configured, writing local checkpoints instead"); + this.checkpointPath = CheckpointUtils.getLocalCheckpointPath(); + this.distributed = false; + } else { + this.checkpointPath = p; + this.distributed = true; + } + } else { + this.checkpointPath = CheckpointUtils.getLocalCheckpointPath(); + this.distributed = false; + } + } + + /** + * {@inheritDoc} + */ + @Override + public void run() { + + while (!this.isCanceled) { + + TransferEnvelope te = null; + + try { + te = this.queuedEnvelopes.take(); + + try { + + if (writeTransferEnvelope(te)) { + break; + } + } catch (IOException ioe) { + ioe.printStackTrace(); + } + + } catch (InterruptedException e) { + if (this.isCanceled) { + break; + } + } + } + + // Clean up in case we were canceled + while (!this.queuedEnvelopes.isEmpty()) { + final TransferEnvelope te = this.queuedEnvelopes.poll(); + final Buffer buffer = te.getBuffer(); + if (buffer != null) { + buffer.recycleBuffer(); + } + } + + // No more data left to be processed in this write thread + this.hasDataLeft = false; + } + + void write(final TransferEnvelope transferEnvelope) throws InterruptedException { + + this.hasDataLeft = true; + + this.queuedEnvelopes.put(transferEnvelope); + } + + void cancelAndDestroy() { + + this.isCanceled = true; + interrupt(); + } + + private boolean writeTransferEnvelope(final TransferEnvelope transferEnvelope) throws IOException, + InterruptedException { + + Buffer buffer = transferEnvelope.getBuffer(); + if (buffer != null) { + if (buffer.isBackedByMemory()) { + + // Make sure we transfer the encapsulated buffer to a file and release the memory buffer again + final Buffer fileBuffer = BufferFactory.createFromFile(buffer.size(), this.vertexID, + this.fileBufferManager, this.distributed, false); + buffer.copyToBuffer(fileBuffer); + transferEnvelope.setBuffer(fileBuffer); + buffer.recycleBuffer(); + } + } + + if (this.fileSystem == null) { + this.fileSystem = this.checkpointPath.getFileSystem(); + } + + if (this.defaultBlockSize < 0L) { + this.defaultBlockSize = this.fileSystem.getDefaultBlockSize(); + } + + // Finish meta data file when the corresponding checkpoint fraction is 10 times the file system's block size + if (this.numberOfBytesPerMetaDataFile > 10L * this.defaultBlockSize && !this.distributed) { + + if (this.metaDataFileChannel != null) { + this.metaDataFileChannel.close(); + this.metaDataFileChannel = null; + + // Rename file + renameCheckpointPart(); + + // Increase the meta data suffix + ++this.metaDataSuffix; + } + + // Reset counter + this.numberOfBytesPerMetaDataFile = 0L; + } + + if (this.metaDataFileChannel == null) { + this.metaDataFileChannel = getMetaDataFileChannel("_part"); + } + + this.transferEnvelopeSerializer.setTransferEnvelope(transferEnvelope); + while (this.transferEnvelopeSerializer.write(this.metaDataFileChannel)) { + } + + // The following code will prevent the underlying file from being closed + buffer = transferEnvelope.getBuffer(); + if (buffer != null) { + if (this.firstSerializedFileBuffer == null) { + this.firstSerializedFileBuffer = buffer; + } else { + buffer.recycleBuffer(); + } + + // Increase the number of serialized transfer envelopes + this.numberOfBytesPerMetaDataFile += buffer.size(); + } + + // Look for close event + final EventList eventList = transferEnvelope.getEventList(); + if (eventList != null) { + final Iterator it = eventList.iterator(); + while (it.hasNext()) { + if (it.next() instanceof ByteBufferedChannelCloseEvent) { + ++this.numberOfClosedChannels; + } + } + } + + if (this.numberOfClosedChannels == this.numberOfConnectedChannels) { + + // Finally, close the underlying file + if (this.firstSerializedFileBuffer != null) { + this.firstSerializedFileBuffer.recycleBuffer(); + } + + // Finish meta data file + if (this.metaDataFileChannel != null) { + this.metaDataFileChannel.close(); + + // Rename file + renameCheckpointPart(); + } + + // Write the meta data file to indicate the checkpoint is complete + this.metaDataFileChannel = getMetaDataFileChannel(CheckpointUtils.COMPLETED_CHECKPOINT_SUFFIX); + this.metaDataFileChannel.write(ByteBuffer.allocate(0)); + this.metaDataFileChannel.close(); + + LOG.info("Finished persistent checkpoint for vertex " + this.vertexID); + + // Send notification that checkpoint is completed + // this.task.checkpointStateChanged(CheckpointState.COMPLETE); //TODO: Fix me + return true; + } + + return false; + } + + private boolean renameCheckpointPart() throws IOException { + + final Path oldFile = this.checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + "_" + + this.vertexID + "_part"); + + final Path newFile = this.checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + "_" + + this.vertexID + "_" + this.metaDataSuffix); + + if (!this.fileSystem.rename(oldFile, newFile)) { + LOG.error("Unable to rename " + oldFile + " to " + newFile); + return false; + } + + return true; + } + + private FileChannel getMetaDataFileChannel(final String suffix) throws IOException { + + if (LOG.isDebugEnabled()) { + LOG.debug("Writing checkpointing meta data to directory " + this.checkpointPath); + } + + // Bypass FileSystem API for local checkpoints + if (!this.distributed) { + + final FileOutputStream fos = new FileOutputStream(this.checkpointPath.toUri().getPath() + + Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + "_" + this.vertexID + suffix); + + return fos.getChannel(); + } + + return new FileChannelWrapper(this.fileSystem, this.checkpointPath.suffix(Path.SEPARATOR + + CheckpointUtils.METADATA_PREFIX + "_" + this.vertexID + suffix), BUFFER_SIZE, (short) 2); // TODO: + // Make + // replication + // configurable + } + + boolean hasDataLeft() { + + return this.hasDataLeft; + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java index 41a75128d3020..7c13e3f82d789 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java @@ -12,4 +12,6 @@ public interface OutputChannelForwarder { boolean hasDataLeft(); void processEvent(AbstractEvent event); + + void destroy(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java index 879f929064b8f..7fb8ef402042d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java @@ -74,4 +74,13 @@ public void processEvent(final AbstractEvent event) { } } } + + /** + * {@inheritDoc} + */ + @Override + public void destroy() { + + // Nothing to do here + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java index 53c8c44251e67..fad6435681391 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java @@ -12,7 +12,7 @@ public final class RuntimeDispatcher implements OutputChannelForwarder { private final TransferEnvelopeDispatcher dispatcher; public RuntimeDispatcher(final TransferEnvelopeDispatcher dispatcher) { - + this.dispatcher = dispatcher; } @@ -38,6 +38,17 @@ public boolean hasDataLeft() { */ @Override public void processEvent(final AbstractEvent event) { + + // Nothing to do here + } + + /** + * {@inheritDoc} + */ + @Override + public void destroy() { + + // Nothing to do here } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java index 3ce115bba0959..3e5d172bb1de4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java @@ -230,4 +230,13 @@ public void transferEventToInputChannel(final AbstractEvent event) throws IOExce this.forwarder.forwardEnvelope(ephemeralTransferEnvelope); } } + + /** + * {@inheritDoc} + */ + @Override + public void destroy() { + + // Nothing to do here + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java index 5ead352966605..e07a37bcf02d0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java @@ -31,12 +31,12 @@ public SpillingBarrier(final boolean isReceiverRunning, final boolean mergeSpill @Override public boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - /*if (!this.isReceiverRunning) { - - // TODO: Add this to the spilling queue - - return false; - }*/ + /* + * if (!this.isReceiverRunning) { + * // TODO: Add this to the spilling queue + * return false; + * } + */ return true; } @@ -61,4 +61,13 @@ public void processEvent(final AbstractEvent event) { } } + /** + * {@inheritDoc} + */ + @Override + public void destroy() { + + // Nothing to do here + } + } From 6e14c1fd8fcacd04517f1b11060f9c77e94bfa6c Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 28 Feb 2012 17:21:44 +0100 Subject: [PATCH 287/310] Fixed problem with shut down of checkpoint's asynchronous write thread --- .../checkpointing/EphemeralCheckpoint.java | 27 +++++++++++++++++-- .../checkpointing/ReplayOutputBroker.java | 2 +- .../nephele/checkpointing/ReplayThread.java | 2 +- .../nephele/checkpointing/WriteThread.java | 2 -- .../bytebuffered/OutputChannelForwarder.java | 2 +- .../OutputChannelForwardingChain.java | 2 +- .../CheckpointDeserializer.java | 7 +++-- 7 files changed, 34 insertions(+), 10 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 972bb72e02bdc..2cf84cc62f80a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -66,6 +66,11 @@ public class EphemeralCheckpoint implements OutputChannelForwarder { */ private final int numberOfConnectedChannels; + /** + * Stores whether a completed checkpoint has already been announced to the task. + */ + private boolean completeCheckpointAnnounced = false; + /** * Reference to a write thread that may be spawned to write the checkpoint data asynchronously */ @@ -196,9 +201,27 @@ public boolean isUndecided() { * {@inheritDoc} */ @Override - public boolean hasDataLeft() { + public boolean hasDataLeft() throws IOException, InterruptedException { + + if (isUndecided()) { + setCheckpointDecisionSynchronously(true); + } + + if (this.writeThread == null) { + return false; + } + + if (this.writeThread.hasDataLeft()) { + return true; + } + + if (!this.completeCheckpointAnnounced) { + this.completeCheckpointAnnounced = true; + // Send notification that checkpoint is completed + this.task.checkpointStateChanged(CheckpointState.COMPLETE); + } - return this.writeThread.hasDataLeft(); + return false; } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java index 554ec72c9d2d9..d39728876a9ad 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java @@ -95,7 +95,7 @@ int getNextEnvelopeToSend() { return this.nextEnvelopeToSend; } - boolean hasFinished() { + boolean hasFinished() throws IOException, InterruptedException { // Check for events this.incomingEventQueue.processQueuedEvents(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index decd242882074..bdb0a398e3b2d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -152,7 +152,7 @@ void restart() { private void replayCheckpoint() throws Exception { - final CheckpointDeserializer deserializer = new CheckpointDeserializer(this.vertexID); + final CheckpointDeserializer deserializer = new CheckpointDeserializer(this.vertexID, !this.isCheckpointLocal); final Path checkpointPath = this.isCheckpointLocal ? CheckpointUtils.getLocalCheckpointPath() : CheckpointUtils .getDistributedCheckpointPath(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java index 585b46b62707b..1aee3750052fe 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java @@ -275,8 +275,6 @@ private boolean writeTransferEnvelope(final TransferEnvelope transferEnvelope) t LOG.info("Finished persistent checkpoint for vertex " + this.vertexID); - // Send notification that checkpoint is completed - // this.task.checkpointStateChanged(CheckpointState.COMPLETE); //TODO: Fix me return true; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java index 7c13e3f82d789..dc60093be6f6a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java @@ -9,7 +9,7 @@ public interface OutputChannelForwarder { boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException; - boolean hasDataLeft(); + boolean hasDataLeft() throws IOException, InterruptedException; void processEvent(AbstractEvent event); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java index a8422498f8c11..303ad0ab7d0ed 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java @@ -53,7 +53,7 @@ public void processEvent(final AbstractEvent event) { } } - public boolean anyForwarderHasDataLeft() { + public boolean anyForwarderHasDataLeft() throws IOException, InterruptedException { final Iterator it = this.forwardingChain.iterator(); while (it.hasNext()) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java index 577b6dbe9c6ba..ac4fdcd005309 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CheckpointDeserializer.java @@ -32,11 +32,14 @@ public final class CheckpointDeserializer extends AbstractDeserializer { private final FileBufferManager fileBufferManager; + private final boolean distributed; + private boolean bufferDataSerializationStarted = false; - public CheckpointDeserializer(final AbstractID ownerID) { + public CheckpointDeserializer(final AbstractID ownerID, final boolean distributed) { this.ownerID = ownerID; this.fileBufferManager = FileBufferManager.getInstance(); + this.distributed = distributed; } @Override @@ -57,7 +60,7 @@ protected boolean readBufferData(final ReadableByteChannel readableByteChannel) final long offset = byteBufferToLong(tempBuffer); final Buffer fileBuffer = BufferFactory.createFromCheckpoint(getSizeOfBuffer(), offset, this.ownerID, - this.fileBufferManager, true); + this.fileBufferManager, this.distributed); setBuffer(fileBuffer); From b3114e26bd9ba5f9493c296a735d3585f371c2a5 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 28 Feb 2012 18:30:45 +0100 Subject: [PATCH 288/310] Fixed problem with FQDN in instance connection info --- .../instance/InstanceConnectionInfo.java | 57 ++++++++++++------- 1 file changed, 37 insertions(+), 20 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java index f3db50cfd610d..1a94ab496ff45 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java @@ -54,22 +54,9 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparablenull if the domain name could not be determined + */ + public String getDomainName() { + + return this.domainname; + } + /** * {@inheritDoc} */ @@ -145,6 +160,7 @@ public void read(DataInput in) throws IOException { byte[] address = new byte[addr_length]; in.readFully(address); this.hostname = StringRecord.readString(in); + this.domainname = StringRecord.readString(in); try { this.inetAddress = InetAddress.getByAddress(address); @@ -165,6 +181,7 @@ public void write(DataOutput out) throws IOException { out.writeInt(this.inetAddress.getAddress().length); out.write(this.inetAddress.getAddress()); StringRecord.writeString(out, this.hostname); + StringRecord.writeString(out, this.domainname); out.writeInt(this.ipcPort); out.writeInt(this.dataPort); } From 8556577d2193add5f78b1dafa71538dcb0602419 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 29 Feb 2012 10:49:14 +0100 Subject: [PATCH 289/310] Improved robustness of checkpoint utility class --- .../checkpointing/CheckpointUtils.java | 75 ++++++++++++------- .../nephele/checkpointing/ReplayTask.java | 27 +------ 2 files changed, 52 insertions(+), 50 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java index 775a030b4b838..18e4d57b689a5 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java @@ -17,13 +17,19 @@ import java.io.IOException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.fs.FileSystem; import eu.stratosphere.nephele.fs.Path; +import eu.stratosphere.nephele.util.StringUtils; public final class CheckpointUtils { + private static final Log LOG = LogFactory.getLog(CheckpointUtils.class); + /** * The prefix for the name of the file containing the checkpoint meta data. */ @@ -75,12 +81,12 @@ public static Path getDistributedCheckpointPath() { return DISTRIBUTED_CHECKPOINT_PATH; } - public static boolean hasCompleteCheckpointAvailable(final ExecutionVertexID vertexID) throws IOException { + public static boolean hasCompleteCheckpointAvailable(final ExecutionVertexID vertexID) { return checkForCheckpoint(vertexID, COMPLETED_CHECKPOINT_SUFFIX); } - public static boolean hasPartialCheckpointAvailable(final ExecutionVertexID vertexID) throws IOException { + public static boolean hasPartialCheckpointAvailable(final ExecutionVertexID vertexID) { if (checkForCheckpoint(vertexID, "_0")) { return true; @@ -89,46 +95,61 @@ public static boolean hasPartialCheckpointAvailable(final ExecutionVertexID vert return checkForCheckpoint(vertexID, "_part"); } - public static boolean hasLocalCheckpointAvailable(final ExecutionVertexID vertexID) throws IOException { + public static boolean hasLocalCheckpointAvailable(final ExecutionVertexID vertexID) { - Path local = new Path(getLocalCheckpointPath() + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID - + "_0"); + try { + Path local = new Path(getLocalCheckpointPath() + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID + + "_0"); - final FileSystem localFs = local.getFileSystem(); + final FileSystem localFs = local.getFileSystem(); - if (localFs.exists(local)) { - return true; - } + if (localFs.exists(local)) { + return true; + } + + local = new Path(getLocalCheckpointPath() + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID + + "_part"); - local = new Path(getLocalCheckpointPath() + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID - + "_part"); + return localFs.exists(local); + + } catch (IOException ioe) { + LOG.warn(StringUtils.stringifyException(ioe)); + } - return localFs.exists(local); + return false; } - private static boolean checkForCheckpoint(final ExecutionVertexID vertexID, final String suffix) throws IOException { + private static boolean checkForCheckpoint(final ExecutionVertexID vertexID, final String suffix) { - final Path local = new Path(getLocalCheckpointPath() + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID - + suffix); + try { - final FileSystem localFs = local.getFileSystem(); + final Path local = new Path(getLocalCheckpointPath() + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID + + suffix); - if (localFs.exists(local)) { - return true; - } + final FileSystem localFs = local.getFileSystem(); - final Path distributedCheckpointPath = getDistributedCheckpointPath(); - if (distributedCheckpointPath == null) { - return false; - } + if (localFs.exists(local)) { + return true; + } - final Path distributed = new Path(distributedCheckpointPath + Path.SEPARATOR + METADATA_PREFIX + "_" + vertexID - + suffix); + final Path distributedCheckpointPath = getDistributedCheckpointPath(); + if (distributedCheckpointPath == null) { + return false; + } + + final Path distributed = new Path(distributedCheckpointPath + Path.SEPARATOR + METADATA_PREFIX + "_" + + vertexID + + suffix); - final FileSystem distFs = distributed.getFileSystem(); + final FileSystem distFs = distributed.getFileSystem(); - return distFs.exists(distributed); + return distFs.exists(distributed); + + } catch (IOException ioe) { + LOG.warn(StringUtils.stringifyException(ioe)); + } + return false; } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index df8f994ce7cfa..57f8e71922747 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -15,7 +15,6 @@ package eu.stratosphere.nephele.checkpointing; -import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; @@ -169,7 +168,8 @@ public ReplayTask(final ExecutionVertexID vertexID, final Environment environmen this.vertexID = vertexID; this.environment = new CheckpointEnvironment(this.vertexID, environment, - hasLocalCheckpointAvailable(this.vertexID), hasCompleteCheckpointAvailable(this.vertexID), + CheckpointUtils.hasLocalCheckpointAvailable(this.vertexID), + CheckpointUtils.hasCompleteCheckpointAvailable(this.vertexID), this.outputBrokerMap); this.environment.setExecutionObserver(new ReplayTaskExecutionObserver(null)); @@ -183,7 +183,8 @@ public ReplayTask(final RuntimeTask encapsulatedTask, final TaskManager taskMana this.vertexID = encapsulatedTask.getVertexID(); this.environment = new CheckpointEnvironment(this.vertexID, encapsulatedTask.getEnvironment(), - hasLocalCheckpointAvailable(this.vertexID), hasCompleteCheckpointAvailable(this.vertexID), + CheckpointUtils.hasLocalCheckpointAvailable(this.vertexID), + CheckpointUtils.hasCompleteCheckpointAvailable(this.vertexID), this.outputBrokerMap); this.environment.setExecutionObserver(new ReplayTaskExecutionObserver(null)); @@ -196,26 +197,6 @@ public ReplayTask(final RuntimeTask encapsulatedTask, final TaskManager taskMana this.taskManager = taskManager; } - private static boolean hasCompleteCheckpointAvailable(final ExecutionVertexID vertexID) { - - try { - return CheckpointUtils.hasCompleteCheckpointAvailable(vertexID); - } catch (IOException ioe) { - } - - return false; - } - - private static boolean hasLocalCheckpointAvailable(final ExecutionVertexID vertexID) { - - try { - return CheckpointUtils.hasLocalCheckpointAvailable(vertexID); - } catch (IOException ioe) { - } - - return false; - } - /** * {@inheritDoc} */ From ea84ab28962cef3fd842c5049f75ee3c72d16be1 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 29 Feb 2012 14:04:23 +0100 Subject: [PATCH 290/310] Improved robustness of clean-up procedure when task is canceled --- .../ReplayOutputChannelContext.java | 9 ++++++--- .../checkpointing/ReplayOutputGateContext.java | 2 +- .../AbstractOutputChannelContext.java | 18 +++++++++++++++++- .../OutputChannelForwardingChain.java | 8 ++++++++ .../runtime/RuntimeOutputChannelContext.java | 14 +++----------- .../runtime/RuntimeOutputGateContext.java | 2 +- 6 files changed, 36 insertions(+), 17 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java index 6634dc61dfbd1..6f03021013ae8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java @@ -6,6 +6,7 @@ import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; public final class ReplayOutputChannelContext extends AbstractOutputChannelContext implements OutputChannelContext { @@ -17,8 +18,9 @@ public final class ReplayOutputChannelContext extends AbstractOutputChannelConte private final OutputChannelContext encapsulatedContext; ReplayOutputChannelContext(final JobID jobID, final ChannelID channelID, - final IncomingEventQueue incomingEventQueue, final OutputChannelContext encapsulatedContext) { - super(incomingEventQueue); + final OutputChannelForwardingChain forwardingChain, final IncomingEventQueue incomingEventQueue, + final OutputChannelContext encapsulatedContext) { + super(forwardingChain, incomingEventQueue); this.jobID = jobID; this.channelID = channelID; @@ -83,9 +85,10 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { @Override public void destroy() { + super.destroy(); + if (this.encapsulatedContext != null) { this.encapsulatedContext.destroy(); } } - } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java index 59e650b09aa6c..70a042f897929 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java @@ -44,7 +44,7 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp // Register output broker this.taskContext.registerReplayOutputBroker(channelID, outputBroker); - return new ReplayOutputChannelContext(null, channelID, incomingEventQueue, previousContext); + return new ReplayOutputChannelContext(null, channelID, forwardingChain, incomingEventQueue, previousContext); } private static void activateForwardingBarrier(final OutputChannelContext previousContext) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java index f59474883b95b..65adcc24790bd 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java @@ -12,8 +12,15 @@ public abstract class AbstractOutputChannelContext implements OutputChannelConte */ private final IncomingEventQueue incomingEventQueue; - public AbstractOutputChannelContext(final IncomingEventQueue incomingEventQueue) { + /** + * The forwarding chain used by this output channel context. + */ + private final OutputChannelForwardingChain forwardingChain; + + public AbstractOutputChannelContext(final OutputChannelForwardingChain forwardingChain, + final IncomingEventQueue incomingEventQueue) { + this.forwardingChain = forwardingChain; this.incomingEventQueue = incomingEventQueue; } @@ -37,4 +44,13 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { this.incomingEventQueue.offer(it.next()); } } + + /** + * {@inheritDoc} + */ + @Override + public void destroy() { + + this.forwardingChain.destroy(); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java index 303ad0ab7d0ed..a1e6797238fd4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java @@ -74,4 +74,12 @@ private void recycleEnvelope(final TransferEnvelope transferEnvelope) { } } + public void destroy() { + + final Iterator it = this.forwardingChain.iterator(); + while (it.hasNext()) { + it.next().destroy(); + } + } + } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java index a106f7f255225..38bead44a589f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java @@ -21,14 +21,15 @@ import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; +import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; public final class RuntimeOutputChannelContext extends AbstractOutputChannelContext { private final AbstractByteBufferedOutputChannel byteBufferedOutputChannel; RuntimeOutputChannelContext(final AbstractByteBufferedOutputChannel byteBufferedOutputChannel, - final IncomingEventQueue incomingEventQueue) { - super(incomingEventQueue); + final OutputChannelForwardingChain forwardingChain, final IncomingEventQueue incomingEventQueue) { + super(forwardingChain, incomingEventQueue); this.byteBufferedOutputChannel = byteBufferedOutputChannel; } @@ -77,13 +78,4 @@ public ChannelType getType() { return this.byteBufferedOutputChannel.getType(); } - - /** - * {@inheritDoc} - */ - @Override - public void destroy() { - // TODO Auto-generated method stub - - } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java index 93a017eb4220e..9c3726836ba94 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java @@ -161,6 +161,6 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp forwardingChain.addForwarder(new SpillingBarrier(isReceiverRunning, mergeSpillBuffers)); forwardingChain.addForwarder(this.taskContext.getRuntimeDispatcher()); - return new RuntimeOutputChannelContext(outputChannel, incomingEventQueue); + return new RuntimeOutputChannelContext(outputChannel, forwardingChain, incomingEventQueue); } } From fc045f3384f2e0bedc91e7c6057754d470ef189a Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 29 Feb 2012 14:08:28 +0100 Subject: [PATCH 291/310] Fixed bug in method to cancel/kill task --- .../nephele/checkpointing/ReplayTask.java | 19 +++++----- .../taskmanager/runtime/RuntimeTask.java | 35 ++++++++++--------- 2 files changed, 28 insertions(+), 26 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index 57f8e71922747..611dde228bb61 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -325,19 +325,20 @@ private void cancelOrKillExecution(final boolean cancel) { if (this.encapsulatedExecutionState != null) { this.encapsulatedExecutionState = ExecutionState.CANCELING; } + reportExecutionStateChange(true, null); - } - // Request user code to shut down - if (this.encapsulatedTask != null) { + // Request user code to shut down + if (this.encapsulatedTask != null) { - try { - final AbstractInvokable invokable = this.encapsulatedTask.getRuntimeEnvironment().getInvokable(); - if (invokable != null) { - invokable.cancel(); + try { + final AbstractInvokable invokable = this.encapsulatedTask.getRuntimeEnvironment().getInvokable(); + if (invokable != null) { + invokable.cancel(); + } + } catch (Throwable e) { + LOG.error(StringUtils.stringifyException(e)); } - } catch (Throwable e) { - LOG.error(StringUtils.stringifyException(e)); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index 338eeff205c82..bddcf0726583b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -69,13 +69,14 @@ public final class RuntimeTask implements Task, ExecutionObserver { private volatile ExecutionState executionState = ExecutionState.STARTING; private Queue registeredListeners = new ConcurrentLinkedQueue(); - + // DW: Start of temporary code private double pactInputOutputRatioSum = 0.0; - + private int numberOfPactInputOutputRatioEntries = 0; + // DW: End of temporay code - + public RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, final TaskManager taskManager) { @@ -230,16 +231,16 @@ private void cancelOrKillExecution(final boolean cancel) { this.isCanceled = true; // Change state executionStateChanged(ExecutionState.CANCELING, null); - } - // Request user code to shut down - try { - final AbstractInvokable invokable = this.environment.getInvokable(); - if (invokable != null) { - invokable.cancel(); + // Request user code to shut down + try { + final AbstractInvokable invokable = this.environment.getInvokable(); + if (invokable != null) { + invokable.cancel(); + } + } catch (Throwable e) { + LOG.error(StringUtils.stringifyException(e)); } - } catch (Throwable e) { - LOG.error(StringUtils.stringifyException(e)); } // Continuously interrupt the user thread until it changed to state CANCELED @@ -435,19 +436,19 @@ public ExecutionState getExecutionState() { // DW: Start of temporary code @Override public void reportPACTDataStatistics(final long numberOfConsumedBytes, final long numberOfProducedBytes) { - + this.pactInputOutputRatioSum += ((double) numberOfProducedBytes / (double) numberOfConsumedBytes); ++this.numberOfPactInputOutputRatioEntries; } - + double getPACTInputOutputRatio() { - - if(this.numberOfPactInputOutputRatioEntries == 0) { + + if (this.numberOfPactInputOutputRatioEntries == 0) { return -1.0; } - + return (this.pactInputOutputRatioSum / (double) this.numberOfPactInputOutputRatioEntries); } // DW: End of temporary code - + } From a1cb0293788e396aeb27fafccdf655cd57e7394b Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 29 Feb 2012 15:38:37 +0100 Subject: [PATCH 292/310] Introduced new checkpoint state undecided --- .../nephele/checkpointing/EphemeralCheckpoint.java | 1 + .../nephele/executiongraph/CheckpointState.java | 5 +++++ .../nephele/executiongraph/ExecutionVertex.java | 7 ++++--- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 2cf84cc62f80a..534b506e8f5f6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -168,6 +168,7 @@ public void setCheckpointDecisionSynchronously(final boolean checkpointDecision) this.checkpointingDecision = CheckpointingDecisionState.NO_CHECKPOINTING; // Simply destroy the checkpoint destroy(); + this.task.checkpointStateChanged(CheckpointState.NONE); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/CheckpointState.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/CheckpointState.java index cb0ea0e3afc5d..93dcbf2430961 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/CheckpointState.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/CheckpointState.java @@ -22,6 +22,11 @@ */ public enum CheckpointState { + /** + * The vertex has not yet decided to create a checkpoint or not. + */ + UNDECIDED, + /** * The vertex has no checkpoint and cannot be recovered. */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index efe4a3e5cea0c..1876be409613c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -137,7 +137,8 @@ public final class ExecutionVertex { /** * The current checkpoint state of this vertex. */ - private final AtomicEnum checkpointState = new AtomicEnum(CheckpointState.NONE); + private final AtomicEnum checkpointState = new AtomicEnum( + CheckpointState.UNDECIDED); /** * The execution pipeline this vertex is part of. @@ -647,10 +648,10 @@ public TaskCancelResult cancelTask() { final ExecutionState previousState = this.executionState.get(); - if(previousState == ExecutionState.CANCELED) { + if (previousState == ExecutionState.CANCELED) { return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); } - + if (updateExecutionState(ExecutionState.CANCELING) != ExecutionState.CANCELING) { if (this.groupVertex.getStageNumber() != this.executionGraph.getIndexOfCurrentExecutionStage()) { From 640df79805ffd767041fa08498c7a5c96ef9c217 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 29 Feb 2012 21:35:08 +0100 Subject: [PATCH 293/310] Added method to trigger checkpoint decision asynchronously --- .../CheckpointDecisionRequester.java | 21 +++++++ .../checkpointing/EphemeralCheckpoint.java | 31 +++++++++- .../nephele/checkpointing/ReplayTask.java | 2 - .../executiongraph/ExecutionVertex.java | 48 +++++++++++++++ .../nephele/instance/AbstractInstance.java | 8 ++- .../jobmanager/scheduler/RecoveryLogic.java | 20 +++++++ .../protocols/TaskOperationProtocol.java | 6 +- .../nephele/taskmanager/Task.java | 7 +-- .../taskmanager/TaskCheckpointResult.java | 45 ++++++++++++++ .../nephele/taskmanager/TaskManager.java | 59 +++++++++++++++---- .../bufferprovider/LocalBufferPoolOwner.java | 6 ++ .../ByteBufferedChannelManager.java | 16 +++-- .../taskmanager/runtime/RuntimeTask.java | 40 ++++++++++++- .../runtime/RuntimeTaskContext.java | 8 +-- 14 files changed, 280 insertions(+), 37 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionRequester.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskCheckpointResult.java diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionRequester.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionRequester.java new file mode 100644 index 0000000000000..39a721243f9c9 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecisionRequester.java @@ -0,0 +1,21 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; + +public interface CheckpointDecisionRequester { + + void requestCheckpointDecision(); +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 534b506e8f5f6..fe9b9db529292 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -44,7 +44,7 @@ * * @author warneke */ -public class EphemeralCheckpoint implements OutputChannelForwarder { +public class EphemeralCheckpoint implements OutputChannelForwarder, CheckpointDecisionRequester { /** * The log object used to report problems. @@ -91,6 +91,20 @@ private enum CheckpointingDecisionState { */ private CheckpointingDecisionState checkpointingDecision; + /** + * Stores whether a checkpoint decision has been requested asynchronously. + */ + private volatile boolean asyncronousCheckpointDecisionRequested = false; + + /** + * Constructs a new ephemeral checkpoint. + * + * @param task + * the task this checkpoint belongs to + * @param ephemeral + * true if the checkpoint is initially ephemeral, false if the checkpoint shall be + * persistent from the beginning + */ public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { this.task = task; @@ -186,6 +200,12 @@ public boolean forward(final TransferEnvelope transferEnvelope) throws IOExcepti if (this.checkpointingDecision == CheckpointingDecisionState.UNDECIDED) { this.queuedEnvelopes.add(dup); + + if (this.asyncronousCheckpointDecisionRequested) { + // TODO: Move decision logic here + setCheckpointDecisionSynchronously(true); + } + } else { this.writeThread.write(dup); } @@ -233,4 +253,13 @@ public void processEvent(final AbstractEvent event) { // TODO Auto-generated method stub } + + /** + * {@inheritDoc} + */ + @Override + public void requestCheckpointDecision() { + + this.asyncronousCheckpointDecisionRequested = true; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index 611dde228bb61..4c9109f0bced8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -38,7 +38,6 @@ import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; -import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; import eu.stratosphere.nephele.template.AbstractInvokable; import eu.stratosphere.nephele.template.InputSplitProvider; @@ -427,7 +426,6 @@ public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) { */ @Override public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher, - final Map tasksWithUndecidedCheckpoints, final LocalBufferPoolOwner previousBufferPoolOwner) { return new ReplayTaskContext(this, transferEnvelopeDispatcher, previousBufferPoolOwner, this.environment diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 1876be409613c..104aa44556f52 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -45,6 +45,7 @@ import eu.stratosphere.nephele.taskmanager.AbstractTaskResult; import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; +import eu.stratosphere.nephele.taskmanager.TaskCheckpointResult; import eu.stratosphere.nephele.taskmanager.TaskKillResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionWrapper; @@ -379,6 +380,33 @@ public void updateCheckpointState(final CheckpointState newCheckpointState) { } } + public void waitForCheckpointStateChange(final CheckpointState initialValue, final long timeout) + throws InterruptedException { + + if (timeout <= 0L) { + throw new IllegalArgumentException("Argument timeout must be greather than zero"); + } + + final long startTime = System.currentTimeMillis(); + + while (this.checkpointState.get() == initialValue) { + + Thread.sleep(1); + + if (startTime + timeout < System.currentTimeMillis()) { + break; + } + } + } + + public void waitForCheckpointStateChange(final CheckpointState initialValue) throws InterruptedException { + + while (this.checkpointState.get() == initialValue) { + + Thread.sleep(1); + } + } + /** * Assigns the execution vertex with an {@link AllocatedResource}. * @@ -636,6 +664,26 @@ public TaskKillResult killTask() { } } + public TaskCheckpointResult requestCheckpointDecision() { + + if (this.allocatedResource == null) { + final TaskCheckpointResult result = new TaskCheckpointResult(getID(), + AbstractTaskResult.ReturnCode.NO_INSTANCE); + result.setDescription("Assigned instance of vertex " + this.toString() + " is null!"); + return result; + } + + try { + return this.allocatedResource.getInstance().requestCheckpointDecision(this.vertexID); + + } catch (IOException e) { + final TaskCheckpointResult result = new TaskCheckpointResult(getID(), + AbstractTaskResult.ReturnCode.IPC_ERROR); + result.setDescription(StringUtils.stringifyException(e)); + return result; + } + } + /** * Cancels and removes the task represented by this vertex * from the instance it is currently running on. If the task diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java index 6015cf5d9d85f..0a9383f28e203 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java @@ -34,6 +34,7 @@ import eu.stratosphere.nephele.protocols.PluginCommunicationProtocol; import eu.stratosphere.nephele.protocols.TaskOperationProtocol; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; +import eu.stratosphere.nephele.taskmanager.TaskCheckpointResult; import eu.stratosphere.nephele.taskmanager.TaskKillResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionWrapper; @@ -225,6 +226,11 @@ public synchronized TaskCancelResult cancelTask(final ExecutionVertexID id) thro return getTaskManager().cancelTask(id); } + public synchronized TaskCheckpointResult requestCheckpointDecision(final ExecutionVertexID id) throws IOException { + + return getTaskManager().requestCheckpointDecision(id); + } + /** * Kills the task identified by the given ID at the instance's * {@link eu.stratosphere.nephele.taskmanager.TaskManager}. @@ -311,7 +317,7 @@ public synchronized void killTaskManager() throws IOException { getTaskManager().killTaskManager(); } -/** + /** * Connects to the plugin component of this instance's task manager and sends data to the plugin with the given ID. * * @param pluginID diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java index a81534788f7f0..e4316453a5805 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java @@ -42,6 +42,7 @@ import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode; +import eu.stratosphere.nephele.taskmanager.TaskCheckpointResult; import eu.stratosphere.nephele.types.Record; import eu.stratosphere.nephele.util.SerializableHashSet; import eu.stratosphere.nephele.util.StringUtils; @@ -163,6 +164,25 @@ private static void findVerticesToRestart(final ExecutionVertex failedVertex, if (hasInstanceAssigned(predecessor)) { + // At the moment, there no checkpoint decision for this vertex + if (predecessor.getCheckpointState() == CheckpointState.UNDECIDED) { + final TaskCheckpointResult result = predecessor.requestCheckpointDecision(); + if (result.getReturnCode() != ReturnCode.SUCCESS) { + // Assume we do not have a checkpoint in this case + predecessor.updateCheckpointState(CheckpointState.NONE); + } else { + + try { + predecessor.waitForCheckpointStateChange(CheckpointState.UNDECIDED, 100L); + } catch (InterruptedException e) { + } + + if (predecessor.getCheckpointState() == CheckpointState.UNDECIDED) { + predecessor.updateCheckpointState(CheckpointState.NONE); + } + } + } + if (predecessor.getCheckpointState() == CheckpointState.NONE) { verticesToBeCanceled.add(predecessor); } else { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index 7462b965ba111..bd947473bdc03 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -26,6 +26,7 @@ import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.protocols.VersionedProtocol; import eu.stratosphere.nephele.taskmanager.TaskCancelResult; +import eu.stratosphere.nephele.taskmanager.TaskCheckpointResult; import eu.stratosphere.nephele.taskmanager.TaskKillResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionWrapper; @@ -73,6 +74,9 @@ List submitTasks(List tasks) */ TaskKillResult killTask(ExecutionVertexID id) throws IOException; + + TaskCheckpointResult requestCheckpointDecision(ExecutionVertexID id) throws IOException; + /** * Queries the task manager about the cache status of the libraries stated in the {@link LibraryCacheProfileRequest} * object. @@ -95,7 +99,7 @@ List submitTasks(List tasks) * thrown if an error occurs during this remote procedure call */ void updateLibraryCache(LibraryCacheUpdate update) throws IOException; - + /** * Removes the checkpoints which are identified by the provided list of vertex IDs. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java index c7dd480c262b2..9efb247341777 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java @@ -15,8 +15,6 @@ package eu.stratosphere.nephele.taskmanager; -import java.util.Map; - import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionState; @@ -27,7 +25,6 @@ import eu.stratosphere.nephele.services.memorymanager.MemoryManager; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; -import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; import eu.stratosphere.nephele.template.InputSplitProvider; @@ -139,7 +136,5 @@ public interface Task { */ ExecutionState getExecutionState(); - TaskContext createTaskContext(TransferEnvelopeDispatcher transferEnvelopeDispatcher, - Map tasksWithUndecidedCheckpoints, - LocalBufferPoolOwner previousBufferPoolOwner); + TaskContext createTaskContext(TransferEnvelopeDispatcher transferEnvelopeDispatcher, LocalBufferPoolOwner previousBufferPoolOwner); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskCheckpointResult.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskCheckpointResult.java new file mode 100644 index 0000000000000..0202a27f395cf --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskCheckpointResult.java @@ -0,0 +1,45 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager; + +import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; + +/** + * A TaskCheckResult is used to report the results of a checkpoint decision request. + * + * @author warneke + */ +public class TaskCheckpointResult extends AbstractTaskResult { + + /** + * Constructs a new task checkpoint result. + * + * @param vertexID + * the task ID this result belongs to + * @param returnCode + * the return code of the cancel + */ + public TaskCheckpointResult(final ExecutionVertexID vertexID, final ReturnCode returnCode) { + super(vertexID, returnCode); + } + + /** + * Constructs an empty task checkpoint result. + */ + public TaskCheckpointResult() { + super(); + } +} \ No newline at end of file diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index 632c8d4e7671b..e9321bab35d81 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -432,22 +432,15 @@ public void run() { @Override public TaskKillResult killTask(final ExecutionVertexID id) throws IOException { - // Check if the task is registered with our task manager - Task tmpTask; - - synchronized (this.runningTasks) { - - tmpTask = this.runningTasks.get(id); + final Task task = this.runningTasks.get(id); - if (tmpTask == null) { - final TaskKillResult taskKillResult = new TaskKillResult(id, + if (task == null) { + final TaskKillResult taskKillResult = new TaskKillResult(id, AbstractTaskResult.ReturnCode.TASK_NOT_FOUND); - taskKillResult.setDescription("No task with ID + " + id + " is currently running"); - return taskKillResult; - } + taskKillResult.setDescription("No task with ID + " + id + " is currently running"); + return taskKillResult; } - final Task task = tmpTask; // Execute call in a new thread so IPC thread can return immediately final Thread tmpThread = new Thread(new Runnable() { @@ -463,6 +456,47 @@ public void run() { return new TaskKillResult(id, AbstractTaskResult.ReturnCode.SUCCESS); } + /** + * {@inheritDoc} + */ + @Override + public TaskCheckpointResult requestCheckpointDecision(ExecutionVertexID id) throws IOException { + + final Task task = this.runningTasks.get(id); + + if (task == null) { + final TaskCheckpointResult taskCheckpointResult = new TaskCheckpointResult(id, + AbstractTaskResult.ReturnCode.TASK_NOT_FOUND); + taskCheckpointResult.setDescription("No task with ID + " + id + " is currently running"); + return taskCheckpointResult; + } + + if (!(task instanceof RuntimeTask)) { + final TaskCheckpointResult taskCheckpointResult = new TaskCheckpointResult(id, + AbstractTaskResult.ReturnCode.TASK_NOT_FOUND); + taskCheckpointResult.setDescription("No task with ID + " + id + " is not a runtime task"); + return taskCheckpointResult; + } + + final RuntimeTask runtimeTask = (RuntimeTask) task; + + // Request a checkpoint decision and return + if (!runtimeTask.requestCheckpointDecision()) { + final TaskCheckpointResult taskCheckpointResult = new TaskCheckpointResult(id, + AbstractTaskResult.ReturnCode.TASK_NOT_FOUND); + taskCheckpointResult.setDescription("No task with ID + " + id + " has not yet created a checkpoint"); + } + + reportAsyncronousEvent(id); + + return new TaskCheckpointResult(id, AbstractTaskResult.ReturnCode.SUCCESS); + } + + private void reportAsyncronousEvent(final ExecutionVertexID vertexID) { + + this.byteBufferedChannelManager.reportAsynchronousEvent(vertexID); + } + /** * {@inheritDoc} */ @@ -699,7 +733,6 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID id, } } - public void checkpointStateChanged(final JobID jobID, final ExecutionVertexID id, final CheckpointState newCheckpointState) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java index a565959b1e455..724afd8e7d180 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java @@ -48,4 +48,10 @@ public interface LocalBufferPoolOwner { * Logs the current status of the local buffer pool. This method is intended mainly for debugging purposes. */ void logBufferUtilization(); + + /** + * Reports an asynchronous event. Calling this method interrupts each blocking method of the buffer pool owner and + * allows the blocked thread to respond to the event. + */ + void reportAsynchronousEvent(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index bf314d0192301..112a424c6729d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -44,7 +44,6 @@ import eu.stratosphere.nephele.taskmanager.bufferprovider.GlobalBufferPool; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner; -import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; @@ -66,8 +65,6 @@ public final class ByteBufferedChannelManager implements TransferEnvelopeDispatc private final Map localBufferPoolOwner = new ConcurrentHashMap(); - private final Map tasksWithUndecidedCheckpoints = new ConcurrentHashMap(); - private final NetworkConnectionManager networkConnectionManager; private final ChannelLookupProtocol channelLookupService; @@ -130,7 +127,7 @@ public void register(final Task task, final Set activeOutputChannels) final Environment environment = task.getEnvironment(); - final TaskContext taskContext = task.createTaskContext(this, this.tasksWithUndecidedCheckpoints, + final TaskContext taskContext = task.createTaskContext(this, this.localBufferPoolOwner.remove(task.getVertexID())); final Set outputGateIDs = environment.getOutputGateIDs(); @@ -700,4 +697,15 @@ public void invalidateLookupCacheEntries(final Set channelIDs) { this.receiverCache.remove(it.next()); } } + + public void reportAsynchronousEvent(final ExecutionVertexID vertexID) { + + final LocalBufferPoolOwner lbpo = this.localBufferPoolOwner.get(vertexID); + if (lbpo == null) { + System.out.println("Cannot find local buffer pool owner for " + vertexID); + return; + } + + lbpo.reportAsynchronousEvent(); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index bddcf0726583b..0db44af4f0735 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -16,13 +16,13 @@ package eu.stratosphere.nephele.taskmanager.runtime; import java.util.Iterator; -import java.util.Map; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.stratosphere.nephele.checkpointing.CheckpointDecisionRequester; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.ExecutionListener; @@ -68,6 +68,12 @@ public final class RuntimeTask implements Task, ExecutionObserver { */ private volatile ExecutionState executionState = ExecutionState.STARTING; + /** + * If the task creates a checkpoint at runtime, a checkpoint decision can be asynchronously requested through this + * interface. + */ + private volatile CheckpointDecisionRequester checkpointDecisionRequester = null; + private Queue registeredListeners = new ConcurrentLinkedQueue(); // DW: Start of temporary code @@ -414,14 +420,13 @@ public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) { */ @Override public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher, - final Map tasksWithUndecidedCheckpoints, final LocalBufferPoolOwner previousBufferPoolOwner) { if (previousBufferPoolOwner != null) { throw new IllegalStateException("Vertex " + this.vertexID + " has a previous buffer pool owner"); } - return new RuntimeTaskContext(this, transferEnvelopeDispatcher, tasksWithUndecidedCheckpoints); + return new RuntimeTaskContext(this, transferEnvelopeDispatcher); } /** @@ -449,6 +454,35 @@ public void reportPACTDataStatistics(final long numberOfConsumedBytes, final lon return (this.pactInputOutputRatioSum / (double) this.numberOfPactInputOutputRatioEntries); } + // DW: End of temporary code + /** + * Registers a checkpoint decision requester object with this task. + * + * @param checkpointDecisionRequester + * the checkpoint decision requester object to register + */ + void registerCheckpointDecisionRequester(final CheckpointDecisionRequester checkpointDecisionRequester) { + this.checkpointDecisionRequester = checkpointDecisionRequester; + } + + /** + * Requests a checkpoint decision from the task. + * + * @return true if the operation was successful, false if the task has not yet created a + * checkpoint + */ + public boolean requestCheckpointDecision() { + + if (this.checkpointDecisionRequester == null) { + return false; + } + + LOG.info("Requesting checkpoint decision for task " + this.environment.getTaskNameWithIndex()); + + this.checkpointDecisionRequester.requestCheckpointDecision(); + + return true; + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 1545ff5ac33cc..79ded67e4920a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -31,7 +31,6 @@ import eu.stratosphere.nephele.checkpointing.EphemeralCheckpoint; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.execution.RuntimeEnvironment; -import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.AbstractID; import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.InputGate; @@ -75,8 +74,7 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve private long startTime; - RuntimeTaskContext(final RuntimeTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, - final Map tasksWithUndecidedCheckpoints) { + RuntimeTaskContext(final RuntimeTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher) { this.localBufferPool = new LocalBufferPool(1, false, this); this.task = task; @@ -96,9 +94,7 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve this.numberOfOutputChannels = nooc; this.ephemeralCheckpoint = new EphemeralCheckpoint(task, ephemeral); - if (ephemeral) { - tasksWithUndecidedCheckpoints.put(task.getVertexID(), this); - } + this.task.registerCheckpointDecisionRequester(this.ephemeralCheckpoint); this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.runtimeDispatcher = new RuntimeDispatcher(transferEnvelopeDispatcher); From 50e4594c9dc254913d1acd5337f64fb517e8410e Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 29 Feb 2012 21:51:19 +0100 Subject: [PATCH 294/310] Added method call to make checkpoint decision as a response to an asynchronous event --- .../nephele/taskmanager/runtime/RuntimeTaskContext.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 79ded67e4920a..44680bacf2bcb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -311,6 +311,8 @@ public void reportAsynchronousEvent() { @Override public void asynchronousEventOccurred() throws IOException, InterruptedException { + // Trigger checkpoint decision here + reportExhaustionOfMemoryBuffers(); } /** From 5b4e8f7fb1074dfd54e7103ca6c5e2c59823532f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 1 Mar 2012 16:28:40 +0100 Subject: [PATCH 295/310] Enabled lazy deployment by default --- .../scheduler/queue/QueueScheduler.java | 2 +- .../nephele/jobmanager/JobManager.java | 13 ++- .../scheduler/AbstractExecutionListener.java | 14 ++- .../scheduler/AbstractScheduler.java | 106 +++++++++++++----- .../jobmanager/scheduler/RecoveryLogic.java | 13 ++- .../scheduler/local/LocalScheduler.java | 8 +- 6 files changed, 112 insertions(+), 44 deletions(-) diff --git a/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java b/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java index db0fff7dfa02b..53e82708b8cb1 100644 --- a/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java +++ b/nephele/nephele-queuescheduler/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueScheduler.java @@ -227,6 +227,6 @@ public void nextExecutionStageEntered(final JobID jobID, final ExecutionStage ex } // Deploy the assigned vertices - deployAssignedVertices(executionStage.getExecutionGraph()); + deployAssignedInputVertices(executionStage.getExecutionGraph()); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 724301836de4c..059da106b1ffa 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -757,7 +757,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio if (assignedInstance == null) { LOG.error("Cannot resolve lookup: vertex found for channel ID " + connectedChannelID + " but no instance assigned"); - //LOG.info("Created receiverNotReady for " + connectedVertex + " 1"); + // LOG.info("Created receiverNotReady for " + connectedVertex + " 1"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } @@ -769,7 +769,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.REPLAYING && executionState != ExecutionState.FINISHING) { - //LOG.info("Created receiverNotReady for " + connectedVertex + " in state " + executionState + " 2"); + // LOG.info("Created receiverNotReady for " + connectedVertex + " in state " + executionState + " 2"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } @@ -801,7 +801,12 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.REPLAYING && executionState != ExecutionState.FINISHING && executionState != ExecutionState.FINISHED) { - //LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3"); + + if (executionState == ExecutionState.ASSIGNED) { + this.scheduler.deployAssignedVertices(targetVertex); + } + + // LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } @@ -810,7 +815,7 @@ public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectio LOG.error("Cannot resolve lookup: vertex found for channel ID " + outputChannel.getConnectedChannelID() + " but no instance assigned"); - //LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4"); + // LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4"); return ConnectionInfoLookupResponse.createReceiverNotReady(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java index 9b078cd1a7015..9cba293547c58 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractExecutionListener.java @@ -15,6 +15,9 @@ package eu.stratosphere.nephele.jobmanager.scheduler; +import java.util.HashSet; +import java.util.Set; + import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.executiongraph.ExecutionGraph; @@ -78,7 +81,7 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver pipelineToBeDeployed.setAllocatedResource(this.executionVertex.getAllocatedResource()); pipelineToBeDeployed.updateExecutionState(ExecutionState.ASSIGNED); - this.scheduler.deployAssignedVertices(eg); + this.scheduler.deployAssignedVertices(groupMember); return; } } @@ -92,7 +95,7 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver this.executionVertex.updateExecutionState(ExecutionState.ASSIGNED, "Restart as part of recovery"); // Run through the deployment procedure - this.scheduler.deployAssignedVertices(eg); + this.scheduler.deployAssignedVertices(this.executionVertex); return; } } @@ -108,11 +111,14 @@ public void executionStateChanged(final JobID jobID, final ExecutionVertexID ver if (newExecutionState == ExecutionState.FAILED) { if (this.executionVertex.decrementRetriesLeftAndCheck()) { - if (RecoveryLogic.recover(this.executionVertex, this.scheduler.getVerticesToBeRestarted())) { + final Set assignedVertices = new HashSet(); + + if (RecoveryLogic.recover(this.executionVertex, this.scheduler.getVerticesToBeRestarted(), + assignedVertices)) { if (RecoveryLogic.hasInstanceAssigned(this.executionVertex)) { // Run through the deployment procedure - this.scheduler.deployAssignedVertices(eg); + this.scheduler.deployAssignedVertices(assignedVertices); } } else { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index db66461e240d5..cd16776098f15 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -16,6 +16,7 @@ package eu.stratosphere.nephele.jobmanager.scheduler; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -28,7 +29,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.util.StringUtils; -import eu.stratosphere.nephele.configuration.GlobalConfiguration; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; @@ -71,16 +71,6 @@ public abstract class AbstractScheduler implements InstanceListener { */ protected static final Log LOG = LogFactory.getLog(AbstractScheduler.class); - /** - * The configuration key to check whether task merging is allowed. - */ - private static final String ALLOW_TASK_MERGING_KEY = "scheduler.queue.allowTaskMerging"; - - /** - * The default setting for task merging. - */ - private static final boolean DEFAULT_ALLOW_TASK_MERGING = false; - /** * The instance manager assigned to this scheduler. */ @@ -91,11 +81,6 @@ public abstract class AbstractScheduler implements InstanceListener { */ private final DeploymentManager deploymentManager; - /** - * Stores whether task merging is allowed. - */ - private final boolean allowTaskMerging; - /** * Stores the vertices to be restarted once they have switched to the CANCELED state. */ @@ -113,12 +98,7 @@ protected AbstractScheduler(final DeploymentManager deploymentManager, final Ins this.deploymentManager = deploymentManager; this.instanceManager = instanceManager; - this.allowTaskMerging = GlobalConfiguration.getBoolean(ALLOW_TASK_MERGING_KEY, - DEFAULT_ALLOW_TASK_MERGING); - this.instanceManager.setInstanceListener(this); - - LOG.info("initialized scheduler with task merging " + (this.allowTaskMerging ? "enabled" : "disabled")); } /** @@ -240,7 +220,7 @@ void findVerticesToBeDeployed(final ExecutionVertex vertex, deployTarget = false; break; case NETWORK: - deployTarget = !this.allowTaskMerging; + deployTarget = false; break; case INMEMORY: deployTarget = true; @@ -263,13 +243,80 @@ void findVerticesToBeDeployed(final ExecutionVertex vertex, } /** - * Collects all execution vertices with the state ASSIGNED from the current execution stage and deploys them on the - * assigned {@link AllocatedResource} objects. + * Collects all execution vertices with the state ASSIGNED starting from the given start vertex and + * deploys them on the assigned {@link AllocatedResource} objects. + * + * @param startVertex + * the execution vertex to start the deployment from + */ + public void deployAssignedVertices(final ExecutionVertex startVertex) { + + final JobID jobID = startVertex.getExecutionGraph().getJobID(); + + final Map> verticesToBeDeployed = new HashMap>(); + final Set alreadyVisited = new HashSet(); + + findVerticesToBeDeployed(startVertex, verticesToBeDeployed, alreadyVisited); + + if (!verticesToBeDeployed.isEmpty()) { + + final Iterator>> it2 = verticesToBeDeployed + .entrySet() + .iterator(); + + while (it2.hasNext()) { + + final Map.Entry> entry = it2.next(); + this.deploymentManager.deploy(jobID, entry.getKey(), entry.getValue()); + } + } + } + + /** + * Collects all execution vertices with the state ASSIGNED starting from the given collection of start vertices and + * deploys them on the assigned {@link AllocatedResource} objects. + * + * @param startVertices + * the collection of execution vertices to start the deployment from + */ + public void deployAssignedVertices(final Collection startVertices) { + + JobID jobID = null; + + final Map> verticesToBeDeployed = new HashMap>(); + final Set alreadyVisited = new HashSet(); + + for (final ExecutionVertex startVertex : startVertices) { + + if (jobID == null) { + jobID = startVertex.getExecutionGraph().getJobID(); + } + + findVerticesToBeDeployed(startVertex, verticesToBeDeployed, alreadyVisited); + } + + if (!verticesToBeDeployed.isEmpty()) { + + final Iterator>> it2 = verticesToBeDeployed + .entrySet() + .iterator(); + + while (it2.hasNext()) { + + final Map.Entry> entry = it2.next(); + this.deploymentManager.deploy(jobID, entry.getKey(), entry.getValue()); + } + } + } + + /** + * Collects all execution vertices with the state ASSIGNED starting from the input vertices of the current execution + * stage and deploys them on the assigned {@link AllocatedResource} objects. * * @param executionGraph * the execution graph to collect the vertices from */ - public void deployAssignedVertices(final ExecutionGraph executionGraph) { + public void deployAssignedInputVertices(final ExecutionGraph executionGraph) { final Map> verticesToBeDeployed = new HashMap>(); final ExecutionStage executionStage = executionGraph.getCurrentExecutionStage(); @@ -397,8 +444,7 @@ public void resourcesAllocated(final JobID jobID, final List } // Deploy the assigned vertices - deployAssignedVertices(eg); - + deployAssignedInputVertices(eg); } /** @@ -472,7 +518,7 @@ protected void replayCheckpointsFromPreviousStage(final ExecutionGraph execution vertex.updateExecutionState(ExecutionState.ASSIGNED); } - deployAssignedVertices(executionGraph); + deployAssignedInputVertices(executionGraph); } /** @@ -551,6 +597,7 @@ public void allocatedResourcesDied(final JobID jobID, final List verticesToBeRestarted) { + final Map verticesToBeRestarted, + final Set assignedVertices) { // Perform initial sanity check if (failedVertex.getExecutionState() != ExecutionState.FAILED) { @@ -91,6 +92,9 @@ public static boolean recover(final ExecutionVertex failedVertex, if (vertex.compareAndUpdateExecutionState(ExecutionState.FINISHED, getStateToUpdate(vertex))) { LOG.info("Vertex " + vertex + " has already finished and will not be canceled"); + if (vertex.getExecutionState() == ExecutionState.ASSIGNED) { + assignedVertices.add(vertex); + } continue; } @@ -121,11 +125,16 @@ public static boolean recover(final ExecutionVertex failedVertex, while (checkpointIterator.hasNext()) { - checkpointIterator.next().updateExecutionState(ExecutionState.ASSIGNED); + final ExecutionVertex checkpoint = checkpointIterator.next(); + checkpoint.updateExecutionState(ExecutionState.ASSIGNED); + assignedVertices.add(checkpoint); } // Restart failed vertex failedVertex.updateExecutionState(getStateToUpdate(failedVertex)); + if (failedVertex.getExecutionState() == ExecutionState.ASSIGNED) { + assignedVertices.add(failedVertex); + } } LOG.info("Recovery FINISHED at " + System.currentTimeMillis()); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java index 95dba03d526f6..3af2928b9ffe7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/local/LocalScheduler.java @@ -62,7 +62,7 @@ public LocalScheduler(final DeploymentManager deploymentManager, final InstanceM void removeJobFromSchedule(final ExecutionGraph executionGraphToRemove) { boolean removedFromQueue = false; - + synchronized (this.jobQueue) { final Iterator it = this.jobQueue.iterator(); @@ -83,8 +83,8 @@ void removeJobFromSchedule(final ExecutionGraph executionGraphToRemove) { LOG.error("Cannot find job " + executionGraphToRemove.getJobName() + " (" + executionGraphToRemove.getJobID() + ") to remove"); } - - //TODO: Remove vertices from restart map + + // TODO: Remove vertices from restart map } /** @@ -218,7 +218,7 @@ public void nextExecutionStageEntered(final JobID jobID, final ExecutionStage ex } // Deploy the assigned vertices - deployAssignedVertices(executionStage.getExecutionGraph()); + deployAssignedInputVertices(executionStage.getExecutionGraph()); // Initialize the replay of the previous stage's checkpoints replayCheckpointsFromPreviousStage(executionStage.getExecutionGraph()); From 4bac7a54370f7dff313388e0369058402a44d180 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Mon, 5 Mar 2012 19:51:17 +0100 Subject: [PATCH 296/310] Removed temporary class --- .../eu/stratosphere/nephele/execution/Mapper.java | 14 -------------- 1 file changed, 14 deletions(-) delete mode 100644 nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java deleted file mode 100644 index fa602362e0079..0000000000000 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/execution/Mapper.java +++ /dev/null @@ -1,14 +0,0 @@ -package eu.stratosphere.nephele.execution; - -import java.util.Queue; - -import eu.stratosphere.nephele.types.Record; - -public interface Mapper { - - void map(I input) throws Exception; - - Queue getOutputCollector(); - - void close(); -} From 7745bdfb25609e93fe91839990ab0cede6e60632 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 6 Mar 2012 13:22:08 +0100 Subject: [PATCH 297/310] Refactored envelope forwarding API --- .../nephele/annotations/TaskAnnotation.java | 15 ++ .../nephele/fs/FileChannelWrapper.java | 136 ++++++++++-------- .../checkpointing/EphemeralCheckpoint.java | 31 +--- .../EphemeralCheckpointForwarder.java | 61 ++++++++ .../checkpointing/ReplayOutputBroker.java | 55 ++----- .../ReplayOutputChannelContext.java | 6 +- .../ReplayOutputGateContext.java | 31 ++-- .../checkpointing/ReplayTaskContext.java | 9 +- .../execution/ExecutionStateTransition.java | 17 ++- .../AbstractOutputChannelContext.java | 16 +-- .../AbstractOutputChannelForwarder.java | 113 +++++++++++++++ .../bytebuffered/IncomingEventQueue.java | 31 ---- .../bytebuffered/OutputChannelForwarder.java | 17 --- .../OutputChannelForwardingChain.java | 73 +++++----- .../runtime/EnvelopeConsumptionLog.java | 15 ++ .../runtime/ForwardingBarrier.java | 39 +++-- .../runtime/RuntimeDispatcher.java | 52 +++---- .../runtime/RuntimeOutputChannelBroker.java | 67 ++++----- .../runtime/RuntimeOutputChannelContext.java | 5 +- .../runtime/RuntimeOutputGateContext.java | 31 ++-- .../runtime/RuntimeTaskContext.java | 9 +- .../taskmanager/runtime/SpillingBarrier.java | 51 ++++--- .../transferenvelope/SpillingQueueThread.java | 15 ++ 23 files changed, 503 insertions(+), 392 deletions(-) create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpointForwarder.java create mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingEventQueue.java delete mode 100644 nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java index 0fabfc830d339..d77b63440e821 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.annotations; import java.lang.annotation.ElementType; diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java index 78177bf86c332..1554bee3dfd4c 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/fs/FileChannelWrapper.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.fs; import java.io.IOException; @@ -35,46 +50,49 @@ public FileChannelWrapper(final FileSystem fs, final Path checkpointFile, final this.replication = replication; } + /** + * {@inheritDoc} + */ @Override - public void force(boolean metaData) throws IOException { - // TODO Auto-generated method stub - System.out.println("force called"); + public void force(final boolean metaData) throws IOException { + + throw new UnsupportedOperationException("Method force is not implemented"); } + /** + * {@inheritDoc} + */ @Override - public FileLock lock(long position, long size, boolean shared) throws IOException { - // TODO Auto-generated method stub - - System.out.println("lock called"); + public FileLock lock(final long position, final long size, final boolean shared) throws IOException { - return null; + throw new UnsupportedOperationException("Method lock is not implemented"); } + /** + * {@inheritDoc} + */ @Override - public MappedByteBuffer map(MapMode mode, long position, long size) throws IOException { - // TODO Auto-generated method stub - - System.out.println("map called"); + public MappedByteBuffer map(final MapMode mode, final long position, final long size) throws IOException { - return null; + throw new UnsupportedOperationException("Method map is not implemented"); } + /** + * {@inheritDoc} + */ @Override public long position() throws IOException { - // TODO Auto-generated method stub - System.out.println("position called"); - - return 0; + throw new UnsupportedOperationException("Method position is not implemented"); } + /** + * {@inheritDoc} + */ @Override - public FileChannel position(long newPosition) throws IOException { - // TODO Auto-generated method stub - - System.out.println("position2 called"); + public FileChannel position(final long newPosition) throws IOException { - return null; + throw new UnsupportedOperationException("Method position is not implemented"); } /** @@ -83,7 +101,7 @@ public FileChannel position(long newPosition) throws IOException { @Override public int read(final ByteBuffer dst) throws IOException { - return read(dst, this.nextExpectedReadPosition); + throw new UnsupportedOperationException("Method read is not implemented"); } /** @@ -114,65 +132,65 @@ public int read(final ByteBuffer dst, final long position) throws IOException { return bytesRead; } + /** + * {@inheritDoc} + */ @Override - public long read(ByteBuffer[] dsts, int offset, int length) throws IOException { - // TODO Auto-generated method stub - - System.out.println("read3 called"); + public long read(final ByteBuffer[] dsts, final int offset, final int length) throws IOException { - return 0; + throw new UnsupportedOperationException("Method read is not implemented"); } + /** + * {@inheritDoc} + */ @Override public long size() throws IOException { - // TODO Auto-generated method stub - System.out.println("size called"); - - return 0; + throw new UnsupportedOperationException("Method size is not implemented"); } + /** + * {@inheritDoc} + */ @Override - public long transferFrom(ReadableByteChannel src, long position, long count) throws IOException { - - System.out.println("transferFrom called"); + public long transferFrom(final ReadableByteChannel src, final long position, final long count) throws IOException { - // TODO Auto-generated method stub - return 0; + throw new UnsupportedOperationException("Method transferFrom is not implemented"); } + /** + * {@inheritDoc} + */ @Override - public long transferTo(long position, long count, WritableByteChannel target) throws IOException { - - System.out.println("transferTo called"); + public long transferTo(final long position, final long count, final WritableByteChannel target) throws IOException { - // TODO Auto-generated method stub - return 0; + throw new UnsupportedOperationException("Method transferTo is not implemented"); } + /** + * {@inheritDoc} + */ @Override - public FileChannel truncate(long size) throws IOException { + public FileChannel truncate(final long size) throws IOException { - System.out.println("truncate called"); - - // TODO Auto-generated method stub - return null; + throw new UnsupportedOperationException("Method truncate is not implemented"); } + /** + * {@inheritDoc} + */ @Override - public FileLock tryLock(long position, long size, boolean shared) throws IOException { - - System.out.println("tryLock called"); + public FileLock tryLock(final long position, final long size, final boolean shared) throws IOException { - // TODO Auto-generated method stub - return null; + throw new UnsupportedOperationException("Method tryLock is not implemented"); } /** * {@inheritDoc} */ @Override - public int write(ByteBuffer src) throws IOException { + public int write(final ByteBuffer src) throws IOException { return write(src, this.nextExpectedWritePosition); } @@ -204,13 +222,13 @@ public int write(final ByteBuffer src, final long position) throws IOException { return totalBytesWritten; } + /** + * {@inheritDoc} + */ @Override - public long write(ByteBuffer[] srcs, int offset, int length) throws IOException { - - System.out.println("write3 called"); + public long write(final ByteBuffer[] srcs, final int offset, final int length) throws IOException { - // TODO Auto-generated method stub - return 0; + throw new UnsupportedOperationException("Method write is not implemented"); } private FSDataOutputStream getOutputStream() throws IOException { @@ -238,7 +256,7 @@ private FSDataInputStream getInputStream() throws IOException { @Override protected void implCloseChannel() throws IOException { - if(this.outputStream != null) { + if (this.outputStream != null) { this.outputStream.close(); this.outputStream = null; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index fe9b9db529292..f58e7a956c24d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -22,10 +22,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; @@ -44,7 +42,7 @@ * * @author warneke */ -public class EphemeralCheckpoint implements OutputChannelForwarder, CheckpointDecisionRequester { +public class EphemeralCheckpoint implements CheckpointDecisionRequester { /** * The log object used to report problems. @@ -132,10 +130,6 @@ public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { } } - /** - * {@inheritDoc} - */ - @Override public void destroy() { while (!this.queuedEnvelopes.isEmpty()) { @@ -186,14 +180,10 @@ public void setCheckpointDecisionSynchronously(final boolean checkpointDecision) } } - /** - * {@inheritDoc} - */ - @Override - public boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + public void forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { if (this.checkpointingDecision == CheckpointingDecisionState.NO_CHECKPOINTING) { - return true; + return; } final TransferEnvelope dup = transferEnvelope.duplicate(); @@ -209,8 +199,6 @@ public boolean forward(final TransferEnvelope transferEnvelope) throws IOExcepti } else { this.writeThread.write(dup); } - - return true; } public boolean isUndecided() { @@ -218,10 +206,6 @@ public boolean isUndecided() { return (this.checkpointingDecision == CheckpointingDecisionState.UNDECIDED); } - /** - * {@inheritDoc} - */ - @Override public boolean hasDataLeft() throws IOException, InterruptedException { if (isUndecided()) { @@ -245,15 +229,6 @@ public boolean hasDataLeft() throws IOException, InterruptedException { return false; } - /** - * {@inheritDoc} - */ - @Override - public void processEvent(final AbstractEvent event) { - // TODO Auto-generated method stub - - } - /** * {@inheritDoc} */ diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpointForwarder.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpointForwarder.java new file mode 100644 index 0000000000000..d982c75616343 --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpointForwarder.java @@ -0,0 +1,61 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; + +import java.io.IOException; + +import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +public final class EphemeralCheckpointForwarder extends AbstractOutputChannelForwarder { + + private final EphemeralCheckpoint ephemeralCheckpoint; + + public EphemeralCheckpointForwarder(final EphemeralCheckpoint ephemeralCheckpoint, + final AbstractOutputChannelForwarder next) { + super(next); + + if (next == null) { + throw new IllegalArgumentException("Argument next must not be null"); + } + + this.ephemeralCheckpoint = ephemeralCheckpoint; + } + + /** + * {@inheritDoc} + */ + @Override + public void push(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + + this.ephemeralCheckpoint.forward(transferEnvelope); + + getNext().push(transferEnvelope); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasDataLeft() throws IOException, InterruptedException { + + if (this.ephemeralCheckpoint.hasDataLeft()) { + return true; + } + + return getNext().hasDataLeft(); + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java index d39728876a9ad..9aa1f6b794264 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java @@ -9,13 +9,12 @@ import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; -import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; -import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; +import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -final class ReplayOutputBroker implements OutputChannelForwarder, BufferProvider { +final class ReplayOutputBroker extends AbstractOutputChannelForwarder implements BufferProvider { /** * The logger to report information and problems. @@ -24,41 +23,18 @@ final class ReplayOutputBroker implements OutputChannelForwarder, BufferProvider private final BufferProvider bufferProvider; - private final OutputChannelForwardingChain forwardingChain; - - private final IncomingEventQueue incomingEventQueue; + private OutputChannelForwardingChain forwardingChain; private int nextEnvelopeToSend = 0; - ReplayOutputBroker(final BufferProvider bufferProvider, final OutputChannelForwardingChain forwardingChain, - final IncomingEventQueue incomingEventQueue) { + ReplayOutputBroker(final BufferProvider bufferProvider, final AbstractOutputChannelForwarder next) { + super(next); this.bufferProvider = bufferProvider; - this.forwardingChain = forwardingChain; - this.incomingEventQueue = incomingEventQueue; - } - - /** - * {@inheritDoc} - */ - @Override - public boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - - // Nothing to do here - - return true; } - /** - * {@inheritDoc} - */ - @Override - public boolean hasDataLeft() { - - // A replay task will not wait for a close acknowledgement as it may have been sent to the corresponding runtime - // task before. - - return false; + public void setForwardingChain(final OutputChannelForwardingChain forwardingChain) { + this.forwardingChain = forwardingChain; } /** @@ -77,17 +53,19 @@ public void processEvent(final AbstractEvent event) { } else { LOG.warn("Received unknown event: " + event); } + + getNext().processEvent(event); } void outputEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - this.incomingEventQueue.processQueuedEvents(); + this.forwardingChain.processQueuedEvents(); if (transferEnvelope.getSequenceNumber() == this.nextEnvelopeToSend) { ++this.nextEnvelopeToSend; } - this.forwardingChain.forwardEnvelope(transferEnvelope); + this.forwardingChain.pushEnvelope(transferEnvelope); } int getNextEnvelopeToSend() { @@ -98,7 +76,7 @@ int getNextEnvelopeToSend() { boolean hasFinished() throws IOException, InterruptedException { // Check for events - this.incomingEventQueue.processQueuedEvents(); + this.forwardingChain.processQueuedEvents(); return (!this.forwardingChain.anyForwarderHasDataLeft()); } @@ -147,13 +125,4 @@ public void reportAsynchronousEvent() { this.bufferProvider.reportAsynchronousEvent(); } - - /** - * {@inheritDoc} - */ - @Override - public void destroy() { - - // Nothing to do here - } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java index 6f03021013ae8..b777e89f86b46 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java @@ -4,7 +4,6 @@ import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext; -import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; @@ -18,9 +17,8 @@ public final class ReplayOutputChannelContext extends AbstractOutputChannelConte private final OutputChannelContext encapsulatedContext; ReplayOutputChannelContext(final JobID jobID, final ChannelID channelID, - final OutputChannelForwardingChain forwardingChain, final IncomingEventQueue incomingEventQueue, - final OutputChannelContext encapsulatedContext) { - super(forwardingChain, incomingEventQueue); + final OutputChannelForwardingChain forwardingChain, final OutputChannelContext encapsulatedContext) { + super(forwardingChain); this.jobID = jobID; this.channelID = channelID; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java index 70a042f897929..6c19c8e897f0a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java @@ -2,13 +2,12 @@ import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext; -import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent; import eu.stratosphere.nephele.taskmanager.runtime.ForwardingBarrier; +import eu.stratosphere.nephele.taskmanager.runtime.RuntimeDispatcher; import eu.stratosphere.nephele.taskmanager.runtime.SpillingBarrier; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; @@ -22,6 +21,9 @@ final class ReplayOutputGateContext extends AbstractReplayGateContext implements this.taskContext = taskContext; } + /** + * {@inheritDoc} + */ @Override public OutputChannelContext createOutputChannelContext(ChannelID channelID, OutputChannelContext previousContext, boolean isReceiverRunning, boolean mergeSpillBuffers) { @@ -31,20 +33,23 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp } // Construct new forwarding chain for the replay output channel context - final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(); - final IncomingEventQueue incomingEventQueue = AbstractOutputChannelContext - .createIncomingEventQueue(forwardingChain); - final ReplayOutputBroker outputBroker = new ReplayOutputBroker(this.taskContext, forwardingChain, - incomingEventQueue); - forwardingChain.addForwarder(outputBroker); - forwardingChain.addForwarder(new ForwardingBarrier(channelID)); - forwardingChain.addForwarder(new SpillingBarrier(isReceiverRunning, mergeSpillBuffers)); - forwardingChain.addForwarder(this.taskContext.getRuntimeDispatcher()); + final RuntimeDispatcher runtimeDispatcher = new RuntimeDispatcher( + this.taskContext.getTransferEnvelopeDispatcher()); + final SpillingBarrier spillingBarrier = new SpillingBarrier(isReceiverRunning, mergeSpillBuffers, + runtimeDispatcher); + final ForwardingBarrier forwardingBarrier = new ForwardingBarrier(channelID, spillingBarrier); + final ReplayOutputBroker outputChannelBroker = new ReplayOutputBroker(this.taskContext, forwardingBarrier); + + final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(outputChannelBroker, + runtimeDispatcher); + + // Set forwarding chain for broker + outputChannelBroker.setForwardingChain(forwardingChain); // Register output broker - this.taskContext.registerReplayOutputBroker(channelID, outputBroker); + this.taskContext.registerReplayOutputBroker(channelID, outputChannelBroker); - return new ReplayOutputChannelContext(null, channelID, forwardingChain, incomingEventQueue, previousContext); + return new ReplayOutputChannelContext(null, channelID, forwardingChain, previousContext); } private static void activateForwardingBarrier(final OutputChannelContext previousContext) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java index 08bfbb9d1cf02..ebb759c8fa8aa 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java @@ -13,7 +13,6 @@ import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext; -import eu.stratosphere.nephele.taskmanager.runtime.RuntimeDispatcher; import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTaskContext; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; @@ -21,7 +20,7 @@ final class ReplayTaskContext implements TaskContext, BufferProvider, Asynchrono private final ReplayTask task; - private final RuntimeDispatcher runtimeDispatcher; + private final TransferEnvelopeDispatcher transferEnvelopeDispatcher; private final int numberOfChannels; @@ -30,7 +29,7 @@ final class ReplayTaskContext implements TaskContext, BufferProvider, Asynchrono ReplayTaskContext(final ReplayTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher, final LocalBufferPoolOwner previousBufferPoolOwner, final int numberOfChannels) { this.task = task; - this.runtimeDispatcher = new RuntimeDispatcher(transferEnvelopeDispatcher); + this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; if (previousBufferPoolOwner == null) { this.localBufferPool = new LocalBufferPool(1, false, this); } else { @@ -67,9 +66,9 @@ void registerReplayOutputBroker(final ChannelID channelID, final ReplayOutputBro this.task.registerReplayOutputBroker(channelID, outputBroker); } - RuntimeDispatcher getRuntimeDispatcher() { + TransferEnvelopeDispatcher getTransferEnvelopeDispatcher() { - return this.runtimeDispatcher; + return this.transferEnvelopeDispatcher; } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java index 494cbeb16d193..125fad0d11377 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.execution; import org.apache.commons.logging.Log; @@ -40,7 +55,7 @@ public static void checkTransition(final boolean jobManager, final String taskNa final ExecutionState newState) { LOG.info((jobManager ? "JM: " : "TM: ") + "ExecutionState set from " + oldState + " to " + newState - + " for task " + taskName+ " at " + System.currentTimeMillis()); + + " for task " + taskName + " at " + System.currentTimeMillis()); boolean unexpectedStateChange = true; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java index 65adcc24790bd..c47746a58a0b4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java @@ -7,26 +7,14 @@ public abstract class AbstractOutputChannelContext implements OutputChannelContext { - /** - * Stores incoming events for this output channel. - */ - private final IncomingEventQueue incomingEventQueue; - /** * The forwarding chain used by this output channel context. */ private final OutputChannelForwardingChain forwardingChain; - public AbstractOutputChannelContext(final OutputChannelForwardingChain forwardingChain, - final IncomingEventQueue incomingEventQueue) { + public AbstractOutputChannelContext(final OutputChannelForwardingChain forwardingChain) { this.forwardingChain = forwardingChain; - this.incomingEventQueue = incomingEventQueue; - } - - public static IncomingEventQueue createIncomingEventQueue(final OutputChannelForwardingChain forwardingChain) { - - return new IncomingEventQueue(forwardingChain); } /** @@ -41,7 +29,7 @@ public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) { final Iterator it = transferEnvelope.getEventList().iterator(); while (it.hasNext()) { - this.incomingEventQueue.offer(it.next()); + this.forwardingChain.offerEvent(it.next()); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java new file mode 100644 index 0000000000000..62221359481ec --- /dev/null +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java @@ -0,0 +1,113 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.bytebuffered; + +import java.io.IOException; + +import eu.stratosphere.nephele.event.task.AbstractEvent; +import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; +import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; + +/** + * An output channel forwarder is a component which processes a {@link TransferEnvelope} after it has been produced by + * an {@link AbstractByteBufferedOutputChannel}. The component can decide based on the transfer envelope whether to + * forward the envelope, discard it, or to store it. + * + * @author warneke + */ +public abstract class AbstractOutputChannelForwarder { + + private final AbstractOutputChannelForwarder next; + + private volatile AbstractOutputChannelForwarder prev = null; + + protected AbstractOutputChannelForwarder(final AbstractOutputChannelForwarder next) { + this.next = next; + if (this.next != null) { + this.next.prev = this; + } + } + + /** + * Called by the framework to push a produced transfer envelope towards its receiver. This method will always be + * called by the task thread itself. + * + * @param transferEnvelope + * the transfer envelope to be processed + * @throws IOException + * thrown if an I/O error occurs while processing the transfer envelope + * @throws InterruptedException + * thrown if the task thread was interrupted while processing the transfer envelope + */ + public void push(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + + if (this.next != null) { + this.next.push(transferEnvelope); + } + } + + public TransferEnvelope pull() { + + if (this.prev != null) { + return this.prev.pull(); + } + + return null; + } + + public boolean hasDataLeft() throws IOException, InterruptedException { + + if (this.next != null) { + this.next.hasDataLeft(); + } + + return false; + } + + public void processEvent(final AbstractEvent event) { + + if (this.next != null) { + this.next.processEvent(event); + } + } + + public void destroy() { + + if (this.next != null) { + this.next.destroy(); + } + } + + protected final void recycleTransferEnvelope(final TransferEnvelope transferEnvelope) { + + final Buffer buffer = transferEnvelope.getBuffer(); + if (buffer != null) { + buffer.recycleBuffer(); + + } + } + + protected final AbstractOutputChannelForwarder getNext() { + + return this.next; + } + + protected final AbstractOutputChannelForwarder getPrev() { + + return this.prev; + } +} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingEventQueue.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingEventQueue.java deleted file mode 100644 index c8bbb4093c32b..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingEventQueue.java +++ /dev/null @@ -1,31 +0,0 @@ -package eu.stratosphere.nephele.taskmanager.bytebuffered; - -import java.util.Queue; -import java.util.concurrent.LinkedBlockingDeque; - -import eu.stratosphere.nephele.event.task.AbstractEvent; - -public final class IncomingEventQueue { - - private final Queue incomingEventQueue = new LinkedBlockingDeque(); - - private final OutputChannelForwardingChain forwardingChain; - - IncomingEventQueue(final OutputChannelForwardingChain forwardingChain) { - this.forwardingChain = forwardingChain; - } - - public void processQueuedEvents() { - - AbstractEvent event = this.incomingEventQueue.poll(); - while (event != null) { - - this.forwardingChain.processEvent(event); - event = this.incomingEventQueue.poll(); - } - } - - void offer(final AbstractEvent event) { - this.incomingEventQueue.offer(event); - } -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java deleted file mode 100644 index dc60093be6f6a..0000000000000 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwarder.java +++ /dev/null @@ -1,17 +0,0 @@ -package eu.stratosphere.nephele.taskmanager.bytebuffered; - -import java.io.IOException; - -import eu.stratosphere.nephele.event.task.AbstractEvent; -import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; - -public interface OutputChannelForwarder { - - boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException; - - boolean hasDataLeft() throws IOException, InterruptedException; - - void processEvent(AbstractEvent event); - - void destroy(); -} diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java index a1e6797238fd4..edd71213a2441 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java @@ -16,70 +16,71 @@ package eu.stratosphere.nephele.taskmanager.bytebuffered; import java.io.IOException; -import java.util.Iterator; -import java.util.concurrent.CopyOnWriteArrayList; +import java.util.Queue; +import java.util.concurrent.LinkedBlockingDeque; import eu.stratosphere.nephele.event.task.AbstractEvent; -import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; public final class OutputChannelForwardingChain { - private final CopyOnWriteArrayList forwardingChain = new CopyOnWriteArrayList(); + private final Queue incomingEventQueue = new LinkedBlockingDeque(); - public void addForwarder(final OutputChannelForwarder forwarder) { + private final AbstractOutputChannelForwarder first; - this.forwardingChain.add(forwarder); - } - - public void forwardEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + private final AbstractOutputChannelForwarder last; - final Iterator it = this.forwardingChain.iterator(); - while (it.hasNext()) { + public OutputChannelForwardingChain(final AbstractOutputChannelForwarder first, + final AbstractOutputChannelForwarder last) { - if (!it.next().forward(transferEnvelope)) { - recycleEnvelope(transferEnvelope); - break; - } + if (first == null) { + throw new IllegalArgumentException("Argument first must not be null"); + } + if (last == null) { + throw new IllegalArgumentException("Argument last must not be null"); } + + this.first = first; + this.last = last; } - public void processEvent(final AbstractEvent event) { + public void pushEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - final Iterator it = this.forwardingChain.iterator(); - while (it.hasNext()) { - it.next().processEvent(event); - } + this.first.push(transferEnvelope); } - public boolean anyForwarderHasDataLeft() throws IOException, InterruptedException { + public TransferEnvelope pullEnvelope() { - final Iterator it = this.forwardingChain.iterator(); - while (it.hasNext()) { + return this.last.pull(); + } - if (it.next().hasDataLeft()) { - return true; - } - } + public void processEvent(final AbstractEvent event) { - return false; + this.first.processEvent(event); } - private void recycleEnvelope(final TransferEnvelope transferEnvelope) { + public boolean anyForwarderHasDataLeft() throws IOException, InterruptedException { - final Buffer buffer = transferEnvelope.getBuffer(); - if (buffer != null) { - buffer.recycleBuffer(); - } + return this.first.hasDataLeft(); } public void destroy() { - final Iterator it = this.forwardingChain.iterator(); - while (it.hasNext()) { - it.next().destroy(); + this.first.destroy(); + } + + public void processQueuedEvents() { + + AbstractEvent event = this.incomingEventQueue.poll(); + while (event != null) { + + this.first.processEvent(event); + event = this.incomingEventQueue.poll(); } } + void offerEvent(final AbstractEvent event) { + this.incomingEventQueue.offer(event); + } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java index ec357aa0b63ea..e5f93be7c0a85 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/EnvelopeConsumptionLog.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.runtime; import java.io.File; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java index 7fb8ef402042d..5cd22310e5e92 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java @@ -15,16 +15,18 @@ package eu.stratosphere.nephele.taskmanager.runtime; +import java.io.IOException; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; +import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -public final class ForwardingBarrier implements OutputChannelForwarder { +public final class ForwardingBarrier extends AbstractOutputChannelForwarder { private static final Log LOG = LogFactory.getLog(ForwardingBarrier.class); @@ -32,7 +34,13 @@ public final class ForwardingBarrier implements OutputChannelForwarder { private int forwardingBarrier = -1; - public ForwardingBarrier(final ChannelID outputChannelID) { + public ForwardingBarrier(final ChannelID outputChannelID, final AbstractOutputChannelForwarder next) { + super(next); + + if (next == null) { + throw new IllegalArgumentException("Argument next must not be null"); + } + this.outputChannelID = outputChannelID; } @@ -40,22 +48,14 @@ public ForwardingBarrier(final ChannelID outputChannelID) { * {@inheritDoc} */ @Override - public boolean forward(final TransferEnvelope transferEnvelope) { + public void push(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { if (transferEnvelope.getSequenceNumber() < this.forwardingBarrier) { - return false; + recycleTransferEnvelope(transferEnvelope); + return; } - return true; - } - - /** - * {@inheritDoc} - */ - @Override - public boolean hasDataLeft() { - - return false; + getNext().push(transferEnvelope); } /** @@ -73,14 +73,7 @@ public void processEvent(final AbstractEvent event) { + " for output channel " + this.outputChannelID); } } - } - - /** - * {@inheritDoc} - */ - @Override - public void destroy() { - // Nothing to do here + getNext().processEvent(event); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java index fad6435681391..8e1d70844b737 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java @@ -1,54 +1,42 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.runtime; import java.io.IOException; -import eu.stratosphere.nephele.event.task.AbstractEvent; -import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; +import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher; -public final class RuntimeDispatcher implements OutputChannelForwarder { +public final class RuntimeDispatcher extends AbstractOutputChannelForwarder { private final TransferEnvelopeDispatcher dispatcher; public RuntimeDispatcher(final TransferEnvelopeDispatcher dispatcher) { + super(null); this.dispatcher = dispatcher; } - @Override - public boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - - dispatcher.processEnvelopeFromOutputChannel(transferEnvelope); - - return true; - } - /** * {@inheritDoc} */ @Override - public boolean hasDataLeft() { + public void push(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { - return false; + this.dispatcher.processEnvelopeFromOutputChannel(transferEnvelope); } - - /** - * {@inheritDoc} - */ - @Override - public void processEvent(final AbstractEvent event) { - - // Nothing to do here - } - - /** - * {@inheritDoc} - */ - @Override - public void destroy() { - - // Nothing to do here - } - } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java index 3e5d172bb1de4..c9e970ed8a7be 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java @@ -13,13 +13,13 @@ import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedOutputChannelBroker; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; -import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; -import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; +import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; import eu.stratosphere.nephele.taskmanager.bytebuffered.ReceiverNotFoundEvent; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -final class RuntimeOutputChannelBroker implements ByteBufferedOutputChannelBroker, OutputChannelForwarder { +final class RuntimeOutputChannelBroker extends AbstractOutputChannelForwarder implements + ByteBufferedOutputChannelBroker { /** * The static object used for logging. @@ -31,20 +31,15 @@ final class RuntimeOutputChannelBroker implements ByteBufferedOutputChannelBroke */ private final AbstractByteBufferedOutputChannel byteBufferedOutputChannel; - /** - * Reference to the queue with the incoming events. - */ - private final IncomingEventQueue incomingEventQueue; - /** * The buffer provider this channel broker to obtain buffers from. */ private final BufferProvider bufferProvider; /** - * The output channel forwarder which will take care of the produced transfer envelopes. + * The forwarding chain along which the created transfer envelopes will be pushed. */ - private final OutputChannelForwardingChain forwarder; + private OutputChannelForwardingChain forwardingChain; /** * Points to the {@link TransferEnvelope} object that will be passed to the framework upon @@ -67,40 +62,37 @@ final class RuntimeOutputChannelBroker implements ByteBufferedOutputChannelBroke */ private int sequenceNumber = 0; - RuntimeOutputChannelBroker(final BufferProvider bufferProvider, final OutputChannelForwardingChain forwarder, - final IncomingEventQueue incomingEventQueue, - final AbstractByteBufferedOutputChannel byteBufferedOutputChannel) { + RuntimeOutputChannelBroker(final BufferProvider bufferProvider, + final AbstractByteBufferedOutputChannel byteBufferedOutputChannel, + final AbstractOutputChannelForwarder next) { + + super(next); + + if (next == null) { + throw new IllegalArgumentException("Argument next must not be null"); + } this.bufferProvider = bufferProvider; - this.forwarder = forwarder; - this.incomingEventQueue = incomingEventQueue; this.byteBufferedOutputChannel = byteBufferedOutputChannel; this.byteBufferedOutputChannel.setByteBufferedOutputChannelBroker(this); } - /** - * {@inheritDoc} - */ - @Override - public boolean forward(final TransferEnvelope transferEnvelope) { - - // Nothing to do here - - return true; + public void setForwardingChain(final OutputChannelForwardingChain forwardingChain) { + this.forwardingChain = forwardingChain; } /** * {@inheritDoc} */ @Override - public boolean hasDataLeft() { + public boolean hasDataLeft() throws IOException, InterruptedException { if (this.closeAcknowledgementReceived) { - return false; + return getNext().hasDataLeft(); } if ((this.lastSequenceNumberWithReceiverNotFound + 1) == this.sequenceNumber) { - return false; + return getNext().hasDataLeft(); } return true; @@ -119,6 +111,8 @@ public void processEvent(final AbstractEvent event) { } else if (event instanceof AbstractTaskEvent) { this.byteBufferedOutputChannel.processEvent(event); } + + getNext().processEvent(event); } @Override @@ -181,7 +175,7 @@ private int calculateBufferSize() { public void releaseWriteBuffers() throws IOException, InterruptedException { // Check for events - this.incomingEventQueue.processQueuedEvents(); + this.forwardingChain.processQueuedEvents(); if (this.outgoingTransferEnvelope == null) { LOG.error("Cannot find transfer envelope for channel with ID " + this.byteBufferedOutputChannel.getID()); @@ -198,7 +192,7 @@ public void releaseWriteBuffers() throws IOException, InterruptedException { final Buffer buffer = this.outgoingTransferEnvelope.getBuffer(); buffer.finishWritePhase(); - this.forwarder.forwardEnvelope(this.outgoingTransferEnvelope); + this.forwardingChain.pushEnvelope(this.outgoingTransferEnvelope); this.outgoingTransferEnvelope = null; } @@ -209,9 +203,9 @@ public void releaseWriteBuffers() throws IOException, InterruptedException { public boolean hasDataLeftToTransmit() throws IOException, InterruptedException { // Check for events - this.incomingEventQueue.processQueuedEvents(); + this.forwardingChain.processQueuedEvents(); - return this.forwarder.anyForwarderHasDataLeft(); + return this.forwardingChain.anyForwarderHasDataLeft(); } /** @@ -227,16 +221,7 @@ public void transferEventToInputChannel(final AbstractEvent event) throws IOExce final TransferEnvelope ephemeralTransferEnvelope = createNewOutgoingTransferEnvelope(); ephemeralTransferEnvelope.addEvent(event); - this.forwarder.forwardEnvelope(ephemeralTransferEnvelope); + this.forwardingChain.pushEnvelope(ephemeralTransferEnvelope); } } - - /** - * {@inheritDoc} - */ - @Override - public void destroy() { - - // Nothing to do here - } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java index 38bead44a589f..6ad5316c19a4f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java @@ -20,7 +20,6 @@ import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext; -import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; public final class RuntimeOutputChannelContext extends AbstractOutputChannelContext { @@ -28,8 +27,8 @@ public final class RuntimeOutputChannelContext extends AbstractOutputChannelCont private final AbstractByteBufferedOutputChannel byteBufferedOutputChannel; RuntimeOutputChannelContext(final AbstractByteBufferedOutputChannel byteBufferedOutputChannel, - final OutputChannelForwardingChain forwardingChain, final IncomingEventQueue incomingEventQueue) { - super(forwardingChain, incomingEventQueue); + final OutputChannelForwardingChain forwardingChain) { + super(forwardingChain); this.byteBufferedOutputChannel = byteBufferedOutputChannel; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java index 9c3726836ba94..8020a9ac1459e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java @@ -18,6 +18,7 @@ import java.io.IOException; import eu.stratosphere.nephele.checkpointing.EphemeralCheckpoint; +import eu.stratosphere.nephele.checkpointing.EphemeralCheckpointForwarder; import eu.stratosphere.nephele.io.AbstractID; import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.OutputGate; @@ -26,8 +27,6 @@ import eu.stratosphere.nephele.io.channels.ChannelID; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; -import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext; -import eu.stratosphere.nephele.taskmanager.bytebuffered.IncomingEventQueue; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; @@ -148,19 +147,27 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp AbstractByteBufferedOutputChannel outputChannel = (AbstractByteBufferedOutputChannel) channel; // Construct forwarding chain for this output channel - final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(); - final IncomingEventQueue incomingEventQueue = AbstractOutputChannelContext - .createIncomingEventQueue(forwardingChain); - forwardingChain.addForwarder(new RuntimeOutputChannelBroker(this, forwardingChain, incomingEventQueue, - outputChannel)); + final RuntimeDispatcher runtimeDispatcher = new RuntimeDispatcher( + this.taskContext.getTransferEnvelopeDispatcher()); + final SpillingBarrier spillingBarrier = new SpillingBarrier(isReceiverRunning, mergeSpillBuffers, + runtimeDispatcher); + final ForwardingBarrier forwardingBarrier = new ForwardingBarrier(channelID, spillingBarrier); final EphemeralCheckpoint checkpoint = this.taskContext.getEphemeralCheckpoint(); + RuntimeOutputChannelBroker outputChannelBroker; if (checkpoint != null) { - forwardingChain.addForwarder(checkpoint); + final EphemeralCheckpointForwarder checkpointForwarder = new EphemeralCheckpointForwarder(checkpoint, + forwardingBarrier); + outputChannelBroker = new RuntimeOutputChannelBroker(this, outputChannel, checkpointForwarder); + } else { + outputChannelBroker = new RuntimeOutputChannelBroker(this, outputChannel, forwardingBarrier); } - forwardingChain.addForwarder(new ForwardingBarrier(channelID)); - forwardingChain.addForwarder(new SpillingBarrier(isReceiverRunning, mergeSpillBuffers)); - forwardingChain.addForwarder(this.taskContext.getRuntimeDispatcher()); - return new RuntimeOutputChannelContext(outputChannel, forwardingChain, incomingEventQueue); + final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(outputChannelBroker, + runtimeDispatcher); + + // Set forwarding chain for broker + outputChannelBroker.setForwardingChain(forwardingChain); + + return new RuntimeOutputChannelContext(outputChannel, forwardingChain); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 44680bacf2bcb..8797ee8e7d62f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -66,8 +66,6 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve private final TransferEnvelopeDispatcher transferEnvelopeDispatcher; - private final RuntimeDispatcher runtimeDispatcher; - private final EphemeralCheckpoint ephemeralCheckpoint; private final EnvelopeConsumptionLog envelopeConsumptionLog; @@ -95,18 +93,15 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve this.ephemeralCheckpoint = new EphemeralCheckpoint(task, ephemeral); this.task.registerCheckpointDecisionRequester(this.ephemeralCheckpoint); - this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; - this.runtimeDispatcher = new RuntimeDispatcher(transferEnvelopeDispatcher); - this.envelopeConsumptionLog = new EnvelopeConsumptionLog(task.getVertexID(), environment); this.startTime = System.currentTimeMillis(); } - RuntimeDispatcher getRuntimeDispatcher() { + TransferEnvelopeDispatcher getTransferEnvelopeDispatcher() { - return this.runtimeDispatcher; + return this.transferEnvelopeDispatcher; } EphemeralCheckpoint getEphemeralCheckpoint() { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java index e07a37bcf02d0..b407fbd657c8b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/SpillingBarrier.java @@ -1,14 +1,29 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.runtime; import java.io.IOException; import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelActivateEvent; -import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwarder; +import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.transferenvelope.SpillingQueue; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -public final class SpillingBarrier implements OutputChannelForwarder { +public final class SpillingBarrier extends AbstractOutputChannelForwarder { /** * Queue to store outgoing transfer envelope in case the receiver of the envelopes is not yet running. @@ -20,7 +35,14 @@ public final class SpillingBarrier implements OutputChannelForwarder { */ private boolean isReceiverRunning = false; - public SpillingBarrier(final boolean isReceiverRunning, final boolean mergeSpillBuffers) { + public SpillingBarrier(final boolean isReceiverRunning, final boolean mergeSpillBuffers, + final AbstractOutputChannelForwarder next) { + super(next); + + if (next == null) { + throw new IllegalArgumentException("Argument next must not be null"); + } + this.isReceiverRunning = isReceiverRunning; this.queuedOutgoingEnvelopes = null; } @@ -29,7 +51,7 @@ public SpillingBarrier(final boolean isReceiverRunning, final boolean mergeSpill * {@inheritDoc} */ @Override - public boolean forward(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { + public void push(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException { /* * if (!this.isReceiverRunning) { @@ -38,16 +60,7 @@ public boolean forward(final TransferEnvelope transferEnvelope) throws IOExcepti * } */ - return true; - } - - /** - * {@inheritDoc} - */ - @Override - public boolean hasDataLeft() { - - return false; + getNext().push(transferEnvelope); } /** @@ -59,15 +72,7 @@ public void processEvent(final AbstractEvent event) { if (event instanceof ByteBufferedChannelActivateEvent) { this.isReceiverRunning = true; } - } - /** - * {@inheritDoc} - */ - @Override - public void destroy() { - - // Nothing to do here + getNext().processEvent(event); } - } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueThread.java index ca34de43c9b8b..604748ddfac85 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueThread.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.transferenvelope; import java.io.IOException; From 2aa93503a2d954d06f9c2efa4f0d7498843fb564 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 6 Mar 2012 13:43:46 +0100 Subject: [PATCH 298/310] Added missing Stratosphere headers --- .../managementgraph/ManagementGateID.java | 17 ++++++++++++- .../AbstractReplayGateContext.java | 24 +++++++++++++++++++ .../checkpointing/CheckpointEnvironment.java | 4 ++-- .../ReplayInputChannelContext.java | 24 +++++++++++++++++++ .../checkpointing/ReplayInputGateContext.java | 21 ++++++++++++++++ ...er.java => ReplayOutputChannelBroker.java} | 6 ++--- .../ReplayOutputChannelContext.java | 15 ++++++++++++ .../ReplayOutputGateContext.java | 2 +- .../nephele/checkpointing/ReplayTask.java | 4 ++-- .../checkpointing/ReplayTaskContext.java | 2 +- .../nephele/checkpointing/ReplayThread.java | 23 ++++++++++++++---- .../nephele/checkpointing/WriteThread.java | 15 ++++++++++++ .../io/channels/ChannelWithAccessInfo.java | 15 ++++++++++++ .../DistributedChannelWithAccessInfo.java | 15 ++++++++++++ .../channels/LocalChannelWithAccessInfo.java | 15 ++++++++++++ .../AbstractOutputChannelContext.java | 15 ++++++++++++ .../taskmanager/bytebuffered/GateContext.java | 17 ++++++++++++- .../bytebuffered/TransferEnvelopeQueue.java | 15 ++++++++++++ .../runtime/RuntimeOutputChannelBroker.java | 15 ++++++++++++ 19 files changed, 249 insertions(+), 15 deletions(-) rename nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/{ReplayOutputBroker.java => ReplayOutputChannelBroker.java} (91%) diff --git a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java index 40c7912a9e9fe..c1c5017f7a0d5 100644 --- a/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java +++ b/nephele/nephele-management/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.managementgraph; import eu.stratosphere.nephele.io.AbstractID; @@ -9,6 +24,6 @@ * * @author Bjoern Lohrmann */ -public class ManagementGateID extends AbstractID { +public final class ManagementGateID extends AbstractID { } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/AbstractReplayGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/AbstractReplayGateContext.java index f723c5e5b5f34..2df75dba2437d 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/AbstractReplayGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/AbstractReplayGateContext.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; import eu.stratosphere.nephele.io.GateID; @@ -5,8 +20,17 @@ abstract class AbstractReplayGateContext implements GateContext { + /** + * The ID of the gate this context has been created for. + */ private final GateID gateID; + /** + * Constructs a new replay gate context. + * + * @param gateID + * the ID of the gate this context has been created for + */ AbstractReplayGateContext(final GateID gateID) { this.gateID = gateID; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java index 39ec8dbf69bc0..2ca9e5205bc94 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java @@ -30,7 +30,7 @@ final class CheckpointEnvironment implements Environment { private final boolean hasCompleteCheckpoint; - private final Map outputBrokerMap; + private final Map outputBrokerMap; /** * The observer object for the task's execution. @@ -41,7 +41,7 @@ final class CheckpointEnvironment implements Environment { CheckpointEnvironment(final ExecutionVertexID vertexID, final Environment environment, final boolean hasLocalCheckpoint, final boolean hasCompleteCheckpoint, - final Map outputBrokerMap) { + final Map outputBrokerMap) { this.vertexID = vertexID; this.environment = environment; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java index 015ef80dc834a..26cf00a45d722 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; import java.io.IOException; @@ -11,8 +26,17 @@ final class ReplayInputChannelContext implements InputChannelContext { + /** + * The encapsulated input channel context. + */ private final InputChannelContext encapsulatedContext; + /** + * Constructs a new replay input channel context. + * + * @param encapsulatedContext + * the encapsulated input channel context + */ ReplayInputChannelContext(final InputChannelContext encapsulatedContext) { this.encapsulatedContext = encapsulatedContext; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java index 5960085388bb9..8f77e3333e4cb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; import eu.stratosphere.nephele.io.GateID; @@ -8,6 +23,12 @@ final class ReplayInputGateContext extends AbstractReplayGateContext implements InputGateContext { + /** + * Constructs a new replay input gate context. + * + * @param gateID + * the ID of the gate this context is created for + */ ReplayInputGateContext(final GateID gateID) { super(gateID); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java similarity index 91% rename from nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java rename to nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java index 9aa1f6b794264..d90f249a47ae1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java @@ -14,12 +14,12 @@ import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; -final class ReplayOutputBroker extends AbstractOutputChannelForwarder implements BufferProvider { +final class ReplayOutputChannelBroker extends AbstractOutputChannelForwarder implements BufferProvider { /** * The logger to report information and problems. */ - private static final Log LOG = LogFactory.getLog(ReplayOutputBroker.class); + private static final Log LOG = LogFactory.getLog(ReplayOutputChannelBroker.class); private final BufferProvider bufferProvider; @@ -27,7 +27,7 @@ final class ReplayOutputBroker extends AbstractOutputChannelForwarder implements private int nextEnvelopeToSend = 0; - ReplayOutputBroker(final BufferProvider bufferProvider, final AbstractOutputChannelForwarder next) { + ReplayOutputChannelBroker(final BufferProvider bufferProvider, final AbstractOutputChannelForwarder next) { super(next); this.bufferProvider = bufferProvider; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java index b777e89f86b46..9326f2fd2b0f9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelContext.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; import eu.stratosphere.nephele.io.channels.ChannelID; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java index 6c19c8e897f0a..83c8769bc1cc2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java @@ -38,7 +38,7 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp final SpillingBarrier spillingBarrier = new SpillingBarrier(isReceiverRunning, mergeSpillBuffers, runtimeDispatcher); final ForwardingBarrier forwardingBarrier = new ForwardingBarrier(channelID, spillingBarrier); - final ReplayOutputBroker outputChannelBroker = new ReplayOutputBroker(this.taskContext, forwardingBarrier); + final ReplayOutputChannelBroker outputChannelBroker = new ReplayOutputChannelBroker(this.taskContext, forwardingBarrier); final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(outputChannelBroker, runtimeDispatcher); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index 4c9109f0bced8..c749d2a0dec1a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -159,7 +159,7 @@ public void reportPACTDataStatistics(final long numberOfConsumedBytes, final lon */ private volatile boolean isCanceled = false; - private final Map outputBrokerMap = new ConcurrentHashMap(); + private final Map outputBrokerMap = new ConcurrentHashMap(); public ReplayTask(final ExecutionVertexID vertexID, final Environment environment, final TaskManager taskManager) { @@ -453,7 +453,7 @@ private void reportExecutionStateChange(final boolean replayTaskStateChanged, fi } } - void registerReplayOutputBroker(final ChannelID channelID, final ReplayOutputBroker outputBroker) { + void registerReplayOutputBroker(final ChannelID channelID, final ReplayOutputChannelBroker outputBroker) { this.outputBrokerMap.put(channelID, outputBroker); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java index ebb759c8fa8aa..5f154f62317da 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java @@ -61,7 +61,7 @@ public InputGateContext createInputGateContext(final GateID gateID) { return new ReplayInputGateContext(gateID); } - void registerReplayOutputBroker(final ChannelID channelID, final ReplayOutputBroker outputBroker) { + void registerReplayOutputBroker(final ChannelID channelID, final ReplayOutputChannelBroker outputBroker) { this.task.registerReplayOutputBroker(channelID, outputBroker); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index bdb0a398e3b2d..5a7d8c9419597 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; import java.io.EOFException; @@ -44,13 +59,13 @@ final class ReplayThread extends Thread { private final boolean isCheckpointComplete; - private final Map outputBrokerMap; + private final Map outputBrokerMap; private final AtomicBoolean restartRequested = new AtomicBoolean(false); ReplayThread(final ExecutionVertexID vertexID, final ExecutionObserver executionObserver, final String taskName, final boolean isCheckpointLocal, final boolean isCheckpointComplete, - final Map outputBrokerMap) { + final Map outputBrokerMap) { super((taskName == null ? "Unkown" : taskName) + REPLAY_SUFFIX); this.vertexID = vertexID; @@ -122,7 +137,7 @@ private void waitForAllOutputBrokerToFinish() throws IOException, InterruptedExc while (!this.executionObserver.isCanceled()) { boolean finished = true; - final Iterator it = this.outputBrokerMap.values().iterator(); + final Iterator it = this.outputBrokerMap.values().iterator(); while (it.hasNext()) { if (!it.next().hasFinished()) { @@ -211,7 +226,7 @@ private void replayCheckpoint() throws Exception { final TransferEnvelope transferEnvelope = deserializer.getFullyDeserializedTransferEnvelope(); if (transferEnvelope != null) { - final ReplayOutputBroker broker = this.outputBrokerMap.get(transferEnvelope.getSource()); + final ReplayOutputChannelBroker broker = this.outputBrokerMap.get(transferEnvelope.getSource()); if (broker == null) { throw new IOException("Cannot find output broker for channel " + transferEnvelope.getSource()); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java index 1aee3750052fe..e676d83f6a5f9 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/WriteThread.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; import java.io.FileOutputStream; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java index a308893429671..f7ef92fb10039 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.io.channels; import java.nio.channels.FileChannel; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java index a15ddab5daf86..36cc508f7a2c4 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.io.channels; import java.io.IOException; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java index 8ac08ec15b4f8..e8cb67335f645 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.io.channels; import java.io.File; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java index c47746a58a0b4..f1c33f291c565 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.bytebuffered; import java.util.Iterator; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java index 63ba4e7620629..a9dababed6a0c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java @@ -1,8 +1,23 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.bytebuffered; import eu.stratosphere.nephele.io.GateID; public interface GateContext { - + GateID getGateID(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TransferEnvelopeQueue.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TransferEnvelopeQueue.java index efb4355fb6395..45e6eeac9bf69 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TransferEnvelopeQueue.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TransferEnvelopeQueue.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.bytebuffered; import java.util.ArrayDeque; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java index c9e970ed8a7be..d03747b58de34 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.runtime; import java.io.IOException; From 6534ac6c0128340a53656fc23f73f96fc5c5c184 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 6 Mar 2012 13:53:17 +0100 Subject: [PATCH 299/310] Added missing Stratosphere headers --- .../checkpointing/CheckpointEnvironment.java | 15 +++++++++++++++ .../ReplayOutputChannelBroker.java | 17 ++++++++++++++++- .../checkpointing/ReplayOutputGateContext.java | 18 +++++++++++++++++- .../checkpointing/ReplayTaskContext.java | 15 +++++++++++++++ .../SpillingQueueElementTest.java | 15 +++++++++++++++ 5 files changed, 78 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java index 2ca9e5205bc94..c4f1626aa3c0f 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointEnvironment.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; import java.util.Map; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java index d90f249a47ae1..c47147a367644 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; import java.io.IOException; @@ -53,7 +68,7 @@ public void processEvent(final AbstractEvent event) { } else { LOG.warn("Received unknown event: " + event); } - + getNext().processEvent(event); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java index 83c8769bc1cc2..3abc3c38ea458 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputGateContext.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.checkpointing; import eu.stratosphere.nephele.io.GateID; @@ -38,7 +53,8 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp final SpillingBarrier spillingBarrier = new SpillingBarrier(isReceiverRunning, mergeSpillBuffers, runtimeDispatcher); final ForwardingBarrier forwardingBarrier = new ForwardingBarrier(channelID, spillingBarrier); - final ReplayOutputChannelBroker outputChannelBroker = new ReplayOutputChannelBroker(this.taskContext, forwardingBarrier); + final ReplayOutputChannelBroker outputChannelBroker = new ReplayOutputChannelBroker(this.taskContext, + forwardingBarrier); final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(outputChannelBroker, runtimeDispatcher); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java index 5f154f62317da..1b097965e74ee 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTaskContext.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** +* +* Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) +* +* 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 eu.stratosphere.nephele.checkpointing; import java.io.IOException; diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElementTest.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElementTest.java index ea9079640ce66..a2e5d04730235 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElementTest.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/SpillingQueueElementTest.java @@ -1,3 +1,18 @@ +/*********************************************************************************************************************** + * + * Copyright (C) 2010 by the Stratosphere project (http://stratosphere.eu) + * + * 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 eu.stratosphere.nephele.taskmanager.transferenvelope; import static org.junit.Assert.assertEquals; From 685423972ec4d00b584afbff6c4a92c63d7e05e1 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 6 Mar 2012 14:27:44 +0100 Subject: [PATCH 300/310] Fixed unit test --- .../io/channels/bytebuffered/FileOutputChannelTest.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-common/src/test/java/eu/stratosphere/nephele/io/channels/bytebuffered/FileOutputChannelTest.java b/nephele/nephele-common/src/test/java/eu/stratosphere/nephele/io/channels/bytebuffered/FileOutputChannelTest.java index 5ed0794266bb0..885451061cb32 100644 --- a/nephele/nephele-common/src/test/java/eu/stratosphere/nephele/io/channels/bytebuffered/FileOutputChannelTest.java +++ b/nephele/nephele-common/src/test/java/eu/stratosphere/nephele/io/channels/bytebuffered/FileOutputChannelTest.java @@ -109,7 +109,10 @@ public void writeRecordTest() throws IOException, InterruptedException { final ByteBufferedOutputChannelBroker outputBroker = mock(ByteBufferedOutputChannelBroker.class); when(outputBroker.requestEmptyWriteBuffers()).thenReturn(bufferPair); - when(this.serializationBuffer.dataLeftFromPreviousSerialization()).thenReturn(true, false, false, true, false); + when(outputBroker.hasDataLeftToTransmit()).thenReturn(true); + + when(this.serializationBuffer.dataLeftFromPreviousSerialization()).thenReturn(false, false, true, true, false, + false); // try { // when(this.serializationBuffer.readData(Matchers.any(ReadableByteChannel.class))).thenReturn(null, record); // } catch (IOException e) { @@ -136,7 +139,7 @@ public void writeRecordTest() throws IOException, InterruptedException { fileOutputChannel.requestClose(); // No acknowledgment from consumer yet so the channel should still be open assertEquals(false, fileOutputChannel.isClosed()); - fileOutputChannel.processEvent(new ByteBufferedChannelCloseEvent()); + when(outputBroker.hasDataLeftToTransmit()).thenReturn(false); // Received acknowledgment the channel should be closed now assertEquals(true, fileOutputChannel.isClosed()); try { From 06d9299d82ecb31bd20dac4107425aa5f53124fa Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 6 Mar 2012 17:35:40 +0100 Subject: [PATCH 301/310] Fixed various bugs related to file channels --- .../checkpointing/EphemeralCheckpoint.java | 1 + .../EphemeralCheckpointForwarder.java | 18 ++++--- .../ReplayInputChannelContext.java | 16 +++++-- .../checkpointing/ReplayInputGateContext.java | 5 +- .../nephele/checkpointing/ReplayTask.java | 2 +- .../scheduler/AbstractScheduler.java | 5 +- .../ByteBufferedChannelManager.java | 6 --- .../runtime/RuntimeOutputChannelBroker.java | 12 +++-- .../runtime/RuntimeOutputGateContext.java | 48 +++++++++++++------ .../nephele/jobmanager/JobManagerITCase.java | 3 +- 10 files changed, 78 insertions(+), 38 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index f58e7a956c24d..7485bb322da79 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -127,6 +127,7 @@ public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { this.task.checkpointStateChanged(CheckpointState.PARTIAL); this.writeThread = new WriteThread(FileBufferManager.getInstance(), this.task.getVertexID(), this.numberOfConnectedChannels); + this.writeThread.start(); } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpointForwarder.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpointForwarder.java index d982c75616343..d7e26f33a1cb2 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpointForwarder.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpointForwarder.java @@ -28,10 +28,6 @@ public EphemeralCheckpointForwarder(final EphemeralCheckpoint ephemeralCheckpoin final AbstractOutputChannelForwarder next) { super(next); - if (next == null) { - throw new IllegalArgumentException("Argument next must not be null"); - } - this.ephemeralCheckpoint = ephemeralCheckpoint; } @@ -43,7 +39,12 @@ public void push(final TransferEnvelope transferEnvelope) throws IOException, In this.ephemeralCheckpoint.forward(transferEnvelope); - getNext().push(transferEnvelope); + final AbstractOutputChannelForwarder next = getNext(); + if (next != null) { + next.push(transferEnvelope); + } else { + recycleTransferEnvelope(transferEnvelope); + } } /** @@ -56,6 +57,11 @@ public boolean hasDataLeft() throws IOException, InterruptedException { return true; } - return getNext().hasDataLeft(); + final AbstractOutputChannelForwarder next = getNext(); + if (next != null) { + return getNext().hasDataLeft(); + } + + return false; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java index 26cf00a45d722..04a6572083d91 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java @@ -31,13 +31,21 @@ final class ReplayInputChannelContext implements InputChannelContext { */ private final InputChannelContext encapsulatedContext; + /** + * The ID of the input channel this context belongs to. + */ + private final ChannelID channelID; + /** * Constructs a new replay input channel context. * + * @param channelID + * the ID of the input channel this context belongs to * @param encapsulatedContext * the encapsulated input channel context */ - ReplayInputChannelContext(final InputChannelContext encapsulatedContext) { + ReplayInputChannelContext(final ChannelID channelID, final InputChannelContext encapsulatedContext) { + this.channelID = channelID; this.encapsulatedContext = encapsulatedContext; } @@ -65,7 +73,7 @@ public JobID getJobID() { @Override public ChannelID getChannelID() { - return this.encapsulatedContext.getChannelID(); + return this.channelID; } /** @@ -146,7 +154,9 @@ public void reportAsynchronousEvent() { @Override public void destroy() { - this.encapsulatedContext.destroy(); + if (this.encapsulatedContext != null) { + this.encapsulatedContext.destroy(); + } } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java index 8f77e3333e4cb..9675f22437d89 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputGateContext.java @@ -37,9 +37,10 @@ final class ReplayInputGateContext extends AbstractReplayGateContext implements * {@inheritDoc} */ @Override - public InputChannelContext createInputChannelContext(ChannelID channelID, InputChannelContext previousContext) { + public InputChannelContext createInputChannelContext(final ChannelID channelID, + final InputChannelContext previousContext) { - return new ReplayInputChannelContext(previousContext); + return new ReplayInputChannelContext(channelID, previousContext); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java index c749d2a0dec1a..2b6f9f709be7a 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayTask.java @@ -470,7 +470,7 @@ public ExecutionState getExecutionState() { private static ExecutionState determineOverallExecutionState(final ExecutionState unchangedExecutionState, final ExecutionState changedExecutionState) { - if (changedExecutionState == null) { + if (unchangedExecutionState == null) { return changedExecutionState; } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java index cd16776098f15..ef8e0d3eb8ba5 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java @@ -512,13 +512,16 @@ protected void replayCheckpointsFromPreviousStage(final ExecutionGraph execution final int currentStageIndex = executionGraph.getIndexOfCurrentExecutionStage(); final ExecutionStage previousStage = executionGraph.getStage(currentStageIndex - 1); + final List verticesToBeReplayed = new ArrayList(); + for (int i = 0; i < previousStage.getNumberOfOutputExecutionVertices(); ++i) { final ExecutionVertex vertex = previousStage.getOutputExecutionVertex(i); vertex.updateExecutionState(ExecutionState.ASSIGNED); + verticesToBeReplayed.add(vertex); } - deployAssignedInputVertices(executionGraph); + deployAssignedVertices(verticesToBeReplayed); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java index 112a424c6729d..1f834ae6cee05 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java @@ -180,12 +180,6 @@ public void register(final Task task, final Set activeOutputChannels) addReceiverListHint(inputChannelContext); } - final boolean isActive = activeOutputChannels.contains(inputChannelContext.getChannelID()); - - if (LOG.isDebugEnabled()) - LOG.debug("Registering byte buffered input channel " + inputChannelContext.getChannelID() + " (" - + (isActive ? "active" : "inactive") + ")"); - this.registeredChannels.put(inputChannelContext.getChannelID(), inputChannelContext); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java index d03747b58de34..a388dd42b0101 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java @@ -23,6 +23,7 @@ import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.event.task.AbstractTaskEvent; import eu.stratosphere.nephele.io.channels.Buffer; +import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; import eu.stratosphere.nephele.io.channels.bytebuffered.BufferPairResponse; import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent; @@ -65,7 +66,7 @@ final class RuntimeOutputChannelBroker extends AbstractOutputChannelForwarder im /** * Stores whether the receiver has acknowledged the close request from this channel. */ - private boolean closeAcknowledgementReceived = false; + private boolean closeAcknowledgmentReceived = false; /** * Stores the last sequence number of the transfer envelope for which the receiver could not be found. @@ -102,7 +103,12 @@ public void setForwardingChain(final OutputChannelForwardingChain forwardingChai @Override public boolean hasDataLeft() throws IOException, InterruptedException { - if (this.closeAcknowledgementReceived) { + // Don't wait for an acknowledgment in case of a file channel, receiver is not running anyway + if (this.byteBufferedOutputChannel.getType() == ChannelType.FILE) { + return getNext().hasDataLeft(); + } + + if (this.closeAcknowledgmentReceived) { return getNext().hasDataLeft(); } @@ -120,7 +126,7 @@ public boolean hasDataLeft() throws IOException, InterruptedException { public void processEvent(final AbstractEvent event) { if (event instanceof ByteBufferedChannelCloseEvent) { - this.closeAcknowledgementReceived = true; + this.closeAcknowledgmentReceived = true; } else if (event instanceof ReceiverNotFoundEvent) { this.lastSequenceNumberWithReceiverNotFound = ((ReceiverNotFoundEvent) event).getSequenceNumber(); } else if (event instanceof AbstractTaskEvent) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java index 8020a9ac1459e..2ede8ba7777ec 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java @@ -25,8 +25,10 @@ import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; +import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext; @@ -144,26 +146,44 @@ public OutputChannelContext createOutputChannelContext(ChannelID channelID, Outp + " is not of type AbstractByteBufferedOutputChannel"); } - AbstractByteBufferedOutputChannel outputChannel = (AbstractByteBufferedOutputChannel) channel; + // The output channel for this context + final AbstractByteBufferedOutputChannel outputChannel = (AbstractByteBufferedOutputChannel) channel; - // Construct forwarding chain for this output channel - final RuntimeDispatcher runtimeDispatcher = new RuntimeDispatcher( - this.taskContext.getTransferEnvelopeDispatcher()); - final SpillingBarrier spillingBarrier = new SpillingBarrier(isReceiverRunning, mergeSpillBuffers, - runtimeDispatcher); - final ForwardingBarrier forwardingBarrier = new ForwardingBarrier(channelID, spillingBarrier); - final EphemeralCheckpoint checkpoint = this.taskContext.getEphemeralCheckpoint(); + // Construct the forwarding chain RuntimeOutputChannelBroker outputChannelBroker; - if (checkpoint != null) { - final EphemeralCheckpointForwarder checkpointForwarder = new EphemeralCheckpointForwarder(checkpoint, - forwardingBarrier); + AbstractOutputChannelForwarder last; + if (outputChannel.getType() == ChannelType.FILE) { + + // Special case for file channels + final EphemeralCheckpoint checkpoint = this.taskContext.getEphemeralCheckpoint(); + if (checkpoint == null) { + throw new IllegalStateException("No ephemeral checkpoint for file channel " + outputChannel.getID()); + } + + final EphemeralCheckpointForwarder checkpointForwarder = new EphemeralCheckpointForwarder(checkpoint, null); outputChannelBroker = new RuntimeOutputChannelBroker(this, outputChannel, checkpointForwarder); + last = checkpointForwarder; + } else { - outputChannelBroker = new RuntimeOutputChannelBroker(this, outputChannel, forwardingBarrier); + + // Construction for in-memory and network channels + final RuntimeDispatcher runtimeDispatcher = new RuntimeDispatcher( + this.taskContext.getTransferEnvelopeDispatcher()); + final SpillingBarrier spillingBarrier = new SpillingBarrier(isReceiverRunning, mergeSpillBuffers, + runtimeDispatcher); + final ForwardingBarrier forwardingBarrier = new ForwardingBarrier(channelID, spillingBarrier); + final EphemeralCheckpoint checkpoint = this.taskContext.getEphemeralCheckpoint(); + if (checkpoint != null) { + final EphemeralCheckpointForwarder checkpointForwarder = new EphemeralCheckpointForwarder(checkpoint, + forwardingBarrier); + outputChannelBroker = new RuntimeOutputChannelBroker(this, outputChannel, checkpointForwarder); + } else { + outputChannelBroker = new RuntimeOutputChannelBroker(this, outputChannel, forwardingBarrier); + } + last = runtimeDispatcher; } - final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(outputChannelBroker, - runtimeDispatcher); + final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(outputChannelBroker, last); // Set forwarding chain for broker outputChannelBroker.setForwardingChain(forwardingChain); diff --git a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java index e2820ec1f17eb..180ee9c4488f6 100644 --- a/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java +++ b/nephele/nephele-server/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java @@ -132,8 +132,7 @@ public static void startNephele() { Constructor c = JobManager.class.getDeclaredConstructor(new Class[] { String.class, String.class }); c.setAccessible(true); - jobManager = c.newInstance(new Object[] { new String(System.getProperty("user.dir") + "/correct-conf"), - new String("local") }); + jobManager = c.newInstance(new Object[] { ServerTestUtils.getConfigDir(), new String("local") }); } catch (SecurityException e) { fail(e.getMessage()); From 83009e73ee1ee3689fbb05b5bdd56394ca8cbbbc Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Tue, 6 Mar 2012 19:57:10 +0100 Subject: [PATCH 302/310] Fix several bugs related to recovery from file channels --- .../ReplayInputChannelContext.java | 4 ++- .../ReplayOutputChannelBroker.java | 4 +++ .../executiongraph/ExecutionVertex.java | 27 +++++++++++++++---- .../nephele/taskmanager/TaskManager.java | 6 +++++ 4 files changed, 35 insertions(+), 6 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java index 04a6572083d91..ff98c6d3be7d7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayInputChannelContext.java @@ -165,6 +165,8 @@ public void destroy() { @Override public void logQueuedEnvelopes() { - this.encapsulatedContext.logQueuedEnvelopes(); + if (this.encapsulatedContext != null) { + this.encapsulatedContext.logQueuedEnvelopes(); + } } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java index c47147a367644..c1b2e3bfe1959 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java @@ -26,6 +26,7 @@ import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder; import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain; +import eu.stratosphere.nephele.taskmanager.bytebuffered.ReceiverNotFoundEvent; import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; @@ -65,6 +66,9 @@ public void processEvent(final AbstractEvent event) { if (uee.getExpectedSequenceNumber() > this.nextEnvelopeToSend) { this.nextEnvelopeToSend = uee.getExpectedSequenceNumber(); } + } else if (event instanceof ReceiverNotFoundEvent) { + final ReceiverNotFoundEvent rnfe = (ReceiverNotFoundEvent) event; + LOG.warn("Cannot find receiver " + rnfe.getReceiverID() + " for envelope " + rnfe.getSequenceNumber()); } else { LOG.warn("Received unknown event: " + event); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 104aa44556f52..14852fdf44bd0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -313,11 +313,20 @@ public ExecutionState updateExecutionState(ExecutionState newExecutionState, fin throw new IllegalArgumentException("Argument newExecutionState must not be null"); } - // Rewrite FINISHED to CANCELED if the task has been marked to be canceled - if (this.executionState.get() == ExecutionState.CANCELING && newExecutionState == ExecutionState.FINISHED) { - LOG.info("Received transition from CANCELING to FINISHED for vertex " + toString() - + ", converting it to CANCELED"); - newExecutionState = ExecutionState.CANCELED; + final ExecutionState currentExecutionState = this.executionState.get(); + if (currentExecutionState == ExecutionState.CANCELING) { + + // If we are in CANCELING, ignore state changes to FINISHING + if (newExecutionState == ExecutionState.FINISHING) { + return currentExecutionState; + } + + // Rewrite FINISHED to CANCELED if the task has been marked to be canceled + if (newExecutionState == ExecutionState.FINISHED) { + LOG.info("Received transition from CANCELING to FINISHED for vertex " + toString() + + ", converting it to CANCELED"); + newExecutionState = ExecutionState.CANCELED; + } } // Check and save the new execution state @@ -700,6 +709,14 @@ public TaskCancelResult cancelTask() { return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); } + if (previousState == ExecutionState.FAILED) { + return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); + } + + if (previousState == ExecutionState.FINISHED) { + return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS); + } + if (updateExecutionState(ExecutionState.CANCELING) != ExecutionState.CANCELING) { if (this.groupVertex.getStageNumber() != this.executionGraph.getIndexOfCurrentExecutionStage()) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index e9321bab35d81..dc22256815fb3 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -588,6 +588,7 @@ private Task createAndRegisterTask(final ExecutionVertexID id, final Configurati } } else { // There is already a replay task running, we will simply restart it + task = runningTask; registerTask = false; } @@ -716,6 +717,11 @@ public void updateLibraryCache(LibraryCacheUpdate update) throws IOException { public void executionStateChanged(final JobID jobID, final ExecutionVertexID id, final ExecutionState newExecutionState, final String optionalDescription) { + // Don't propagate state CANCELING back to the job manager + if (newExecutionState == ExecutionState.CANCELING) { + return; + } + if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED || newExecutionState == ExecutionState.FAILED) { From 8427b4477adce1cdadb5fa1a881248340bfb174f Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 7 Mar 2012 16:15:24 +0100 Subject: [PATCH 303/310] Fixed bugs related to task recovery --- .../ReplayOutputChannelBroker.java | 4 + .../nephele/checkpointing/ReplayThread.java | 85 +++++++++++++------ .../execution/ExecutionStateTransition.java | 5 ++ 3 files changed, 69 insertions(+), 25 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java index c1b2e3bfe1959..894e741f07ed1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayOutputChannelBroker.java @@ -87,6 +87,10 @@ void outputEnvelope(final TransferEnvelope transferEnvelope) throws IOException, this.forwardingChain.pushEnvelope(transferEnvelope); } + void reset() { + this.nextEnvelopeToSend = 0; + } + int getNextEnvelopeToSend() { return this.nextEnvelopeToSend; diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index 5a7d8c9419597..18060c34642a1 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -81,35 +81,58 @@ final class ReplayThread extends Thread { @Override public void run() { - // Now the actual program starts to run - changeExecutionState(ExecutionState.REPLAYING, null); + while (true) { - // If the task has been canceled in the mean time, do not even start it - if (this.executionObserver.isCanceled()) { - changeExecutionState(ExecutionState.CANCELED, null); - return; - } - - try { - - replayCheckpoint(); + // Now the actual program starts to run + changeExecutionState(ExecutionState.REPLAYING, null); - // Make sure, we enter the catch block when the task has been canceled + // If the task has been canceled in the mean time, do not even start it if (this.executionObserver.isCanceled()) { - throw new InterruptedException(); + changeExecutionState(ExecutionState.CANCELED, null); + return; } - } catch (Exception e) { + // Reset all the output broker in case we here restarted + resetAllOutputBroker(); - e.printStackTrace(); + try { - if (this.executionObserver.isCanceled()) { - changeExecutionState(ExecutionState.CANCELED, null); - } else { - changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(e)); + // Replay the actual checkpoint + replayCheckpoint(); + + // Make sure, we enter the catch block when the task has been canceled + if (this.executionObserver.isCanceled()) { + throw new InterruptedException(); + } + + } catch (Exception e) { + + if (this.restartRequested.compareAndSet(true, false)) { + // Wait for the thread to be interrupted, then clear interrupted flag + while (!Thread.currentThread().isInterrupted()) { + } + Thread.interrupted(); + continue; + } + + if (this.executionObserver.isCanceled()) { + changeExecutionState(ExecutionState.CANCELED, null); + } else { + changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(e)); + } + + return; } - return; + if (this.restartRequested.compareAndSet(true, false)) { + // Wait for the thread to be interrupted, then clear interrupted flag + while (!Thread.currentThread().isInterrupted()) { + } + Thread.interrupted(); + continue; + } + + break; } // Task finished running, but there may be some unconsumed data in the brokers @@ -133,6 +156,15 @@ public void run() { changeExecutionState(ExecutionState.FINISHED, null); } + private void resetAllOutputBroker() { + + final Iterator it = this.outputBrokerMap.values().iterator(); + while (it.hasNext()) { + it.next().reset(); + } + + } + private void waitForAllOutputBrokerToFinish() throws IOException, InterruptedException { while (!this.executionObserver.isCanceled()) { @@ -162,7 +194,9 @@ private void changeExecutionState(final ExecutionState newExecutionState, final void restart() { + changeExecutionState(ExecutionState.STARTING, null); this.restartRequested.set(true); + interrupt(); } private void replayCheckpoint() throws Exception { @@ -188,10 +222,6 @@ private void replayCheckpoint() throws Exception { while (true) { - if (this.restartRequested.compareAndSet(true, false)) { - metaDataIndex = 0; - } - // Try to locate the meta data file final Path metaDataFile = checkpointPath.suffix(Path.SEPARATOR + CheckpointUtils.METADATA_PREFIX + "_" + this.vertexID + "_" + metaDataIndex); @@ -226,7 +256,8 @@ private void replayCheckpoint() throws Exception { final TransferEnvelope transferEnvelope = deserializer.getFullyDeserializedTransferEnvelope(); if (transferEnvelope != null) { - final ReplayOutputChannelBroker broker = this.outputBrokerMap.get(transferEnvelope.getSource()); + final ReplayOutputChannelBroker broker = this.outputBrokerMap.get(transferEnvelope + .getSource()); if (broker == null) { throw new IOException("Cannot find output broker for channel " + transferEnvelope.getSource()); @@ -252,6 +283,10 @@ private void replayCheckpoint() throws Exception { } broker.outputEnvelope(transferEnvelope); + + if (this.restartRequested.get()) { + return; + } } } catch (EOFException eof) { // Close the file channel diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java index 125fad0d11377..69218c2e7ebf7 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java @@ -153,6 +153,11 @@ public static void checkTransition(final boolean jobManager, final String taskNa unexpectedStateChange = false; } + // This is a regular transition in case a replay task is restarted + if (oldState == ExecutionState.REPLAYING && newState == ExecutionState.ASSIGNED) { + unexpectedStateChange = false; + } + // This is a regular transition in case a task replay is triggered. if (oldState == ExecutionState.FINISHING && newState == ExecutionState.ASSIGNED) { unexpectedStateChange = false; From dd15ef00c9161a7ad5c2305429ee8c86cbdf59a6 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 7 Mar 2012 18:08:09 +0100 Subject: [PATCH 304/310] Extended user API to set the number of execution retries per vertex --- .../nephele/jobgraph/AbstractJobVertex.java | 34 +++++++++++++++++++ .../executiongraph/ExecutionGraph.java | 2 +- .../executiongraph/ExecutionGroupVertex.java | 34 +++++++++++++++++-- .../executiongraph/ExecutionVertex.java | 2 +- 4 files changed, 68 insertions(+), 4 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java index 67d87e332c627..97af4314d6ec1 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java @@ -78,6 +78,11 @@ public abstract class AbstractJobVertex implements IOReadableWritable { */ private int numberOfSubtasksPerInstance = -1; + /** + * Number of retries in case of an error before the task represented by this vertex is considered as failed. + */ + private int numberOfExecutionRetries = -1; + /** * Other task to share a (set of) of instances with at runtime. */ @@ -385,6 +390,9 @@ public void read(final DataInput in) throws IOException { // Read number of subtasks per instance this.numberOfSubtasksPerInstance = in.readInt(); + // Number of execution retries + this.numberOfExecutionRetries = in.readInt(); + // Read vertex to share instances with if (in.readBoolean()) { final JobVertexID id = new JobVertexID(); @@ -466,6 +474,9 @@ public void write(final DataOutput out) throws IOException { // Number of subtasks per instance out.writeInt(this.numberOfSubtasksPerInstance); + // Number of execution retries + out.writeInt(this.numberOfExecutionRetries); + // Vertex to share instance with if (this.vertexToShareInstancesWith != null) { out.writeBoolean(true); @@ -536,6 +547,29 @@ public int getNumberOfSubtasks() { return this.numberOfSubtasks; } + /** + * Sets the number of retries in case of an error before the task represented by this vertex is considered as + * failed. + * + * @param numberOfExecutionRetries + * the number of retries in case of an error before the task represented by this vertex is considered as + * failed + */ + public void setNumberOfExecutionRetries(final int numberOfExecutionRetries) { + this.numberOfExecutionRetries = numberOfExecutionRetries; + } + + /** + * Returns the number of retries in case of an error before the task represented by this vertex is considered as + * failed. + * + * @return the number of retries in case of an error before the task represented by this vertex is considered as + * failed or -1 if unspecified + */ + public int getNumberOfExecutionRetries() { + return this.numberOfExecutionRetries; + } + /** * Sets the instance type the task this vertex represents should run on. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index 76dc588cf5174..bbc9cb31fa951 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -556,7 +556,7 @@ private ExecutionVertex createVertex(final AbstractJobVertex jobVertex, final In final ExecutionGroupVertex groupVertex = new ExecutionGroupVertex(jobVertex.getName(), jobVertex.getID(), this, jobVertex.getNumberOfSubtasks(), instanceType, userDefinedInstanceType, jobVertex .getNumberOfSubtasksPerInstance(), jobVertex.getVertexToShareInstancesWith() != null ? true : false, - jobVertex.getConfiguration(), signature); + jobVertex.getNumberOfExecutionRetries(), jobVertex.getConfiguration(), signature); // Create an initial execution vertex for the job vertex final Class invokableClass = jobVertex.getInvokableClass(); if (invokableClass == null) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java index d3feb7579c87e..02902060103dc 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java @@ -46,6 +46,12 @@ */ public class ExecutionGroupVertex { + /** + * The default number of retries in case of an error before the task represented by this vertex is considered as + * failed. + */ + private final int DEFAULT_NUMBER_OF_EXECUTION_RETRIES = 2; + /** * The name of the vertex. */ @@ -101,6 +107,11 @@ public class ExecutionGroupVertex { */ private final boolean userDefinedNumberOfSubtasksPerInstance; + /** + * Number of retries in case of an error before the task represented by this vertex is considered as failed. + */ + private final int numberOfExecutionRetries; + /** * The execution group vertex to share instances with. */ @@ -172,6 +183,9 @@ public class ExecutionGroupVertex { * @param userDefinedVertexToShareInstanceWith * true if the user specified another vertex to share instances with, false * otherwise + * @param numberOfExecutionRetries + * the number of retries in case of an error before the task represented by this vertex is considered as + * failed, -1 if the user did not specify the number * @param configuration * the vertex's configuration object * @param signature @@ -180,8 +194,8 @@ public class ExecutionGroupVertex { public ExecutionGroupVertex(final String name, final JobVertexID jobVertexID, final ExecutionGraph executionGraph, final int userDefinedNumberOfMembers, final InstanceType instanceType, final boolean userDefinedInstanceType, final int numberOfSubtasksPerInstance, - final boolean userDefinedVertexToShareInstanceWith, final Configuration configuration, - final ExecutionSignature signature) { + final boolean userDefinedVertexToShareInstanceWith, final int numberOfExecutionRetries, + final Configuration configuration, final ExecutionSignature signature) { this.name = name; this.jobVertexID = jobVertexID; @@ -196,6 +210,11 @@ public ExecutionGroupVertex(final String name, final JobVertexID jobVertexID, fi this.numberOfSubtasksPerInstance = 1; this.userDefinedNumberOfSubtasksPerInstance = false; } + if (numberOfExecutionRetries >= 0) { + this.numberOfExecutionRetries = numberOfExecutionRetries; + } else { + this.numberOfExecutionRetries = DEFAULT_NUMBER_OF_EXECUTION_RETRIES; + } this.userDefinedVertexToShareInstancesWith = userDefinedVertexToShareInstanceWith; this.configuration = configuration; this.executionSignature = signature; @@ -694,6 +713,17 @@ int getNumberOfSubtasksPerInstance() { return this.numberOfSubtasksPerInstance; } + /** + * Returns the number of retries in case of an error before the task represented by this vertex is considered as + * failed. + * + * @return the number of retries in case of an error before the task represented by this vertex is considered as + * failed + */ + int getNumberOfExecutionRetries() { + return this.numberOfExecutionRetries; + } + void shareInstancesWith(final ExecutionGroupVertex groupVertex) throws GraphConversionException { if (this.userDefinedVertexToShareInstancesWith && this.vertexToShareInstancesWith.get() != null) { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 14852fdf44bd0..39fb9182e7bc6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -201,7 +201,7 @@ private ExecutionVertex(final ExecutionVertexID vertexID, final Class Date: Wed, 7 Mar 2012 19:37:48 +0100 Subject: [PATCH 305/310] Improved computation of initial checkpoint state --- .../stratosphere/nephele/util/AtomicEnum.java | 11 ++++++ .../checkpointing/EphemeralCheckpoint.java | 23 +++++------- .../executiongraph/ExecutionGroupVertex.java | 36 +++++++++++++++++++ .../executiongraph/ExecutionVertex.java | 34 ++++++++++++++++-- .../nephele/jobmanager/JobManager.java | 2 +- .../protocols/TaskOperationProtocol.java | 8 ++--- .../nephele/taskmanager/TaskManager.java | 15 ++++++-- .../taskmanager/TaskSubmissionWrapper.java | 34 +++++++++++++++--- .../taskmanager/runtime/RuntimeTask.java | 7 ++-- .../runtime/RuntimeTaskContext.java | 23 +++++++----- 10 files changed, 151 insertions(+), 42 deletions(-) diff --git a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/AtomicEnum.java b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/AtomicEnum.java index be2a536decc9f..652ba9f1c8e2d 100644 --- a/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/AtomicEnum.java +++ b/nephele/nephele-common/src/main/java/eu/stratosphere/nephele/util/AtomicEnum.java @@ -43,6 +43,17 @@ public AtomicEnum(final T initialValue) { this.ref = new AtomicReference(initialValue); } + /** + * Sets to the given value. + * + * @param newValue + * the new value + */ + public void set(final T newValue) { + + this.ref.set(newValue); + } + /** * Gets the current value. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java index 7485bb322da79..294c537474551 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/EphemeralCheckpoint.java @@ -25,7 +25,6 @@ import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask; import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope; import eu.stratosphere.nephele.execution.Environment; -import eu.stratosphere.nephele.execution.RuntimeEnvironment; import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.FileBufferManager; @@ -60,9 +59,9 @@ public class EphemeralCheckpoint implements CheckpointDecisionRequester { private final RuntimeTask task; /** - * The number of channels connected to this checkpoint. + * The total number of output channels connected to this checkpoint. */ - private final int numberOfConnectedChannels; + private final int totalNumberOfOutputChannels; /** * Stores whether a completed checkpoint has already been announced to the task. @@ -99,22 +98,16 @@ private enum CheckpointingDecisionState { * * @param task * the task this checkpoint belongs to + * @param totalNumberOfOutputChannels + * the total number of output channels connected to this checkpoint * @param ephemeral * true if the checkpoint is initially ephemeral, false if the checkpoint shall be * persistent from the beginning */ - public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { + public EphemeralCheckpoint(final RuntimeTask task, final int totalNumberOfOutputChannels, final boolean ephemeral) { this.task = task; - - // Determine number of output channel - int nooc = 0; - final RuntimeEnvironment environment = task.getRuntimeEnvironment(); - for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { - nooc += environment.getOutputGate(i).getNumberOfOutputChannels(); - } - - this.numberOfConnectedChannels = nooc; + this.totalNumberOfOutputChannels = totalNumberOfOutputChannels; this.checkpointingDecision = (ephemeral ? CheckpointingDecisionState.UNDECIDED : CheckpointingDecisionState.CHECKPOINTING); @@ -126,7 +119,7 @@ public EphemeralCheckpoint(final RuntimeTask task, final boolean ephemeral) { if (this.checkpointingDecision == CheckpointingDecisionState.CHECKPOINTING) { this.task.checkpointStateChanged(CheckpointState.PARTIAL); this.writeThread = new WriteThread(FileBufferManager.getInstance(), this.task.getVertexID(), - this.numberOfConnectedChannels); + this.totalNumberOfOutputChannels); this.writeThread.start(); } } @@ -152,7 +145,7 @@ private void write() throws IOException, InterruptedException { if (this.writeThread == null) { this.writeThread = new WriteThread(FileBufferManager.getInstance(), task.getVertexID(), - this.numberOfConnectedChannels); + this.totalNumberOfOutputChannels); this.writeThread.start(); } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java index 02902060103dc..4cb2b3138d985 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java @@ -158,6 +158,11 @@ public class ExecutionGroupVertex { */ private volatile ExecutionStage executionStage = null; + /** + * The initial checkpoint state for vertices assigned to this group vertex. + */ + private volatile CheckpointState initialCheckpointState = null; + /** * The configuration object of the original job vertex. */ @@ -747,6 +752,37 @@ boolean isVertexToShareInstanceWithUserDefined() { return this.userDefinedVertexToShareInstancesWith; } + /** + * Sets the initial checkpoint state for vertices assigned to this group vertex. + * + * @param initialCheckpointState + * the initial checkpoint state for vertices assigned to this group vertex + */ + void setInitialCheckpointState(final CheckpointState initialCheckpointState) { + + if (initialCheckpointState == null) { + throw new IllegalArgumentException("Argument initialCheckpointState must not be null"); + } + + if (this.initialCheckpointState != null) { + throw new IllegalStateException("Initial checkpoint state for group vertex " + getName() + + " is already set"); + } + + this.initialCheckpointState = initialCheckpointState; + } + + /** + * Returns the initial checkpoint state for vertices assigned to this group vertex. + * + * @return the initial checkpoint state for vertices assigned to this group vertex or null if the state + * is not yet set + */ + CheckpointState getInitialCheckpointState() { + + return this.initialCheckpointState; + } + /** * Returns the configuration object of the original job vertex. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 39fb9182e7bc6..477d57b36369b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.util.StringUtils; +import eu.stratosphere.nephele.annotations.ForceCheckpoint; import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.ExecutionListener; import eu.stratosphere.nephele.execution.ExecutionState; @@ -138,8 +139,7 @@ public final class ExecutionVertex { /** * The current checkpoint state of this vertex. */ - private final AtomicEnum checkpointState = new AtomicEnum( - CheckpointState.UNDECIDED); + private final AtomicEnum checkpointState; /** * The execution pipeline this vertex is part of. @@ -176,6 +176,32 @@ public ExecutionVertex(final JobID jobID, final Class(groupVertex.getInitialCheckpointState()); + this.retriesLeft = new AtomicInteger(groupVertex.getNumberOfExecutionRetries()); // Register the vertex itself as a listener for state changes @@ -624,7 +652,7 @@ public TaskSubmissionResult startTask() { final List tasks = new SerializableArrayList(); final TaskSubmissionWrapper tsw = new TaskSubmissionWrapper(this.vertexID, this.environment, - this.executionGraph.getJobConfiguration(), activeOutputChannels); + this.executionGraph.getJobConfiguration(), this.checkpointState.get(), activeOutputChannels); tasks.add(tsw); try { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java index 059da106b1ffa..e10082006d03b 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java @@ -1180,7 +1180,7 @@ public void run() { for (final ExecutionVertex vertex : verticesToBeDeployed) { submissionList.add(new TaskSubmissionWrapper(vertex.getID(), vertex.getEnvironment(), vertex - .getExecutionGraph().getJobConfiguration(), vertex + .getExecutionGraph().getJobConfiguration(), vertex.getCheckpointState(), vertex .constructInitialActiveOutputChannelsSet())); LOG.info("Starting task " + vertex + " on " + vertex.getAllocatedResource().getInstance()); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java index bd947473bdc03..5622de43bd149 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java @@ -49,8 +49,7 @@ public interface TaskOperationProtocol extends VersionedProtocol { * @throws IOException * thrown if an error occurs during this remote procedure call */ - List submitTasks(List tasks) - throws IOException; + List submitTasks(List tasks) throws IOException; /** * Advises the task manager to cancel the task with the given ID. @@ -74,9 +73,8 @@ List submitTasks(List tasks) */ TaskKillResult killTask(ExecutionVertexID id) throws IOException; - TaskCheckpointResult requestCheckpointDecision(ExecutionVertexID id) throws IOException; - + /** * Queries the task manager about the cache status of the libraries stated in the {@link LibraryCacheProfileRequest} * object. @@ -99,7 +97,7 @@ List submitTasks(List tasks) * thrown if an error occurs during this remote procedure call */ void updateLibraryCache(LibraryCacheUpdate update) throws IOException; - + /** * Removes the checkpoints which are identified by the provided list of vertex IDs. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java index dc22256815fb3..031b33114facb 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java @@ -512,10 +512,12 @@ public List submitTasks(final List final RuntimeEnvironment re = tsw.getEnvironment(); final ExecutionVertexID id = tsw.getVertexID(); final Configuration jobConfiguration = tsw.getConfiguration(); + final CheckpointState initialCheckpointState = tsw.getInitialCheckpointState(); final Set activeOutputChannels = tsw.getActiveOutputChannels(); // Register the task - final Task task = createAndRegisterTask(id, jobConfiguration, re, activeOutputChannels); + final Task task = createAndRegisterTask(id, jobConfiguration, re, initialCheckpointState, + activeOutputChannels); if (task == null) { final TaskSubmissionResult result = new TaskSubmissionResult(id, AbstractTaskResult.ReturnCode.TASK_NOT_FOUND); @@ -545,12 +547,15 @@ public List submitTasks(final List * the job configuration that has been attached to the original job graph * @param environment * the environment of the task to be registered + * @param initialCheckpointState + * the task's initial checkpoint state * @param activeOutputChannels * the set of initially active output channels * @return the task to be started or null if a task with the same ID was already running */ private Task createAndRegisterTask(final ExecutionVertexID id, final Configuration jobConfiguration, - final RuntimeEnvironment environment, final Set activeOutputChannels) throws IOException { + final RuntimeEnvironment environment, final CheckpointState initialCheckpointState, + final Set activeOutputChannels) throws IOException { if (id == null) { throw new IllegalArgumentException("Argument id is null"); @@ -560,6 +565,10 @@ private Task createAndRegisterTask(final ExecutionVertexID id, final Configurati throw new IllegalArgumentException("Argument environment is null"); } + if (initialCheckpointState == null) { + throw new IllegalArgumentException("Argument initialCheckpointState is null"); + } + // Task creation and registration must be atomic Task task = null; @@ -573,7 +582,7 @@ private Task createAndRegisterTask(final ExecutionVertexID id, final Configurati if (CheckpointUtils.hasCompleteCheckpointAvailable(id)) { task = new ReplayTask(id, environment, this); } else { - task = new RuntimeTask(id, environment, this); + task = new RuntimeTask(id, environment, initialCheckpointState, this); } } else { diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskSubmissionWrapper.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskSubmissionWrapper.java index bc15442f87dc1..c36e8b99de435 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskSubmissionWrapper.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/TaskSubmissionWrapper.java @@ -22,9 +22,11 @@ import eu.stratosphere.nephele.configuration.Configuration; import eu.stratosphere.nephele.execution.RuntimeEnvironment; +import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.io.IOReadableWritable; import eu.stratosphere.nephele.io.channels.ChannelID; +import eu.stratosphere.nephele.util.EnumUtils; import eu.stratosphere.nephele.util.SerializableHashSet; /** @@ -50,6 +52,11 @@ public final class TaskSubmissionWrapper implements IOReadableWritable { */ Configuration configuration = null; + /** + * The task's initial checkpoint state. + */ + CheckpointState initialCheckpointState = null; + /** * The set of initially active output channels. */ @@ -64,9 +71,14 @@ public final class TaskSubmissionWrapper implements IOReadableWritable { * the task's execution environment * @param configuration * the task's configuration + * @param initialCheckpointState + * the initial state the task's checkpoint is in + * @param activeOutputChannels + * the set of initially active output channels */ public TaskSubmissionWrapper(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, - final Configuration configuration, final SerializableHashSet activeOutputChannels) { + final Configuration configuration, final CheckpointState initialCheckpointState, + final SerializableHashSet activeOutputChannels) { if (vertexID == null) { throw new IllegalArgumentException("Argument vertexID is null"); @@ -80,6 +92,10 @@ public TaskSubmissionWrapper(final ExecutionVertexID vertexID, final RuntimeEnvi throw new IllegalArgumentException("Argument configuration is null"); } + if (initialCheckpointState == null) { + throw new IllegalArgumentException("Argument initialCheckpointState in null"); + } + if (activeOutputChannels == null) { throw new IllegalArgumentException("Argument activeOutputChannels is null"); } @@ -87,6 +103,7 @@ public TaskSubmissionWrapper(final ExecutionVertexID vertexID, final RuntimeEnvi this.vertexID = vertexID; this.environment = environment; this.configuration = configuration; + this.initialCheckpointState = initialCheckpointState; this.activeOutputChannels = activeOutputChannels; } @@ -105,6 +122,7 @@ public void write(final DataOutput out) throws IOException { this.vertexID.write(out); this.environment.write(out); this.configuration.write(out); + EnumUtils.writeEnum(out, this.initialCheckpointState); this.activeOutputChannels.write(out); } @@ -120,6 +138,7 @@ public void read(final DataInput in) throws IOException { this.environment.read(in); this.configuration = new Configuration(); this.configuration.read(in); + this.initialCheckpointState = EnumUtils.readEnum(in, CheckpointState.class); this.activeOutputChannels = new SerializableHashSet(); this.activeOutputChannels.read(in); } @@ -130,7 +149,6 @@ public void read(final DataInput in) throws IOException { * @return the task's execution vertex ID */ public ExecutionVertexID getVertexID() { - return this.vertexID; } @@ -140,7 +158,6 @@ public ExecutionVertexID getVertexID() { * @return the task's execution environment */ public RuntimeEnvironment getEnvironment() { - return this.environment; } @@ -150,17 +167,24 @@ public RuntimeEnvironment getEnvironment() { * @return the task's configuration object */ public Configuration getConfiguration() { - return this.configuration; } + /** + * Returns the task's initial checkpoint state. + * + * @return the task's initial checkpoint state + */ + public CheckpointState getInitialCheckpointState() { + return this.initialCheckpointState; + } + /** * Returns the set of initially active output channels. * * @return the set of initially active output channels */ public Set getActiveOutputChannels() { - return this.activeOutputChannels; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java index 0db44af4f0735..be2f4939fc2e0 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java @@ -56,6 +56,8 @@ public final class RuntimeTask implements Task, ExecutionObserver { private final RuntimeEnvironment environment; + private final CheckpointState initialCheckpointState; + private final TaskManager taskManager; /** @@ -84,10 +86,11 @@ public final class RuntimeTask implements Task, ExecutionObserver { // DW: End of temporay code public RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, - final TaskManager taskManager) { + final CheckpointState initialCheckpointState, final TaskManager taskManager) { this.vertexID = vertexID; this.environment = environment; + this.initialCheckpointState = initialCheckpointState; this.taskManager = taskManager; this.environment.setExecutionObserver(this); @@ -426,7 +429,7 @@ public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEn throw new IllegalStateException("Vertex " + this.vertexID + " has a previous buffer pool owner"); } - return new RuntimeTaskContext(this, transferEnvelopeDispatcher); + return new RuntimeTaskContext(this, this.initialCheckpointState, transferEnvelopeDispatcher); } /** diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java index 8797ee8e7d62f..e69c09b89da17 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java @@ -31,6 +31,7 @@ import eu.stratosphere.nephele.checkpointing.EphemeralCheckpoint; import eu.stratosphere.nephele.execution.ResourceUtilizationSnapshot; import eu.stratosphere.nephele.execution.RuntimeEnvironment; +import eu.stratosphere.nephele.executiongraph.CheckpointState; import eu.stratosphere.nephele.io.AbstractID; import eu.stratosphere.nephele.io.GateID; import eu.stratosphere.nephele.io.InputGate; @@ -39,7 +40,6 @@ import eu.stratosphere.nephele.io.channels.AbstractOutputChannel; import eu.stratosphere.nephele.io.channels.Buffer; import eu.stratosphere.nephele.io.channels.ChannelID; -import eu.stratosphere.nephele.io.channels.ChannelType; import eu.stratosphere.nephele.taskmanager.bufferprovider.AsynchronousEventListener; import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider; import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool; @@ -72,7 +72,8 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve private long startTime; - RuntimeTaskContext(final RuntimeTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher) { + RuntimeTaskContext(final RuntimeTask task, final CheckpointState initialCheckpointState, + final TransferEnvelopeDispatcher transferEnvelopeDispatcher) { this.localBufferPool = new LocalBufferPool(1, false, this); this.task = task; @@ -81,18 +82,20 @@ public final class RuntimeTaskContext implements BufferProvider, AsynchronousEve // Compute number of output input channels int nooc = 0; - boolean ephemeral = true; for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) { final OutputGate outputGate = environment.getOutputGate(i); nooc += outputGate.getNumberOfOutputChannels(); - if (outputGate.getChannelType() == ChannelType.FILE) { - ephemeral = false; - } } this.numberOfOutputChannels = nooc; - this.ephemeralCheckpoint = new EphemeralCheckpoint(task, ephemeral); - this.task.registerCheckpointDecisionRequester(this.ephemeralCheckpoint); + if (initialCheckpointState == CheckpointState.NONE) { + this.ephemeralCheckpoint = null; + } else { + this.ephemeralCheckpoint = new EphemeralCheckpoint(task, this.numberOfOutputChannels, + initialCheckpointState == CheckpointState.UNDECIDED); + this.task.registerCheckpointDecisionRequester(this.ephemeralCheckpoint); + } + this.transferEnvelopeDispatcher = transferEnvelopeDispatcher; this.envelopeConsumptionLog = new EnvelopeConsumptionLog(task.getVertexID(), environment); @@ -184,6 +187,10 @@ public void logBufferUtilization() { */ void reportExhaustionOfMemoryBuffers() throws IOException, InterruptedException { + if (this.ephemeralCheckpoint == null) { + return; + } + if (!this.ephemeralCheckpoint.isUndecided()) { return; } From c1ca650d848545575947342d7196bc173024132e Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 7 Mar 2012 19:45:20 +0100 Subject: [PATCH 306/310] Clean-up --- .../checkpointing/CheckpointDecision.java | 6 -- .../ResourceUtilizationSnapshot.java | 64 +++++++------------ .../executiongraph/ExecutionVertex.java | 2 + .../runtime/RuntimeTaskContext.java | 25 +------- 4 files changed, 26 insertions(+), 71 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java index 8394f049c8cb4..49cddef29b45c 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointDecision.java @@ -38,12 +38,6 @@ public static boolean getDecision(final RuntimeTask task, final ResourceUtilizat return isNetworkTask(task); } - if (rus.getForced() != null) { - LOG.info("Checkpoint decision was forced to " + rus.getForced()); - // checkpoint decision was forced by the user - return rus.getForced(); - } - final double CPlower = CheckpointUtils.getCPLower(); final double CPupper = CheckpointUtils.getCPUpper(); diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java index 6893c60faae88..ab4675d6c368e 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/execution/ResourceUtilizationSnapshot.java @@ -41,21 +41,17 @@ public final class ResourceUtilizationSnapshot implements IOReadableWritable { * Stores the utilization of task's output channels at the time when the snapshot was created. */ private final Map channelUtilization; - + /** * userCPu Time in percent */ private long userCPU; - - /** - * The forced decision if annotated - */ - private Boolean forced; - + /** * amount of input bytes of all input-channels */ private long totalInputAmount; + /** * amount of transmitted bytes of all output-channels */ @@ -69,8 +65,9 @@ public final class ResourceUtilizationSnapshot implements IOReadableWritable { private boolean isDam; - - public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU, Boolean force, long totalInputAmount2, long totalOutputAmount2, long averageOutputRecordSize2, long averageInputRecordSize2, double pactRatio, boolean isDam) { + public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization, + long userCPU, long totalInputAmount2, long totalOutputAmount2, long averageOutputRecordSize2, + long averageInputRecordSize2, double pactRatio, boolean isDam) { if (timestamp <= 0L) { throw new IllegalArgumentException("Argument timestamp must be larger than zero"); @@ -83,16 +80,17 @@ public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization,long userCPU, final Boolean forced, final long totalInputAmount, final long totalOutputAmount) { + + public ResourceUtilizationSnapshot(final long timestamp, final Map channelUtilization, + long userCPU, final Boolean forced, final long totalInputAmount, final long totalOutputAmount) { if (timestamp <= 0L) { throw new IllegalArgumentException("Argument timestamp must be larger than zero"); @@ -105,9 +103,8 @@ public ResourceUtilizationSnapshot(final long timestamp, final Map channelU this.timestamp = timestamp; this.channelUtilization = channelUtilization; this.userCPU = userCPU; - this.forced = force; this.totalInputAmount = totalInputAmount; - this.totalOutputAmount = totalOutputAmount; + this.totalOutputAmount = totalOutputAmount; this.averageOutputRecordSize = averageOutputRecordSize; this.averageInputRecordSize = averageInputRecordSize; - + } + /** * {@inheritDoc} */ @@ -155,13 +152,7 @@ public void write(final DataOutput out) throws IOException { } // Write the userCPU out.writeLong(this.userCPU); - // Write forced decision - if(this.forced == null){ - out.writeByte(0); - }else{ - out.writeByte(1); - out.writeBoolean(this.forced); - } + out.writeLong(this.totalInputAmount); out.writeLong(this.totalOutputAmount); out.writeLong(this.averageInputRecordSize); @@ -187,10 +178,7 @@ public void read(final DataInput in) throws IOException { this.channelUtilization.put(channelID, l); } this.userCPU = in.readLong(); - - if(in.readByte() == 1){ - this.forced = in.readBoolean(); - } + this.totalInputAmount = in.readLong(); this.totalOutputAmount = in.readLong(); this.averageInputRecordSize = in.readLong(); @@ -226,7 +214,6 @@ public long getAmountOfDataTransmitted(final ChannelID channelID) { return l.longValue(); } - /** * Returns the userCPU. * @@ -235,33 +222,28 @@ public long getAmountOfDataTransmitted(final ChannelID channelID) { public long getUserCPU() { return this.userCPU; } - /** - * Returns whether a decision was forced by the user - * @return - */ - public Boolean getForced() { - return forced; - } - + public long getTotalInputAmount() { return this.totalInputAmount; } - public long getTotalOutputAmount() { return this.totalOutputAmount; } + public long getAverageOutputRecordSize() { return averageOutputRecordSize; } + public long getAverageInputRecordSize() { return averageInputRecordSize; } - - public double getPactRatio(){ + + public double getPactRatio() { return this.pactRatio; } - public boolean isDam(){ + + public boolean isDam() { return this.isDam; } } diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 477d57b36369b..804c431bb0183 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -198,6 +198,8 @@ public ExecutionVertex(final JobID jobID, final Class Date: Wed, 7 Mar 2012 19:51:06 +0100 Subject: [PATCH 307/310] Made local checkpoints the default again --- .../eu/stratosphere/nephele/checkpointing/CheckpointUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java index 18e4d57b689a5..bf90de178d1dc 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/CheckpointUtils.java @@ -242,7 +242,7 @@ public static boolean useAVG() { public static boolean createDistributedCheckpoint() { - return GlobalConfiguration.getBoolean("checkpoint.distributed", true); + return GlobalConfiguration.getBoolean("checkpoint.distributed", false); } public static String getSummary() { From 67816468e793b0295281d85bbc6b2bd39f43a8d0 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Wed, 7 Mar 2012 20:46:38 +0100 Subject: [PATCH 308/310] Fixed restart bug in class ReplayThread --- .../nephele/checkpointing/ReplayThread.java | 33 +++++++++++++------ 1 file changed, 23 insertions(+), 10 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index 18060c34642a1..690681ac08cb8 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -63,6 +63,8 @@ final class ReplayThread extends Thread { private final AtomicBoolean restartRequested = new AtomicBoolean(false); + private final AtomicBoolean interruptCalled = new AtomicBoolean(false); + ReplayThread(final ExecutionVertexID vertexID, final ExecutionObserver executionObserver, final String taskName, final boolean isCheckpointLocal, final boolean isCheckpointComplete, final Map outputBrokerMap) { @@ -107,11 +109,7 @@ public void run() { } catch (Exception e) { - if (this.restartRequested.compareAndSet(true, false)) { - // Wait for the thread to be interrupted, then clear interrupted flag - while (!Thread.currentThread().isInterrupted()) { - } - Thread.interrupted(); + if (isRestartRequested()) { continue; } @@ -124,11 +122,7 @@ public void run() { return; } - if (this.restartRequested.compareAndSet(true, false)) { - // Wait for the thread to be interrupted, then clear interrupted flag - while (!Thread.currentThread().isInterrupted()) { - } - Thread.interrupted(); + if (isRestartRequested()) { continue; } @@ -156,6 +150,24 @@ public void run() { changeExecutionState(ExecutionState.FINISHED, null); } + private boolean isRestartRequested() { + + if (this.restartRequested.compareAndSet(true, false)) { + // Check if the interrupt call has already been made + if (!this.interruptCalled.compareAndSet(true, false)) { + // Wait for the thread to be interrupted + while (!Thread.currentThread().isInterrupted()) { + } + this.interruptCalled.set(false); + } + // Clear interrupted flag + Thread.interrupted(); + return true; + } + + return false; + } + private void resetAllOutputBroker() { final Iterator it = this.outputBrokerMap.values().iterator(); @@ -197,6 +209,7 @@ void restart() { changeExecutionState(ExecutionState.STARTING, null); this.restartRequested.set(true); interrupt(); + this.interruptCalled.set(true); } private void replayCheckpoint() throws Exception { From 46c992b19db470c04b1ea61220d66435f6bf9a53 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 8 Mar 2012 11:10:55 +0100 Subject: [PATCH 309/310] Fixed problem with wrong initial checkpoint state --- .../executiongraph/ExecutionGraph.java | 4 +- .../executiongraph/ExecutionGroupVertex.java | 36 ---------- .../executiongraph/ExecutionVertex.java | 69 +++++++++++-------- 3 files changed, 42 insertions(+), 67 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java index bbc9cb31fa951..c86af2001a138 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java @@ -473,8 +473,10 @@ void wire(final ExecutionGroupVertex source, final int indexOfOutputGate, final // Update channel type of output gate outputGate.setChannelType(channelType); - } + // Update the initial checkpoint state + sourceVertex.checkInitialCheckpointState(); + } } @SuppressWarnings({ "unchecked", "rawtypes" }) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java index 4cb2b3138d985..02902060103dc 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java @@ -158,11 +158,6 @@ public class ExecutionGroupVertex { */ private volatile ExecutionStage executionStage = null; - /** - * The initial checkpoint state for vertices assigned to this group vertex. - */ - private volatile CheckpointState initialCheckpointState = null; - /** * The configuration object of the original job vertex. */ @@ -752,37 +747,6 @@ boolean isVertexToShareInstanceWithUserDefined() { return this.userDefinedVertexToShareInstancesWith; } - /** - * Sets the initial checkpoint state for vertices assigned to this group vertex. - * - * @param initialCheckpointState - * the initial checkpoint state for vertices assigned to this group vertex - */ - void setInitialCheckpointState(final CheckpointState initialCheckpointState) { - - if (initialCheckpointState == null) { - throw new IllegalArgumentException("Argument initialCheckpointState must not be null"); - } - - if (this.initialCheckpointState != null) { - throw new IllegalStateException("Initial checkpoint state for group vertex " + getName() - + " is already set"); - } - - this.initialCheckpointState = initialCheckpointState; - } - - /** - * Returns the initial checkpoint state for vertices assigned to this group vertex. - * - * @return the initial checkpoint state for vertices assigned to this group vertex or null if the state - * is not yet set - */ - CheckpointState getInitialCheckpointState() { - - return this.initialCheckpointState; - } - /** * Returns the configuration object of the original job vertex. * diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java index 804c431bb0183..28c7d6e424849 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java @@ -139,7 +139,8 @@ public final class ExecutionVertex { /** * The current checkpoint state of this vertex. */ - private final AtomicEnum checkpointState; + private final AtomicEnum checkpointState = new AtomicEnum( + CheckpointState.UNDECIDED); /** * The execution pipeline this vertex is part of. @@ -177,33 +178,7 @@ public ExecutionVertex(final JobID jobID, final Class(groupVertex.getInitialCheckpointState()); - this.retriesLeft = new AtomicInteger(groupVertex.getNumberOfExecutionRetries()); // Register the vertex itself as a listener for state changes @@ -290,6 +263,9 @@ public ExecutionVertex duplicateVertex(final boolean preserveVertexID) throws Ex final ExecutionVertex duplicatedVertex = new ExecutionVertex(newVertexID, this.invokableClass, this.executionGraph, this.groupVertex, duplicatedEnvironment); + // Copy checkpoint state from original vertex + duplicatedVertex.checkpointState.set(this.checkpointState.get()); + // TODO set new profiling record with new vertex id duplicatedVertex.setAllocatedResource(this.allocatedResource); @@ -950,4 +926,37 @@ public ExecutionPipeline getExecutionPipeline() { return this.executionPipeline.get(); } + + /** + * Checks and if necessary silently updates the initial checkpoint state of the vertex. + */ + void checkInitialCheckpointState() { + + // Determine the vertex' initial checkpoint state + CheckpointState ics = CheckpointState.UNDECIDED; + boolean hasFileChannels = false; + for (int i = 0; i < this.environment.getNumberOfOutputGates(); ++i) { + if (this.environment.getOutputGate(i).getChannelType() == ChannelType.FILE) { + hasFileChannels = true; + break; + } + } + + // The vertex has at least one file channel, so we must write a checkpoint anyways + if (hasFileChannels) { + ics = CheckpointState.PARTIAL; + } else { + // Look for a user annotation + ForceCheckpoint forcedCheckpoint = this.environment.getInvokable().getClass() + .getAnnotation(ForceCheckpoint.class); + + if (forcedCheckpoint != null) { + ics = forcedCheckpoint.checkpoint() ? CheckpointState.PARTIAL : CheckpointState.NONE; + } + + // TODO: Consider state annotation here + } + + this.checkpointState.set(ics); + } } From bdeab0f61cdeec05fb7cb6fcd1743b27a915d6d2 Mon Sep 17 00:00:00 2001 From: Daniel Warneke Date: Thu, 8 Mar 2012 11:33:22 +0100 Subject: [PATCH 310/310] Fixed problem with missing thread interruption in class ReplayThread --- .../eu/stratosphere/nephele/checkpointing/ReplayThread.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java index 690681ac08cb8..993c34b0c92e6 100644 --- a/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java +++ b/nephele/nephele-server/src/main/java/eu/stratosphere/nephele/checkpointing/ReplayThread.java @@ -155,8 +155,9 @@ private boolean isRestartRequested() { if (this.restartRequested.compareAndSet(true, false)) { // Check if the interrupt call has already been made if (!this.interruptCalled.compareAndSet(true, false)) { - // Wait for the thread to be interrupted - while (!Thread.currentThread().isInterrupted()) { + try { + Thread.sleep(10L); + } catch (InterruptedException e) { } this.interruptCalled.set(false); }