From 2ac08a6c88930d04dbc6287490e8e3df999a32e6 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 8 Sep 2014 23:29:42 +0200 Subject: [PATCH] Remove management graph and simplify historic job status --- .../client/CliFrontendListCancelTest.java | 6 - .../org/apache/flink/util/ExceptionUtils.java | 6 + .../apache/flink/util/InstantiationUtil.java | 2 - .../org/apache/flink/util/StringUtils.java | 54 +- .../core/io/LocatableSplitAssignerTest.java | 2 - .../runtime/event/job/AbstractEvent.java | 90 +- .../event/job/ExecutionStateChangeEvent.java | 116 +-- .../flink/runtime/event/job/JobEvent.java | 91 +- .../runtime/event/job/RecentJobEvent.java | 134 ++- .../flink/runtime/event/job/VertexEvent.java | 155 ++-- .../runtime/execution/RuntimeEnvironment.java | 6 +- .../runtime/executiongraph/ExecutionEdge.java | 90 -- .../executiongraph/ExecutionGraph.java | 53 +- .../executiongraph/ExecutionGroupEdge.java | 202 ----- .../executiongraph/ExecutionGroupVertex.java | 774 ------------------ .../executiongraph/ExecutionJobVertex.java | 25 +- .../executiongraph/ExecutionVertex2.java | 13 +- .../executiongraph/IntermediateResult.java | 8 + .../IntermediateResultPartition.java | 1 - .../ManagementGraphFactory.java | 181 ---- .../flink/runtime/instance/Hardware.java | 6 +- .../runtime/instance/InstanceManager.java | 8 +- .../instance/LocalInstanceManager.java | 4 +- .../runtime/io/disk/iomanager/IOManager.java | 9 +- .../flink/runtime/jobgraph/JobVertexID.java | 15 +- .../runtime/jobmanager/EventCollector.java | 83 +- .../flink/runtime/jobmanager/JobManager.java | 63 +- .../jobmanager/archive/ArchiveListener.java | 62 +- .../jobmanager/archive/MemoryArchivist.java | 143 ++-- .../jobmanager/web/JobmanagerInfoServlet.java | 185 ++--- .../runtime/jobmanager/web/JsonFactory.java | 114 +++ .../jobmanager/web/SetupInfoServlet.java | 54 +- .../managementgraph/ManagementAttachment.java | 54 -- .../managementgraph/ManagementEdge.java | 166 ---- .../managementgraph/ManagementEdgeID.java | 55 -- .../managementgraph/ManagementGate.java | 219 ----- .../managementgraph/ManagementGateID.java | 28 - .../managementgraph/ManagementGraph.java | 562 ------------- .../ManagementGraphIterator.java | 456 ----------- .../managementgraph/ManagementGroupEdge.java | 131 --- .../ManagementGroupVertex.java | 485 ----------- .../ManagementGroupVertexID.java | 52 -- .../ManagementGroupVertexIterator.java | 330 -------- .../managementgraph/ManagementStage.java | 341 -------- .../managementgraph/ManagementVertex.java | 321 -------- .../managementgraph/ManagementVertexID.java | 40 - .../impl/EnvironmentListenerImpl.java | 3 +- .../profiling/impl/EnvironmentThreadSet.java | 127 +-- .../impl/JobManagerProfilerImpl.java | 80 +- .../profiling/impl/JobProfilingData.java | 80 +- .../impl/TaskManagerProfilerImpl.java | 5 +- .../InternalExecutionVertexProfilingData.java | 44 +- ...nalExecutionVertexThreadProfilingData.java | 5 +- .../types/InternalInputGateProfilingData.java | 95 --- .../types/InternalInstanceProfilingData.java | 2 - .../InternalOutputGateProfilingData.java | 94 --- .../types/InputGateProfilingEvent.java | 152 ---- .../types/InstanceProfilingEvent.java | 110 +-- .../types/InstanceSummaryProfilingEvent.java | 30 +- .../types/OutputGateProfilingEvent.java | 150 ---- .../profiling/types/ProfilingEvent.java | 70 +- .../types/SingleInstanceProfilingEvent.java | 55 +- .../profiling/types/ThreadProfilingEvent.java | 65 +- .../profiling/types/VertexProfilingEvent.java | 88 +- .../protocols/ExtendedManagementProtocol.java | 13 - .../runtime/taskmanager/TaskManager.java | 54 +- .../apache/flink/runtime/util/AtomicEnum.java | 95 --- .../flink/runtime/util/AtomicEnumerator.java | 57 -- .../runtime/util/EnvironmentInformation.java | 23 +- .../runtime/util/ExecutorThreadFactory.java | 6 +- .../flink/runtime/event/job/JobEventTest.java | 57 -- .../event/job/ManagementEventTest.java | 85 -- .../instance/LocalInstanceManagerTest.java | 101 +-- .../org/apache/flink/runtime/ipc/RpcTest.java | 8 - .../managementgraph/ManagementGraphTest.java | 453 ---------- .../profiling/types/ProfilingTypesTest.java | 122 +-- .../runtime/taskmanager/TaskManagerTest.java | 9 +- 77 files changed, 1218 insertions(+), 7115 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupEdge.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupVertex.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ManagementGraphFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementAttachment.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdge.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdgeID.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGate.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGateID.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGraph.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGraphIterator.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupEdge.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertex.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexID.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexIterator.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementStage.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertex.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInputGateProfilingData.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalOutputGateProfilingData.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InputGateProfilingEvent.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/OutputGateProfilingEvent.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicEnum.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicEnumerator.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/event/job/JobEventTest.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/event/job/ManagementEventTest.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/managementgraph/ManagementGraphTest.java diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java index 593916fbe4d..2f311817213 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java @@ -34,7 +34,6 @@ import org.apache.flink.runtime.event.job.AbstractEvent; import org.apache.flink.runtime.event.job.RecentJobEvent; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.managementgraph.ManagementGraph; import org.apache.flink.runtime.protocols.ExtendedManagementProtocol; import org.apache.flink.runtime.types.IntegerRecord; import org.junit.Assert; @@ -173,11 +172,6 @@ public class CliFrontendListCancelTest { throw new UnsupportedOperationException(); } - @Override - public ManagementGraph getManagementGraph(JobID jobID) throws IOException { - throw new UnsupportedOperationException(); - } - @Override public List getRecentJobs() throws IOException { return new ArrayList(); diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java index 8f9822e278c..fd4b37571f5 100644 --- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java @@ -16,6 +16,12 @@ * limitations under the License. */ +// +// The function "stringifyException" is based on source code from the Hadoop Project (http://hadoop.apache.org/), +// licensed by the Apache Software Foundation (ASF) under the Apache License, Version 2.0. +// See the NOTICE file distributed with this work for additional information regarding copyright ownership. +// + package org.apache.flink.util; import java.io.PrintWriter; diff --git a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java index f1a8a9458f3..678ef111794 100644 --- a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.util; import java.io.ByteArrayInputStream; @@ -31,7 +30,6 @@ import java.lang.reflect.Modifier; import org.apache.flink.configuration.Configuration; - /** * Utility class to create instances from class objects and checking failure reasons. */ diff --git a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java index 2699c9e2e7a..4473c8c8dbb 100644 --- a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java @@ -16,20 +16,16 @@ * limitations under the License. */ - -/** - * This file is based on source code from the Hadoop Project (http://hadoop.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. - */ - package org.apache.flink.util; -import java.io.PrintWriter; -import java.io.StringWriter; +import java.io.IOException; import java.util.Arrays; import java.util.Random; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.types.StringValue; + /** * Utility class to convert objects into strings in vice-versa. */ @@ -48,11 +44,7 @@ public final class StringUtils { * @return A string with exception name and call stack. */ public static String stringifyException(final Throwable e) { - final StringWriter stm = new StringWriter(); - final PrintWriter wrt = new PrintWriter(stm); - e.printStackTrace(wrt); - wrt.close(); - return stm.toString(); + return ExceptionUtils.stringifyException(e); } /** @@ -71,6 +63,7 @@ public final class StringUtils { if (bytes == null) { throw new IllegalArgumentException("bytes == null"); } + final StringBuilder s = new StringBuilder(); for (int i = start; i < end; i++) { s.append(String.format("%02x", bytes[i])); @@ -302,4 +295,37 @@ public final class StringUtils { } return new String(data); } + + /** + * Writes a String to the given output. The string may be null. + * The written string can be read with {@link #readNullableString(DataInputView)}- + * + * @param str The string to write, or null. + * @param out The output to write to. + * @throws IOException Throws if the writing or the serialization fails. + */ + public static void writeNullableString(String str, DataOutputView out) throws IOException { + if (str != null) { + out.writeBoolean(true); + StringValue.writeString(str, out); + } else { + out.writeBoolean(false); + } + } + + /** + * Reads a String from the given input. The string may be null and must have been written with + * {@link #writeNullableString(String, DataOutputView)}. + * + * @param in The input to read from. + * @return The deserialized string, or null. + * @throws IOException Throws if the reading or the deserialization fails. + */ + public static String readNullableString(DataInputView in) throws IOException { + if (in.readBoolean()) { + return StringValue.readString(in); + } else { + return null; + } + } } diff --git a/flink-core/src/test/java/org/apache/flink/core/io/LocatableSplitAssignerTest.java b/flink-core/src/test/java/org/apache/flink/core/io/LocatableSplitAssignerTest.java index aa56c1ce7f2..fd43fa0d752 100644 --- a/flink-core/src/test/java/org/apache/flink/core/io/LocatableSplitAssignerTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/io/LocatableSplitAssignerTest.java @@ -27,9 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.flink.api.common.io.LocatableInputSplitAssigner; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.LocatableInputSplit; -import org.apache.flink.util.LogUtils; -import org.junit.BeforeClass; import org.junit.Test; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/AbstractEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/AbstractEvent.java index d7e47129a43..ba072730aa0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/AbstractEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/AbstractEvent.java @@ -31,7 +31,9 @@ import org.apache.flink.core.memory.DataOutputView; * An abstract event is transmitted from the job manager to the * job client in order to inform the user about the job progress */ -public abstract class AbstractEvent implements IOReadableWritable { +public abstract class AbstractEvent implements IOReadableWritable, java.io.Serializable { + + private static final long serialVersionUID = 1L; /** Static variable that points to the current global sequence number */ private static final AtomicLong GLOBAL_SEQUENCE_NUMBER = new AtomicLong(0); @@ -39,12 +41,21 @@ public abstract class AbstractEvent implements IOReadableWritable { /** Auxiliary object which helps to convert a {@link Date} object to the given string representation. */ private static final SimpleDateFormat DATA_FORMATTER = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss"); + /** The timestamp of the event. */ private long timestamp = -1; /** The sequence number of the event. */ private long sequenceNumber = -1; + // -------------------------------------------------------------------------------------------- + + /** + * Constructs a new abstract event object. This constructor + * is required for the deserialization process and is not + * supposed to be called directly. + */ + public AbstractEvent() {} /** * Constructs a new abstract event object. * @@ -56,17 +67,27 @@ public abstract class AbstractEvent implements IOReadableWritable { this.sequenceNumber = GLOBAL_SEQUENCE_NUMBER.incrementAndGet(); } + // -------------------------------------------------------------------------------------------- + /** + * Returns the timestamp of the event. + * + * @return the timestamp of the event + */ + public long getTimestamp() { + return this.timestamp; + } + + public String getTimestampString() { + return timestampToString(timestamp); + } + public long getSequenceNumber() { return this.sequenceNumber; } - - /** - * Constructs a new abstract event object. This constructor - * is required for the deserialization process and is not - * supposed to be called directly. - */ - public AbstractEvent() {} - + + // -------------------------------------------------------------------------------------------- + // Serialization + // -------------------------------------------------------------------------------------------- @Override public void read(DataInputView in) throws IOException { @@ -80,15 +101,33 @@ public abstract class AbstractEvent implements IOReadableWritable { out.writeLong(this.sequenceNumber); } - /** - * Returns the timestamp of the event. - * - * @return the timestamp of the event - */ - public long getTimestamp() { - return this.timestamp; + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + @Override + public boolean equals(Object obj) { + if (obj instanceof AbstractEvent) { + final AbstractEvent abstractEvent = (AbstractEvent) obj; + return this.timestamp == abstractEvent.timestamp; + } + else { + return false; + } } + @Override + public int hashCode() { + return (int) (this.timestamp ^ (this.timestamp >>> 32)); + } + + @Override + public String toString() { + return String.format("AbstractEvent #%d at %s", sequenceNumber, timestampToString(timestamp)); + } + + // -------------------------------------------------------------------------------------------- + /** * Converts the timestamp of an event from its "milliseconds since beginning the epoch" * representation into a unified string representation. @@ -100,23 +139,4 @@ public abstract class AbstractEvent implements IOReadableWritable { public static String timestampToString(long timestamp) { return DATA_FORMATTER.format(new Date(timestamp)); } - - - @Override - public boolean equals(final Object obj) { - if (obj instanceof AbstractEvent) { - final AbstractEvent abstractEvent = (AbstractEvent) obj; - if (this.timestamp == abstractEvent.getTimestamp()) { - return true; - } - } - - return false; - } - - - @Override - public int hashCode() { - return (int) (this.timestamp ^ (this.timestamp >>> 32)); - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java index 15bae605b93..f233d497127 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.event.job; import java.io.IOException; @@ -24,21 +23,24 @@ import java.io.IOException; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.execution.ExecutionState2; -import org.apache.flink.runtime.managementgraph.ManagementVertexID; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.jobgraph.JobVertexID; + +import com.google.common.base.Preconditions; /** - * An {@link ExecutionStateChangeEvent} can be used to notify other objects about an execution state change of a vertex. + * An ecutionStateChangeEvent can be used to notify other objects about an execution state change of a vertex. */ public final class ExecutionStateChangeEvent extends AbstractEvent implements ManagementEvent { - /** - * The ID identifies the vertex this events refers to. - */ - private ManagementVertexID managementVertexID; + private static final long serialVersionUID = 1L; + + private JobVertexID vertexId; + + private int subtask; + + private ExecutionAttemptID executionAttemptId; - /** - * The new execution state of the vertex this event refers to. - */ private ExecutionState2 newExecutionState; /** @@ -46,14 +48,24 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma * * @param timestamp * the timestamp of the event - * @param managementVertexID + * @param executionAttemptId * identifies the vertex this event refers to * @param newExecutionState * the new execution state of the vertex this event refers to */ - public ExecutionStateChangeEvent(long timestamp, ManagementVertexID managementVertexID, ExecutionState2 newExecutionState) { + public ExecutionStateChangeEvent(long timestamp, JobVertexID vertexId, int subtask, + ExecutionAttemptID executionAttemptId, ExecutionState2 newExecutionState) + { super(timestamp); - this.managementVertexID = managementVertexID; + + Preconditions.checkNotNull(vertexId); + Preconditions.checkArgument(subtask >= 0); + Preconditions.checkNotNull(executionAttemptId); + Preconditions.checkNotNull(newExecutionState); + + this.vertexId = vertexId; + this.subtask = subtask; + this.executionAttemptId = executionAttemptId; this.newExecutionState = newExecutionState; } @@ -65,17 +77,28 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma public ExecutionStateChangeEvent() { super(); - this.managementVertexID = new ManagementVertexID(); + this.vertexId = new JobVertexID(); + this.executionAttemptId = new ExecutionAttemptID(); this.newExecutionState = ExecutionState2.CREATED; } + // -------------------------------------------------------------------------------------------- + + public JobVertexID getVertexId() { + return vertexId; + } + + public int getSubtaskIndex() { + return subtask; + } + /** * Returns the ID of the vertex this event refers to. * * @return the ID of the vertex this event refers to */ - public ManagementVertexID getVertexID() { - return this.managementVertexID; + public ExecutionAttemptID getExecutionAttemptID() { + return this.executionAttemptId; } /** @@ -87,11 +110,14 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma return this.newExecutionState; } - + // -------------------------------------------------------------------------------------------- + @Override public void read(DataInputView in) throws IOException { super.read(in); - this.managementVertexID.read(in); + this.vertexId.read(in); + this.executionAttemptId.read(in); + this.subtask = in.readInt(); this.newExecutionState = ExecutionState2.values()[in.readInt()]; } @@ -99,46 +125,38 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma @Override public void write(DataOutputView out) throws IOException { super.write(out); - this.managementVertexID.write(out); + this.vertexId.write(out); + this.executionAttemptId.write(out); + out.writeInt(subtask); out.writeInt(this.newExecutionState.ordinal()); } - + // -------------------------------------------------------------------------------------------- + @Override - public boolean equals(final Object obj) { - - if (!super.equals(obj)) { - return false; - } - - if (!(obj instanceof ExecutionStateChangeEvent)) { + public boolean equals(Object obj) { + if (obj instanceof ExecutionStateChangeEvent) { + ExecutionStateChangeEvent other = (ExecutionStateChangeEvent) obj; + + return other.newExecutionState == this.newExecutionState && + other.executionAttemptId.equals(this.executionAttemptId) && + super.equals(obj); + + } else { return false; } - - ExecutionStateChangeEvent stateChangeEvent = (ExecutionStateChangeEvent) obj; - if (!stateChangeEvent.getNewExecutionState().equals(this.newExecutionState)) { - return false; - } - - if (!stateChangeEvent.getVertexID().equals(this.managementVertexID)) { - return false; - } - - return true; } - @Override public int hashCode() { - - if (this.newExecutionState != null) { - return this.newExecutionState.hashCode(); - } - - if (this.managementVertexID != null) { - return this.managementVertexID.hashCode(); - } - - return super.hashCode(); + return super.hashCode() ^ + (127 * newExecutionState.ordinal()) ^ + this.executionAttemptId.hashCode(); + } + + @Override + public String toString() { + return String.format("ExecutionStateChangeEvent %d at %d , executionAttempt=%s, newState=%s", getSequenceNumber(), getTimestamp(), + executionAttemptId, newExecutionState); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/JobEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/JobEvent.java index 2f32686c066..805016dbc0f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/JobEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/JobEvent.java @@ -20,11 +20,10 @@ package org.apache.flink.runtime.event.job; import java.io.IOException; -import org.apache.flink.core.io.StringRecord; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.util.EnumUtils; +import org.apache.flink.util.StringUtils; /** * A job event object is used by the job manager to inform a client about @@ -32,11 +31,13 @@ import org.apache.flink.runtime.util.EnumUtils; */ public class JobEvent extends AbstractEvent { + private static final long serialVersionUID = 1846424770472758893L; + /** The current status of the job. */ private JobStatus currentJobStatus; /** An optional message attached to the event, possibly null. */ - private String optionalMessage = null; + private String optionalMessage; /** * Constructs a new job event object. @@ -48,7 +49,7 @@ public class JobEvent extends AbstractEvent { * @param optionalMessage * an optional message that shall be attached to this event, possibly null */ - public JobEvent(final long timestamp, final JobStatus currentJobStatus, final String optionalMessage) { + public JobEvent(long timestamp, JobStatus currentJobStatus, String optionalMessage) { super(timestamp); this.currentJobStatus = currentJobStatus; @@ -65,30 +66,7 @@ public class JobEvent extends AbstractEvent { this.currentJobStatus = JobStatus.CREATED; } - - @Override - public void read(final DataInputView in) throws IOException { - super.read(in); - - // Read job status - this.currentJobStatus = EnumUtils.readEnum(in, JobStatus.class); - - // Read optional message - this.optionalMessage = StringRecord.readString(in); - } - - - @Override - public void write(final DataOutputView out) throws IOException { - super.write(out); - - // Write job status - EnumUtils.writeEnum(out, this.currentJobStatus); - - // Write optional message - StringRecord.writeString(out, this.optionalMessage); - } - + /** * Returns the current status of the job. * @@ -107,43 +85,50 @@ public class JobEvent extends AbstractEvent { return this.optionalMessage; } + // -------------------------------------------------------------------------------------------- + // Serialization + // -------------------------------------------------------------------------------------------- + + @Override + public void read(final DataInputView in) throws IOException { + super.read(in); - public String toString() { - return timestampToString(getTimestamp()) + ":\tJob execution switched to status " + this.currentJobStatus; + this.currentJobStatus = JobStatus.values()[in.readInt()]; + this.optionalMessage = StringUtils.readNullableString(in); } - @Override - public boolean equals(final Object obj) { - if (!super.equals(obj)) { - return false; - } - - if (!(obj instanceof JobEvent)) { - return false; - } + public void write(final DataOutputView out) throws IOException { + super.write(out); - final JobEvent jobEvent = (JobEvent) obj; + out.writeInt(this.currentJobStatus.ordinal()); + StringUtils.writeNullableString(optionalMessage, out); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- - if (!this.currentJobStatus.equals(jobEvent.getCurrentJobStatus())) { - return false; + @Override + public boolean equals(Object obj) { + if (obj instanceof JobEvent) { + JobEvent other = (JobEvent) obj; + + return super.equals(other) && this.currentJobStatus == other.currentJobStatus && + this.optionalMessage == null ? other.optionalMessage == null : + (other.optionalMessage != null && this.optionalMessage.equals(other.optionalMessage)); } - - if (this.optionalMessage == null) { - - if (jobEvent.getOptionalMessage() == null) { - return true; - } else { - return false; - } + else { + return false; } - - return this.optionalMessage.equals(jobEvent.getOptionalMessage()); } - @Override public int hashCode() { return super.hashCode(); } + + public String toString() { + return timestampToString(getTimestamp()) + ":\tJob execution switched to status " + this.currentJobStatus; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/RecentJobEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/RecentJobEvent.java index 376d8af0948..17f25ef9efd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/RecentJobEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/RecentJobEvent.java @@ -16,49 +16,41 @@ * limitations under the License. */ - package org.apache.flink.runtime.event.job; import java.io.IOException; -import org.apache.flink.core.io.StringRecord; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.util.EnumUtils; +import org.apache.flink.util.StringUtils; + +import com.google.common.base.Preconditions; /** * A {@link RecentJobEvent} provides a summary of a job which is either currently running or has been running recently. - * */ public final class RecentJobEvent extends AbstractEvent implements ManagementEvent { - /** - * The ID of the new job. - */ + private static final long serialVersionUID = -3361778351490181333L; + + /** The ID of the new job. */ private JobID jobID; - /** - * The name of the new job. - */ + /** The name of the new job. */ private String jobName; - /** - * The last known status of the job. - */ + /** The last known status of the job. */ private JobStatus jobStatus; - /** - * true if profiling is enabled for this job, false otherwise. - */ + /** true if profiling is enabled for this job, false otherwise. */ private boolean isProfilingEnabled; - /** - * The time stamp of the job submission. - */ + /** The time stamp of the job submission. */ private long submissionTimestamp; + /** * Constructs a new event. * @@ -75,13 +67,12 @@ public final class RecentJobEvent extends AbstractEvent implements ManagementEve * @param timestamp * the time stamp of the event */ - public RecentJobEvent(final JobID jobID, final String jobName, final JobStatus jobStatus, - final boolean isProfilingEnabled, final long submissionTimestamp, final long timestamp) { + public RecentJobEvent(JobID jobID, String jobName, JobStatus jobStatus, + boolean isProfilingEnabled, long submissionTimestamp, long timestamp) { super(timestamp); - if (jobStatus == null) { - throw new IllegalArgumentException("job status must not be null"); - } + Preconditions.checkNotNull(jobID); + Preconditions.checkNotNull(jobStatus); this.jobID = jobID; this.jobName = jobName; @@ -95,8 +86,11 @@ public final class RecentJobEvent extends AbstractEvent implements ManagementEve */ public RecentJobEvent() { super(); + this.jobID = new JobID(); } + // -------------------------------------------------------------------------------------------- + /** * Returns the ID of the new job. * @@ -139,98 +133,62 @@ public final class RecentJobEvent extends AbstractEvent implements ManagementEve * @return the time stamp of the job submission */ public long getSubmissionTimestamp() { - return this.submissionTimestamp; } + // -------------------------------------------------------------------------------------------- + // Serialization + // -------------------------------------------------------------------------------------------- @Override public void read(final DataInputView in) throws IOException { super.read(in); - - // Read the job ID - this.jobID = new JobID(); + this.jobID.read(in); - - // Read the job name - this.jobName = StringRecord.readString(in); - - // Read the job status - this.jobStatus = EnumUtils.readEnum(in, JobStatus.class); - - // Read if profiling is enabled + this.jobName = StringUtils.readNullableString(in); + this.jobStatus = JobStatus.values()[in.readInt()]; this.isProfilingEnabled = in.readBoolean(); - - // Read the submission time stamp this.submissionTimestamp = in.readLong(); } - @Override public void write(final DataOutputView out) throws IOException { super.write(out); - // Write the job ID this.jobID.write(out); - - // Write the job name - StringRecord.writeString(out, this.jobName); - - // Writes the job status - EnumUtils.writeEnum(out, this.jobStatus); - - // Write out if profiling is enabled + StringUtils.writeNullableString(jobName, out); + out.writeInt(jobStatus.ordinal()); out.writeBoolean(this.isProfilingEnabled); - - // Write out the submission time stamp out.writeLong(this.submissionTimestamp); } - + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + @Override - public boolean equals(final Object obj) { - - if (!super.equals(obj)) { - return false; - } - - if (!(obj instanceof RecentJobEvent)) { - return false; - } - - final RecentJobEvent newJobEvent = (RecentJobEvent) obj; - - if (!this.jobID.equals(newJobEvent.getJobID())) { - return false; - } - - if (!this.jobName.equals(newJobEvent.getJobName())) { - return false; + public boolean equals(Object obj) { + if (obj instanceof RecentJobEvent) { + final RecentJobEvent other = (RecentJobEvent) obj; + return super.equals(other) && this.jobID.equals(other.jobID) && this.isProfilingEnabled == other.isProfilingEnabled && + this.submissionTimestamp == other.submissionTimestamp && + (this.jobName == null ? other.jobName == null : (other.jobName != null && + this.jobName.equals(other.jobName))); } - - if (this.isProfilingEnabled != newJobEvent.isProfilingAvailable()) { - return false; - } - - if (this.submissionTimestamp != newJobEvent.getSubmissionTimestamp()) { + else { return false; } - - return true; } - @Override public int hashCode() { - - if (this.jobID != null) { - return this.jobID.hashCode(); - } - - if (this.jobName != null) { - return this.jobName.hashCode(); - } - - return super.hashCode(); + return super.hashCode() ^ jobID.hashCode() ^ jobStatus.ordinal(); + } + + @Override + public String toString() { + return String.format("RecentJobEvent #%d at %s - jobId=%s, jobName=%s, status=%s, jobSubmission=%s, profiling=%s", + getSequenceNumber(), getTimestampString(), jobID, jobName, jobStatus, timestampToString(submissionTimestamp), + isProfilingEnabled); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java index fb3b2476e81..c697aa44cc3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java @@ -20,11 +20,14 @@ package org.apache.flink.runtime.event.job; import java.io.IOException; -import org.apache.flink.core.io.StringRecord; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.execution.ExecutionState2; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.util.StringUtils; + +import com.google.common.base.Preconditions; /** * Vertex events are transmitted from the job manager to the job client in order to inform the user about @@ -32,6 +35,8 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; */ public class VertexEvent extends AbstractEvent { + private static final long serialVersionUID = -521556020344465262L; + /** The ID of the job vertex this event belongs to. */ private JobVertexID jobVertexID; @@ -43,6 +48,9 @@ public class VertexEvent extends AbstractEvent { /** The index of the subtask that this event belongs to. */ private int indexOfSubtask; + + /** The id of the execution attempt */ + private ExecutionAttemptID executionAttemptId; /** The current execution state of the subtask this event belongs to. */ private ExecutionState2 currentExecutionState; @@ -69,15 +77,19 @@ public class VertexEvent extends AbstractEvent { * an optional description */ public VertexEvent(long timestamp, JobVertexID jobVertexID, String jobVertexName, - int totalNumberOfSubtasks, int indexOfSubtask, ExecutionState2 currentExecutionState, - String description) + int totalNumberOfSubtasks, int indexOfSubtask, ExecutionAttemptID executionAttemptId, + ExecutionState2 currentExecutionState, String description) { super(timestamp); + Preconditions.checkNotNull(jobVertexID); + Preconditions.checkNotNull(currentExecutionState); + this.jobVertexID = jobVertexID; this.jobVertexName = jobVertexName; this.totalNumberOfSubtasks = totalNumberOfSubtasks; this.indexOfSubtask = indexOfSubtask; + this.executionAttemptId = executionAttemptId; this.currentExecutionState = currentExecutionState; this.description = description; } @@ -91,39 +103,10 @@ public class VertexEvent extends AbstractEvent { super(); this.jobVertexID = new JobVertexID(); - this.jobVertexName = null; this.totalNumberOfSubtasks = -1; this.indexOfSubtask = -1; + this.executionAttemptId = new ExecutionAttemptID(); this.currentExecutionState = ExecutionState2.CREATED; - this.description = null; - } - - - @Override - public void read(final DataInputView in) throws IOException { - - super.read(in); - - this.jobVertexID.read(in); - this.jobVertexName = StringRecord.readString(in); - this.totalNumberOfSubtasks = in.readInt(); - this.indexOfSubtask = in.readInt(); - this.currentExecutionState = ExecutionState2.values()[in.readInt()]; - this.description = StringRecord.readString(in); - } - - - @Override - public void write(final DataOutputView out) throws IOException { - - super.write(out); - - this.jobVertexID.write(out); - StringRecord.writeString(out, this.jobVertexName); - out.writeInt(this.totalNumberOfSubtasks); - out.writeInt(this.indexOfSubtask); - out.writeInt(this.currentExecutionState.ordinal()); - StringRecord.writeString(out, this.description); } /** @@ -181,72 +164,76 @@ public class VertexEvent extends AbstractEvent { public String getDescription() { return description; } - - - public String toString() { - - return timestampToString(getTimestamp()) + ":\t" + this.jobVertexName + " (" + (this.indexOfSubtask + 1) + "/" - + this.totalNumberOfSubtasks + ") switched to " + this.currentExecutionState - + ((this.description != null) ? ("\n" + this.description) : ""); + + public ExecutionAttemptID getExecutionAttemptId() { + return executionAttemptId; } - + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + @Override - public boolean equals(final Object obj) { - - if (!super.equals(obj)) { - return false; - } + public void read(DataInputView in) throws IOException { + super.read(in); - if (!(obj instanceof VertexEvent)) { - return false; - } + this.jobVertexID.read(in); + this.executionAttemptId.read(in); + this.totalNumberOfSubtasks = in.readInt(); + this.indexOfSubtask = in.readInt(); + this.currentExecutionState = ExecutionState2.values()[in.readInt()]; + this.jobVertexName = StringUtils.readNullableString(in); + this.description = StringUtils.readNullableString(in); + } - final VertexEvent vertexEvent = (VertexEvent) obj; - if (!this.jobVertexID.equals(vertexEvent.getJobVertexID())) { - return false; - } + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); - if (this.jobVertexName != null && vertexEvent.getJobVertexName() != null) { - if (!this.jobVertexName.equals(vertexEvent.getJobVertexName())) { - return false; - } - } else { - if (this.jobVertexName != vertexEvent.getJobVertexName()) { - return false; - } - } + this.jobVertexID.write(out); + this.executionAttemptId.write(out); + out.writeInt(this.totalNumberOfSubtasks); + out.writeInt(this.indexOfSubtask); + out.writeInt(this.currentExecutionState.ordinal()); + StringUtils.writeNullableString(jobVertexName, out); + StringUtils.writeNullableString(description, out); + } - if (this.totalNumberOfSubtasks != vertexEvent.getTotalNumberOfSubtasks()) { - return false; - } + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- - if (this.indexOfSubtask != vertexEvent.getIndexOfSubtask()) { - return false; + @Override + public boolean equals(Object obj) { + if (obj instanceof VertexEvent) { + final VertexEvent other = (VertexEvent) obj; + + return super.equals(other) && + this.jobVertexID.equals(other.jobVertexID) && + this.totalNumberOfSubtasks == other.totalNumberOfSubtasks && + this.indexOfSubtask == other.indexOfSubtask && + this.currentExecutionState == other.currentExecutionState && + + (this.jobVertexName == null ? other.jobVertexName == null : + (other.jobVertexName != null && this.jobVertexName.equals(other.jobVertexName))) && + + (this.description == null ? other.description == null : + (other.description != null && this.description.equals(other.description))); } - - if (!this.currentExecutionState.equals(vertexEvent.getCurrentExecutionState())) { + else { return false; } - - if (this.description != null && vertexEvent.getDescription() != null) { - - if (!this.description.equals(vertexEvent.getDescription())) { - return false; - } - } else { - if (this.description != vertexEvent.getDescription()) { - return false; - } - } - - return true; } - @Override public int hashCode() { - return super.hashCode(); + return super.hashCode() ^ jobVertexID.hashCode() ^ (31*indexOfSubtask) ^ (17*currentExecutionState.ordinal()); + } + + public String toString() { + return timestampToString(getTimestamp()) + ":\t" + this.jobVertexName + " (" + (this.indexOfSubtask + 1) + "/" + + this.totalNumberOfSubtasks + ") switched to " + this.currentExecutionState + + ((this.description != null) ? ("\n" + this.description) : ""); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java index cb7b290e327..eef081c0ec5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java @@ -32,8 +32,6 @@ import java.util.Set; import java.util.concurrent.FutureTask; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.io.IOReadableWritable; @@ -58,6 +56,8 @@ import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.protocols.AccumulatorProtocol; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.google.common.base.Preconditions; @@ -65,7 +65,7 @@ import com.google.common.base.Preconditions; public class RuntimeEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner, Runnable { /** The log object used for debugging. */ - private static final Log LOG = LogFactory.getLog(RuntimeEnvironment.class); + private static final Logger LOG = LoggerFactory.getLogger(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; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java deleted file mode 100644 index 81523848d93..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java +++ /dev/null @@ -1,90 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.executiongraph; - -import org.apache.flink.runtime.io.network.channels.ChannelID; -import org.apache.flink.runtime.io.network.channels.ChannelType; - -/** - * Objects of this class represent a pair of {@link org.apache.flink.runtime.io.network.channels.InputChannel} and - * {@link org.apache.flink.runtime.io.network.channels.OutputChannel} objects - * within an {@link ExecutionGraph}, Nephele's internal scheduling representation for jobs. - */ -public final class ExecutionEdge { - - private final ExecutionGroupEdge groupEdge; - - private final ExecutionGate outputGate; - - private final ExecutionGate inputGate; - - private final ChannelID outputChannelID; - - private final ChannelID inputChannelID; - - private final int outputGateIndex; - - private final int inputGateIndex; - - ExecutionEdge(final ExecutionGate outputGate, final ExecutionGate inputGate, final ExecutionGroupEdge groupEdge, - final ChannelID outputChannelID, final ChannelID inputChannelID, final int outputGateIndex, - final int inputGateIndex) { - - this.outputGate = outputGate; - this.inputGate = inputGate; - this.groupEdge = groupEdge; - this.outputChannelID = outputChannelID; - this.inputChannelID = inputChannelID; - this.outputGateIndex = outputGateIndex; - this.inputGateIndex = inputGateIndex; - } - - public ExecutionGate getInputGate() { - return this.inputGate; - } - - public ExecutionGate getOutputGate() { - return this.outputGate; - } - - public ChannelID getOutputChannelID() { - return this.outputChannelID; - } - - public ChannelID getInputChannelID() { - return this.inputChannelID; - } - - public int getOutputGateIndex() { - return this.outputGateIndex; - } - - public int getInputGateIndex() { - return this.inputGateIndex; - } - - public ChannelType getChannelType() { - return this.groupEdge.getChannelType(); - } - - public int getConnectionID() { - return this.groupEdge.getConnectionID(); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index da1d28c998a..4c0c7308dc3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -21,9 +21,11 @@ package org.apache.flink.runtime.executiongraph; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collections; +import java.util.ConcurrentModificationException; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; @@ -50,6 +52,8 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler; import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import com.google.common.base.Preconditions; + public class ExecutionGraph { @@ -73,6 +77,8 @@ public class ExecutionGraph { /** All job vertices that are part of this graph */ private final ConcurrentHashMap tasks; + private final List verticesInCreationOrder; + /** All intermediate results that are part of this graph */ private final ConcurrentHashMap intermediateResults; @@ -94,6 +100,8 @@ public class ExecutionGraph { private volatile JobStatus state = JobStatus.CREATED; + private final long[] stateTimestamps; + public ExecutionGraph(JobID jobId, String jobName, Configuration jobConfig) { @@ -112,10 +120,13 @@ public class ExecutionGraph { this.tasks = new ConcurrentHashMap(); this.intermediateResults = new ConcurrentHashMap(); + this.verticesInCreationOrder = new ArrayList(); this.userCodeJarFiles = new ArrayList(); this.jobStatusListeners = new CopyOnWriteArrayList(); this.executionListeners = new CopyOnWriteArrayList(); + + this.stateTimestamps = new long[JobStatus.values().length]; } // -------------------------------------------------------------------------------------------- @@ -145,6 +156,8 @@ public class ExecutionGraph { res.getId(), res, previousDataSet)); } } + + this.verticesInCreationOrder.add(ejv); } } @@ -182,6 +195,40 @@ public class ExecutionGraph { return Collections.unmodifiableMap(this.tasks); } + public Iterable getVerticesTopologically() { + // we return a specific iterator that does not fail with concurrent modifications + // the list is append only, so it is safe for that + final int numElements = this.verticesInCreationOrder.size(); + + return new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + private int pos = 0; + + @Override + public boolean hasNext() { + return pos < numElements; + } + + @Override + public ExecutionJobVertex next() { + if (hasNext()) { + return verticesInCreationOrder.get(pos++); + } else { + throw new NoSuchElementException(); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + }; + } + public Map getAllIntermediateResults() { return Collections.unmodifiableMap(this.intermediateResults); } @@ -190,11 +237,15 @@ public class ExecutionGraph { return new Iterable() { @Override public Iterator iterator() { - return new AllVerticesIterator(tasks.values().iterator()); + return new AllVerticesIterator(getVerticesTopologically().iterator()); } }; } + public long getStatusTimestamp(JobStatus status) { + return this.stateTimestamps[status.ordinal()]; + } + public boolean isQueuedSchedulingAllowed() { return this.allowQueuedScheduling; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupEdge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupEdge.java deleted file mode 100644 index 89e05e2b08e..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupEdge.java +++ /dev/null @@ -1,202 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.executiongraph; - -import org.apache.flink.runtime.io.network.channels.ChannelType; -import org.apache.flink.runtime.jobgraph.DistributionPattern; - -/** - * An execution group edge represents an edge between two execution group vertices. - *

- * This class is thread-safe. - * - */ -public class ExecutionGroupEdge { - - /** - * Stores if the channel type has been specified by the user. - */ - private final boolean userDefinedChannelType; - - /** - * The channel type to be used between the execution vertices of the two connected group vertices. - */ - private volatile ChannelType channelType; - - /** - * The edge's connection ID. The connection ID determines to which physical TCP connection channels represented by - * this edge will be mapped in case the edge's channel type is NETWORK. - */ - private volatile int connectionID; - - /** - * The group vertex connected to this edge. - */ - private final ExecutionGroupVertex targetVertex; - - /** - * The group vertex this edge starts from. - */ - private final ExecutionGroupVertex sourceVertex; - - /** - * The index of the producing task's output gate. - */ - private final int indexOfOutputGate; - - /** - * The index of the consuming task's input gate. - */ - private final int indexOfInputGate; - - /** - * The distribution pattern used to connect the vertices within two groups. - */ - private final DistributionPattern distributionPattern; - - /** - * Constructs a new group edge. - * - * @param sourceVertex - * the source vertex this edge originates from - * @param indexOfOutputGate - * the index of the source vertex's output gate this edge originates from - * @param targetVertex - * the group vertex to be connected - * @param indexOfInputGate - * the index of the consuming task's input gate - * @param channelType - * the channel type for the edge - * @param userDefinedChannelType - * true if the channel type has been specified by the user, false otherwise - * @param distributionPattern - * the distribution pattern to create the wiring - */ - public ExecutionGroupEdge(final ExecutionGroupVertex sourceVertex, final int indexOfOutputGate, - final ExecutionGroupVertex targetVertex, final int indexOfInputGate, final ChannelType channelType, - final boolean userDefinedChannelType, final DistributionPattern distributionPattern) { - this.sourceVertex = sourceVertex; - this.indexOfOutputGate = indexOfOutputGate; - this.channelType = channelType; - this.indexOfInputGate = indexOfInputGate; - this.userDefinedChannelType = userDefinedChannelType; - this.targetVertex = targetVertex; - this.distributionPattern = distributionPattern; - } - - /** - * Returns the channel type assigned to this edge. - * - * @return the channel type assigned to this edge - */ - public ChannelType getChannelType() { - return this.channelType; - } - - /** - * Changes the channel type for this edge. - * - * @param newChannelType - * the channel type for this edge - * @throws GraphConversionException - * thrown if the new channel type violates a user setting - */ - void changeChannelType(final ChannelType newChannelType) throws GraphConversionException { - - if (!this.channelType.equals(newChannelType) && this.userDefinedChannelType) { - throw new GraphConversionException("Cannot overwrite user defined channel type"); - } - - this.channelType = newChannelType; - } - - /** - * Returns the group vertex connected to this edge. - * - * @return the group vertex connected to this edge - */ - public ExecutionGroupVertex getTargetVertex() { - return this.targetVertex; - } - - /** - * Sets the edge's connection ID. - * - * @param connectionID - * the edge's connection ID - */ - void setConnectionID(final int connectionID) { - this.connectionID = connectionID; - } - - /** - * Returns the edge's connection ID. - * - * @return the edge's connection ID - */ - public int getConnectionID() { - return this.connectionID; - } - - /** - * Returns if the edge's channel type is user defined. - * - * @return true if the channel type is user defined, false otherwise - */ - public boolean isChannelTypeUserDefined() { - return this.userDefinedChannelType; - } - - /** - * Returns the index of the input gate this edge starts from. - * - * @return the index of the input gate this edge starts from - */ - public int getIndexOfInputGate() { - return this.indexOfInputGate; - } - - /** - * Returns the source vertex this edge starts from. - * - * @return the source vertex this edge starts from - */ - public ExecutionGroupVertex getSourceVertex() { - return this.sourceVertex; - } - - /** - * Returns the index of the output gate this edge arrives at. - * - * @return the index of the output gate this edge arrives at - */ - public int getIndexOfOutputGate() { - return this.indexOfOutputGate; - } - - /** - * Returns the distribution pattern to create the wiring between the group members. - * - * @return the distribution pattern to create the wiring between the group members - */ - public DistributionPattern getDistributionPattern() { - return this.distributionPattern; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupVertex.java deleted file mode 100644 index 8bd0e60039e..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupVertex.java +++ /dev/null @@ -1,774 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.executiongraph; - -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.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.io.InputSplit; -import org.apache.flink.runtime.instance.AllocatedResource; -import org.apache.flink.runtime.instance.DummyInstance; -import org.apache.flink.runtime.io.network.channels.ChannelType; -import org.apache.flink.runtime.jobgraph.DistributionPattern; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; - -/** - * An ExecutionGroupVertex is created for every JobVertex of the initial job graph. It represents a number of execution - * vertices that originate from the same job vertex. - *

- * This class is thread-safe. - */ -public final class ExecutionGroupVertex { - - /** - * The default number of retries in case of an error before the task represented by this vertex is considered as - * failed. - */ - private static final int DEFAULT_EXECUTION_RETRIES = 0; - - /** - * The name of the vertex. - */ - private final String name; - - /** - * The ID of the job vertex which is represented by this group vertex. - */ - private final JobVertexID jobVertexID; - - /** - * Stores whether the initial group member has already been added to this group vertex. - */ - private final AtomicBoolean initialGroupMemberAdded = new AtomicBoolean(false); - - /** - * The list of execution vertices which are managed by this group vertex. - */ - private final CopyOnWriteArrayList groupMembers = new CopyOnWriteArrayList(); - - /** - * The user defined number of execution vertices, -1 if the user has not specified it. - */ - private final int userDefinedNumberOfMembers; - - /** - * 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. - */ - private final AtomicReference vertexToShareInstancesWith = new AtomicReference( - null); - - /** - * Set of execution vertices sharing instances with this vertex. - */ - private final CopyOnWriteArrayList verticesSharingInstances = new CopyOnWriteArrayList(); - - /** - * Stores whether the group vertex to share instances with is user defined. - */ - private final boolean userDefinedVertexToShareInstancesWith; - - /** - * The cryptographic signature of the vertex. - */ - private final ExecutionSignature executionSignature; - - /** - * List of outgoing edges. - */ - private final CopyOnWriteArrayList forwardLinks = new CopyOnWriteArrayList(); - - /** - * List of incoming edges. - */ - private final CopyOnWriteArrayList backwardLinks = new CopyOnWriteArrayList(); - - /** - * List of input splits assigned to this group vertex. - */ - private volatile InputSplit[] inputSplits = null; - - /** - * Input split type - */ - private volatile Class inputSplitType = null; - - /** - * The execution stage this vertex belongs to. - */ - private volatile ExecutionStage executionStage = null; - - /** - * The configuration object of the original job vertex. - */ - private final Configuration configuration; - - /** - * The task class that is assigned to execution vertices of this group - */ - private final Class invokableClass; - - /** - * Constructs a new group vertex. - * - * @param name - * the name of the group vertex - * @param jobVertexID - * the ID of the job vertex which is represented by this group vertex - * @param executionGraph - * the execution graph is group vertex belongs to - * @param userDefinedNumberOfMembers - * the user defined number of subtasks, -1 if the user did not specify the number - * @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 - * the cryptographic signature of the vertex - * @param invokableClass - * the task class that is assigned to execution vertices of this group - * @throws Exception - * throws if an error occurs while instantiating the {@link AbstractInvokable} - */ - public ExecutionGroupVertex(final String name, final JobVertexID jobVertexID, final ExecutionGraph executionGraph, - final int userDefinedNumberOfMembers, final boolean userDefinedVertexToShareInstanceWith, - final int numberOfExecutionRetries, final Configuration configuration, final ExecutionSignature signature, - final Class invokableClass) throws Exception { - - this.name = (name != null) ? name : ""; - this.jobVertexID = jobVertexID; - this.userDefinedNumberOfMembers = userDefinedNumberOfMembers; - if (numberOfExecutionRetries >= 0) { - this.numberOfExecutionRetries = numberOfExecutionRetries; - } else { - this.numberOfExecutionRetries = DEFAULT_EXECUTION_RETRIES; - } - this.userDefinedVertexToShareInstancesWith = userDefinedVertexToShareInstanceWith; - this.configuration = configuration; - this.executionSignature = signature; - - this.invokableClass = invokableClass; - } - - /** - * Returns the name of the group vertex, usually copied from the initial job vertex. - * - * @return the name of the group vertex. - */ - public String getName() { - return this.name; - } - - /** - * Sets the execution stage this group vertex is associated with. - * - * @param executionStage - * The new execution stage. - */ - public void setExecutionStage(final ExecutionStage executionStage) { - - this.executionStage = executionStage; - } - - /** - * Returns the execution stage this group vertex is associated with. - * - * @return The execution stage this vertex is associated with. - */ - public ExecutionStage getExecutionStage() { - - return this.executionStage; - } - - /** - * Adds the initial execution vertex to this group vertex. - * - * @param ev - * The new execution vertex to be added. - */ - void addInitialSubtask(final ExecutionVertex ev) { - - if (ev == null) { - throw new IllegalArgumentException("Argument ev must not be null"); - } - - if (this.initialGroupMemberAdded.compareAndSet(false, true)) { - this.groupMembers.add(ev); - } - } - - /** - * Returns a specific execution vertex from the list of members. - * - * @param pos - * The position of the execution vertex to be returned. - * @return The execution vertex at position pos of the member list, null if there is no - * such position. - */ - public ExecutionVertex getGroupMember(final int pos) { - - if (pos < 0) { - throw new IllegalArgumentException("Argument pos must be greater or equal to 0"); - } - - try { - return this.groupMembers.get(pos); - } catch (ArrayIndexOutOfBoundsException e) { - return null; - } - } - - /** - * Returns the current number of members this group vertex has. - * - * @return the current number of members this group vertex has - */ - public int getCurrentNumberOfGroupMembers() { - - return this.groupMembers.size(); - } - - /** - * Wires this group vertex to the specified group vertex and creates - * a back link. - * - * @param groupVertex - * the group vertex that should be the target of the wiring - * @param indexOfInputGate - * the index of the consuming task's input gate - * @param indexOfOutputGate - * the index of the producing tasks's output gate - * @param channelType - * the channel type to be used for this edge - * @param userDefinedChannelType - * true if the channel type is user defined, false otherwise - * @param distributionPattern - * the distribution pattern to create the wiring between the group members - * @param isBroadcast - * indicates that the edge is part of broadcast group - * @return the created edge. - */ - ExecutionGroupEdge wireTo(final ExecutionGroupVertex groupVertex, final int indexOfInputGate, - final int indexOfOutputGate, final ChannelType channelType, final boolean userDefinedChannelType, - final DistributionPattern distributionPattern) throws GraphConversionException { - - try { - final ExecutionGroupEdge previousEdge = this.forwardLinks.get(indexOfOutputGate); - if (previousEdge != null) { - throw new GraphConversionException("Output gate " + indexOfOutputGate + " of" + getName() - + " already has an outgoing edge"); - } - } catch (ArrayIndexOutOfBoundsException e) { - // Ignore exception - } - - final ExecutionGroupEdge edge = new ExecutionGroupEdge(this, indexOfOutputGate, groupVertex, indexOfInputGate, - channelType, userDefinedChannelType, distributionPattern); - - this.forwardLinks.add(edge); - - groupVertex.wireBackLink(edge); - - return edge; - } - - /** - * Checks if this group vertex is wired to the given group vertex. - * - * @param groupVertex - * the group vertex to check for - * @return true if there is a wire from the current group vertex to the specified group vertex, - * otherwise false - */ - boolean isWiredTo(final ExecutionGroupVertex groupVertex) { - - final Iterator it = this.forwardLinks.iterator(); - while (it.hasNext()) { - final ExecutionGroupEdge edge = it.next(); - if (edge.getTargetVertex() == groupVertex) { - return true; - } - } - - return false; - } - - /** - * Creates a back link from the current group vertex to the specified group vertex. - * - * @param groupVertex - * the target of the back link - */ - private void wireBackLink(final ExecutionGroupEdge edge) { - - this.backwardLinks.add(edge); - } - - /** - * Returns the number of forward links the current group vertex has. - * - * @return the number of forward links the current group vertex has - */ - public int getNumberOfForwardLinks() { - - return this.forwardLinks.size(); - } - - /** - * Returns the number of backward links the current group vertex has. - * - * @return the number of backward links the current group vertex has - */ - public int getNumberOfBackwardLinks() { - - return this.backwardLinks.size(); - } - - /** - * Returns the number of the stage this group vertex belongs to. - * - * @return the number of the stage this group vertex belongs to - */ - public int getStageNumber() { - - return this.executionStage.getStageNumber(); - } - - /** - * Creates the initial execution vertices managed by this group vertex. - * - * @param initialNumberOfVertices - * the initial number of execution vertices - * @throws GraphConversionException - * thrown if the number of execution vertices for this group vertex cannot be set to the desired value - */ - void createInitialExecutionVertices(final int initialNumberOfVertices) throws GraphConversionException { - - // If the requested number of group vertices does not change, do nothing - if (initialNumberOfVertices == this.getCurrentNumberOfGroupMembers()) { - return; - } - - // Make sure the method is only called for the initial setup of the graph - if (this.getCurrentNumberOfGroupMembers() != 1) { - throw new IllegalStateException( - "This method can only be called for the initial setup of the execution graph"); - } - - // If the number of group vertices is user defined, prevent overwriting - if (this.userDefinedNumberOfMembers != -1) { - if (this.userDefinedNumberOfMembers == getCurrentNumberOfGroupMembers()) { // Note that - // this.userDefinedNumberOfMembers - // is final and requires no - // locking! - throw new GraphConversionException("Cannot overwrite user defined number of group members"); - } - } - - final ExecutionVertex originalVertex = this.getGroupMember(0); - int currentNumberOfExecutionVertices = this.getCurrentNumberOfGroupMembers(); - - while (currentNumberOfExecutionVertices++ < initialNumberOfVertices) { - - final ExecutionVertex vertex = originalVertex.splitVertex(); - vertex.setAllocatedResource(new AllocatedResource(DummyInstance - .createDummyInstance(), null)); - this.groupMembers.add(vertex); - } - - // Update the index and size information attached to the vertices - int index = 0; - final Iterator it = this.groupMembers.iterator(); - while (it.hasNext()) { - final ExecutionVertex vertex = it.next(); - vertex.setIndexInVertexGroup(index++); - } - } - - /** - * Sets the input splits that should be assigned to this group vertex. - * - * @param inputSplits - * the input splits that shall be assigned to this group vertex - */ - public void setInputSplits(final InputSplit[] inputSplits) { - - this.inputSplits = inputSplits; - } - - /** - * Sets the input split type class - * - * @param inputSplitType Input split type class - */ - public void setInputSplitType(final Class inputSplitType) { - this.inputSplitType = inputSplitType; - } - - /** - * Returns the input splits assigned to this group vertex. - * - * @return the input splits, possibly null if the group vertex does not represent an input vertex - */ - public InputSplit[] getInputSplits() { - return this.inputSplits; - } - - /** - * Returns the input split type class - * - * @return the input split type class, possibly null if the group vertex does not represent an input - * vertex - */ - public Class getInputSplitType() { return this.inputSplitType; } - - public ExecutionGroupEdge getForwardEdge(int index) { - - if (index < 0) { - throw new IllegalArgumentException("Argument index must be greater than or equal to 0"); - } - - try { - return this.forwardLinks.get(index); - } catch (ArrayIndexOutOfBoundsException e) { - return null; - } - } - - public ExecutionGroupEdge getBackwardEdge(int index) { - - if (index < 0) { - throw new IllegalArgumentException("Argument index must be greater than or equal to 0"); - } - - try { - return this.backwardLinks.get(index); - } catch (ArrayIndexOutOfBoundsException e) { - return null; - } - } - - public List getForwardEdges(final ExecutionGroupVertex groupVertex) { - - if (groupVertex == null) { - throw new IllegalArgumentException("Argument groupVertex must not be null"); - } - - final List edges = new ArrayList(); - - final Iterator it = this.forwardLinks.iterator(); - while (it.hasNext()) { - - final ExecutionGroupEdge edge = it.next(); - if (edge.getTargetVertex() == groupVertex) { - edges.add(edge); - } - } - - return edges; - } - - public List getBackwardEdges(final ExecutionGroupVertex groupVertex) { - - if (groupVertex == null) { - throw new IllegalArgumentException("Argument groupVertex must not be null"); - } - - final List edges = new ArrayList(); - - final Iterator it = this.backwardLinks.iterator(); - while (it.hasNext()) { - - final ExecutionGroupEdge edge = it.next(); - if (edge.getSourceVertex() == groupVertex) { - edges.add(edge); - } - } - - return edges; - } - - boolean isNumberOfMembersUserDefined() { - - return (this.userDefinedNumberOfMembers == -1) ? false : true; - } - - int getUserDefinedNumberOfMembers() { - - return this.userDefinedNumberOfMembers; - } - - - /** - * 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) { - throw new GraphConversionException("Cannot overwrite user defined vertex to share instances with"); - } - - if (groupVertex == null) { - throw new IllegalArgumentException("shareInstancesWith: argument is null!"); - } - - final ExecutionGroupVertex oldVertex = this.vertexToShareInstancesWith.getAndSet(groupVertex); - if (oldVertex != null) { - oldVertex.removeFromVerticesSharingInstances(this); - } - - groupVertex.addToVerticesSharingInstances(this); - } - - boolean isVertexToShareInstanceWithUserDefined() { - - return this.userDefinedVertexToShareInstancesWith; - } - - /** - * Returns the configuration object of the original job vertex. - * - * @return the configuration object of the original job vertex - */ - public Configuration getConfiguration() { - - return this.configuration; - } - - /** - * Returns the execution signature of this vertex. - * - * @return the execution signature of this vertex - */ - public ExecutionSignature getExecutionSignature() { - - return this.executionSignature; - } - - private void addToVerticesSharingInstances(final ExecutionGroupVertex groupVertex) { - - if (groupVertex == null) { - throw new IllegalArgumentException("Argument groupVertex must not be null"); - } - - this.verticesSharingInstances.addIfAbsent(groupVertex); - } - - private void removeFromVerticesSharingInstances(final ExecutionGroupVertex groupVertex) { - - if (groupVertex == null) { - throw new IllegalArgumentException("Argument groupVertex must not be null"); - } - - this.verticesSharingInstances.remove(groupVertex); - - } - - void repairInstanceSharing(final Set availableResources) { - - // Number of required resources by this group vertex - final int numberOfRequiredSlots = this.groupMembers.size(); - - // Number of resources to be replaced - final int resourcesToBeReplaced = Math.min(availableResources.size(), numberOfRequiredSlots); - - // Build the replacement map if necessary - final Map replacementMap = new HashMap(); - - if (resourcesToBeReplaced > 0) { - - final Iterator vertexIt = this.groupMembers.iterator(); - final Iterator resourceIt = availableResources.iterator(); - - while (replacementMap.size() < resourcesToBeReplaced) { - - if (!vertexIt.hasNext()) { - break; - } - - if (!resourceIt.hasNext()) { - break; - } - - final ExecutionVertex vertex = vertexIt.next(); - final AllocatedResource originalResource = vertex.getAllocatedResource(); - - if (!replacementMap.containsKey(originalResource)) { - - final AllocatedResource replacementResource = resourceIt.next(); - replacementMap.put(originalResource, replacementResource); - } - } - } - - // Now replace the instances - final Iterator vertexIt = this.groupMembers.iterator(); - while (vertexIt.hasNext()) { - - final ExecutionVertex vertex = vertexIt.next(); - final AllocatedResource originalResource = vertex.getAllocatedResource(); - final AllocatedResource replacementResource = replacementMap.get(originalResource); - if (replacementResource != null) { - vertex.setAllocatedResource(replacementResource); - } else { - availableResources.add(originalResource); - } - } - - final Iterator groupVertexIt = this.verticesSharingInstances.iterator(); - while (groupVertexIt.hasNext()) { - groupVertexIt.next().repairInstanceSharing(availableResources); - } - } - - /** - * Checks if this vertex is an input vertex in its stage, i.e. has either no - * incoming connections or only incoming connections to group vertices in a lower stage. - * - * @return true if this vertex is an input vertex, false otherwise - */ - public boolean isInputVertex() { - - if (this.backwardLinks.size() == 0) { - return true; - } - - final Iterator it = this.backwardLinks.iterator(); - while (it.hasNext()) { - if (it.next().getSourceVertex().getStageNumber() == this.getStageNumber()) { - return false; - } - } - - return true; - } - - /** - * Checks if this vertex is an output vertex in its stage, i.e. has either no - * outgoing connections or only outgoing connections to group vertices in a higher stage. - * - * @return true if this vertex is an output vertex, false otherwise - */ - public boolean isOutputVertex() { - - if (this.forwardLinks.size() == 0) { - return true; - } - - final Iterator it = this.forwardLinks.iterator(); - while (it.hasNext()) { - if (it.next().getTargetVertex().getStageNumber() == this.getStageNumber()) { - return false; - } - } - - return true; - } - - public ExecutionGroupVertex getVertexToShareInstancesWith() { - return this.vertexToShareInstancesWith.get(); - } - - /** - * Returns the ID of the job vertex which is represented by - * this group vertex. - * - * @return the ID of the job vertex which is represented by - * this group vertex - */ - public JobVertexID getJobVertexID() { - - return this.jobVertexID; - } - - /** - * Returns an iterator over all members of this execution group vertex. - * - * @return an iterator over all members of this execution group vertex - */ - public Iterator iterator() { - - return this.groupMembers.iterator(); - } - - /** - * Recursive method to calculate the connection IDs of the {@link ExecutionGraph}. - * - * @param currentConnectionID - * the current connection ID - * @param alreadyVisited - * the set of already visited group vertices - * @return maximum assigned connectionID - */ - int calculateConnectionID(int currentConnectionID, final Set alreadyVisited) { - - if (!alreadyVisited.add(this)) { - return currentConnectionID; - } - - for (final ExecutionGroupEdge backwardLink : this.backwardLinks) { - - backwardLink.setConnectionID(currentConnectionID); - - ++currentConnectionID; - - currentConnectionID = backwardLink.getSourceVertex() - .calculateConnectionID(currentConnectionID, alreadyVisited); - } - - return currentConnectionID; - } - - - /** - * Returns the task class that is assigned to execution vertices of this group. - * - * @return the task class that is assigned to execution vertices of this group - */ - Class getInvokableClass() { - - return this.invokableClass; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java index 72f03b4a5bd..dcad957bae4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java @@ -18,11 +18,11 @@ package org.apache.flink.runtime.executiongraph; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.logging.Log; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.core.io.InputSplitSource; @@ -35,11 +35,13 @@ import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; +import org.slf4j.Logger; + public class ExecutionJobVertex { /** Use the same log for all ExecutionGraph classes */ - private static final Log LOG = ExecutionGraph.LOG; + private static final Logger LOG = ExecutionGraph.LOG; private final ExecutionGraph graph; @@ -50,6 +52,8 @@ public class ExecutionJobVertex { private final IntermediateResult[] producedDataSets; + private final List inputs; + private final InputSplitAssigner splitAssigner; private final int parallelism; @@ -79,6 +83,8 @@ public class ExecutionJobVertex { this.parallelism = numTaskVertices; this.taskVertices = new ExecutionVertex2[numTaskVertices]; + this.inputs = new ArrayList(jobVertex.getInputs().size()); + // create the intermediate results this.producedDataSets = new IntermediateResult[jobVertex.getNumberOfProducedIntermediateDataSets()]; for (int i = 0; i < jobVertex.getProducedDataSets().size(); i++) { @@ -92,6 +98,13 @@ public class ExecutionJobVertex { this.taskVertices[i] = vertex; } + // sanity check for the double referencing between intermediate result partitions and execution vertices + for (IntermediateResult ir : this.producedDataSets) { + if (ir.getNumberOfAssignedPartitions() != parallelism) { + throw new RuntimeException("The intermediate result's partitions were not correctly assiged."); + } + } + // take the sharing group this.slotSharingGroup = jobVertex.getSlotSharingGroup(); @@ -151,6 +164,10 @@ public class ExecutionJobVertex { return slotSharingGroup; } + public List getInputs() { + return inputs; + } + //--------------------------------------------------------------------------------------------- public void connectToPredecessors(Map intermediateDataSets) throws JobException { @@ -182,14 +199,14 @@ public class ExecutionJobVertex { + edge.getSourceId()); } + this.inputs.add(ires); + int consumerIndex = ires.registerConsumer(); for (int i = 0; i < parallelism; i++) { ExecutionVertex2 ev = taskVertices[i]; ev.connectSource(num, ires, edge, consumerIndex); } - - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java index d703b4efcf8..4a4d0e7aa95 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java @@ -40,6 +40,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture; import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFutureAction; import org.apache.flink.runtime.taskmanager.TaskOperationResult; import org.apache.flink.util.StringUtils; +import org.slf4j.Logger; import static org.apache.flink.runtime.execution.ExecutionState2.*; @@ -67,7 +68,7 @@ public class ExecutionVertex2 { private static final AtomicReferenceFieldUpdater ASSIGNED_SLOT_UPDATER = AtomicReferenceFieldUpdater.newUpdater(ExecutionVertex2.class, AllocatedSlot.class, "assignedSlot"); - private static final Log LOG = ExecutionGraph.LOG; + private static final Logger LOG = ExecutionGraph.LOG; private static final int NUM_CANCEL_CALL_TRIES = 3; @@ -82,6 +83,9 @@ public class ExecutionVertex2 { private final int subTaskIndex; + private final long[] stateTimestamps; + + private volatile ExecutionState2 state = CREATED; private volatile AllocatedSlot assignedSlot; @@ -101,6 +105,8 @@ public class ExecutionVertex2 { } this.inputEdges = new ExecutionEdge2[jobVertex.getJobVertex().getInputs().size()][]; + + this.stateTimestamps = new long[ExecutionState2.values().length]; } @@ -151,6 +157,11 @@ public class ExecutionVertex2 { return assignedSlot; } + public long getStateTimestamp(ExecutionState2 state) { + return this.stateTimestamps[state.ordinal()]; + } + + private ExecutionGraph getExecutionGraph() { return this.jobVertex.getGraph(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java index 540996f2421..0b822abf943 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java @@ -64,10 +64,18 @@ public class IntermediateResult { return id; } + public ExecutionJobVertex getProducer() { + return producer; + } + public IntermediateResultPartition[] getPartitions() { return partitions; } + public int getNumberOfAssignedPartitions() { + return partitionsAssigned; + } + public int registerConsumer() { final int index = numConsumers; numConsumers++; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java index 13bb9308484..b733baa396d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.executiongraph; import java.util.ArrayList; -import java.util.Collections; import java.util.List; public class IntermediateResultPartition { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ManagementGraphFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ManagementGraphFactory.java deleted file mode 100644 index 56571f892a4..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ManagementGraphFactory.java +++ /dev/null @@ -1,181 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.executiongraph; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; - -import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.io.network.channels.ChannelType; -import org.apache.flink.runtime.managementgraph.ManagementEdge; -import org.apache.flink.runtime.managementgraph.ManagementEdgeID; -import org.apache.flink.runtime.managementgraph.ManagementGate; -import org.apache.flink.runtime.managementgraph.ManagementGateID; -import org.apache.flink.runtime.managementgraph.ManagementGraph; -import org.apache.flink.runtime.managementgraph.ManagementGroupEdge; -import org.apache.flink.runtime.managementgraph.ManagementGroupVertex; -import org.apache.flink.runtime.managementgraph.ManagementStage; -import org.apache.flink.runtime.managementgraph.ManagementVertex; - -public class ManagementGraphFactory { - - private ManagementGraphFactory() { - } - - public static ManagementGraph fromExecutionGraph(ExecutionGraph executionGraph) { - - final ManagementGraph managementGraph = new ManagementGraph(executionGraph.getJobID()); - - final Map stageMap = addExecutionStages(managementGraph, executionGraph); - final Map groupMap = addGroupVertices(stageMap); - addExecutionVertices(groupMap, executionGraph); - - return managementGraph; - } - - private static Map addExecutionStages(ManagementGraph managementGraph, - ExecutionGraph executionGraph) { - - final Map stageMap = new HashMap(); - - for (int i = 0; i < executionGraph.getNumberOfStages(); i++) { - - final ExecutionStage executionStage = executionGraph.getStage(i); - final ManagementStage managementStage = new ManagementStage(managementGraph, i); - stageMap.put(executionStage, managementStage); - } - - return stageMap; - } - - private static Map addGroupVertices( - Map stageMap) { - - final Map groupMap = new HashMap(); - - // First, create all vertices - Iterator> iterator = stageMap.entrySet().iterator(); - while (iterator.hasNext()) { - - final Map.Entry entry = iterator.next(); - final ExecutionStage executionStage = entry.getKey(); - final ManagementStage parent = entry.getValue(); - - for (int i = 0; i < executionStage.getNumberOfStageMembers(); i++) { - - final ExecutionGroupVertex groupVertex = executionStage.getStageMember(i); - final ManagementGroupVertex managementGroupVertex = new ManagementGroupVertex(parent, groupVertex - .getName()); - - groupMap.put(groupVertex, managementGroupVertex); - } - } - - // Second, make sure all edges are created and connected properly - iterator = stageMap.entrySet().iterator(); - while (iterator.hasNext()) { - - final Map.Entry entry = iterator.next(); - final ExecutionStage executionStage = entry.getKey(); - for (int i = 0; i < executionStage.getNumberOfStageMembers(); i++) { - - final ExecutionGroupVertex sourceVertex = executionStage.getStageMember(i); - final ManagementGroupVertex sourceGroupVertex = groupMap.get(sourceVertex); - - for (int j = 0; j < sourceVertex.getNumberOfForwardLinks(); j++) { - - final ExecutionGroupEdge edge = sourceVertex.getForwardEdge(j); - final ExecutionGroupVertex targetVertex = edge.getTargetVertex(); - final ManagementGroupVertex targetGroupVertex = groupMap.get(targetVertex); - new ManagementGroupEdge(sourceGroupVertex, j, targetGroupVertex, edge.getIndexOfInputGate(), edge - .getChannelType()); - } - } - } - - return groupMap; - } - - private static void addExecutionVertices(Map groupMap, - ExecutionGraph executionGraph) { - - ExecutionGraphIterator iterator = new ExecutionGraphIterator(executionGraph, true); - final Map vertexMap = new HashMap(); - final Map gateMap = new HashMap(); - - while (iterator.hasNext()) { - - final ExecutionVertex ev = iterator.next(); - final ManagementGroupVertex parent = groupMap.get(ev.getGroupVertex()); - - final Instance instance = ev.getAllocatedResource().getInstance(); - final ManagementVertex managementVertex = new ManagementVertex( - parent, - ev.getID().toManagementVertexID(), - (instance.getInstanceConnectionInfo() != null) ? instance.getInstanceConnectionInfo().toString() : instance.toString(), - ev.getIndexInVertexGroup() - ); - managementVertex.setExecutionState(ev.getExecutionState()); - vertexMap.put(ev, managementVertex); - - for (int i = 0; i < ev.getNumberOfOutputGates(); i++) { - final ExecutionGate outputGate = ev.getOutputGate(i); - final ManagementGate managementGate = new ManagementGate(managementVertex, new ManagementGateID(), i, - false); - gateMap.put(outputGate, managementGate); - } - - for (int i = 0; i < ev.getNumberOfInputGates(); i++) { - final ExecutionGate inputGate = ev.getInputGate(i); - final ManagementGate managementGate = new ManagementGate(managementVertex, new ManagementGateID(), i, - true); - gateMap.put(inputGate, managementGate); - } - } - - iterator = new ExecutionGraphIterator(executionGraph, true); - - // Setup connections - while (iterator.hasNext()) { - - final ExecutionVertex source = iterator.next(); - - for (int i = 0; i < source.getNumberOfOutputGates(); i++) { - - final ExecutionGate outputGate = source.getOutputGate(i); - final ManagementGate manangementOutputGate = gateMap.get(outputGate); - final ChannelType channelType = outputGate.getChannelType(); - - for (int j = 0; j < outputGate.getNumberOfEdges(); j++) { - - final ExecutionEdge outputChannel = outputGate.getEdge(j); - - final ManagementGate managementInputGate = gateMap.get(outputChannel.getInputGate()); - - final ManagementEdgeID sourceEdgeID = new ManagementEdgeID(outputChannel.getOutputChannelID()); - final ManagementEdgeID targetEdgeID = new ManagementEdgeID(outputChannel.getInputChannelID()); - new ManagementEdge(sourceEdgeID, targetEdgeID, manangementOutputGate, j, managementInputGate, - outputChannel.getInputGateIndex(), channelType); - } - } - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Hardware.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Hardware.java index aa619274154..71e428e5728 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Hardware.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Hardware.java @@ -25,16 +25,16 @@ import java.io.InputStreamReader; import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.flink.util.OperatingSystem; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Convenience class to extract hardware specifics of the computer executing this class */ public class Hardware { - private static final Log LOG = LogFactory.getLog(Hardware.class); + private static final Logger LOG = LoggerFactory.getLogger(Hardware.class); private static final String LINUX_MEMORY_INFO_PATH = "/proc/meminfo"; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java index 5e139f32d0e..94ec3d3baf9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java @@ -28,17 +28,17 @@ import java.util.Set; import java.util.Timer; import java.util.TimerTask; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.GlobalConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Simple manager that keeps track of which TaskManager are available and alive. */ public class InstanceManager { - private static final Log LOG = LogFactory.getLog(InstanceManager.class); + private static final Logger LOG = LoggerFactory.getLogger(InstanceManager.class); // ------------------------------------------------------------------------ // Fields @@ -201,6 +201,8 @@ public class InstanceManager { public Map getAllRegisteredInstances() { synchronized (this.lock) { + // return a copy (rather than a Collections.unmodifiable(...) wrapper), such that + // concurrent modifications do not interfere with the traversals or lookups return new HashMap(this.registeredHostsById); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java index 832b8cd17e3..e9f9c73bc57 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java @@ -21,12 +21,12 @@ package org.apache.flink.runtime.instance; import java.util.ArrayList; import java.util.List; -import org.apache.commons.logging.LogFactory; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.ExecutionMode; import org.apache.flink.runtime.taskmanager.TaskManager; +import org.slf4j.LoggerFactory; /** * A variant of the {@link InstanceManager} that internally spawn task managers as instances, rather than waiting for external @@ -73,7 +73,7 @@ public class LocalInstanceManager extends InstanceManager { // log and continue in any case // we initialize the log lazily, because this is the only place we log // and most likely we never log. - LogFactory.getLog(LocalInstanceManager.class).error("Error shutting down local embedded TaskManager.", t); + LoggerFactory.getLogger(LocalInstanceManager.class).error("Error shutting down local embedded TaskManager.", t); } } } finally { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java index fc39db2a42e..d3007c53bbc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java @@ -188,13 +188,10 @@ public class IOManager implements UncaughtExceptionHandler { return this.isClosed && writersShutDown && readersShutDown; } - /* (non-Javadoc) - * @see java.lang.Thread.UncaughtExceptionHandler#uncaughtException(java.lang.Thread, java.lang.Throwable) - */ + @Override - public void uncaughtException(Thread t, Throwable e) - { - LOG.fatal("IO Thread '" + t.getName() + "' terminated due to an exception. Closing I/O Manager.", e); + public void uncaughtException(Thread t, Throwable e) { + LOG.error("IO Thread '" + t.getName() + "' terminated due to an exception. Closing I/O Manager.", e); shutdown(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java index d6d3e37d6af..11229ec5ca3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java @@ -18,8 +18,9 @@ package org.apache.flink.runtime.jobgraph; +import javax.xml.bind.DatatypeConverter; + import org.apache.flink.runtime.AbstractID; -import org.apache.flink.runtime.managementgraph.ManagementVertexID; /** * A class for statistically unique job vertex IDs. @@ -28,7 +29,15 @@ public class JobVertexID extends AbstractID { private static final long serialVersionUID = 1L; - public ManagementVertexID toManagementVertexId(int subtaskIndex) { - return new ManagementVertexID(subtaskIndex, getLowerPart() + getUpperPart()); + public JobVertexID() { + super(); + } + + public JobVertexID(byte[] bytes) { + super(bytes); + } + + public static JobVertexID fromHexString(String hexString) { + return new JobVertexID(DatatypeConverter.parseHexBinary(hexString)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java index d36659f0be3..3bd24a67567 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java @@ -38,13 +38,10 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.JobStatusListener; -import org.apache.flink.runtime.executiongraph.ManagementGraphFactory; import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmanager.archive.ArchiveListener; -import org.apache.flink.runtime.managementgraph.ManagementGraph; -import org.apache.flink.runtime.managementgraph.ManagementVertex; import org.apache.flink.runtime.profiling.ProfilingListener; import org.apache.flink.runtime.profiling.types.ProfilingEvent; @@ -52,8 +49,6 @@ import org.apache.flink.runtime.profiling.types.ProfilingEvent; * The event collector collects events which occurred during the execution of a job and prepares them * for being fetched by a client. The collected events have an expiration time. In a configurable interval * the event collector removes all intervals which are older than the interval. - *

- * This class is thread-safe. */ public final class EventCollector extends TimerTask implements ProfilingListener { @@ -90,14 +85,13 @@ public final class EventCollector extends TimerTask implements ProfilingListener // Create a new vertex event final VertexEvent vertexEvent = new VertexEvent(timestamp, vertexId, taskName, totalNumberOfSubtasks, - subtask, newExecutionState, optionalMessage); + subtask, executionId, newExecutionState, optionalMessage); this.eventCollector.addEvent(jobID, vertexEvent); - final ExecutionStateChangeEvent executionStateChangeEvent = new ExecutionStateChangeEvent(timestamp, - vertexId.toManagementVertexId(subtask), newExecutionState); + final ExecutionStateChangeEvent executionStateChangeEvent = new ExecutionStateChangeEvent(timestamp, vertexId, subtask, + executionId, newExecutionState); - this.eventCollector.updateManagementGraph(jobID, executionStateChangeEvent, optionalMessage); this.eventCollector.addEvent(jobID, executionStateChangeEvent); } } @@ -150,9 +144,7 @@ public final class EventCollector extends TimerTask implements ProfilingListener final JobID jobID = executionGraph.getJobID(); if (newJobStatus == JobStatus.RUNNING) { - - final ManagementGraph managementGraph = ManagementGraphFactory.fromExecutionGraph(executionGraph); - this.eventCollector.addManagementGraph(jobID, managementGraph); + this.eventCollector.addExecutionGraph(jobID, executionGraph); } // Update recent job event @@ -180,7 +172,7 @@ public final class EventCollector extends TimerTask implements ProfilingListener /** * Map of management graphs belonging to recently started jobs with the time stamp of the last received job event. */ - private final Map recentManagementGraphs = new HashMap(); + private final Map recentManagementGraphs = new HashMap(); /** * The timer used to trigger the cleanup routine. @@ -237,13 +229,8 @@ public final class EventCollector extends TimerTask implements ProfilingListener } public void getRecentJobs(List eventList) { - synchronized (this.recentJobs) { - - final Iterator it = this.recentJobs.values().iterator(); - while (it.hasNext()) { - eventList.add(it.next()); - } + eventList.addAll(this.recentJobs.values()); } } @@ -409,60 +396,28 @@ public final class EventCollector extends TimerTask implements ProfilingListener } /** - * Adds a {@link ManagementGraph} to the map of recently created management graphs. + * Adds an execution graph to the map of recently created management graphs. * - * @param jobID - * the ID of the job the management graph belongs to - * @param managementGraph - * the management graph to be added + * @param jobID The ID of the graph + * @param executionGraph The graph to be added */ - void addManagementGraph(final JobID jobID, final ManagementGraph managementGraph) { + void addExecutionGraph(JobID jobID, ExecutionGraph executionGraph) { synchronized (this.recentManagementGraphs) { - this.recentManagementGraphs.put(jobID, managementGraph); + this.recentManagementGraphs.put(jobID, executionGraph); } } /** - * Returns the {@link ManagementGraph} object for the job with the given ID from the map of recently created - * management graphs. + * Returns the execution graph object for the job with the given ID from the map of recently added graphs. * - * @param jobID - * the ID of the job the management graph shall be retrieved for + * @param jobID The ID of the job the management graph shall be retrieved for * @return the management graph for the job with the given ID or null if no such graph exists */ - public ManagementGraph getManagementGraph(final JobID jobID) { + public ExecutionGraph getManagementGraph(JobID jobID) { synchronized (this.recentManagementGraphs) { return this.recentManagementGraphs.get(jobID); } } - - /** - * Applies changes in the state of an execution vertex to the stored management graph. - * - * @param jobID - * the ID of the job whose management graph shall be updated - * @param executionStateChangeEvent - * the event describing the changes in the execution state of the vertex - */ - private void updateManagementGraph(JobID jobID, ExecutionStateChangeEvent executionStateChangeEvent, String optionalMessage) { - - synchronized (this.recentManagementGraphs) { - - final ManagementGraph managementGraph = this.recentManagementGraphs.get(jobID); - if (managementGraph == null) { - return; - } - final ManagementVertex vertex = managementGraph.getVertexByID(executionStateChangeEvent.getVertexID()); - if (vertex == null) { - return; - } - - vertex.setExecutionState(executionStateChangeEvent.getNewExecutionState()); - if (optionalMessage != null) { - vertex.setOptMessage(optionalMessage); - } - } - } /** * Register Archivist to archive @@ -472,20 +427,20 @@ public final class EventCollector extends TimerTask implements ProfilingListener } private void archiveEvent(JobID jobId, AbstractEvent event) { - for(ArchiveListener al : archivists) { + for (ArchiveListener al : archivists) { al.archiveEvent(jobId, event); } } private void archiveJobevent(JobID jobId, RecentJobEvent event) { - for(ArchiveListener al : archivists) { + for (ArchiveListener al : archivists) { al.archiveJobevent(jobId, event); } } - private void archiveManagementGraph(JobID jobId, ManagementGraph graph) { - for(ArchiveListener al : archivists) { - al.archiveManagementGraph(jobId, graph); + private void archiveManagementGraph(JobID jobId, ExecutionGraph graph) { + for (ArchiveListener al : archivists) { + al.archiveExecutionGraph(jobId, graph); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java index fc76d73e00c..880abd5ad2a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java @@ -40,8 +40,7 @@ import org.apache.commons.cli.Option; import org.apache.commons.cli.OptionBuilder; import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; + import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -82,7 +81,6 @@ import org.apache.flink.runtime.jobmanager.archive.ArchiveListener; import org.apache.flink.runtime.jobmanager.archive.MemoryArchivist; import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler; import org.apache.flink.runtime.jobmanager.web.WebInfoServer; -import org.apache.flink.runtime.managementgraph.ManagementGraph; import org.apache.flink.runtime.protocols.AccumulatorProtocol; import org.apache.flink.runtime.protocols.ChannelLookupProtocol; import org.apache.flink.runtime.protocols.ExtendedManagementProtocol; @@ -94,10 +92,9 @@ import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.runtime.util.SerializableArrayList; import org.apache.flink.util.StringUtils; -import org.apache.log4j.ConsoleAppender; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; -import org.apache.log4j.PatternLayout; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.google.common.base.Preconditions; @@ -109,7 +106,7 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide JobManagerProtocol, ChannelLookupProtocol, JobStatusListener, AccumulatorProtocol { - private static final Log LOG = LogFactory.getLog(JobManager.class); + private static final Logger LOG = LoggerFactory.getLogger(JobManager.class); private final static int FAILURE_RETURN_CODE = 1; @@ -251,7 +248,7 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide try { this.executorService.awaitTermination(5000L, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { - LOG.debug(e); + LOG.debug("Shutdown of executor thread pool interrupted", e); } } @@ -523,28 +520,6 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide public InstanceManager getInstanceManager() { return this.instanceManager; } - - /** - * Returns current ManagementGraph from eventCollector and, if not current, from archive - * - * {@inheritDoc} - */ - @Override - public ManagementGraph getManagementGraph(final JobID jobID) throws IOException { - - ManagementGraph mg = this.eventCollector.getManagementGraph(jobID); - if (mg == null) { - if(this.archive != null) { - mg = this.archive.getManagementGraph(jobID); - } - - if (mg == null) { - throw new IOException("Cannot find job with ID " + jobID); - } - } - - return mg; - } @Override public IntegerRecord getRecommendedPollingInterval() throws IOException { @@ -641,6 +616,21 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide public Map getCurrentJobs() { return Collections.unmodifiableMap(currentJobs); } + + public ExecutionGraph getRecentExecutionGraph(JobID jobID) throws IOException { + ExecutionGraph eg = currentJobs.get(jobID); + if (eg == null) { + eg = this.eventCollector.getManagementGraph(jobID); + if (eg == null && this.archive != null) { + eg = this.archive.getExecutionGraph(jobID); + } + } + + if (eg == null) { + throw new IOException("Cannot find execution graph for job with ID " + jobID); + } + return eg; + } // -------------------------------------------------------------------------------------------- // TaskManager to JobManager communication @@ -669,15 +659,6 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide */ public static void main(String[] args) { - // determine if a valid log4j config exists and initialize a default logger if not - if (System.getProperty("log4j.configuration") == null) { - Logger root = Logger.getRootLogger(); - root.removeAllAppenders(); - PatternLayout layout = new PatternLayout("%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n"); - ConsoleAppender appender = new ConsoleAppender(layout, "System.err"); - root.addAppender(appender); - root.setLevel(Level.INFO); - } JobManager jobManager; try { @@ -686,7 +667,7 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide jobManager.startInfoServer(); } catch (Exception e) { - LOG.fatal(e.getMessage(), e); + LOG.error(e.getMessage(), e); System.exit(FAILURE_RETURN_CODE); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/ArchiveListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/ArchiveListener.java index d93551b4ebc..aade7aaed16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/ArchiveListener.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/ArchiveListener.java @@ -16,96 +16,48 @@ * limitations under the License. */ - package org.apache.flink.runtime.jobmanager.archive; import java.util.List; import org.apache.flink.runtime.event.job.AbstractEvent; import org.apache.flink.runtime.event.job.RecentJobEvent; -import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.managementgraph.ManagementGraph; -import org.apache.flink.runtime.managementgraph.ManagementVertexID; /** - * Interface used to implement Archivists, that store old jobmanager information discarded by the EventCollector. + * Interface used to implement Archivists, that store old JobManager information discarded by the EventCollector. * Archivists can decide how to store the data (memory, database, files...) */ public interface ArchiveListener { + + void archiveExecutionGraph(JobID jobId, ExecutionGraph graph); + /** * Stores event in archive - * - * @param jobId - * @param event */ void archiveEvent(JobID jobId, AbstractEvent event); /** * Stores old job in archive - * - * @param jobId - * @param event */ void archiveJobevent(JobID jobId, RecentJobEvent event); - /** - * Stores old ManagementGraph in archive - * - * @param jobId - * @param graph - */ - void archiveManagementGraph(JobID jobId, ManagementGraph graph); - /** * Get all archived Jobs - * - * @return */ List getJobs(); /** * Return archived job - * - * @param JobId - * @return */ RecentJobEvent getJob(JobID JobId); - /** - * Get archived ManagementGraph for a job - * - * @param jobID - * @return - */ - ManagementGraph getManagementGraph(JobID jobID); - /** * Get all archived Events for a job - * - * @param jobID - * @return */ - List getEvents(JobID jobID); + List getEvents(JobID jobID); - /** - * Returns the time when the status of the given job changed to jobStatus - * - * @param jobID - * @param jobStatus - * @return - */ - long getJobTime(JobID jobID, JobStatus jobStatus); - - /** - * returns the time, when the status of the given vertex changed to executionState - * - * @param jobID - * @param jobVertexID - * @param executionState - * @return - */ - long getVertexTime(JobID jobID, ManagementVertexID jobVertexID, ExecutionState executionState); + ExecutionGraph getExecutionGraph(JobID jid); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java index 35a43b43d95..2579b752f4d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.jobmanager.archive; import java.util.ArrayList; @@ -26,128 +25,112 @@ import java.util.List; import java.util.Map; import org.apache.flink.runtime.event.job.AbstractEvent; -import org.apache.flink.runtime.event.job.ExecutionStateChangeEvent; -import org.apache.flink.runtime.event.job.JobEvent; import org.apache.flink.runtime.event.job.RecentJobEvent; -import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.managementgraph.ManagementGraph; -import org.apache.flink.runtime.managementgraph.ManagementVertexID; /** - * Implementation of the ArchiveListener, that archives old data of the jobmanager in memory - * + * Implementation of the ArchiveListener, that archives old data of the JobManager in memory. + * + * This class must be thread safe, because it is accessed by the JobManager events and by the + * web server concurrently. */ public class MemoryArchivist implements ArchiveListener { + /** The global lock */ + private final Object lock = new Object(); - private int max_entries; /** * The map which stores all collected events until they are either * fetched by the client or discarded. */ private final Map> collectedEvents = new HashMap>(); - /** - * Map of recently started jobs with the time stamp of the last received job event. - */ + /** Map of recently started jobs with the time stamp of the last received job event. */ private final Map oldJobs = new HashMap(); - /** - * Map of management graphs belonging to recently started jobs with the time stamp of the last received job event. - */ - private final Map managementGraphs = new HashMap(); - + /** Map of management graphs belonging to recently started jobs with the time stamp of the last received job event. */ + private final Map graphs = new HashMap(); private final LinkedList lru = new LinkedList(); + private final int max_entries; + + // -------------------------------------------------------------------------------------------- + public MemoryArchivist(int max_entries) { this.max_entries = max_entries; } + // -------------------------------------------------------------------------------------------- - public void archiveEvent(JobID jobId, AbstractEvent event) { - - if(!collectedEvents.containsKey(jobId)) { - collectedEvents.put(jobId, new ArrayList()); + @Override + public void archiveExecutionGraph(JobID jobId, ExecutionGraph graph) { + synchronized (lock) { + graphs.put(jobId, graph); + cleanup(jobId); } - - collectedEvents.get(jobId).add(event); - - cleanup(jobId); } - public void archiveJobevent(JobID jobId, RecentJobEvent event) { - - oldJobs.put(jobId, event); - - cleanup(jobId); + @Override + public void archiveEvent(JobID jobId, AbstractEvent event) { + synchronized (lock) { + if(!collectedEvents.containsKey(jobId)) { + collectedEvents.put(jobId, new ArrayList()); + } + + collectedEvents.get(jobId).add(event); + cleanup(jobId); + } } - public void archiveManagementGraph(JobID jobId, ManagementGraph graph) { - - managementGraphs.put(jobId, graph); - - cleanup(jobId); + @Override + public void archiveJobevent(JobID jobId, RecentJobEvent event) { + synchronized (lock) { + oldJobs.put(jobId, event); + cleanup(jobId); + } } + @Override public List getJobs() { - - return new ArrayList(oldJobs.values()); - } - - private void cleanup(JobID jobId) { - if(!lru.contains(jobId)) { - lru.addFirst(jobId); - } - if(lru.size() > this.max_entries) { - JobID toRemove = lru.removeLast(); - collectedEvents.remove(toRemove); - oldJobs.remove(toRemove); - managementGraphs.remove(toRemove); + synchronized (lock) { + return new ArrayList(oldJobs.values()); } } + @Override public RecentJobEvent getJob(JobID jobId) { - - return oldJobs.get(jobId); - } + synchronized (lock) { + return oldJobs.get(jobId); + }} - public ManagementGraph getManagementGraph(final JobID jobID) { - - synchronized (this.managementGraphs) { - return this.managementGraphs.get(jobID); - } - } + @Override public List getEvents(JobID jobID) { - return collectedEvents.get(jobID); + synchronized (graphs) { + return collectedEvents.get(jobID); + } } - - public long getJobTime(JobID jobID, JobStatus jobStatus) { - for(AbstractEvent event : this.getEvents(jobID)) { - if(event instanceof JobEvent) - { - if(((JobEvent) event).getCurrentJobStatus() == jobStatus) { - return event.getTimestamp(); - } - } + + @Override + public ExecutionGraph getExecutionGraph(JobID jid) { + synchronized (lock) { + return graphs.get(jid); } - return 0; } - public long getVertexTime(JobID jobID, ManagementVertexID jobVertexID, ExecutionState executionState) { - for(AbstractEvent event : this.getEvents(jobID)) { - if(event instanceof ExecutionStateChangeEvent) - { - if(((ExecutionStateChangeEvent) event).getVertexID().equals(jobVertexID) && ((ExecutionStateChangeEvent) event).getNewExecutionState().equals(executionState)) { - return event.getTimestamp(); - } - } + + + private void cleanup(JobID jobId) { + if (!lru.contains(jobId)) { + lru.addFirst(jobId); + } + if (lru.size() > this.max_entries) { + JobID toRemove = lru.removeLast(); + collectedEvents.remove(toRemove); + oldJobs.remove(toRemove); + graphs.remove(toRemove); } - return 0; } - - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java index 9b94f851d24..b086fc14e07 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java @@ -16,14 +16,12 @@ * limitations under the License. */ - package org.apache.flink.runtime.jobmanager.web; import java.io.IOException; import java.io.PrintWriter; import java.util.Collections; import java.util.Comparator; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -35,21 +33,22 @@ import javax.servlet.http.HttpServletResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.runtime.event.job.AbstractEvent; import org.apache.flink.runtime.event.job.ExecutionStateChangeEvent; import org.apache.flink.runtime.event.job.JobEvent; import org.apache.flink.runtime.event.job.RecentJobEvent; -import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.execution.ExecutionState2; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex2; import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmanager.JobManager; -import org.apache.flink.runtime.managementgraph.ManagementGraph; -import org.apache.flink.runtime.managementgraph.ManagementGraphIterator; -import org.apache.flink.runtime.managementgraph.ManagementGroupVertex; -import org.apache.flink.runtime.managementgraph.ManagementGroupVertexID; -import org.apache.flink.runtime.managementgraph.ManagementVertex; import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.StringUtils; import org.eclipse.jetty.io.EofException; @@ -58,20 +57,17 @@ public class JobmanagerInfoServlet extends HttpServlet { private static final long serialVersionUID = 1L; - /** - * The log for this class. - */ private static final Logger LOG = LoggerFactory.getLogger(JobmanagerInfoServlet.class); - /** - * Underlying JobManager - */ + /** Underlying JobManager */ private final JobManager jobmanager; + public JobmanagerInfoServlet(JobManager jobmanager) { this.jobmanager = jobmanager; } + @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { @@ -89,7 +85,7 @@ public class JobmanagerInfoServlet extends HttpServlet { else if("groupvertex".equals(req.getParameter("get"))) { String jobId = req.getParameter("job"); String groupvertexId = req.getParameter("groupvertex"); - writeJsonForArchivedJobGroupvertex(resp.getWriter(), jobmanager.getArchive().getJob(JobID.fromHexString(jobId)), ManagementGroupVertexID.fromHexString(groupvertexId)); + writeJsonForArchivedJobGroupvertex(resp.getWriter(), jobmanager.getArchive().getJob(JobID.fromHexString(jobId)), JobVertexID.fromHexString(groupvertexId)); } else if("taskmanagers".equals(req.getParameter("get"))) { resp.getWriter().write("{\"taskmanagers\": " + jobmanager.getNumberOfTaskManagers() +"}"); @@ -152,7 +148,7 @@ public class JobmanagerInfoServlet extends HttpServlet { private void writeJsonForJob(PrintWriter wrt, RecentJobEvent jobEvent) throws IOException { - ManagementGraph jobManagementGraph = jobmanager.getManagementGraph(jobEvent.getJobID()); + ExecutionGraph graph = jobmanager.getRecentExecutionGraph(jobEvent.getJobID()); //Serialize job to json wrt.write("{"); @@ -164,14 +160,15 @@ public class JobmanagerInfoServlet extends HttpServlet { // Serialize ManagementGraph to json wrt.write("\"groupvertices\": ["); boolean first = true; - for(ManagementGroupVertex groupVertex : jobManagementGraph.getGroupVerticesInTopologicalOrder()) { + + for (ExecutionJobVertex groupVertex : graph.getVerticesTopologically()) { //Write seperator between json objects if(first) { first = false; } else { wrt.write(","); } - wrt.write(groupVertex.toJson()); + wrt.write(JsonFactory.toJson(groupVertex)); } wrt.write("]"); wrt.write("}"); @@ -235,39 +232,33 @@ public class JobmanagerInfoServlet extends HttpServlet { wrt.write("["); - ManagementGraph jobManagementGraph = jobmanager.getManagementGraph(jobEvent.getJobID()); + ExecutionGraph graph = jobmanager.getRecentExecutionGraph(jobEvent.getJobID()); //Serialize job to json wrt.write("{"); wrt.write("\"jobid\": \"" + jobEvent.getJobID() + "\","); wrt.write("\"jobname\": \"" + jobEvent.getJobName()+"\","); wrt.write("\"status\": \""+ jobEvent.getJobStatus() + "\","); - wrt.write("\"SCHEDULED\": "+ jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.SCHEDULED) + ","); - wrt.write("\"RUNNING\": "+ jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.RUNNING) + ","); - wrt.write("\"FINISHED\": "+ jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.FINISHED) + ","); - wrt.write("\"FAILED\": "+ jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.FAILED) + ","); - wrt.write("\"CANCELED\": "+ jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.CANCELED) + ","); - wrt.write("\"CREATED\": " + jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.CREATED)+","); + wrt.write("\"SCHEDULED\": "+ graph.getStatusTimestamp(JobStatus.CREATED) + ","); + wrt.write("\"RUNNING\": "+ graph.getStatusTimestamp(JobStatus.RUNNING) + ","); + wrt.write("\"FINISHED\": "+ graph.getStatusTimestamp(JobStatus.FINISHED) + ","); + wrt.write("\"FAILED\": "+ graph.getStatusTimestamp(JobStatus.FAILED) + ","); + wrt.write("\"CANCELED\": "+ graph.getStatusTimestamp(JobStatus.CANCELED) + ","); if (jobEvent.getJobStatus() == JobStatus.FAILED) { - ManagementGraphIterator managementGraphIterator = new ManagementGraphIterator(jobManagementGraph,true); - wrt.write("\"failednodes\": ["); - HashSet map = new HashSet(); - boolean first = true; - while (managementGraphIterator.hasNext()) { - ManagementVertex managementVertex = managementGraphIterator.next(); - String instanceName = managementVertex.getInstanceName(); - if (managementVertex.getExecutionState() == ExecutionState.FAILED && !map.contains(instanceName)) { + wrt.write("\"failednodes\": ["); + boolean first = true; + for (ExecutionVertex2 vertex : graph.getAllExecutionVertices()) { + if (vertex.getExecutionState() == ExecutionState2.FAILED) { if (first) { first = false; } else { wrt.write(","); } wrt.write("{"); - wrt.write("\"node\": \"" + instanceName + "\","); - wrt.write("\"message\": \"" + StringUtils.escapeHtml(managementVertex.getOptMessage()) + "\""); + wrt.write("\"node\": \"" + vertex.getAssignedResource().getInstance().getInstanceConnectionInfo().hostname() + "\","); + wrt.write("\"message\": \"" + StringUtils.escapeHtml(ExceptionUtils.stringifyException(vertex.getFailureCause())) + "\""); wrt.write("}"); - map.add(instanceName); } } wrt.write("],"); @@ -276,14 +267,14 @@ public class JobmanagerInfoServlet extends HttpServlet { // Serialize ManagementGraph to json wrt.write("\"groupvertices\": ["); boolean first = true; - for(ManagementGroupVertex groupVertex : jobManagementGraph.getGroupVerticesInTopologicalOrder()) { + for (ExecutionJobVertex groupVertex : graph.getVerticesTopologically()) { //Write seperator between json objects if(first) { first = false; } else { wrt.write(","); } - wrt.write(groupVertex.toJson()); + wrt.write(JsonFactory.toJson(groupVertex)); } wrt.write("],"); @@ -304,7 +295,7 @@ public class JobmanagerInfoServlet extends HttpServlet { wrt.write("\"groupverticetimes\": {"); first = true; - for(ManagementGroupVertex groupVertex : jobManagementGraph.getGroupVerticesInTopologicalOrder()) { + for (ExecutionJobVertex groupVertex : graph.getVerticesTopologically()) { if(first) { first = false; @@ -316,17 +307,16 @@ public class JobmanagerInfoServlet extends HttpServlet { long ended = 0; // Take earliest running state and latest endstate of groupmembers - for(int j = 0; j < groupVertex.getNumberOfGroupMembers(); j++) { - ManagementVertex vertex = groupVertex.getGroupMember(j); + for (ExecutionVertex2 vertex : groupVertex.getTaskVertices()) { - long running = jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.RUNNING); - if(running != 0 && running < started) { + long running = vertex.getStateTimestamp(ExecutionState2.RUNNING); + if (running != 0 && running < started) { started = running; } - long finished = jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.FINISHED); - long canceled = jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.CANCELED); - long failed = jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.FAILED); + long finished = vertex.getStateTimestamp(ExecutionState2.FINISHED); + long canceled = vertex.getStateTimestamp(ExecutionState2.CANCELED); + long failed = vertex.getStateTimestamp(ExecutionState2.FAILED); if(finished != 0 && finished > ended) { ended = finished; @@ -342,8 +332,8 @@ public class JobmanagerInfoServlet extends HttpServlet { } - wrt.write("\""+groupVertex.getID()+"\": {"); - wrt.write("\"groupvertexid\": \"" + groupVertex.getID() + "\","); + wrt.write("\""+groupVertex.getJobVertexId()+"\": {"); + wrt.write("\"groupvertexid\": \"" + groupVertex.getJobVertexId() + "\","); wrt.write("\"groupvertexname\": \"" + groupVertex + "\","); wrt.write("\"STARTED\": "+ started + ","); wrt.write("\"ENDED\": "+ ended); @@ -400,9 +390,9 @@ public class JobmanagerInfoServlet extends HttpServlet { wrt.write("\"vertexevents\": ["); first = true; - for(AbstractEvent event: events) { + for (AbstractEvent event: events) { - if(event instanceof ExecutionStateChangeEvent) { + if (event instanceof ExecutionStateChangeEvent) { if(first) { first = false; @@ -411,7 +401,7 @@ public class JobmanagerInfoServlet extends HttpServlet { ExecutionStateChangeEvent vertexevent = (ExecutionStateChangeEvent) event; wrt.write("{"); - wrt.write("\"vertexid\": \"" + vertexevent.getVertexID() + "\","); + wrt.write("\"vertexid\": \"" + vertexevent.getExecutionAttemptID() + "\","); wrt.write("\"newstate\": \"" + vertexevent.getNewExecutionState() + "\","); wrt.write("\"timestamp\": \"" + vertexevent.getTimestamp() + "\""); wrt.write("}"); @@ -454,62 +444,59 @@ public class JobmanagerInfoServlet extends HttpServlet { } /** - * Writes infos about one particular archived groupvertex in a job, including all groupmembers, their times and status - * - * @param wrt - * @param jobEvent - * @param groupvertexId + * Writes info about one particular archived JobVertex in a job, including all member execution vertices, their times and statuses. */ - private void writeJsonForArchivedJobGroupvertex(PrintWriter wrt, RecentJobEvent jobEvent, ManagementGroupVertexID groupvertexId) { - - + private void writeJsonForArchivedJobGroupvertex(PrintWriter wrt, RecentJobEvent jobEvent, JobVertexID vertexId) { try { - - ManagementGraph jobManagementGraph = jobmanager.getManagementGraph(jobEvent.getJobID()); - - ManagementGroupVertex groupvertex = jobManagementGraph.getGroupVertexByID(groupvertexId); - - // Serialize ManagementGraph to json - wrt.write("{\"groupvertex\": "+groupvertex.toJson()+","); - - wrt.write("\"verticetimes\": {"); - boolean first = true; - for(ManagementGroupVertex groupVertex : jobManagementGraph.getGroupVerticesInTopologicalOrder()) { + ExecutionGraph graph = jobmanager.getRecentExecutionGraph(jobEvent.getJobID()); + + ExecutionJobVertex jobVertex = graph.getJobVertex(vertexId); - for(int j = 0; j < groupVertex.getNumberOfGroupMembers(); j++) { - ManagementVertex vertex = groupVertex.getGroupMember(j); + // Serialize ManagementGraph to json + wrt.write("{\"groupvertex\": " + JsonFactory.toJson(jobVertex) + ","); + + wrt.write("\"verticetimes\": {"); + boolean first = true; + for (ExecutionJobVertex groupVertex : graph.getAllVertices().values()) { - if(first) { - first = false; - } else { - wrt.write(","); } + int num = 0; + for (ExecutionVertex2 vertex : groupVertex.getTaskVertices()) { + + if(first) { + first = false; + } else { + wrt.write(","); } + + wrt.write("\""+jobVertex.getJobVertex()+"-"+num +"\": {"); + wrt.write("\"vertexid\": \"" + vertex.getJobvertexId() + "\","); + wrt.write("\"vertexname\": \"" + vertex + "\","); + wrt.write("\"CREATED\": "+ vertex.getStateTimestamp(ExecutionState2.CREATED) + ","); + wrt.write("\"SCHEDULED\": "+ vertex.getStateTimestamp(ExecutionState2.SCHEDULED) + ","); + wrt.write("\"STARTING\": "+ vertex.getStateTimestamp(ExecutionState2.DEPLOYING) + ","); + wrt.write("\"RUNNING\": "+ vertex.getStateTimestamp(ExecutionState2.RUNNING) + ","); + wrt.write("\"FINISHED\": "+ vertex.getStateTimestamp(ExecutionState2.FINISHED) + ","); + wrt.write("\"CANCELING\": "+ vertex.getStateTimestamp(ExecutionState2.CANCELING) + ","); + wrt.write("\"CANCELED\": "+ vertex.getStateTimestamp(ExecutionState2.CANCELED) + ","); + wrt.write("\"FAILED\": "+ vertex.getStateTimestamp(ExecutionState2.FAILED) + ""); + wrt.write("}"); + + num++; + } - wrt.write("\""+vertex.getID()+"\": {"); - wrt.write("\"vertexid\": \"" + vertex.getID() + "\","); - wrt.write("\"vertexname\": \"" + vertex + "\","); - wrt.write("\"CREATED\": "+ jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.CREATED) + ","); - wrt.write("\"SCHEDULED\": "+ jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.SCHEDULED) + ","); - wrt.write("\"ASSIGNED\": "+ jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.ASSIGNED) + ","); - wrt.write("\"READY\": "+ jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.READY) + ","); - wrt.write("\"STARTING\": "+ jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.STARTING) + ","); - wrt.write("\"RUNNING\": "+ jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.RUNNING) + ","); - wrt.write("\"FINISHING\": "+ jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.FINISHING) + ","); - wrt.write("\"FINISHED\": "+ jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.FINISHED) + ","); - wrt.write("\"CANCELING\": "+ jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.CANCELING) + ","); - wrt.write("\"CANCELED\": "+ jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.CANCELED) + ","); - wrt.write("\"FAILED\": "+ jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.FAILED) + ""); - wrt.write("}"); } + wrt.write("}}"); } - wrt.write("}}"); - - } catch (EofException eof) { // Connection closed by client - LOG.info("Info server for jobmanager: Connection closed by client, EofException"); - } catch (IOException ioe) { // Connection closed by client - LOG.info("Info server for jobmanager: Connection closed by client, IOException"); - } - + catch (IOException ioe) { // Connection closed by client + String message = "Info server for jobmanager: Connection closed by client - " + ioe.getClass().getSimpleName(); + + if (LOG.isDebugEnabled()) { + LOG.debug(message, ioe); + } + else if (LOG.isInfoEnabled()) { + LOG.info(message); + } + } } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java new file mode 100644 index 00000000000..908401063b8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmanager.web; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.flink.runtime.execution.ExecutionState2; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex2; +import org.apache.flink.runtime.executiongraph.IntermediateResult; +import org.apache.flink.runtime.instance.AllocatedSlot; +import org.apache.flink.runtime.io.network.channels.ChannelType; +import org.apache.flink.util.StringUtils; + +public class JsonFactory { + + public static String toJson(ExecutionVertex2 vertex) { + StringBuilder json = new StringBuilder(""); + json.append("{"); + json.append("\"vertexid\": \"" + vertex.getJobvertexId() + "\","); + json.append("\"vertexname\": \"" + StringUtils.escapeHtml(vertex.getSimpleName()) + "\","); + json.append("\"vertexstatus\": \"" + vertex.getExecutionState() + "\","); + + AllocatedSlot slot = vertex.getAssignedResource(); + String instanceName = slot == null ? "(null)" : slot.getInstance().getInstanceConnectionInfo().hostname(); + + json.append("\"vertexinstancename\": \"" + instanceName + "\""); + json.append("}"); + return json.toString(); + } + + public static String toJson(ExecutionJobVertex jobVertex) { + StringBuilder json = new StringBuilder(""); + + json.append("{"); + json.append("\"groupvertexid\": \"" + jobVertex.getJobVertexId() + "\","); + json.append("\"groupvertexname\": \"" + StringUtils.escapeHtml(jobVertex.getJobVertex().getName()) + "\","); + json.append("\"numberofgroupmembers\": " + jobVertex.getParallelism() + ","); + json.append("\"groupmembers\": ["); + + // Count state status of group members + Map stateCounts = new HashMap(); + + // initialize with 0 + for (ExecutionState2 state : ExecutionState2.values()) { + stateCounts.put(state, new Integer(0)); + } + + ExecutionVertex2[] vertices = jobVertex.getTaskVertices(); + + for(int j = 0; j < vertices.length; j++) { + ExecutionVertex2 vertex = vertices[j]; + + json.append(toJson(vertex)); + + // print delimiter + if (j != vertices.length - 1) { + json.append(","); + } + + // Increment state status count + Integer count = stateCounts.get(vertex.getExecutionState()) + new Integer(1); + stateCounts.put(vertex.getExecutionState(), count); + } + + json.append("],"); + json.append("\"backwardEdges\": ["); + + List inputs = jobVertex.getInputs(); + + for (int inputNumber = 0; inputNumber < inputs.size(); inputNumber++) { + ExecutionJobVertex input = inputs.get(inputNumber).getProducer(); + + json.append("{"); + json.append("\"groupvertexid\": \"" + input.getJobVertexId() + "\","); + json.append("\"groupvertexname\": \"" + StringUtils.escapeHtml(jobVertex.getJobVertex().getName()) + "\","); + json.append("\"channelType\": \"" + ChannelType.NETWORK + "\""); + json.append("}"); + + // print delimiter + if(inputNumber != inputs.size() - 1) { + json.append(","); + } + } + json.append("]"); + + // list number of members for each status + for (Map.Entry stateCount : stateCounts.entrySet()) { + json.append(",\""+stateCount.getKey()+"\": " + stateCount.getValue()); + } + + json.append("}"); + + return json.toString(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java index 7e5ad3e24eb..79d253439f0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java @@ -16,13 +16,13 @@ * limitations under the License. */ - package org.apache.flink.runtime.jobmanager.web; import java.io.IOException; import java.io.PrintWriter; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.Date; import java.util.List; import java.util.Set; @@ -32,36 +32,34 @@ import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.jobmanager.JobManager; + import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** - * A Servlet that displays the Configruation in the webinterface. - * + * A Servlet that displays the Configuration in the web interface. */ public class SetupInfoServlet extends HttpServlet { - /** - * Serial UID for serialization interoperability. - */ + /** Serial UID for serialization interoperability. */ private static final long serialVersionUID = 3704963598772630435L; - /** - * The log for this class. - */ + /** The log for this class. */ private static final Logger LOG = LoggerFactory.getLogger(SetupInfoServlet.class); + private Configuration globalC; private JobManager jobmanager; + public SetupInfoServlet(JobManager jm) { globalC = GlobalConfiguration.getConfiguration(); this.jobmanager = jm; @@ -79,9 +77,6 @@ public class SetupInfoServlet extends HttpServlet { } else if ("taskmanagers".equals(req.getParameter("get"))) { writeTaskmanagers(resp); } - - - } private void writeGlobalConfiguration(HttpServletResponse resp) throws IOException { @@ -101,27 +96,25 @@ public class SetupInfoServlet extends HttpServlet { PrintWriter w = resp.getWriter(); w.write(obj.toString()); - } private void writeTaskmanagers(HttpServletResponse resp) throws IOException { - Set keys = jobmanager.getInstances().keySet(); - List list = new ArrayList(keys); - Collections.sort(list); + List instances = new ArrayList(jobmanager.getInstanceManager().getAllRegisteredInstances().values()); + + Collections.sort(instances, INSTANCE_SORTER); JSONObject obj = new JSONObject(); JSONArray array = new JSONArray(); - for (InstanceConnectionInfo k : list) { + for (Instance instance : instances) { JSONObject objInner = new JSONObject(); - - Instance instance = jobmanager.getInstances().get(k); + long time = new Date().getTime() - instance.getLastHeartBeat(); try { - objInner.put("inetAdress", k.getInetAdress()); - objInner.put("ipcPort", k.ipcPort()); - objInner.put("dataPort", k.dataPort()); + objInner.put("inetAdress", instance.getInstanceConnectionInfo().getInetAdress()); + objInner.put("ipcPort", instance.getInstanceConnectionInfo().ipcPort()); + objInner.put("dataPort", instance.getInstanceConnectionInfo().dataPort()); objInner.put("timeSinceLastHeartbeat", time / 1000); objInner.put("slotsNumber", instance.getTotalNumberOfSlots()); objInner.put("freeSlots", instance.getNumberOfAvailableSlots()); @@ -130,7 +123,8 @@ public class SetupInfoServlet extends HttpServlet { objInner.put("freeMemory", instance.getResources().getSizeOfJvmHeap() >>> 20); objInner.put("managedMemory", instance.getResources().getSizeOfManagedMemory() >>> 20); array.put(objInner); - } catch (JSONException e) { + } + catch (JSONException e) { LOG.warn("Json object creation failed", e); } @@ -145,4 +139,12 @@ public class SetupInfoServlet extends HttpServlet { w.write(obj.toString()); } + // -------------------------------------------------------------------------------------------- + + private static final Comparator INSTANCE_SORTER = new Comparator() { + @Override + public int compare(Instance o1, Instance o2) { + return o1.getInstanceConnectionInfo().compareTo(o2.getInstanceConnectionInfo()); + } + }; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementAttachment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementAttachment.java deleted file mode 100644 index b858820874d..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementAttachment.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.managementgraph; - -/** - * This class allows to attach an arbitrary {@link Object} to a part of a {@link ManagementGraph}. However, the - * attachment is not included in a possible serialization/deserialization of the graph or its components. - *

- * This class is not thread-safe. - * - */ -public abstract class ManagementAttachment { - - /** - * The attachment to this part of the {@link ManagementGraph}. - */ - private Object attachment; - - /** - * Sets an attachment for this part of the {@link ManagementGraph}. - * - * @param attachment - * the attachment for this part of the {@link ManagementGraph} - */ - public final void setAttachment(final Object attachment) { - this.attachment = attachment; - } - - /** - * Returns the attachment for this part of the {@link ManagementGraph}. - * - * @return the attachment for this part of the {@link ManagementGraph} or null if no attachment is set - */ - public final Object getAttachment() { - return this.attachment; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdge.java deleted file mode 100644 index 9bd37f53770..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdge.java +++ /dev/null @@ -1,166 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.managementgraph; - -import org.apache.flink.runtime.io.network.channels.ChannelType; - -/** - * This class implements a directed edge of a {@link ManagementGraph}. The edge is derived from a channel of the actual - * execution graph. - *

- * This class is not thread-safe. - * - */ -public final class ManagementEdge extends ManagementAttachment { - - /** - * The source of the edge referring to the output gate of an execution vertex. - */ - private final ManagementGate source; - - /** - * The target of the edge referring to the input gate of an execution vertex. - */ - private final ManagementGate target; - - /** - * The edge's index in the source gate. - */ - private final int sourceIndex; - - /** - * The edge's index in the target gate. - */ - private final int targetIndex; - - /** - * The type of the channel this edge refers to. - */ - private final ChannelType channelType; - - /** - * The source ID of the management edge. - */ - private final ManagementEdgeID sourceEdgeID; - - /** - * The target ID of the management edge. - */ - private final ManagementEdgeID targetEdgeID; - - /** - * Constructs a new edge object. - * - * @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 - * the edge's index in the source gate - * @param target - * the target of the edge referring to the input gate of an execution vertex - * @param targetIndex - * the edge's index in the target gate - * @param channelType - * the type of the channel this edge refers to - * @param compressionLevel - * the compression level of the channel this edge refers to - */ - public ManagementEdge(final ManagementEdgeID sourceEdgeID, final ManagementEdgeID targetEdgeID, - final ManagementGate source, final int sourceIndex, final ManagementGate target, final int targetIndex, - final ChannelType channelType) { - - this.sourceEdgeID = sourceEdgeID; - this.targetEdgeID = targetEdgeID; - this.source = source; - this.target = target; - this.sourceIndex = sourceIndex; - this.targetIndex = targetIndex; - this.channelType = channelType; - - this.source.insertForwardEdge(this, sourceIndex); - this.target.insertBackwardEdge(this, targetIndex); - } - - /** - * Returns the type of the channel this edge refers to. - * - * @return the type of the channel this edge refers to - */ - public ChannelType getChannelType() { - return this.channelType; - } - - /** - * Returns the source of the edge referring to the output gate of an execution vertex. - * - * @return the source of the edge referring to the output gate of an execution vertex - */ - public ManagementGate getSource() { - return this.source; - } - - /** - * Returns the target of the edge referring to the input gate of an execution vertex. - * - * @return the target of the edge referring to the input gate of an execution vertex - */ - public ManagementGate getTarget() { - return this.target; - } - - /** - * Returns the edge's index in the source gate. - * - * @return the edge's index in the source gate - */ - public int getSourceIndex() { - return this.sourceIndex; - } - - /** - * Returns the edge's index in the target gate. - * - * @return the edge's index in the target gate - */ - public int getTargetIndex() { - return this.targetIndex; - } - - /** - * 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 target ID of the edge - */ - public ManagementEdgeID getTargetEdgeID() { - return targetEdgeID; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdgeID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdgeID.java deleted file mode 100644 index 16ba9741ba6..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdgeID.java +++ /dev/null @@ -1,55 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.managementgraph; - -import org.apache.flink.runtime.AbstractID; -import org.apache.flink.runtime.io.network.channels.ChannelID; - -/** - * A management edge ID uniquely identifies a {@link ManagementEdge}. - */ -public class ManagementEdgeID extends AbstractID { - - private static final long serialVersionUID = 1L; - - /** - * Initializes ManagementEdgeID. - */ - public ManagementEdgeID() {} - - /** - * A ManagementEdgeID is derived from the #{@link ChannelID} of the corresponding - * output channel in the execution graph. - * - * @param source - * ID of the corresponding output channel - */ - public ManagementEdgeID(ChannelID source) { - super(source); - } - - /** - * Converts the management edge ID into a {@link ChannelID}. - * - * @return the corresponding channelID. - */ - public ChannelID toChannelID() { - return new ChannelID(this); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGate.java deleted file mode 100644 index ce965737e0e..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGate.java +++ /dev/null @@ -1,219 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.managementgraph; - -import java.util.ArrayList; -import java.util.List; - -/** - * This class implements an input or output gate of a {@link ManagementVertex}. The gate is derived an input or output - * gate of the actual execution vertex. - *

- * This class is not thread-safe. - * - */ -public final class ManagementGate extends ManagementAttachment { - - /** - * The management vertex this gate belongs to. - */ - private final ManagementVertex managementVertex; - - /** - * true if this gate represents an input gate in the actual execution graph, false - * otherwise. - */ - private final boolean isInputGate; - - /** - * The index of this gate. - */ - private final int index; - - /** - * A list of edges originating from this gate. - */ - private final List forwardEdges = new ArrayList(); - - /** - * A list of edges arriving at this gate. - */ - private final List backwardEdges = new ArrayList(); - - /** - * The id of the management gate. - */ - private ManagementGateID gateID; - - /** - * Constructs a new management gate. - * - * @param managementVertex - * 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 - */ - public ManagementGate(final ManagementVertex managementVertex, final ManagementGateID gateID, - final int index, final boolean isInputGate) { - this.isInputGate = isInputGate; - this.managementVertex = managementVertex; - this.gateID = gateID; - this.index = index; - - managementVertex.addGate(this); - } - - /** - * Checks if this gate represents an input gate. - * - * @return true if this gate represents an input gate in the actual execution graph, false - * otherwise - */ - public boolean isInputGate() { - return this.isInputGate; - } - - /** - * Adds a new edge which originates at this gate. - * - * @param managementEdge - * the edge to be added - * @param index - * the index at which the edge shall be added - */ - void insertForwardEdge(final ManagementEdge managementEdge, final int index) { - - while (index >= this.forwardEdges.size()) { - this.forwardEdges.add(null); - } - - this.forwardEdges.set(index, managementEdge); - } - - /** - * Adds a new edge which arrives at this gate. - * - * @param managementEdge - * the edge to be added - * @param index - * the index at which the edge shall be added - */ - void insertBackwardEdge(final ManagementEdge managementEdge, final int index) { - - while (index >= this.backwardEdges.size()) { - this.backwardEdges.add(null); - } - - this.backwardEdges.set(index, managementEdge); - } - - /** - * Returns the {@link ManagementGraph} this gate belongs to. - * - * @return the management graph this gate belongs to - */ - public ManagementGraph getGraph() { - return this.managementVertex.getGraph(); - } - - /** - * Returns the number of edges originating at this gate. - * - * @return the number of edges originating at this gate - */ - public int getNumberOfForwardEdges() { - - return this.forwardEdges.size(); - } - - /** - * Returns the number of edges arriving at this gate. - * - * @return the number of edges arriving at this gate - */ - public int getNumberOfBackwardEdges() { - - return this.backwardEdges.size(); - } - - /** - * Returns the index of this gate. - * - * @return the index of this gate - */ - public int getIndex() { - return this.index; - } - - /** - * Returns the edge originating at the given index. - * - * @param index - * the index of the edge to be returned - * @return the edge at the given index or null if no such edge exists - */ - public ManagementEdge getForwardEdge(final int index) { - - if (index < this.forwardEdges.size()) { - return this.forwardEdges.get(index); - } - - return null; - } - - /** - * Returns the edge arriving at the given index. - * - * @param index - * the index of the edge to be returned - * @return the edge at the given index or null if no such edge exists - */ - public ManagementEdge getBackwardEdge(final int index) { - - if (index < this.backwardEdges.size()) { - return this.backwardEdges.get(index); - } - - return null; - } - - /** - * Returns the vertex this gate belongs to. - * - * @return the vertex this gate belongs to - */ - 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/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGateID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGateID.java deleted file mode 100644 index 3e890d4a61d..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGateID.java +++ /dev/null @@ -1,28 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.managementgraph; - -import org.apache.flink.runtime.AbstractID; - -/** - * A management gate ID uniquely identifies a {@link ManagementGate}. - */ -public final class ManagementGateID extends AbstractID { - private static final long serialVersionUID = 1L; -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGraph.java deleted file mode 100644 index 1ad4d101364..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGraph.java +++ /dev/null @@ -1,562 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.managementgraph; - -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; -import java.util.List; -import java.util.Map; - -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.core.io.StringRecord; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.io.network.channels.ChannelType; -import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.util.EnumUtils; - -/** - * A management graph is structurally equal to the graph Nephele uses internally for scheduling jobs. Management graphs - * are intended to provide more fine-grained information about a job at runtime than available through the regular - * client interface, however, without exposing Nephele's internal scheduling data structures. - *

- * This class is not thread-safe. - * - */ -public final class ManagementGraph extends ManagementAttachment implements IOReadableWritable { - - /** - * List of stages the graph is divided into. - */ - private final List stages = new ArrayList(); - - /** - * The ID of the job this graph describes. - */ - private final JobID jobID; - - /** - * A map of vertices this graph consists of. - */ - private final Map vertices = new HashMap(); - - /** - * A map of group vertices this graph consists of. - */ - private final Map groupVertices = new HashMap(); - - /** - * Constructs a new management graph with the given job ID. - * - * @param jobID - * the job ID of the graph. - */ - public ManagementGraph(final JobID jobID) { - this.jobID = jobID; - } - - /** - * Constructs a new management graph with a random job ID. - */ - public ManagementGraph() { - this.jobID = new JobID(); - } - - /** - * Adds a new management stage to the graph. - * - * @param mangementStage - * the management stage to be added. - */ - void addStage(final ManagementStage mangementStage) { - - this.stages.add(mangementStage); - } - - /** - * Returns the ID of the job this graph describes. - * - * @return the ID of the job this graph describes - */ - public JobID getJobID() { - return this.jobID; - } - - /** - * Adds the given vertex to the graph's internal vertex map. - * - * @param id - * the ID of the vertex to be added - * @param vertex - * the vertex to be added - */ - void addVertex(final ManagementVertexID id, final ManagementVertex vertex) { - - this.vertices.put(id, vertex); - } - - /** - * Returns the vertex with the given ID from the graph's internal vertex map. - * - * @param id - * the ID of the vertex to be returned - * @return the vertex with the given ID or null if no such vertex exists - */ - public ManagementVertex getVertexByID(final ManagementVertexID id) { - - return this.vertices.get(id); - } - - /** - * Returns the group vertex with the given ID from the graph's internal group vertex map. - * - * @param id - * the ID of the group vertex to be returned - * @return the group vertex with the given ID or null if no such group vertex exists - */ - public ManagementGroupVertex getGroupVertexByID(final ManagementGroupVertexID id) { - - return this.groupVertices.get(id); - } - - /** - * Adds the given group vertex to the graph's internal group vertex map. - * - * @param id - * the ID of the group vertex to be added - * @param groupVertex - * the group vertex to be added - */ - void addGroupVertex(final ManagementGroupVertexID id, final ManagementGroupVertex groupVertex) { - - this.groupVertices.put(id, groupVertex); - } - - /** - * Returns the number of stages in this management graph. - * - * @return the number of stages in this management graph - */ - public int getNumberOfStages() { - - return this.stages.size(); - } - - /** - * Returns the management stage with the given index. - * - * @param index - * the index of the management stage to be returned - * @return the management stage with the given index or null if no such management stage exists - */ - public ManagementStage getStage(final int index) { - - if (index >= 0 && index < this.stages.size()) { - return this.stages.get(index); - } - - return null; - } - - /** - * Returns the number of input group vertices in the management stage with the given index. - * - * @param stage - * the index to the management stage - * @return the number of input group vertices in this stage, possibly 0. - */ - public int getNumberOfInputGroupVertices(final int stage) { - - if (stage < 0 || stage >= this.stages.size()) { - return 0; - } - - return this.stages.get(stage).getNumberOfInputGroupVertices(); - } - - /** - * Returns the number of output group vertices in the management stage with the given index. - * - * @param stage - * the index to the management stage - * @return the number of output group vertices in this stage, possibly 0. - */ - public int getNumberOfOutputGroupVertices(final int stage) { - - if (stage >= this.stages.size()) { - return 0; - } - - return this.stages.get(stage).getNumberOfOutputGroupVertices(); - } - - /** - * Returns the input group vertex at the given index in the given stage. - * - * @param stage - * the index to the management stage - * @param index - * the index to the input group vertex - * @return the input group vertex at the given index in the given stage or null if either the stage - * does not exists or the given index is invalid in this stage - */ - public ManagementGroupVertex getInputGroupVertex(final int stage, final int index) { - - if (stage >= this.stages.size()) { - return null; - } - - return this.stages.get(stage).getInputGroupVertex(index); - } - - /** - * Returns the output group vertex at the given index in the given stage. - * - * @param stage - * the index to the management stage - * @param index - * the index to the output group vertex - * @return the output group vertex at the given index in the given stage or null if either the stage - * does not exists or the given index is invalid in this stage - */ - public ManagementGroupVertex getOutputGroupVertex(final int stage, final int index) { - - if (stage >= this.stages.size()) { - return null; - } - - return this.stages.get(stage).getOutputGroupVertex(index); - } - - /** - * Returns the number of input vertices for the given stage. - * - * @param stage - * the index of the management stage - * @return the number of input vertices for the given stage - */ - public int getNumberOfInputVertices(final int stage) { - - if (stage >= this.stages.size()) { - return 0; - } - - return this.stages.get(stage).getNumberOfInputManagementVertices(); - } - - /** - * Returns the number of output vertices for the given stage. - * - * @param stage - * the index of the management stage - * @return the number of input vertices for the given stage - */ - public int getNumberOfOutputVertices(final int stage) { - - if (stage >= this.stages.size()) { - return 0; - } - - return this.stages.get(stage).getNumberOfInputManagementVertices(); - } - - /** - * Returns the input vertex with the specified index for the given stage. - * - * @param stage - * the index of the stage - * @param index - * the index of the input vertex to return - * @return the input vertex with the specified index or null if no input vertex with such an index - * exists in that stage - */ - public ManagementVertex getInputVertex(final int stage, final int index) { - - if (stage >= this.stages.size()) { - return null; - } - - return this.stages.get(stage).getInputManagementVertex(index); - } - - /** - * Returns the output vertex with the specified index for the given stage. - * - * @param stage - * the index of the stage - * @param index - * the index of the output vertex to return - * @return the output vertex with the specified index or null if no output vertex with such an index - * exists in that stage - */ - public ManagementVertex getOutputVertex(final int stage, final int index) { - - if (stage >= this.stages.size()) { - return null; - } - - 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. - * - * @return a list of group vertices sorted in topological order - */ - public List getGroupVerticesInTopologicalOrder() { - - final List topologicalSort = new ArrayList(); - final Deque noIncomingEdges = new ArrayDeque(); - final Map indegrees = new HashMap(); - - final Iterator it = new ManagementGroupVertexIterator(this, true, -1); - while (it.hasNext()) { - final ManagementGroupVertex groupVertex = it.next(); - indegrees.put(groupVertex, Integer.valueOf(groupVertex.getNumberOfBackwardEdges())); - if (groupVertex.getNumberOfBackwardEdges() == 0) { - noIncomingEdges.add(groupVertex); - } - } - - while (!noIncomingEdges.isEmpty()) { - - final ManagementGroupVertex groupVertex = noIncomingEdges.removeFirst(); - topologicalSort.add(groupVertex); - // Decrease indegree of connected vertices - for (int i = 0; i < groupVertex.getNumberOfForwardEdges(); i++) { - final ManagementGroupVertex targetVertex = groupVertex.getForwardEdge(i).getTarget(); - Integer indegree = indegrees.get(targetVertex); - indegree = Integer.valueOf(indegree.intValue() - 1); - indegrees.put(targetVertex, indegree); - if (indegree.intValue() == 0) { - noIncomingEdges.add(targetVertex); - } - - } - } - - return topologicalSort; - } - - /** - * Returns a list of group vertices sorted in reverse topological order. - * - * @return a list of group vertices sorted in reverse topological order - */ - public List getGroupVerticesInReverseTopologicalOrder() { - - final List reverseTopologicalSort = new ArrayList(); - final Deque noOutgoingEdges = new ArrayDeque(); - final Map outdegrees = new HashMap(); - - final Iterator it = new ManagementGroupVertexIterator(this, false, -1); - while (it.hasNext()) { - final ManagementGroupVertex groupVertex = it.next(); - outdegrees.put(groupVertex, Integer.valueOf(groupVertex.getNumberOfForwardEdges())); - if (groupVertex.getNumberOfForwardEdges() == 0) { - noOutgoingEdges.add(groupVertex); - } - } - - while (!noOutgoingEdges.isEmpty()) { - - final ManagementGroupVertex groupVertex = noOutgoingEdges.removeFirst(); - reverseTopologicalSort.add(groupVertex); - // Decrease indegree of connected vertices - for (int i = 0; i < groupVertex.getNumberOfBackwardEdges(); i++) { - final ManagementGroupVertex sourceVertex = groupVertex.getBackwardEdge(i).getSource(); - Integer outdegree = outdegrees.get(sourceVertex); - outdegree = Integer.valueOf(outdegree.intValue() - 1); - outdegrees.put(sourceVertex, outdegree); - if (outdegree.intValue() == 0) { - noOutgoingEdges.add(sourceVertex); - } - - } - } - - return reverseTopologicalSort; - } - - - @Override - public void read(final DataInputView in) throws IOException { - - // Read job ID - this.jobID.read(in); - - // Recreate stages - final int numberOfStages = in.readInt(); - for (int i = 0; i < numberOfStages; i++) { - new ManagementStage(this, i); - } - - // Read number of group vertices and their corresponding IDs - final int numberOfGroupVertices = in.readInt(); - for (int i = 0; i < numberOfGroupVertices; i++) { - - final ManagementGroupVertexID groupVertexID = new ManagementGroupVertexID(); - groupVertexID.read(in); - final ManagementStage stage = this.stages.get(in.readInt()); - final String groupVertexName = StringRecord.readString(in); - new ManagementGroupVertex(stage, groupVertexID, groupVertexName); - } - - for (int i = 0; i < numberOfGroupVertices; i++) { - final ManagementGroupVertexID groupVertexID = new ManagementGroupVertexID(); - groupVertexID.read(in); - final ManagementGroupVertex groupVertex = this.groupVertices.get(groupVertexID); - groupVertex.read(in); - } - - // Read the management vertices - int numberOfVertices = in.readInt(); - for (int i = 0; i < numberOfVertices; i++) { - - final ManagementVertexID vertexID = new ManagementVertexID(); - vertexID.read(in); - final ManagementGroupVertexID groupVertexID = new ManagementGroupVertexID(); - groupVertexID.read(in); - final ManagementGroupVertex groupVertex = this.getGroupVertexByID(groupVertexID); - final String instanceName = StringRecord.readString(in); - final int indexInGroup = in.readInt(); - final ManagementVertex vertex = new ManagementVertex(groupVertex, vertexID, instanceName, indexInGroup); - vertex.read(in); - } - - for (int i = 0; i < numberOfVertices; i++) { - - final ManagementVertexID sourceID = new ManagementVertexID(); - sourceID.read(in); - final ManagementVertex sourceVertex = getVertexByID(sourceID); - for (int j = 0; j < sourceVertex.getNumberOfOutputGates(); j++) { - 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); - final int targetGateIndex = in.readInt(); - final ManagementGate targetGate = targetVertex.getInputGate(targetGateIndex); - - final int sourceIndex = in.readInt(); - final int targetIndex = in.readInt(); - - final ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class); - new ManagementEdge(sourceEdgeID, targetEdgeID, sourceGate, sourceIndex, targetGate, targetIndex, - channelType); - } - - } - } - } - - - @Override - public void write(final DataOutputView out) throws IOException { - - // Write job ID - this.jobID.write(out); - - // Write number of stages - out.writeInt(this.stages.size()); - - // Write number of group vertices and their corresponding IDs - out.writeInt(this.groupVertices.size()); - Iterator it = new ManagementGroupVertexIterator(this, true, -1); - - while (it.hasNext()) { - - final ManagementGroupVertex groupVertex = it.next(); - groupVertex.getID().write(out); - out.writeInt(groupVertex.getStage().getStageNumber()); - StringRecord.writeString(out, groupVertex.getName()); - } - - it = new ManagementGroupVertexIterator(this, true, -1); - while (it.hasNext()) { - - final ManagementGroupVertex groupVertex = it.next(); - groupVertex.getID().write(out); - groupVertex.write(out); - } - - // Write out the management vertices and their corresponding IDs - out.writeInt(this.vertices.size()); - Iterator it2 = new ManagementGraphIterator(this, true); - while (it2.hasNext()) { - - final ManagementVertex managementVertex = it2.next(); - managementVertex.getID().write(out); - managementVertex.getGroupVertex().getID().write(out); - StringRecord.writeString(out, managementVertex.getInstanceName()); - out.writeInt(managementVertex.getIndexInGroup()); - managementVertex.write(out); - } - - // Finally, serialize the edges between the management vertices - it2 = vertices.values().iterator(); - while (it2.hasNext()) { - - final ManagementVertex managementVertex = it2.next(); - managementVertex.getID().write(out); - for (int i = 0; i < managementVertex.getNumberOfOutputGates(); i++) { - final ManagementGate outputGate = managementVertex.getOutputGate(i); - 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()); - - out.writeInt(edge.getSourceIndex()); - out.writeInt(edge.getTargetIndex()); - - EnumUtils.writeEnum(out, edge.getChannelType()); - } - } - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGraphIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGraphIterator.java deleted file mode 100644 index c2b8ef3fa6f..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGraphIterator.java +++ /dev/null @@ -1,456 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.managementgraph; - -import java.util.HashSet; -import java.util.Iterator; -import java.util.Set; -import java.util.Stack; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This class provides an implementation of the {@link java.util.Iterator} interface which allows to - * traverse a management graph and visit every reachable vertex exactly once. The order - * in which the vertices are visited corresponds to the order of their discovery in a depth first - * search. - *

- * This class is not thread-safe. - * - */ -public final class ManagementGraphIterator implements Iterator { - - /** - * The log object used for debugging. - */ - private static final Logger LOG = LoggerFactory.getLogger(ManagementGraphIterator.class); - - /** - * The management graph this iterator traverses. - */ - private final ManagementGraph managementGraph; - - /** - * Stores whether the graph is traversed starting from the input or the output vertices. - */ - private final boolean forward; - - /** - * The stage that should be traversed by this iterator. - */ - private final int startStage; - - /** - * Stores whether the iterator is confined to the start stage or not. - */ - private final boolean confinedToStage; - - /** - * The number of visited vertices from the entry set (either input or output vertices). - */ - private int numVisitedEntryVertices = 0; - - /** - * Stack used for the depth first search. - */ - private final Stack traversalStack = new Stack(); - - /** - * Set of already visited vertices during traversal. - */ - private final Set alreadyVisited = new HashSet(); - - /** - * Auxiliary class which stores which vertices have already been visited. - * - */ - private static class TraversalEntry { - - /** - * Management vertex this entry has been created for. - */ - private final ManagementVertex managementVertex; - - /** - * Next gate to traverse. - */ - private int currentGate; - - /** - * Next channel to traverse. - */ - private int currentChannel; - - /** - * Constructs a new traversal entry. - * - * @param managementVertex - * the management vertex this entry belongs to - * @param currentGate - * the gate index to use to visit the next vertex - * @param currentChannel - * the channel index to use to visit the next vertex - */ - public TraversalEntry(final ManagementVertex managementVertex, final int currentGate, final int currentChannel) { - this.managementVertex = managementVertex; - this.currentGate = currentGate; - this.currentChannel = currentChannel; - } - - /** - * Returns the management vertex this entry belongs to. - * - * @return the managenemt vertex this entry belongs to - */ - public ManagementVertex getManagementVertex() { - return this.managementVertex; - } - - /** - * Returns the gate index to use to visit the next vertex. - * - * @return the gate index to use to visit the next vertex - */ - public int getCurrentGate() { - return this.currentGate; - } - - /** - * Returns the channel index to use to visit the next vertex. - * - * @return the channel index to use to visit the next vertex - */ - public int getCurrentChannel() { - return this.currentChannel; - } - - /** - * Increases the channel index by one. - */ - public void increaseCurrentChannel() { - ++this.currentChannel; - } - - /** - * Increases the gate index by one. - */ - public void increaseCurrentGate() { - ++this.currentGate; - } - - /** - * Resets the channel index. - */ - public void resetCurrentChannel() { - this.currentChannel = 0; - } - - } - - /** - * Creates a new management graph iterator. - * - * @param managementGraph - * the management graph that should be traversed - * @param forward - * true if the graph should be traversed in correct order, false to traverse it in - * reverse order - */ - public ManagementGraphIterator(final ManagementGraph managementGraph, final boolean forward) { - this(managementGraph, forward ? 0 : (managementGraph.getNumberOfStages() - 1), false, forward); - } - - /** - * Creates a new management graph iterator. - * - * @param managementGraph - * the management graph that should be traversed - * @param startStage - * the index of the stage of the graph where the traversal is supposed to begin - * @param confinedToStage - * false if the graph iterator is allowed to traverse to upper (in case of reverse order - * traversal lower) stages, true otherwise. - * @param forward - * true if the graph should be traversed in correct order, false to traverse it in - * reverse order - */ - public ManagementGraphIterator(final ManagementGraph managementGraph, final int startStage, - final boolean confinedToStage, final boolean forward) { - - this.managementGraph = managementGraph; - this.forward = forward; - this.startStage = startStage; - this.confinedToStage = confinedToStage; - - if (startStage >= this.managementGraph.getNumberOfStages()) { - return; - } - - if (forward) { - if (managementGraph.getNumberOfInputVertices(startStage) > 0) { - - final TraversalEntry te = new TraversalEntry(managementGraph.getInputVertex(startStage, 0), 0, 0); - this.traversalStack.push(te); - this.alreadyVisited.add(te.getManagementVertex()); - - } - } else { - if (managementGraph.getNumberOfOutputVertices(startStage) > 0) { - - final TraversalEntry te = new TraversalEntry(managementGraph.getOutputVertex(startStage, 0), 0, 0); - this.traversalStack.push(te); - this.alreadyVisited.add(te.getManagementVertex()); - } - - } - } - - /** - * Creates a new management graph iterator. This constructor can be used to - * traverse only specific parts of the graph starting at startVertex. - * The iterator will not switch to the next input/output vertex of an output/input vertex - * has been reached. - * - * @param managementGraph - * the management graph that should be traversed - * @param startVertex - * the vertex to start the traversal from - * @param forward - * true if the graph should be traversed in correct order, false to reverse it in - * reverse order - */ - public ManagementGraphIterator(final ManagementGraph managementGraph, final ManagementVertex startVertex, - final boolean forward) { - - this.managementGraph = managementGraph; - this.forward = forward; - this.numVisitedEntryVertices = -1; - this.startStage = 0; - this.confinedToStage = false; - - final TraversalEntry te = new TraversalEntry(startVertex, 0, 0); - this.traversalStack.push(te); - this.alreadyVisited.add(te.getManagementVertex()); - } - - - @Override - public boolean hasNext() { - - if (this.traversalStack.isEmpty()) { - - if (this.numVisitedEntryVertices < 0) { - // User chose a specific starting vertex - return false; - } - - ++this.numVisitedEntryVertices; - - if (this.forward) { - if (this.managementGraph.getNumberOfInputVertices(this.startStage) <= this.numVisitedEntryVertices) { - return false; - } - } else { - if (this.managementGraph.getNumberOfOutputVertices(this.startStage) <= this.numVisitedEntryVertices) { - return false; - } - } - } - - return true; - } - - - @Override - public ManagementVertex next() { - - if (this.traversalStack.isEmpty()) { - - if (this.numVisitedEntryVertices < 0) { - // User chose a specific entry vertex - return null; - } - - TraversalEntry newentry; - - if (this.forward) { - newentry = new TraversalEntry(this.managementGraph.getInputVertex(this.startStage, this.numVisitedEntryVertices), - 0, 0); - } else { - newentry = new TraversalEntry( - managementGraph.getOutputVertex(this.startStage, this.numVisitedEntryVertices), 0, 0); - } - - this.traversalStack.push(newentry); - this.alreadyVisited.add(newentry.getManagementVertex()); - } - - final ManagementVertex returnVertex = this.traversalStack.peek().getManagementVertex(); - - // Propose vertex to be visited next - do { - - final TraversalEntry te = this.traversalStack.peek(); - - // Check if we can traverse deeper into the graph - final ManagementVertex candidateVertex = getCandidateVertex(te, this.forward); - if (candidateVertex == null) { - // Pop it from the stack - this.traversalStack.pop(); - } else { - // Create new entry and put it on the stack - final TraversalEntry newte = new TraversalEntry(candidateVertex, 0, 0); - this.traversalStack.push(newte); - this.alreadyVisited.add(candidateVertex); - break; - } - - } while (!this.traversalStack.isEmpty()); - - return returnVertex; - } - - /** - * Returns a candidate vertex which could potentially be visited next because it is reachable from the - * currently considered vertex. - * - * @param te - * the traversal entry for the current source vertex - * @param forward - * true if the graph should be traversed in correct order, false to traverse it in - * reverse order - * @return a candidate vertex which could potentially be visited next - */ - private ManagementVertex getCandidateVertex(final TraversalEntry te, final boolean forward) { - - if (forward) { - - while (true) { - - if (this.confinedToStage && te.getCurrentChannel() == 0) { - while (currentGateLeadsToOtherStage(te, this.forward)) { - te.increaseCurrentGate(); - } - } - - // No more outgoing edges to consider - if (te.getCurrentGate() >= te.getManagementVertex().getNumberOfOutputGates()) { - break; - } - - if (te.getCurrentChannel() >= te.getManagementVertex().getOutputGate(te.getCurrentGate()) - .getNumberOfForwardEdges()) { - te.increaseCurrentGate(); - te.resetCurrentChannel(); - } else { - final ManagementEdge forwardEdge = te.getManagementVertex().getOutputGate(te.getCurrentGate()) - .getForwardEdge(te.getCurrentChannel()); - final ManagementVertex target = forwardEdge.getTarget().getVertex(); - te.increaseCurrentChannel(); - if (!alreadyVisited.contains(target)) { - return target; - } - } - } - } else { - - while (true) { - - if (this.confinedToStage && te.getCurrentChannel() == 0) { - while (currentGateLeadsToOtherStage(te, this.forward)) { - te.increaseCurrentGate(); - } - } - - // No more incoming edges to consider - if (te.getCurrentGate() >= te.getManagementVertex().getNumberOfInputGates()) { - break; - } - - if (te.getCurrentChannel() >= te.getManagementVertex().getInputGate(te.getCurrentGate()) - .getNumberOfBackwardEdges()) { - te.increaseCurrentGate(); - te.resetCurrentChannel(); - } else { - final ManagementEdge backwardEdge = te.getManagementVertex().getInputGate(te.getCurrentGate()) - .getBackwardEdge(te.getCurrentChannel()); - final ManagementVertex source = backwardEdge.getSource().getVertex(); - if (source == null) { - LOG.error("Inconsistency in vertex map found (backward)!"); - } - te.increaseCurrentChannel(); - if (!this.alreadyVisited.contains(source)) { - return source; - } - } - } - } - - return null; - } - - /** - * Checks if the current gate leads to another stage or not. - * - * @param te - * the current traversal entry - * @param forward - * true if the graph should be traversed in correct order, false to traverse it in - * reverse order - * @return true if current gate leads to another stage, otherwise false - */ - private boolean currentGateLeadsToOtherStage(final TraversalEntry te, final boolean forward) { - - final ManagementGroupVertex groupVertex = te.getManagementVertex().getGroupVertex(); - - if (forward) { - - if (te.getCurrentGate() >= groupVertex.getNumberOfForwardEdges()) { - return false; - } - - final ManagementGroupEdge edge = groupVertex.getForwardEdge(te.getCurrentGate()); - if (edge.getTarget().getStageNumber() == groupVertex.getStageNumber()) { - return false; - } - - } else { - - if (te.getCurrentGate() >= groupVertex.getNumberOfBackwardEdges()) { - return false; - } - - final ManagementGroupEdge edge = groupVertex.getBackwardEdge(te.getCurrentGate()); - if (edge.getSource().getStageNumber() == groupVertex.getStageNumber()) { - return false; - } - } - - return true; - } - - - @Override - public void remove() { - // According to the Iterator documentation this method is optional. - } - -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupEdge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupEdge.java deleted file mode 100644 index 7c955cfd3d4..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupEdge.java +++ /dev/null @@ -1,131 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.managementgraph; - -import org.apache.flink.runtime.io.network.channels.ChannelType; - -/** - * This class implements a directed edge of between two {@link ManagementGroupVertex} objects. The edge is derived from - * a group edge of the actual execution graph. - *

- * This class is not thread-safe. - * - */ -public final class ManagementGroupEdge extends ManagementAttachment { - - /** - * The source vertex of this group edge. - */ - private final ManagementGroupVertex source; - - /** - * The target vertex of this group edge. - */ - private final ManagementGroupVertex target; - - /** - * The edge's index at the source vertex. - */ - private final int sourceIndex; - - /** - * The edge's index at the target vertex. - */ - private final int targetIndex; - - /** - * The type of the channels represented by this group edge. - */ - private final ChannelType channelType; - - /** - * Constructs a new management group edge. - * - * @param source - * the source vertex of the new group edge - * @param sourceIndex - * the index at the source vertex - * @param target - * the target vertex of the new group edge - * @param targetIndex - * the index at the target vertex - * @param channelType - * the type of the channels represented by the new group edge - * @param compressionLevel - * the compression level of the channels represented by the new group edge - */ - public ManagementGroupEdge(final ManagementGroupVertex source, final int sourceIndex, - final ManagementGroupVertex target, final int targetIndex, final ChannelType channelType) { - this.source = source; - this.target = target; - this.sourceIndex = sourceIndex; - this.targetIndex = targetIndex; - this.channelType = channelType; - - source.insertForwardEdge(this, sourceIndex); - target.insertBackwardEdge(this, targetIndex); - } - - /** - * Returns the type of the channels represented by this group edge. - * - * @return the type of the channels represented by this group edge - */ - public ChannelType getChannelType() { - return this.channelType; - } - - - /** - * Returns the source vertex of this group edge. - * - * @return the source vertex of this group edge - */ - public ManagementGroupVertex getSource() { - return this.source; - } - - /** - * Returns the target vertex of this group edge. - * - * @return the target vertex of this group edge - */ - public ManagementGroupVertex getTarget() { - return this.target; - } - - /** - * Returns the edge's index at the source vertex. - * - * @return the edge's index at the source vertex - */ - public int getSourceIndex() { - return this.sourceIndex; - } - - /** - * Returns the edge's index at the target vertex. - * - * @return the edges's index at the target vertex - */ - public int getTargetIndex() { - return this.targetIndex; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertex.java deleted file mode 100644 index 4e168b478cf..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertex.java +++ /dev/null @@ -1,485 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.managementgraph; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.io.network.channels.ChannelType; -import org.apache.flink.runtime.util.EnumUtils; -import org.apache.flink.util.StringUtils; - -/** - * This class implements a management group vertex of a {@link ManagementGraph}. A management group vertex is derived - * from the type of group vertices Nephele uses in its internal scheduling structures. - *

- * This class is not thread-safe. - * - */ -public final class ManagementGroupVertex extends ManagementAttachment implements IOReadableWritable { - - /** - * The ID of the management group vertex. - */ - private final ManagementGroupVertexID id; - - /** - * The name of the management group vertex. - */ - private final String name; - - /** - * The stage this management group vertex belongs to. - */ - private final ManagementStage stage; - - /** - * The list of {@link ManagementVertex} contained in this group vertex. - */ - private final List groupMembers = new ArrayList(); - - /** - * The list of group edges which originate from this group vertex. - */ - private final List forwardEdges = new ArrayList(); - - /** - * The list of group edges which arrive at this group vertex. - */ - private final List backwardEdges = new ArrayList(); - - /** - * Constructs a new management group vertex. - * - * @param stage - * the stage this group vertex belongs to - * @param name - * the name of the new management group vertex - */ - public ManagementGroupVertex(final ManagementStage stage, final String name) { - this(stage, new ManagementGroupVertexID(), name); - } - - /** - * Constructs a new management group vertex. - * - * @param stage - * the stage this group vertex belongs to - * @param id - * the ID of the management group vertex - * @param name - * the name of the new management group vertex - */ - public ManagementGroupVertex(final ManagementStage stage, final ManagementGroupVertexID id, final String name) { - - this.stage = stage; - this.id = id; - this.name = name; - - stage.addGroupVertex(this); - } - - /** - * Returns the ID of this management group vertex. - * - * @return the ID of this management group vertex - */ - public ManagementGroupVertexID getID() { - return this.id; - } - - /** - * Returns the name of this management group vertex. - * - * @return the anme of this management group vertex, possibly null - */ - public String getName() { - return name; - } - - /** - * Inserts a new edge starting at this group vertex at the given index. - * - * @param edge - * the edge to be added - * @param index - * the index at which the edge shall be added - */ - void insertForwardEdge(final ManagementGroupEdge edge, final int index) { - - while (index >= this.forwardEdges.size()) { - this.forwardEdges.add(null); - } - - this.forwardEdges.set(index, edge); - } - - /** - * Inserts a new edge arriving at this group vertex at the given index. - * - * @param edge - * the edge to be added - * @param index - * the index at which the edge shall be added - */ - void insertBackwardEdge(final ManagementGroupEdge edge, final int index) { - - while (index >= this.backwardEdges.size()) { - this.backwardEdges.add(null); - } - - this.backwardEdges.set(index, edge); - } - - /** - * Returns the number of edges originating at this group vertex. - * - * @return the number of edges originating at this group vertex - */ - public int getNumberOfForwardEdges() { - return this.forwardEdges.size(); - } - - /** - * Returns the number of edges arriving at this group vertex. - * - * @return the number of edges arriving at this group vertex - */ - public int getNumberOfBackwardEdges() { - return this.backwardEdges.size(); - } - - /** - * Returns the group edge which leaves this group vertex at the given index. - * - * @param index - * the index of the group edge - * @return the group edge which leaves this group vertex at the given index or null if no such group - * edge exists - */ - public ManagementGroupEdge getForwardEdge(final int index) { - - if (index < this.forwardEdges.size()) { - return this.forwardEdges.get(index); - } - - return null; - } - - /** - * Returns the group edge which arrives at this group vertex at the given index. - * - * @param index - * the index of the group edge - * @return the group edge which arrives at this group vertex at the given index or null if no such - * group edge exists - */ - public ManagementGroupEdge getBackwardEdge(final int index) { - - if (index < this.backwardEdges.size()) { - return this.backwardEdges.get(index); - } - - return null; - } - - /** - * Adds a {@link ManagementVertex} to this group vertex. - * - * @param vertex - * the vertex to be added - */ - void addGroupMember(final ManagementVertex vertex) { - - while (this.groupMembers.size() <= vertex.getIndexInGroup()) { - this.groupMembers.add(null); - } - - this.groupMembers.set(vertex.getIndexInGroup(), vertex); - - this.getGraph().addVertex(vertex.getID(), vertex); - } - - /** - * Adds all management vertices which are included in this group vertex to the given list. - * - * @param vertices - * the list to which the vertices shall be added - */ - void collectVertices(final List vertices) { - - final Iterator it = this.groupMembers.iterator(); - while (it.hasNext()) { - vertices.add(it.next()); - } - } - - /** - * Returns the stage this management group vertex belongs to. - * - * @return the stage this management group vertex belongs to - */ - public ManagementStage getStage() { - return this.stage; - } - - /** - * Returns the management graph this group vertex is part of. - * - * @return the management graph this group vertex is part of - */ - public ManagementGraph getGraph() { - return this.stage.getGraph(); - } - - /** - * Returns the number of management vertices included in this group vertex. - * - * @return the number of management vertices included in this group vertex - */ - public int getNumberOfGroupMembers() { - return this.groupMembers.size(); - } - - /** - * Returns the management vertex with the given index. - * - * @param index - * the index of the management vertex to be returned - * @return the management vertex with the given index or null if no such vertex exists - */ - public ManagementVertex getGroupMember(final int index) { - - if (index < this.groupMembers.size()) { - return this.groupMembers.get(index); - } - - return null; - } - - /** - * Returns alle management vertices containted in this group - * - * @return the management vertex with the given index or null if no such vertex exists - */ - public List getGroupMembers() { - - return this.groupMembers; - } - - /** - * Checks if this vertex is an input vertex in its stage, i.e. has either no - * incoming connections or only incoming connections to group vertices in a lower stage. - * - * @return true if this vertex is an input vertex, false otherwise - */ - public boolean isInputVertex() { - - if (this.backwardEdges.size() == 0) { - return true; - } - - final Iterator it = this.backwardEdges.iterator(); - while (it.hasNext()) { - if (it.next().getSource().getStageNumber() == this.getStageNumber()) { - return false; - } - } - - return true; - } - - /** - * Checks if this vertex is an output vertex in its stage, i.e. has either no - * outgoing connections or only outgoing connections to group vertices in a higher stage. - * - * @return true if this vertex is an output vertex, false otherwise - */ - public boolean isOutputVertex() { - - if (this.forwardEdges.size() == 0) { - return true; - } - - final Iterator it = this.forwardEdges.iterator(); - while (it.hasNext()) { - if (it.next().getTarget().getStageNumber() == this.getStageNumber()) { - return false; - } - } - - return true; - } - - /** - * Returns the number of the management stage this group vertex belongs to. - * - * @return the number of the management stage this group vertex belongs to - */ - public int getStageNumber() { - - return this.stage.getStageNumber(); - } - - - @Override - public void read(final DataInputView in) throws IOException { - - int numberOfForwardLinks = in.readInt(); - for (int i = 0; i < numberOfForwardLinks; i++) { - final ManagementGroupVertexID targetGroupVertexID = new ManagementGroupVertexID(); - targetGroupVertexID.read(in); - final ManagementGroupVertex targetGroupVertex = getGraph().getGroupVertexByID(targetGroupVertexID); - final int sourceIndex = in.readInt(); - final int targetIndex = in.readInt(); - final ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class); - new ManagementGroupEdge(this, sourceIndex, targetGroupVertex, targetIndex, channelType); - } - - } - - - @Override - public void write(final DataOutputView out) throws IOException { - - // Write the number of forward links - out.writeInt(this.forwardEdges.size()); - final Iterator it = this.forwardEdges.iterator(); - while (it.hasNext()) { - final ManagementGroupEdge groupEdge = it.next(); - groupEdge.getTarget().getID().write(out); - out.writeInt(groupEdge.getSourceIndex()); - out.writeInt(groupEdge.getTargetIndex()); - EnumUtils.writeEnum(out, groupEdge.getChannelType()); - } - } - - /** - * Returns the list of successors of this group vertex. A successor is a group vertex which can be reached via a - * group edge originating at this group vertex. - * - * @return the list of successors of this group vertex. - */ - public List getSuccessors() { - - final List successors = new ArrayList(); - - for (ManagementGroupEdge edge : this.forwardEdges) { - successors.add(edge.getTarget()); - } - - return successors; - } - - /** - * Returns the list of predecessors of this group vertex. A predecessors is a group vertex which can be reached via - * a group edge arriving at this group vertex. - * - * @return the list of predecessors of this group vertex. - */ - public List getPredecessors() { - - final List predecessors = new ArrayList(); - - for (ManagementGroupEdge edge : this.backwardEdges) { - predecessors.add(edge.getSource()); - } - - return predecessors; - } - - @Override - public String toString() { - return String.format("ManagementGroupVertex(%s)", getName()); - } - - /** - * Returns Json representation of this ManagementGroupVertex - * - * @return - */ - public String toJson() { - StringBuilder json = new StringBuilder(""); - - json.append("{"); - json.append("\"groupvertexid\": \"" + this.getID() + "\","); - json.append("\"groupvertexname\": \"" + StringUtils.escapeHtml(this.getName()) + "\","); - json.append("\"numberofgroupmembers\": " + this.getNumberOfGroupMembers() + ","); - json.append("\"groupmembers\": ["); - - // Count state status of group members - Map stateCounts = new HashMap(); - - // initialize with 0 - for (ExecutionState state : ExecutionState.values()) { - stateCounts.put(state, new Integer(0)); - } - - for(int j = 0; j < this.getNumberOfGroupMembers(); j++) { - ManagementVertex vertex = this.getGroupMember(j); - - json.append(vertex.toJson()); - - // print delimiter - if(j != this.getNumberOfGroupMembers() - 1) { - json.append(","); - } - - // Increment state status count - Integer count = stateCounts.get(vertex.getExecutionState()) + new Integer(1); - stateCounts.put(vertex.getExecutionState(), count); - } - json.append("],"); - json.append("\"backwardEdges\": ["); - - for(int edgeIndex = 0; edgeIndex < this.getNumberOfBackwardEdges(); edgeIndex++) { - ManagementGroupEdge edge = this.getBackwardEdge(edgeIndex); - - json.append("{"); - json.append("\"groupvertexid\": \"" + edge.getSource().getID() + "\","); - json.append("\"groupvertexname\": \"" + StringUtils.escapeHtml(edge.getSource().getName()) + "\","); - json.append("\"channelType\": \"" + edge.getChannelType() + "\""); - json.append("}"); - - // print delimiter - if(edgeIndex != this.getNumberOfBackwardEdges() - 1) { - json.append(","); - } - } - json.append("]"); - - // list number of members for each status - for (Map.Entry stateCount : stateCounts.entrySet()) { - json.append(",\""+stateCount.getKey()+"\": " + stateCount.getValue()); - } - - json.append("}"); - - return json.toString(); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexID.java deleted file mode 100644 index 66465ddc3b7..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexID.java +++ /dev/null @@ -1,52 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.managementgraph; - -import javax.xml.bind.DatatypeConverter; - -import org.apache.flink.runtime.AbstractID; - -/** - * A management group vertex ID uniquely identifies a {@link ManagementGroupVertex}. - */ -public final class ManagementGroupVertexID extends AbstractID { - - private static final long serialVersionUID = 1L; - - /** - * Constructs a new ManagementGroupVertexID - */ - public ManagementGroupVertexID() { - super(); - } - - /** - * Constructs a new ManagementGroupVertexID from the given bytes. - * - * @param bytes - * the bytes to initialize the job ID with - */ - public ManagementGroupVertexID(final byte[] bytes) { - super(bytes); - } - - public static ManagementGroupVertexID fromHexString(final String hexString) { - return new ManagementGroupVertexID(DatatypeConverter.parseHexBinary(hexString)); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexIterator.java deleted file mode 100644 index 0ddfd70fa12..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexIterator.java +++ /dev/null @@ -1,330 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.managementgraph; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; -import java.util.Stack; - -/** - * This class provides an implementation of the {@link Iterator} interface which allows to - * traverse a management graph and visit every reachable group vertex exactly once. The order - * in which the group vertices are visited corresponds to the order of their discovery in a depth first - * search. - * This class is not thread-safe. - * - */ -public final class ManagementGroupVertexIterator implements Iterator { - - /** - * Stores whether the group graph is traversed starting from the input or the output vertices. - */ - private final boolean forward; - - /** - * The stage to traverse, -1 to traverse all stages of the graph. - */ - private final int stage; - - /** - * List of entry vertices for the traversal (either input or output vertices). - */ - private final List entryVertices = new ArrayList(); - - /** - * Number of already visited entry vertices. - */ - private int numVisitedEntryVertices = 0; - - /** - * Stack used for the traversal. - */ - private final Stack traversalStack = new Stack(); - - /** - * Set storing the vertices already visited during traversal. - */ - private final Set alreadyVisited = new HashSet(); - - /** - * Auxiliary class which stores which vertices have already been visited. - * - */ - private static class TraversalEntry { - - /** - * The group vertex this entry is created for. - */ - private final ManagementGroupVertex groupVertex; - - /** - * The current outgoing edge of the group vertex. - */ - private int currentEdge = 0; - - /** - * Constructs a new traversal entry. - * - * @param groupVertex - * the group vertex this traversal entry belongs to - * @param currentEdge - * the edge index to use to visit the next group vertex - */ - public TraversalEntry(final ManagementGroupVertex groupVertex, final int currentEdge) { - this.groupVertex = groupVertex; - this.currentEdge = currentEdge; - } - - /** - * Returns the group vertex this traversal entry belongs to. - * - * @return the group vertex this traversal entry belongs to - */ - public ManagementGroupVertex getGroupVertex() { - return this.groupVertex; - } - - /** - * Returns the edge index to use to visit the next group vertex. - * - * @return the edge index to use to visit the next group vertex - */ - public int getCurrentEdge() { - return this.currentEdge; - } - - /** - * Increases the edge index by one. - */ - public void increaseCurrentEdge() { - this.currentEdge++; - } - } - - /** - * Creates a new management group vertex iterator. - * - * @param managementGraph - * the management graph that should be traversed - * @param forward - * true if the graph should be traversed in correct order, false to reverse it in - * reverse order - * @param stage - * the number of the stage whose vertices should be traversed or -1 if all stages should be included in the - * traversal - */ - public ManagementGroupVertexIterator(final ManagementGraph managementGraph, final boolean forward, final int stage) { - - this.forward = forward; - this.stage = stage; - - // Collect start vertices - if (stage < 0) { - - for (int i = 0; i < managementGraph.getNumberOfStages(); i++) { - collectStartVertices(managementGraph.getStage(i)); - } - } else { - if (stage < managementGraph.getNumberOfStages()) { - collectStartVertices(managementGraph.getStage(stage)); - } - } - - if (this.entryVertices.size() > 0) { - final TraversalEntry te = new TraversalEntry(this.entryVertices.get(0), 0); - this.traversalStack.push(te); - this.alreadyVisited.add(te.getGroupVertex()); - } - } - - /** - * Collects all input group vertices (i.e. vertices with no incoming edge or incoming edges from other stages) in - * the given stage and adds them to an internal list. - * - * @param stage - * the number of the stage whose input vertices should be collected - */ - private void collectStartVertices(final ManagementStage stage) { - - for (int i = 0; i < stage.getNumberOfGroupVertices(); i++) { - - final ManagementGroupVertex groupVertex = stage.getGroupVertex(i); - - if (this.forward) { - if ((groupVertex.getNumberOfBackwardEdges() == 0) - || ((this.stage >= 0) && allConnectionsFromOtherStage(groupVertex, true))) { - this.entryVertices.add(groupVertex); - } - } else { - if ((groupVertex.getNumberOfForwardEdges() == 0) - || ((this.stage >= 0) && allConnectionsFromOtherStage(groupVertex, false))) { - this.entryVertices.add(groupVertex); - } - } - } - } - - /** - * Checks if for the given group vertex all incoming (if forward is true) or outgoing edges (if forward - * is false) come from - * other stages than the one the given vertex is in. - * - * @param groupVertex - * the group vertex to check for - * @param forward - * true if incoming edges should be considered, false for outgoing edges - * @return true if all incoming or outgoing edges (depends on the forward switch) come from other - * stages, false otherwise - */ - private boolean allConnectionsFromOtherStage(final ManagementGroupVertex groupVertex, final boolean forward) { - - if (forward) { - for (int i = 0; i < groupVertex.getNumberOfBackwardEdges(); i++) { - if (this.stage == groupVertex.getBackwardEdge(i).getSource().getStageNumber()) { - return false; - } - } - } else { - for (int i = 0; i < groupVertex.getNumberOfForwardEdges(); i++) { - if (this.stage == groupVertex.getForwardEdge(i).getTarget().getStageNumber()) { - return false; - } - } - } - - return true; - } - - - @Override - public boolean hasNext() { - - if (this.traversalStack.isEmpty()) { - - this.numVisitedEntryVertices++; - if (this.entryVertices.size() <= this.numVisitedEntryVertices) { - return false; - } - } - - return true; - - } - - - @Override - public ManagementGroupVertex next() { - - if (this.traversalStack.isEmpty()) { - - final TraversalEntry newentry = new TraversalEntry(this.entryVertices.get(this.numVisitedEntryVertices), 0); - this.traversalStack.push(newentry); - this.alreadyVisited.add(newentry.getGroupVertex()); - } - - final ManagementGroupVertex returnVertex = this.traversalStack.peek().getGroupVertex(); - - // Propose vertex to be visited next - do { - - final TraversalEntry te = this.traversalStack.peek(); - - // Check if we can traverse deeper into the graph - final ManagementGroupVertex candidateVertex = getCandidateVertex(te, forward); - if (candidateVertex == null) { - // Pop it from the stack - this.traversalStack.pop(); - } else { - // Create new entry and put it on the stack - final TraversalEntry newte = new TraversalEntry(candidateVertex, 0); - this.traversalStack.push(newte); - this.alreadyVisited.add(newte.getGroupVertex()); - break; - } - - } while (!this.traversalStack.isEmpty()); - - return returnVertex; - - } - - /** - * Returns a candidate group vertex which could potentially be visited next because it is reachable from the - * currently considered group vertex. - * - * @param te - * the traversal entry for the current source group vertex - * @param forward - * true if the graph should be traversed in correct order, false to traverse it in - * reverse order - * @return a candidate group vertex which could potentially be visited next - */ - private ManagementGroupVertex getCandidateVertex(final TraversalEntry te, final boolean forward) { - - while (true) { - - if (forward) { - // No more outgoing edges to consider - if (te.getCurrentEdge() >= te.getGroupVertex().getNumberOfForwardEdges()) { - break; - } - } else { - // No more outgoing edges to consider - if (te.getCurrentEdge() >= te.getGroupVertex().getNumberOfBackwardEdges()) { - break; - } - } - - ManagementGroupVertex tmp = null; - if (forward) { - tmp = te.getGroupVertex().getForwardEdge(te.getCurrentEdge()).getTarget(); - } else { - tmp = te.getGroupVertex().getBackwardEdge(te.getCurrentEdge()).getSource(); - } - - // Increase the current edge index by one - te.increaseCurrentEdge(); - - // If stage >= 0, tmp must be in the same stage as te.getGroupVertex() - if (this.stage >= 0) { - if (tmp.getStageNumber() != this.stage) { - continue; - } - } - - if (!this.alreadyVisited.contains(tmp)) { - return tmp; - } - } - - return null; - } - - - @Override - public void remove() { - // According to the documentation this method is optional and does not need to be implemented - - } - -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementStage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementStage.java deleted file mode 100644 index 92635946cbb..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementStage.java +++ /dev/null @@ -1,341 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.managementgraph; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -/** - * This class implements a management stage of a {@link ManagementGraph}. The stage is derived from an execution stage - * which is used in Nephele's internal scheduling structure. - *

- * This class is not thread-safe. - * - */ -public final class ManagementStage extends ManagementAttachment { - - /** - * The management graph this management stage belongs to. - */ - private final ManagementGraph managementGraph; - - /** - * The unique number of this management stage. - */ - private int stageNumber = -1; - - /** - * The list of management group vertices which are contained in this management stage. - */ - private final List groupVertices = new ArrayList(); - - /** - * Constructs a new management stage. - * - * @param managementGraph - * the management graph this management stage belongs to - * @param stageNumber - * the number of the stage - */ - public ManagementStage(final ManagementGraph managementGraph, final int stageNumber) { - this.managementGraph = managementGraph; - this.stageNumber = stageNumber; - - this.managementGraph.addStage(this); - } - - /** - * Returns the management graph this management stage is part of. - * - * @return the management graph this management stage is part of - */ - public ManagementGraph getGraph() { - return this.managementGraph; - } - - /** - * Returns this management stage's number. - * - * @return this management stage's number - */ - public int getStageNumber() { - return this.stageNumber; - } - - /** - * Returns the number of group vertices included in this management stage. - * - * @return the number of group vertices included in this management stage - */ - public int getNumberOfGroupVertices() { - - return this.groupVertices.size(); - } - - /** - * Returns the management group vertex with the given index. - * - * @param index - * the index of the group vertex to be returned - * @return the group vertex with the given index or null if no such vertex exists - */ - public ManagementGroupVertex getGroupVertex(final int index) { - - if (index < this.groupVertices.size()) { - return this.groupVertices.get(index); - } - - return null; - } - - /** - * Adds the given group vertex to this management stage. - * - * @param groupVertex - * the group vertex to be added to this management stage - */ - void addGroupVertex(final ManagementGroupVertex groupVertex) { - - this.groupVertices.add(groupVertex); - - this.managementGraph.addGroupVertex(groupVertex.getID(), groupVertex); - } - - /** - * Adds all management group vertices contained in this stage to the given list. - * - * @param groupVertices - * the list to which the group vertices in this stage shall be added - */ - void collectGroupVertices(final List groupVertices) { - - final Iterator it = this.groupVertices.iterator(); - - while (it.hasNext()) { - groupVertices.add(it.next()); - } - } - - /** - * Adds all management vertices contained in this stage's group vertices to the given list. - * - * @param vertices - * the list to which the vertices in this stage shall be added - */ - void collectVertices(final List vertices) { - - final Iterator it = this.groupVertices.iterator(); - - while (it.hasNext()) { - it.next().collectVertices(vertices); - } - } - - /** - * Returns the number of input management vertices in this stage, i.e. the number - * of management vertices which are connected to vertices in a lower stage - * or have no input channels. - * - * @return the number of input vertices in this stage - */ - public int getNumberOfInputManagementVertices() { - - int retVal = 0; - - final Iterator it = this.groupVertices.iterator(); - while (it.hasNext()) { - - final ManagementGroupVertex groupVertex = it.next(); - if (groupVertex.isInputVertex()) { - retVal += groupVertex.getNumberOfGroupMembers(); - } - } - - return retVal; - } - - /** - * Returns the number of output management vertices in this stage, i.e. the number - * of management vertices which are connected to vertices in a higher stage - * or have no output channels. - * - * @return the number of output vertices in this stage - */ - public int getNumberOfOutputManagementVertices() { - - int retVal = 0; - - final Iterator it = this.groupVertices.iterator(); - while (it.hasNext()) { - - final ManagementGroupVertex groupVertex = it.next(); - if (groupVertex.isOutputVertex()) { - retVal += groupVertex.getNumberOfGroupMembers(); - } - } - - return retVal; - } - - /** - * Returns the output management vertex with the given index or null if no such vertex exists. - * - * @param index - * the index of the vertex to be selected. - * @return the output management vertex with the given index or null if no such vertex exists - */ - public ManagementVertex getInputManagementVertex(int index) { - - final Iterator it = this.groupVertices.iterator(); - while (it.hasNext()) { - - final ManagementGroupVertex groupVertex = it.next(); - if (groupVertex.isInputVertex()) { - final int numberOfMembers = groupVertex.getNumberOfGroupMembers(); - if (index >= numberOfMembers) { - index -= numberOfMembers; - } else { - return groupVertex.getGroupMember(index); - } - } - } - - return null; - } - - /** - * Returns the input management vertex with the given index or null if no such vertex exists. - * - * @param index - * the index of the vertex to be selected. - * @return the input management vertex with the given index or null if no such vertex exists - */ - public ManagementVertex getOutputManagementVertex(int index) { - - final Iterator it = this.groupVertices.iterator(); - while (it.hasNext()) { - - final ManagementGroupVertex groupVertex = it.next(); - if (groupVertex.isOutputVertex()) { - final int numberOfMembers = groupVertex.getNumberOfGroupMembers(); - if (index >= numberOfMembers) { - index -= numberOfMembers; - } else { - return groupVertex.getGroupMember(index); - } - } - } - - return null; - } - - /** - * Returns the number of input group vertices in this stage. Input group vertices are those vertices which have - * incoming edges from group vertices of a lower stage. - * - * @return the number of input group vertices in this stage - */ - public int getNumberOfInputGroupVertices() { - - int retVal = 0; - - final Iterator it = this.groupVertices.iterator(); - while (it.hasNext()) { - if (it.next().isInputVertex()) { - ++retVal; - } - } - - return retVal; - } - - /** - * Returns the input group vertex in this stage with the given index. Input group vertices are those vertices which - * have incoming edges from group vertices of a lower stage. - * - * @param index - * the index of the input group vertex to return - * @return the input group vertex with the given index or null if no such vertex exists - */ - public ManagementGroupVertex getInputGroupVertex(int index) { - - final Iterator it = this.groupVertices.iterator(); - while (it.hasNext()) { - - final ManagementGroupVertex groupVertex = it.next(); - if (groupVertex.isInputVertex()) { - if (index == 0) { - return groupVertex; - } else { - --index; - } - } - } - - return null; - } - - /** - * Returns the number of output group vertices in this stage. Output group vertices are those vertices which have - * outgoing edges to group vertices of a higher stage. - * - * @return the number of output group vertices in this stage - */ - public int getNumberOfOutputGroupVertices() { - - int retVal = 0; - - final Iterator it = this.groupVertices.iterator(); - while (it.hasNext()) { - if (it.next().isOutputVertex()) { - ++retVal; - } - } - - return retVal; - } - - /** - * Returns the output group vertex in this stage with the given index. Output group vertices are those vertices - * which have outgoing edges to group vertices of a higher stage. - * - * @param index - * the index of the output group vertex to return - * @return the output group vertex with the given index or null if no such vertex exists - */ - public ManagementGroupVertex getOutputGroupVertex(int index) { - - final Iterator it = this.groupVertices.iterator(); - while (it.hasNext()) { - - final ManagementGroupVertex groupVertex = it.next(); - if (groupVertex.isOutputVertex()) { - if (index == 0) { - return groupVertex; - } else { - --index; - } - } - } - - return null; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertex.java deleted file mode 100644 index c7d8f0adced..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertex.java +++ /dev/null @@ -1,321 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.managementgraph; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.core.io.StringRecord; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.execution.ExecutionState2; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.StringUtils; - -/** - * This class implements a management vertex of a {@link ManagementGraph}. A management vertex is derived from the type - * of vertices Nephele uses in its internal scheduling structures. - */ -public final class ManagementVertex extends ManagementAttachment implements IOReadableWritable { - - /** - * The management group vertex this vertex belongs to. - */ - private final ManagementGroupVertex groupVertex; - - /** - * The ID of this management group vertex. - */ - private final ManagementVertexID id; - - /** - * A list of input gates which belong to this vertex. - */ - private final List inputGates = new ArrayList(); - - /** - * A list of output gates which belong to this vertex. - */ - private final List outputGates = new ArrayList(); - - /** - * The current execution state of the vertex represented by this management vertex. - */ - private ExecutionState2 executionState = ExecutionState2.CREATED; - - /** - * The name of the instance the vertex represented by this management vertex currently runs on. - */ - private String instanceName; - - /** - * The index of this vertex in the management group vertex it belongs to. - */ - private final int indexInGroup; - - /** - * The error message when executionState is FAILED. - */ - private String optMessage; - - /** - * Constructs a new management vertex. - * - * @param groupVertex - * the management group vertex the new vertex belongs to - * @param id - * the ID of the new management vertex - * @param instanceName - * the name of the instance the vertex represented by this new management vertex currently runs on - * @param indexInGroup - * the index of this vertex in the management group vertex it belongs to - */ - public ManagementVertex(final ManagementGroupVertex groupVertex, final ManagementVertexID id, - final String instanceName, final int indexInGroup) { - this.groupVertex = groupVertex; - this.id = id; - this.instanceName = instanceName; - - this.indexInGroup = indexInGroup; - - groupVertex.addGroupMember(this); - } - - /** - * Adds a management gate to this vertex. - * - * @param gate - * the management gate to be added - */ - void addGate(final ManagementGate gate) { - - if (gate.isInputGate()) { - this.inputGates.add(gate); - } else { - this.outputGates.add(gate); - } - } - - /** - * Returns the name of the instance the vertex represented by this management vertex currently runs on. - * - * @return the name of the instance the vertex represented by this management vertex currently runs on - */ - public String getInstanceName() { - return this.instanceName; - } - - /** - * Returns the number of input gates this management vertex contains. - * - * @return the number of input gates this management vertex contains - */ - public int getNumberOfInputGates() { - return this.inputGates.size(); - } - - /** - * Returns the number of output gates this management vertex contains. - * - * @return the number of output gates this management vertex contains - */ - public int getNumberOfOutputGates() { - return this.outputGates.size(); - } - - /** - * Returns the input gate at the given index. - * - * @param index - * the index of the input gate to be returned - * @return the input gate at the given index or null if no such input gate exists - */ - public ManagementGate getInputGate(final int index) { - - if (index < this.inputGates.size()) { - return this.inputGates.get(index); - } - - return null; - } - - /** - * Returns the output gate at the given index. - * - * @param index - * the index of the output gate to be returned - * @return the output gate at the given index or null if no such output gate exists - */ - public ManagementGate getOutputGate(final int index) { - - if (index < this.outputGates.size()) { - return this.outputGates.get(index); - } - - return null; - } - - /** - * Returns the group vertex this management vertex belongs to. - * - * @return the group vertex this management vertex belongs to - */ - public ManagementGroupVertex getGroupVertex() { - return this.groupVertex; - } - - /** - * The management graph this management vertex belongs to. - * - * @return the management graph this management vertex belongs to - */ - public ManagementGraph getGraph() { - return this.groupVertex.getGraph(); - } - - /** - * Returns the ID of this management vertex. - * - * @return the ID of this management vertex - */ - public ManagementVertexID getID() { - return this.id; - } - - /** - * Returns the name of this management vertex. - * - * @return the name of this management vertex, possibly null - */ - public String getName() { - - return this.groupVertex.getName(); - } - - /** - * Returns the number of vertices which belong to the same group vertex as this management vertex. - * - * @return the number of vertices which belong to the same group vertex as this management vertex - */ - public int getNumberOfVerticesInGroup() { - - return this.groupVertex.getNumberOfGroupMembers(); - } - - /** - * Returns the index at which this vertex is stored inside its corresponding group vertex. - * - * @return the index at which this vertex is stored inside its corresponding group vertex - */ - public int getIndexInGroup() { - - return this.indexInGroup; - } - - /** - * Sets the current execution state of this management vertex. - * - * @param executionState - * the current execution state of this vertex - */ - public void setExecutionState(ExecutionState2 executionState) { - this.executionState = executionState; - } - - /** - * Returns the current execution state of this management vertex. - * - * @return the current execution state of this management vertex - */ - public ExecutionState2 getExecutionState() { - return this.executionState; - } - - /** - * Sets the name of the instance this vertex currently runs on. - * - * @param instanceName - * the name of the instance this vertex currently runs on - */ - public void setInstanceName(final String instanceName) { - this.instanceName = instanceName; - } - - public void setOptMessage(final String optMessage) { - this.optMessage = optMessage; - } - - public String getOptMessage() { - return this.optMessage; - } - - @Override - public void read(final DataInputView in) throws IOException { - - // Read the execution state - this.executionState = ExecutionState2.values()[in.readInt()]; - - // Read number of input gates - int numberOfInputGates = in.readInt(); - for (int i = 0; i < numberOfInputGates; i++) { - new ManagementGate(this, new ManagementGateID(), i, true); - } - - // Read number of input gates - int numberOfOutputGates = in.readInt(); - for (int i = 0; i < numberOfOutputGates; i++) { - new ManagementGate(this, new ManagementGateID(), i, false); - } - - this.instanceName = StringRecord.readString(in); - } - - - @Override - public void write(final DataOutputView out) throws IOException { - out.writeInt(this.executionState.ordinal()); - out.writeInt(this.inputGates.size()); - out.writeInt(this.outputGates.size()); - - StringValue.writeString(this.instanceName, out); - } - - @Override - public String toString() { - return String.format("%s_%d", getGroupVertex().getName(), indexInGroup); - } - - /** - * Returns JSON representation of this ManagementVertex - * - * @return A JSON representation of this ManagementVertex - */ - public String toJson() { - StringBuilder json = new StringBuilder(""); - json.append("{"); - json.append("\"vertexid\": \"" + this.getID() + "\","); - json.append("\"vertexname\": \"" + StringUtils.escapeHtml(this.toString()) + "\","); - json.append("\"vertexstatus\": \"" + this.getExecutionState() + "\","); - json.append("\"vertexinstancename\": \"" + this.getInstanceName() + "\""); - json.append("}"); - return json.toString(); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java deleted file mode 100644 index 6e7277dc236..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java +++ /dev/null @@ -1,40 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.managementgraph; - -import org.apache.flink.runtime.AbstractID; - -/** - * A management vertex ID uniquely identifies a {@link ManagementVertex}. - */ -public final class ManagementVertexID extends AbstractID { - private static final long serialVersionUID = 1L; - - public ManagementVertexID() { - super(); - } - - public ManagementVertexID(AbstractID toCopy) { - super(toCopy); - } - - public ManagementVertexID(long lowerPart, long upperPart) { - super(lowerPart, upperPart); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java index 77c4f5bffd2..4a14e13dd43 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java @@ -35,6 +35,7 @@ public class EnvironmentListenerImpl implements ExecutionListener { private final RuntimeEnvironment environment; + public EnvironmentListenerImpl(TaskManagerProfilerImpl taskManagerProfiler, RuntimeEnvironment environment) { this.taskManagerProfiler = taskManagerProfiler; this.environment = environment; @@ -46,7 +47,7 @@ public class EnvironmentListenerImpl implements ExecutionListener { switch (newExecutionState) { case RUNNING: - this.taskManagerProfiler.registerMainThreadForCPUProfiling(this.environment, this.environment.getExecutingThread(), executionId); + this.taskManagerProfiler.registerMainThreadForCPUProfiling(this.environment, this.environment.getExecutingThread(), vertexId, subtaskIndex, executionId); break; case FINISHED: diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java index 7eac666f286..29685a17331 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java @@ -26,6 +26,7 @@ import java.util.Map; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobID; +import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.profiling.impl.types.InternalExecutionVertexThreadProfilingData; public class EnvironmentThreadSet { @@ -34,58 +35,25 @@ public class EnvironmentThreadSet { private static final long PERCENT = 100; - private class CPUUtilizationSnapshot { - - private final long timestamp; - - private final long totalCPUTime; - - private final long totalCPUUserTime; - - private final long totalCPUWaitTime; - - private final long totalCPUBlockTime; - - public CPUUtilizationSnapshot(long timestamp, long totalCPUTime, long totalCPUUserTime, long totalCPUWaitTime, - long totalCPUBlockTime) { - this.timestamp = timestamp; - this.totalCPUTime = totalCPUTime; - this.totalCPUUserTime = totalCPUUserTime; - this.totalCPUWaitTime = totalCPUWaitTime; - this.totalCPUBlockTime = totalCPUBlockTime; - } - - public long getTimestamp() { - return this.timestamp; - } - - public long getTotalCPUTime() { - return this.totalCPUTime; - } - - public long getTotalCPUUserTime() { - return this.totalCPUUserTime; - } - - public long getTotalCPUWaitTime() { - return this.totalCPUWaitTime; - } - - public long getTotalCPUBlockTime() { - return this.totalCPUBlockTime; - } - } + private final Thread mainThread; + private final JobVertexID vertexId; + + private final int subtask; + private final ExecutionAttemptID executionId; private final Map userThreads = new HashMap(); - private CPUUtilizationSnapshot mainThreadSnapshot = null; + private CPUUtilizationSnapshot mainThreadSnapshot; - public EnvironmentThreadSet(ThreadMXBean tmx, Thread mainThread, ExecutionAttemptID executionId) { + + public EnvironmentThreadSet(ThreadMXBean tmx, Thread mainThread, JobVertexID vertexId, int subtask, ExecutionAttemptID executionId) { this.mainThread = mainThread; + this.vertexId = vertexId; + this.subtask = subtask; this.executionId = executionId; this.mainThreadSnapshot = createCPUUtilizationSnapshot(tmx, mainThread, System.currentTimeMillis()); @@ -96,29 +64,24 @@ public class EnvironmentThreadSet { } public void addUserThread(ThreadMXBean tmx, Thread thread) { - synchronized (this.userThreads) { this.userThreads.put(thread, createCPUUtilizationSnapshot(tmx, thread, System.currentTimeMillis())); } } public void removeUserThread(Thread thread) { - synchronized (this.userThreads) { this.userThreads.remove(thread); } } public int getNumberOfUserThreads() { - synchronized (this.userThreads) { return this.userThreads.size(); } - } private CPUUtilizationSnapshot createCPUUtilizationSnapshot(ThreadMXBean tmx, Thread thread, long timestamp) { - final long threadId = thread.getId(); final ThreadInfo threadInfo = tmx.getThreadInfo(threadId); @@ -126,13 +89,14 @@ public class EnvironmentThreadSet { return null; } - return new CPUUtilizationSnapshot(timestamp, tmx.getThreadCpuTime(threadId) / NANO_TO_MILLISECONDS, tmx - .getThreadUserTime(threadId) / NANO_TO_MILLISECONDS, threadInfo.getWaitedTime(), - threadInfo.getBlockedTime()); + return new CPUUtilizationSnapshot(timestamp, + tmx.getThreadCpuTime(threadId) / NANO_TO_MILLISECONDS, + tmx.getThreadUserTime(threadId) / NANO_TO_MILLISECONDS, + threadInfo.getWaitedTime(), + threadInfo.getBlockedTime()); } - public InternalExecutionVertexThreadProfilingData captureCPUUtilization(JobID jobID, ThreadMXBean tmx, - long timestamp) { + public InternalExecutionVertexThreadProfilingData captureCPUUtilization(JobID jobID, ThreadMXBean tmx, long timestamp) { synchronized (this.userThreads) { @@ -182,13 +146,10 @@ public class EnvironmentThreadSet { } cputime = newUtilizationSnaphot.getTotalCPUTime() - oldUtilizationSnapshot.getTotalCPUTime(); - usrtime = newUtilizationSnaphot.getTotalCPUUserTime() - - oldUtilizationSnapshot.getTotalCPUUserTime(); + usrtime = newUtilizationSnaphot.getTotalCPUUserTime() - oldUtilizationSnapshot.getTotalCPUUserTime(); systime = cputime - usrtime; - waitime = newUtilizationSnaphot.getTotalCPUWaitTime() - - oldUtilizationSnapshot.getTotalCPUWaitTime(); - blktime = newUtilizationSnaphot.getTotalCPUBlockTime() - - oldUtilizationSnapshot.getTotalCPUBlockTime(); + waitime = newUtilizationSnaphot.getTotalCPUWaitTime() - oldUtilizationSnapshot.getTotalCPUWaitTime(); + blktime = newUtilizationSnaphot.getTotalCPUBlockTime() - oldUtilizationSnapshot.getTotalCPUBlockTime(); sumUsrTime += (int) ((usrtime * PERCENT) / interval); sumSysTime += (int) ((systime * PERCENT) / interval); @@ -205,8 +166,52 @@ public class EnvironmentThreadSet { sumWaiTime /= (divisor + 1); } - return new InternalExecutionVertexThreadProfilingData(jobID, this.executionId, (int) mainInterval, - sumUsrTime, sumSysTime, sumBlkTime, sumWaiTime); + return new InternalExecutionVertexThreadProfilingData(jobID, this.vertexId, this.subtask, this.executionId, + (int) mainInterval, sumUsrTime, sumSysTime, sumBlkTime, sumWaiTime); + } + } + + // -------------------------------------------------------------------------------------------- + + private class CPUUtilizationSnapshot { + + private final long timestamp; + + private final long totalCPUTime; + + private final long totalCPUUserTime; + + private final long totalCPUWaitTime; + + private final long totalCPUBlockTime; + + public CPUUtilizationSnapshot(long timestamp, long totalCPUTime, long totalCPUUserTime, long totalCPUWaitTime, + long totalCPUBlockTime) { + this.timestamp = timestamp; + this.totalCPUTime = totalCPUTime; + this.totalCPUUserTime = totalCPUUserTime; + this.totalCPUWaitTime = totalCPUWaitTime; + this.totalCPUBlockTime = totalCPUBlockTime; + } + + public long getTimestamp() { + return this.timestamp; + } + + public long getTotalCPUTime() { + return this.totalCPUTime; + } + + public long getTotalCPUUserTime() { + return this.totalCPUUserTime; + } + + public long getTotalCPUWaitTime() { + return this.totalCPUWaitTime; + } + + public long getTotalCPUBlockTime() { + return this.totalCPUBlockTime; } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobManagerProfilerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobManagerProfilerImpl.java index faae3455f8d..da97c0469ae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobManagerProfilerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobManagerProfilerImpl.java @@ -40,14 +40,10 @@ import org.apache.flink.runtime.profiling.ProfilingException; import org.apache.flink.runtime.profiling.ProfilingListener; import org.apache.flink.runtime.profiling.ProfilingUtils; import org.apache.flink.runtime.profiling.impl.types.InternalExecutionVertexThreadProfilingData; -import org.apache.flink.runtime.profiling.impl.types.InternalInputGateProfilingData; import org.apache.flink.runtime.profiling.impl.types.InternalInstanceProfilingData; -import org.apache.flink.runtime.profiling.impl.types.InternalOutputGateProfilingData; import org.apache.flink.runtime.profiling.impl.types.InternalProfilingData; import org.apache.flink.runtime.profiling.impl.types.ProfilingDataContainer; -import org.apache.flink.runtime.profiling.types.InputGateProfilingEvent; import org.apache.flink.runtime.profiling.types.InstanceSummaryProfilingEvent; -import org.apache.flink.runtime.profiling.types.OutputGateProfilingEvent; import org.apache.flink.runtime.profiling.types.SingleInstanceProfilingEvent; import org.apache.flink.runtime.profiling.types.ThreadProfilingEvent; import org.apache.flink.util.StringUtils; @@ -60,12 +56,15 @@ public class JobManagerProfilerImpl implements JobManagerProfiler, ProfilerImplP private static final int DEFAULT_NUM_HANLDER = 3; + private final Server profilingServer; private final Map> registeredListeners = new HashMap>(); private final Map registeredJobs = new HashMap(); + // -------------------------------------------------------------------------------------------- + public JobManagerProfilerImpl(InetAddress jobManagerbindAddress) throws ProfilingException { // Start profiling IPC server @@ -89,17 +88,14 @@ public class JobManagerProfilerImpl implements JobManagerProfiler, ProfilerImplP @Override public void registerProfilingJob(ExecutionGraph executionGraph) { - synchronized (this.registeredJobs) { this.registeredJobs.put(executionGraph.getJobID(), new JobProfilingData(executionGraph)); } - } @Override public void unregisterProfilingJob(ExecutionGraph executionGraph) { - synchronized (this.registeredListeners) { this.registeredListeners.remove(executionGraph.getJobID()); } @@ -107,12 +103,10 @@ public class JobManagerProfilerImpl implements JobManagerProfiler, ProfilerImplP synchronized (this.registeredJobs) { this.registeredJobs.remove(executionGraph.getJobID()); } - } @Override public void shutdown() { - // Stop the RPC server if (this.profilingServer != null) { LOG.debug("Stopping profiling RPC server"); @@ -135,10 +129,10 @@ public class JobManagerProfilerImpl implements JobManagerProfiler, ProfilerImplP return; } - final ThreadProfilingEvent threadProfilingEvent = new ThreadProfilingEvent(profilingData.getUserTime(), - profilingData.getSystemTime(), profilingData.getBlockedTime(), profilingData.getWaitedTime(), - profilingData.getExecutionVertexID().toManagementVertexID(), profilingData.getProfilingInterval(), - profilingData.getJobID(), timestamp, (timestamp - profilingStart)); + final ThreadProfilingEvent threadProfilingEvent = new ThreadProfilingEvent( + profilingData.getUserTime(), profilingData.getSystemTime(), profilingData.getBlockedTime(), profilingData.getWaitedTime(), + profilingData.getVertexId(), profilingData.getSubtask(), profilingData.getExecutionAttemptId(), + profilingData.getProfilingInterval(), profilingData.getJobID(), timestamp, (timestamp - profilingStart)); final Iterator it = jobListeners.iterator(); while (it.hasNext()) { @@ -156,7 +150,7 @@ public class JobManagerProfilerImpl implements JobManagerProfiler, ProfilerImplP while (it.hasNext()) { final JobID jobID = it.next(); final JobProfilingData jobProfilingData = this.registeredJobs.get(jobID); - if (!jobProfilingData.instanceAllocatedByJob(profilingData)) { + if (!jobProfilingData.addIfInstanceIsAllocatedByJob(profilingData)) { continue; } @@ -192,60 +186,6 @@ public class JobManagerProfilerImpl implements JobManagerProfiler, ProfilerImplP } } - private void dispatchInputGateData(long timestamp, InternalInputGateProfilingData profilingData) { - - final long profilingStart = getProfilingStart(profilingData.getJobID()); - if (profilingStart < 0) { - LOG.error("Received profiling data for unregistered job " + profilingData.getJobID()); - return; - } - - synchronized (this.registeredListeners) { - - final List jobListeners = this.registeredListeners.get(profilingData.getJobID()); - if (jobListeners == null) { - return; - } - - final InputGateProfilingEvent inputGateProfilingEvent = new InputGateProfilingEvent(profilingData - .getGateIndex(), profilingData.getNoRecordsAvailableCounter(), profilingData.getExecutionVertexID() - .toManagementVertexID(), profilingData.getProfilingInterval(), profilingData.getJobID(), timestamp, - timestamp - profilingStart); - - final Iterator it = jobListeners.iterator(); - while (it.hasNext()) { - it.next().processProfilingEvents(inputGateProfilingEvent); - } - } - } - - private void dispatchOutputGateData(long timestamp, InternalOutputGateProfilingData profilingData) { - - final long profilingStart = getProfilingStart(profilingData.getJobID()); - if (profilingStart < 0) { - LOG.error("Received profiling data for unregistered job " + profilingData.getJobID()); - return; - } - - synchronized (this.registeredListeners) { - - final List jobListeners = this.registeredListeners.get(profilingData.getJobID()); - if (jobListeners == null) { - return; - } - - final OutputGateProfilingEvent outputGateProfilingEvent = new OutputGateProfilingEvent(profilingData - .getGateIndex(), profilingData.getChannelCapacityExhaustedCounter(), profilingData - .getExecutionVertexID().toManagementVertexID(), profilingData.getProfilingInterval(), profilingData - .getJobID(), timestamp, timestamp - profilingStart); - - final Iterator it = jobListeners.iterator(); - while (it.hasNext()) { - it.next().processProfilingEvents(outputGateProfilingEvent); - } - } - } - private long getProfilingStart(JobID jobID) { synchronized (this.registeredJobs) { @@ -274,10 +214,6 @@ public class JobManagerProfilerImpl implements JobManagerProfiler, ProfilerImplP dispatchThreadData(timestamp, (InternalExecutionVertexThreadProfilingData) internalProfilingData); } else if (internalProfilingData instanceof InternalInstanceProfilingData) { dispatchInstanceData(timestamp, (InternalInstanceProfilingData) internalProfilingData); - } else if (internalProfilingData instanceof InternalInputGateProfilingData) { - dispatchInputGateData(timestamp, (InternalInputGateProfilingData) internalProfilingData); - } else if (internalProfilingData instanceof InternalOutputGateProfilingData) { - dispatchOutputGateData(timestamp, (InternalOutputGateProfilingData) internalProfilingData); } else { LOG.error("Received unknown profiling data: " + internalProfilingData.getClass().getName()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java index c7efb8c271f..16c2bf5832e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java @@ -25,8 +25,8 @@ import java.util.Map; import java.util.Set; import org.apache.flink.runtime.executiongraph.ExecutionGraph; -import org.apache.flink.runtime.executiongraph.ExecutionGroupVertex; -import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex2; +import org.apache.flink.runtime.instance.AllocatedSlot; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.profiling.impl.types.InternalInstanceProfilingData; @@ -41,36 +41,30 @@ public class JobProfilingData { private final Map collectedInstanceProfilingData = new HashMap(); + public JobProfilingData(ExecutionGraph executionGraph) { this.executionGraph = executionGraph; - this.profilingStart = System.currentTimeMillis(); } + public long getProfilingStart() { return this.profilingStart; } public ExecutionGraph getExecutionGraph() { - return this.executionGraph; } - public boolean instanceAllocatedByJob(InternalInstanceProfilingData instanceProfilingData) { - - final ExecutionGroupVertexIterator it = new ExecutionGroupVertexIterator(this.executionGraph, true, - this.executionGraph.getIndexOfCurrentExecutionStage()); - while (it.hasNext()) { - - final ExecutionGroupVertex groupVertex = it.next(); - for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); i++) { - final ExecutionVertex executionVertex = groupVertex.getGroupMember(i); - if (instanceProfilingData.getInstanceConnectionInfo().equals( - executionVertex.getAllocatedResource().getInstance().getInstanceConnectionInfo())) { - this.collectedInstanceProfilingData.put(instanceProfilingData.getInstanceConnectionInfo(), - instanceProfilingData); - return true; - } + public boolean addIfInstanceIsAllocatedByJob(InternalInstanceProfilingData instanceProfilingData) { + + for (ExecutionVertex2 executionVertex : this.executionGraph.getAllExecutionVertices()) { + AllocatedSlot slot = executionVertex.getAssignedResource(); + if (slot != null && slot.getInstance().getInstanceConnectionInfo().equals( + instanceProfilingData.getInstanceConnectionInfo())) + { + this.collectedInstanceProfilingData.put(instanceProfilingData.getInstanceConnectionInfo(), instanceProfilingData); + return true; } } @@ -80,25 +74,16 @@ public class JobProfilingData { public InstanceSummaryProfilingEvent getInstanceSummaryProfilingData(long timestamp) { final Set tempSet = new HashSet(); - // First determine the number of allocated instances in the current stage - final ExecutionGroupVertexIterator it = new ExecutionGroupVertexIterator(this.executionGraph, true, - this.executionGraph.getIndexOfCurrentExecutionStage()); - while (it.hasNext()) { - - final ExecutionGroupVertex groupVertex = it.next(); - for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); i++) { - final ExecutionVertex executionVertex = groupVertex.getGroupMember(i); - final Instance instance = executionVertex.getAllocatedResource().getInstance(); - if(!(instance instanceof DummyInstance)) { - tempSet.add(instance); - } + + for (ExecutionVertex2 executionVertex : this.executionGraph.getAllExecutionVertices()) { + AllocatedSlot slot = executionVertex.getAssignedResource(); + if (slot != null) { + tempSet.add(slot.getInstance()); } } - /* - * Now compare the size of the collected data set and the allocated instance set. - * If their sizes are equal we can issue an instance summary. - */ + // Now compare the size of the collected data set and the allocated instance set. + // If their sizes are equal we can issue an instance summary. if (tempSet.size() != this.collectedInstanceProfilingData.size()) { return null; } @@ -109,6 +94,7 @@ public class JobProfilingData { private InstanceSummaryProfilingEvent constructInstanceSummary(long timestamp) { final int numberOfInstances = this.collectedInstanceProfilingData.size(); + final Iterator instanceIterator = this.collectedInstanceProfilingData.keySet().iterator(); long freeMemorySum = 0; @@ -149,14 +135,22 @@ public class JobProfilingData { cachedSwapMemorySum += profilingData.getCachedSwapMemory(); } - final InstanceSummaryProfilingEvent instanceSummary = new InstanceSummaryProfilingEvent(profilingIntervalSum - / numberOfInstances, ioWaitCPUSum / numberOfInstances, idleCPUSum / numberOfInstances, userCPUSum - / numberOfInstances, systemCPUSum / numberOfInstances, hardIrqCPUSum / numberOfInstances, softIrqCPUSum - / numberOfInstances, totalMemorySum / (long) numberOfInstances, freeMemorySum / (long) numberOfInstances, - bufferedMemorySum / (long) numberOfInstances, cachedMemorySum / (long) numberOfInstances, - cachedSwapMemorySum / (long) numberOfInstances, receivedBytesSum / (long) numberOfInstances, - transmittedBytesSum / (long) numberOfInstances, this.executionGraph.getJobID(), timestamp, - (timestamp - this.profilingStart)); + final InstanceSummaryProfilingEvent instanceSummary = new InstanceSummaryProfilingEvent( + profilingIntervalSum / numberOfInstances, + ioWaitCPUSum / numberOfInstances, + idleCPUSum / numberOfInstances, + userCPUSum / numberOfInstances, + systemCPUSum / numberOfInstances, + hardIrqCPUSum / numberOfInstances, + softIrqCPUSum / numberOfInstances, + totalMemorySum / numberOfInstances, + freeMemorySum / numberOfInstances, + bufferedMemorySum / numberOfInstances, + cachedMemorySum / numberOfInstances, + cachedSwapMemorySum / numberOfInstances, + receivedBytesSum / numberOfInstances, + transmittedBytesSum / numberOfInstances, + this.executionGraph.getJobID(), timestamp, (timestamp - this.profilingStart)); this.collectedInstanceProfilingData.clear(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java index f80e6246ce2..00daeb02b85 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.ipc.RPC; +import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.net.NetUtils; import org.apache.flink.runtime.profiling.ProfilingException; import org.apache.flink.runtime.profiling.ProfilingUtils; @@ -172,7 +173,7 @@ public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerPro } } - public void registerMainThreadForCPUProfiling(Environment environment, Thread thread, ExecutionAttemptID executionID) { + public void registerMainThreadForCPUProfiling(Environment environment, Thread thread, JobVertexID vertexId, int subtask, ExecutionAttemptID executionID) { synchronized (this.monitoredThreads) { LOG.debug("Registering thread " + thread.getName() + " for CPU monitoring"); @@ -181,7 +182,7 @@ public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerPro + environment.getTaskName()); } - this.monitoredThreads.put(environment, new EnvironmentThreadSet(this.tmx, thread, executionID)); + this.monitoredThreads.put(environment, new EnvironmentThreadSet(this.tmx, thread, vertexId, subtask, executionID)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java index f2a23e37e69..71d982c11f8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java @@ -24,41 +24,67 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobID; +import org.apache.flink.runtime.jobgraph.JobVertexID; public abstract class InternalExecutionVertexProfilingData implements InternalProfilingData { - private final ExecutionAttemptID executionId; - private final JobID jobId; + + private final JobVertexID vertexId; + + private int subtask; + + private final ExecutionAttemptID executionId; public InternalExecutionVertexProfilingData() { this.jobId = new JobID(); + this.vertexId = new JobVertexID(); this.executionId = new ExecutionAttemptID(); + this.subtask = -1; } - public InternalExecutionVertexProfilingData(JobID jobID, ExecutionAttemptID executionId) { - this.jobId = jobID; + + public InternalExecutionVertexProfilingData(JobID jobId, JobVertexID vertexId, int subtask, ExecutionAttemptID executionId) { + this.jobId = jobId; + this.vertexId = vertexId; + this.subtask = subtask; this.executionId = executionId; } - public ExecutionAttemptID getExecutionAttemptId() { - return this.executionId; - } - + // -------------------------------------------------------------------------------------------- + public JobID getJobID() { return this.jobId; } - + + public JobVertexID getVertexId() { + return vertexId; + } + + public int getSubtask() { + return subtask; + } + + public ExecutionAttemptID getExecutionAttemptId() { + return this.executionId; + } + + // -------------------------------------------------------------------------------------------- + @Override public void read(DataInputView in) throws IOException { this.jobId.read(in); + this.vertexId.read(in); this.executionId.read(in); + this.subtask = in.readInt(); } @Override public void write(DataOutputView out) throws IOException { this.jobId.write(out); + this.vertexId.write(out); this.executionId.write(out); + out.writeInt(this.subtask); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java index 892968d1f5e..0135b122a8a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java @@ -24,6 +24,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobID; +import org.apache.flink.runtime.jobgraph.JobVertexID; public class InternalExecutionVertexThreadProfilingData extends InternalExecutionVertexProfilingData { @@ -37,10 +38,10 @@ public class InternalExecutionVertexThreadProfilingData extends InternalExecutio private int waitedTime = 0; - public InternalExecutionVertexThreadProfilingData(JobID jobID, ExecutionAttemptID executionId, + public InternalExecutionVertexThreadProfilingData(JobID jobID, JobVertexID vertexId, int subtask, ExecutionAttemptID executionId, int profilingInterval, int userTime, int systemTime, int blockedTime, int waitedTime) { - super(jobID, executionId); + super(jobID, vertexId, subtask, executionId); this.profilingInterval = profilingInterval; this.userTime = userTime; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInputGateProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInputGateProfilingData.java deleted file mode 100644 index 3d001612ee1..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInputGateProfilingData.java +++ /dev/null @@ -1,95 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.profiling.impl.types; - -import java.io.IOException; - -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.jobgraph.JobID; - -public class InternalInputGateProfilingData implements InternalProfilingData { - - private final ExecutionAttemptID executionID; - - private final JobID jobId; - - private int gateIndex; - - private int profilingInternval; - - private int noRecordsAvailableCounter; - - public InternalInputGateProfilingData() { - this.jobId = new JobID(); - this.executionID = new ExecutionAttemptID(); - this.gateIndex = 0; - this.profilingInternval = 0; - this.noRecordsAvailableCounter = 0; - } - - public InternalInputGateProfilingData(JobID jobID, ExecutionAttemptID executionID, int gateIndex, - int profilingInterval, int noRecordsAvailableCounter) - { - this.jobId = jobID; - this.executionID = executionID; - this.gateIndex = gateIndex; - this.profilingInternval = profilingInterval; - this.noRecordsAvailableCounter = noRecordsAvailableCounter; - } - - @Override - public void read(DataInputView in) throws IOException { - this.jobId.read(in); - this.executionID.read(in); - this.gateIndex = in.readInt(); - this.profilingInternval = in.readInt(); - this.noRecordsAvailableCounter = in.readInt(); - } - - @Override - public void write(DataOutputView out) throws IOException { - this.jobId.write(out); - this.executionID.write(out); - out.writeInt(this.gateIndex); - out.writeInt(this.profilingInternval); - out.writeInt(this.noRecordsAvailableCounter); - } - - public JobID getJobID() { - return this.jobId; - } - - public ExecutionAttemptID getExecutionID() { - return this.executionID; - } - - public int getGateIndex() { - return this.gateIndex; - } - - public int getProfilingInterval() { - return this.profilingInternval; - } - - public int getNoRecordsAvailableCounter() { - return this.noRecordsAvailableCounter; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInstanceProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInstanceProfilingData.java index 5d8138dd79d..f85f87f1f00 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInstanceProfilingData.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInstanceProfilingData.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.profiling.impl.types; import java.io.IOException; @@ -243,5 +242,4 @@ public class InternalInstanceProfilingData implements InternalProfilingData { public void setTransmittedBytes(long transmittedBytes) { this.transmittedBytes = transmittedBytes; } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalOutputGateProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalOutputGateProfilingData.java deleted file mode 100644 index c05fbe7431a..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalOutputGateProfilingData.java +++ /dev/null @@ -1,94 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.profiling.impl.types; - -import java.io.IOException; - -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.jobgraph.JobID; - -public class InternalOutputGateProfilingData implements InternalProfilingData { - - private final ExecutionAttemptID executionID; - - private final JobID jobId; - - private int gateIndex; - - private int profilingInternval; - - private int channelCapacityExhaustedCounter; - - public InternalOutputGateProfilingData() { - this.jobId = new JobID(); - this.executionID = new ExecutionAttemptID(); - this.gateIndex = 0; - this.profilingInternval = 0; - this.channelCapacityExhaustedCounter = 0; - } - - public InternalOutputGateProfilingData(JobID jobID, ExecutionAttemptID executionVertexID, int gateIndex, - int profilingInterval, int channelCapacityExhaustedCounter) { - this.jobId = jobID; - this.executionID = executionVertexID; - this.gateIndex = gateIndex; - this.profilingInternval = profilingInterval; - this.channelCapacityExhaustedCounter = channelCapacityExhaustedCounter; - } - - @Override - public void read(DataInputView in) throws IOException { - this.jobId.read(in); - this.executionID.read(in); - this.gateIndex = in.readInt(); - this.profilingInternval = in.readInt(); - this.channelCapacityExhaustedCounter = in.readInt(); - } - - @Override - public void write(DataOutputView out) throws IOException { - this.jobId.write(out); - this.executionID.write(out); - out.writeInt(this.gateIndex); - out.writeInt(this.profilingInternval); - out.writeInt(this.channelCapacityExhaustedCounter); - } - - public JobID getJobID() { - return this.jobId; - } - - public ExecutionAttemptID getExecutionID() { - return this.executionID; - } - - public int getGateIndex() { - return this.gateIndex; - } - - public int getProfilingInterval() { - return this.profilingInternval; - } - - public int getChannelCapacityExhaustedCounter() { - return this.channelCapacityExhaustedCounter; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InputGateProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InputGateProfilingEvent.java deleted file mode 100644 index 1c25d201de7..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InputGateProfilingEvent.java +++ /dev/null @@ -1,152 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.profiling.types; - -import java.io.IOException; - -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.managementgraph.ManagementVertexID; - -/** - * Input gate profiling events are a special subclass of vertex profiling events. They contain profiling information - * which refer to particular input gates of a task. - *

- * This class is not thread-safe. - * - */ -public final class InputGateProfilingEvent extends VertexProfilingEvent { - - /** - * The index of the input gate at the corresponding management vertex. - */ - private int gateIndex; - - /** - * Stores how often the input gate had no records available during the last time period. - */ - private int noRecordsAvailableCounter; - - /** - * Constructs a new input gate profiling event. - * - * @param gateIndex - * the index of the input gate at the corresponding management vertex - * @param noRecordsAvailableCounter - * indicates how often the input gate had no records available during the last time period - * @param vertexID - * the ID of the management vertex this event refers to - * @param profilingInterval - * the interval of time this profiling event covers - * @param jobID - * the ID of the job this event refers to - * @param timestamp - * the time stamp of the event - * @param profilingTimestamp - * the time stamp of the profiling data - */ - public InputGateProfilingEvent(final int gateIndex, final int noRecordsAvailableCounter, - final ManagementVertexID vertexID, final int profilingInterval, final JobID jobID, final long timestamp, - final long profilingTimestamp) { - - super(vertexID, profilingInterval, jobID, timestamp, profilingTimestamp); - - this.gateIndex = gateIndex; - this.noRecordsAvailableCounter = noRecordsAvailableCounter; - } - - /** - * Default constructor for the serialization/deserialization process. Should not be called for other purposes. - */ - public InputGateProfilingEvent() { - super(); - } - - /** - * Returns the index of input gate. - * - * @return the index of the input gate - */ - public int getGateIndex() { - return this.gateIndex; - } - - /** - * Returns the number of times no records were available - * on any of the channels attached to the input gate in - * the given profiling internval. - * - * @return the number of times no records were available - */ - public int getNoRecordsAvailableCounter() { - return this.noRecordsAvailableCounter; - } - - - @Override - public void read(final DataInputView in) throws IOException { - super.read(in); - - this.gateIndex = in.readInt(); - this.noRecordsAvailableCounter = in.readInt(); - } - - - @Override - public void write(final DataOutputView out) throws IOException { - super.write(out); - - out.writeInt(this.gateIndex); - out.writeInt(this.noRecordsAvailableCounter); - } - - - @Override - public boolean equals(final Object obj) { - - if (!super.equals(obj)) { - return false; - } - - if (!(obj instanceof InputGateProfilingEvent)) { - return false; - } - - final InputGateProfilingEvent inputGateProfilingEvent = (InputGateProfilingEvent) obj; - - if (this.gateIndex != inputGateProfilingEvent.getGateIndex()) { - return false; - } - - if (this.noRecordsAvailableCounter != inputGateProfilingEvent.getNoRecordsAvailableCounter()) { - return false; - } - - return true; - } - - - @Override - public int hashCode() { - - return super.hashCode(); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceProfilingEvent.java index 2b7265208c9..73c8f6faedc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceProfilingEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceProfilingEvent.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.profiling.types; import java.io.IOException; @@ -28,12 +27,11 @@ import org.apache.flink.runtime.jobgraph.JobID; /** * Instance profiling events are a special subclass of profiling events. They contain profiling information about the * utilization of a particular instance during a job execution. - *

- * This class is not thread-safe. - * */ public abstract class InstanceProfilingEvent extends ProfilingEvent { + private static final long serialVersionUID = 5964092674722506040L; + /** * The interval of time this profiling event covers in milliseconds. */ @@ -176,6 +174,8 @@ public abstract class InstanceProfilingEvent extends ProfilingEvent { super(); } + // -------------------------------------------------------------------------------------------- + /** * Returns the interval of time this profiling event covers in milliseconds. * @@ -304,7 +304,10 @@ public abstract class InstanceProfilingEvent extends ProfilingEvent { return this.transmittedBytes; } - + // -------------------------------------------------------------------------------------------- + // Serialization + // -------------------------------------------------------------------------------------------- + @Override public void read(final DataInputView in) throws IOException { super.read(in); @@ -353,87 +356,44 @@ public abstract class InstanceProfilingEvent extends ProfilingEvent { } + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + @Override public boolean equals(final Object obj) { - - if (!super.equals(obj)) { - return false; - } - - if (!(obj instanceof InstanceProfilingEvent)) { - return false; - } - - final InstanceProfilingEvent instanceProfilingEvent = (InstanceProfilingEvent) obj; - - if (this.profilingInterval != instanceProfilingEvent.getProfilingInterval()) { - return false; - } - - if (this.ioWaitCPU != instanceProfilingEvent.getIOWaitCPU()) { - return false; + if (obj instanceof InstanceProfilingEvent) { + final InstanceProfilingEvent other = (InstanceProfilingEvent) obj; + + return super.equals(obj) && + profilingInterval == other.profilingInterval && + ioWaitCPU == other.ioWaitCPU && + idleCPU == other.idleCPU && + userCPU == other.userCPU && + systemCPU == other.systemCPU && + hardIrqCPU == other.hardIrqCPU && + softIrqCPU == other.softIrqCPU && + totalMemory == other.totalMemory && + freeMemory == other.freeMemory && + bufferedMemory == other.bufferedMemory && + cachedMemory == other.cachedMemory && + cachedSwapMemory == other.cachedSwapMemory && + receivedBytes == other.receivedBytes && + transmittedBytes == other.transmittedBytes; } - if (this.idleCPU != instanceProfilingEvent.getIdleCPU()) { + else { return false; } - - if (this.userCPU != instanceProfilingEvent.getUserCPU()) { - return false; - } - - if (this.systemCPU != instanceProfilingEvent.getSystemCPU()) { - return false; - } - - if (this.hardIrqCPU != instanceProfilingEvent.getHardIrqCPU()) { - return false; - } - - if (this.softIrqCPU != instanceProfilingEvent.getSoftIrqCPU()) { - return false; - } - - if (this.totalMemory != instanceProfilingEvent.getTotalMemory()) { - return false; - } - - if (this.freeMemory != instanceProfilingEvent.getFreeMemory()) { - return false; - } - - if (this.bufferedMemory != instanceProfilingEvent.getBufferedMemory()) { - return false; - } - - if (this.cachedMemory != instanceProfilingEvent.getCachedMemory()) { - return false; - } - - if (this.cachedSwapMemory != instanceProfilingEvent.getCachedSwapMemory()) { - return false; - } - - if (this.receivedBytes != instanceProfilingEvent.getReceivedBytes()) { - return false; - } - - if (this.transmittedBytes != instanceProfilingEvent.getTransmittedBytes()) { - return false; - } - - return true; } @Override public int hashCode() { - long hashCode = getJobID().hashCode() + getTimestamp() + getProfilingTimestamp(); - hashCode += (this.profilingInterval + this.ioWaitCPU + this.idleCPU + this.userCPU + this.systemCPU - + this.hardIrqCPU + this.softIrqCPU); + hashCode += this.profilingInterval + this.ioWaitCPU + this.idleCPU + this.userCPU + this.systemCPU + + this.hardIrqCPU + this.softIrqCPU; hashCode += (this.totalMemory + this.freeMemory + this.bufferedMemory + this.cachedMemory + this.cachedSwapMemory); - hashCode -= Integer.MAX_VALUE; - - return (int) (hashCode % Integer.MAX_VALUE); + hashCode ^= hashCode >>> 32; + return (int) hashCode; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceSummaryProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceSummaryProfilingEvent.java index 87c89177265..0a3e22189a0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceSummaryProfilingEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceSummaryProfilingEvent.java @@ -16,20 +16,17 @@ * limitations under the License. */ - package org.apache.flink.runtime.profiling.types; import org.apache.flink.runtime.jobgraph.JobID; /** - * Instance summary profiling events summarize the profiling events of all instances involved in computing a Nephele - * job. - *

- * This class is not thread-safe. - * + * Instance summary profiling events summarize the profiling events of all instances involved in computing a job. */ public final class InstanceSummaryProfilingEvent extends InstanceProfilingEvent { + private static final long serialVersionUID = 1L; + /** * Constructs a new instance summary profiling event. * @@ -85,25 +82,8 @@ public final class InstanceSummaryProfilingEvent extends InstanceProfilingEvent super(); } - @Override - public boolean equals(final Object obj) { - - if (!super.equals(obj)) { - return false; - } - - if (!(obj instanceof InstanceSummaryProfilingEvent)) { - return false; - } - - return true; - } - - - @Override - public int hashCode() { - - return super.hashCode(); + public boolean equals(Object obj) { + return (obj instanceof InstanceSummaryProfilingEvent) && super.equals(obj); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/OutputGateProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/OutputGateProfilingEvent.java deleted file mode 100644 index 7acb7981b50..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/OutputGateProfilingEvent.java +++ /dev/null @@ -1,150 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.profiling.types; - -import java.io.IOException; - -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.managementgraph.ManagementVertexID; - -/** - * Output gate profiling events are a special subclass of vertex profiling events. They contain profiling information - * which refer to particular output gates of a task. - *

- * This class is not thread-safe. - * - */ -public final class OutputGateProfilingEvent extends VertexProfilingEvent { - - /** - * The index of the output gate at the corresponding management vertex. - */ - private int gateIndex; - - /** - * Stores how often the output gate had exhausted one of its channels capacity during the last time period. - */ - private int channelCapacityExhausted; - - /** - * Constructs a new output gate profiling event. - * - * @param gateIndex - * the index of the output gate at the corresponding management vertex - * @param channelCapacityExhausted - * indicates how often the output gate had exhausted one of its channels capacity during the last time period - * @param vertexID - * the ID of the management vertex this event refers to - * @param profilingInterval - * the interval of time this profiling event covers - * @param jobID - * the ID of the job this event refers to - * @param timestamp - * the time stamp of the event - * @param profilingTimestamp - * the time stamp of the profiling data - */ - public OutputGateProfilingEvent(final int gateIndex, final int channelCapacityExhausted, - final ManagementVertexID vertexID, final int profilingInterval, final JobID jobID, final long timestamp, - final long profilingTimestamp) { - super(vertexID, profilingInterval, jobID, timestamp, profilingTimestamp); - - this.gateIndex = gateIndex; - this.channelCapacityExhausted = channelCapacityExhausted; - } - - /** - * Default constructor for the serialization/deserialization process. Should not be called for other purposes. - */ - public OutputGateProfilingEvent() { - super(); - } - - /** - * Returns the index of input gate. - * - * @return the index of the input gate - */ - public int getGateIndex() { - return this.gateIndex; - } - - /** - * Returns the number of times the capacity of an attached output channel was exhausted during the given profiling - * interval. - * - * @return the number of times a channel reached its capacity limit - */ - public int getChannelCapacityExhausted() { - return this.channelCapacityExhausted; - } - - - @Override - public void read(final DataInputView in) throws IOException { - super.read(in); - - this.gateIndex = in.readInt(); - this.channelCapacityExhausted = in.readInt(); - } - - - @Override - public void write(final DataOutputView out) throws IOException { - super.write(out); - - out.writeInt(this.gateIndex); - out.writeInt(this.channelCapacityExhausted); - } - - - @Override - public boolean equals(final Object obj) { - - if (!super.equals(obj)) { - return false; - } - - if (!(obj instanceof OutputGateProfilingEvent)) { - return false; - } - - final OutputGateProfilingEvent outputGateProfilingEvent = (OutputGateProfilingEvent) obj; - - if (this.gateIndex != outputGateProfilingEvent.getGateIndex()) { - return false; - } - - if (this.channelCapacityExhausted != outputGateProfilingEvent.getChannelCapacityExhausted()) { - return false; - } - - return true; - } - - - @Override - public int hashCode() { - - return super.hashCode(); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ProfilingEvent.java index 0350394f3d2..0ea74a32ac9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ProfilingEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ProfilingEvent.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.profiling.types; import java.io.IOException; @@ -27,23 +26,20 @@ import org.apache.flink.runtime.event.job.AbstractEvent; import org.apache.flink.runtime.event.job.ManagementEvent; import org.apache.flink.runtime.jobgraph.JobID; +import com.google.common.base.Preconditions; + /** * A profiling event is a special type of event. It is intended to transport profiling data of a Nephele job to external * components. - *

- * This class is not thread-safe. - * */ public abstract class ProfilingEvent extends AbstractEvent implements ManagementEvent { - /** - * The ID of the job the profiling data belongs to. - */ - private JobID jobID; + private static final long serialVersionUID = 1L; - /** - * The profiling time stamp. - */ + /** The ID of the job the profiling data belongs to. */ + private final JobID jobID; + + /** The profiling time stamp. */ private long profilingTimestamp; /** @@ -56,9 +52,10 @@ public abstract class ProfilingEvent extends AbstractEvent implements Management * @param profilingTimestamp * the time stamp of the profiling data */ - public ProfilingEvent(final JobID jobID, final long timestamp, final long profilingTimestamp) { + public ProfilingEvent(JobID jobID, long timestamp, long profilingTimestamp) { super(timestamp); + Preconditions.checkNotNull(jobID); this.jobID = jobID; this.profilingTimestamp = profilingTimestamp; } @@ -68,8 +65,11 @@ public abstract class ProfilingEvent extends AbstractEvent implements Management */ public ProfilingEvent() { super(); + this.jobID = new JobID(); } + // -------------------------------------------------------------------------------------------- + /** * Returns the ID of the job this profiling information belongs to. * @@ -90,59 +90,47 @@ public abstract class ProfilingEvent extends AbstractEvent implements Management return this.profilingTimestamp; } + // -------------------------------------------------------------------------------------------- + // Serialization + // -------------------------------------------------------------------------------------------- @Override - public void read(final DataInputView in) throws IOException { + public void read(DataInputView in) throws IOException { super.read(in); - this.jobID = new JobID(); this.jobID.read(in); - this.profilingTimestamp = in.readLong(); } @Override - public void write(final DataOutputView out) throws IOException { + public void write(DataOutputView out) throws IOException { super.write(out); this.jobID.write(out); out.writeLong(this.profilingTimestamp); } + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- @Override - public boolean equals(final Object obj) { - - if (!super.equals(obj)) { - return false; + public boolean equals(Object obj) { + if (obj instanceof ProfilingEvent) { + final ProfilingEvent other = (ProfilingEvent) obj; + + return super.equals(obj) && this.profilingTimestamp == other.profilingTimestamp && + this.jobID.equals(other.jobID); } - - if (!(obj instanceof ProfilingEvent)) { + else { return false; } - - final ProfilingEvent profilingEvent = (ProfilingEvent) obj; - - if (!this.jobID.equals(profilingEvent.getJobID())) { - return false; - } - - if (this.profilingTimestamp != profilingEvent.getProfilingTimestamp()) { - return false; - } - - return true; } - @Override public int hashCode() { - - if (this.jobID != null) { - return this.jobID.hashCode(); - } - - return super.hashCode(); + return this.jobID.hashCode() ^ ((int) (profilingTimestamp >>> 32)) ^ ((int) (profilingTimestamp)) ^ + super.hashCode(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/SingleInstanceProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/SingleInstanceProfilingEvent.java index 17bfe6d9ebe..59e6d5a9e56 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/SingleInstanceProfilingEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/SingleInstanceProfilingEvent.java @@ -16,22 +16,24 @@ * limitations under the License. */ - package org.apache.flink.runtime.profiling.types; import java.io.IOException; -import org.apache.flink.core.io.StringRecord; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.jobgraph.JobID; +import org.apache.flink.types.StringValue; + +import com.google.common.base.Preconditions; /** * A single instance profiling event encapsulates profiling information for one particular instance. - * */ public final class SingleInstanceProfilingEvent extends InstanceProfilingEvent { + private static final long serialVersionUID = 1L; + private String instanceName; /** @@ -78,11 +80,13 @@ public final class SingleInstanceProfilingEvent extends InstanceProfilingEvent { final int userCPU, final int systemCPU, final int hardIrqCPU, final int softIrqCPU, final long totalMemory, final long freeMemory, final long bufferedMemory, final long cachedMemory, final long cachedSwapMemory, final long receivedBytes, final long transmittedBytes, final JobID jobID, final long timestamp, - final long profilingTimestamp, final String instanceName) { + final long profilingTimestamp, final String instanceName) + { super(profilingInterval, ioWaitCPU, idleCPU, userCPU, systemCPU, hardIrqCPU, softIrqCPU, totalMemory, freeMemory, bufferedMemory, cachedMemory, cachedSwapMemory, receivedBytes, transmittedBytes, jobID, timestamp, profilingTimestamp); + Preconditions.checkNotNull(instanceName); this.instanceName = instanceName; } @@ -93,6 +97,8 @@ public final class SingleInstanceProfilingEvent extends InstanceProfilingEvent { super(); } + // -------------------------------------------------------------------------------------------- + /** * Returns the name of the instance. * @@ -102,47 +108,40 @@ public final class SingleInstanceProfilingEvent extends InstanceProfilingEvent { return this.instanceName; } - + // -------------------------------------------------------------------------------------------- + // Serialization + // -------------------------------------------------------------------------------------------- + @Override public void read(DataInputView in) throws IOException { super.read(in); - - this.instanceName = StringRecord.readString(in); + this.instanceName = StringValue.readString(in); } - @Override public void write(DataOutputView out) throws IOException { super.write(out); - - StringRecord.writeString(out, this.instanceName); + StringValue.writeString(this.instanceName, out); } - + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- @Override public boolean equals(Object obj) { - - if (!super.equals(obj)) { - return false; + if (obj instanceof SingleInstanceProfilingEvent) { + SingleInstanceProfilingEvent other = (SingleInstanceProfilingEvent) obj; + return super.equals(obj) && this.instanceName.equals(other.instanceName); + } - - if (!(obj instanceof SingleInstanceProfilingEvent)) { - return false; - } - - final SingleInstanceProfilingEvent singleInstanceProfilingEvent = (SingleInstanceProfilingEvent) obj; - - if (!this.instanceName.equals(singleInstanceProfilingEvent.getInstanceName())) { + else { return false; } - - return true; } - - + @Override public int hashCode() { - - return super.hashCode(); + return super.hashCode() + 31*instanceName.hashCode(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ThreadProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ThreadProfilingEvent.java index 0e2a151815f..306a3969976 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ThreadProfilingEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ThreadProfilingEvent.java @@ -16,23 +16,24 @@ * limitations under the License. */ - package org.apache.flink.runtime.profiling.types; import java.io.IOException; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.managementgraph.ManagementVertexID; +import org.apache.flink.runtime.jobgraph.JobVertexID; /** * Through this interface it is possible to access profiling data about the CPU utilization * of the corresponding execution thread during its execution. - * */ public class ThreadProfilingEvent extends VertexProfilingEvent { + private static final long serialVersionUID = -3006867830244444710L; + private int userTime; private int systemTime; @@ -42,8 +43,10 @@ public class ThreadProfilingEvent extends VertexProfilingEvent { private int waitedTime; public ThreadProfilingEvent(int userTime, int systemTime, int blockedTime, int waitedTime, - ManagementVertexID vertexID, int profilingInterval, JobID jobID, long timestamp, long profilingTimestamp) { - super(vertexID, profilingInterval, jobID, timestamp, profilingTimestamp); + JobVertexID vertexId, int subtask, ExecutionAttemptID executionId, + int profilingInterval, JobID jobID, long timestamp, long profilingTimestamp) + { + super(vertexId, subtask, executionId, profilingInterval, jobID, timestamp, profilingTimestamp); this.userTime = userTime; this.systemTime = systemTime; @@ -54,6 +57,8 @@ public class ThreadProfilingEvent extends VertexProfilingEvent { public ThreadProfilingEvent() { super(); } + + // -------------------------------------------------------------------------------------------- /** * Returns the percentage of time the execution thread spent in @@ -95,6 +100,9 @@ public class ThreadProfilingEvent extends VertexProfilingEvent { return this.waitedTime; } + // -------------------------------------------------------------------------------------------- + // Serialization + // -------------------------------------------------------------------------------------------- @Override public void read(DataInputView in) throws IOException { @@ -118,42 +126,25 @@ public class ThreadProfilingEvent extends VertexProfilingEvent { } + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + @Override public boolean equals(Object obj) { - - if (!super.equals(obj)) { - return false; - } - - if (!(obj instanceof ThreadProfilingEvent)) { - return false; - } - - final ThreadProfilingEvent threadProfilingEvent = (ThreadProfilingEvent) obj; - - if (this.userTime != threadProfilingEvent.getUserTime()) { - return false; - } - - if (this.systemTime != threadProfilingEvent.getSystemTime()) { - return false; - } - - if (this.blockedTime != threadProfilingEvent.getBlockedTime()) { - return false; + if (obj instanceof ThreadProfilingEvent) { + final ThreadProfilingEvent other = (ThreadProfilingEvent) obj; + + return this.userTime == other.userTime && + this.systemTime == other.systemTime && + this.blockedTime == other.blockedTime && + this.waitedTime == other.waitedTime && + super.equals(obj); } - - if (this.waitedTime != threadProfilingEvent.getWaitedTime()) { + else { return false; } - - return true; - } - - - @Override - public int hashCode() { - - return super.hashCode(); } + + // hash code is inherited from the superclass } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/VertexProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/VertexProfilingEvent.java index ae064ca43f1..9de85556609 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/VertexProfilingEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/VertexProfilingEvent.java @@ -23,31 +23,45 @@ import java.io.IOException; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.managementgraph.ManagementVertexID; +import org.apache.flink.runtime.jobgraph.JobVertexID; /** * This interface is a base interface for profiling data which * pertains to the execution of tasks. - * */ public abstract class VertexProfilingEvent extends ProfilingEvent { - private ManagementVertexID vertexID; + private static final long serialVersionUID = -5364961557518174880L; + + private final JobVertexID vertexId; + + private int subtask; + + private final ExecutionAttemptID executionId; private int profilingInterval; - public VertexProfilingEvent(ManagementVertexID vertexID, int profilingInterval, JobID jobID, long timestamp, - long profilingTimestamp) { + + public VertexProfilingEvent(JobVertexID vertexId, int subtask, ExecutionAttemptID executionId, + int profilingInterval, JobID jobID, long timestamp, long profilingTimestamp) + { super(jobID, timestamp, profilingTimestamp); - this.vertexID = vertexID; + this.vertexId = vertexId; + this.subtask = subtask; + this.executionId = executionId; this.profilingInterval = profilingInterval; } public VertexProfilingEvent() { super(); + this.vertexId = new JobVertexID(); + this.executionId = new ExecutionAttemptID(); } + + // -------------------------------------------------------------------------------------------- /** * Returns the ID of the vertex this profiling information @@ -55,8 +69,8 @@ public abstract class VertexProfilingEvent extends ProfilingEvent { * * @return the ID of the vertex this profiling information belongs to */ - public ManagementVertexID getVertexID() { - return this.vertexID; + public JobVertexID getVertexID() { + return this.vertexId; } /** @@ -68,15 +82,26 @@ public abstract class VertexProfilingEvent extends ProfilingEvent { public int getProfilingInterval() { return this.profilingInterval; } + + public int getSubtask() { + return subtask; + } + + public ExecutionAttemptID getExecutionId() { + return executionId; + } - + // -------------------------------------------------------------------------------------------- + // Serialization + // -------------------------------------------------------------------------------------------- + @Override public void read(DataInputView in) throws IOException { super.read(in); - this.vertexID = new ManagementVertexID(); - this.vertexID.read(in); - + this.vertexId.read(in); + this.executionId.read(in); + this.subtask = in.readInt(); this.profilingInterval = in.readInt(); } @@ -85,32 +110,33 @@ public abstract class VertexProfilingEvent extends ProfilingEvent { public void write(DataOutputView out) throws IOException { super.write(out); - this.vertexID.write(out); + this.vertexId.write(out); + this.executionId.write(out); + out.writeInt(subtask); out.writeInt(this.profilingInterval); } - + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + @Override public boolean equals(Object obj) { - - if (!super.equals(obj)) { - return false; + if (obj instanceof VertexProfilingEvent) { + final VertexProfilingEvent other = (VertexProfilingEvent) obj; + + return super.equals(other) && this.subtask == other.subtask && + this.profilingInterval == other.profilingInterval && + this.vertexId.equals(other.vertexId) && + this.executionId.equals(other.executionId); } - - if (!(obj instanceof VertexProfilingEvent)) { + else { return false; } - - final VertexProfilingEvent vertexProfilingEvent = (VertexProfilingEvent) obj; - - if (!this.vertexID.equals(vertexProfilingEvent.getVertexID())) { - return false; - } - - if (this.profilingInterval != vertexProfilingEvent.getProfilingInterval()) { - return false; - } - - return true; + } + + @Override + public int hashCode() { + return super.hashCode() ^ vertexId.hashCode() ^ (31*subtask) ^ executionId.hashCode(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java index b0806063c4e..9e3e22e7c3e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java @@ -24,7 +24,6 @@ import java.util.List; import org.apache.flink.runtime.event.job.AbstractEvent; import org.apache.flink.runtime.event.job.RecentJobEvent; import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.managementgraph.ManagementGraph; /** * This protocol provides extended management capabilities beyond the @@ -34,18 +33,6 @@ import org.apache.flink.runtime.managementgraph.ManagementGraph; */ public interface ExtendedManagementProtocol extends JobManagementProtocol { - /** - * Retrieves the management graph for the job - * with the given ID. - * - * @param jobID - * the ID identifying the job - * @return the management graph for the job - * @throws IOException - * thrown if an error occurs while retrieving the management graph - */ - ManagementGraph getManagementGraph(JobID jobID) throws IOException; - /** * Retrieves a list of jobs which have either running or have been started recently. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java index 67e47008fab..f2259c4cdeb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java @@ -51,8 +51,7 @@ import org.apache.commons.cli.Option; import org.apache.commons.cli.OptionBuilder; import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; + import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry; import org.apache.flink.configuration.ConfigConstants; @@ -98,6 +97,9 @@ import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * A task manager receives tasks from the job manager and executes them. After having executed them * (or in case of an execution error) it reports the execution result back to the job manager. @@ -106,7 +108,7 @@ import org.apache.flink.util.StringUtils; */ public class TaskManager implements TaskOperationProtocol { - private static final Log LOG = LogFactory.getLog(TaskManager.class); + private static final Logger LOG = LoggerFactory.getLogger(TaskManager.class); private static final int STARTUP_FAILURE_RETURN_CODE = 1; @@ -219,7 +221,7 @@ public class TaskManager implements TaskOperationProtocol { this.taskManagerServer = RPC.getServer(this, taskManagerBindAddress.getHostAddress(), ipcPort, numberOfSlots); this.taskManagerServer.start(); } catch (IOException e) { - LOG.fatal("Failed to start TaskManager server. " + e.getMessage(), e); + LOG.error("Failed to start TaskManager server. " + e.getMessage(), e); throw new Exception("Failed to start taskmanager server. " + e.getMessage(), e); } @@ -266,21 +268,20 @@ public class TaskManager implements TaskOperationProtocol { networkConnectionManager = new LocalConnectionManager(); break; case CLUSTER: - int numInThreads = GlobalConfiguration.getInteger( - ConfigConstants.TASK_MANAGER_NET_NUM_IN_THREADS_KEY, + int numInThreads = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NET_NUM_IN_THREADS_KEY, ConfigConstants.DEFAULT_TASK_MANAGER_NET_NUM_IN_THREADS); - - int numOutThreads = GlobalConfiguration.getInteger( - ConfigConstants.TASK_MANAGER_NET_NUM_OUT_THREADS_KEY, + + int numOutThreads = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NET_NUM_OUT_THREADS_KEY, ConfigConstants.DEFAULT_TASK_MANAGER_NET_NUM_OUT_THREADS); - - int closeAfterIdleForMs = GlobalConfiguration.getInteger( - ConfigConstants.TASK_MANAGER_NET_CLOSE_AFTER_IDLE_FOR_MS_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_NET_CLOSE_AFTER_IDLE_FOR_MS); - - networkConnectionManager = new NettyConnectionManager( - localInstanceConnectionInfo.address(), localInstanceConnectionInfo.dataPort(), - bufferSize, numInThreads, numOutThreads, closeAfterIdleForMs); + + int lowWaterMark = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NET_NETTY_LOW_WATER_MARK, + ConfigConstants.DEFAULT_TASK_MANAGER_NET_NETTY_LOW_WATER_MARK); + + int highWaterMark = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NET_NETTY_HIGH_WATER_MARK, + ConfigConstants.DEFAULT_TASK_MANAGER_NET_NETTY_HIGH_WATER_MARK); + + networkConnectionManager = new NettyConnectionManager(localInstanceConnectionInfo.address(), + localInstanceConnectionInfo.dataPort(), bufferSize, numInThreads, numOutThreads, lowWaterMark, highWaterMark); break; } @@ -323,7 +324,7 @@ public class TaskManager implements TaskOperationProtocol { this.memoryManager = new DefaultMemoryManager(memorySize, this.numberOfSlots, pageSize); } catch (Throwable t) { - LOG.fatal("Unable to initialize memory manager with " + (memorySize >>> 20) + " megabytes of memory.", t); + LOG.error("Unable to initialize memory manager with " + (memorySize >>> 20) + " megabytes of memory.", t); throw new Exception("Unable to initialize memory manager.", t); } } @@ -449,7 +450,7 @@ public class TaskManager implements TaskOperationProtocol { try { this.executorService.awaitTermination(5000L, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { - LOG.debug(e); + LOG.debug("Shutdown of executor thread pool interrupted", e); } } @@ -898,7 +899,7 @@ public class TaskManager implements TaskOperationProtocol { jobManagerAddress = new InetSocketAddress(tmpAddress, port); } catch (UnknownHostException e) { - LOG.fatal("Could not resolve JobManager host name."); + LOG.error("Could not resolve JobManager host name."); throw new Exception("Could not resolve JobManager host name: " + e.getMessage(), e); } @@ -940,7 +941,7 @@ public class TaskManager implements TaskOperationProtocol { jobManager = RPC.getProxy(JobManagerProtocol.class, jobManagerAddress, NetUtils.getSocketFactory()); } catch (IOException e) { - LOG.fatal("Could not connect to the JobManager: " + e.getMessage(), e); + LOG.error("Could not connect to the JobManager: " + e.getMessage(), e); throw new Exception("Failed to initialize connection to JobManager: " + e.getMessage(), e); } @@ -949,7 +950,7 @@ public class TaskManager implements TaskOperationProtocol { splitProvider = RPC.getProxy(InputSplitProviderProtocol.class, jobManagerAddress, NetUtils.getSocketFactory()); } catch (IOException e) { - LOG.fatal(e.getMessage(), e); + LOG.error(e.getMessage(), e); throw new Exception("Failed to initialize connection to global input split provider: " + e.getMessage(), e); } @@ -958,7 +959,7 @@ public class TaskManager implements TaskOperationProtocol { channelLookup = RPC.getProxy(ChannelLookupProtocol.class, jobManagerAddress, NetUtils.getSocketFactory()); } catch (IOException e) { - LOG.fatal(e.getMessage(), e); + LOG.error(e.getMessage(), e); throw new Exception("Failed to initialize channel lookup protocol. " + e.getMessage(), e); } @@ -967,7 +968,7 @@ public class TaskManager implements TaskOperationProtocol { accumulators = RPC.getProxy(AccumulatorProtocol.class, jobManagerAddress, NetUtils.getSocketFactory()); } catch (IOException e) { - LOG.fatal("Failed to initialize accumulator protocol: " + e.getMessage(), e); + LOG.error("Failed to initialize accumulator protocol: " + e.getMessage(), e); throw new Exception("Failed to initialize accumulator protocol: " + e.getMessage(), e); } @@ -1040,8 +1041,9 @@ public class TaskManager implements TaskOperationProtocol { // Create a new task manager object try { createTaskManager(ExecutionMode.CLUSTER); - } catch (Exception e) { - LOG.fatal("Taskmanager startup failed: " + e.getMessage(), e); + } + catch (Throwable t) { + LOG.error("Taskmanager startup failed: " + t.getMessage(), t); System.exit(STARTUP_FAILURE_RETURN_CODE); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicEnum.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicEnum.java deleted file mode 100644 index d4634db43ed..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicEnum.java +++ /dev/null @@ -1,95 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.util; - -import java.util.concurrent.atomic.AtomicReference; - -/** - * Auxiliary class which provides atomic operations on enumerations. Internally, the class uses an - * {@link AtomicReference} object to guarantee the atomicity. - *

- * This class is thread-safe. - * - */ -public final class AtomicEnum> { - - /** - * The atomic reference which provides the atomicity internally. - */ - private final AtomicReference ref; - - /** - * Constructs a new atomic enumeration object. - * - * @param initialValue - * the initial value of the enumeration - */ - 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. - * - * @return the current value. - */ - public T get() { - - return this.ref.get(); - } - - /** - * Sets the given value and returns the old value. - * - * @param newValue - * the new value - * @return the previous value - */ - public T getAndSet(final T newValue) { - - return this.ref.getAndSet(newValue); - } - - /** - * Atomically set the value to the given updated value if the current value == the expected value. - * - * @param expect - * the expected value - * @param update - * the new value - * @return true if successful, false otherwise - */ - public boolean compareAndSet(final T expect, final T update) { - - return this.ref.compareAndSet(expect, update); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicEnumerator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicEnumerator.java deleted file mode 100644 index 94bcca684db..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/AtomicEnumerator.java +++ /dev/null @@ -1,57 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.util; - -import java.util.concurrent.atomic.AtomicInteger; - - -/** - */ -public class AtomicEnumerator -{ - private final E[] values; - - private final AtomicInteger next; - - - public AtomicEnumerator(E[] values) - { - this.values = values; - this.next = new AtomicInteger(0); - } - - - public E getNext() - { - int n, nv; - do { - n = this.next.get(); - nv = n+1; - } while (!this.next.compareAndSet(n, nv == this.values.length ? 0 : nv)); - - return this.values[n]; - } - - - public static final AtomicEnumerator get(T[] values) - { - return new AtomicEnumerator(values); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java index c2efc0ab5d2..ca6bf6c62ee 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.util; import java.io.InputStream; @@ -33,6 +32,12 @@ public class EnvironmentInformation { private static final Logger LOG = LoggerFactory.getLogger(EnvironmentInformation.class); private static final String UNKNOWN = ""; + + private static final String[] IGNORED_STARTUP_OPTIONS = { + "-Dlog.file", + "-Dlogback.configurationFile", + "-Dlog4j.configuration" + }; /** * Returns the version of the code as String. If version == null, then the JobManager does not run from a @@ -156,6 +161,22 @@ public class EnvironmentInformation { try { final RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean(); final StringBuilder bld = new StringBuilder(); + + for (String s : bean.getInputArguments()) { + + boolean append = true; + for (String ignored : IGNORED_STARTUP_OPTIONS) { + if (s.startsWith(ignored)) { + append = false; + break; + } + } + + if (append) { + bld.append(s).append(' '); + } + } + return bld.toString(); } catch (Throwable t) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java index fa7ad1c37a7..91d9973f818 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java @@ -21,12 +21,12 @@ package org.apache.flink.runtime.util; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ExecutorThreadFactory implements ThreadFactory { - private static final Log LOG = LogFactory.getLog(ExecutorThreadFactory.class); + private static final Logger LOG = LoggerFactory.getLogger(ExecutorThreadFactory.class); private static final String THREAD_NAME_PREFIX = "Flink Executor Thread - "; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/JobEventTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/JobEventTest.java deleted file mode 100644 index c8ded868a8f..00000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/JobEventTest.java +++ /dev/null @@ -1,57 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.event.job; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.IOException; - -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.testutils.CommonTestUtils; -import org.junit.Test; - -/** - * This class contains tests concerning the serialization/deserialization of job events which have been derived from - * {@link org.apache.flink.runtime.event.job.AbstractEvent}. - */ -public class JobEventTest { - - /** - * This test checks the correct serialization/deserialization of a {@link JobEvent}. - */ - @Test - public void testJobEvent() { - - try { - - final JobEvent orig = new JobEvent(1234567L, JobStatus.FINISHED, null); - final JobEvent copy = (JobEvent) CommonTestUtils.createCopyWritable(orig); - - assertEquals(orig.getTimestamp(), copy.getTimestamp()); - assertEquals(orig.getCurrentJobStatus(), copy.getCurrentJobStatus()); - assertEquals(orig.hashCode(), copy.hashCode()); - assertTrue(orig.equals(copy)); - - } catch (IOException ioe) { - fail(ioe.getMessage()); - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/ManagementEventTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/ManagementEventTest.java deleted file mode 100644 index 99e750bb887..00000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/ManagementEventTest.java +++ /dev/null @@ -1,85 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.event.job; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import org.apache.flink.runtime.event.job.ExecutionStateChangeEvent; -import org.apache.flink.runtime.event.job.RecentJobEvent; -import org.apache.flink.runtime.execution.ExecutionState2; -import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.managementgraph.ManagementVertexID; -import org.apache.flink.runtime.testutils.ManagementTestUtils; -import org.junit.Test; - -/** - * This test checks the proper serialization and deserialization of job events. - */ -public class ManagementEventTest { - - /** - * The time stamp used during the tests. - */ - private static final long TIMESTAMP = 123456789L; - - /** - * The name of the job used during the tests. - */ - private static final String JOBNAME = "Test Job Name"; - - /** - * Tests serialization/deserialization for {@link ExecutionStateChangeEvent}. - */ - @Test - public void testExecutionStateChangeEvent() { - - final ExecutionStateChangeEvent orig = new ExecutionStateChangeEvent(TIMESTAMP, new ManagementVertexID(), - ExecutionState2.DEPLOYING); - - final ExecutionStateChangeEvent copy = (ExecutionStateChangeEvent) ManagementTestUtils.createCopy(orig); - - assertEquals(orig.getTimestamp(), copy.getTimestamp()); - assertEquals(orig.getVertexID(), copy.getVertexID()); - assertEquals(orig.getNewExecutionState(), copy.getNewExecutionState()); - assertEquals(orig.hashCode(), copy.hashCode()); - assertTrue(orig.equals(copy)); - } - - /** - * Tests serialization/deserialization for {@link RecentJobEvent}. - */ - @Test - public void testRecentJobEvent() { - - final RecentJobEvent orig = new RecentJobEvent(new JobID(), JOBNAME, JobStatus.RUNNING, true, TIMESTAMP, TIMESTAMP); - - final RecentJobEvent copy = (RecentJobEvent) ManagementTestUtils.createCopy(orig); - - assertEquals(orig.getJobID(), copy.getJobID()); - assertEquals(orig.getJobName(), copy.getJobName()); - assertEquals(orig.getJobStatus(), copy.getJobStatus()); - assertEquals(orig.isProfilingAvailable(), copy.isProfilingAvailable()); - assertEquals(orig.getTimestamp(), copy.getTimestamp()); - assertEquals(orig.getSubmissionTimestamp(), copy.getSubmissionTimestamp()); - assertEquals(orig.hashCode(), copy.hashCode()); - assertTrue(orig.equals(copy)); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java index 09da1a58ff9..25837717f64 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java @@ -18,49 +18,28 @@ package org.apache.flink.runtime.instance; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import java.io.IOException; import java.net.ServerSocket; -import java.util.List; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.core.io.InputSplit; import org.apache.flink.runtime.ExecutionMode; -import org.apache.flink.runtime.accumulators.AccumulatorEvent; -import org.apache.flink.runtime.client.JobCancelResult; -import org.apache.flink.runtime.client.JobProgressResult; -import org.apache.flink.runtime.client.JobSubmissionResult; -import org.apache.flink.runtime.event.job.AbstractEvent; -import org.apache.flink.runtime.event.job.RecentJobEvent; -import org.apache.flink.runtime.executiongraph.ExecutionGraph; -import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.executiongraph.JobStatusListener; -import org.apache.flink.runtime.io.network.ConnectionInfoLookupResponse; -import org.apache.flink.runtime.io.network.channels.ChannelID; import org.apache.flink.runtime.ipc.RPC; import org.apache.flink.runtime.ipc.RPC.Server; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobmanager.DeploymentManager; -import org.apache.flink.runtime.managementgraph.ManagementGraph; -import org.apache.flink.runtime.protocols.AccumulatorProtocol; -import org.apache.flink.runtime.protocols.ChannelLookupProtocol; -import org.apache.flink.runtime.protocols.ExtendedManagementProtocol; -import org.apache.flink.runtime.protocols.InputSplitProviderProtocol; import org.apache.flink.runtime.protocols.JobManagerProtocol; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManager; -import org.apache.flink.runtime.types.IntegerRecord; import org.junit.After; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; -import static org.junit.Assert.*; - public class LocalInstanceManagerTest { private Server jmServer; @@ -71,7 +50,7 @@ public class LocalInstanceManagerTest { public void startJobManagerServer() { try { this.port = getAvailablePort(); - this.jmServer = RPC.getServer(new EmptyRPCs(), "localhost", this.port, 1); + this.jmServer = RPC.getServer(new MockRPC(), "localhost", this.port, 1); this.jmServer.start(); Configuration cfg = new Configuration(); @@ -170,72 +149,11 @@ public class LocalInstanceManagerTest { throw new IOException("Could not find a free port."); } - private static final class EmptyRPCs implements DeploymentManager, ExtendedManagementProtocol, InputSplitProviderProtocol, - JobManagerProtocol, ChannelLookupProtocol, JobStatusListener, AccumulatorProtocol - { - - @Override - public JobSubmissionResult submitJob(JobGraph job) { - return null; - } - - @Override - public JobProgressResult getJobProgress(JobID jobID) { - return null; - } - - @Override - public JobCancelResult cancelJob(JobID jobID) { - return null; - } - - @Override - public IntegerRecord getRecommendedPollingInterval() { - return null; - } - - @Override - public void reportAccumulatorResult(AccumulatorEvent accumulatorEvent) {} - - @Override - public AccumulatorEvent getAccumulatorResults(JobID jobID) { - return null; - } - - @Override - public void jobStatusHasChanged(ExecutionGraph executionGraph, JobStatus newJobStatus, String optionalMessage) {} - - @Override - public ConnectionInfoLookupResponse lookupConnectionInfo(InstanceConnectionInfo caller, JobID jobID, ChannelID sourceChannelID) { - return null; - } + private static final class MockRPC implements JobManagerProtocol { @Override public void updateTaskExecutionState(TaskExecutionState taskExecutionState) {} - @Override - public ManagementGraph getManagementGraph(JobID jobID) { - return null; - } - - @Override - public List getRecentJobs() { - return null; - } - - @Override - public List getEvents(JobID jobID) { - return null; - } - - @Override - public int getAvailableSlots() { - return 0; - } - - @Override - public void deploy(JobID jobID, Instance instance, List verticesToBeDeployed) {} - @Override public boolean sendHeartbeat(InstanceID taskManagerId) { return true; @@ -245,10 +163,5 @@ public class LocalInstanceManagerTest { public InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription hardwareDescription, int numberOfSlots) { return new InstanceID(); } - - @Override - public InputSplit requestNextInputSplit(JobID jobID, JobVertexID vertex) throws IOException { - return null; - } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/ipc/RpcTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/ipc/RpcTest.java index d45b131ae74..12e7a40842d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/ipc/RpcTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/ipc/RpcTest.java @@ -20,7 +20,6 @@ package org.apache.flink.runtime.ipc; import static org.junit.Assert.*; -import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; @@ -32,16 +31,9 @@ import org.apache.flink.runtime.net.NetUtils; import org.apache.flink.types.DoubleValue; import org.apache.flink.types.LongValue; import org.apache.flink.types.StringValue; -import org.apache.flink.util.LogUtils; public class RpcTest { - - @BeforeClass - public static void initLogger() { - LogUtils.initializeDefaultConsoleLogger(); - } - @Test public void testRpc() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/managementgraph/ManagementGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/managementgraph/ManagementGraphTest.java deleted file mode 100644 index 5293dc6cc00..00000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/managementgraph/ManagementGraphTest.java +++ /dev/null @@ -1,453 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.runtime.managementgraph; - -import static org.junit.Assert.assertEquals; - -import java.util.Iterator; - -import org.apache.flink.runtime.io.network.channels.ChannelType; -import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.managementgraph.ManagementEdge; -import org.apache.flink.runtime.managementgraph.ManagementEdgeID; -import org.apache.flink.runtime.managementgraph.ManagementGate; -import org.apache.flink.runtime.managementgraph.ManagementGateID; -import org.apache.flink.runtime.managementgraph.ManagementGraph; -import org.apache.flink.runtime.managementgraph.ManagementGraphIterator; -import org.apache.flink.runtime.managementgraph.ManagementGroupEdge; -import org.apache.flink.runtime.managementgraph.ManagementGroupVertex; -import org.apache.flink.runtime.managementgraph.ManagementGroupVertexIterator; -import org.apache.flink.runtime.managementgraph.ManagementStage; -import org.apache.flink.runtime.managementgraph.ManagementVertex; -import org.apache.flink.runtime.managementgraph.ManagementVertexID; -import org.apache.flink.runtime.testutils.ManagementTestUtils; -import org.junit.Test; - -/** - * This class contains tests concerning the {@link ManagementGraph} and its components. - * - */ -public class ManagementGraphTest { - - /** - * This test checks the serialization/deserilization of a management graph. - */ - @Test - public void testManagementGraph() { - - final ManagementGraph orig = constructTestManagementGraph(); - final ManagementGraph copy = (ManagementGraph) ManagementTestUtils.createCopy(orig); - - assertEquals(orig.getJobID(), copy.getJobID()); - assertEquals(orig.getNumberOfStages(), copy.getNumberOfStages()); - - for (int i = 0; i < orig.getNumberOfStages(); i++) { - - final ManagementStage origStage = orig.getStage(i); - final ManagementStage copyStage = copy.getStage(i); - - assertEquals(origStage.getNumberOfGroupVertices(), copyStage.getNumberOfGroupVertices()); - assertEquals(origStage.getNumberOfInputGroupVertices(), copyStage.getNumberOfInputGroupVertices()); - assertEquals(origStage.getNumberOfOutputGroupVertices(), copyStage.getNumberOfOutputGroupVertices()); - - for (int j = 0; j < origStage.getNumberOfInputGroupVertices(); j++) { - - final ManagementGroupVertex origGroupVertex = origStage.getInputGroupVertex(j); - final ManagementGroupVertex copyGroupVertex = copyStage.getInputGroupVertex(j); - - assertEquals(origGroupVertex.getID(), copyGroupVertex.getID()); - } - - for (int j = 0; j < origStage.getNumberOfOutputGroupVertices(); j++) { - - final ManagementGroupVertex origGroupVertex = origStage.getOutputGroupVertex(j); - final ManagementGroupVertex copyGroupVertex = copyStage.getOutputGroupVertex(j); - - assertEquals(origGroupVertex.getID(), copyGroupVertex.getID()); - } - - for (int j = 0; j < origStage.getNumberOfGroupVertices(); j++) { - - final ManagementGroupVertex origGroupVertex = origStage.getGroupVertex(j); - final ManagementGroupVertex copyGroupVertex = copyStage.getGroupVertex(j); - - testGroupVertex(origGroupVertex, copyGroupVertex); - } - } - } - - /** - * Auxiliary method to test serialization/deserialization of management group vertices. - * - * @param origGroupVertex - * the original management group vertex - * @param copyGroupVertex - * the deserialized copy of the management group vertex - */ - private void testGroupVertex(final ManagementGroupVertex origGroupVertex, - final ManagementGroupVertex copyGroupVertex) { - - assertEquals(origGroupVertex.getID(), copyGroupVertex.getID()); - assertEquals(origGroupVertex.getName(), copyGroupVertex.getName()); - assertEquals(origGroupVertex.getNumberOfGroupMembers(), copyGroupVertex.getNumberOfGroupMembers()); - - for (int k = 0; k < origGroupVertex.getNumberOfForwardEdges(); k++) { - - final ManagementGroupEdge origGroupEdge = origGroupVertex.getForwardEdge(k); - final ManagementGroupEdge copyGroupEdge = copyGroupVertex.getForwardEdge(k); - - testGroupEdge(origGroupEdge, copyGroupEdge); - } - - for (int k = 0; k < origGroupVertex.getNumberOfBackwardEdges(); k++) { - - final ManagementGroupEdge origGroupEdge = origGroupVertex.getBackwardEdge(k); - final ManagementGroupEdge copyGroupEdge = copyGroupVertex.getBackwardEdge(k); - - testGroupEdge(origGroupEdge, copyGroupEdge); - } - - for (int k = 0; k < origGroupVertex.getNumberOfGroupMembers(); k++) { - - final ManagementVertex origVertex = origGroupVertex.getGroupMember(k); - final ManagementVertex copyVertex = copyGroupVertex.getGroupMember(k); - - testVertex(origVertex, copyVertex); - } - - } - - /** - * Auxiliary method to test serialization/deserialization of management vertices. - * - * @param origVertex - * the original management vertex - * @param copyVertex - * the deserialized copy of the management vertex - */ - private void testVertex(final ManagementVertex origVertex, final ManagementVertex copyVertex) { - - assertEquals(origVertex.getID(), copyVertex.getID()); - assertEquals(origVertex.getExecutionState(), copyVertex.getExecutionState()); - assertEquals(origVertex.getIndexInGroup(), copyVertex.getIndexInGroup()); - assertEquals(origVertex.getInstanceName(), copyVertex.getInstanceName()); - assertEquals(origVertex.getNumberOfInputGates(), copyVertex.getNumberOfInputGates()); - assertEquals(origVertex.getNumberOfOutputGates(), copyVertex.getNumberOfOutputGates()); - - for (int i = 0; i < origVertex.getNumberOfInputGates(); i++) { - - final ManagementGate origGate = origVertex.getInputGate(i); - final ManagementGate copyGate = copyVertex.getInputGate(i); - - testGate(origGate, copyGate); - } - - for (int i = 0; i < origVertex.getNumberOfOutputGates(); i++) { - - final ManagementGate origGate = origVertex.getOutputGate(i); - final ManagementGate copyGate = copyVertex.getOutputGate(i); - - testGate(origGate, copyGate); - } - } - - /** - * Auxiliary method to test serialization/deserialization of management gates. - * - * @param origGate - * the original management gate - * @param copyGate - * the deserialized copy of the management gate - */ - private void testGate(final ManagementGate origGate, final ManagementGate copyGate) { - - assertEquals(origGate.getIndex(), copyGate.getIndex()); - assertEquals(origGate.isInputGate(), copyGate.isInputGate()); - assertEquals(origGate.getNumberOfForwardEdges(), copyGate.getNumberOfForwardEdges()); - assertEquals(origGate.getNumberOfBackwardEdges(), copyGate.getNumberOfBackwardEdges()); - - for (int i = 0; i < origGate.getNumberOfForwardEdges(); i++) { - - final ManagementEdge origEdge = origGate.getForwardEdge(i); - final ManagementEdge copyEdge = copyGate.getForwardEdge(i); - - testEdge(origEdge, copyEdge); - } - - for (int i = 0; i < origGate.getNumberOfBackwardEdges(); i++) { - - final ManagementEdge origEdge = origGate.getBackwardEdge(i); - final ManagementEdge copyEdge = copyGate.getBackwardEdge(i); - - testEdge(origEdge, copyEdge); - } - } - - /** - * Auxiliary method to test serialization/deserialization of management group edges. - * - * @param origGroupEdge - * the original management group edge - * @param copyGroupEdge - * the deserialized copy of the management group edge - */ - private void testGroupEdge(final ManagementGroupEdge origGroupEdge, final ManagementGroupEdge copyGroupEdge) { - - assertEquals(origGroupEdge.getChannelType(), copyGroupEdge.getChannelType()); - assertEquals(origGroupEdge.getSourceIndex(), copyGroupEdge.getSourceIndex()); - assertEquals(origGroupEdge.getTargetIndex(), copyGroupEdge.getTargetIndex()); - } - - /** - * Auxiliary method to test serialization/deserialization of management edges. - * - * @param origEdge - * the original management edge - * @param copyEdge - * the deserialized copy of the management edge - */ - private void testEdge(final ManagementEdge origEdge, final ManagementEdge copyEdge) { - - assertEquals(origEdge.getChannelType(), copyEdge.getChannelType()); - assertEquals(origEdge.getSourceIndex(), copyEdge.getSourceIndex()); - assertEquals(origEdge.getTargetIndex(), copyEdge.getTargetIndex()); - } - - /** - * Constructs a sample management graph that is used during the unit tests. - * - * @return the sample management graph used during the tests - */ - private static ManagementGraph constructTestManagementGraph() { - - /** - * This is the structure of the constructed test graph. The graph - * contains two stages and all three channel types. - * 4 - * | In-memory - * 3 - * --/ \-- Network (was FILE) - * 2 2 - * \ / Network - * 1 - */ - - // Graph - final ManagementGraph graph = new ManagementGraph(new JobID()); - - // Stages - final ManagementStage lowerStage = new ManagementStage(graph, 0); - final ManagementStage upperStage = new ManagementStage(graph, 1); - - // Group vertices - final ManagementGroupVertex groupVertex1 = new ManagementGroupVertex(lowerStage, "Group Vertex 1"); - final ManagementGroupVertex groupVertex2 = new ManagementGroupVertex(lowerStage, "Group Vertex 2"); - final ManagementGroupVertex groupVertex3 = new ManagementGroupVertex(upperStage, "Group Vertex 3"); - final ManagementGroupVertex groupVertex4 = new ManagementGroupVertex(upperStage, "Group Vertex 4"); - - // Vertices - final ManagementVertex vertex1_1 = new ManagementVertex(groupVertex1, new ManagementVertexID(), "Host 1", - 0); - final ManagementVertex vertex2_1 = new ManagementVertex(groupVertex2, new ManagementVertexID(), "Host 2", - 0); - final ManagementVertex vertex2_2 = new ManagementVertex(groupVertex2, new ManagementVertexID(), "Host 2", - 1); - final ManagementVertex vertex3_1 = new ManagementVertex(groupVertex3, new ManagementVertexID(), "Host 2", - 0); - final ManagementVertex vertex4_1 = new ManagementVertex(groupVertex4, new ManagementVertexID(), "Host 2", - 0); - - // Input/output gates - final ManagementGate outputGate1_1 = new ManagementGate(vertex1_1, new ManagementGateID(), 0, false); - - final ManagementGate inputGate2_1 = new ManagementGate(vertex2_1, new ManagementGateID(), 0, true); - final ManagementGate outputGate2_1 = new ManagementGate(vertex2_1, new ManagementGateID(), 0, false); - - final ManagementGate inputGate2_2 = new ManagementGate(vertex2_2, new ManagementGateID(), 0, true); - final ManagementGate outputGate2_2 = new ManagementGate(vertex2_2, new ManagementGateID(), 0, false); - - final ManagementGate inputGate3_1 = new ManagementGate(vertex3_1, new ManagementGateID(), 0, true); - final ManagementGate outputGate3_1 = new ManagementGate(vertex3_1, new ManagementGateID(), 0, false); - - final ManagementGate inputGate4_1 = new ManagementGate(vertex4_1, new ManagementGateID(), 0, true); - - // Group Edges - new ManagementGroupEdge(groupVertex1, 0, groupVertex2, 0, ChannelType.NETWORK); - new ManagementGroupEdge(groupVertex2, 0, groupVertex3, 0, ChannelType.NETWORK); - new ManagementGroupEdge(groupVertex3, 0, groupVertex4, 0, ChannelType.IN_MEMORY); - - // Edges - new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate1_1, 0, inputGate2_1, 0, - ChannelType.NETWORK); - new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate1_1, 1, inputGate2_2, 0, - ChannelType.NETWORK); - new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate2_1, 0, inputGate3_1, 0, - ChannelType.NETWORK); - new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate2_2, 0, inputGate3_1, 1, - ChannelType.NETWORK); - new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate3_1, 0, inputGate4_1, 0, - ChannelType.IN_MEMORY); - - return graph; - } - - /** - * This test checks the correctness of the {@link ManagementGraphIterator}. In particular it checks whether the - * vertices are visited in the correct order (depth first). - */ - @Test - public void testManagementGraphIterator() { - - final ManagementGraph testGraph = constructTestManagementGraph(); - - // Forward traversal - Iterator it = new ManagementGraphIterator(testGraph, true); - - ManagementVertex[] expectedOrder = new ManagementVertex[5]; - expectedOrder[0] = testGraph.getStage(0).getGroupVertex(0).getGroupMember(0); - expectedOrder[1] = testGraph.getStage(0).getGroupVertex(1).getGroupMember(0); - expectedOrder[2] = testGraph.getStage(1).getGroupVertex(0).getGroupMember(0); - expectedOrder[3] = testGraph.getStage(1).getGroupVertex(1).getGroupMember(0); - expectedOrder[4] = testGraph.getStage(0).getGroupVertex(1).getGroupMember(1); - - checkManagementVertexOrder(it, expectedOrder); - - it = new ManagementGraphIterator(testGraph, 0, false, true); - - checkManagementVertexOrder(it, expectedOrder); - - it = new ManagementGraphIterator(testGraph, expectedOrder[0], true); - - checkManagementVertexOrder(it, expectedOrder); - - expectedOrder = new ManagementVertex[3]; - expectedOrder[0] = testGraph.getStage(0).getGroupVertex(0).getGroupMember(0); - expectedOrder[1] = testGraph.getStage(0).getGroupVertex(1).getGroupMember(0); - expectedOrder[2] = testGraph.getStage(0).getGroupVertex(1).getGroupMember(1); - - it = new ManagementGraphIterator(testGraph, 0, true, true); - - checkManagementVertexOrder(it, expectedOrder); - - // Backward traversal - it = new ManagementGraphIterator(testGraph, false); - - expectedOrder = new ManagementVertex[5]; - expectedOrder[0] = testGraph.getStage(1).getGroupVertex(1).getGroupMember(0); - expectedOrder[1] = testGraph.getStage(1).getGroupVertex(0).getGroupMember(0); - expectedOrder[2] = testGraph.getStage(0).getGroupVertex(1).getGroupMember(0); - expectedOrder[3] = testGraph.getStage(0).getGroupVertex(0).getGroupMember(0); - expectedOrder[4] = testGraph.getStage(0).getGroupVertex(1).getGroupMember(1); - - checkManagementVertexOrder(it, expectedOrder); - - it = new ManagementGraphIterator(testGraph, 1, true, false); - - expectedOrder = new ManagementVertex[2]; - expectedOrder[0] = testGraph.getStage(1).getGroupVertex(1).getGroupMember(0); - expectedOrder[1] = testGraph.getStage(1).getGroupVertex(0).getGroupMember(0); - - checkManagementVertexOrder(it, expectedOrder); - } - - /** - * This test checks the correctness of the {@link ManagementGroupVertexIterator}. In particular it checks whether - * the - * vertices are visited in the correct order (depth first). - */ - @Test - public void testManagementGroupVertexIterator() { - - final ManagementGraph testGraph = constructTestManagementGraph(); - - Iterator it = new ManagementGroupVertexIterator(testGraph, true, -1); - ManagementGroupVertex[] expectedOrder = new ManagementGroupVertex[4]; - expectedOrder[0] = testGraph.getStage(0).getGroupVertex(0); - expectedOrder[1] = testGraph.getStage(0).getGroupVertex(1); - expectedOrder[2] = testGraph.getStage(1).getGroupVertex(0); - expectedOrder[3] = testGraph.getStage(1).getGroupVertex(1); - - checkManagementGroupVertexOrder(it, expectedOrder); - - it = new ManagementGroupVertexIterator(testGraph, false, -1); - expectedOrder[0] = testGraph.getStage(1).getGroupVertex(1); - expectedOrder[1] = testGraph.getStage(1).getGroupVertex(0); - expectedOrder[2] = testGraph.getStage(0).getGroupVertex(1); - expectedOrder[3] = testGraph.getStage(0).getGroupVertex(0); - - checkManagementGroupVertexOrder(it, expectedOrder); - - it = new ManagementGroupVertexIterator(testGraph, true, 0); - expectedOrder = new ManagementGroupVertex[2]; - expectedOrder[0] = testGraph.getStage(0).getGroupVertex(0); - expectedOrder[1] = testGraph.getStage(0).getGroupVertex(1); - - checkManagementGroupVertexOrder(it, expectedOrder); - - it = new ManagementGroupVertexIterator(testGraph, false, 1); - expectedOrder[0] = testGraph.getStage(1).getGroupVertex(1); - expectedOrder[1] = testGraph.getStage(1).getGroupVertex(0); - - checkManagementGroupVertexOrder(it, expectedOrder); - } - - /** - * Auxiliary method to check if the order in which the management vertices are visited by the given iterator - * corresponds to the one in the given array. - * - * @param it - * the iterator to be used to traverse the vertices - * @param expectedOrder - * array with the expected order of vertices - */ - private void checkManagementVertexOrder(final Iterator it, final ManagementVertex[] expectedOrder) { - - int i = 0; - while (it.hasNext()) { - - final ManagementVertex vertex = it.next(); - assertEquals(expectedOrder[i++], vertex); - } - - assertEquals(expectedOrder.length, i); - } - - /** - * Auxiliary method to check if the order in which the management group vertices are visited by the given iterator - * corresponds to the one in the given array. - * - * @param it - * the iterator to be used to traverse the vertices - * @param expectedOrder - * array with the expected order of vertices - */ - private void checkManagementGroupVertexOrder(final Iterator it, - final ManagementGroupVertex[] expectedOrder) { - - int i = 0; - while (it.hasNext()) { - - final ManagementGroupVertex groupVertex = it.next(); - assertEquals(expectedOrder[i++], groupVertex); - } - - assertEquals(expectedOrder.length, i); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/types/ProfilingTypesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/types/ProfilingTypesTest.java index f933dec8371..051e171857c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/types/ProfilingTypesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/types/ProfilingTypesTest.java @@ -16,25 +16,24 @@ * limitations under the License. */ - package org.apache.flink.runtime.profiling.types; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.managementgraph.ManagementVertexID; -import org.apache.flink.runtime.profiling.types.InputGateProfilingEvent; +import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.profiling.types.InstanceSummaryProfilingEvent; -import org.apache.flink.runtime.profiling.types.OutputGateProfilingEvent; import org.apache.flink.runtime.profiling.types.SingleInstanceProfilingEvent; import org.apache.flink.runtime.profiling.types.ThreadProfilingEvent; +import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.ManagementTestUtils; import org.junit.Test; /** * This test checks the proper serialization and deserialization of profiling events. - * */ public class ProfilingTypesTest { @@ -72,12 +71,6 @@ public class ProfilingTypesTest { private static final String INSTANCE_NAME = "Test Instance"; - private static final int GATE_INDEX = 7; - - private static final int NO_RECORD_AVAILABLE_COUNTER = 999; - - private static final int CHANNEL_CAPACITY_EXHAUSTED = 998; - private static final int USER_TIME = 17; private static final int SYSTEM_TIME = 18; @@ -125,79 +118,40 @@ public class ProfilingTypesTest { */ @Test public void testSingleInstanceProfilingEvent() { + try { + final SingleInstanceProfilingEvent orig = new SingleInstanceProfilingEvent(PROFILING_INTERVAL, IOWAIT_CPU, + IDLE_CPU, USER_CPU, SYSTEM_CPU, HARD_IRQ_CPU, SOFT_IRQ_CPU, TOTAL_MEMORY, FREE_MEMORY, BUFFERED_MEMORY, + CACHED_MEMORY, CACHED_SWAP_MEMORY, RECEIVED_BYTES, TRANSMITTED_BYTES, new JobID(), TIMESTAMP, + PROFILING_TIMESTAMP, INSTANCE_NAME); + + final SingleInstanceProfilingEvent copy = (SingleInstanceProfilingEvent) CommonTestUtils.createCopyWritable(orig); + + assertEquals(orig.getProfilingInterval(), copy.getProfilingInterval()); + assertEquals(orig.getIOWaitCPU(), copy.getIOWaitCPU()); + assertEquals(orig.getIdleCPU(), copy.getIdleCPU()); + assertEquals(orig.getUserCPU(), copy.getUserCPU()); + assertEquals(orig.getSystemCPU(), copy.getSystemCPU()); + assertEquals(orig.getHardIrqCPU(), copy.getHardIrqCPU()); + assertEquals(orig.getSoftIrqCPU(), copy.getSoftIrqCPU()); + assertEquals(orig.getTotalMemory(), copy.getTotalMemory()); + assertEquals(orig.getFreeMemory(), copy.getFreeMemory()); + assertEquals(orig.getBufferedMemory(), copy.getBufferedMemory()); + assertEquals(orig.getCachedMemory(), copy.getCachedMemory()); + assertEquals(orig.getCachedSwapMemory(), copy.getCachedSwapMemory()); + assertEquals(orig.getReceivedBytes(), copy.getReceivedBytes()); + assertEquals(orig.getTransmittedBytes(), copy.getTransmittedBytes()); + assertEquals(orig.getJobID(), copy.getJobID()); + assertEquals(orig.getTimestamp(), copy.getTimestamp()); + assertEquals(orig.getProfilingTimestamp(), copy.getProfilingTimestamp()); + assertEquals(orig.getInstanceName(), copy.getInstanceName()); + assertEquals(orig.hashCode(), copy.hashCode()); + assertTrue(orig.equals(copy)); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } - final SingleInstanceProfilingEvent orig = new SingleInstanceProfilingEvent(PROFILING_INTERVAL, IOWAIT_CPU, - IDLE_CPU, USER_CPU, SYSTEM_CPU, HARD_IRQ_CPU, SOFT_IRQ_CPU, TOTAL_MEMORY, FREE_MEMORY, BUFFERED_MEMORY, - CACHED_MEMORY, CACHED_SWAP_MEMORY, RECEIVED_BYTES, TRANSMITTED_BYTES, new JobID(), TIMESTAMP, - PROFILING_TIMESTAMP, INSTANCE_NAME); - - final SingleInstanceProfilingEvent copy = (SingleInstanceProfilingEvent) ManagementTestUtils.createCopy(orig); - - assertEquals(orig.getProfilingInterval(), copy.getProfilingInterval()); - assertEquals(orig.getIOWaitCPU(), copy.getIOWaitCPU()); - assertEquals(orig.getIdleCPU(), copy.getIdleCPU()); - assertEquals(orig.getUserCPU(), copy.getUserCPU()); - assertEquals(orig.getSystemCPU(), copy.getSystemCPU()); - assertEquals(orig.getHardIrqCPU(), copy.getHardIrqCPU()); - assertEquals(orig.getSoftIrqCPU(), copy.getSoftIrqCPU()); - assertEquals(orig.getTotalMemory(), copy.getTotalMemory()); - assertEquals(orig.getFreeMemory(), copy.getFreeMemory()); - assertEquals(orig.getBufferedMemory(), copy.getBufferedMemory()); - assertEquals(orig.getCachedMemory(), copy.getCachedMemory()); - assertEquals(orig.getCachedSwapMemory(), copy.getCachedSwapMemory()); - assertEquals(orig.getReceivedBytes(), copy.getReceivedBytes()); - assertEquals(orig.getTransmittedBytes(), copy.getTransmittedBytes()); - assertEquals(orig.getJobID(), copy.getJobID()); - assertEquals(orig.getTimestamp(), copy.getTimestamp()); - assertEquals(orig.getProfilingTimestamp(), copy.getProfilingTimestamp()); - assertEquals(orig.getInstanceName(), copy.getInstanceName()); - assertEquals(orig.hashCode(), copy.hashCode()); - assertTrue(orig.equals(copy)); - - } - - /** - * Tests serialization/deserialization for {@link InputGateProfilingEvent}. - */ - @Test - public void testInputGateProfilingEvent() { - - final InputGateProfilingEvent orig = new InputGateProfilingEvent(GATE_INDEX, NO_RECORD_AVAILABLE_COUNTER, - new ManagementVertexID(), PROFILING_INTERVAL, new JobID(), TIMESTAMP, PROFILING_TIMESTAMP); - - final InputGateProfilingEvent copy = (InputGateProfilingEvent) ManagementTestUtils.createCopy(orig); - - assertEquals(orig.getGateIndex(), copy.getGateIndex()); - assertEquals(orig.getNoRecordsAvailableCounter(), copy.getNoRecordsAvailableCounter()); - assertEquals(orig.getVertexID(), copy.getVertexID()); - assertEquals(orig.getProfilingInterval(), copy.getProfilingInterval()); - assertEquals(orig.getJobID(), copy.getJobID()); - assertEquals(orig.getTimestamp(), copy.getTimestamp()); - assertEquals(orig.getProfilingTimestamp(), copy.getProfilingTimestamp()); - assertEquals(orig.hashCode(), copy.hashCode()); - assertTrue(orig.equals(copy)); - } - - /** - * Tests serialization/deserialization for {@link OutputGateProfilingEvent}. - */ - @Test - public void testOutputGateProfilingEvent() { - - final OutputGateProfilingEvent orig = new OutputGateProfilingEvent(GATE_INDEX, CHANNEL_CAPACITY_EXHAUSTED, - new ManagementVertexID(), PROFILING_INTERVAL, new JobID(), TIMESTAMP, PROFILING_TIMESTAMP); - - final OutputGateProfilingEvent copy = (OutputGateProfilingEvent) ManagementTestUtils.createCopy(orig); - - assertEquals(orig.getGateIndex(), copy.getGateIndex()); - assertEquals(orig.getChannelCapacityExhausted(), copy.getChannelCapacityExhausted()); - assertEquals(orig.getVertexID(), copy.getVertexID()); - assertEquals(orig.getProfilingInterval(), copy.getProfilingInterval()); - assertEquals(orig.getJobID(), copy.getJobID()); - assertEquals(orig.getTimestamp(), copy.getTimestamp()); - assertEquals(orig.getProfilingTimestamp(), copy.getProfilingTimestamp()); - assertEquals(orig.hashCode(), copy.hashCode()); - assertTrue(orig.equals(copy)); } /** @@ -207,7 +161,7 @@ public class ProfilingTypesTest { public void testThreadProfilingEvent() { final ThreadProfilingEvent orig = new ThreadProfilingEvent(USER_TIME, SYSTEM_TIME, BLOCKED_TIME, WAITED_TIME, - new ManagementVertexID(), PROFILING_INTERVAL, new JobID(), TIMESTAMP, PROFILING_TIMESTAMP); + new JobVertexID(), 17, new ExecutionAttemptID(), PROFILING_INTERVAL, new JobID(), TIMESTAMP, PROFILING_TIMESTAMP); final ThreadProfilingEvent copy = (ThreadProfilingEvent) ManagementTestUtils.createCopy(orig); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index e7639e35bf1..3eb4a610484 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -48,18 +48,13 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.types.IntegerRecord; -import org.apache.flink.util.LogUtils; -import org.junit.BeforeClass; + import org.junit.Test; + import org.mockito.Matchers; public class TaskManagerTest { - - @BeforeClass - public static void reduceLogLevel() { - LogUtils.initializeDefaultTestConsoleLogger(); - } @Test public void testSetupTaskManager() { -- GitLab