From 38f302460a42311a28d9480624ae96fee06110c8 Mon Sep 17 00:00:00 2001 From: Matthew Ho Date: Mon, 26 Jun 2023 11:29:58 -0700 Subject: [PATCH 1/8] CI / CD for branch --- .github/workflows/build_and_test.yaml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/build_and_test.yaml b/.github/workflows/build_and_test.yaml index b70e18ab34..249ca1889d 100644 --- a/.github/workflows/build_and_test.yaml +++ b/.github/workflows/build_and_test.yaml @@ -19,12 +19,13 @@ name: Build and Run Tests on: push: - # Publish only on `master` branches: - master + - temporal-poc pull_request: branches: - master + - temporal-poc release: types: [published, edited] From eb1603de5927d1245206b44a56ad76714f18cb97 Mon Sep 17 00:00:00 2001 From: Matthew Ho Date: Mon, 26 Jun 2023 15:31:13 -0700 Subject: [PATCH 2/8] Add the temporal SDK as a dependency --- gradle/scripts/dependencyDefinitions.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/gradle/scripts/dependencyDefinitions.gradle b/gradle/scripts/dependencyDefinitions.gradle index 95f8f6c328..ffc81c7352 100644 --- a/gradle/scripts/dependencyDefinitions.gradle +++ b/gradle/scripts/dependencyDefinitions.gradle @@ -101,6 +101,7 @@ ext.externalDependency = [ "confluentJsonSerializer": "io.confluent:kafka-json-serializer:" + confluentVersion, "zkClient": "com.101tec:zkclient:0.7", "quartz": "org.quartz-scheduler:quartz:2.2.3", + "temporal-sdk": "io.temporal:temporal-sdk:1.20.0", "testng": "org.testng:testng:6.14.3", "junit": "junit:junit:4.13.2", "mockserver":"org.mock-server:mockserver-netty:3.10.4", From 6ff4fa0407d7adf2c031ad3fe30dfaa011240f52 Mon Sep 17 00:00:00 2001 From: Peiying Ye <112960226+Peiyingy@users.noreply.github.com> Date: Mon, 24 Jul 2023 14:49:41 -0700 Subject: [PATCH 3/8] Milestone 2: Temporal on Yarn * Add hello world temporal implementation * Add Gobblin temporal implementation * Add setSslContext * remove helix parts * clear up code --- gobblin-cluster/build.gradle | 1 + .../GobblinTemporalClusterManager.java | 539 +++++++++ .../cluster/GobblinTemporalTaskRunner.java | 379 +++++++ .../temporal/GobblinTemporalActivity.java | 30 + .../temporal/GobblinTemporalActivityImpl.java | 35 + .../temporal/GobblinTemporalWorkflow.java | 34 + .../temporal/GobblinTemporalWorkflowImpl.java | 72 ++ .../gobblin/cluster/temporal/Shared.java | 27 + .../security/ssl/SSLContextFactory.java | 2 +- gobblin-yarn/build.gradle | 2 +- .../GobblinTemporalApplicationMaster.java | 172 +++ .../yarn/GobblinTemporalYarnTaskRunner.java | 134 +++ .../gobblin/yarn/GobblinYarnAppLauncher.java | 8 +- .../YarnTemporalAppMasterSecurityManager.java | 49 + .../yarn/YarnTemporalAutoScalingManager.java | 328 ++++++ .../gobblin/yarn/YarnTemporalService.java | 1007 +++++++++++++++++ gradle/scripts/dependencyDefinitions.gradle | 3 +- 17 files changed, 2815 insertions(+), 7 deletions(-) create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java create mode 100644 gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java create mode 100644 gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalYarnTaskRunner.java create mode 100644 gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalAppMasterSecurityManager.java create mode 100644 gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalAutoScalingManager.java create mode 100644 gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java diff --git a/gobblin-cluster/build.gradle b/gobblin-cluster/build.gradle index 4b68445c12..f785eb8761 100644 --- a/gobblin-cluster/build.gradle +++ b/gobblin-cluster/build.gradle @@ -47,6 +47,7 @@ dependencies { compile externalDependency.hadoopCommon compile externalDependency.avroMapredH2 compile externalDependency.findBugsAnnotations + compile externalDependency."temporal-sdk" compile (externalDependency.helix) { exclude group: 'io.dropwizard.metrics', module: 'metrics-core' } diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java new file mode 100644 index 0000000000..4a910c09aa --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java @@ -0,0 +1,539 @@ +/* + * 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.gobblin.cluster; + +import java.io.File; +import java.io.IOException; +import java.security.KeyStore; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.DefaultParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.eventbus.EventBus; +import com.google.common.eventbus.Subscribe; +import com.google.common.util.concurrent.Service; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import io.grpc.netty.shaded.io.grpc.netty.GrpcSslContexts; +import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext; +import io.temporal.client.WorkflowClient; +import io.temporal.client.WorkflowClientOptions; +import io.temporal.client.WorkflowOptions; +import io.temporal.client.WorkflowStub; +import io.temporal.serviceclient.WorkflowServiceStubs; +import io.temporal.serviceclient.WorkflowServiceStubsOptions; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.TrustManagerFactory; +import lombok.AccessLevel; +import lombok.Getter; +import lombok.Setter; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.cluster.event.ClusterManagerShutdownRequest; +import org.apache.gobblin.cluster.temporal.GobblinTemporalWorkflow; +import org.apache.gobblin.cluster.temporal.Shared; +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.instrumented.StandardMetricsBridge; +import org.apache.gobblin.metrics.Tag; +import org.apache.gobblin.runtime.api.MutableJobCatalog; +import org.apache.gobblin.runtime.app.ApplicationException; +import org.apache.gobblin.runtime.app.ApplicationLauncher; +import org.apache.gobblin.runtime.app.ServiceBasedAppLauncher; +import org.apache.gobblin.scheduler.SchedulerService; +import org.apache.gobblin.util.ConfigUtils; +import org.apache.gobblin.util.JvmUtils; +import org.apache.gobblin.util.reflection.GobblinConstructorUtils; + +import static org.apache.gobblin.security.ssl.SSLContextFactory.toInputStream; + + +/** + * The central cluster manager for Gobblin Clusters. + * + * + *

+ * This class will initiates a graceful shutdown of the cluster in the following conditions: + * + *

+ *

