Skip to content

Commit

Permalink
[Part 10]: Task Executor Starter and other related classes (#160)
Browse files Browse the repository at this point in the history
* [Part 10]: Task Executor Starter and other related classes

TaskExecutorStarter is the starting point for the task executor from the worker side. Initializing this class and starting it as part of the runtime framework (guice/springboot) would start the task executor thread capable of running any task from the client side.

Co-authored-by: Sundaram Ananthanarayanan <[email protected]>
  • Loading branch information
sundargates and sundargates authored Apr 6, 2022
1 parent 45d804d commit ecf0d4e
Show file tree
Hide file tree
Showing 8 changed files with 368 additions and 19 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ dependencies {

api "org.skife.config:config-magic:$configMagicVersion"
api libraries.flinkRpcApi
api libraries.flinkRpcImpl

api "org.apache.mesos:mesos:$mesosVersion"
api "org.json:json:$jsonVersion"
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* Copyright 2022 Netflix, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package io.mantisrx.server.core;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.rpc.RpcSystem;
import org.apache.flink.runtime.rpc.RpcSystemLoader;
import org.apache.flink.runtime.rpc.akka.AkkaRpcSystem;

/**
* RpcSystemLoader for mantis task executor and other services that need to expose an RPC API.
* This particular implementation uses the akka RPC implementation under the hood.
*/
public class MantisAkkaRpcSystemLoader implements RpcSystemLoader {

private static final RpcSystem INSTANCE = new AkkaRpcSystem();
@Override
public RpcSystem loadRpcSystem(Configuration config) {
return INSTANCE;
}

public static RpcSystem getInstance() {
return INSTANCE;
}
}
20 changes: 10 additions & 10 deletions mantis-control-plane/mantis-control-plane-server/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -17,34 +17,34 @@
apply plugin: 'application'

ext {
akkaVersion = '2.5.23'
akkaHttpVersion = '10.1.8'
akkaVersion = '2.6.15'
akkaHttpVersion = '10.2.7'
fenzoVersion = '0.13.8'
cliParserVersion = '1.1.1'
configMagicVersion = '0.11'
rxJavaReactiveStreamsVersion = '1.+'
curatorVersion = '2.11.0'
testngVersion = '6.+'
scalaBinaryVersion = '2.12'
}

dependencies {
compile project(':mantis-control-plane:mantis-control-plane-core')

api "com.typesafe.akka:akka-http_2.13:$akkaHttpVersion"
api "com.typesafe.akka:akka-http-jackson_2.13:$akkaHttpVersion"
api "com.typesafe.akka:akka-http-caching_2.13:$akkaHttpVersion"
api "com.typesafe.akka:akka-stream_2.13:$akkaVersion"
compile group: 'com.typesafe.akka', name: 'akka-stream_2.13', version: '2.5.23'
api "com.typesafe.akka:akka-slf4j_2.13:$akkaVersion"
api "com.typesafe.akka:akka-http_$scalaBinaryVersion:$akkaHttpVersion"
api "com.typesafe.akka:akka-http-jackson_$scalaBinaryVersion:$akkaHttpVersion"
api "com.typesafe.akka:akka-http-caching_$scalaBinaryVersion:$akkaHttpVersion"
api "com.typesafe.akka:akka-stream_$scalaBinaryVersion:$akkaVersion"
api "com.typesafe.akka:akka-slf4j_$scalaBinaryVersion:$akkaVersion"
api "com.typesafe.akka:akka-actor_$scalaBinaryVersion:$akkaVersion"

api "com.typesafe.akka:akka-actor_2.13:$akkaVersion"
api "io.reactivex:rxjava-reactive-streams:$rxJavaReactiveStreamsVersion"
api "com.netflix.fenzo:fenzo-core:$fenzoVersion"
api "com.netflix.fenzo:fenzo-triggers:$fenzoVersion"
api "com.github.spullara.cli-parser:cli-parser:$cliParserVersion"
api "org.skife.config:config-magic:$configMagicVersion"

testCompile "com.typesafe.akka:akka-testkit_2.13:$akkaVersion"
testCompile "com.typesafe.akka:akka-testkit_$scalaBinaryVersion:$akkaVersion"
testCompile libraries.junit4
testCompile libraries.mockitoAll
testCompile "org.testng:testng:$testngVersion"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,16 @@ default BlobStore withZipCapabilities() {
return new ZipHandlingBlobStore(this);
}

static BlobStore forHadoopFileSystem(URI clusterStoragePath, File localStoreDir) throws Exception {
final org.apache.hadoop.fs.FileSystem fileSystem =
FileSystemInitializer.create(clusterStoragePath);

return
new HadoopFileSystemBlobStore(fileSystem, localStoreDir)
.withPrefix(clusterStoragePath)
.withZipCapabilities();
}

@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
class PrefixedBlobStore implements BlobStore {
private final URI rootUri;
Expand Down Expand Up @@ -109,6 +119,4 @@ private ZipFile getZipFile(File file) {
}
}
}


}
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* Copyright 2022 Netflix, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.mantisrx.server.worker;

import java.io.IOException;
import java.net.URI;
import org.apache.hadoop.fs.FileSystem;

public interface FileSystemFactory {
/** Gets the scheme of the file system created by this factory. */
String getScheme();

/**
* Creates a new file system for the given file system URI. The URI describes the type of file
* system (via its scheme) and optionally the authority (for example the host) of the file
* system.
*
* @param fsUri The URI that describes the file system.
* @return A new instance of the specified file system.
* @throws IOException Thrown if the file system could not be instantiated.
*/
FileSystem create(URI fsUri) throws IOException;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* Copyright 2022 Netflix, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.mantisrx.server.worker;

import java.io.IOException;
import java.net.URI;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.ServiceLoader;
import lombok.extern.slf4j.Slf4j;
import org.apache.hadoop.fs.FileSystem;

@Slf4j
public class FileSystemInitializer {

private static final Map<String, FileSystemFactory> FS_FACTORIES = new HashMap<>();

static {
FileSystemInitializer.initialize();
}

/**
* initialize needs to be called at the start of the JVM.
*/
public static void initialize() {
FS_FACTORIES.clear();

// let's get all the registered implementations
Iterator<FileSystemFactory> fileSystemFactoryIterator =
ServiceLoader.load(FileSystemFactory.class).iterator();
fileSystemFactoryIterator.forEachRemaining(fileSystemFactory -> {
log.info("Initializing FileSystem Factory {}", fileSystemFactory);
FS_FACTORIES.putIfAbsent(fileSystemFactory.getScheme(), fileSystemFactory);
});
}

public static FileSystem create(URI fsUri) throws IOException {
FileSystemFactory factory = FS_FACTORIES.get(fsUri.getScheme());
if (factory != null) {
return factory.create(fsUri);
} else {
throw new IllegalArgumentException(
String.format("Unknown schema %s", fsUri.getScheme()));
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,173 @@
/*
* Copyright 2022 Netflix, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package io.mantisrx.server.worker;

import com.mantisrx.common.utils.Services;
import io.mantisrx.server.core.MantisAkkaRpcSystemLoader;
import io.mantisrx.server.master.client.HighAvailabilityServices;
import io.mantisrx.server.master.client.HighAvailabilityServicesUtil;
import io.mantisrx.server.worker.config.WorkerConfiguration;
import io.mantisrx.shaded.com.google.common.base.Preconditions;
import io.mantisrx.shaded.com.google.common.util.concurrent.AbstractIdleService;
import io.mantisrx.shaded.com.google.common.util.concurrent.MoreExecutors;
import java.time.Clock;
import java.util.Optional;
import javax.annotation.Nullable;
import lombok.AccessLevel;
import lombok.RequiredArgsConstructor;
import lombok.extern.slf4j.Slf4j;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.runtime.rpc.RpcSystem;
import org.apache.flink.runtime.rpc.RpcUtils;

/**
* TaskExecutorStarter class represents the starting point for a task executor.
* Use the {@link TaskExecutorStarterBuilder} to build {@link TaskExecutorStarter}.
* Once the service is build, start and stop it during the lifecycle of your runtime framework such as spring.
*/
@RequiredArgsConstructor(access = AccessLevel.PRIVATE)
@Slf4j
public class TaskExecutorStarter extends AbstractIdleService {

private final TaskExecutor taskExecutor;
private final HighAvailabilityServices highAvailabilityServices;

@Override
protected void startUp() {
highAvailabilityServices.startAsync().awaitRunning();

taskExecutor.start();
}

@Override
protected void shutDown() throws Exception {
taskExecutor
.closeAsync()
.exceptionally(throwable -> null)
.thenCompose(dontCare -> Services.stopAsync(highAvailabilityServices,
MoreExecutors.directExecutor()))
.get();
}

public static TaskExecutorStarterBuilder builder(WorkerConfiguration workerConfiguration) {
return new TaskExecutorStarterBuilder(workerConfiguration);
}

@SuppressWarnings("unused")
public static class TaskExecutorStarterBuilder {
private final WorkerConfiguration workerConfiguration;
private Configuration configuration;
@Nullable
private RpcSystem rpcSystem;
@Nullable
private RpcService rpcService;
@Nullable
private ClassLoaderHandle classLoaderHandle;
private final HighAvailabilityServices highAvailabilityServices;
@Nullable
private SinkSubscriptionStateHandler.Factory sinkSubscriptionHandlerFactory;

private TaskExecutorStarterBuilder(WorkerConfiguration workerConfiguration) {
this.workerConfiguration = workerConfiguration;
this.configuration = new Configuration();
this.highAvailabilityServices = HighAvailabilityServicesUtil.createHAServices(workerConfiguration);
}

public TaskExecutorStarterBuilder configuration(Configuration configuration) {
this.configuration = configuration;
return this;
}

public TaskExecutorStarterBuilder rpcSystem(RpcSystem rpcSystem) {
Preconditions.checkNotNull(rpcSystem);
this.rpcSystem = rpcSystem;
return this;
}

private RpcSystem getRpcSystem() {
if (this.rpcSystem == null) {
return MantisAkkaRpcSystemLoader.getInstance();
} else {
return this.rpcSystem;
}
}

public TaskExecutorStarterBuilder rpcService(RpcService rpcService) {
Preconditions.checkNotNull(rpcService);
this.rpcService = rpcService;
return this;
}

private RpcService getRpcService() throws Exception {
if (this.rpcService == null) {
return RpcUtils.createRemoteRpcService(
getRpcSystem(),
configuration,
workerConfiguration.getExternalAddress(),
workerConfiguration.getExternalPortRange(),
workerConfiguration.getBindAddress(),
Optional.ofNullable(workerConfiguration.getBindPort()));
} else {
return this.rpcService;
}
}

public TaskExecutorStarterBuilder classLoaderHandle(ClassLoaderHandle classLoaderHandle) {
this.classLoaderHandle = classLoaderHandle;
return this;
}

private ClassLoaderHandle getClassLoaderHandle() throws Exception {
if (this.classLoaderHandle == null) {
return new BlobStoreAwareClassLoaderHandle(
BlobStore.forHadoopFileSystem(
workerConfiguration.getBlobStoreArtifactDir(),
workerConfiguration.getLocalStorageDir()));
} else {
return this.classLoaderHandle;
}
}

public TaskExecutorStarterBuilder sinkSubscriptionHandlerFactory(SinkSubscriptionStateHandler.Factory sinkSubscriptionHandlerFactory) {
this.sinkSubscriptionHandlerFactory = sinkSubscriptionHandlerFactory;
return this;
}

private SinkSubscriptionStateHandler.Factory getSinkSubscriptionHandlerFactory() {
if (this.sinkSubscriptionHandlerFactory == null) {
return SinkSubscriptionStateHandler.Factory.forEphemeralJobsThatNeedToBeKilledInAbsenceOfSubscriber(
highAvailabilityServices.getMasterClientApi(),
Clock.systemDefaultZone());
} else {
return this.sinkSubscriptionHandlerFactory;
}
}

public TaskExecutorStarter build() throws Exception {
final TaskExecutor taskExecutor =
new TaskExecutor(
getRpcService(),
workerConfiguration,
highAvailabilityServices,
getClassLoaderHandle(),
getSinkSubscriptionHandlerFactory());

return new TaskExecutorStarter(taskExecutor, highAvailabilityServices);
}
}
}
Loading

0 comments on commit ecf0d4e

Please sign in to comment.