Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
/*
* 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.util;

/**
* Sets a context class loader in a "try-with-resources" pattern.
*
* <pre>
* {@code
* try (AutoContextClassLoader ignored = AutoContextClassLoader.of(classloader)) {
* // code that needs the context class loader
* }
* }
* </pre>
*
* <p>This is conceptually the same as the code below.

* <pre>
* {@code
* ClassLoader original = Thread.currentThread().getContextClassLoader();
* Thread.currentThread().setContextClassLoader(classloader);
* try {
* // code that needs the context class loader
* } finally {
* Thread.currentThread().setContextClassLoader(original);
* }
* }
* </pre>
*/
public final class AutoContextClassLoader implements AutoCloseable {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

duplicate of TemporaryClassLoaderContext

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would consolidate that in a separate commit. If that gets used beyond plugins, it should reside in a more "universal" package. The AutoContextClassLoader is a tad bit nicer, especially javadoc wise.


/**
* Sets the context class loader to the given ClassLoader and returns a resource
* that sets it back to the current context ClassLoader when the resource is closed.
*
* <pre>{@code
* try (AutoContextClassLoader ignored = AutoContextClassLoader.of(classloader)) {
* // code that needs the context class loader
* }
* }</pre>
*/
public static AutoContextClassLoader of(ClassLoader cl) {
final Thread t = Thread.currentThread();
final ClassLoader original = t.getContextClassLoader();

t.setContextClassLoader(cl);

return new AutoContextClassLoader(t, original);
}

// ------------------------------------------------------------------------

private final Thread thread;

private final ClassLoader originalContextClassLoader;

private AutoContextClassLoader(Thread thread, ClassLoader originalContextClassLoader) {
this.thread = thread;
this.originalContextClassLoader = originalContextClassLoader;
}

@Override
public void close() {
thread.setContextClassLoader(originalContextClassLoader);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -38,16 +38,20 @@
import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway;
import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.flink.runtime.query.KvStateRegistry;
import org.apache.flink.runtime.query.TaskKvStateRegistry;
import org.apache.flink.runtime.state.TaskStateManager;
import org.apache.flink.runtime.taskexecutor.GlobalAggregateManager;
import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SerializedValue;

import java.util.Collections;
import java.util.Map;
Expand Down Expand Up @@ -218,6 +222,11 @@ public void declineCheckpoint(long checkpointId, Throwable cause) {
throw new UnsupportedOperationException(ERROR_MSG);
}

@Override
public TaskOperatorEventGateway getOperatorCoordinatorEventGateway() {
return new NoOpTaskOperatorEventGateway();
}

@Override
public void failExternally(Throwable cause) {
ExceptionUtils.rethrow(cause);
Expand Down Expand Up @@ -297,5 +306,14 @@ public SavepointEnvironment build() {
prioritizedOperatorSubtaskState);
}
}

// ------------------------------------------------------------------------
// mocks / stand-ins
// ------------------------------------------------------------------------

private static final class NoOpTaskOperatorEventGateway implements TaskOperatorEventGateway {
@Override
public void sendOperatorEventToCoordinator(OperatorID operator, SerializedValue<OperatorEvent> event) {}
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway;
import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
import org.apache.flink.runtime.query.TaskKvStateRegistry;
Expand Down Expand Up @@ -124,6 +125,11 @@ public interface Environment {
*/
InputSplitProvider getInputSplitProvider();

/**
* Gets the gateway through which operators can send events to the operator coordinators.
*/
TaskOperatorEventGateway getOperatorCoordinatorEventGateway();

/**
* Returns the current {@link IOManager}.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
import org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint;
import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
Expand All @@ -53,17 +54,21 @@
import org.apache.flink.runtime.jobmaster.SlotRequestId;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.messages.TaskBackPressureResponse;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.flink.runtime.operators.coordination.TaskNotRunningException;
import org.apache.flink.runtime.shuffle.NettyShuffleMaster;
import org.apache.flink.runtime.shuffle.PartitionDescriptor;
import org.apache.flink.runtime.shuffle.ProducerDescriptor;
import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
import org.apache.flink.runtime.shuffle.ShuffleMaster;
import org.apache.flink.runtime.taskexecutor.TaskExecutorOperatorEventGateway;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.OptionalFailure;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SerializedValue;
import org.apache.flink.util.function.ThrowingRunnable;

import org.slf4j.Logger;
Expand Down Expand Up @@ -1030,6 +1035,23 @@ private void triggerCheckpointHelper(long checkpointId, long timestamp, Checkpoi
}
}

/**
* Sends the operator event to the Task on the Task Executor.
*
* @return True, of the message was sent, false is the task is currently not running.
*/
public CompletableFuture<Acknowledge> sendOperatorEvent(OperatorID operatorId, SerializedValue<OperatorEvent> event) {
final LogicalSlot slot = assignedResource;

if (slot != null && getState() == RUNNING) {
final TaskExecutorOperatorEventGateway eventGateway = slot.getTaskManagerGateway();
return eventGateway.sendOperatorEventToTask(getAttemptId(), operatorId, event);
} else {
return FutureUtils.completedExceptionally(new TaskNotRunningException(
'"' + vertex.getTaskNameWithSubtaskIndex() + "\" is currently not running or ready."));
}
}

// --------------------------------------------------------------------------------------------
// Callbacks
// --------------------------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,8 @@
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinatorUtil;
import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
import org.apache.flink.types.Either;
import org.apache.flink.util.OptionalFailure;
Expand Down Expand Up @@ -134,6 +136,8 @@ public class ExecutionJobVertex implements AccessExecutionJobVertex, Archiveable
*/
private Either<SerializedValue<TaskInformation>, PermanentBlobKey> taskInformationOrBlobKey = null;

private final Map<OperatorID, OperatorCoordinator> operatorCoordinators;

private InputSplitAssigner splitAssigner;

/**
Expand Down Expand Up @@ -244,6 +248,16 @@ public ExecutionJobVertex(
}
}

try {
this.operatorCoordinators = OperatorCoordinatorUtil.instantiateCoordinators(
jobVertex.getOperatorCoordinators(),
graph.getUserClassLoader(),
(opId) -> new ExecutionJobVertexCoordinatorContext(opId, this));
}
catch (IOException | ClassNotFoundException e) {
throw new JobException("Cannot instantiate the coordinator for operator " + getName(), e);
}

// set up the input splits, if the vertex has any
try {
@SuppressWarnings("unchecked")
Expand Down Expand Up @@ -383,6 +397,15 @@ public InputDependencyConstraint getInputDependencyConstraint() {
return getJobVertex().getInputDependencyConstraint();
}

@Nullable
public OperatorCoordinator getOperatorCoordinator(OperatorID operatorId) {
return operatorCoordinators.get(operatorId);
}

public Collection<OperatorCoordinator> getOperatorCoordinators() {
return Collections.unmodifiableCollection(operatorCoordinators.values());
}

public Either<SerializedValue<TaskInformation>, PermanentBlobKey> getTaskInformationOrBlobKey() throws IOException {
// only one thread should offload the task information, so let's also let only one thread
// serialize the task information!
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.runtime.executiongraph;

import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.SerializedValue;

import java.io.IOException;
import java.util.concurrent.CompletableFuture;

/**
* An implementation of the {@link OperatorCoordinator.Context} that delegates call to an
* {@link ExecutionJobVertex}.
*/
final class ExecutionJobVertexCoordinatorContext implements OperatorCoordinator.Context {

private final OperatorID operatorId;

private final ExecutionJobVertex jobVertex;

ExecutionJobVertexCoordinatorContext(OperatorID operatorId, ExecutionJobVertex jobVertex) {
this.operatorId = operatorId;
this.jobVertex = jobVertex;
}

@Override
public OperatorID getOperatorId() {
return operatorId;
}

@Override
public CompletableFuture<Acknowledge> sendEvent(OperatorEvent evt, int targetSubtask) {
final SerializedValue<OperatorEvent> serializedEvent;
try {
serializedEvent = new SerializedValue<>(evt);
}
catch (IOException e) {
// we do not expect that this exception is handled by the caller, so we make it
// unchecked so that it can bubble up
throw new FlinkRuntimeException("Cannot serialize operator event", e);
}

return getTaskExecution(targetSubtask).sendOperatorEvent(operatorId, serializedEvent);
}

@Override
public void failTask(int subtask, Throwable cause) {
final Execution taskExecution = getTaskExecution(subtask);
taskExecution.fail(cause);
}

@Override
public void failJob(Throwable cause) {
jobVertex.getGraph().failGlobal(cause);
}

private Execution getTaskExecution(int subtask) {
return jobVertex.getTaskVertices()[subtask].getCurrentExecutionAttempt();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -650,6 +650,8 @@ private Execution resetForNewExecutionInternal(final long timestamp, final long
}
}

jobVertex.getOperatorCoordinators().forEach((c -> c.subtaskFailed(getParallelSubtaskIndex())));

CoLocationGroup grp = jobVertex.getCoLocationGroup();
if (grp != null) {
locationConstraint = grp.getLocationConstraint(subTaskIndex);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,11 +27,14 @@
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SerializedValue;

import javax.annotation.Nullable;

import java.util.ArrayList;
import java.util.Collections;
import java.util.List;

import static org.apache.flink.util.Preconditions.checkNotNull;
Expand Down Expand Up @@ -67,7 +70,10 @@ public class JobVertex implements java.io.Serializable {
/** List of edges with incoming data. One per Reader. */
private final ArrayList<JobEdge> inputs = new ArrayList<>();

/** Number of subtasks to split this task into at runtime. */
/** The list of factories for operator coordinators. */
private final ArrayList<SerializedValue<OperatorCoordinator.Provider>> operatorCoordinators = new ArrayList<>();

/** Number of subtasks to split this task into at runtime.*/
private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT;

/** Maximum number of subtasks to split this task into a runtime. */
Expand Down Expand Up @@ -359,6 +365,14 @@ public List<JobEdge> getInputs() {
return this.inputs;
}

public List<SerializedValue<OperatorCoordinator.Provider>> getOperatorCoordinators() {
return Collections.unmodifiableList(operatorCoordinators);
}

public void addOperatorCoordinator(SerializedValue<OperatorCoordinator.Provider> serializedCoordinatorProvider) {
operatorCoordinators.add(serializedCoordinatorProvider);
}

/**
* Associates this vertex with a slot sharing group for scheduling. Different vertices in the same
* slot sharing group can run one subtask each in the same slot.
Expand Down
Loading