forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-11417] Make access to ExecutionGraph single threaded from JobM…
…aster main thread This closes apache#7568.
- Loading branch information
1 parent
2296299
commit 85bae44
Showing
55 changed files
with
1,893 additions
and
1,403 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
83 changes: 83 additions & 0 deletions
83
...untime/src/main/java/org/apache/flink/runtime/concurrent/ComponentMainThreadExecutor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,83 @@ | ||
/* | ||
* 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.concurrent; | ||
|
||
import javax.annotation.Nonnull; | ||
|
||
import java.util.concurrent.Callable; | ||
import java.util.concurrent.ScheduledFuture; | ||
import java.util.concurrent.TimeUnit; | ||
|
||
/** | ||
* Interface for an executor that runs tasks in the main thread of an {@link org.apache.flink.runtime.rpc.RpcEndpoint}. | ||
*/ | ||
public interface ComponentMainThreadExecutor extends ScheduledExecutor { | ||
|
||
/** | ||
* Returns true if the method was called in the thread of this executor. | ||
*/ | ||
void assertRunningInMainThread(); | ||
|
||
/** | ||
* Dummy implementation of ComponentMainThreadExecutor. | ||
*/ | ||
final class DummyComponentMainThreadExecutor implements ComponentMainThreadExecutor { | ||
|
||
/** Customized message for the exception that is thrown on method invocation. */ | ||
private final String exceptionMessageOnInvocation; | ||
|
||
public DummyComponentMainThreadExecutor(String exceptionMessageOnInvocation) { | ||
this.exceptionMessageOnInvocation = exceptionMessageOnInvocation; | ||
} | ||
|
||
@Override | ||
public void assertRunningInMainThread() { | ||
throw createException(); | ||
} | ||
|
||
@Override | ||
public ScheduledFuture<?> schedule(Runnable command, long delay, TimeUnit unit) { | ||
throw createException(); | ||
} | ||
|
||
@Override | ||
public <V> ScheduledFuture<V> schedule(Callable<V> callable, long delay, TimeUnit unit) { | ||
throw createException(); | ||
} | ||
|
||
@Override | ||
public ScheduledFuture<?> scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit) { | ||
throw createException(); | ||
} | ||
|
||
@Override | ||
public ScheduledFuture<?> scheduleWithFixedDelay(Runnable command, long initialDelay, long delay, TimeUnit unit) { | ||
throw createException(); | ||
} | ||
|
||
@Override | ||
public void execute(@Nonnull Runnable command) { | ||
throw createException(); | ||
} | ||
|
||
private UnsupportedOperationException createException() { | ||
return new UnsupportedOperationException(exceptionMessageOnInvocation); | ||
} | ||
} | ||
} |
48 changes: 48 additions & 0 deletions
48
...n/java/org/apache/flink/runtime/concurrent/ComponentMainThreadExecutorServiceAdapter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,48 @@ | ||
/* | ||
* 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.concurrent; | ||
|
||
import javax.annotation.Nonnull; | ||
|
||
import java.util.concurrent.ScheduledExecutorService; | ||
|
||
/** | ||
* Adapter class for a {@link ScheduledExecutorService} which shall be used as a | ||
* {@link ComponentMainThreadExecutor}. It enhances the given executor with an assert that the current thread is the | ||
* main thread of the executor. | ||
*/ | ||
public class ComponentMainThreadExecutorServiceAdapter | ||
extends ScheduledExecutorServiceAdapter implements ComponentMainThreadExecutor { | ||
|
||
/** A runnable that should assert that the current thread is the expected main thread. */ | ||
@Nonnull | ||
private final Runnable mainThreadCheck; | ||
|
||
public ComponentMainThreadExecutorServiceAdapter( | ||
@Nonnull ScheduledExecutorService scheduledExecutorService, | ||
@Nonnull Runnable mainThreadCheck) { | ||
super(scheduledExecutorService); | ||
this.mainThreadCheck = mainThreadCheck; | ||
} | ||
|
||
@Override | ||
public void assertRunningInMainThread() { | ||
mainThreadCheck.run(); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.