|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | + |
| 20 | +package org.apache.samza.runtime; |
| 21 | + |
| 22 | +import org.apache.samza.SamzaException; |
| 23 | +import org.apache.samza.application.descriptors.ApplicationDescriptor; |
| 24 | +import org.apache.samza.application.descriptors.ApplicationDescriptorImpl; |
| 25 | +import org.apache.samza.config.Config; |
| 26 | +import org.apache.samza.config.ShellCommandConfig; |
| 27 | +import org.apache.samza.container.ContainerHeartbeatClient; |
| 28 | +import org.apache.samza.container.ContainerHeartbeatMonitor; |
| 29 | +import org.apache.samza.container.LocalityManager; |
| 30 | +import org.apache.samza.container.SamzaContainer; |
| 31 | +import org.apache.samza.container.SamzaContainer$; |
| 32 | +import org.apache.samza.container.SamzaContainerListener; |
| 33 | +import org.apache.samza.context.ExternalContext; |
| 34 | +import org.apache.samza.context.JobContextImpl; |
| 35 | +import org.apache.samza.job.model.JobModel; |
| 36 | +import org.apache.samza.metrics.MetricsRegistryMap; |
| 37 | +import org.apache.samza.metrics.MetricsReporter; |
| 38 | +import org.apache.samza.task.TaskFactory; |
| 39 | +import org.apache.samza.task.TaskFactoryUtil; |
| 40 | +import org.apache.samza.util.ScalaJavaUtil; |
| 41 | +import org.slf4j.Logger; |
| 42 | +import org.slf4j.LoggerFactory; |
| 43 | +import scala.Option; |
| 44 | + |
| 45 | +import java.util.HashMap; |
| 46 | +import java.util.Map; |
| 47 | +import java.util.Optional; |
| 48 | + |
| 49 | +public class ContainerLaunchUtil { |
| 50 | + private static final Logger log = LoggerFactory.getLogger(ContainerLaunchUtil.class); |
| 51 | + |
| 52 | + private static volatile Throwable containerRunnerException = null; |
| 53 | + |
| 54 | + /** |
| 55 | + * This method launches a Samza container in a managed cluster, e.g. Yarn. |
| 56 | + * |
| 57 | + * NOTE: this util method is also invoked by Beam SamzaRunner. |
| 58 | + * Any change here needs to take Beam into account. |
| 59 | + */ |
| 60 | + public static void run( |
| 61 | + ApplicationDescriptorImpl<? extends ApplicationDescriptor> appDesc, |
| 62 | + String containerId, |
| 63 | + JobModel jobModel) { |
| 64 | + |
| 65 | + Config config = jobModel.getConfig(); |
| 66 | + run(appDesc, containerId, jobModel, config, buildExternalContext(config)); |
| 67 | + |
| 68 | + System.exit(0); |
| 69 | + } |
| 70 | + |
| 71 | + private static void run( |
| 72 | + ApplicationDescriptorImpl<? extends ApplicationDescriptor> appDesc, |
| 73 | + String containerId, |
| 74 | + JobModel jobModel, |
| 75 | + Config config, |
| 76 | + Optional<ExternalContext> externalContextOptional) { |
| 77 | + TaskFactory taskFactory = TaskFactoryUtil.getTaskFactory(appDesc); |
| 78 | + LocalityManager localityManager = new LocalityManager(config, new MetricsRegistryMap()); |
| 79 | + SamzaContainer container = SamzaContainer$.MODULE$.apply( |
| 80 | + containerId, |
| 81 | + jobModel, |
| 82 | + ScalaJavaUtil.toScalaMap(loadMetricsReporters(appDesc, containerId, config)), |
| 83 | + taskFactory, |
| 84 | + JobContextImpl.fromConfigWithDefaults(config), |
| 85 | + Option.apply(appDesc.getApplicationContainerContextFactory().orElse(null)), |
| 86 | + Option.apply(appDesc.getApplicationTaskContextFactory().orElse(null)), |
| 87 | + Option.apply(externalContextOptional.orElse(null)), localityManager); |
| 88 | + |
| 89 | + ProcessorLifecycleListener listener = appDesc.getProcessorLifecycleListenerFactory() |
| 90 | + .createInstance(new ProcessorContext() { }, config); |
| 91 | + |
| 92 | + container.setContainerListener( |
| 93 | + new SamzaContainerListener() { |
| 94 | + @Override |
| 95 | + public void beforeStart() { |
| 96 | + log.info("Before starting the container."); |
| 97 | + listener.beforeStart(); |
| 98 | + } |
| 99 | + |
| 100 | + @Override |
| 101 | + public void afterStart() { |
| 102 | + log.info("Container Started"); |
| 103 | + listener.afterStart(); |
| 104 | + } |
| 105 | + |
| 106 | + @Override |
| 107 | + public void afterStop() { |
| 108 | + log.info("Container Stopped"); |
| 109 | + listener.afterStop(); |
| 110 | + } |
| 111 | + |
| 112 | + @Override |
| 113 | + public void afterFailure(Throwable t) { |
| 114 | + log.info("Container Failed"); |
| 115 | + containerRunnerException = t; |
| 116 | + listener.afterFailure(t); |
| 117 | + } |
| 118 | + }); |
| 119 | + |
| 120 | + ContainerHeartbeatMonitor heartbeatMonitor = createContainerHeartbeatMonitor(container); |
| 121 | + if (heartbeatMonitor != null) { |
| 122 | + heartbeatMonitor.start(); |
| 123 | + } |
| 124 | + |
| 125 | + container.run(); |
| 126 | + if (heartbeatMonitor != null) { |
| 127 | + heartbeatMonitor.stop(); |
| 128 | + } |
| 129 | + |
| 130 | + if (containerRunnerException != null) { |
| 131 | + log.error("Container stopped with Exception. Exiting process now.", containerRunnerException); |
| 132 | + System.exit(1); |
| 133 | + } |
| 134 | + } |
| 135 | + |
| 136 | + private static Optional<ExternalContext> buildExternalContext(Config config) { |
| 137 | + /* |
| 138 | + * By default, use an empty ExternalContext here. In a custom fork of Samza, this can be implemented to pass |
| 139 | + * a non-empty ExternalContext to SamzaContainer. Only config should be used to build the external context. In the |
| 140 | + * future, components like the application descriptor may not be available to LocalContainerRunner. |
| 141 | + */ |
| 142 | + return Optional.empty(); |
| 143 | + } |
| 144 | + |
| 145 | + // TODO: this is going away when SAMZA-1168 is done and the initialization of metrics reporters are done via |
| 146 | + // LocalApplicationRunner#createStreamProcessor() |
| 147 | + private static Map<String, MetricsReporter> loadMetricsReporters( |
| 148 | + ApplicationDescriptorImpl<? extends ApplicationDescriptor> appDesc, String containerId, Config config) { |
| 149 | + Map<String, MetricsReporter> reporters = new HashMap<>(); |
| 150 | + appDesc.getMetricsReporterFactories().forEach((name, factory) -> |
| 151 | + reporters.put(name, factory.getMetricsReporter(name, containerId, config))); |
| 152 | + return reporters; |
| 153 | + } |
| 154 | + |
| 155 | + /** |
| 156 | + * Creates a new container heartbeat monitor if possible. |
| 157 | + * @param container the container to monitor |
| 158 | + * @return a new {@link ContainerHeartbeatMonitor} instance, or null if could not create one |
| 159 | + */ |
| 160 | + private static ContainerHeartbeatMonitor createContainerHeartbeatMonitor(SamzaContainer container) { |
| 161 | + String coordinatorUrl = System.getenv(ShellCommandConfig.ENV_COORDINATOR_URL()); |
| 162 | + String executionEnvContainerId = System.getenv(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID()); |
| 163 | + if (executionEnvContainerId != null) { |
| 164 | + log.info("Got execution environment container id: {}", executionEnvContainerId); |
| 165 | + return new ContainerHeartbeatMonitor(() -> { |
| 166 | + try { |
| 167 | + container.shutdown(); |
| 168 | + containerRunnerException = new SamzaException("Container shutdown due to expired heartbeat"); |
| 169 | + } catch (Exception e) { |
| 170 | + log.error("Heartbeat monitor failed to shutdown the container gracefully. Exiting process.", e); |
| 171 | + System.exit(1); |
| 172 | + } |
| 173 | + }, new ContainerHeartbeatClient(coordinatorUrl, executionEnvContainerId)); |
| 174 | + } else { |
| 175 | + log.warn("Execution environment container id not set. Container heartbeat monitor will not be created"); |
| 176 | + return null; |
| 177 | + } |
| 178 | + } |
| 179 | +} |
0 commit comments