+ * + * @author Yinan Li + */ +@Alpha +@Slf4j +public class GobblinTemporalClusterManager implements ApplicationLauncher, StandardMetricsBridge, LeadershipChangeAwareComponent { + + private static final Logger LOGGER = LoggerFactory.getLogger(GobblinTemporalClusterManager.class); + + private StopStatus stopStatus = new StopStatus(false); + + protected ServiceBasedAppLauncher applicationLauncher; + + // An EventBus used for communications between services running in the ApplicationMaster + @Getter(AccessLevel.PUBLIC) + protected final EventBus eventBus = new EventBus(GobblinTemporalClusterManager.class.getSimpleName()); + + protected final Path appWorkDir; + + @Getter + protected final FileSystem fs; + + protected final String applicationId; + + // thread used to keep process up for an idle controller + private Thread idleProcessThread; + + // set to true to stop the idle process thread + private volatile boolean stopIdleProcessThread = false; + + private final boolean isStandaloneMode; + + @Getter + private MutableJobCatalog jobCatalog; + @Getter + private JobConfigurationManager jobConfigurationManager; + @Getter + private volatile boolean started = false; + + protected final String clusterName; + @Getter + protected final Config config; + + public GobblinTemporalClusterManager(String clusterName, String applicationId, Config sysConfig, + Optional appWorkDirOptional) throws Exception { + // Set system properties passed in via application config. As an example, Helix uses System#getProperty() for ZK configuration + // overrides such as sessionTimeout. In this case, the overrides specified + // in the application configuration have to be extracted and set before initializing HelixManager. + GobblinClusterUtils.setSystemProperties(sysConfig); + + //Add dynamic config + this.config = GobblinClusterUtils.addDynamicConfig(sysConfig); + + this.clusterName = clusterName; + this.isStandaloneMode = ConfigUtils.getBoolean(this.config, GobblinClusterConfigurationKeys.STANDALONE_CLUSTER_MODE_KEY, + GobblinClusterConfigurationKeys.DEFAULT_STANDALONE_CLUSTER_MODE); + + this.applicationId = applicationId; + + this.fs = GobblinClusterUtils.buildFileSystem(this.config, new Configuration()); + this.appWorkDir = appWorkDirOptional.isPresent() ? appWorkDirOptional.get() + : GobblinClusterUtils.getAppWorkDirPathFromConfig(this.config, this.fs, clusterName, applicationId); + LOGGER.info("Configured GobblinClusterManager work dir to: {}", this.appWorkDir); + + initializeAppLauncherAndServices(); + } + + /** + * Create the service based application launcher and other associated services + * @throws Exception + */ + private void initializeAppLauncherAndServices() throws Exception { + // Done to preserve backwards compatibility with the previously hard-coded timeout of 5 minutes + Properties properties = ConfigUtils.configToProperties(this.config); + if (!properties.contains(ServiceBasedAppLauncher.APP_STOP_TIME_SECONDS)) { + properties.setProperty(ServiceBasedAppLauncher.APP_STOP_TIME_SECONDS, Long.toString(300)); + } + this.applicationLauncher = new ServiceBasedAppLauncher(properties, this.clusterName); + + // create a job catalog for keeping track of received jobs if a job config path is specified + if (this.config.hasPath(GobblinClusterConfigurationKeys.GOBBLIN_CLUSTER_PREFIX + + ConfigurationKeys.JOB_CONFIG_FILE_GENERAL_PATH_KEY)) { + String jobCatalogClassName = ConfigUtils.getString(config, GobblinClusterConfigurationKeys.JOB_CATALOG_KEY, + GobblinClusterConfigurationKeys.DEFAULT_JOB_CATALOG); + + this.jobCatalog = + (MutableJobCatalog) GobblinConstructorUtils.invokeFirstConstructor(Class.forName(jobCatalogClassName), + ImmutableList.of(config + .getConfig(StringUtils.removeEnd(GobblinClusterConfigurationKeys.GOBBLIN_CLUSTER_PREFIX, ".")) + .withFallback(this.config))); + } else { + this.jobCatalog = null; + } + + SchedulerService schedulerService = new SchedulerService(properties); + this.applicationLauncher.addService(schedulerService); + this.jobConfigurationManager = buildJobConfigurationManager(config); + this.applicationLauncher.addService(this.jobConfigurationManager); + + if (ConfigUtils.getBoolean(this.config, GobblinClusterConfigurationKeys.CONTAINER_HEALTH_METRICS_SERVICE_ENABLED, + GobblinClusterConfigurationKeys.DEFAULT_CONTAINER_HEALTH_METRICS_SERVICE_ENABLED)) { + this.applicationLauncher.addService(new ContainerHealthMetricsService(config)); + } + } + + /** + * Start any services required by the application launcher then start the application launcher + */ + private void startAppLauncherAndServices() { + // other services such as the job configuration manager have a dependency on the job catalog, so it has be be + // started first + if (this.jobCatalog instanceof Service) { + ((Service) this.jobCatalog).startAsync().awaitRunning(); + } + + this.applicationLauncher.start(); + } + + /** + * Stop the application launcher then any services that were started outside of the application launcher + */ + private void stopAppLauncherAndServices() { + try { + this.applicationLauncher.stop(); + } catch (ApplicationException ae) { + LOGGER.error("Error while stopping Gobblin Cluster application launcher", ae); + } + + if (this.jobCatalog instanceof Service) { + ((Service) this.jobCatalog).stopAsync().awaitTerminated(); + } + } + + + /** + * Start the Gobblin Cluster Manager. + */ + // @Import(clazz = ClientSslContextFactory.class, prefix = ClientSslContextFactory.SCOPE_PREFIX) + @Override + public void start() { + // temporal workflow + LOGGER.info("Starting the Gobblin Temporal Cluster Manager"); + + this.eventBus.register(this); + + if (this.isStandaloneMode) { + // standalone mode starts non-daemon threads later, so need to have this thread to keep process up + this.idleProcessThread = new Thread(new Runnable() { + @Override + public void run() { + while (!GobblinTemporalClusterManager.this.stopStatus.isStopInProgress() && !GobblinTemporalClusterManager.this.stopIdleProcessThread) { + try { + Thread.sleep(300); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + } + }); + + this.idleProcessThread.start(); + + // Need this in case a kill is issued to the process so that the idle thread does not keep the process up + // since GobblinClusterManager.stop() is not called this case. + Runtime.getRuntime().addShutdownHook(new Thread(() -> GobblinTemporalClusterManager.this.stopIdleProcessThread = true)); + } else { + startAppLauncherAndServices(); + } + this.started = true; + + try { + initiateWorkflow(); + }catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void initiateWorkflow() + throws Exception { + LOGGER.info("Initiating Temporal Workflow"); + WorkflowServiceStubs workflowServiceStubs = createServiceStubs(); + WorkflowClient client = + WorkflowClient.newInstance( + workflowServiceStubs, WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build()); + + /* + * Set Workflow options such as WorkflowId and Task Queue so the worker knows where to list and which workflows to execute. + */ + WorkflowOptions options = WorkflowOptions.newBuilder() + .setTaskQueue(Shared.HELLO_WORLD_TASK_QUEUE) + .build(); + + // Create the workflow client stub. It is used to start our workflow execution. + GobblinTemporalWorkflow workflow = client.newWorkflowStub(GobblinTemporalWorkflow.class, options); + + /* + * Execute our workflow and wait for it to complete. The call to our getGreeting method is + * synchronous. + * + * Replace the parameter "World" in the call to getGreeting() with your name. + */ + String greeting = workflow.getGreeting("World"); + + String workflowId = WorkflowStub.fromTyped(workflow).getExecution().getWorkflowId(); + // Display workflow execution results + LOGGER.info(workflowId + " " + greeting); + } + + public static WorkflowServiceStubs createServiceStubs() + throws Exception { + GobblinClusterUtils.setSystemProperties(ConfigFactory.load()); + Config config = GobblinClusterUtils.addDynamicConfig(ConfigFactory.load()); + String SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT = "gobblin.kafka.sharedConfig."; + String SSL_KEYMANAGER_ALGORITHM = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.keymanager.algorithm"; + String SSL_KEYSTORE_TYPE = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.keystore.type"; + String SSL_KEYSTORE_LOCATION = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.keystore.location"; + String SSL_KEY_PASSWORD = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.key.password"; + String SSL_TRUSTSTORE_LOCATION = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.truststore.location"; + String SSL_TRUSTSTORE_PASSWORD = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.truststore.password"; + + List SSL_CONFIG_DEFAULT_SSL_PROTOCOLS = Collections.unmodifiableList( + Arrays.asList("TLSv1.2")); + List SSL_CONFIG_DEFAULT_CIPHER_SUITES = Collections.unmodifiableList(Arrays.asList( + // The following list is from https://github.com/netty/netty/blob/4.1/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2SecurityUtil.java#L50 + "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256", + + /* REQUIRED BY HTTP/2 SPEC */ + "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", + /* REQUIRED BY HTTP/2 SPEC */ + + "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384", + "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384", + "TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256", + "TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256" + )); + + String keyStoreType = config.getString(SSL_KEYSTORE_TYPE); + File keyStoreFile = new File(config.getString(SSL_KEYSTORE_LOCATION)); + String keyStorePassword = config.getString(SSL_KEY_PASSWORD); + + KeyStore keyStore = KeyStore.getInstance(keyStoreType); + keyStore.load(toInputStream(keyStoreFile), keyStorePassword.toCharArray()); + + // Set key manager from key store + String sslKeyManagerAlgorithm = config.getString(SSL_KEYMANAGER_ALGORITHM); + KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(sslKeyManagerAlgorithm); + keyManagerFactory.init(keyStore, keyStorePassword.toCharArray()); + + // Set trust manager from trust store + KeyStore trustStore = KeyStore.getInstance("JKS"); + File trustStoreFile = new File(config.getString(SSL_TRUSTSTORE_LOCATION)); + LOGGER.info("SSL_TRUSTSTORE_LOCATION " + config.getString(SSL_TRUSTSTORE_LOCATION)); + + String trustStorePassword = config.getString(SSL_TRUSTSTORE_PASSWORD); + trustStore.load(toInputStream(trustStoreFile), trustStorePassword.toCharArray()); + TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("SunX509"); + trustManagerFactory.init(trustStore); + + SslContext sslContext = GrpcSslContexts.forClient() + .keyManager(keyManagerFactory) + .trustManager(trustManagerFactory) + .protocols(SSL_CONFIG_DEFAULT_SSL_PROTOCOLS) + .ciphers(SSL_CONFIG_DEFAULT_CIPHER_SUITES) + .build(); + + LOGGER.info("SSLContext: " + sslContext); + + return WorkflowServiceStubs.newServiceStubs( + WorkflowServiceStubsOptions.newBuilder() + .setTarget("1.nephos-temporal.corp-lca1.atd.corp.linkedin.com:7233") + .setEnableHttps(true) + .setSslContext(sslContext) + .build()); + + } + /** + * Stop the Gobblin Cluster Manager. + */ + @Override + public synchronized void stop() { + if (this.stopStatus.isStopInProgress()) { + return; + } + + this.stopStatus.setStopInprogress(true); + + LOGGER.info("Stopping the Gobblin Cluster Manager"); + + if (this.idleProcessThread != null) { + try { + this.idleProcessThread.join(); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + } + } + + stopAppLauncherAndServices(); + + } + + /** + * Get additional {@link Tag}s required for any type of reporting. + */ + private List> getMetadataTags(String applicationName, String applicationId) { + return Tag.fromMap( + new ImmutableMap.Builder().put(GobblinClusterMetricTagNames.APPLICATION_NAME, applicationName) + .put(GobblinClusterMetricTagNames.APPLICATION_ID, applicationId).build()); + } + + /** + * Build the {@link JobConfigurationManager} for the Application Master. + */ + private JobConfigurationManager buildJobConfigurationManager(Config config) { + try { + List argumentList = (this.jobCatalog != null)? ImmutableList.of(this.eventBus, config, this.jobCatalog, this.fs) : + ImmutableList.of(this.eventBus, config, this.fs); + if (config.hasPath(GobblinClusterConfigurationKeys.JOB_CONFIGURATION_MANAGER_KEY)) { + return (JobConfigurationManager) GobblinConstructorUtils.invokeLongestConstructor(Class.forName( + config.getString(GobblinClusterConfigurationKeys.JOB_CONFIGURATION_MANAGER_KEY)), argumentList.toArray(new Object[argumentList.size()])); + } else { + return new JobConfigurationManager(this.eventBus, config); + } + } catch (ReflectiveOperationException e) { + throw new RuntimeException(e); + } + } + + @SuppressWarnings("unused") + @Subscribe + public void handleApplicationMasterShutdownRequest(ClusterManagerShutdownRequest shutdownRequest) { + stop(); + } + + @Override + public void close() throws IOException { + this.applicationLauncher.close(); + } + + @Override + public Collection getStandardMetricsCollection() { + List list = new ArrayList(); + list.addAll(this.jobCatalog.getStandardMetricsCollection()); + list.addAll(this.jobConfigurationManager.getStandardMetricsCollection()); + return list; + } + + /** + * TODO for now the cluster id is hardcoded to 1 both here and in the {@link GobblinTaskRunner}. In the future, the + * cluster id should be created by the {@link GobblinTemporalClusterManager} and passed to each {@link GobblinTaskRunner} via + * Helix (at least that would be the easiest approach, there are certainly others ways to do it). + */ + private static String getApplicationId() { + return "1"; + } + + private static Options buildOptions() { + Options options = new Options(); + options.addOption("a", GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME, true, "Gobblin application name"); + options.addOption("s", GobblinClusterConfigurationKeys.STANDALONE_CLUSTER_MODE, true, "Standalone cluster mode"); + options.addOption("i", GobblinClusterConfigurationKeys.HELIX_INSTANCE_NAME_OPTION_NAME, true, "Helix instance name"); + return options; + } + + private static void printUsage(Options options) { + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp(GobblinTemporalClusterManager.class.getSimpleName(), options); + } + + public static void main(String[] args) throws Exception { + Options options = buildOptions(); + try { + CommandLine cmd = new DefaultParser().parse(options, args); + if (!cmd.hasOption(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME)) { + printUsage(options); + System.exit(1); + } + + boolean isStandaloneClusterManager = false; + if (cmd.hasOption(GobblinClusterConfigurationKeys.STANDALONE_CLUSTER_MODE)) { + isStandaloneClusterManager = Boolean.parseBoolean(cmd.getOptionValue(GobblinClusterConfigurationKeys.STANDALONE_CLUSTER_MODE, "false")); + } + + LOGGER.info(JvmUtils.getJvmInputArguments()); + Config config = ConfigFactory.load(); + + if (cmd.hasOption(GobblinClusterConfigurationKeys.HELIX_INSTANCE_NAME_OPTION_NAME)) { + config = config.withValue(GobblinClusterConfigurationKeys.HELIX_INSTANCE_NAME_KEY, + ConfigValueFactory.fromAnyRef(cmd.getOptionValue( + GobblinClusterConfigurationKeys.HELIX_INSTANCE_NAME_OPTION_NAME))); + } + + if (isStandaloneClusterManager) { + config = config.withValue(GobblinClusterConfigurationKeys.STANDALONE_CLUSTER_MODE_KEY, + ConfigValueFactory.fromAnyRef(true)); + } + + try (GobblinTemporalClusterManager gobblinClusterManager = new GobblinTemporalClusterManager( + cmd.getOptionValue(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME), getApplicationId(), + config, Optional.absent())) { + gobblinClusterManager.start(); + } + } catch (ParseException pe) { + printUsage(options); + System.exit(1); + } + } + + @Override + public void becomeActive() { + startAppLauncherAndServices(); + } + + @Override + public void becomeStandby() { + stopAppLauncherAndServices(); + try { + initializeAppLauncherAndServices(); + } catch (Exception e) { + throw new RuntimeException("Exception reinitializing app launcher services ", e); + } + } + + static class StopStatus { + @Getter + @Setter + AtomicBoolean isStopInProgress; + public StopStatus(boolean inProgress) { + isStopInProgress = new AtomicBoolean(inProgress); + } + public void setStopInprogress (boolean inProgress) { + isStopInProgress.set(inProgress); + } + public boolean isStopInProgress () { + return isStopInProgress.get(); + } + } +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java new file mode 100644 index 0000000000..1337dfd9aa --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java @@ -0,0 +1,379 @@ +/* + * 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.gobblin.cluster; + +import java.io.IOException; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Properties; +import java.util.UUID; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.DefaultParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.api.client.repackaged.com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import com.google.common.eventbus.EventBus; +import com.google.common.eventbus.Subscribe; +import com.google.common.util.concurrent.Service; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import io.temporal.client.WorkflowClient; +import io.temporal.client.WorkflowClientOptions; +import io.temporal.serviceclient.WorkflowServiceStubs; +import io.temporal.worker.Worker; +import io.temporal.worker.WorkerFactory; +import lombok.Getter; +import lombok.Setter; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.cluster.temporal.GobblinTemporalActivityImpl; +import org.apache.gobblin.cluster.temporal.GobblinTemporalWorkflowImpl; +import org.apache.gobblin.cluster.temporal.Shared; +import org.apache.gobblin.configuration.State; +import org.apache.gobblin.instrumented.StandardMetricsBridge; +import org.apache.gobblin.metrics.GobblinMetrics; +import org.apache.gobblin.metrics.RootMetricContext; +import org.apache.gobblin.metrics.event.EventSubmitter; +import org.apache.gobblin.metrics.event.GobblinEventBuilder; +import org.apache.gobblin.runtime.api.TaskEventMetadataGenerator; +import org.apache.gobblin.util.ConfigUtils; +import org.apache.gobblin.util.FileUtils; +import org.apache.gobblin.util.HadoopUtils; +import org.apache.gobblin.util.JvmUtils; +import org.apache.gobblin.util.TaskEventMetadataUtils; +import org.apache.gobblin.util.event.ContainerHealthCheckFailureEvent; + +import static org.apache.gobblin.cluster.GobblinTemporalClusterManager.createServiceStubs; + + +/** + * The main class running in the containers managing services for running Gobblin + * {@link org.apache.gobblin.source.workunit.WorkUnit}s. + * + *

+ * This class presents a Helix participant that uses a to communicate with Helix. + * It uses Helix task execution framework and details are encapsulated in {@link TaskRunnerSuiteBase}. + *

+ * + *

+ * This class responds to a graceful shutdown initiated by the {@link GobblinTemporalClusterManager} via + * a Helix message of subtype {@link HelixMessageSubTypes#WORK_UNIT_RUNNER_SHUTDOWN}, or it does a + * graceful shutdown when the shutdown hook gets called. In both cases, {@link #stop()} will be + * called to start the graceful shutdown. + *

+ * + *

+ * If for some reason, the container exits or gets killed, the {@link GobblinTemporalClusterManager} will + * be notified for the completion of the container and will start a new container to replace this one. + *

+ * + * @author Yinan Li + */ +@Alpha +public class GobblinTemporalTaskRunner implements StandardMetricsBridge { + // Working directory key for applications. This config is set dynamically. + public static final String CLUSTER_APP_WORK_DIR = GobblinClusterConfigurationKeys.GOBBLIN_CLUSTER_PREFIX + "appWorkDir"; + + private static final Logger logger = LoggerFactory.getLogger(GobblinTemporalTaskRunner.class); + + static final java.nio.file.Path CLUSTER_CONF_PATH = Paths.get("generated-gobblin-cluster.conf"); + private final Optional containerMetrics; + private final Path appWorkPath; + private boolean isTaskDriver; + @Getter + private volatile boolean started = false; + private volatile boolean stopInProgress = false; + private volatile boolean isStopped = false; + @Getter + @Setter + private volatile boolean healthCheckFailed = false; + + protected final String taskRunnerId; + protected final EventBus eventBus = new EventBus(GobblinTemporalTaskRunner.class.getSimpleName()); + protected final Config clusterConfig; + @Getter + protected final FileSystem fs; + protected final String applicationName; + protected final String applicationId; + + public GobblinTemporalTaskRunner(String applicationName, + String applicationId, + String taskRunnerId, + Config config, + Optional appWorkDirOptional) throws Exception { + // Set system properties passed in via application config. As an example, Helix uses System#getProperty() for ZK configuration + // overrides such as sessionTimeout. In this case, the overrides specified + // in the application configuration have to be extracted and set before initializing HelixManager. + GobblinClusterUtils.setSystemProperties(config); + + //Add dynamic config + config = GobblinClusterUtils.addDynamicConfig(config); + + this.isTaskDriver = ConfigUtils.getBoolean(config, GobblinClusterConfigurationKeys.TASK_DRIVER_ENABLED,false); + this.taskRunnerId = taskRunnerId; + this.applicationName = applicationName; + this.applicationId = applicationId; + Configuration conf = HadoopUtils.newConfiguration(); + this.fs = GobblinClusterUtils.buildFileSystem(config, conf); + this.appWorkPath = initAppWorkDir(config, appWorkDirOptional); + this.clusterConfig = saveConfigToFile(config); + + logger.info("Configured GobblinTaskRunner work dir to: {}", this.appWorkPath.toString()); + + this.containerMetrics = buildContainerMetrics(); + + logger.info("GobblinTaskRunner({}): applicationName {}, applicationId {}, taskRunnerId {}, config {}, appWorkDir {}", + this.isTaskDriver ? "taskDriver" : "worker", + applicationName, + applicationId, + taskRunnerId, + config, + appWorkDirOptional); + } + + private Path initAppWorkDir(Config config, Optional appWorkDirOptional) { + return appWorkDirOptional.isPresent() ? appWorkDirOptional.get() : GobblinClusterUtils + .getAppWorkDirPathFromConfig(config, this.fs, this.applicationName, this.applicationId); + } + + private Config saveConfigToFile(Config config) + throws IOException { + Config newConf = config + .withValue(CLUSTER_APP_WORK_DIR, ConfigValueFactory.fromAnyRef(this.appWorkPath.toString())); + ConfigUtils configUtils = new ConfigUtils(new FileUtils()); + configUtils.saveConfigToFile(newConf, CLUSTER_CONF_PATH); + return newConf; + } + + /** + * Start this {@link GobblinTemporalTaskRunner} instance. + */ + public void start() + throws ContainerHealthCheckException { + logger.info("Calling start method in GobblinTemporalTaskRunner"); + logger.info(String.format("Starting in container %s", this.taskRunnerId)); + try { + initiateWorker(); + }catch (Exception e) { + logger.info(e + " for initiate workers"); + throw new RuntimeException(e); + } + + // Add a shutdown hook so the task scheduler gets properly shutdown + addShutdownHook(); + } + + private void initiateWorker() throws Exception{ + logger.info("Starting Temporal Worker"); + WorkflowServiceStubs service = createServiceStubs(); + + // WorkflowClient can be used to start, signal, query, cancel, and terminate Workflows. + WorkflowClient client = + WorkflowClient.newInstance( + service, WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build()); + + /* + * Define the workflow factory. It is used to create workflow workers that poll specific Task Queues. + */ + WorkerFactory factory = WorkerFactory.newInstance(client); + + /* + * Define the workflow worker. Workflow workers listen to a defined task queue and process + * workflows and activities. + */ + Worker worker = factory.newWorker(Shared.HELLO_WORLD_TASK_QUEUE); + + /* + * Register our workflow implementation with the worker. + * Workflow implementations must be known to the worker at runtime in + * order to dispatch workflow tasks. + */ + worker.registerWorkflowImplementationTypes(GobblinTemporalWorkflowImpl.class); + + /* + * Register our Activity Types with the Worker. Since Activities are stateless and thread-safe, + * the Activity Type is a shared instance. + */ + worker.registerActivitiesImplementations(new GobblinTemporalActivityImpl()); + + /* + * Start all the workers registered for a specific task queue. + * The started workers then start polling for workflows and activities. + */ + factory.start(); + logger.info("A new worker is started."); + } + + public synchronized void stop() { + if (this.isStopped) { + logger.info("Gobblin Task runner is already stopped."); + return; + } + + if (this.stopInProgress) { + logger.info("Gobblin Task runner stop already in progress."); + return; + } + + this.stopInProgress = true; + + logger.info("Stopping the Gobblin Task runner"); + + // Stop metric reporting + if (this.containerMetrics.isPresent()) { + this.containerMetrics.get().stopMetricsReporting(); + } + + logger.info("All services are stopped."); + + this.isStopped = true; + } + + /** + * Creates and returns a {@link List} of additional {@link Service}s that should be run in this + * {@link GobblinTemporalTaskRunner}. Sub-classes that need additional {@link Service}s to run, should override this method + * + * @return a {@link List} of additional {@link Service}s to run. + */ + protected List getServices() { + List serviceList = new ArrayList<>(); + if (ConfigUtils.getBoolean(this.clusterConfig, GobblinClusterConfigurationKeys.CONTAINER_HEALTH_METRICS_SERVICE_ENABLED, + GobblinClusterConfigurationKeys.DEFAULT_CONTAINER_HEALTH_METRICS_SERVICE_ENABLED)) { + serviceList.add(new ContainerHealthMetricsService(clusterConfig)); + } + return serviceList; + } + + @VisibleForTesting + boolean isStopped() { + return this.isStopped; + } + + private void addShutdownHook() { + Runtime.getRuntime().addShutdownHook(new Thread() { + + @Override + public void run() { + logger.info("Running the shutdown hook"); + GobblinTemporalTaskRunner.this.stop(); + } + }); + } + + private Optional buildContainerMetrics() { + Properties properties = ConfigUtils.configToProperties(this.clusterConfig); + if (GobblinMetrics.isEnabled(properties)) { + logger.info("Container metrics are enabled"); + return Optional.of(ContainerMetrics + .get(ConfigUtils.configToState(clusterConfig), this.applicationName, this.taskRunnerId)); + } else { + return Optional.absent(); + } + } + + // hard coded for now + @Override + public Collection getStandardMetricsCollection() { + return null; + } + + @Subscribe + public void handleContainerHealthCheckFailureEvent(ContainerHealthCheckFailureEvent event) { + logger.error("Received {} from: {}", event.getClass().getSimpleName(), event.getClassName()); + logger.error("Submitting a ContainerHealthCheckFailureEvent.."); + submitEvent(event); + logger.error("Stopping GobblinTaskRunner..."); + GobblinTemporalTaskRunner.this.setHealthCheckFailed(true); + GobblinTemporalTaskRunner.this.stop(); + } + + private void submitEvent(ContainerHealthCheckFailureEvent event) { + EventSubmitter eventSubmitter = new EventSubmitter.Builder(RootMetricContext.get(), getClass().getPackage().getName()).build(); + GobblinEventBuilder eventBuilder = new GobblinEventBuilder(event.getClass().getSimpleName()); + State taskState = ConfigUtils.configToState(event.getConfig()); + //Add task metadata such as Helix taskId, containerId, and workflowId if configured + TaskEventMetadataGenerator taskEventMetadataGenerator = TaskEventMetadataUtils.getTaskEventMetadataGenerator(taskState); + eventBuilder.addAdditionalMetadata(taskEventMetadataGenerator.getMetadata(taskState, event.getClass().getSimpleName())); + eventBuilder.addAdditionalMetadata(event.getMetadata()); + eventSubmitter.submit(eventBuilder); + } + + private static String getApplicationId() { + return "1"; + } + + private static String getTaskRunnerId() { + return UUID.randomUUID().toString(); + } + + public static Options buildOptions() { + Options options = new Options(); + options.addOption("a", GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME, true, + "Application name"); + options.addOption("d", GobblinClusterConfigurationKeys.APPLICATION_ID_OPTION_NAME, true, + "Application id"); + options.addOption("i", GobblinClusterConfigurationKeys.HELIX_INSTANCE_NAME_OPTION_NAME, true, + "Helix instance name"); + options.addOption(Option.builder("t").longOpt(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_OPTION_NAME) + .hasArg(true).required(false).desc("Helix instance tags").build()); + return options; + } + + public static void printUsage(Options options) { + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp(GobblinTemporalClusterManager.class.getSimpleName(), options); + } + + public static void main(String[] args) + throws Exception { + Options options = buildOptions(); + try { + CommandLine cmd = new DefaultParser().parse(options, args); + if (!cmd.hasOption(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME)) { + printUsage(options); + System.exit(1); + } + + logger.info(JvmUtils.getJvmInputArguments()); + + String applicationName = + cmd.getOptionValue(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME); + GobblinTemporalTaskRunner gobblinWorkUnitRunner = + new GobblinTemporalTaskRunner(applicationName, getApplicationId(), + getTaskRunnerId(), ConfigFactory.load(), Optional.absent()); + gobblinWorkUnitRunner.start(); + } catch (ParseException pe) { + printUsage(options); + System.exit(1); + } + } +} \ No newline at end of file diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java new file mode 100644 index 0000000000..269ff09479 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java @@ -0,0 +1,30 @@ +/* + * 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. + */ + +// @@@SNIPSTART hello-world-project-template-java-activity-interface +package org.apache.gobblin.cluster.temporal; + +import io.temporal.activity.ActivityInterface; + +@ActivityInterface +public interface GobblinTemporalActivity { + + // Define your activity methods which can be called during workflow execution + String composeGreeting(String name); + +} +// @@@SNIPEND diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java new file mode 100644 index 0000000000..c1c5225f50 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java @@ -0,0 +1,35 @@ +/* + * 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. + */ + +// @@@SNIPSTART hello-world-project-template-java-activity +package org.apache.gobblin.cluster.temporal; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class GobblinTemporalActivityImpl implements GobblinTemporalActivity { + + private static final Logger LOGGER = LoggerFactory.getLogger(GobblinTemporalActivityImpl.class); + @Override + public String composeGreeting(String name) { + LOGGER.info("Activity triggered"); + return "Hello " + name + "!"; + } + +} +// @@@SNIPEND diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java new file mode 100644 index 0000000000..7edfcd0e81 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java @@ -0,0 +1,34 @@ +/* + * 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. + */ + +// @@@SNIPSTART hello-world-project-template-java-workflow-interface +package org.apache.gobblin.cluster.temporal; + +import io.temporal.workflow.WorkflowInterface; +import io.temporal.workflow.WorkflowMethod; + +@WorkflowInterface +public interface GobblinTemporalWorkflow { + + /** + * This is the method that is executed when the Workflow Execution is started. The Workflow + * Execution completes when this method finishes execution. + */ + @WorkflowMethod + String getGreeting(String name); +} +// @@@SNIPEND diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java new file mode 100644 index 0000000000..c2f4e76f30 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java @@ -0,0 +1,72 @@ +/* + * 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. + */ + +// @@@SNIPSTART hello-world-project-template-java-workflow +package org.apache.gobblin.cluster.temporal; + +import java.time.Duration; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.temporal.activity.ActivityOptions; +import io.temporal.common.RetryOptions; +import io.temporal.workflow.Workflow; + +public class GobblinTemporalWorkflowImpl implements GobblinTemporalWorkflow { + + /* + * At least one of the following options needs to be defined: + * - setStartToCloseTimeout + * - setScheduleToCloseTimeout + */ + + private final RetryOptions retryoptions = RetryOptions.newBuilder() + .setInitialInterval(Duration.ofSeconds(1)) + .setMaximumInterval(Duration.ofSeconds(100)) + .setBackoffCoefficient(2) + .setMaximumAttempts(500) + .build(); + + private final ActivityOptions options = ActivityOptions.newBuilder() + .setStartToCloseTimeout(Duration.ofSeconds(60)) + .setRetryOptions(retryoptions) + .build(); + + /* + * Define the HelloWorldActivity stub. Activity stubs are proxies for activity invocations that + * are executed outside of the workflow thread on the activity worker, that can be on a + * different host. Temporal is going to dispatch the activity results back to the workflow and + * unblock the stub as soon as activity is completed on the activity worker. + * + * The activity options that were defined above are passed in as a parameter. + */ + private final GobblinTemporalActivity activity = Workflow.newActivityStub(GobblinTemporalActivity.class, options); + private static final Logger LOGGER = LoggerFactory.getLogger(GobblinTemporalWorkflowImpl.class); + // This is the entry point to the Workflow. + @Override + public String getGreeting(String name) { + + /** + * If there were other Activity methods they would be orchestrated here or from within other Activities. + * This is a blocking call that returns only after the activity has completed. + */ + LOGGER.info("Workflow triggered"); + return activity.composeGreeting(name); + } +} +// @@@SNIPEND diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java new file mode 100644 index 0000000000..9aa869af57 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java @@ -0,0 +1,27 @@ +/* + * 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. + */ + +// @@@SNIPSTART hello-world-project-template-java-shared-constants +package org.apache.gobblin.cluster.temporal; + +public interface Shared { + + // Define the task queue name + final String HELLO_WORLD_TASK_QUEUE = "HelloWorldTaskQueue"; + +} +// @@@SNIPEND diff --git a/gobblin-core/src/main/java/org/apache/gobblin/security/ssl/SSLContextFactory.java b/gobblin-core/src/main/java/org/apache/gobblin/security/ssl/SSLContextFactory.java index e7677fe76f..a940090cf9 100644 --- a/gobblin-core/src/main/java/org/apache/gobblin/security/ssl/SSLContextFactory.java +++ b/gobblin-core/src/main/java/org/apache/gobblin/security/ssl/SSLContextFactory.java @@ -119,7 +119,7 @@ public static SSLContext createInstance(Config srcConfig) { new File(trustStoreFilePath), trustStorePassword); } - private static InputStream toInputStream(File storeFile) + public static InputStream toInputStream(File storeFile) throws IOException { byte[] data = FileUtils.readFileToByteArray(storeFile); return new ByteArrayInputStream(data); diff --git a/gobblin-yarn/build.gradle b/gobblin-yarn/build.gradle index 0221c01b4e..8594245a67 100644 --- a/gobblin-yarn/build.gradle +++ b/gobblin-yarn/build.gradle @@ -59,7 +59,7 @@ dependencies { compile (externalDependency.helix) { exclude group: 'io.dropwizard.metrics', module: 'metrics-core' } - + compile externalDependency."temporal-sdk" testCompile project(path: ':gobblin-cluster', configuration: 'tests') testCompile project(":gobblin-example") diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java new file mode 100644 index 0000000000..4ef48c9765 --- /dev/null +++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java @@ -0,0 +1,172 @@ +/* + * 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.gobblin.yarn; + +import java.util.List; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.DefaultParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Optional; +import com.google.common.util.concurrent.Service; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import lombok.Getter; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; +import org.apache.gobblin.cluster.GobblinClusterUtils; +import org.apache.gobblin.cluster.GobblinTemporalClusterManager; +import org.apache.gobblin.util.ConfigUtils; +import org.apache.gobblin.util.JvmUtils; +import org.apache.gobblin.util.PathUtils; +import org.apache.gobblin.util.logs.Log4jConfigurationHelper; +import org.apache.gobblin.util.logs.LogCopier; +import org.apache.gobblin.util.reflection.GobblinConstructorUtils; + + +/** + * The Yarn ApplicationMaster class for Gobblin. + * + *

+ * This class runs the {@link YarnTemporalService} for all Yarn-related stuffs like ApplicationMaster registration + * and un-registration and Yarn container provisioning. + *

+ * + * @author Yinan Li + */ +@Alpha +public class GobblinTemporalApplicationMaster extends GobblinTemporalClusterManager { + private static final Logger LOGGER = LoggerFactory.getLogger(GobblinTemporalApplicationMaster.class); + + @Getter + private final YarnTemporalService _yarnTemporalService; + private LogCopier logCopier; + + public GobblinTemporalApplicationMaster(String applicationName, String applicationId, ContainerId containerId, Config config, + YarnConfiguration yarnConfiguration) throws Exception { + super(applicationName, applicationId, config.withValue(GobblinYarnConfigurationKeys.CONTAINER_NUM_KEY, + ConfigValueFactory.fromAnyRef(YarnHelixUtils.getContainerNum(containerId.toString()))), + Optional.absent()); + + String containerLogDir = config.getString(GobblinYarnConfigurationKeys.LOGS_SINK_ROOT_DIR_KEY); + GobblinYarnLogSource gobblinYarnLogSource = new GobblinYarnLogSource(); + if (gobblinYarnLogSource.isLogSourcePresent()) { + Path appWorkDir = PathUtils.combinePaths(containerLogDir, GobblinClusterUtils.getAppWorkDirPath(this.clusterName, this.applicationId), "AppMaster"); + logCopier = gobblinYarnLogSource.buildLogCopier(this.config, containerId.toString(), this.fs, appWorkDir); + this.applicationLauncher + .addService(logCopier); + } + YarnHelixUtils.setYarnClassPath(config, yarnConfiguration); + YarnHelixUtils.setAdditionalYarnClassPath(config, yarnConfiguration); + this._yarnTemporalService = buildTemporalYarnService(this.config, applicationName, this.applicationId, yarnConfiguration, this.fs); + this.applicationLauncher.addService(this._yarnTemporalService); + + if (UserGroupInformation.isSecurityEnabled()) { + LOGGER.info("Adding YarnContainerSecurityManager since security is enabled"); + this.applicationLauncher.addService(buildYarnContainerSecurityManager(this.config, this.fs)); + } + + // Add additional services + List serviceClassNames = ConfigUtils.getStringList(this.config, + GobblinYarnConfigurationKeys.APP_MASTER_SERVICE_CLASSES); + + for (String serviceClassName : serviceClassNames) { + Class serviceClass = Class.forName(serviceClassName); + this.applicationLauncher.addService((Service) GobblinConstructorUtils.invokeLongestConstructor(serviceClass, this)); + } + } + + /** + * Build the {@link YarnTemporalService} for the Application Master. + */ + protected YarnTemporalService buildTemporalYarnService(Config config, String applicationName, String applicationId, + YarnConfiguration yarnConfiguration, FileSystem fs) + throws Exception { + return new YarnTemporalService(config, applicationName, applicationId, yarnConfiguration, fs, this.eventBus); + } + + /** + * Build the {@link YarnAppMasterSecurityManager} for the Application Master. + */ + private YarnContainerSecurityManager buildYarnContainerSecurityManager(Config config, FileSystem fs) { + return new YarnTemporalAppMasterSecurityManager(config, fs, this.eventBus, this.logCopier, this._yarnTemporalService); + } + + private static Options buildOptions() { + Options options = new Options(); + options.addOption("a", GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME, true, "Yarn application name"); + options.addOption("d", GobblinClusterConfigurationKeys.APPLICATION_ID_OPTION_NAME, true, "Yarn application id"); + return options; + } + + private static void printUsage(Options options) { + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp(GobblinTemporalApplicationMaster.class.getSimpleName(), options); + } + + public static void main(String[] args) throws Exception { + Options options = buildOptions(); + try { + CommandLine cmd = new DefaultParser().parse(options, args); + if (!cmd.hasOption(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME) || + (!cmd.hasOption(GobblinClusterConfigurationKeys.APPLICATION_ID_OPTION_NAME))) { + printUsage(options); + System.exit(1); + } + + //Because AM is restarted with the original AppSubmissionContext, it may have outdated delegation tokens. + //So the refreshed tokens should be added into the container's UGI before any HDFS/Hive/RM access is performed. + YarnHelixUtils.updateToken(GobblinYarnConfigurationKeys.TOKEN_FILE_NAME); + + Log4jConfigurationHelper.updateLog4jConfiguration(GobblinTemporalApplicationMaster.class, + GobblinYarnConfigurationKeys.GOBBLIN_YARN_LOG4J_CONFIGURATION_FILE, + GobblinYarnConfigurationKeys.GOBBLIN_YARN_LOG4J_CONFIGURATION_FILE); + + LOGGER.info(JvmUtils.getJvmInputArguments()); + + ContainerId containerId = + ConverterUtils.toContainerId(System.getenv().get(ApplicationConstants.Environment.CONTAINER_ID.key())); + + try (GobblinTemporalApplicationMaster applicationMaster = new GobblinTemporalApplicationMaster( + cmd.getOptionValue(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME), + cmd.getOptionValue(GobblinClusterConfigurationKeys.APPLICATION_ID_OPTION_NAME), containerId, + ConfigFactory.load(), new YarnConfiguration())) { + + applicationMaster.start(); + } + } catch (ParseException pe) { + printUsage(options); + System.exit(1); + } + } +} diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalYarnTaskRunner.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalYarnTaskRunner.java new file mode 100644 index 0000000000..8e81714c67 --- /dev/null +++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalYarnTaskRunner.java @@ -0,0 +1,134 @@ +/* + * 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.gobblin.yarn; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.DefaultParser; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Optional; +import com.google.common.util.concurrent.Service; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; +import org.apache.gobblin.cluster.GobblinClusterUtils; +import org.apache.gobblin.cluster.GobblinTaskRunner; +import org.apache.gobblin.cluster.GobblinTemporalTaskRunner; +import org.apache.gobblin.util.JvmUtils; +import org.apache.gobblin.util.logs.Log4jConfigurationHelper; +import org.apache.gobblin.util.logs.LogCopier; + + +public class GobblinTemporalYarnTaskRunner extends GobblinTemporalTaskRunner { + + private static final Logger LOGGER = LoggerFactory.getLogger(GobblinTaskRunner.class); + public GobblinTemporalYarnTaskRunner(String applicationName, String applicationId, ContainerId containerId, Config config, + Optional appWorkDirOptional) throws Exception { + super(applicationName, applicationId, getTaskRunnerId(containerId), config + .withValue(GobblinYarnConfigurationKeys.CONTAINER_NUM_KEY, + ConfigValueFactory.fromAnyRef(YarnHelixUtils.getContainerNum(containerId.toString()))), appWorkDirOptional); + } + + @Override + public List getServices() { + List services = new ArrayList<>(); + services.addAll(super.getServices()); + LogCopier logCopier = null; + if (clusterConfig.hasPath(GobblinYarnConfigurationKeys.LOGS_SINK_ROOT_DIR_KEY)) { + GobblinYarnLogSource gobblinYarnLogSource = new GobblinYarnLogSource(); + String containerLogDir = clusterConfig.getString(GobblinYarnConfigurationKeys.LOGS_SINK_ROOT_DIR_KEY); + + if (gobblinYarnLogSource.isLogSourcePresent()) { + try { + logCopier = gobblinYarnLogSource.buildLogCopier(this.clusterConfig, this.taskRunnerId, this.fs, + new Path(containerLogDir, GobblinClusterUtils.getAppWorkDirPath(this.applicationName, this.applicationId))); + services.add(logCopier); + } catch (Exception e) { + LOGGER.warn("Cannot add LogCopier service to the service manager due to", e); + } + } + } + if (UserGroupInformation.isSecurityEnabled()) { + LOGGER.info("Adding YarnContainerSecurityManager since security is enabled"); + services.add(new YarnContainerSecurityManager(this.clusterConfig, this.fs, this.eventBus, logCopier)); + } + return services; + } + + + private static String getApplicationId(ContainerId containerId) { + return containerId.getApplicationAttemptId().getApplicationId().toString(); + } + + private static String getTaskRunnerId(ContainerId containerId) { + return containerId.toString(); + } + + public static void main(String[] args) { + LOGGER.info("Starting GobblinTemporalYarnTaskRunner"); + Options options = buildOptions(); + try { + CommandLine cmd = new DefaultParser().parse(options, args); + if (!cmd.hasOption(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME) || !cmd + .hasOption(GobblinClusterConfigurationKeys.APPLICATION_ID_OPTION_NAME)) { + printUsage(options); + System.exit(1); + } + + Log4jConfigurationHelper.updateLog4jConfiguration(GobblinTaskRunner.class, + GobblinYarnConfigurationKeys.GOBBLIN_YARN_LOG4J_CONFIGURATION_FILE, + GobblinYarnConfigurationKeys.GOBBLIN_YARN_LOG4J_CONFIGURATION_FILE); + + LOGGER.info(JvmUtils.getJvmInputArguments()); + + ContainerId containerId = + ConverterUtils.toContainerId(System.getenv().get(ApplicationConstants.Environment.CONTAINER_ID.key())); + String applicationName = cmd.getOptionValue(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME); + String applicationId = cmd.getOptionValue(GobblinClusterConfigurationKeys.APPLICATION_ID_OPTION_NAME); + Config config = ConfigFactory.load(); + + GobblinTemporalTaskRunner gobblinTemporalTaskRunner = + new GobblinTemporalYarnTaskRunner(applicationName, applicationId, containerId, config, + Optional.absent()); + gobblinTemporalTaskRunner.start(); + + } catch (ParseException pe) { + printUsage(options); + System.exit(1); + } catch (Throwable t) { + // Ideally, we should not be catching non-recoverable exceptions and errors. However, + // simply propagating the exception may prevent the container exit due to the presence of non-daemon threads present + // in the application. Hence, we catch this exception to invoke System.exit() which in turn ensures that all non-daemon threads are killed. + LOGGER.error("Exception encountered: {}", t); + System.exit(1); + } + } +} \ No newline at end of file diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnAppLauncher.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnAppLauncher.java index 99c4094df5..d6b0b36842 100644 --- a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnAppLauncher.java +++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnAppLauncher.java @@ -38,8 +38,6 @@ import org.apache.avro.Schema; import org.apache.commons.io.FileUtils; import org.apache.commons.mail.EmailException; -import org.apache.gobblin.util.hadoop.TokenUtils; -import org.apache.gobblin.util.reflection.GobblinConstructorUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -117,8 +115,10 @@ import org.apache.gobblin.util.EmailUtils; import org.apache.gobblin.util.ExecutorsUtils; import org.apache.gobblin.util.JvmUtils; +import org.apache.gobblin.util.hadoop.TokenUtils; import org.apache.gobblin.util.io.StreamUtils; import org.apache.gobblin.util.logs.LogCopier; +import org.apache.gobblin.util.reflection.GobblinConstructorUtils; import org.apache.gobblin.yarn.event.ApplicationReportArrivalEvent; import org.apache.gobblin.yarn.event.GetApplicationReportFailureEvent; @@ -824,7 +824,7 @@ private void addJobConfPackage(String jobConfPackagePath, Path destDir, Map + UNUSUAL_HELIX_TASK_STATES = Sets.newHashSet(TaskPartitionState.ERROR, TaskPartitionState.DROPPED, TaskPartitionState.COMPLETED, TaskPartitionState.TIMED_OUT); + + public YarnTemporalAutoScalingManager(GobblinTemporalApplicationMaster appMaster) { + this.config = appMaster.getConfig(); + this._yarnTemporalService = appMaster.get_yarnTemporalService(); + this.partitionsPerContainer = ConfigUtils.getInt(this.config, AUTO_SCALING_PARTITIONS_PER_CONTAINER, + DEFAULT_AUTO_SCALING_PARTITIONS_PER_CONTAINER); + + Preconditions.checkArgument(this.partitionsPerContainer > 0, + AUTO_SCALING_PARTITIONS_PER_CONTAINER + " needs to be greater than 0"); + + this.overProvisionFactor = ConfigUtils.getDouble(this.config, AUTO_SCALING_CONTAINER_OVERPROVISION_FACTOR, + DEFAULT_AUTO_SCALING_CONTAINER_OVERPROVISION_FACTOR); + + this.slidingFixedSizeWindow = config.hasPath(AUTO_SCALING_WINDOW_SIZE) + ? new SlidingWindowReservoir(config.getInt(AUTO_SCALING_WINDOW_SIZE), Integer.MAX_VALUE) + : new SlidingWindowReservoir(Integer.MAX_VALUE); + + this.autoScalingExecutor = Executors.newSingleThreadScheduledExecutor( + ExecutorsUtils.newThreadFactory(Optional.of(log), Optional.of("AutoScalingExecutor"))); + + this.defaultHelixInstanceTags = ConfigUtils.getString(config, + GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, GobblinClusterConfigurationKeys.HELIX_DEFAULT_TAG); + this.defaultContainerMemoryMbs = config.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); + this.defaultContainerCores = config.getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); + } + + @Override + protected void startUp() { + int scheduleInterval = ConfigUtils.getInt(this.config, AUTO_SCALING_POLLING_INTERVAL_SECS, + DEFAULT_AUTO_SCALING_POLLING_INTERVAL_SECS); + int initialDelay = ConfigUtils.getInt(this.config, AUTO_SCALING_INITIAL_DELAY, + DEFAULT_AUTO_SCALING_INITIAL_DELAY_SECS); + log.info("Starting the " + YarnTemporalAutoScalingManager.class.getSimpleName()); + log.info("Scheduling the auto scaling task with an interval of {} seconds", scheduleInterval); + +// this.autoScalingExecutor.scheduleAtFixedRate(new TemporalYarnAutoScalingRunnable(new TaskDriver(this.helixManager), +// this.temporalYarnService, this.partitionsPerContainer, this.overProvisionFactor, +// this.slidingFixedSizeWindow, this.defaultHelixInstanceTags, +// this.defaultContainerMemoryMbs, this.defaultContainerCores), +// initialDelay, scheduleInterval, TimeUnit.SECONDS); + } + + @Override + protected void shutDown() { + log.info("Stopping the " + YarnTemporalAutoScalingManager.class.getSimpleName()); + + ExecutorsUtils.shutdownExecutorService(this.autoScalingExecutor, Optional.of(log)); + } + + /** + * A {@link Runnable} that figures out the number of containers required for the workload + * and requests those containers. + */ + @VisibleForTesting + @AllArgsConstructor + static class TemporalYarnAutoScalingRunnable implements Runnable { + private final TaskDriver taskDriver; + private final YarnTemporalService _yarnTemporalService; + private final int partitionsPerContainer; + private final double overProvisionFactor; + private final SlidingWindowReservoir slidingWindowReservoir; + private final String defaultHelixInstanceTags; + private final int defaultContainerMemoryMbs; + private final int defaultContainerCores; + + /** + * A static map that keep track of an idle instance and its latest beginning idle time. + * If an instance is no longer idle when inspected, it will be dropped from this map. + */ + private static final Map instanceIdleSince = new HashMap<>(); + + + @Override + public void run() { + // Suppress errors to avoid interrupting any scheduled executions of this Runnable + try { + runInternal(); + } catch (Throwable t) { + log.warn("Suppressing error from YarnAutoScalingRunnable.run()", t); + } + } + + /** + * Iterate through the workflows configured in Helix to figure out the number of required partitions + * and request the {@link YarnTemporalService} to scale to the desired number of containers. + */ + @VisibleForTesting + void runInternal() { + Set inUseInstances = new HashSet<>(); + YarnContainerRequestBundle yarnContainerRequestBundle = new YarnContainerRequestBundle(); + for (Map.Entry workFlowEntry : taskDriver.getWorkflows().entrySet()) { + WorkflowContext workflowContext = taskDriver.getWorkflowContext(workFlowEntry.getKey()); + + // Only allocate for active workflows + if (workflowContext == null || !workflowContext.getWorkflowState().equals(TaskState.IN_PROGRESS)) { + continue; + } + + log.debug("Workflow name {} config {} context {}", workFlowEntry.getKey(), workFlowEntry.getValue(), + workflowContext); + + WorkflowConfig workflowConfig = workFlowEntry.getValue(); + JobDag jobDag = workflowConfig.getJobDag(); + Set jobs = jobDag.getAllNodes(); + + // sum up the number of partitions + for (String jobName : jobs) { + JobContext jobContext = taskDriver.getJobContext(jobName); + JobConfig jobConfig = taskDriver.getJobConfig(jobName); + Resource resource = Resource.newInstance(this.defaultContainerMemoryMbs, this.defaultContainerCores); + int numPartitions = 0; + String jobTag = defaultHelixInstanceTags; + if (jobContext != null) { + log.debug("JobContext {} num partitions {}", jobContext, jobContext.getPartitionSet().size()); + + numPartitions = jobContext.getPartitionSet().size(); + // Job level config for helix instance tags takes precedence over other tag configurations + if (jobConfig != null) { + if (!Strings.isNullOrEmpty(jobConfig.getInstanceGroupTag())) { + jobTag = jobConfig.getInstanceGroupTag(); + } + Map jobCommandConfigMap = jobConfig.getJobCommandConfigMap(); + if(jobCommandConfigMap.containsKey(GobblinClusterConfigurationKeys.HELIX_JOB_CONTAINER_MEMORY_MBS)){ + resource.setMemory(Integer.parseInt(jobCommandConfigMap.get(GobblinClusterConfigurationKeys.HELIX_JOB_CONTAINER_MEMORY_MBS))); + } + if(jobCommandConfigMap.containsKey(GobblinClusterConfigurationKeys.HELIX_JOB_CONTAINER_CORES)){ + resource.setVirtualCores(Integer.parseInt(jobCommandConfigMap.get(GobblinClusterConfigurationKeys.HELIX_JOB_CONTAINER_CORES))); + } + } + } + // compute the container count as a ceiling of number of partitions divided by the number of containers + // per partition. Scale the result by a constant overprovision factor. + int containerCount = (int) Math.ceil(((double)numPartitions / this.partitionsPerContainer) * this.overProvisionFactor); + yarnContainerRequestBundle.add(jobTag, containerCount, resource); + log.info("jobName={}, jobTag={}, numPartitions={}, targetNumContainers={}", + jobName, jobTag, numPartitions, containerCount); + } + } + slidingWindowReservoir.add(yarnContainerRequestBundle); + + log.debug("There are {} containers being requested in total, tag-count map {}, tag-resource map {}", + yarnContainerRequestBundle.getTotalContainers(), yarnContainerRequestBundle.getHelixTagContainerCountMap(), + yarnContainerRequestBundle.getHelixTagResourceMap()); + + this._yarnTemporalService.requestTargetNumberOfContainers(slidingWindowReservoir.getMax(), inUseInstances); + } + + @VisibleForTesting + /** + * Return true is the condition for tagging an instance as "unused" holds. + * The condition, by default is that if an instance went back to + * active (having partition running on it) within {@link #maxIdleTimeInMinutesBeforeScalingDown} minutes, we will + * not tag that instance as "unused" and have that as the candidate for scaling down. + */ + boolean isInstanceUnused(String participant){ + return System.currentTimeMillis() - instanceIdleSince.get(participant) > + TimeUnit.MINUTES.toMillis(maxIdleTimeInMinutesBeforeScalingDown); + } + } + + /** + * A FIFO queue with fixed size and returns maxValue among all elements within the queue in constant time. + * This data structure prevents temporary fluctuation in the number of active helix partitions as the size of queue + * grows and will be less sensitive when scaling down is actually required. + * + * The interface for this is implemented in a minimal-necessity manner to serve only as a sliding-sized-window + * which captures max value. It is NOT built for general purpose. + */ + static class SlidingWindowReservoir { + private ArrayDeque fifoQueue; + private PriorityQueue priorityQueue; + + // Queue Size + private int maxSize; + private static final int DEFAULT_MAX_SIZE = 10; + + // Upper-bound of value within the queue. + private int upperBound; + + public SlidingWindowReservoir(int maxSize, int upperBound) { + Preconditions.checkArgument(maxSize > 0, "maxSize has to be a value larger than 0"); + + this.maxSize = maxSize; + this.upperBound = upperBound; + this.fifoQueue = new ArrayDeque<>(maxSize); + this.priorityQueue = new PriorityQueue<>(maxSize, new Comparator() { + @Override + public int compare(YarnContainerRequestBundle o1, YarnContainerRequestBundle o2) { + Integer i2 = o2.getTotalContainers(); + return i2.compareTo(o1.getTotalContainers()); + } + }); + } + + public SlidingWindowReservoir(int upperBound) { + this(DEFAULT_MAX_SIZE, upperBound); + } + + /** + * Add element into data structure. + * When a new element is larger than upperbound, reject the value since we may request too many Yarn containers. + * When queue is full, evict head of FIFO-queue (In FIFO queue, elements are inserted from tail). + */ + public void add(YarnContainerRequestBundle e) { + if (e.getTotalContainers() > upperBound) { + log.error(String.format("Request of getting %s containers seems to be excessive, rejected", e)); + return; + } + + if (fifoQueue.size() == maxSize) { + YarnContainerRequestBundle removedElement = fifoQueue.remove(); + priorityQueue.remove(removedElement); + } + + if (fifoQueue.size() == priorityQueue.size()) { + fifoQueue.add(e); + priorityQueue.add(e); + } else { + throw new IllegalStateException("Queue has its internal data structure being inconsistent."); + } + } + + /** + * If queue is empty, throw {@link IllegalStateException}. + */ + public YarnContainerRequestBundle getMax() { + if (priorityQueue.size() > 0) { + return this.priorityQueue.peek(); + } else { + throw new IllegalStateException("Queried before elements added into the queue."); + } + } + } +} diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java new file mode 100644 index 0000000000..a45017b5c2 --- /dev/null +++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java @@ -0,0 +1,1007 @@ +/* + * 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.gobblin.yarn; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.IntStream; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAccessType; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.hadoop.yarn.api.records.LocalResourceType; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.client.api.AMRMClient; +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; +import org.apache.hadoop.yarn.client.api.async.NMClientAsync; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; +import org.apache.hadoop.yarn.util.Records; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.base.Throwables; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.eventbus.EventBus; +import com.google.common.eventbus.Subscribe; +import com.google.common.io.Closer; +import com.google.common.util.concurrent.AbstractIdleService; +import com.typesafe.config.Config; + +import lombok.AccessLevel; +import lombok.Getter; + +import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; +import org.apache.gobblin.cluster.GobblinClusterMetricTagNames; +import org.apache.gobblin.cluster.GobblinClusterUtils; +import org.apache.gobblin.cluster.event.ClusterManagerShutdownRequest; +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.metrics.GobblinMetrics; +import org.apache.gobblin.metrics.MetricReporterException; +import org.apache.gobblin.metrics.MultiReporterException; +import org.apache.gobblin.metrics.Tag; +import org.apache.gobblin.metrics.event.EventSubmitter; +import org.apache.gobblin.util.ConfigUtils; +import org.apache.gobblin.util.ExecutorsUtils; +import org.apache.gobblin.util.JvmUtils; +import org.apache.gobblin.util.executors.ScalingThreadPoolExecutor; +import org.apache.gobblin.yarn.event.ContainerReleaseRequest; +import org.apache.gobblin.yarn.event.ContainerShutdownRequest; +import org.apache.gobblin.yarn.event.NewContainerRequest; + +/** + * This class is responsible for all Yarn-related stuffs including ApplicationMaster registration, + * ApplicationMaster un-registration, Yarn container management, etc. + * + * @author Yinan Li + */ +public class YarnTemporalService extends AbstractIdleService { + + private static final Logger LOGGER = LoggerFactory.getLogger(YarnTemporalService.class); + + private static final String UNKNOWN_HELIX_INSTANCE = "UNKNOWN"; + + private final String applicationName; + private final String applicationId; + private final String appViewAcl; + //Default helix instance tag derived from cluster level config + private final String helixInstanceTags; + + private final Config config; + + private final EventBus eventBus; + + private final Configuration yarnConfiguration; + private final FileSystem fs; + + private final Optional gobblinMetrics; + private final Optional eventSubmitter; + + @VisibleForTesting + @Getter(AccessLevel.PROTECTED) + private final AMRMClientAsync amrmClientAsync; + private final NMClientAsync nmClientAsync; + private final ExecutorService containerLaunchExecutor; + + private final int initialContainers; + private final int requestedContainerMemoryMbs; + private final int requestedContainerCores; + private final int jvmMemoryOverheadMbs; + private final double jvmMemoryXmxRatio; + private final boolean containerHostAffinityEnabled; + + private final int helixInstanceMaxRetries; + + private final Optional containerJvmArgs; + private final String containerTimezone; + + @Getter(AccessLevel.PROTECTED) + private volatile Optional maxResourceCapacity = Optional.absent(); + + // Security tokens for accessing HDFS + private ByteBuffer tokens; + + private final Closer closer = Closer.create(); + + private final Object allContainersStopped = new Object(); + + // A map from container IDs to Container instances, Helix participant IDs of the containers and Helix Tag + @VisibleForTesting + @Getter(AccessLevel.PROTECTED) + private final ConcurrentMap containerMap = Maps.newConcurrentMap(); + + // A cache of the containers with an outstanding container release request. + // This is a cache instead of a set to get the automatic cleanup in case a container completes before the requested + // release. + @VisibleForTesting + @Getter(AccessLevel.PROTECTED) + private final Cache releasedContainerCache; + + // A map from Helix instance names to the number times the instances are retried to be started + private final ConcurrentMap helixInstanceRetryCount = Maps.newConcurrentMap(); + + // A concurrent HashSet of unused Helix instance names. An unused Helix instance name gets put + // into the set if the container running the instance completes. Unused Helix + // instance names get picked up when replacement containers get allocated. + private final Set unusedHelixInstanceNames = ConcurrentHashMap.newKeySet(); + + // The map from helix tag to allocated container count + private final ConcurrentMap allocatedContainerCountMap = Maps.newConcurrentMap(); + private final ConcurrentMap removedContainerID = Maps.newConcurrentMap(); + + private volatile YarnContainerRequestBundle yarnContainerRequest; + private final AtomicInteger priorityNumGenerator = new AtomicInteger(0); + private final Map resourcePriorityMap = new HashMap<>(); + + private volatile boolean shutdownInProgress = false; + + public YarnTemporalService(Config config, String applicationName, String applicationId, YarnConfiguration yarnConfiguration, + FileSystem fs, EventBus eventBus) throws Exception { + this.applicationName = applicationName; + this.applicationId = applicationId; + + this.config = config; + + this.eventBus = eventBus; + + this.gobblinMetrics = config.getBoolean(ConfigurationKeys.METRICS_ENABLED_KEY) ? + Optional.of(buildGobblinMetrics()) : Optional.absent(); + + this.eventSubmitter = config.getBoolean(ConfigurationKeys.METRICS_ENABLED_KEY) ? + Optional.of(buildEventSubmitter()) : Optional.absent(); + + this.yarnConfiguration = yarnConfiguration; + this.fs = fs; + + int amRmHeartbeatIntervalMillis = Long.valueOf(TimeUnit.SECONDS.toMillis( + ConfigUtils.getInt(config, GobblinYarnConfigurationKeys.AMRM_HEARTBEAT_INTERVAL_SECS, + GobblinYarnConfigurationKeys.DEFAULT_AMRM_HEARTBEAT_INTERVAL_SECS))).intValue(); + this.amrmClientAsync = closer.register( + AMRMClientAsync.createAMRMClientAsync(amRmHeartbeatIntervalMillis, new AMRMClientCallbackHandler())); + this.amrmClientAsync.init(this.yarnConfiguration); + this.nmClientAsync = closer.register(NMClientAsync.createNMClientAsync(getNMClientCallbackHandler())); + this.nmClientAsync.init(this.yarnConfiguration); + + this.initialContainers = config.getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); + this.requestedContainerMemoryMbs = config.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); + this.requestedContainerCores = config.getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); + this.containerHostAffinityEnabled = config.getBoolean(GobblinYarnConfigurationKeys.CONTAINER_HOST_AFFINITY_ENABLED); + + this.helixInstanceMaxRetries = config.getInt(GobblinYarnConfigurationKeys.HELIX_INSTANCE_MAX_RETRIES); + this.helixInstanceTags = ConfigUtils.getString(config, + GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, GobblinClusterConfigurationKeys.HELIX_DEFAULT_TAG); + + this.containerJvmArgs = config.hasPath(GobblinYarnConfigurationKeys.CONTAINER_JVM_ARGS_KEY) ? + Optional.of(config.getString(GobblinYarnConfigurationKeys.CONTAINER_JVM_ARGS_KEY)) : + Optional.absent(); + + int numContainerLaunchThreads = + ConfigUtils.getInt(config, GobblinYarnConfigurationKeys.MAX_CONTAINER_LAUNCH_THREADS_KEY, + GobblinYarnConfigurationKeys.DEFAULT_MAX_CONTAINER_LAUNCH_THREADS); + this.containerLaunchExecutor = ScalingThreadPoolExecutor.newScalingThreadPool(5, numContainerLaunchThreads, 0L, + ExecutorsUtils.newThreadFactory(Optional.of(LOGGER), Optional.of("ContainerLaunchExecutor"))); + + this.tokens = getSecurityTokens(); + + this.releasedContainerCache = CacheBuilder.newBuilder().expireAfterAccess(ConfigUtils.getInt(config, + GobblinYarnConfigurationKeys.RELEASED_CONTAINERS_CACHE_EXPIRY_SECS, + GobblinYarnConfigurationKeys.DEFAULT_RELEASED_CONTAINERS_CACHE_EXPIRY_SECS), TimeUnit.SECONDS).build(); + + this.jvmMemoryXmxRatio = ConfigUtils.getDouble(this.config, + GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY, + GobblinYarnConfigurationKeys.DEFAULT_CONTAINER_JVM_MEMORY_XMX_RATIO); + + Preconditions.checkArgument(this.jvmMemoryXmxRatio >= 0 && this.jvmMemoryXmxRatio <= 1, + GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY + " must be between 0 and 1 inclusive"); + + this.jvmMemoryOverheadMbs = ConfigUtils.getInt(this.config, + GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY, + GobblinYarnConfigurationKeys.DEFAULT_CONTAINER_JVM_MEMORY_OVERHEAD_MBS); + + Preconditions.checkArgument(this.jvmMemoryOverheadMbs < this.requestedContainerMemoryMbs * this.jvmMemoryXmxRatio, + GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY + " cannot be more than " + + GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY + " * " + + GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY); + + this.appViewAcl = ConfigUtils.getString(this.config, GobblinYarnConfigurationKeys.APP_VIEW_ACL, + GobblinYarnConfigurationKeys.DEFAULT_APP_VIEW_ACL); + this.containerTimezone = ConfigUtils.getString(this.config, GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_TIMEZONE, + GobblinYarnConfigurationKeys.DEFAULT_GOBBLIN_YARN_CONTAINER_TIMEZONE); + } + + @SuppressWarnings("unused") + @Subscribe + public void handleNewContainerRequest(NewContainerRequest newContainerRequest) { + if (!this.maxResourceCapacity.isPresent()) { + LOGGER.error(String.format( + "Unable to handle new container request as maximum resource capacity is not available: " + + "[memory (MBs) requested = %d, vcores requested = %d]", this.requestedContainerMemoryMbs, + this.requestedContainerCores)); + return; + } + requestContainer(newContainerRequest.getReplacedContainer().transform(container -> container.getNodeId().getHost()), + newContainerRequest.getResource()); + } + + protected NMClientCallbackHandler getNMClientCallbackHandler() { + return new NMClientCallbackHandler(); + } + + @SuppressWarnings("unused") + @Subscribe + public void handleContainerShutdownRequest(ContainerShutdownRequest containerShutdownRequest) { + for (Container container : containerShutdownRequest.getContainers()) { + LOGGER.info(String.format("Stopping container %s running on %s", container.getId(), container.getNodeId())); + this.nmClientAsync.stopContainerAsync(container.getId(), container.getNodeId()); + } + } + + /** + * Request the Resource Manager to release the container + * @param containerReleaseRequest containers to release + */ + @Subscribe + public void handleContainerReleaseRequest(ContainerReleaseRequest containerReleaseRequest) { + for (Container container : containerReleaseRequest.getContainers()) { + LOGGER.info(String.format("Releasing container %s running on %s", container.getId(), container.getNodeId())); + + // Record that this container was explicitly released so that a new one is not spawned to replace it + // Put the container id in the releasedContainerCache before releasing it so that handleContainerCompletion() + // can check for the container id and skip spawning a replacement container. + // Note that this is the best effort since these are asynchronous operations and a container may abort concurrently + // with the release call. So in some cases a replacement container may have already been spawned before + // the container is put into the black list. + this.releasedContainerCache.put(container.getId(), ""); + this.amrmClientAsync.releaseAssignedContainer(container.getId()); + } + } + + @Override + protected synchronized void startUp() throws Exception { + LOGGER.info("Starting the TemporalYarnService"); + + // Register itself with the EventBus for container-related requests + this.eventBus.register(this); + + this.amrmClientAsync.start(); + this.nmClientAsync.start(); + + // The ApplicationMaster registration response is used to determine the maximum resource capacity of the cluster + RegisterApplicationMasterResponse response = this.amrmClientAsync.registerApplicationMaster( + GobblinClusterUtils.getHostname(), -1, ""); + LOGGER.info("ApplicationMaster registration response: " + response); + this.maxResourceCapacity = Optional.of(response.getMaximumResourceCapability()); + + LOGGER.info("Requesting initial containers"); + requestInitialContainers(this.initialContainers); + } + + @Override + protected void shutDown() throws IOException { + LOGGER.info("Stopping the TemporalYarnService"); + + this.shutdownInProgress = true; + + try { + ExecutorsUtils.shutdownExecutorService(this.containerLaunchExecutor, Optional.of(LOGGER)); + + // Stop the running containers + for (ContainerInfo containerInfo : this.containerMap.values()) { + LOGGER.info("Stopping container {} running participant {}", containerInfo.getContainer().getId(), + containerInfo.getHelixParticipantId()); + this.nmClientAsync.stopContainerAsync(containerInfo.getContainer().getId(), containerInfo.getContainer().getNodeId()); + } + + if (!this.containerMap.isEmpty()) { + synchronized (this.allContainersStopped) { + try { + // Wait 5 minutes for the containers to stop + Duration waitTimeout = Duration.ofMinutes(5); + this.allContainersStopped.wait(waitTimeout.toMillis()); + LOGGER.info("All of the containers have been stopped"); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + } + } + } + + this.amrmClientAsync.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, null, null); + } catch (IOException | YarnException e) { + LOGGER.error("Failed to unregister the ApplicationMaster", e); + } finally { + try { + this.closer.close(); + } finally { + if (this.gobblinMetrics.isPresent()) { + this.gobblinMetrics.get().stopMetricsReporting(); + } + } + } + } + + public void updateToken() throws IOException{ + this.tokens = getSecurityTokens(); + } + + private GobblinMetrics buildGobblinMetrics() { + // Create tags list + ImmutableList.Builder> tags = new ImmutableList.Builder<>(); + tags.add(new Tag<>(GobblinClusterMetricTagNames.APPLICATION_ID, this.applicationId)); + tags.add(new Tag<>(GobblinClusterMetricTagNames.APPLICATION_NAME, this.applicationName)); + + // Intialize Gobblin metrics and start reporters + GobblinMetrics gobblinMetrics = GobblinMetrics.get(this.applicationId, null, tags.build()); + try { + gobblinMetrics.startMetricReporting(ConfigUtils.configToProperties(config)); + } catch (MultiReporterException ex) { + for (MetricReporterException e: ex.getExceptions()) { + LOGGER.error("Failed to start {} {} reporter.", e.getSinkType().name(), e.getReporterType().name(), e); + } + } + + return gobblinMetrics; + } + + private EventSubmitter buildEventSubmitter() { + return new EventSubmitter.Builder(this.gobblinMetrics.get().getMetricContext(), + GobblinYarnEventConstants.EVENT_NAMESPACE) + .build(); + } + + /** + * Request an allocation of containers. If numTargetContainers is larger than the max of current and expected number + * of containers then additional containers are requested. + *

+ * If numTargetContainers is less than the current number of allocated containers then release free containers. + * Shrinking is relative to the number of currently allocated containers since it takes time for containers + * to be allocated and assigned work and we want to avoid releasing a container prematurely before it is assigned + * work. This means that a container may not be released even though numTargetContainers is less than the requested + * number of containers. The intended usage is for the caller of this method to make periodic calls to attempt to + * adjust the cluster towards the desired number of containers. + * + * @param yarnContainerRequestBundle the desired containers information, including numbers, resource and helix tag + * @param inUseInstances a set of in use instances + * @return whether successfully requested the target number of containers + */ + public synchronized boolean requestTargetNumberOfContainers(YarnContainerRequestBundle yarnContainerRequestBundle, Set inUseInstances) { + LOGGER.info("Trying to set numTargetContainers={}, in-use helix instances count is {}, container map size is {}", + yarnContainerRequestBundle.getTotalContainers(), inUseInstances.size(), this.containerMap.size()); + + int defaultContainerMemoryMbs = config.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); + int defaultContainerCores = config.getInt(GobblinYarnConfigurationKeys. CONTAINER_CORES_KEY); + int workerPoolSize = ConfigUtils.getInt(config, "temporal.workerpool.size",2); + requestContainers(workerPoolSize, Resource.newInstance(defaultContainerMemoryMbs, defaultContainerCores)); + + this.yarnContainerRequest = yarnContainerRequestBundle; + LOGGER.info("Current tag-container desired count:{}, tag-container allocated: {}", + yarnContainerRequestBundle.getHelixTagContainerCountMap(), this.allocatedContainerCountMap); + return true; + } + + // Request initial containers with default resource and helix tag + private void requestInitialContainers(int containersRequested) { + YarnContainerRequestBundle initialYarnContainerRequest = new YarnContainerRequestBundle(); + Resource capability = Resource.newInstance(this.requestedContainerMemoryMbs, this.requestedContainerCores); + initialYarnContainerRequest.add(this.helixInstanceTags, containersRequested, capability); + requestTargetNumberOfContainers(initialYarnContainerRequest, Collections.EMPTY_SET); + } + + private void requestContainer(Optional preferredNode, Optional resourceOptional) { + Resource desiredResource = resourceOptional.or(Resource.newInstance( + this.requestedContainerMemoryMbs, this.requestedContainerCores)); + requestContainer(preferredNode, desiredResource); + } + + /** + * Request {@param numContainers} from yarn with the specified resource. Resources will be allocated without a preferred + * node + * @param numContainers + * @param resource + */ + private void requestContainers(int numContainers, Resource resource) { + LOGGER.info("Requesting {} containers with resource={}", numContainers, resource); + IntStream.range(0, numContainers) + .forEach(i -> requestContainer(Optional.absent(), resource)); + } + + // Request containers with specific resource requirement + private void requestContainer(Optional preferredNode, Resource resource) { + // Fail if Yarn cannot meet container resource requirements + Preconditions.checkArgument(resource.getMemory() <= this.maxResourceCapacity.get().getMemory() && + resource.getVirtualCores() <= this.maxResourceCapacity.get().getVirtualCores(), + "Resource requirement must less than the max resource capacity. Requested resource" + resource.toString() + + " exceed the max resource limit " + this.maxResourceCapacity.get().toString()); + + // Due to YARN-314, different resource capacity needs different priority, otherwise Yarn will not allocate container + Priority priority = Records.newRecord(Priority.class); + if(!resourcePriorityMap.containsKey(resource.toString())) { + resourcePriorityMap.put(resource.toString(), priorityNumGenerator.getAndIncrement()); + } + int priorityNum = resourcePriorityMap.get(resource.toString()); + priority.setPriority(priorityNum); + + String[] preferredNodes = preferredNode.isPresent() ? new String[] {preferredNode.get()} : null; + this.amrmClientAsync.addContainerRequest( + new AMRMClient.ContainerRequest(resource, preferredNodes, null, priority)); + } + + protected ContainerLaunchContext newContainerLaunchContext(ContainerInfo containerInfo) + throws IOException { + Path appWorkDir = GobblinClusterUtils.getAppWorkDirPathFromConfig(this.config, this.fs, this.applicationName, this.applicationId); + Path containerWorkDir = new Path(appWorkDir, GobblinYarnConfigurationKeys.CONTAINER_WORK_DIR_NAME); + + Map resourceMap = Maps.newHashMap(); + + addContainerLocalResources(new Path(appWorkDir, GobblinYarnConfigurationKeys.LIB_JARS_DIR_NAME), resourceMap); + addContainerLocalResources(new Path(containerWorkDir, GobblinYarnConfigurationKeys.APP_JARS_DIR_NAME), resourceMap); + addContainerLocalResources( + new Path(containerWorkDir, GobblinYarnConfigurationKeys.APP_FILES_DIR_NAME), resourceMap); + + if (this.config.hasPath(GobblinYarnConfigurationKeys.CONTAINER_FILES_REMOTE_KEY)) { + YarnHelixUtils.addRemoteFilesToLocalResources(this.config.getString(GobblinYarnConfigurationKeys.CONTAINER_FILES_REMOTE_KEY), + resourceMap, yarnConfiguration); + } + if (this.config.hasPath(GobblinYarnConfigurationKeys.CONTAINER_ZIPS_REMOTE_KEY)) { + YarnHelixUtils.addRemoteZipsToLocalResources(this.config.getString(GobblinYarnConfigurationKeys.CONTAINER_ZIPS_REMOTE_KEY), + resourceMap, yarnConfiguration); + } + ContainerLaunchContext containerLaunchContext = Records.newRecord(ContainerLaunchContext.class); + containerLaunchContext.setLocalResources(resourceMap); + containerLaunchContext.setEnvironment(YarnHelixUtils.getEnvironmentVariables(this.yarnConfiguration)); + containerLaunchContext.setCommands(Arrays.asList(containerInfo.getStartupCommand())); + + Map acls = new HashMap<>(1); + acls.put(ApplicationAccessType.VIEW_APP, this.appViewAcl); + containerLaunchContext.setApplicationACLs(acls); + + if (UserGroupInformation.isSecurityEnabled()) { + containerLaunchContext.setTokens(this.tokens.duplicate()); + } + + return containerLaunchContext; + } + + private void addContainerLocalResources(Path destDir, Map resourceMap) throws IOException { + if (!this.fs.exists(destDir)) { + LOGGER.warn(String.format("Path %s does not exist so no container LocalResource to add", destDir)); + return; + } + + FileStatus[] statuses = this.fs.listStatus(destDir); + if (statuses != null) { + for (FileStatus status : statuses) { + YarnHelixUtils.addFileAsLocalResource(this.fs, status.getPath(), LocalResourceType.FILE, resourceMap); + } + } + } + + + protected ByteBuffer getSecurityTokens() throws IOException { + Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials(); + Closer closer = Closer.create(); + try { + DataOutputBuffer dataOutputBuffer = closer.register(new DataOutputBuffer()); + credentials.writeTokenStorageToStream(dataOutputBuffer); + + // Remove the AM->RM token so that containers cannot access it + Iterator> tokenIterator = credentials.getAllTokens().iterator(); + while (tokenIterator.hasNext()) { + Token token = tokenIterator.next(); + if (token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) { + tokenIterator.remove(); + } + } + + return ByteBuffer.wrap(dataOutputBuffer.getData(), 0, dataOutputBuffer.getLength()); + } catch (Throwable t) { + throw closer.rethrow(t); + } finally { + closer.close(); + } + } + + @VisibleForTesting + protected String buildContainerCommand(Container container, String helixParticipantId, String helixInstanceTag) { + String containerProcessName = GobblinTemporalYarnTaskRunner.class.getSimpleName(); + StringBuilder containerCommand = new StringBuilder() + .append(ApplicationConstants.Environment.JAVA_HOME.$()).append("/bin/java") + .append(" -Xmx").append((int) (container.getResource().getMemory() * this.jvmMemoryXmxRatio) - + this.jvmMemoryOverheadMbs).append("M") + .append(" -D").append(GobblinYarnConfigurationKeys.JVM_USER_TIMEZONE_CONFIG).append("=").append(this.containerTimezone) + .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_DIR_NAME).append("=").append(ApplicationConstants.LOG_DIR_EXPANSION_VAR) + .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_FILE_NAME).append("=").append(containerProcessName).append(".").append(ApplicationConstants.STDOUT) + .append(" ").append(JvmUtils.formatJvmArguments(this.containerJvmArgs)) + .append(" ").append(GobblinTemporalYarnTaskRunner.class.getName()) + .append(" --").append(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME) + .append(" ").append(this.applicationName) + .append(" --").append(GobblinClusterConfigurationKeys.APPLICATION_ID_OPTION_NAME) + .append(" ").append(this.applicationId) + .append(" --").append(GobblinClusterConfigurationKeys.HELIX_INSTANCE_NAME_OPTION_NAME) + .append(" ").append(helixParticipantId); + + if (!Strings.isNullOrEmpty(helixInstanceTag)) { + containerCommand.append(" --").append(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_OPTION_NAME) + .append(" ").append(helixInstanceTag); + } + + LOGGER.info("Building " + containerProcessName); + return containerCommand.append(" 1>").append(ApplicationConstants.LOG_DIR_EXPANSION_VAR).append(File.separator).append( + containerProcessName).append(".").append(ApplicationConstants.STDOUT) + .append(" 2>").append(ApplicationConstants.LOG_DIR_EXPANSION_VAR).append(File.separator).append( + containerProcessName).append(".").append(ApplicationConstants.STDERR).toString(); + } + + /** + * Check the exit status of a completed container and see if the replacement container + * should try to be started on the same node. Some exit status indicates a disk or + * node failure and in such cases the replacement container should try to be started on + * a different node. + */ + private boolean shouldStickToTheSameNode(int containerExitStatus) { + switch (containerExitStatus) { + case ContainerExitStatus.DISKS_FAILED: + return false; + case ContainerExitStatus.ABORTED: + // Mostly likely this exit status is due to node failures because the + // application itself will not release containers. + return false; + default: + // Stick to the same node for other cases if host affinity is enabled. + return this.containerHostAffinityEnabled; + } + } + + /** + * Handle the completion of a container. A new container will be requested to replace the one + * that just exited. Depending on the exit status and if container host affinity is enabled, + * the new container may or may not try to be started on the same node. + *

+ * A container completes in either of the following conditions: 1) some error happens in the + * container and caused the container to exit, 2) the container gets killed due to some reason, + * for example, if it runs over the allowed amount of virtual or physical memory, 3) the gets + * preempted by the ResourceManager, or 4) the container gets stopped by the ApplicationMaster. + * A replacement container is needed in all but the last case. + */ + protected void handleContainerCompletion(ContainerStatus containerStatus) { + ContainerInfo completedContainerInfo = this.containerMap.remove(containerStatus.getContainerId()); + //Get the Helix instance name for the completed container. Because callbacks are processed asynchronously, we might + //encounter situations where handleContainerCompletion() is called before onContainersAllocated(), resulting in the + //containerId missing from the containersMap. + // We use removedContainerID to remember these containers and remove them from containerMap later when we call requestTargetNumberOfContainers method + if (completedContainerInfo == null) { + removedContainerID.putIfAbsent(containerStatus.getContainerId(), ""); + } + String completedInstanceName = UNKNOWN_HELIX_INSTANCE; + + String helixTag = completedContainerInfo == null ? helixInstanceTags : completedContainerInfo.getHelixTag(); + if (completedContainerInfo != null) { + allocatedContainerCountMap.get(helixTag).decrementAndGet(); + } + + LOGGER.info(String.format("Container %s running Helix instance %s with tag %s has completed with exit status %d", + containerStatus.getContainerId(), completedInstanceName, helixTag, containerStatus.getExitStatus())); + + if (!Strings.isNullOrEmpty(containerStatus.getDiagnostics())) { + LOGGER.info(String.format("Received the following diagnostics information for container %s: %s", + containerStatus.getContainerId(), containerStatus.getDiagnostics())); + } + + switch(containerStatus.getExitStatus()) { + case(ContainerExitStatus.ABORTED): + if (handleAbortedContainer(containerStatus, completedContainerInfo, completedInstanceName)) { + return; + } + break; + case(1): // Same as linux exit status 1 Often occurs when launch_container.sh failed + LOGGER.info("Exit status 1. CompletedContainerInfo={}", completedContainerInfo); + break; + default: + break; + } + + if (this.shutdownInProgress) { + return; + } + if(completedContainerInfo != null) { + this.helixInstanceRetryCount.putIfAbsent(completedInstanceName, new AtomicInteger(0)); + int retryCount = this.helixInstanceRetryCount.get(completedInstanceName).incrementAndGet(); + + // Populate event metadata + Optional> eventMetadataBuilder = Optional.absent(); + if (this.eventSubmitter.isPresent()) { + eventMetadataBuilder = Optional.of(buildContainerStatusEventMetadata(containerStatus)); + eventMetadataBuilder.get().put(GobblinYarnEventConstants.EventMetadata.HELIX_INSTANCE_ID, completedInstanceName); + eventMetadataBuilder.get().put(GobblinYarnEventConstants.EventMetadata.CONTAINER_STATUS_RETRY_ATTEMPT, retryCount + ""); + } + + if (this.helixInstanceMaxRetries > 0 && retryCount > this.helixInstanceMaxRetries) { + if (this.eventSubmitter.isPresent()) { + this.eventSubmitter.get() + .submit(GobblinYarnEventConstants.EventNames.HELIX_INSTANCE_COMPLETION, eventMetadataBuilder.get().build()); + } + + LOGGER.warn("Maximum number of retries has been achieved for Helix instance " + completedInstanceName); + return; + } + + // Add the Helix instance name of the completed container to the set of unused + // instance names so they can be reused by a replacement container. + LOGGER.info("Adding instance {} to the pool of unused instances", completedInstanceName); + this.unusedHelixInstanceNames.add(completedInstanceName); + + if (this.eventSubmitter.isPresent()) { + this.eventSubmitter.get() + .submit(GobblinYarnEventConstants.EventNames.HELIX_INSTANCE_COMPLETION, eventMetadataBuilder.get().build()); + } + } + Optional newContainerResource = completedContainerInfo != null ? + Optional.of(completedContainerInfo.getContainer().getResource()) : Optional.absent(); + LOGGER.info("Requesting a new container to replace {} to run Helix instance {} with helix tag {} and resource {}", + containerStatus.getContainerId(), completedInstanceName, helixTag, newContainerResource.orNull()); + this.eventBus.post(new NewContainerRequest( + shouldStickToTheSameNode(containerStatus.getExitStatus()) && completedContainerInfo != null ? + Optional.of(completedContainerInfo.getContainer()) : Optional.absent(), newContainerResource)); + } + + /** + * Handles containers aborted. This method handles 2 cases: + *

    + *
  1. + * Case 1: Gobblin AM intentionally requested container to be released (often because the number of helix tasks + * has decreased due to decreased traffic) + *
  2. + *
  3. + * Case 2: Unexpected hardware fault and the node is lost. Need to do specific Helix logic to ensure 2 helix tasks + * are not being run by the multiple containers + *
  4. + *
+ * @param containerStatus + * @param completedContainerInfo + * @param completedInstanceName + * @return if release request was intentionally released (Case 1) + */ + private boolean handleAbortedContainer(ContainerStatus containerStatus, ContainerInfo completedContainerInfo, + String completedInstanceName) { + + // Case 1: Container intentionally released + if (this.releasedContainerCache.getIfPresent(containerStatus.getContainerId()) != null) { + LOGGER.info("Container release requested, so not spawning a replacement for containerId {}", containerStatus.getContainerId()); + if (completedContainerInfo != null) { + LOGGER.info("Adding instance {} to the pool of unused instances", completedInstanceName); + this.unusedHelixInstanceNames.add(completedInstanceName); + } + + return true; + } + + // Case 2: Container release was not requested. Likely, the container was running on a node on which the NM died. + // In this case, RM assumes that the containers are "lost", even though the container process may still be + // running on the node. We need to ensure that the Helix instances running on the orphaned containers + // are fenced off from the Helix cluster to avoid double publishing and state being committed by the + // instances. + LOGGER.info("Container {} aborted due to lost NM", containerStatus.getContainerId()); + return false; + } + + private ImmutableMap.Builder buildContainerStatusEventMetadata(ContainerStatus containerStatus) { + ImmutableMap.Builder eventMetadataBuilder = new ImmutableMap.Builder<>(); + eventMetadataBuilder.put(GobblinYarnMetricTagNames.CONTAINER_ID, containerStatus.getContainerId().toString()); + eventMetadataBuilder.put(GobblinYarnEventConstants.EventMetadata.CONTAINER_STATUS_CONTAINER_STATE, + containerStatus.getState().toString()); + if (ContainerExitStatus.INVALID != containerStatus.getExitStatus()) { + eventMetadataBuilder.put(GobblinYarnEventConstants.EventMetadata.CONTAINER_STATUS_EXIT_STATUS, + containerStatus.getExitStatus() + ""); + } + if (!Strings.isNullOrEmpty(containerStatus.getDiagnostics())) { + eventMetadataBuilder.put(GobblinYarnEventConstants.EventMetadata.CONTAINER_STATUS_EXIT_DIAGNOSTICS, + containerStatus.getDiagnostics()); + } + + return eventMetadataBuilder; + } + + /** + * Get the number of matching container requests for the specified resource memory and cores. + * Due to YARN-1902 and YARN-660, this API is not 100% accurate. {@link AMRMClientCallbackHandler#onContainersAllocated(List)} + * contains logic for best effort clean up of requests, and the resource tend to match the allocated container. So in practice the count is pretty accurate. + *

+ * This API call gets the count of container requests for containers that are > resource if there is no request with the exact same resource + * The RM can return containers that are larger (because of normalization etc). + * Container may be larger by memory or cpu (e.g. container (1000M, 3cpu) can fit request (1000M, 1cpu) or request (500M, 3cpu). + *

+ * Thankfully since each helix tag / resource has a different priority, matching requests for one helix tag / resource + * have complete isolation from another helix tag / resource + */ + private int getMatchingRequestsCount(Resource resource) { + Integer priorityNum = resourcePriorityMap.get(resource.toString()); + if (priorityNum == null) { // request has never been made with this resource + return 0; + } + Priority priority = Priority.newInstance(priorityNum); + + // Each collection in the list represents a set of requests with each with the same resource requirement. + // The reason for differing resources can be due to normalization + List> outstandingRequests = getAmrmClientAsync().getMatchingRequests(priority, ResourceRequest.ANY, resource); + return outstandingRequests == null ? 0 : outstandingRequests.stream() + .filter(Objects::nonNull) + .mapToInt(Collection::size) + .sum(); + } + + /** + * A custom implementation of {@link AMRMClientAsync.CallbackHandler}. + */ + private class AMRMClientCallbackHandler implements AMRMClientAsync.CallbackHandler { + + private volatile boolean done = false; + + @Override + public void onContainersCompleted(List statuses) { + for (ContainerStatus containerStatus : statuses) { + handleContainerCompletion(containerStatus); + } + } + + @Override + public void onContainersAllocated(List containers) { + for (final Container container : containers) { + String containerId = container.getId().toString(); + String containerHelixTag = YarnHelixUtils.findHelixTagForContainer(container, allocatedContainerCountMap, yarnContainerRequest); + if (Strings.isNullOrEmpty(containerHelixTag)) { + containerHelixTag = helixInstanceTags; + } + if (eventSubmitter.isPresent()) { + eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_ALLOCATION, + GobblinYarnMetricTagNames.CONTAINER_ID, containerId); + } + + LOGGER.info("Container {} has been allocated with resource {} for helix tag {}", + container.getId(), container.getResource(), containerHelixTag); + + //Iterate over the (thread-safe) set of unused instances to find the first instance that is not currently live. + //Once we find a candidate instance, it is removed from the set. + String instanceName = null; + + //Ensure that updates to unusedHelixInstanceNames are visible to other threads that might concurrently + //invoke the callback on container allocation. + synchronized (this) { + Iterator iterator = unusedHelixInstanceNames.iterator(); + while (iterator.hasNext()) { + instanceName = iterator.next(); + } + } + + ContainerInfo containerInfo = new ContainerInfo(container, instanceName, containerHelixTag); + containerMap.put(container.getId(), containerInfo); + allocatedContainerCountMap.putIfAbsent(containerHelixTag, new AtomicInteger(0)); + allocatedContainerCountMap.get(containerHelixTag).incrementAndGet(); + + // Find matching requests and remove the request (YARN-660). We the scheduler are responsible + // for cleaning up requests after allocation based on the design in the described ticket. + // YARN does not have a delta request API and the requests are not cleaned up automatically. + // Try finding a match first with the host as the resource name then fall back to any resource match. + // Also see YARN-1902. Container count will explode without this logic for removing container requests. + List> matchingRequests = amrmClientAsync + .getMatchingRequests(container.getPriority(), container.getNodeHttpAddress(), container.getResource()); + + if (matchingRequests.isEmpty()) { + LOGGER.debug("Matching request by host {} not found", container.getNodeHttpAddress()); + + matchingRequests = amrmClientAsync + .getMatchingRequests(container.getPriority(), ResourceRequest.ANY, container.getResource()); + } + + if (!matchingRequests.isEmpty()) { + AMRMClient.ContainerRequest firstMatchingContainerRequest = matchingRequests.get(0).iterator().next(); + LOGGER.debug("Found matching requests {}, removing first matching request {}", + matchingRequests, firstMatchingContainerRequest); + + amrmClientAsync.removeContainerRequest(firstMatchingContainerRequest); + } + + containerLaunchExecutor.submit(new Runnable() { + @Override + public void run() { + try { + LOGGER.info("Starting container " + containerId); + + nmClientAsync.startContainerAsync(container, newContainerLaunchContext(containerInfo)); + } catch (IOException ioe) { + LOGGER.error("Failed to start container " + containerId, ioe); + } + } + }); + } + } + + @Override + public void onShutdownRequest() { + if (eventSubmitter.isPresent()) { + eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.SHUTDOWN_REQUEST); + } + + LOGGER.info("Received shutdown request from the ResourceManager"); + this.done = true; + eventBus.post(new ClusterManagerShutdownRequest()); + } + + @Override + public void onNodesUpdated(List updatedNodes) { + for (NodeReport nodeReport : updatedNodes) { + LOGGER.info("Received node update report: " + nodeReport); + } + } + + @Override + public float getProgress() { + return this.done ? 1.0f : 0.0f; + } + + @Override + public void onError(Throwable t) { + if (eventSubmitter.isPresent()) { + eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.ERROR, + GobblinYarnEventConstants.EventMetadata.ERROR_EXCEPTION, Throwables.getStackTraceAsString(t)); + } + + LOGGER.error("Received error: " + t, t); + this.done = true; + eventBus.post(new ClusterManagerShutdownRequest()); + } + } + + /** + * A custom implementation of {@link NMClientAsync.CallbackHandler}. + */ + class NMClientCallbackHandler implements NMClientAsync.CallbackHandler { + + @Override + public void onContainerStarted(ContainerId containerId, Map allServiceResponse) { + if (eventSubmitter.isPresent()) { + eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_STARTED, + GobblinYarnMetricTagNames.CONTAINER_ID, containerId.toString()); + } + + LOGGER.info(String.format("Container %s has been started", containerId)); + } + + @Override + public void onContainerStatusReceived(ContainerId containerId, ContainerStatus containerStatus) { + if (eventSubmitter.isPresent()) { + eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_STATUS_RECEIVED, + buildContainerStatusEventMetadata(containerStatus).build()); + } + + LOGGER.info(String.format("Received container status for container %s: %s", containerId, containerStatus)); + } + + @Override + public void onContainerStopped(ContainerId containerId) { + if (eventSubmitter.isPresent()) { + eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_STOPPED, + GobblinYarnMetricTagNames.CONTAINER_ID, containerId.toString()); + } + + LOGGER.info(String.format("Container %s has been stopped", containerId)); + if (containerMap.isEmpty()) { + synchronized (allContainersStopped) { + allContainersStopped.notify(); + } + } + } + + @Override + public void onStartContainerError(ContainerId containerId, Throwable t) { + if (eventSubmitter.isPresent()) { + eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_START_ERROR, + GobblinYarnMetricTagNames.CONTAINER_ID, containerId.toString(), + GobblinYarnEventConstants.EventMetadata.ERROR_EXCEPTION, Throwables.getStackTraceAsString(t)); + } + + LOGGER.error(String.format("Failed to start container %s due to error %s", containerId, t)); + } + + @Override + public void onGetContainerStatusError(ContainerId containerId, Throwable t) { + if (eventSubmitter.isPresent()) { + eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_GET_STATUS_ERROR, + GobblinYarnMetricTagNames.CONTAINER_ID, containerId.toString(), + GobblinYarnEventConstants.EventMetadata.ERROR_EXCEPTION, Throwables.getStackTraceAsString(t)); + } + + LOGGER.error(String.format("Failed to get status for container %s due to error %s", containerId, t)); + } + + @Override + public void onStopContainerError(ContainerId containerId, Throwable t) { + if (eventSubmitter.isPresent()) { + eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_STOP_ERROR, + GobblinYarnMetricTagNames.CONTAINER_ID, containerId.toString(), + GobblinYarnEventConstants.EventMetadata.ERROR_EXCEPTION, Throwables.getStackTraceAsString(t)); + } + + LOGGER.error(String.format("Failed to stop container %s due to error %s", containerId, t)); + } + } + + // Class encapsulates Container instances, Helix participant IDs of the containers, Helix Tag, and + // initial startup command + @Getter + class ContainerInfo { + private final Container container; + private final String helixParticipantId; + private final String helixTag; + private final String startupCommand; + + public ContainerInfo(Container container, String helixParticipantId, String helixTag) { + this.container = container; + this.helixParticipantId = helixParticipantId; + this.helixTag = helixTag; + this.startupCommand = YarnTemporalService.this.buildContainerCommand(container, helixParticipantId, helixTag); + } + + @Override + public String toString() { + return String.format("ContainerInfo{ container=%s, helixParticipantId=%s, helixTag=%s, startupCommand=%s }", + container.getId(), helixParticipantId, helixTag, startupCommand); + } + } +} diff --git a/gradle/scripts/dependencyDefinitions.gradle b/gradle/scripts/dependencyDefinitions.gradle index ffc81c7352..a26183a192 100644 --- a/gradle/scripts/dependencyDefinitions.gradle +++ b/gradle/scripts/dependencyDefinitions.gradle @@ -101,7 +101,7 @@ ext.externalDependency = [ "confluentJsonSerializer": "io.confluent:kafka-json-serializer:" + confluentVersion, "zkClient": "com.101tec:zkclient:0.7", "quartz": "org.quartz-scheduler:quartz:2.2.3", - "temporal-sdk": "io.temporal:temporal-sdk:1.20.0", + "temporal-sdk": "io.temporal:temporal-sdk:1.18.1", "testng": "org.testng:testng:6.14.3", "junit": "junit:junit:4.13.2", "mockserver":"org.mock-server:mockserver-netty:3.10.4", @@ -215,6 +215,7 @@ ext.externalDependency = [ "postgresConnector": "org.postgresql:postgresql:42.1.4", "testContainers": "org.testcontainers:testcontainers:1.17.3", "testContainersMysql": "org.testcontainers:mysql:1.17.3", + "workflowClient.namespace": "gobblin-fastingest-internpoc", "xz": "org.tukaani:xz:1.8" ] From 593676c13b51d7248bf30f495c70797655ae6dc4 Mon Sep 17 00:00:00 2001 From: Matthew Ho Date: Mon, 21 Aug 2023 15:45:13 -0700 Subject: [PATCH 4/8] Py milestone 3 (#8) * create temporal workflow from GobblinTemporalJobLauncher Still fixing temporal activities changed path settings changed workunitpath * fixed workunitpath * Add EventSubmitter * Clean up code and stop writing orc files * add MetricReporter in taskrunner and make worker size configurable * add workflowid in gobblin task * Add workerOptions to setMaxConcurrentWorkflowTaskExecutionSize --------- Co-authored-by: Peiyingy --- .../GobblinClusterConfigurationKeys.java | 1 + .../GobblinTemporalClusterManager.java | 95 +--- .../cluster/GobblinTemporalJobLauncher.java | 431 ++++++++++++++++++ .../cluster/GobblinTemporalJobScheduler.java | 313 +++++++++++++ .../cluster/GobblinTemporalTaskRunner.java | 104 ++++- .../apache/gobblin/cluster/SingleTask.java | 53 ++- .../temporal/GobblinTemporalActivity.java | 11 +- .../temporal/GobblinTemporalActivityImpl.java | 62 ++- .../temporal/GobblinTemporalWorkflow.java | 12 +- .../temporal/GobblinTemporalWorkflowImpl.java | 89 +++- .../gobblin/cluster/temporal/Shared.java | 6 +- .../apache/gobblin/metrics/MetricContext.java | 8 +- .../gobblin/metrics/event/EventSubmitter.java | 9 +- .../runtime/StreamModelTaskRunner.java | 12 +- .../GobblinTemporalApplicationMaster.java | 2 +- .../yarn/GobblinYarnConfigurationKeys.java | 1 + .../gobblin/yarn/YarnTemporalService.java | 38 +- gradle/scripts/dependencyDefinitions.gradle | 1 - 18 files changed, 1077 insertions(+), 171 deletions(-) create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobScheduler.java diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java index 31a8547aa9..f0dedec42b 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java @@ -222,4 +222,5 @@ public class GobblinClusterConfigurationKeys { public static final String CONTAINER_ID_KEY = GOBBLIN_HELIX_PREFIX + "containerId"; public static final String GOBBLIN_CLUSTER_SYSTEM_PROPERTY_PREFIX = GOBBLIN_CLUSTER_PREFIX + "sysProps"; + public static final String TEMPORAL_WORKER_SIZE = "temporal.worker.size"; } diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java index 4a910c09aa..232e4af233 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java @@ -52,10 +52,6 @@ import io.grpc.netty.shaded.io.grpc.netty.GrpcSslContexts; import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext; -import io.temporal.client.WorkflowClient; -import io.temporal.client.WorkflowClientOptions; -import io.temporal.client.WorkflowOptions; -import io.temporal.client.WorkflowStub; import io.temporal.serviceclient.WorkflowServiceStubs; import io.temporal.serviceclient.WorkflowServiceStubsOptions; import javax.net.ssl.KeyManagerFactory; @@ -67,8 +63,6 @@ import org.apache.gobblin.annotation.Alpha; import org.apache.gobblin.cluster.event.ClusterManagerShutdownRequest; -import org.apache.gobblin.cluster.temporal.GobblinTemporalWorkflow; -import org.apache.gobblin.cluster.temporal.Shared; import org.apache.gobblin.configuration.ConfigurationKeys; import org.apache.gobblin.instrumented.StandardMetricsBridge; import org.apache.gobblin.metrics.Tag; @@ -86,21 +80,6 @@ /** * The central cluster manager for Gobblin Clusters. - * - * - *

- * This class will initiates a graceful shutdown of the cluster in the following conditions: - * - *

    - *
  • A shutdown request is received via a Helix message of subtype - * {@link HelixMessageSubTypes#APPLICATION_MASTER_SHUTDOWN}. Upon receiving such a message, - * it will call {@link #stop()} to initiate a graceful shutdown of the cluster
  • - *
  • The shutdown hook gets called. The shutdown hook will call {@link #stop()}, which will - * start a graceful shutdown of the cluster.
  • - *
- *

- * - * @author Yinan Li */ @Alpha @Slf4j @@ -136,6 +115,8 @@ public class GobblinTemporalClusterManager implements ApplicationLauncher, Stand @Getter private JobConfigurationManager jobConfigurationManager; @Getter + private GobblinTemporalJobScheduler gobblinTemporalJobScheduler; + @Getter private volatile boolean started = false; protected final String clusterName; @@ -144,9 +125,8 @@ public class GobblinTemporalClusterManager implements ApplicationLauncher, Stand public GobblinTemporalClusterManager(String clusterName, String applicationId, Config sysConfig, Optional appWorkDirOptional) throws Exception { - // Set system properties passed in via application config. As an example, Helix uses System#getProperty() for ZK configuration + // Set system properties passed in via application config. // overrides such as sessionTimeout. In this case, the overrides specified - // in the application configuration have to be extracted and set before initializing HelixManager. GobblinClusterUtils.setSystemProperties(sysConfig); //Add dynamic config @@ -161,7 +141,7 @@ public GobblinTemporalClusterManager(String clusterName, String applicationId, C this.fs = GobblinClusterUtils.buildFileSystem(this.config, new Configuration()); this.appWorkDir = appWorkDirOptional.isPresent() ? appWorkDirOptional.get() : GobblinClusterUtils.getAppWorkDirPathFromConfig(this.config, this.fs, clusterName, applicationId); - LOGGER.info("Configured GobblinClusterManager work dir to: {}", this.appWorkDir); + LOGGER.info("Configured GobblinTemporalClusterManager work dir to: {}", this.appWorkDir); initializeAppLauncherAndServices(); } @@ -195,6 +175,9 @@ private void initializeAppLauncherAndServices() throws Exception { SchedulerService schedulerService = new SchedulerService(properties); this.applicationLauncher.addService(schedulerService); + this.gobblinTemporalJobScheduler = buildGobblinTemporalJobScheduler(config, this.appWorkDir, getMetadataTags(clusterName, applicationId), + schedulerService); + this.applicationLauncher.addService(this.gobblinTemporalJobScheduler); this.jobConfigurationManager = buildJobConfigurationManager(config); this.applicationLauncher.addService(this.jobConfigurationManager); @@ -234,7 +217,7 @@ private void stopAppLauncherAndServices() { /** - * Start the Gobblin Cluster Manager. + * Start the Gobblin Temporal Cluster Manager. */ // @Import(clazz = ClientSslContextFactory.class, prefix = ClientSslContextFactory.SCOPE_PREFIX) @Override @@ -263,49 +246,12 @@ public void run() { this.idleProcessThread.start(); // Need this in case a kill is issued to the process so that the idle thread does not keep the process up - // since GobblinClusterManager.stop() is not called this case. + // since GobblinTemporalClusterManager.stop() is not called this case. Runtime.getRuntime().addShutdownHook(new Thread(() -> GobblinTemporalClusterManager.this.stopIdleProcessThread = true)); } else { startAppLauncherAndServices(); } this.started = true; - - try { - initiateWorkflow(); - }catch (Exception e) { - throw new RuntimeException(e); - } - } - - public void initiateWorkflow() - throws Exception { - LOGGER.info("Initiating Temporal Workflow"); - WorkflowServiceStubs workflowServiceStubs = createServiceStubs(); - WorkflowClient client = - WorkflowClient.newInstance( - workflowServiceStubs, WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build()); - - /* - * Set Workflow options such as WorkflowId and Task Queue so the worker knows where to list and which workflows to execute. - */ - WorkflowOptions options = WorkflowOptions.newBuilder() - .setTaskQueue(Shared.HELLO_WORLD_TASK_QUEUE) - .build(); - - // Create the workflow client stub. It is used to start our workflow execution. - GobblinTemporalWorkflow workflow = client.newWorkflowStub(GobblinTemporalWorkflow.class, options); - - /* - * Execute our workflow and wait for it to complete. The call to our getGreeting method is - * synchronous. - * - * Replace the parameter "World" in the call to getGreeting() with your name. - */ - String greeting = workflow.getGreeting("World"); - - String workflowId = WorkflowStub.fromTyped(workflow).getExecution().getWorkflowId(); - // Display workflow execution results - LOGGER.info(workflowId + " " + greeting); } public static WorkflowServiceStubs createServiceStubs() @@ -351,7 +297,6 @@ public static WorkflowServiceStubs createServiceStubs() // Set trust manager from trust store KeyStore trustStore = KeyStore.getInstance("JKS"); File trustStoreFile = new File(config.getString(SSL_TRUSTSTORE_LOCATION)); - LOGGER.info("SSL_TRUSTSTORE_LOCATION " + config.getString(SSL_TRUSTSTORE_LOCATION)); String trustStorePassword = config.getString(SSL_TRUSTSTORE_PASSWORD); trustStore.load(toInputStream(trustStoreFile), trustStorePassword.toCharArray()); @@ -365,8 +310,6 @@ public static WorkflowServiceStubs createServiceStubs() .ciphers(SSL_CONFIG_DEFAULT_CIPHER_SUITES) .build(); - LOGGER.info("SSLContext: " + sslContext); - return WorkflowServiceStubs.newServiceStubs( WorkflowServiceStubsOptions.newBuilder() .setTarget("1.nephos-temporal.corp-lca1.atd.corp.linkedin.com:7233") @@ -375,6 +318,7 @@ public static WorkflowServiceStubs createServiceStubs() .build()); } + /** * Stop the Gobblin Cluster Manager. */ @@ -400,9 +344,15 @@ public synchronized void stop() { } - /** - * Get additional {@link Tag}s required for any type of reporting. - */ + private GobblinTemporalJobScheduler buildGobblinTemporalJobScheduler(Config sysConfig, Path appWorkDir, + List> metadataTags, SchedulerService schedulerService) throws Exception { + return new GobblinTemporalJobScheduler(sysConfig, + this.eventBus, + appWorkDir, + metadataTags, + schedulerService); + } + private List> getMetadataTags(String applicationName, String applicationId) { return Tag.fromMap( new ImmutableMap.Builder().put(GobblinClusterMetricTagNames.APPLICATION_NAME, applicationName) @@ -448,8 +398,7 @@ public Collection getStandardMetricsCollection() { /** * TODO for now the cluster id is hardcoded to 1 both here and in the {@link GobblinTaskRunner}. In the future, the - * cluster id should be created by the {@link GobblinTemporalClusterManager} and passed to each {@link GobblinTaskRunner} via - * Helix (at least that would be the easiest approach, there are certainly others ways to do it). + * cluster id should be created by the {@link GobblinTemporalClusterManager} and passed to each {@link GobblinTaskRunner} */ private static String getApplicationId() { return "1"; @@ -496,10 +445,10 @@ public static void main(String[] args) throws Exception { ConfigValueFactory.fromAnyRef(true)); } - try (GobblinTemporalClusterManager gobblinClusterManager = new GobblinTemporalClusterManager( + try (GobblinTemporalClusterManager GobblinTemporalClusterManager = new GobblinTemporalClusterManager( cmd.getOptionValue(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME), getApplicationId(), config, Optional.absent())) { - gobblinClusterManager.start(); + GobblinTemporalClusterManager.start(); } } catch (ParseException pe) { printUsage(options); diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java new file mode 100644 index 0000000000..5f79e596fc --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java @@ -0,0 +1,431 @@ +/* + * 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.gobblin.cluster; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigValueFactory; + +import io.temporal.client.WorkflowClient; +import io.temporal.client.WorkflowClientOptions; +import io.temporal.client.WorkflowOptions; +import io.temporal.serviceclient.WorkflowServiceStubs; +import javax.annotation.Nullable; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.cluster.temporal.GobblinTemporalWorkflow; +import org.apache.gobblin.cluster.temporal.Shared; +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.metastore.StateStore; +import org.apache.gobblin.metrics.Tag; +import org.apache.gobblin.metrics.event.CountEventBuilder; +import org.apache.gobblin.metrics.event.JobEvent; +import org.apache.gobblin.metrics.event.TimingEvent; +import org.apache.gobblin.rest.LauncherTypeEnum; +import org.apache.gobblin.runtime.AbstractJobLauncher; +import org.apache.gobblin.runtime.JobException; +import org.apache.gobblin.runtime.JobLauncher; +import org.apache.gobblin.runtime.JobState; +import org.apache.gobblin.runtime.TaskStateCollectorService; +import org.apache.gobblin.runtime.listeners.JobListener; +import org.apache.gobblin.runtime.util.StateStores; +import org.apache.gobblin.source.extractor.extract.kafka.KafkaSource; +import org.apache.gobblin.source.workunit.MultiWorkUnit; +import org.apache.gobblin.source.workunit.WorkUnit; +import org.apache.gobblin.util.ConfigUtils; +import org.apache.gobblin.util.Id; +import org.apache.gobblin.util.JobLauncherUtils; +import org.apache.gobblin.util.ParallelRunner; +import org.apache.gobblin.util.PropertiesUtils; +import org.apache.gobblin.util.SerializationUtils; + +import static org.apache.gobblin.cluster.GobblinTemporalClusterManager.createServiceStubs; + + +/** + * An implementation of {@link JobLauncher} that launches a Gobblin job using the Temporal task framework. + * + *

+ * Each {@link WorkUnit} of the job is persisted to the {@link FileSystem} of choice and the path to the file + * storing the serialized {@link WorkUnit} is passed to the Temporal task running the {@link WorkUnit} as a + * user-defined property {@link GobblinClusterConfigurationKeys#WORK_UNIT_FILE_PATH}. Upon startup, the gobblin + * task reads the property for the file path and de-serializes the {@link WorkUnit} from the file. + *

+ * + *

+ * This class is instantiated by the {@link GobblinTemporalJobScheduler} on every job submission to launch the Gobblin job. + * The actual task execution happens in the {@link GobblinTemporalTaskRunner}, usually in a different process. + *

+ */ +@Alpha +@Slf4j +public class GobblinTemporalJobLauncher extends AbstractJobLauncher { + + private static final Logger LOGGER = LoggerFactory.getLogger(GobblinTemporalJobLauncher.class); + + private static final String WORK_UNIT_FILE_EXTENSION = ".wu"; + + private final FileSystem fs; + private final Path appWorkDir; + private final Path inputWorkUnitDir; + private final Path outputTaskStateDir; + + // Number of ParallelRunner threads to be used for state serialization/deserialization + private final int stateSerDeRunnerThreads; + + private final TaskStateCollectorService taskStateCollectorService; + private final ConcurrentHashMap runningMap; + @Getter + private final StateStores stateStores; + + private WorkflowServiceStubs workflowServiceStubs; + private WorkflowClient client; + + public GobblinTemporalJobLauncher(Properties jobProps, Path appWorkDir, + List> metadataTags, ConcurrentHashMap runningMap) + throws Exception { + super(jobProps, initBaseEventTags(jobProps, metadataTags)); + LOGGER.debug("GobblinTemporalJobLauncher: jobProps {}, appWorkDir {}", jobProps, appWorkDir); + this.runningMap = runningMap; + this.appWorkDir = appWorkDir; + this.inputWorkUnitDir = new Path(appWorkDir, GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME); + this.outputTaskStateDir = new Path(this.appWorkDir, + GobblinClusterConfigurationKeys.OUTPUT_TASK_STATE_DIR_NAME + Path.SEPARATOR + this.jobContext.getJobId()); + + this.jobContext.getJobState().setJobLauncherType(LauncherTypeEnum.CLUSTER); + + this.stateSerDeRunnerThreads = Integer.parseInt(jobProps.getProperty(ParallelRunner.PARALLEL_RUNNER_THREADS_KEY, + Integer.toString(ParallelRunner.DEFAULT_PARALLEL_RUNNER_THREADS))); + + Config stateStoreJobConfig = ConfigUtils.propertiesToConfig(jobProps) + .withValue(ConfigurationKeys.STATE_STORE_FS_URI_KEY, ConfigValueFactory.fromAnyRef( + new URI(appWorkDir.toUri().getScheme(), null, appWorkDir.toUri().getHost(), appWorkDir.toUri().getPort(), + "/", null, null).toString())); + + this.stateStores = + new StateStores(stateStoreJobConfig, appWorkDir, GobblinClusterConfigurationKeys.OUTPUT_TASK_STATE_DIR_NAME, + appWorkDir, GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME, appWorkDir, + GobblinClusterConfigurationKeys.JOB_STATE_DIR_NAME); + + URI fsUri = URI.create(jobProps.getProperty(ConfigurationKeys.FS_URI_KEY, ConfigurationKeys.LOCAL_FS_URI)); + this.fs = FileSystem.get(fsUri, new Configuration()); + + this.taskStateCollectorService = + new TaskStateCollectorService(jobProps, this.jobContext.getJobState(), this.eventBus, this.eventSubmitter, + this.stateStores.getTaskStateStore(), this.outputTaskStateDir, this.getIssueRepository()); + + this.workflowServiceStubs = createServiceStubs(); + this.client = WorkflowClient.newInstance( + workflowServiceStubs, WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build()); + + /* + * Set Workflow options such as WorkflowId and Task Queue so the worker knows where to list and which workflows to execute. + */ + startCancellationExecutor(); + } + + @Override + public void close() throws IOException { + try { + executeCancellation(); + } finally { + super.close(); + } + } + + public String getJobId() { + return this.jobContext.getJobId(); + } + + @Override + protected void runWorkUnits(List workUnits) throws Exception { + try { + CountEventBuilder countEventBuilder = new CountEventBuilder(JobEvent.WORK_UNITS_CREATED, workUnits.size()); + this.eventSubmitter.submit(countEventBuilder); + LOGGER.info("Emitting WorkUnitsCreated Count: " + countEventBuilder.getCount()); + + long workUnitStartTime = System.currentTimeMillis(); + workUnits.forEach((k) -> k.setProp(ConfigurationKeys.WORK_UNIT_CREATION_TIME_IN_MILLIS, workUnitStartTime)); + + // Start the output TaskState collector service + this.taskStateCollectorService.startAsync().awaitRunning(); + + TimingEvent jobSubmissionTimer = + this.eventSubmitter.getTimingEvent(TimingEvent.RunJobTimings.HELIX_JOB_SUBMISSION); + + if (!this.cancellationRequested) { + submitJobToTemporal(workUnits); + jobSubmissionTimer.stop(); + LOGGER.info(String.format("Submitted job %s to Temporal", this.jobContext.getJobId())); + } else { + LOGGER.warn("Job {} not submitted to Temporal as it was requested to be cancelled.", this.jobContext.getJobId()); + } + + TimingEvent jobRunTimer = this.eventSubmitter.getTimingEvent(TimingEvent.RunJobTimings.HELIX_JOB_RUN); + jobRunTimer.stop(); + LOGGER.info(String.format("Job %s completed", this.jobContext.getJobId())); + } finally { + // The last iteration of output TaskState collecting will run when the collector service gets stopped + this.taskStateCollectorService.stopAsync().awaitTerminated(); + cleanupWorkingDirectory(); + } + } + + @Override + protected void executeCancellation() { + LOGGER.info("Cancel temporal workflow"); + } + + protected void removeTasksFromCurrentJob(List workUnitIdsToRemove) { + LOGGER.info("Temporal removeTasksFromCurrentJob"); + } + + protected void addTasksToCurrentJob(List workUnitsToAdd) { + LOGGER.info("Temporal addTasksToCurrentJob"); + } + + /** + * Submit a job to run. + */ + private void submitJobToTemporal(List workUnits) throws Exception{ + try (ParallelRunner stateSerDeRunner = new ParallelRunner(this.stateSerDeRunnerThreads, this.fs)) { + Path jobStateFilePath; + + // write the job.state using the state store if present, otherwise serialize directly to the file + if (this.stateStores.haveJobStateStore()) { + jobStateFilePath = GobblinClusterUtils.getJobStateFilePath(true, this.appWorkDir, this.jobContext.getJobId()); + this.stateStores.getJobStateStore() + .put(jobStateFilePath.getParent().getName(), jobStateFilePath.getName(), this.jobContext.getJobState()); + } else { + jobStateFilePath = GobblinClusterUtils.getJobStateFilePath(false, this.appWorkDir, this.jobContext.getJobId()); + SerializationUtils.serializeState(this.fs, jobStateFilePath, this.jobContext.getJobState()); + } + + // Block on persistence of all workunits to be finished. + stateSerDeRunner.waitForTasks(Long.MAX_VALUE); + + LOGGER.debug("GobblinTemporalJobLauncher.createTemporalJob: jobStateFilePath {}, jobState {} jobProperties {}", + jobStateFilePath, this.jobContext.getJobState().toString(), this.jobContext.getJobState().getProperties()); + + String jobStateFilePathStr = jobStateFilePath.toString(); + + List> futures = new ArrayList<>(); + AtomicInteger multiTaskIdSequence = new AtomicInteger(0); + AtomicInteger workflowCount = new AtomicInteger(0); + int workflowSize = 100; + ExecutorService executor = Executors.newFixedThreadPool(workflowSize); + + for (int i = 0; i < workflowSize; i++) { + WorkUnit workUnit = workUnits.get(i); + futures.add(CompletableFuture.runAsync(() -> { + try { + if (workUnit instanceof MultiWorkUnit) { + workUnit.setId(JobLauncherUtils.newMultiTaskId(this.jobContext.getJobId(), multiTaskIdSequence.getAndIncrement())); + } + String workUnitFilePathStr = persistWorkUnit(new Path(this.inputWorkUnitDir, this.jobContext.getJobId()), workUnit, stateSerDeRunner); + String workflowId = workUnit.getProp(KafkaSource.TOPIC_NAME) + "_" + workflowCount.getAndIncrement(); + WorkflowOptions options = WorkflowOptions.newBuilder() + .setTaskQueue(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE) + .setWorkflowId(workflowId) + .build(); + GobblinTemporalWorkflow workflow = this.client.newWorkflowStub(GobblinTemporalWorkflow.class, options); + LOGGER.info("Setting up temporal workflow {}", workflowId); + workflow.runTask(jobProps, appWorkDir.toString(), getJobId(), workUnitFilePathStr, jobStateFilePathStr); + } catch (Exception e) { + throw new RuntimeException(e); + } + }, executor)); + } + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).join(); + } + } + + public void launchJob(@Nullable JobListener jobListener) throws JobException { + LOGGER.info("Launching Temporal Job"); + boolean isLaunched = false; + this.runningMap.putIfAbsent(this.jobContext.getJobName(), false); + + Throwable errorInJobLaunching = null; + try { + if (this.runningMap.replace(this.jobContext.getJobName(), false, true)) { + LOGGER.info("Job {} will be executed, add into running map.", this.jobContext.getJobId()); + isLaunched = true; + launchJobImpl(jobListener); + } else { + LOGGER.warn("Job {} will not be executed because other jobs are still running.", this.jobContext.getJobId()); + } + + // TODO: Better error handling. The current impl swallows exceptions for jobs that were started by this method call. + // One potential way to improve the error handling is to make this error swallowing conifgurable + } catch (Throwable t) { + errorInJobLaunching = t; + } finally { + if (isLaunched) { + if (this.runningMap.replace(this.jobContext.getJobName(), true, false)) { + LOGGER.info("Job {} is done, remove from running map.", this.jobContext.getJobId()); + } else { + throw errorInJobLaunching == null ? new IllegalStateException( + "A launched job should have running state equal to true in the running map.") + : new RuntimeException("Failure in launching job:", errorInJobLaunching); + } + } + } + } + + + /** + * This method looks silly at first glance but exists for a reason. + * + * The method {@link GobblinTemporalJobLauncher#launchJob(JobListener)} contains boiler plate for handling exceptions and + * mutating the runningMap to communicate state back to the {@link GobblinTemporalJobScheduler}. The boiler plate swallows + * exceptions when launching the job because many use cases require that 1 job failure should not affect other jobs by causing the + * entire process to fail through an uncaught exception. + * + * This method is useful for unit testing edge cases where we expect {@link JobException}s during the underlying launch operation. + * It would be nice to not swallow exceptions, but the implications of doing that will require careful refactoring since + * the class {@link GobblinTemporalJobLauncher} and {@link GobblinTemporalJobScheduler} are shared for 2 quite different cases + * between GaaS and streaming. GaaS typically requiring many short lifetime workflows (where a failure is tolerated) and + * streaming requiring a small number of long running workflows (where failure to submit is unexpected and is not + * tolerated) + * + * @throws JobException + */ + @VisibleForTesting + void launchJobImpl(@Nullable JobListener jobListener) throws JobException { + super.launchJob(jobListener); + } + + /** + * Persist a single {@link WorkUnit} (flattened) to a file. + */ + private String persistWorkUnit(final Path workUnitFileDir, final WorkUnit workUnit, ParallelRunner stateSerDeRunner) { + final StateStore stateStore; + String workUnitFileName = workUnit.getId(); + + if (workUnit instanceof MultiWorkUnit) { + workUnitFileName += MULTI_WORK_UNIT_FILE_EXTENSION; + stateStore = stateStores.getMwuStateStore(); + } else { + workUnitFileName += WORK_UNIT_FILE_EXTENSION; + stateStore = stateStores.getWuStateStore(); + } + + Path workUnitFile = new Path(workUnitFileDir, workUnitFileName); + final String fileName = workUnitFile.getName(); + final String storeName = workUnitFile.getParent().getName(); + stateSerDeRunner.submitCallable(new Callable() { + @Override + public Void call() throws Exception { + stateStore.put(storeName, fileName, workUnit); + return null; + } + }, "Serialize state to store " + storeName + " file " + fileName); + + return workUnitFile.toString(); + } + + /** + * Delete persisted {@link WorkUnit}s and {@link JobState} upon job completion. + */ + private void cleanupWorkingDirectory() throws IOException { + LOGGER.info("Deleting persisted work units for job " + this.jobContext.getJobId()); + stateStores.getWuStateStore().delete(this.jobContext.getJobId()); + + // delete the directory that stores the task state files + stateStores.getTaskStateStore().delete(outputTaskStateDir.getName()); + + LOGGER.info("Deleting job state file for job " + this.jobContext.getJobId()); + + if (this.stateStores.haveJobStateStore()) { + this.stateStores.getJobStateStore().delete(this.jobContext.getJobId()); + } else { + Path jobStateFilePath = + GobblinClusterUtils.getJobStateFilePath(false, this.appWorkDir, this.jobContext.getJobId()); + this.fs.delete(jobStateFilePath, false); + } + } + + public static List> initBaseEventTags(Properties jobProps, + List> inputTags) { + List> metadataTags = Lists.newArrayList(inputTags); + String jobId; + + // generate job id if not already set + if (jobProps.containsKey(ConfigurationKeys.JOB_ID_KEY)) { + jobId = jobProps.getProperty(ConfigurationKeys.JOB_ID_KEY); + } else { + jobId = JobLauncherUtils.newJobId(JobState.getJobNameFromProps(jobProps), + PropertiesUtils.getPropAsLong(jobProps, ConfigurationKeys.FLOW_EXECUTION_ID_KEY, System.currentTimeMillis())); + jobProps.put(ConfigurationKeys.JOB_ID_KEY, jobId); + } + + String jobExecutionId = Long.toString(Id.Job.parse(jobId).getSequence()); + + // only inject flow tags if a flow name is defined + if (jobProps.containsKey(ConfigurationKeys.FLOW_NAME_KEY)) { + metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.FLOW_GROUP_FIELD, + jobProps.getProperty(ConfigurationKeys.FLOW_GROUP_KEY, ""))); + metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.FLOW_NAME_FIELD, + jobProps.getProperty(ConfigurationKeys.FLOW_NAME_KEY))); + + // use job execution id if flow execution id is not present + metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD, + jobProps.getProperty(ConfigurationKeys.FLOW_EXECUTION_ID_KEY, jobExecutionId))); + } + + if (jobProps.containsKey(ConfigurationKeys.JOB_CURRENT_ATTEMPTS)) { + metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.CURRENT_ATTEMPTS_FIELD, + jobProps.getProperty(ConfigurationKeys.JOB_CURRENT_ATTEMPTS, "1"))); + metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.CURRENT_GENERATION_FIELD, + jobProps.getProperty(ConfigurationKeys.JOB_CURRENT_GENERATION, "1"))); + metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.SHOULD_RETRY_FIELD, + "false")); + } + + metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.JOB_GROUP_FIELD, + jobProps.getProperty(ConfigurationKeys.JOB_GROUP_KEY, ""))); + metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.JOB_NAME_FIELD, + jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY, ""))); + metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.JOB_EXECUTION_ID_FIELD, jobExecutionId)); + + log.debug("AddAdditionalMetadataTags: metadataTags {}", metadataTags); + + return metadataTags; + } +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobScheduler.java new file mode 100644 index 0000000000..af00ca4995 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobScheduler.java @@ -0,0 +1,313 @@ +/* + * 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.gobblin.cluster; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import com.google.common.eventbus.EventBus; +import com.google.common.eventbus.Subscribe; +import com.typesafe.config.Config; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.cluster.event.CancelJobConfigArrivalEvent; +import org.apache.gobblin.cluster.event.DeleteJobConfigArrivalEvent; +import org.apache.gobblin.cluster.event.NewJobConfigArrivalEvent; +import org.apache.gobblin.cluster.event.UpdateJobConfigArrivalEvent; +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.instrumented.Instrumented; +import org.apache.gobblin.instrumented.StandardMetricsBridge; +import org.apache.gobblin.metrics.MetricContext; +import org.apache.gobblin.metrics.Tag; +import org.apache.gobblin.runtime.JobException; +import org.apache.gobblin.runtime.JobLauncher; +import org.apache.gobblin.runtime.listeners.JobListener; +import org.apache.gobblin.scheduler.JobScheduler; +import org.apache.gobblin.scheduler.SchedulerService; +import org.apache.gobblin.util.ConfigUtils; +import org.apache.gobblin.util.PathUtils; +import org.apache.gobblin.util.PropertiesUtils; + + +/** + * An extension to {@link JobScheduler} that schedules and runs + * Gobblin jobs on Temporal. + * + *

If the job should be launched from the scheduler node, + * {@link GobblinTemporalJobLauncher} is invoked. + * + */ +@Alpha +public class GobblinTemporalJobScheduler extends JobScheduler implements StandardMetricsBridge { + + private static final Logger LOGGER = LoggerFactory.getLogger(GobblinTemporalJobScheduler.class); + private static final String COMMON_JOB_PROPS = "gobblin.common.job.props"; + + private final Properties commonJobProperties; + private final EventBus eventBus; + private final Path appWorkDir; + private final List> metadataTags; + private final ConcurrentHashMap jobRunningMap; + private final MetricContext metricContext; + final GobblinHelixJobSchedulerMetrics jobSchedulerMetrics; + final GobblinHelixJobLauncherMetrics launcherMetrics; + final GobblinHelixPlanningJobLauncherMetrics planningJobLauncherMetrics; + final HelixJobsMapping jobsMapping; + private boolean startServicesCompleted; + + public GobblinTemporalJobScheduler(Config sysConfig, + EventBus eventBus, + Path appWorkDir, List> metadataTags, + SchedulerService schedulerService) throws Exception { + + super(ConfigUtils.configToProperties(sysConfig), schedulerService); + this.commonJobProperties = ConfigUtils.configToProperties(ConfigUtils.getConfigOrEmpty(sysConfig, COMMON_JOB_PROPS)); + this.eventBus = eventBus; + this.jobRunningMap = new ConcurrentHashMap<>(); + this.appWorkDir = appWorkDir; + this.metadataTags = metadataTags; + this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(properties), this.getClass()); + + int metricsWindowSizeInMin = ConfigUtils.getInt(sysConfig, + ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, + ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES); + + this.launcherMetrics = new GobblinHelixJobLauncherMetrics("launcherInScheduler", + this.metricContext, + metricsWindowSizeInMin); + + this.jobSchedulerMetrics = new GobblinHelixJobSchedulerMetrics(this.jobExecutor, + this.metricContext, + metricsWindowSizeInMin); + + this.jobsMapping = new HelixJobsMapping(ConfigUtils.propertiesToConfig(properties), + PathUtils.getRootPath(appWorkDir).toUri(), + appWorkDir.toString()); + + this.planningJobLauncherMetrics = new GobblinHelixPlanningJobLauncherMetrics("planningLauncherInScheduler", + this.metricContext, + metricsWindowSizeInMin, this.jobsMapping); + + this.startServicesCompleted = false; + } + + @Override + public Collection getStandardMetricsCollection() { + return ImmutableList.of(this.launcherMetrics, + this.jobSchedulerMetrics, + this.planningJobLauncherMetrics); + } + + @Override + protected void startUp() throws Exception { + this.eventBus.register(this); + super.startUp(); + this.startServicesCompleted = true; + } + + @Override + public void scheduleJob(Properties jobProps, JobListener jobListener) throws JobException { + try { + while (!startServicesCompleted) { + LOGGER.info("{} service is not fully up, waiting here...", this.getClass().getName()); + Thread.sleep(1000); + } + + scheduleJob(jobProps, + jobListener, + Maps.newHashMap(), + GobblinHelixJob.class); + + } catch (Exception e) { + throw new JobException("Failed to schedule job " + jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), e); + } + } + + @Override + protected void startServices() throws Exception { + + boolean cleanAllDistJobs = PropertiesUtils.getPropAsBoolean(this.properties, + GobblinClusterConfigurationKeys.CLEAN_ALL_DIST_JOBS, + String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CLEAN_ALL_DIST_JOBS)); + + if (cleanAllDistJobs) { + for (org.apache.gobblin.configuration.State state : this.jobsMapping.getAllStates()) { + String jobUri = state.getId(); + LOGGER.info("Delete mapping for job " + jobUri); + this.jobsMapping.deleteMapping(jobUri); + } + } + } + + @Override + public void runJob(Properties jobProps, JobListener jobListener) throws JobException { + } + + @Override + public GobblinTemporalJobLauncher buildJobLauncher(Properties jobProps) + throws Exception { + Properties combinedProps = new Properties(); + combinedProps.putAll(properties); + combinedProps.putAll(jobProps); + + return new GobblinTemporalJobLauncher(combinedProps, + this.appWorkDir, + this.metadataTags, + this.jobRunningMap); + } + + @Subscribe + public void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { + String jobUri = newJobArrival.getJobName(); + LOGGER.info("Received new job configuration of job " + jobUri); + try { + Properties jobProps = new Properties(); + jobProps.putAll(this.commonJobProperties); + jobProps.putAll(newJobArrival.getJobConfig()); + + // set uri so that we can delete this job later + jobProps.setProperty(GobblinClusterConfigurationKeys.JOB_SPEC_URI, jobUri); + + this.jobSchedulerMetrics.updateTimeBeforeJobScheduling(jobProps); + + if (jobProps.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)) { + LOGGER.info("Scheduling job " + jobUri); + scheduleJob(jobProps, + new GobblinHelixJobLauncherListener(this.launcherMetrics)); + } else { + LOGGER.info("No job schedule found, so running job " + jobUri); + GobblinHelixJobLauncherListener listener = new GobblinHelixJobLauncherListener(this.launcherMetrics); + JobLauncher launcher = buildJobLauncher(newJobArrival.getJobConfig()); + launcher.launchJob(listener); + } + } catch (Exception je) { + LOGGER.error("Failed to schedule or run job " + jobUri, je); + } + } + + @Subscribe + public void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobArrival) { + LOGGER.info("Received update for job configuration of job " + updateJobArrival.getJobName()); + try { + handleDeleteJobConfigArrival(new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(), + updateJobArrival.getJobConfig())); + } catch (Exception je) { + LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je); + } + + try { + handleNewJobConfigArrival(new NewJobConfigArrivalEvent(updateJobArrival.getJobName(), + updateJobArrival.getJobConfig())); + } catch (Exception je) { + LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je); + } + } + + private void waitForJobCompletion(String jobName) { + while (this.jobRunningMap.getOrDefault(jobName, false)) { + LOGGER.info("Waiting for job {} to stop...", jobName); + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + LOGGER.warn("Interrupted exception encountered: ", e); + } + } + } + + @Subscribe + public void handleDeleteJobConfigArrival(DeleteJobConfigArrivalEvent deleteJobArrival) throws InterruptedException { + LOGGER.info("Received delete for job configuration of job " + deleteJobArrival.getJobName()); + try { + unscheduleJob(deleteJobArrival.getJobName()); + cancelJobIfRequired(deleteJobArrival); + } catch (JobException je) { + LOGGER.error("Failed to unschedule job " + deleteJobArrival.getJobName()); + } + } + + @Subscribe + public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobArrival) + throws InterruptedException { + String jobUri = cancelJobArrival.getJoburi(); + LOGGER.info("Received cancel for job configuration of job " + jobUri); + Optional distributedJobMode; + Optional planningJob = Optional.empty(); + Optional actualJob = Optional.empty(); + boolean cancelByDelete = PropertiesUtils.getPropAsBoolean(this.commonJobProperties, GobblinClusterConfigurationKeys.CANCEL_HELIX_JOB_BY_DELETE, + String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CANCEL_HELIX_JOB_BY_DELETE)); + + this.jobSchedulerMetrics.numCancellationStart.incrementAndGet(); + + try { + distributedJobMode = this.jobsMapping.getDistributedJobMode(jobUri); + if (distributedJobMode.isPresent() && Boolean.parseBoolean(distributedJobMode.get())) { + planningJob = this.jobsMapping.getPlanningJobId(jobUri); + } else { + actualJob = this.jobsMapping.getActualJobId(jobUri); + } + } catch (IOException e) { + LOGGER.warn("jobsMapping could not be retrieved for job {}", jobUri); + return; + } + + this.jobSchedulerMetrics.numCancellationStart.decrementAndGet(); + } + + + private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) throws InterruptedException { + + } + + /** + * This class is responsible for running non-scheduled jobs. + */ + class NonScheduledJobRunner implements Runnable { + private final Properties jobProps; + private final GobblinHelixJobLauncherListener jobListener; + private final Long creationTimeInMillis; + + public NonScheduledJobRunner(Properties jobProps, + GobblinHelixJobLauncherListener jobListener) { + + this.jobProps = jobProps; + this.jobListener = jobListener; + this.creationTimeInMillis = System.currentTimeMillis(); + } + + @Override + public void run() { + try { + GobblinTemporalJobScheduler.this.jobSchedulerMetrics.updateTimeBeforeJobLaunching(this.jobProps); + GobblinTemporalJobScheduler.this.jobSchedulerMetrics.updateTimeBetweenJobSchedulingAndJobLaunching(this.creationTimeInMillis, System.currentTimeMillis()); + GobblinTemporalJobScheduler.this.runJob(this.jobProps, this.jobListener); + } catch (JobException je) { + LOGGER.error("Failed to run job " + this.jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), je); + } + } + } +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java index 1337dfd9aa..0633530e60 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java @@ -18,6 +18,8 @@ package org.apache.gobblin.cluster; import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collection; @@ -51,6 +53,7 @@ import io.temporal.serviceclient.WorkflowServiceStubs; import io.temporal.worker.Worker; import io.temporal.worker.WorkerFactory; +import io.temporal.worker.WorkerOptions; import lombok.Getter; import lombok.Setter; @@ -58,19 +61,24 @@ import org.apache.gobblin.cluster.temporal.GobblinTemporalActivityImpl; import org.apache.gobblin.cluster.temporal.GobblinTemporalWorkflowImpl; import org.apache.gobblin.cluster.temporal.Shared; +import org.apache.gobblin.configuration.ConfigurationKeys; import org.apache.gobblin.configuration.State; import org.apache.gobblin.instrumented.StandardMetricsBridge; import org.apache.gobblin.metrics.GobblinMetrics; +import org.apache.gobblin.metrics.MultiReporterException; import org.apache.gobblin.metrics.RootMetricContext; import org.apache.gobblin.metrics.event.EventSubmitter; import org.apache.gobblin.metrics.event.GobblinEventBuilder; +import org.apache.gobblin.metrics.reporter.util.MetricReportUtils; import org.apache.gobblin.runtime.api.TaskEventMetadataGenerator; +import org.apache.gobblin.util.ClassAliasResolver; import org.apache.gobblin.util.ConfigUtils; import org.apache.gobblin.util.FileUtils; import org.apache.gobblin.util.HadoopUtils; import org.apache.gobblin.util.JvmUtils; import org.apache.gobblin.util.TaskEventMetadataUtils; import org.apache.gobblin.util.event.ContainerHealthCheckFailureEvent; +import org.apache.gobblin.util.reflection.GobblinConstructorUtils; import static org.apache.gobblin.cluster.GobblinTemporalClusterManager.createServiceStubs; @@ -80,18 +88,6 @@ * {@link org.apache.gobblin.source.workunit.WorkUnit}s. * *

- * This class presents a Helix participant that uses a to communicate with Helix. - * It uses Helix task execution framework and details are encapsulated in {@link TaskRunnerSuiteBase}. - *

- * - *

- * This class responds to a graceful shutdown initiated by the {@link GobblinTemporalClusterManager} via - * a Helix message of subtype {@link HelixMessageSubTypes#WORK_UNIT_RUNNER_SHUTDOWN}, or it does a - * graceful shutdown when the shutdown hook gets called. In both cases, {@link #stop()} will be - * called to start the graceful shutdown. - *

- * - *

* If for some reason, the container exits or gets killed, the {@link GobblinTemporalClusterManager} will * be notified for the completion of the container and will start a new container to replace this one. *

@@ -106,6 +102,8 @@ public class GobblinTemporalTaskRunner implements StandardMetricsBridge { private static final Logger logger = LoggerFactory.getLogger(GobblinTemporalTaskRunner.class); static final java.nio.file.Path CLUSTER_CONF_PATH = Paths.get("generated-gobblin-cluster.conf"); + + private static TaskRunnerSuiteBase.Builder builder; private final Optional containerMetrics; private final Path appWorkPath; private boolean isTaskDriver; @@ -124,15 +122,15 @@ public class GobblinTemporalTaskRunner implements StandardMetricsBridge { protected final FileSystem fs; protected final String applicationName; protected final String applicationId; + protected final int temporalWorkerSize; + private final boolean isMetricReportingFailureFatal; + private final boolean isEventReportingFailureFatal; public GobblinTemporalTaskRunner(String applicationName, String applicationId, String taskRunnerId, Config config, Optional appWorkDirOptional) throws Exception { - // Set system properties passed in via application config. As an example, Helix uses System#getProperty() for ZK configuration - // overrides such as sessionTimeout. In this case, the overrides specified - // in the application configuration have to be extracted and set before initializing HelixManager. GobblinClusterUtils.setSystemProperties(config); //Add dynamic config @@ -150,6 +148,17 @@ public GobblinTemporalTaskRunner(String applicationName, logger.info("Configured GobblinTaskRunner work dir to: {}", this.appWorkPath.toString()); this.containerMetrics = buildContainerMetrics(); + this.builder = initBuilder(); + // The default worker size would be 1 + this.temporalWorkerSize = ConfigUtils.getInt(config, GobblinClusterConfigurationKeys.TEMPORAL_WORKER_SIZE,1); + + this.isMetricReportingFailureFatal = ConfigUtils.getBoolean(this.clusterConfig, + ConfigurationKeys.GOBBLIN_TASK_METRIC_REPORTING_FAILURE_FATAL, + ConfigurationKeys.DEFAULT_GOBBLIN_TASK_METRIC_REPORTING_FAILURE_FATAL); + + this.isEventReportingFailureFatal = ConfigUtils.getBoolean(this.clusterConfig, + ConfigurationKeys.GOBBLIN_TASK_EVENT_REPORTING_FAILURE_FATAL, + ConfigurationKeys.DEFAULT_GOBBLIN_TASK_EVENT_REPORTING_FAILURE_FATAL); logger.info("GobblinTaskRunner({}): applicationName {}, applicationId {}, taskRunnerId {}, config {}, appWorkDir {}", this.isTaskDriver ? "taskDriver" : "worker", @@ -160,6 +169,35 @@ public GobblinTemporalTaskRunner(String applicationName, appWorkDirOptional); } + public static TaskRunnerSuiteBase.Builder getBuilder() { + return builder; + } + + private TaskRunnerSuiteBase.Builder initBuilder() throws ReflectiveOperationException { + String builderStr = ConfigUtils.getString(this.clusterConfig, + GobblinClusterConfigurationKeys.TASK_RUNNER_SUITE_BUILDER, + TaskRunnerSuiteBase.Builder.class.getName()); + + String hostName = ""; + try { + hostName = InetAddress.getLocalHost().getHostName(); + } catch (UnknownHostException e) { + logger.warn("Cannot find host name for Helix instance: {}"); + } + + TaskRunnerSuiteBase.Builder builder = GobblinConstructorUtils.invokeLongestConstructor( + new ClassAliasResolver(TaskRunnerSuiteBase.Builder.class) + .resolveClass(builderStr), this.clusterConfig); + + return builder.setAppWorkPath(this.appWorkPath) + .setContainerMetrics(this.containerMetrics) + .setFileSystem(this.fs) + .setApplicationId(applicationId) + .setApplicationName(applicationName) + .setContainerId(taskRunnerId) + .setHostName(hostName); + } + private Path initAppWorkDir(Config config, Optional appWorkDirOptional) { return appWorkDirOptional.isPresent() ? appWorkDirOptional.get() : GobblinClusterUtils .getAppWorkDirPathFromConfig(config, this.fs, this.applicationName, this.applicationId); @@ -181,21 +219,32 @@ public void start() throws ContainerHealthCheckException { logger.info("Calling start method in GobblinTemporalTaskRunner"); logger.info(String.format("Starting in container %s", this.taskRunnerId)); + + // Start metric reporting + initMetricReporter(); + + // Add a shutdown hook so the task scheduler gets properly shutdown + addShutdownHook(); + try { - initiateWorker(); + for (int i = 0; i < this.temporalWorkerSize; i++) { + initiateWorker(); + } }catch (Exception e) { logger.info(e + " for initiate workers"); throw new RuntimeException(e); } - - // Add a shutdown hook so the task scheduler gets properly shutdown - addShutdownHook(); } private void initiateWorker() throws Exception{ logger.info("Starting Temporal Worker"); WorkflowServiceStubs service = createServiceStubs(); + WorkerOptions workerOptions = WorkerOptions.newBuilder() + .setMaxConcurrentWorkflowTaskExecutionSize(1) + .setMaxConcurrentActivityExecutionSize(1) + .build(); + // WorkflowClient can be used to start, signal, query, cancel, and terminate Workflows. WorkflowClient client = WorkflowClient.newInstance( @@ -210,7 +259,7 @@ private void initiateWorker() throws Exception{ * Define the workflow worker. Workflow workers listen to a defined task queue and process * workflows and activities. */ - Worker worker = factory.newWorker(Shared.HELLO_WORLD_TASK_QUEUE); + Worker worker = factory.newWorker(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE, workerOptions); /* * Register our workflow implementation with the worker. @@ -233,6 +282,19 @@ private void initiateWorker() throws Exception{ logger.info("A new worker is started."); } + private void initMetricReporter() { + if (this.containerMetrics.isPresent()) { + try { + this.containerMetrics.get() + .startMetricReportingWithFileSuffix(ConfigUtils.configToState(this.clusterConfig), this.taskRunnerId); + } catch (MultiReporterException ex) { + if (MetricReportUtils.shouldThrowException(logger, ex, this.isMetricReportingFailureFatal, this.isEventReportingFailureFatal)) { + throw new RuntimeException(ex); + } + } + } + } + public synchronized void stop() { if (this.isStopped) { logger.info("Gobblin Task runner is already stopped."); @@ -320,7 +382,7 @@ private void submitEvent(ContainerHealthCheckFailureEvent event) { EventSubmitter eventSubmitter = new EventSubmitter.Builder(RootMetricContext.get(), getClass().getPackage().getName()).build(); GobblinEventBuilder eventBuilder = new GobblinEventBuilder(event.getClass().getSimpleName()); State taskState = ConfigUtils.configToState(event.getConfig()); - //Add task metadata such as Helix taskId, containerId, and workflowId if configured + //Add task metadata such as taskId, containerId, and workflowId if configured TaskEventMetadataGenerator taskEventMetadataGenerator = TaskEventMetadataUtils.getTaskEventMetadataGenerator(taskState); eventBuilder.addAdditionalMetadata(taskEventMetadataGenerator.getMetadata(taskState, event.getClass().getSimpleName())); eventBuilder.addAdditionalMetadata(event.getMetadata()); diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java index 93caab41c7..67fdcd4f14 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java @@ -18,6 +18,9 @@ package org.apache.gobblin.cluster; import java.io.IOException; +import java.time.Instant; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; import java.util.List; import java.util.Properties; import java.util.concurrent.TimeUnit; @@ -39,6 +42,9 @@ import org.apache.gobblin.broker.gobblin_scopes.GobblinScopeTypes; import org.apache.gobblin.broker.gobblin_scopes.JobScopeInstance; import org.apache.gobblin.broker.iface.SharedResourcesBroker; +import org.apache.gobblin.metrics.MetricContext; +import org.apache.gobblin.metrics.event.EventSubmitter; +import org.apache.gobblin.metrics.event.GobblinEventBuilder; import org.apache.gobblin.runtime.AbstractJobLauncher; import org.apache.gobblin.runtime.GobblinMultiTaskAttempt; import org.apache.gobblin.runtime.JobState; @@ -72,22 +78,32 @@ public class SingleTask { private JobState _jobState; // Preventing Helix calling cancel before taskAttempt is created - // Checking if taskAttempt is empty is not enough, since canceller runs in different thread as runner, the case to + // Checking if taskAttempt is empty is not enough, since canceller runs in different thread as runner, the case // to avoid here is taskAttempt being created and start to run after cancel has been called. private Condition _taskAttemptBuilt; private Lock _lock; + private String workflowId; - SingleTask(String jobId, Path workUnitFilePath, Path jobStateFilePath, FileSystem fs, + public SingleTask(String jobId, Path workUnitFilePath, Path jobStateFilePath, FileSystem fs, TaskAttemptBuilder taskAttemptBuilder, StateStores stateStores, Config dynamicConfig) { this(jobId, workUnitFilePath, jobStateFilePath, fs, taskAttemptBuilder, stateStores, dynamicConfig, false); + this.workflowId = ""; + } + + public SingleTask(String jobId, Path workUnitFilePath, Path jobStateFilePath, FileSystem fs, + TaskAttemptBuilder taskAttemptBuilder, StateStores stateStores, Config dynamicConfig, String workflowId) { + this(jobId, workUnitFilePath, jobStateFilePath, fs, taskAttemptBuilder, stateStores, dynamicConfig, false); + _logger.info("Gobblin task workflowid: {}", workflowId); + this.workflowId = workflowId; } /** * Do all heavy-lifting of initialization in constructor which could be retried if failed, * see the example in {@link GobblinHelixTask}. */ - SingleTask(String jobId, Path workUnitFilePath, Path jobStateFilePath, FileSystem fs, + public SingleTask(String jobId, Path workUnitFilePath, Path jobStateFilePath, FileSystem fs, TaskAttemptBuilder taskAttemptBuilder, StateStores stateStores, Config dynamicConfig, boolean skipGetJobState) { + _logger.info("Constructing SingleTask"); _jobId = jobId; _workUnitFilePath = workUnitFilePath; _jobStateFilePath = jobStateFilePath; @@ -109,23 +125,25 @@ public class SingleTask { } } - public void run() - throws IOException, InterruptedException { + public void run() throws IOException, InterruptedException { + _logger.info("Running SingleTask"); if (_jobState == null) { throw new RuntimeException("jobState is null. Task may have already been cancelled."); } + MetricContext metricContext = MetricContext.builder("SingleTaskContext").build(); + EventSubmitter eventSubmitter = new EventSubmitter.Builder(metricContext, "gobblin.task").build(); + submitEvent(eventSubmitter, "GobblinTaskStarted"); + // Add dynamic configuration to the job state _dynamicConfig.entrySet().forEach(e -> _jobState.setProp(e.getKey(), e.getValue().unwrapped().toString())); Config jobConfig = getConfigFromJobState(_jobState); - _logger.debug("SingleTask.run: jobId {} workUnitFilePath {} jobStateFilePath {} jobState {} jobConfig {}", - _jobId, _workUnitFilePath, _jobStateFilePath, _jobState, jobConfig); + _logger.debug("SingleTask.run: jobId {} workUnitFilePath {} jobStateFilePath {} jobState {} jobConfig {}", _jobId, _workUnitFilePath, _jobStateFilePath, _jobState, jobConfig); - try (SharedResourcesBroker globalBroker = SharedResourcesBrokerFactory - .createDefaultTopLevelBroker(jobConfig, GobblinScopeTypes.GLOBAL.defaultScopeInstance())) { + try (SharedResourcesBroker globalBroker = SharedResourcesBrokerFactory.createDefaultTopLevelBroker(jobConfig, GobblinScopeTypes.GLOBAL.defaultScopeInstance())) { SharedResourcesBroker jobBroker = getJobBroker(_jobState, globalBroker); // Secure atomicity of taskAttempt's execution. @@ -135,19 +153,34 @@ public void run() _lock.lock(); try { _taskAttemptBuilt.signal(); + submitEvent(eventSubmitter, "GobblinTaskAttemptBuiltSignal"); } finally { _lock.unlock(); } // This is a blocking call. - _taskAttempt.runAndOptionallyCommitTaskAttempt(GobblinMultiTaskAttempt.CommitPolicy.IMMEDIATE); + submitEvent(eventSubmitter, "GobblinTaskAttemptRunAndOptionallyCommit"); + _taskAttempt.runAndOptionallyCommitTaskAttempt(GobblinMultiTaskAttempt.CommitPolicy.IMMEDIATE); } finally { _logger.info("Clearing all metrics object in cache."); + submitEvent(eventSubmitter, "GobblinTaskAttemptCleanMetrics"); _taskAttempt.cleanMetrics(); } } + private void submitEvent(EventSubmitter eventSubmitter, String eventName) { + GobblinEventBuilder eventBuilder = new GobblinEventBuilder(eventName); + Instant instant = Instant.now(); + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss") + .withZone(ZoneId.systemDefault()); + String formattedDateTime = formatter.format(instant); + eventBuilder.addMetadata("EventType", eventName); + eventBuilder.addMetadata("WorkflowId", this.workflowId); + eventBuilder.addMetadata("EventTime", formattedDateTime); + eventSubmitter.submit(eventBuilder); + } + private SharedResourcesBroker getJobBroker(JobState jobState, SharedResourcesBroker globalBroker) { return globalBroker.newSubscopedBuilder(new JobScopeInstance(jobState.getJobName(), jobState.getJobId())).build(); diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java index 269ff09479..70cbcda2a3 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java @@ -15,16 +15,21 @@ * limitations under the License. */ -// @@@SNIPSTART hello-world-project-template-java-activity-interface package org.apache.gobblin.cluster.temporal; +import java.util.Properties; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; + import io.temporal.activity.ActivityInterface; +@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.ANY) @ActivityInterface public interface GobblinTemporalActivity { - // Define your activity methods which can be called during workflow execution String composeGreeting(String name); + // Method to run Gobblin Task in activity + void run(Properties jobProps, String appWorkDir, String jobId, String workUnitFilePath, String jobStateFilePath, String workflowId) + throws Exception; } -// @@@SNIPEND diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java index c1c5225f50..d6ef6f6e46 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java @@ -15,21 +15,79 @@ * limitations under the License. */ -// @@@SNIPSTART hello-world-project-template-java-activity package org.apache.gobblin.cluster.temporal; +import java.net.URI; +import java.util.Properties; + +import org.apache.hadoop.fs.Path; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; +import org.apache.gobblin.cluster.GobblinHelixTaskStateTracker; +import org.apache.gobblin.cluster.GobblinTemporalTaskRunner; +import org.apache.gobblin.cluster.SingleTask; +import org.apache.gobblin.cluster.TaskAttemptBuilder; +import org.apache.gobblin.cluster.TaskRunnerSuiteBase; +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.runtime.TaskExecutor; +import org.apache.gobblin.runtime.TaskStateTracker; +import org.apache.gobblin.runtime.util.StateStores; +import org.apache.gobblin.util.ConfigUtils; public class GobblinTemporalActivityImpl implements GobblinTemporalActivity { private static final Logger LOGGER = LoggerFactory.getLogger(GobblinTemporalActivityImpl.class); + private TaskRunnerSuiteBase.Builder builder; + private StateStores stateStores; + private TaskAttemptBuilder taskAttemptBuilder; + + @Override public String composeGreeting(String name) { LOGGER.info("Activity triggered"); return "Hello " + name + "!"; } + private TaskAttemptBuilder createTaskAttemptBuilder() { + Properties properties = ConfigUtils.configToProperties(builder.getConfig()); + TaskStateTracker taskStateTracker = new GobblinHelixTaskStateTracker(properties); + TaskExecutor taskExecutor = new TaskExecutor(ConfigUtils.configToProperties(builder.getConfig())); + TaskAttemptBuilder taskAttemptBuilder = new TaskAttemptBuilder(taskStateTracker, taskExecutor); + taskAttemptBuilder.setTaskStateStore(this.stateStores.getTaskStateStore()); + return taskAttemptBuilder; + } + + @Override + public void run(Properties jobProps, String appWorkDirStr, String jobId, String workUnitFilePath, String jobStateFilePath, String workflowId) + throws Exception { + Path appWorkDir = new Path(appWorkDirStr); + this.builder = GobblinTemporalTaskRunner.getBuilder(); + + Config stateStoreJobConfig = ConfigUtils.propertiesToConfig(jobProps) + .withValue(ConfigurationKeys.STATE_STORE_FS_URI_KEY, ConfigValueFactory.fromAnyRef( + new URI(appWorkDir.toUri().getScheme(), null, appWorkDir.toUri().getHost(), appWorkDir.toUri().getPort(), + "/", null, null).toString())); + + this.stateStores = + new StateStores(stateStoreJobConfig, appWorkDir, GobblinClusterConfigurationKeys.OUTPUT_TASK_STATE_DIR_NAME, + appWorkDir, GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME, appWorkDir, + GobblinClusterConfigurationKeys.JOB_STATE_DIR_NAME); + + this.taskAttemptBuilder = createTaskAttemptBuilder(); + + // Dynamic config is considered as part of JobState in SingleTask + // Important to distinguish between dynamicConfig and Config + final Config dynamicConfig = builder.getDynamicConfig() + .withValue(GobblinClusterConfigurationKeys.TASK_RUNNER_HOST_NAME_KEY, ConfigValueFactory.fromAnyRef(builder.getHostName())) + .withValue(GobblinClusterConfigurationKeys.CONTAINER_ID_KEY, ConfigValueFactory.fromAnyRef(builder.getContainerId())) + .withValue(GobblinClusterConfigurationKeys.HELIX_INSTANCE_NAME_KEY, ConfigValueFactory.fromAnyRef(builder.getInstanceName())); + + SingleTask singleTask = new SingleTask(jobId, new Path(workUnitFilePath), new Path(jobStateFilePath), builder.getFs(), this.taskAttemptBuilder, this.stateStores, dynamicConfig, workflowId); + singleTask.run(); + } } -// @@@SNIPEND diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java index 7edfcd0e81..351a9ec7f4 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java @@ -15,20 +15,26 @@ * limitations under the License. */ -// @@@SNIPSTART hello-world-project-template-java-workflow-interface package org.apache.gobblin.cluster.temporal; +import java.util.Properties; + +import io.temporal.workflow.QueryMethod; import io.temporal.workflow.WorkflowInterface; import io.temporal.workflow.WorkflowMethod; + @WorkflowInterface public interface GobblinTemporalWorkflow { + @QueryMethod + String getGreeting(String name); + /** * This is the method that is executed when the Workflow Execution is started. The Workflow * Execution completes when this method finishes execution. */ @WorkflowMethod - String getGreeting(String name); + void runTask(Properties jobProps, String appWorkDir, String jobId, String workUnitFilePath, String jobStateFilePath) + throws Exception; } -// @@@SNIPEND diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java index c2f4e76f30..49d974384c 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java @@ -19,14 +19,35 @@ package org.apache.gobblin.cluster.temporal; import java.time.Duration; +import java.time.Instant; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.Properties; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.protobuf.Timestamp; + import io.temporal.activity.ActivityOptions; +import io.temporal.api.common.v1.WorkflowExecution; +import io.temporal.api.history.v1.HistoryEvent; +import io.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryRequest; +import io.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryResponse; import io.temporal.common.RetryOptions; +import io.temporal.serviceclient.WorkflowServiceStubs; import io.temporal.workflow.Workflow; +import org.apache.gobblin.metrics.MetricContext; +import org.apache.gobblin.metrics.event.EventSubmitter; +import org.apache.gobblin.metrics.event.GobblinEventBuilder; + +import static org.apache.gobblin.cluster.GobblinTemporalClusterManager.createServiceStubs; + + public class GobblinTemporalWorkflowImpl implements GobblinTemporalWorkflow { /* @@ -36,19 +57,17 @@ public class GobblinTemporalWorkflowImpl implements GobblinTemporalWorkflow { */ private final RetryOptions retryoptions = RetryOptions.newBuilder() - .setInitialInterval(Duration.ofSeconds(1)) - .setMaximumInterval(Duration.ofSeconds(100)) - .setBackoffCoefficient(2) - .setMaximumAttempts(500) + .setMaximumAttempts(1) .build(); + int yearPeriod = 365 * 24 * 60 * 60; private final ActivityOptions options = ActivityOptions.newBuilder() - .setStartToCloseTimeout(Duration.ofSeconds(60)) + .setStartToCloseTimeout(Duration.ofSeconds(yearPeriod)) .setRetryOptions(retryoptions) .build(); /* - * Define the HelloWorldActivity stub. Activity stubs are proxies for activity invocations that + * Define the GobblinTemporalActivity stub. Activity stubs are proxies for activity invocations that * are executed outside of the workflow thread on the activity worker, that can be on a * different host. Temporal is going to dispatch the activity results back to the workflow and * unblock the stub as soon as activity is completed on the activity worker. @@ -60,7 +79,6 @@ public class GobblinTemporalWorkflowImpl implements GobblinTemporalWorkflow { // This is the entry point to the Workflow. @Override public String getGreeting(String name) { - /** * If there were other Activity methods they would be orchestrated here or from within other Activities. * This is a blocking call that returns only after the activity has completed. @@ -68,5 +86,58 @@ public String getGreeting(String name) { LOGGER.info("Workflow triggered"); return activity.composeGreeting(name); } -} -// @@@SNIPEND + + @Override + public void runTask(Properties jobProps, String appWorkDir, String jobId, String workUnitFilePath, String jobStateFilePath) + throws Exception{ + String workflowId = Workflow.getInfo().getWorkflowId(); + String runId = Workflow.getInfo().getRunId(); + WorkflowExecution execution = WorkflowExecution.newBuilder() + .setWorkflowId(workflowId) + .setRunId(runId) + .build(); + + ScheduledExecutorService executorService = Executors.newScheduledThreadPool(1); + + MetricContext metricContext = MetricContext.builder("TemporalWorkflowContext").build(); + EventSubmitter eventSubmitter = new EventSubmitter.Builder(metricContext, "gobblin.temporal").build(); + + final long[] lastLoggedEventId = {0}; + executorService.scheduleAtFixedRate(() -> { + try { + GetWorkflowExecutionHistoryRequest request = + GetWorkflowExecutionHistoryRequest.newBuilder().setNamespace("gobblin-fastingest-internpoc").setExecution(execution).build(); + + WorkflowServiceStubs workflowServiceStubs = createServiceStubs(); + GetWorkflowExecutionHistoryResponse response = + workflowServiceStubs.blockingStub().getWorkflowExecutionHistory(request); + + for (HistoryEvent event : response.getHistory().getEventsList()) { + // Only log events that are newer than the last one we logged + if (event.getEventId() > lastLoggedEventId[0]) { + Timestamp timestamp = event.getEventTime(); + Instant instant = Instant.ofEpochSecond(timestamp.getSeconds(), timestamp.getNanos()); + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss") + .withZone(ZoneId.systemDefault()); + String formattedDateTime = formatter.format(instant); + + GobblinEventBuilder eventBuilder = new GobblinEventBuilder("TemporalEvent"); + eventBuilder.addMetadata("WorkflowId", workflowId); + eventBuilder.addMetadata("EventType", event.getEventType().name()); + eventBuilder.addMetadata("EventTime", formattedDateTime); + // add metadata of workflow topic + eventSubmitter.submit(eventBuilder); + + lastLoggedEventId[0] = event.getEventId(); + } + } + } catch (Exception e) { + LOGGER.error("Error retrieving workflow history", e); + } + }, 0, 10, TimeUnit.SECONDS); + + activity.run(jobProps, appWorkDir, jobId, workUnitFilePath, jobStateFilePath, workflowId); + + executorService.shutdown(); + } +} \ No newline at end of file diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java index 9aa869af57..b282d36275 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java @@ -15,13 +15,11 @@ * limitations under the License. */ -// @@@SNIPSTART hello-world-project-template-java-shared-constants package org.apache.gobblin.cluster.temporal; public interface Shared { // Define the task queue name - final String HELLO_WORLD_TASK_QUEUE = "HelloWorldTaskQueue"; + final String GOBBLIN_TEMPORAL_TASK_QUEUE = "GobblinTemporalTaskQueue"; -} -// @@@SNIPEND +} \ No newline at end of file diff --git a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/MetricContext.java b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/MetricContext.java index 7a851e4a15..31c820f247 100644 --- a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/MetricContext.java +++ b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/MetricContext.java @@ -17,8 +17,6 @@ package org.apache.gobblin.metrics; -import lombok.Getter; - import java.io.Closeable; import java.io.IOException; import java.util.Collection; @@ -38,15 +36,14 @@ import org.slf4j.LoggerFactory; import com.codahale.metrics.Counter; -import com.codahale.metrics.MetricFilter; import com.codahale.metrics.Gauge; import com.codahale.metrics.Histogram; import com.codahale.metrics.Meter; import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricFilter; import com.codahale.metrics.MetricRegistry; import com.codahale.metrics.MetricSet; import com.codahale.metrics.Timer; - import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Optional; @@ -59,6 +56,8 @@ import com.google.common.io.Closer; import com.google.common.util.concurrent.MoreExecutors; +import lombok.Getter; + import org.apache.gobblin.metrics.context.NameConflictException; import org.apache.gobblin.metrics.context.ReportableContext; import org.apache.gobblin.metrics.notification.EventNotification; @@ -211,6 +210,7 @@ public void submitEvent(GobblinTrackingEvent nonReusableEvent) { EventNotification notification = new EventNotification(nonReusableEvent); sendNotification(notification); + LOG.info("EventBuilder {} is submitted.", nonReusableEvent); } /** diff --git a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/event/EventSubmitter.java b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/event/EventSubmitter.java index 891f980b7f..73ea591b7e 100644 --- a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/event/EventSubmitter.java +++ b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/event/EventSubmitter.java @@ -19,15 +19,18 @@ import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import lombok.Getter; + import org.apache.gobblin.metrics.GobblinTrackingEvent; import org.apache.gobblin.metrics.MetricContext; -import lombok.Getter; - /** * Wrapper around Avro {@link org.apache.gobblin.metrics.GobblinTrackingEvent.Builder} simplifying handling {@link org.apache.gobblin.metrics.GobblinTrackingEvent}s. @@ -47,6 +50,8 @@ public class EventSubmitter { @Getter private final Optional metricContext; + private static final Logger LOGGER = LoggerFactory.getLogger(EventSubmitter.class); + public static class Builder { private final Optional metricContext; private final Map metadata; diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/StreamModelTaskRunner.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/StreamModelTaskRunner.java index 9e3a7f70cc..124d1286f0 100644 --- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/StreamModelTaskRunner.java +++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/StreamModelTaskRunner.java @@ -28,6 +28,11 @@ import com.google.common.io.Closer; import com.google.common.util.concurrent.Futures; +import io.reactivex.Flowable; +import io.reactivex.flowables.ConnectableFlowable; +import io.reactivex.schedulers.Schedulers; +import lombok.AllArgsConstructor; + import org.apache.gobblin.configuration.ConfigurationKeys; import org.apache.gobblin.converter.Converter; import org.apache.gobblin.fork.ForkOperator; @@ -45,11 +50,6 @@ import org.apache.gobblin.writer.WatermarkManager; import org.apache.gobblin.writer.WatermarkStorage; -import io.reactivex.Flowable; -import io.reactivex.flowables.ConnectableFlowable; -import io.reactivex.schedulers.Schedulers; -import lombok.AllArgsConstructor; - /** * A helper class to run {@link Task} in stream mode. Prevents {@link Task} from loading reactivex classes when not @@ -147,7 +147,7 @@ protected void run() throws Exception { forkedStream = forkedStream.mapStream(f -> f.observeOn(Schedulers.from(this.taskExecutor.getForkExecutor()), false, bufferSize)); } Fork fork = new Fork(this.taskContext, forkedStream.getGlobalMetadata().getSchema(), forkedStreams.getForkedStreams().size(), fidx, this.taskMode); - fork.consumeRecordStream(forkedStream); + // fork.consumeRecordStream(forkedStream); this.forks.put(Optional.of(fork), Optional.of(Futures.immediateFuture(null))); this.task.configureStreamingFork(fork); } diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java index 4ef48c9765..00c8e746dc 100644 --- a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java +++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java @@ -55,7 +55,7 @@ /** - * The Yarn ApplicationMaster class for Gobblin. + * The Yarn ApplicationMaster class for Gobblin using Temporal. * *

* This class runs the {@link YarnTemporalService} for all Yarn-related stuffs like ApplicationMaster registration diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnConfigurationKeys.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnConfigurationKeys.java index 7088dfa996..490ed72b94 100644 --- a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnConfigurationKeys.java +++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnConfigurationKeys.java @@ -151,4 +151,5 @@ public class GobblinYarnConfigurationKeys { //Config to control Heartbeat interval for Yarn AMRM client. public static final String AMRM_HEARTBEAT_INTERVAL_SECS = GOBBLIN_YARN_PREFIX + "amRmHeartbeatIntervalSecs"; public static final Integer DEFAULT_AMRM_HEARTBEAT_INTERVAL_SECS = 15; + public static final String TEMPORAL_WORKERPOOL_SIZE = "temporal.workerpool.size"; } diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java index a45017b5c2..421a4a6300 100644 --- a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java +++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java @@ -422,12 +422,14 @@ private EventSubmitter buildEventSubmitter() { * @return whether successfully requested the target number of containers */ public synchronized boolean requestTargetNumberOfContainers(YarnContainerRequestBundle yarnContainerRequestBundle, Set inUseInstances) { - LOGGER.info("Trying to set numTargetContainers={}, in-use helix instances count is {}, container map size is {}", - yarnContainerRequestBundle.getTotalContainers(), inUseInstances.size(), this.containerMap.size()); - int defaultContainerMemoryMbs = config.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); int defaultContainerCores = config.getInt(GobblinYarnConfigurationKeys. CONTAINER_CORES_KEY); - int workerPoolSize = ConfigUtils.getInt(config, "temporal.workerpool.size",2); + // making workerPoolSize configurable, the default value would be 10 + int workerPoolSize = ConfigUtils.getInt(config, GobblinYarnConfigurationKeys.TEMPORAL_WORKERPOOL_SIZE,10); + + LOGGER.info("Trying to set numTargetContainers={}, in-use helix instances count is {}, container map size is {}", + workerPoolSize, inUseInstances.size(), this.containerMap.size()); + requestContainers(workerPoolSize, Resource.newInstance(defaultContainerMemoryMbs, defaultContainerCores)); this.yarnContainerRequest = yarnContainerRequestBundle; @@ -701,42 +703,16 @@ protected void handleContainerCompletion(ContainerStatus containerStatus) { Optional.of(completedContainerInfo.getContainer()) : Optional.absent(), newContainerResource)); } - /** - * Handles containers aborted. This method handles 2 cases: - *

    - *
  1. - * Case 1: Gobblin AM intentionally requested container to be released (often because the number of helix tasks - * has decreased due to decreased traffic) - *
  2. - *
  3. - * Case 2: Unexpected hardware fault and the node is lost. Need to do specific Helix logic to ensure 2 helix tasks - * are not being run by the multiple containers - *
  4. - *
- * @param containerStatus - * @param completedContainerInfo - * @param completedInstanceName - * @return if release request was intentionally released (Case 1) - */ private boolean handleAbortedContainer(ContainerStatus containerStatus, ContainerInfo completedContainerInfo, String completedInstanceName) { - - // Case 1: Container intentionally released if (this.releasedContainerCache.getIfPresent(containerStatus.getContainerId()) != null) { LOGGER.info("Container release requested, so not spawning a replacement for containerId {}", containerStatus.getContainerId()); if (completedContainerInfo != null) { LOGGER.info("Adding instance {} to the pool of unused instances", completedInstanceName); this.unusedHelixInstanceNames.add(completedInstanceName); } - return true; } - - // Case 2: Container release was not requested. Likely, the container was running on a node on which the NM died. - // In this case, RM assumes that the containers are "lost", even though the container process may still be - // running on the node. We need to ensure that the Helix instances running on the orphaned containers - // are fenced off from the Helix cluster to avoid double publishing and state being committed by the - // instances. LOGGER.info("Container {} aborted due to lost NM", containerStatus.getContainerId()); return false; } @@ -767,8 +743,6 @@ private ImmutableMap.Builder buildContainerStatusEventMetadata(C * The RM can return containers that are larger (because of normalization etc). * Container may be larger by memory or cpu (e.g. container (1000M, 3cpu) can fit request (1000M, 1cpu) or request (500M, 3cpu). *

- * Thankfully since each helix tag / resource has a different priority, matching requests for one helix tag / resource - * have complete isolation from another helix tag / resource */ private int getMatchingRequestsCount(Resource resource) { Integer priorityNum = resourcePriorityMap.get(resource.toString()); diff --git a/gradle/scripts/dependencyDefinitions.gradle b/gradle/scripts/dependencyDefinitions.gradle index a26183a192..ba961bbf8f 100644 --- a/gradle/scripts/dependencyDefinitions.gradle +++ b/gradle/scripts/dependencyDefinitions.gradle @@ -215,7 +215,6 @@ ext.externalDependency = [ "postgresConnector": "org.postgresql:postgresql:42.1.4", "testContainers": "org.testcontainers:testcontainers:1.17.3", "testContainersMysql": "org.testcontainers:mysql:1.17.3", - "workflowClient.namespace": "gobblin-fastingest-internpoc", "xz": "org.tukaani:xz:1.8" ] From f49aedb1b42584b146b678d2f770faf0c88921df Mon Sep 17 00:00:00 2001 From: Yiming Yang Date: Mon, 28 Aug 2023 22:39:04 -0700 Subject: [PATCH 5/8] Temporal POC --- .../cluster/GobblinTemporalJobLauncher.java | 76 ++++++++---- .../cluster/GobblinTemporalTaskRunner.java | 43 ++----- .../AbstractNestingExecWorkflowImpl.java | 116 ++++++++++++++++++ .../temporal/AbstractTemporalWorker.java | 42 +++++++ .../temporal/CollectionBackedTaskSpan.java | 51 ++++++++ .../cluster/temporal/IllustrationTask.java | 16 +++ .../temporal/IllustrationTaskActivity.java | 16 +++ .../IllustrationTaskActivityImpl.java | 12 ++ .../cluster/temporal/NestingExecWorker.java | 21 ++++ .../cluster/temporal/NestingExecWorkflow.java | 30 +++++ .../temporal/NestingExecWorkflowImpl.java | 34 +++++ .../gobblin/cluster/temporal/Shared.java | 1 - .../temporal/SimpleGeneratedWorkload.java | 44 +++++++ .../TemporalWorkflowClientFactory.java | 92 ++++++++++++++ .../gobblin/cluster/temporal/WFAddr.java | 48 ++++++++ .../gobblin/cluster/temporal/Workload.java | 37 ++++++ 16 files changed, 619 insertions(+), 60 deletions(-) create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractNestingExecWorkflowImpl.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractTemporalWorker.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/CollectionBackedTaskSpan.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTask.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivity.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivityImpl.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorker.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflow.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflowImpl.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/SimpleGeneratedWorkload.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/WFAddr.java create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Workload.java diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java index 5f79e596fc..ee7b803b81 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java @@ -28,6 +28,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; +import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -51,6 +52,11 @@ import org.apache.gobblin.annotation.Alpha; import org.apache.gobblin.cluster.temporal.GobblinTemporalWorkflow; import org.apache.gobblin.cluster.temporal.Shared; +import org.apache.gobblin.cluster.temporal.IllustrationTask; +import org.apache.gobblin.cluster.temporal.Workload; +import org.apache.gobblin.cluster.temporal.SimpleGeneratedWorkload; +import org.apache.gobblin.cluster.temporal.NestingExecWorkflow; +import org.apache.gobblin.cluster.temporal.WFAddr; import org.apache.gobblin.configuration.ConfigurationKeys; import org.apache.gobblin.metastore.StateStore; import org.apache.gobblin.metrics.Tag; @@ -76,7 +82,8 @@ import org.apache.gobblin.util.SerializationUtils; import static org.apache.gobblin.cluster.GobblinTemporalClusterManager.createServiceStubs; - +import static org.apache.gobblin.cluster.temporal.TemporalWorkflowClientFactory.createServiceInstance; +import static org.apache.gobblin.cluster.temporal.TemporalWorkflowClientFactory.createClientInstance; /** * An implementation of {@link JobLauncher} that launches a Gobblin job using the Temporal task framework. @@ -150,10 +157,8 @@ public GobblinTemporalJobLauncher(Properties jobProps, Path appWorkDir, new TaskStateCollectorService(jobProps, this.jobContext.getJobState(), this.eventBus, this.eventSubmitter, this.stateStores.getTaskStateStore(), this.outputTaskStateDir, this.getIssueRepository()); - this.workflowServiceStubs = createServiceStubs(); - this.client = WorkflowClient.newInstance( - workflowServiceStubs, WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build()); - + this.workflowServiceStubs = createServiceInstance(); + this.client = createClientInstance(workflowServiceStubs); /* * Set Workflow options such as WorkflowId and Task Queue so the worker knows where to list and which workflows to execute. */ @@ -246,27 +251,50 @@ private void submitJobToTemporal(List workUnits) throws Exception{ String jobStateFilePathStr = jobStateFilePath.toString(); List> futures = new ArrayList<>(); - AtomicInteger multiTaskIdSequence = new AtomicInteger(0); - AtomicInteger workflowCount = new AtomicInteger(0); - int workflowSize = 100; - ExecutorService executor = Executors.newFixedThreadPool(workflowSize); - - for (int i = 0; i < workflowSize; i++) { - WorkUnit workUnit = workUnits.get(i); + boolean fastIngest = false; + if (fastIngest) { + AtomicInteger multiTaskIdSequence = new AtomicInteger(0); + AtomicInteger workflowCount = new AtomicInteger(0); + int workflowSize = 100; + ExecutorService executor = Executors.newFixedThreadPool(workflowSize); + for (int i = 0; i < workflowSize; i++) { + WorkUnit workUnit = workUnits.get(i); + futures.add(CompletableFuture.runAsync(() -> { + try { + if (workUnit instanceof MultiWorkUnit) { + workUnit.setId(JobLauncherUtils.newMultiTaskId(this.jobContext.getJobId(), multiTaskIdSequence.getAndIncrement())); + } + String workUnitFilePathStr = persistWorkUnit(new Path(this.inputWorkUnitDir, this.jobContext.getJobId()), workUnit, stateSerDeRunner); + String workflowId = workUnit.getProp(KafkaSource.TOPIC_NAME) + "_" + workflowCount.getAndIncrement(); + WorkflowOptions options = WorkflowOptions.newBuilder() + .setTaskQueue(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE) + .setWorkflowId(workflowId) + .build(); + // TODO(yiyang): change up the workflow + GobblinTemporalWorkflow workflow = this.client.newWorkflowStub(GobblinTemporalWorkflow.class, options); + LOGGER.info("Setting up temporal workflow {}", workflowId); + workflow.runTask(jobProps, appWorkDir.toString(), getJobId(), workUnitFilePathStr, jobStateFilePathStr); + } catch (Exception e) { + throw new RuntimeException(e); + } + }, executor)); + } + } else { + // TODO(yiyang): how do we determine 100 tasks in total. what if more tasks come in + int numTasks = 100; + int maxBranchesPerTree = 20; + int maxSubTreesPerTree = 5; + // TODO(yiyang): workflow size is also variable but we fix our thread pool ahead of time + ExecutorService executor = Executors.newFixedThreadPool(100); futures.add(CompletableFuture.runAsync(() -> { try { - if (workUnit instanceof MultiWorkUnit) { - workUnit.setId(JobLauncherUtils.newMultiTaskId(this.jobContext.getJobId(), multiTaskIdSequence.getAndIncrement())); - } - String workUnitFilePathStr = persistWorkUnit(new Path(this.inputWorkUnitDir, this.jobContext.getJobId()), workUnit, stateSerDeRunner); - String workflowId = workUnit.getProp(KafkaSource.TOPIC_NAME) + "_" + workflowCount.getAndIncrement(); - WorkflowOptions options = WorkflowOptions.newBuilder() - .setTaskQueue(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE) - .setWorkflowId(workflowId) - .build(); - GobblinTemporalWorkflow workflow = this.client.newWorkflowStub(GobblinTemporalWorkflow.class, options); - LOGGER.info("Setting up temporal workflow {}", workflowId); - workflow.runTask(jobProps, appWorkDir.toString(), getJobId(), workUnitFilePathStr, jobStateFilePathStr); + Workload workload = SimpleGeneratedWorkload.createAs(numTasks); + // WARNING: although type param must agree w/ that of `workload`, it's entirely unverified by type checker! + // ...and more to the point, mismatch would occur at runtime (`performWork` on whichever workflow underpins stub)! + WorkflowOptions options = WorkflowOptions.newBuilder().setTaskQueue(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE).build(); + NestingExecWorkflow workflow = + this.client.newWorkflowStub(NestingExecWorkflow.class, options); + workflow.performWork(WFAddr.ROOT, workload, 0, maxBranchesPerTree, maxSubTreesPerTree, Optional.empty()); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java index 0633530e60..de99b17d97 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java @@ -33,6 +33,7 @@ import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; +import org.apache.gobblin.cluster.temporal.NestingExecWorker; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -81,6 +82,8 @@ import org.apache.gobblin.util.reflection.GobblinConstructorUtils; import static org.apache.gobblin.cluster.GobblinTemporalClusterManager.createServiceStubs; +import static org.apache.gobblin.cluster.temporal.TemporalWorkflowClientFactory.createClientInstance; +import static org.apache.gobblin.cluster.temporal.TemporalWorkflowClientFactory.createServiceInstance; /** @@ -238,47 +241,17 @@ public void start() private void initiateWorker() throws Exception{ logger.info("Starting Temporal Worker"); - WorkflowServiceStubs service = createServiceStubs(); + + WorkflowServiceStubs service = createServiceInstance(); + WorkflowClient client = createClientInstance(service); WorkerOptions workerOptions = WorkerOptions.newBuilder() .setMaxConcurrentWorkflowTaskExecutionSize(1) .setMaxConcurrentActivityExecutionSize(1) .build(); - // WorkflowClient can be used to start, signal, query, cancel, and terminate Workflows. - WorkflowClient client = - WorkflowClient.newInstance( - service, WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build()); - - /* - * Define the workflow factory. It is used to create workflow workers that poll specific Task Queues. - */ - WorkerFactory factory = WorkerFactory.newInstance(client); - - /* - * Define the workflow worker. Workflow workers listen to a defined task queue and process - * workflows and activities. - */ - Worker worker = factory.newWorker(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE, workerOptions); - - /* - * Register our workflow implementation with the worker. - * Workflow implementations must be known to the worker at runtime in - * order to dispatch workflow tasks. - */ - worker.registerWorkflowImplementationTypes(GobblinTemporalWorkflowImpl.class); - - /* - * Register our Activity Types with the Worker. Since Activities are stateless and thread-safe, - * the Activity Type is a shared instance. - */ - worker.registerActivitiesImplementations(new GobblinTemporalActivityImpl()); - - /* - * Start all the workers registered for a specific task queue. - * The started workers then start polling for workflows and activities. - */ - factory.start(); + NestingExecWorker worker = new NestingExecWorker(client, Shared.GOBBLIN_TEMPORAL_TASK_QUEUE); + worker.start(); logger.info("A new worker is started."); } diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractNestingExecWorkflowImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractNestingExecWorkflowImpl.java new file mode 100644 index 0000000000..b04c4e9a35 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractNestingExecWorkflowImpl.java @@ -0,0 +1,116 @@ +package org.apache.gobblin.cluster.temporal; + +import io.temporal.api.enums.v1.ParentClosePolicy; +import io.temporal.workflow.Async; +import io.temporal.workflow.ChildWorkflowOptions; +import io.temporal.workflow.Promise; +import io.temporal.workflow.Workflow; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.compress.utils.Lists; + +/** Core skeleton of {@link NestingExecWorkflow}: realizing classes need only define {@link #launchAsyncActivity} */ +@Slf4j +public abstract class AbstractNestingExecWorkflowImpl implements NestingExecWorkflow { + @Override + public int performWork( + final WFAddr addr, + final Workload workload, + final int startIndex, + final int maxBranchesPerTree, + final int maxSubTreesPerTree, + final Optional maxSubTreesForCurrentTreeOverride) { + final int maxSubTreesForCurrent = maxSubTreesForCurrentTreeOverride.orElse(maxSubTreesPerTree); + final int maxLeaves = maxBranchesPerTree - maxSubTreesForCurrent; + final Optional> optSpan = workload.getSpan(startIndex, maxLeaves); + log.info("[" + addr + "] " + workload + " w/ start '" + startIndex + "'" + "; tree (" + maxBranchesPerTree + "/" + maxSubTreesPerTree + "): " + optSpan); + if (!optSpan.isPresent()) { + return 0; + } else { + final Workload.TaskSpan taskSpan = optSpan.get(); + final Iterable iterable = () -> taskSpan; + final List> childActivities = StreamSupport.stream(iterable.spliterator(), false) + .map(t -> launchAsyncActivity(t)) + .collect(Collectors.toList()); + final List> childSubTrees = new ArrayList<>(); + if (taskSpan.getNumElems() == maxLeaves) { // received as many as requested (did not stop short) + int subTreeId = 0; + for (int subTreeChildMaxSubTreesPerTree + : consolidateSubTreeGrandChildren(maxSubTreesForCurrent, maxBranchesPerTree, maxSubTreesPerTree)) { + // CAUTION: calc these *before* incrementing `subTreeId`! + final int childStartIndex = startIndex + maxLeaves + (maxBranchesPerTree * subTreeId); + final int nextChildId = maxLeaves + subTreeId; + final WFAddr childAddr = addr.createChild(nextChildId); + final NestingExecWorkflow child = createChildWorkflow(childAddr); + if (!workload.isIndexKnownToExceed(childStartIndex)) { // best-effort short-circuiting + childSubTrees.add( + Async.function(child::performWork, childAddr, workload, childStartIndex, maxBranchesPerTree, + maxSubTreesPerTree, Optional.of(subTreeChildMaxSubTreesPerTree))); + ++subTreeId; + } + } + } + final Promise allActivityChildren = Promise.allOf(childActivities); + allActivityChildren.get(); // ensure all complete prior to counting them in `overallActivitiesRollupCount` + // TODO: determine whether any benefit to unordered `::get` blocking for any next ready (perhaps no difference...) + final int descendantActivitiesRollupCount = childSubTrees.stream().map(Promise::get).reduce(0, (x, y) -> x + y); + final int overallActivitiesRollupCount = taskSpan.getNumElems() + descendantActivitiesRollupCount; + log.info("[" + addr + "] activites finished coordinating: " + overallActivitiesRollupCount); + return overallActivitiesRollupCount; + } + } + + /** Factory for invoking the specific activity by providing it args via {@link Async::function} */ + protected abstract Promise launchAsyncActivity(TASK task); + + protected NestingExecWorkflow createChildWorkflow(final WFAddr childAddr) { + ChildWorkflowOptions childOpts = ChildWorkflowOptions.newBuilder() + .setParentClosePolicy(ParentClosePolicy.PARENT_CLOSE_POLICY_ABANDON) + .setWorkflowId("NestingExecWorkflow-" + childAddr) + .build(); + return Workflow.newChildWorkflowStub(NestingExecWorkflow.class, childOpts); + } + + /** + * "right-tilt" sub-tree's grandchildren, so final child gets all grandchildren (vs. constant grandchildren/child) + * i.e. NOT!: + * List naiveUniformity = Collections.nCopies(numSubTreesPerSubTree, numSubTreeChildren); + * @return each sub-tree's desired size, in ascending sub-tree order + */ + protected static List consolidateSubTreeGrandChildren( + final int numSubTreesPerSubTree, + final int numChildrenTotal, + final int numSubTreeChildren + ) { + if (numSubTreesPerSubTree <= 0) { + return Lists.newArrayList(); + } else if (isSqrt(numSubTreeChildren, numChildrenTotal)) { + // redistribute all grandchild sub-trees to pack every grandchild beneath the final child sub-tree + final List grandChildCounts = new ArrayList<>(Collections.nCopies(numSubTreesPerSubTree - 1, 0)); + grandChildCounts.add(numChildrenTotal); + return grandChildCounts; + } else { + final int totalGrandChildSubTrees = numSubTreesPerSubTree * numSubTreeChildren; + final int numTreesWithSolelySubTreeBranches = totalGrandChildSubTrees / numChildrenTotal; + final int numSubTreesRemaining = totalGrandChildSubTrees % numChildrenTotal; + assert (numTreesWithSolelySubTreeBranches == 1 && numSubTreesRemaining == 0) || numTreesWithSolelySubTreeBranches == 0 + : "present limitation: at most one sub-tree may use further branching: (found: numSubTreesPerSubTree: " + + numSubTreesPerSubTree + "; numChildrenTotal: " + numChildrenTotal + " / numSubTreeChildren: " + + numSubTreeChildren + ")"; + final List grandChildCounts = new ArrayList<>(Collections.nCopies(numSubTreesPerSubTree - (numTreesWithSolelySubTreeBranches + 1), 0)); + grandChildCounts.addAll(Collections.nCopies(Math.min(1, numSubTreesPerSubTree - numTreesWithSolelySubTreeBranches), numSubTreesRemaining)); + grandChildCounts.addAll(Collections.nCopies(Math.min(numTreesWithSolelySubTreeBranches, numSubTreesPerSubTree), numChildrenTotal)); + return grandChildCounts; + } + } + + /** @return whether `maxSubTrees` == `Math.sqrt(maxBranches)` */ + private static boolean isSqrt(int maxSubTrees, int maxBranches) { + return maxSubTrees > 0 && maxSubTrees * maxSubTrees == maxBranches; + } +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractTemporalWorker.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractTemporalWorker.java new file mode 100644 index 0000000000..0613eed020 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractTemporalWorker.java @@ -0,0 +1,42 @@ +package org.apache.gobblin.cluster.temporal; + +import io.temporal.client.WorkflowClient; +import io.temporal.worker.Worker; +import io.temporal.worker.WorkerOptions; +import io.temporal.worker.WorkerFactory; +public abstract class AbstractTemporalWorker { + private final WorkflowClient workflowClient; + private final String queueName; + private final WorkerFactory workerFactory; + + public AbstractTemporalWorker(WorkflowClient client, String queue) { + workflowClient = client; + queueName = queue; + // Create a Worker factory that can be used to create Workers that poll specific Task Queues. + workerFactory = WorkerFactory.newInstance(workflowClient); + } + + public void start() { + Worker worker = workerFactory.newWorker(queueName); + // This Worker hosts both Workflow and Activity implementations. + // Workflows are stateful, so you need to supply a type to create instances. + worker.registerWorkflowImplementationTypes(getWorkflowImplClasses()); + // Activities are stateless and thread safe, so a shared instance is used. + worker.registerActivitiesImplementations(getActivityImplInstances()); + // Start polling the Task Queue. + workerFactory.start(); + } + + /** + * Shuts down the worker. + */ + public void shutdown() { + workerFactory.shutdown(); + } + + /** @return workflow types for *implementation* classes (not interface) */ + protected abstract Class[] getWorkflowImplClasses(); + + /** @return activity instances; NOTE: activities must be stateless and thread-safe, so a shared instance is used. */ + protected abstract Object[] getActivityImplInstances(); +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/CollectionBackedTaskSpan.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/CollectionBackedTaskSpan.java new file mode 100644 index 0000000000..a1b302afe9 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/CollectionBackedTaskSpan.java @@ -0,0 +1,51 @@ +package org.apache.gobblin.cluster.temporal; +import java.util.Iterator; +import java.util.List; +import lombok.NoArgsConstructor; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; + + +/** Logical sub-sequence of `Task`s, backed for simplicity's sake by an in-memory collection */ +@NoArgsConstructor +@RequiredArgsConstructor +public class CollectionBackedTaskSpan implements Workload.TaskSpan { + @NonNull + private List elems; + // CAUTION: despite the "warning: @NonNull is meaningless on a primitive @lombok.RequiredArgsConstructor"... + // if removed, no two-arg ctor is generated, so syntax error on `new CollectionBackedTaskSpan(elems, startIndex)` + @NonNull + private int startingIndex; + private transient Iterator statefulDelegatee = null; + + @Override + public int getNumElems() { + return elems.size(); + } + + @Override + public boolean hasNext() { + if (statefulDelegatee == null) { + statefulDelegatee = elems.iterator(); + } + return statefulDelegatee.hasNext(); + } + + @Override + public T next() { + if (statefulDelegatee == null) { + throw new IllegalStateException("first call `hasNext()`!"); + } + return statefulDelegatee.next(); + } + + @Override + public String toString() { + return getClassNickname() + "(" + startingIndex + "... {+" + getNumElems() + "})"; + } + + protected String getClassNickname() { + // return getClass().getSimpleName(); + return "TaskSpan"; + } +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTask.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTask.java new file mode 100644 index 0000000000..43f1e6bd6f --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTask.java @@ -0,0 +1,16 @@ +package org.apache.gobblin.cluster.temporal; + +import lombok.Data; +import lombok.NoArgsConstructor; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; + + +/** Generally, this would specify what "work" needs performing plus how to perform; now just a unique name (to log) */ +@Data +@NoArgsConstructor // IMPORTANT: for jackson (de)serialization +@RequiredArgsConstructor +public class IllustrationTask { + @NonNull + private String name; +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivity.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivity.java new file mode 100644 index 0000000000..d24217fe45 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivity.java @@ -0,0 +1,16 @@ +package org.apache.gobblin.cluster.temporal; + +import io.temporal.activity.ActivityInterface; +import io.temporal.activity.ActivityMethod; +/** + * Activity for processing {@link IllustrationTask}s + * + * CAUTION/FINDING: an `@ActivityInterface` must not be parameterized (e.g. here, by TASK), as doing so results in: + * io.temporal.failure.ApplicationFailure: message='class java.util.LinkedHashMap cannot be cast to class + * com.linkedin.temporal.app.work.IllustrationTask', type='java.lang.ClassCastException' + */ +@ActivityInterface +public interface IllustrationTaskActivity { + @ActivityMethod + String doTask(IllustrationTask task); +} \ No newline at end of file diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivityImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivityImpl.java new file mode 100644 index 0000000000..c170829f09 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivityImpl.java @@ -0,0 +1,12 @@ +package org.apache.gobblin.cluster.temporal; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class IllustrationTaskActivityImpl implements IllustrationTaskActivity { + @Override + public String doTask(final IllustrationTask task) { + log.info("Now performing - '" + task.getName() + "'"); + return task.getName(); + } +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorker.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorker.java new file mode 100644 index 0000000000..327a28c8d9 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorker.java @@ -0,0 +1,21 @@ +package org.apache.gobblin.cluster.temporal; + +import io.temporal.client.WorkflowClient; +import io.temporal.worker.Worker; +import io.temporal.worker.WorkerOptions; +import io.temporal.worker.WorkerFactory; +public class NestingExecWorker extends AbstractTemporalWorker{ + public NestingExecWorker(WorkflowClient workflowClient, String queueName) { + super(workflowClient, queueName); + } + + @Override + protected Class[] getWorkflowImplClasses() { + return new Class[] { NestingExecWorkflowImpl.class }; + } + + @Override + protected Object[] getActivityImplInstances() { + return new Object[] { new IllustrationTaskActivityImpl() }; + } +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflow.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflow.java new file mode 100644 index 0000000000..8887d71b74 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflow.java @@ -0,0 +1,30 @@ +package org.apache.gobblin.cluster.temporal; + +import io.temporal.workflow.WorkflowInterface; +import io.temporal.workflow.WorkflowMethod; +import java.util.Optional; + +/** + * Process all `TASK`s of `workload`, from `startIndex` to the end by creating child workflows, where this and + * descendants should have at most `maxBranchesPerTree`, with at most `maxSubTreesPerTree` of those being child + * workflows. (Non-child-workflow branches being activities.) + * + * IMPORTANT: `Math.sqrt(maxBranchesPerTree) == maxSubTreesPerTree` provides a good rule-of-thumb; `maxSubTreesPerTree + * should not exceed that. + * + * @param the type of task for which to invoke an appropriate activity + * @param maxSubTreesForCurrentTreeOverride when the current tree should use different max sub-trees than descendants + */ + +@WorkflowInterface +public interface NestingExecWorkflow { + @WorkflowMethod + int performWork( + WFAddr addr, + Workload workload, + int startIndex, + int maxBranchesPerTree, + int maxSubTreesPerTree, + Optional maxSubTreesForCurrentTreeOverride + ); +} \ No newline at end of file diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflowImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflowImpl.java new file mode 100644 index 0000000000..4f16c1101c --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflowImpl.java @@ -0,0 +1,34 @@ +package org.apache.gobblin.cluster.temporal; + +import io.temporal.activity.ActivityOptions; +import io.temporal.common.RetryOptions; +import io.temporal.workflow.Async; +import io.temporal.workflow.Promise; +import io.temporal.workflow.Workflow; +import java.time.Duration; + +/** {@link com.linkedin.temporal.app.workflow.nesting.NestingExecWorkflow} for {@link IllustrationTask} */ +public class NestingExecWorkflowImpl + extends AbstractNestingExecWorkflowImpl { + + // RetryOptions specify how to automatically handle retries when Activities fail. + private static final RetryOptions ACTIVITY_RETRY_OPTS = RetryOptions.newBuilder() + .setInitialInterval(Duration.ofSeconds(1)) + .setMaximumInterval(Duration.ofSeconds(100)) + .setBackoffCoefficient(2) + .setMaximumAttempts(3) + .build(); + + private static final ActivityOptions ACTIVITY_OPTS = ActivityOptions.newBuilder() + .setStartToCloseTimeout(Duration.ofSeconds(10)) + .setRetryOptions(ACTIVITY_RETRY_OPTS) + .build(); + + private final IllustrationTaskActivity activityStub = + Workflow.newActivityStub(IllustrationTaskActivity.class, ACTIVITY_OPTS); + + @Override + protected Promise launchAsyncActivity(final IllustrationTask t) { + return Async.function(activityStub::doTask, t); + } +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java index b282d36275..3d7e51a459 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java @@ -21,5 +21,4 @@ public interface Shared { // Define the task queue name final String GOBBLIN_TEMPORAL_TASK_QUEUE = "GobblinTemporalTaskQueue"; - } \ No newline at end of file diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/SimpleGeneratedWorkload.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/SimpleGeneratedWorkload.java new file mode 100644 index 0000000000..93461bfe27 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/SimpleGeneratedWorkload.java @@ -0,0 +1,44 @@ +package org.apache.gobblin.cluster.temporal; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import lombok.AccessLevel; + +/** Example, illustration workload that synthesizes tasks; genuine {@link Workload}s likely arise from query/calc */ +@lombok.AllArgsConstructor(access = AccessLevel.PRIVATE) +@lombok.NoArgsConstructor // IMPORTANT: for jackson (de)serialization +@lombok.ToString +public class SimpleGeneratedWorkload implements Workload { + private int numTasks; + + /** Factory method */ + public static SimpleGeneratedWorkload createAs(final int numTasks) { + return new SimpleGeneratedWorkload(numTasks); + } + + @Override + public Optional> getSpan(final int startIndex, final int numElements) { + if (startIndex >= numTasks || startIndex < 0) { + return Optional.empty(); + } else { + List elems = IntStream.range(startIndex, Math.min(startIndex + numElements, numTasks)) + .mapToObj(n -> new IllustrationTask("task-" + n + "-of-" + numTasks)) + .collect(Collectors.toList()); + return Optional.of(new CollectionBackedTaskSpan<>(elems, startIndex)); + } + } + + @Override + public boolean isIndexKnownToExceed(final int index) { + return isDefiniteSize() && index >= numTasks; + } + + @Override + @JsonIgnore // (because no-arg method resembles 'java bean property') + public boolean isDefiniteSize() { + return true; + } +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java new file mode 100644 index 0000000000..76b6dd983c --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java @@ -0,0 +1,92 @@ +package org.apache.gobblin.cluster.temporal; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import io.grpc.netty.shaded.io.grpc.netty.GrpcSslContexts; +import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext; +import io.temporal.client.WorkflowClient; +import io.temporal.client.WorkflowClientOptions; +import io.temporal.serviceclient.WorkflowServiceStubs; +import io.temporal.serviceclient.WorkflowServiceStubsOptions; +import org.apache.gobblin.cluster.GobblinClusterUtils; + +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.TrustManagerFactory; +import java.io.File; +import java.security.KeyStore; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.gobblin.security.ssl.SSLContextFactory.toInputStream; + +public class TemporalWorkflowClientFactory { + + public static WorkflowServiceStubs createServiceInstance() throws Exception { + GobblinClusterUtils.setSystemProperties(ConfigFactory.load()); + Config config = GobblinClusterUtils.addDynamicConfig(ConfigFactory.load()); + String SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT = "gobblin.kafka.sharedConfig."; + String SSL_KEYMANAGER_ALGORITHM = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.keymanager.algorithm"; + String SSL_KEYSTORE_TYPE = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.keystore.type"; + String SSL_KEYSTORE_LOCATION = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.keystore.location"; + String SSL_KEY_PASSWORD = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.key.password"; + String SSL_TRUSTSTORE_LOCATION = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.truststore.location"; + String SSL_TRUSTSTORE_PASSWORD = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.truststore.password"; + + List SSL_CONFIG_DEFAULT_SSL_PROTOCOLS = Collections.unmodifiableList( + Arrays.asList("TLSv1.2")); + List SSL_CONFIG_DEFAULT_CIPHER_SUITES = Collections.unmodifiableList(Arrays.asList( + // The following list is from https://github.com/netty/netty/blob/4.1/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2SecurityUtil.java#L50 + "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256", + + /* REQUIRED BY HTTP/2 SPEC */ + "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", + /* REQUIRED BY HTTP/2 SPEC */ + + "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384", + "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384", + "TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256", + "TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256" + )); + + String keyStoreType = config.getString(SSL_KEYSTORE_TYPE); + File keyStoreFile = new File(config.getString(SSL_KEYSTORE_LOCATION)); + String keyStorePassword = config.getString(SSL_KEY_PASSWORD); + + KeyStore keyStore = KeyStore.getInstance(keyStoreType); + keyStore.load(toInputStream(keyStoreFile), keyStorePassword.toCharArray()); + + // Set key manager from key store + String sslKeyManagerAlgorithm = config.getString(SSL_KEYMANAGER_ALGORITHM); + KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(sslKeyManagerAlgorithm); + keyManagerFactory.init(keyStore, keyStorePassword.toCharArray()); + + // Set trust manager from trust store + KeyStore trustStore = KeyStore.getInstance("JKS"); + File trustStoreFile = new File(config.getString(SSL_TRUSTSTORE_LOCATION)); + + String trustStorePassword = config.getString(SSL_TRUSTSTORE_PASSWORD); + trustStore.load(toInputStream(trustStoreFile), trustStorePassword.toCharArray()); + TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("SunX509"); + trustManagerFactory.init(trustStore); + + SslContext sslContext = GrpcSslContexts.forClient() + .keyManager(keyManagerFactory) + .trustManager(trustManagerFactory) + .protocols(SSL_CONFIG_DEFAULT_SSL_PROTOCOLS) + .ciphers(SSL_CONFIG_DEFAULT_CIPHER_SUITES) + .build(); + + WorkflowServiceStubsOptions options = WorkflowServiceStubsOptions.newBuilder() + .setTarget("1.nephos-temporal.corp-lca1.atd.corp.linkedin.com:7233") + .setEnableHttps(true) + .setSslContext(sslContext) + .build(); + return WorkflowServiceStubs.newServiceStubs(options); + } + + public static WorkflowClient createClientInstance(WorkflowServiceStubs service) { + WorkflowClientOptions options = WorkflowClientOptions.newBuilder().setNamespace("gobblin-gaas-poc").build(); + return WorkflowClient.newInstance(service, options); + } +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/WFAddr.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/WFAddr.java new file mode 100644 index 0000000000..e7de7f347c --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/WFAddr.java @@ -0,0 +1,48 @@ +package org.apache.gobblin.cluster.temporal; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; + + +/** Hierarchical address for nesting workflows (0-based). */ +@NoArgsConstructor // IMPORTANT: for jackson (de)serialization +@RequiredArgsConstructor +public class WFAddr { + public static final String SEP = "."; + + /** initial, top-level address */ + public static final WFAddr ROOT = new WFAddr(0); + + @Getter + @NonNull // IMPORTANT: for jackson (de)serialization (which won't permit `final`) + private List segments; + + public WFAddr(final int firstLevelOnly) { + this(Lists.newArrayList(firstLevelOnly)); + } + + /** @return 0-based depth */ + @JsonIgnore // (because no-arg method resembles 'java bean property') + public int getDepth() { + return segments.size() - 1; + } + + /** Create a child of the current `WFAddr` */ + public WFAddr createChild(int childLevel) { + final List copy = new ArrayList<>(segments); + copy.add(childLevel); + return new WFAddr(copy); + } + + @Override + public String toString() { + return Joiner.on(SEP).join(segments); + } +} diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Workload.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Workload.java new file mode 100644 index 0000000000..2957eeb2a7 --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Workload.java @@ -0,0 +1,37 @@ +package org.apache.gobblin.cluster.temporal; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import java.util.Iterator; +import java.util.Optional; + + +/** + * An assemblage of "work", modeled as sequential "task" specifications. Given Temporal's required determinism, tasks + * and task spans should remain unchanged, with stable sequential ordering. This need not constrain `Workload`s to + * eager, advance elaboration: "streaming" definition is possible, so long as producing a deterministic result. + * + * A actual, real-world workload might correspond to datastore contents, such as records serialized into HDFS files + * or ordered DB query results. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.CLASS, include = JsonTypeInfo.As.PROPERTY, property = "@class") // to handle impls + +public interface Workload { + + /** + * @return a sequential sub-sequence, from `startIndex` (0-based), unless it falls beyond the underlying sequence + * NOTE: this is a blocking call that forces elaboration: `TaskSpan.getNumElems() < numElements` signifies end of seq + */ + Optional> getSpan(int startIndex, int numElements); + + /** Non-blocking, best-effort advice: to support non-strict elaboration, does NOT guarantee `index` will not exceed */ + boolean isIndexKnownToExceed(int index); + + default boolean isDefiniteSize() { + return false; + } + + /** Logical sub-sequence 'slice' of contiguous "tasks" */ + public interface TaskSpan extends Iterator { + int getNumElems(); + } +} \ No newline at end of file From b21def1df5693986c918f5f735150dbd786ece64 Mon Sep 17 00:00:00 2001 From: Yiming Yang Date: Tue, 29 Aug 2023 11:39:48 -0700 Subject: [PATCH 6/8] minor changes --- .../org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java | 2 +- .../gobblin/cluster/temporal/TemporalWorkflowClientFactory.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java index ee7b803b81..239ce55c82 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java @@ -285,7 +285,7 @@ private void submitJobToTemporal(List workUnits) throws Exception{ int maxBranchesPerTree = 20; int maxSubTreesPerTree = 5; // TODO(yiyang): workflow size is also variable but we fix our thread pool ahead of time - ExecutorService executor = Executors.newFixedThreadPool(100); + ExecutorService executor = Executors.newFixedThreadPool(1); futures.add(CompletableFuture.runAsync(() -> { try { Workload workload = SimpleGeneratedWorkload.createAs(numTasks); diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java index 76b6dd983c..7a958bb785 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java @@ -86,7 +86,7 @@ public static WorkflowServiceStubs createServiceInstance() throws Exception { } public static WorkflowClient createClientInstance(WorkflowServiceStubs service) { - WorkflowClientOptions options = WorkflowClientOptions.newBuilder().setNamespace("gobblin-gaas-poc").build(); + WorkflowClientOptions options = WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build(); return WorkflowClient.newInstance(service, options); } } From ec381f92c68499a2fa2e9270f9501788176d2734 Mon Sep 17 00:00:00 2001 From: Yiming Yang Date: Thu, 31 Aug 2023 10:20:24 -0700 Subject: [PATCH 7/8] fix a few things --- .../org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java index 239ce55c82..a823fafa72 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java @@ -280,11 +280,9 @@ private void submitJobToTemporal(List workUnits) throws Exception{ }, executor)); } } else { - // TODO(yiyang): how do we determine 100 tasks in total. what if more tasks come in int numTasks = 100; int maxBranchesPerTree = 20; int maxSubTreesPerTree = 5; - // TODO(yiyang): workflow size is also variable but we fix our thread pool ahead of time ExecutorService executor = Executors.newFixedThreadPool(1); futures.add(CompletableFuture.runAsync(() -> { try { From 2d08d53c1461ab21185e3bc37a04f139785b3e4a Mon Sep 17 00:00:00 2001 From: Yiming Yang Date: Fri, 1 Sep 2023 13:58:01 -0700 Subject: [PATCH 8/8] make properties --- .../gobblin/cluster/GobblinTemporalJobLauncher.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java index a823fafa72..6c06255861 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java @@ -270,7 +270,6 @@ private void submitJobToTemporal(List workUnits) throws Exception{ .setTaskQueue(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE) .setWorkflowId(workflowId) .build(); - // TODO(yiyang): change up the workflow GobblinTemporalWorkflow workflow = this.client.newWorkflowStub(GobblinTemporalWorkflow.class, options); LOGGER.info("Setting up temporal workflow {}", workflowId); workflow.runTask(jobProps, appWorkDir.toString(), getJobId(), workUnitFilePathStr, jobStateFilePathStr); @@ -280,9 +279,11 @@ private void submitJobToTemporal(List workUnits) throws Exception{ }, executor)); } } else { - int numTasks = 100; - int maxBranchesPerTree = 20; - int maxSubTreesPerTree = 5; + // l0: (root)workflow + // l1: act0, act2, .... act14, (sub)workflow0, ... (sub)workflow4 + int numTasks = PropertiesUtils.getPropAsInt(this.jobProps, "temporal.task.size", 100); + int maxBranchesPerTree = PropertiesUtils.getPropAsInt(this.jobProps, "temporal.task.maxBranchesPerTree", 20); + int maxSubTreesPerTree = PropertiesUtils.getPropAsInt(this.jobProps, "temporal.task.maxSubTreesPerTree", 5); ExecutorService executor = Executors.newFixedThreadPool(1); futures.add(CompletableFuture.runAsync(() -> { try {