cameronlee314 commented on a change in pull request #1529: URL: https://github.com/apache/samza/pull/1529#discussion_r712393785
########## File path: samza-core/src/main/java/org/apache/samza/coordinator/StaticResourceJobCoordinator.java ########## @@ -0,0 +1,232 @@ +/* + * 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.samza.coordinator; + +import java.io.IOException; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.annotations.VisibleForTesting; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.container.LocalityManager; +import org.apache.samza.container.grouper.task.TaskAssignmentManager; +import org.apache.samza.container.grouper.task.TaskPartitionAssignmentManager; +import org.apache.samza.coordinator.communication.CoordinatorCommunication; +import org.apache.samza.coordinator.communication.CoordinatorCommunicationContext; +import org.apache.samza.coordinator.communication.HttpCoordinatorToWorkerCommunicationFactory; +import org.apache.samza.coordinator.communication.JobModelServingContext; +import org.apache.samza.coordinator.metadatastore.NamespaceAwareCoordinatorStreamStore; +import org.apache.samza.coordinator.stream.messages.SetChangelogMapping; +import org.apache.samza.coordinator.stream.messages.SetContainerHostMapping; +import org.apache.samza.coordinator.stream.messages.SetJobCoordinatorMetadataMessage; +import org.apache.samza.coordinator.stream.messages.SetTaskContainerMapping; +import org.apache.samza.coordinator.stream.messages.SetTaskModeMapping; +import org.apache.samza.coordinator.stream.messages.SetTaskPartitionMapping; +import org.apache.samza.job.JobCoordinatorMetadata; +import org.apache.samza.job.JobMetadataChange; +import org.apache.samza.job.metadata.JobCoordinatorMetadataManager; +import org.apache.samza.job.model.JobModel; +import org.apache.samza.job.model.JobModelUtil; +import org.apache.samza.metadatastore.MetadataStore; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.startpoint.StartpointManager; +import org.apache.samza.storage.ChangelogStreamManager; +import org.apache.samza.system.StreamMetadataCache; +import org.apache.samza.system.SystemAdmins; +import org.apache.samza.util.SystemClock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Handles coordination with the workers for a Samza job. This includes generating the job model, managing metadata, + * and coordinating with the workers when the job model changes. + * This does certain similar things as the older ClusterBasedJobCoordinator, but one notable difference is that this + * coordinator does no management of execution resources. It relies on an external component to manage those resources + * for a Samza job. + */ +public class StaticResourceJobCoordinator { + private static final Logger LOG = LoggerFactory.getLogger(StaticResourceJobCoordinator.class); + + private final JobModelHelper jobModelHelper; + private final JobModelServingContext jobModelServingContext; + private final CoordinatorCommunication coordinatorCommunication; + private final JobCoordinatorMetadataManager jobCoordinatorMetadataManager; + /** + * This can be null if startpoints are not enabled. + */ + private final StartpointManager startpointManager; + private final ChangelogStreamManager changelogStreamManager; + private final MetricsRegistry metrics; + private final SystemAdmins systemAdmins; + private final Config config; + + private final AtomicBoolean isStarted = new AtomicBoolean(false); + private final AtomicBoolean shouldShutdown = new AtomicBoolean(false); + private final CountDownLatch shutdownLatch = new CountDownLatch(1); + + public static StaticResourceJobCoordinator build(MetricsRegistry metrics, MetadataStore metadataStore, Review comment: I will be adding a factory class due to some other comments as well. ########## File path: samza-core/src/main/java/org/apache/samza/coordinator/StaticResourceJobCoordinator.java ########## @@ -0,0 +1,232 @@ +/* + * 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.samza.coordinator; + +import java.io.IOException; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.annotations.VisibleForTesting; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.container.LocalityManager; +import org.apache.samza.container.grouper.task.TaskAssignmentManager; +import org.apache.samza.container.grouper.task.TaskPartitionAssignmentManager; +import org.apache.samza.coordinator.communication.CoordinatorCommunication; +import org.apache.samza.coordinator.communication.CoordinatorCommunicationContext; +import org.apache.samza.coordinator.communication.HttpCoordinatorToWorkerCommunicationFactory; +import org.apache.samza.coordinator.communication.JobModelServingContext; +import org.apache.samza.coordinator.metadatastore.NamespaceAwareCoordinatorStreamStore; +import org.apache.samza.coordinator.stream.messages.SetChangelogMapping; +import org.apache.samza.coordinator.stream.messages.SetContainerHostMapping; +import org.apache.samza.coordinator.stream.messages.SetJobCoordinatorMetadataMessage; +import org.apache.samza.coordinator.stream.messages.SetTaskContainerMapping; +import org.apache.samza.coordinator.stream.messages.SetTaskModeMapping; +import org.apache.samza.coordinator.stream.messages.SetTaskPartitionMapping; +import org.apache.samza.job.JobCoordinatorMetadata; +import org.apache.samza.job.JobMetadataChange; +import org.apache.samza.job.metadata.JobCoordinatorMetadataManager; +import org.apache.samza.job.model.JobModel; +import org.apache.samza.job.model.JobModelUtil; +import org.apache.samza.metadatastore.MetadataStore; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.startpoint.StartpointManager; +import org.apache.samza.storage.ChangelogStreamManager; +import org.apache.samza.system.StreamMetadataCache; +import org.apache.samza.system.SystemAdmins; +import org.apache.samza.util.SystemClock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Handles coordination with the workers for a Samza job. This includes generating the job model, managing metadata, + * and coordinating with the workers when the job model changes. + * This does certain similar things as the older ClusterBasedJobCoordinator, but one notable difference is that this + * coordinator does no management of execution resources. It relies on an external component to manage those resources + * for a Samza job. + */ +public class StaticResourceJobCoordinator { + private static final Logger LOG = LoggerFactory.getLogger(StaticResourceJobCoordinator.class); + + private final JobModelHelper jobModelHelper; + private final JobModelServingContext jobModelServingContext; + private final CoordinatorCommunication coordinatorCommunication; + private final JobCoordinatorMetadataManager jobCoordinatorMetadataManager; + /** + * This can be null if startpoints are not enabled. + */ + private final StartpointManager startpointManager; + private final ChangelogStreamManager changelogStreamManager; + private final MetricsRegistry metrics; + private final SystemAdmins systemAdmins; + private final Config config; + + private final AtomicBoolean isStarted = new AtomicBoolean(false); + private final AtomicBoolean shouldShutdown = new AtomicBoolean(false); + private final CountDownLatch shutdownLatch = new CountDownLatch(1); + + public static StaticResourceJobCoordinator build(MetricsRegistry metrics, MetadataStore metadataStore, + Config config) { + JobModelServingContext jobModelServingContext = new JobModelServingContext(); + JobConfig jobConfig = new JobConfig(config); + CoordinatorCommunicationContext context = + new CoordinatorCommunicationContext(jobModelServingContext, config, metrics); + CoordinatorCommunication coordinatorCommunication = + new HttpCoordinatorToWorkerCommunicationFactory().coordinatorCommunication(context); + JobCoordinatorMetadataManager jobCoordinatorMetadataManager = new JobCoordinatorMetadataManager( + new NamespaceAwareCoordinatorStreamStore(metadataStore, SetJobCoordinatorMetadataMessage.TYPE), + JobCoordinatorMetadataManager.ClusterType.NON_YARN, metrics); + ChangelogStreamManager changelogStreamManager = + new ChangelogStreamManager(new NamespaceAwareCoordinatorStreamStore(metadataStore, SetChangelogMapping.TYPE)); + StartpointManager startpointManager = + jobConfig.getStartpointEnabled() ? new StartpointManager(metadataStore) : null; + SystemAdmins systemAdmins = new SystemAdmins(config, StaticResourceJobCoordinator.class.getSimpleName()); + StreamMetadataCache streamMetadataCache = new StreamMetadataCache(systemAdmins, 0, SystemClock.instance()); + JobModelHelper jobModelHelper = buildJobModelHelper(metadataStore, streamMetadataCache); + return new StaticResourceJobCoordinator(jobModelHelper, jobModelServingContext, coordinatorCommunication, + jobCoordinatorMetadataManager, startpointManager, changelogStreamManager, metrics, systemAdmins, config); + } + + @VisibleForTesting + StaticResourceJobCoordinator(JobModelHelper jobModelHelper, JobModelServingContext jobModelServingContext, + CoordinatorCommunication coordinatorCommunication, JobCoordinatorMetadataManager jobCoordinatorMetadataManager, + StartpointManager startpointManager, ChangelogStreamManager changelogStreamManager, MetricsRegistry metrics, + SystemAdmins systemAdmins, Config config) { + this.jobModelHelper = jobModelHelper; + this.jobModelServingContext = jobModelServingContext; + this.coordinatorCommunication = coordinatorCommunication; + this.jobCoordinatorMetadataManager = jobCoordinatorMetadataManager; + this.startpointManager = startpointManager; + this.changelogStreamManager = changelogStreamManager; + this.metrics = metrics; + this.systemAdmins = systemAdmins; + this.config = config; + } + + /** + * Run the coordinator. + */ + public void run() { + if (!isStarted.compareAndSet(false, true)) { + LOG.warn("Already running; not to going execute run() again"); + return; + } + LOG.info("Starting job coordinator"); + this.systemAdmins.start(); + this.startpointManager.start(); + try { + JobModel jobModel = newJobModel(); + JobCoordinatorMetadata newMetadata = + this.jobCoordinatorMetadataManager.generateJobCoordinatorMetadata(jobModel, jobModel.getConfig()); + Set<JobMetadataChange> jobMetadataChanges = checkForMetadataChanges(newMetadata); + prepareWorkerExecution(jobModel, newMetadata, jobMetadataChanges); + this.coordinatorCommunication.start(); + waitForShutdownQuietly(); + } catch (Exception e) { + LOG.error("Error while running job coordinator; exiting", e); + throw new SamzaException("Error while running job coordinator", e); + } finally { + this.coordinatorCommunication.stop(); + this.startpointManager.stop(); + this.systemAdmins.stop(); + } + } + + /** + * Set shutdown flag for coordinator and release any threads waiting for the shutdown. + */ + public void signalShutdown() { + if (this.shouldShutdown.compareAndSet(false, true)) { + LOG.info("Shutdown signalled"); + this.shutdownLatch.countDown(); + } + } + + private static JobModelHelper buildJobModelHelper(MetadataStore metadataStore, + StreamMetadataCache streamMetadataCache) { + LocalityManager localityManager = + new LocalityManager(new NamespaceAwareCoordinatorStreamStore(metadataStore, SetContainerHostMapping.TYPE)); + TaskAssignmentManager taskAssignmentManager = + new TaskAssignmentManager(new NamespaceAwareCoordinatorStreamStore(metadataStore, SetTaskContainerMapping.TYPE), + new NamespaceAwareCoordinatorStreamStore(metadataStore, SetTaskModeMapping.TYPE)); + TaskPartitionAssignmentManager taskPartitionAssignmentManager = new TaskPartitionAssignmentManager( + new NamespaceAwareCoordinatorStreamStore(metadataStore, SetTaskPartitionMapping.TYPE)); + return new JobModelHelper(localityManager, taskAssignmentManager, taskPartitionAssignmentManager, + streamMetadataCache, JobModelCalculator.INSTANCE); + } + + private JobModel newJobModel() { + return this.jobModelHelper.newJobModel(this.config, this.changelogStreamManager.readPartitionMapping()); + } + + /** + * Run set up steps so that workers can begin processing: + * 1. Persist job coordinator metadata + * 2. Publish new job model on coordinator-to-worker communication channel + * 3. Create metadata resources + * 4. Handle startpoints + */ + private void prepareWorkerExecution(JobModel jobModel, JobCoordinatorMetadata newMetadata, + Set<JobMetadataChange> jobMetadataChanges) throws IOException { + if (!jobMetadataChanges.isEmpty()) { + this.jobCoordinatorMetadataManager.writeJobCoordinatorMetadata(newMetadata); + } + this.jobModelServingContext.setJobModel(jobModel); + + metadataResourceUtil(jobModel).createResources(); + + // the fan out trigger logic comes from ClusterBasedJobCoordinator, in which a new job model can trigger a fan out + if (this.startpointManager != null && !jobMetadataChanges.isEmpty()) { + startpointManager.fanOut(JobModelUtil.getTaskToSystemStreamPartitions(jobModel)); + } + } + + @VisibleForTesting + MetadataResourceUtil metadataResourceUtil(JobModel jobModel) { + return new MetadataResourceUtil(jobModel, this.metrics, this.config); + } + + private void waitForShutdown() throws InterruptedException { + LOG.info("Waiting for coordinator to be signalled for shutdown"); + boolean latchReleased = false; + while (!latchReleased && !this.shouldShutdown.get()) { + /* + * Using a timeout as a defensive measure in case we are waiting for a shutdown but the latch is not triggered + * for some reason. + */ + latchReleased = this.shutdownLatch.await(15, TimeUnit.SECONDS); Review comment: 15 seconds was arbitrary. It shouldn't need to be configurable or need to be changed, because it is not expected to actually be needed. I would always expect the shutdown flag to get set along with the latch being released, so it shouldn't matter what this timeout is. Based on your questions, I'm going to remove this timeout, since it causes more confusion and it isn't helpful right now. ########## File path: samza-core/src/main/java/org/apache/samza/coordinator/StaticResourceJobCoordinator.java ########## @@ -0,0 +1,299 @@ +/* + * 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.samza.coordinator; + +import java.io.IOException; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.annotations.VisibleForTesting; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.JobCoordinatorConfig; +import org.apache.samza.container.LocalityManager; +import org.apache.samza.container.grouper.task.TaskAssignmentManager; +import org.apache.samza.container.grouper.task.TaskPartitionAssignmentManager; +import org.apache.samza.coordinator.communication.CoordinatorCommunication; +import org.apache.samza.coordinator.communication.CoordinatorCommunicationContext; +import org.apache.samza.coordinator.communication.HttpCoordinatorToWorkerCommunicationFactory; +import org.apache.samza.coordinator.communication.JobModelServingContext; +import org.apache.samza.coordinator.lifecycle.JobRestartSignal; +import org.apache.samza.coordinator.lifecycle.JobRestartSignalFactory; +import org.apache.samza.coordinator.lifecycle.JobRestartSignalFactoryContext; +import org.apache.samza.coordinator.metadatastore.NamespaceAwareCoordinatorStreamStore; +import org.apache.samza.coordinator.stream.messages.SetChangelogMapping; +import org.apache.samza.coordinator.stream.messages.SetContainerHostMapping; +import org.apache.samza.coordinator.stream.messages.SetJobCoordinatorMetadataMessage; +import org.apache.samza.coordinator.stream.messages.SetTaskContainerMapping; +import org.apache.samza.coordinator.stream.messages.SetTaskModeMapping; +import org.apache.samza.coordinator.stream.messages.SetTaskPartitionMapping; +import org.apache.samza.job.JobCoordinatorMetadata; +import org.apache.samza.job.JobMetadataChange; +import org.apache.samza.job.metadata.JobCoordinatorMetadataManager; +import org.apache.samza.job.model.JobModel; +import org.apache.samza.job.model.JobModelUtil; +import org.apache.samza.metadatastore.MetadataStore; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.startpoint.StartpointManager; +import org.apache.samza.storage.ChangelogStreamManager; +import org.apache.samza.system.StreamMetadataCache; +import org.apache.samza.system.SystemAdmins; +import org.apache.samza.util.ReflectionUtil; +import org.apache.samza.util.SystemClock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Handles coordination with the workers for a Samza job. This includes generating the job model, managing metadata, + * and coordinating with the workers when the job model changes. + * This does certain similar things as the older ClusterBasedJobCoordinator, but one notable difference is that this + * coordinator does no management of execution resources. It relies on an external component to manage those resources + * for a Samza job. + */ +public class StaticResourceJobCoordinator { + private static final Logger LOG = LoggerFactory.getLogger(StaticResourceJobCoordinator.class); + + private final JobModelHelper jobModelHelper; + private final JobModelServingContext jobModelServingContext; + private final CoordinatorCommunication coordinatorCommunication; + private final JobCoordinatorMetadataManager jobCoordinatorMetadataManager; + private final StreamPartitionCountMonitorFactory streamPartitionCountMonitorFactory; + private final StreamRegexMonitorFactory streamRegexMonitorFactory; + /** + * This can be null if startpoints are not enabled. + */ + private final StartpointManager startpointManager; + private final ChangelogStreamManager changelogStreamManager; + private final JobRestartSignal jobRestartSignal; + private final MetricsRegistry metrics; + private final SystemAdmins systemAdmins; + private final Config config; + + private final AtomicBoolean isStarted = new AtomicBoolean(false); + private final AtomicBoolean shouldShutdown = new AtomicBoolean(false); + private final CountDownLatch shutdownLatch = new CountDownLatch(1); + + public static StaticResourceJobCoordinator build(MetricsRegistry metrics, MetadataStore metadataStore, + Config config) { + JobModelServingContext jobModelServingContext = new JobModelServingContext(); + JobConfig jobConfig = new JobConfig(config); + CoordinatorCommunicationContext context = + new CoordinatorCommunicationContext(jobModelServingContext, config, metrics); + CoordinatorCommunication coordinatorCommunication = + new HttpCoordinatorToWorkerCommunicationFactory().coordinatorCommunication(context); + JobCoordinatorMetadataManager jobCoordinatorMetadataManager = new JobCoordinatorMetadataManager( + new NamespaceAwareCoordinatorStreamStore(metadataStore, SetJobCoordinatorMetadataMessage.TYPE), + JobCoordinatorMetadataManager.ClusterType.NON_YARN, metrics); Review comment: Config rewriting in open source is currently agnostic of cluster type from what I can tell. Certain specific implementations of config rewriters may care about cluster type, but I think it would be good to keep the core samza engine agnostic of it. The impls that need cluster type will need to figure out cluster type in their own way, so that the samza engine can stay decoupled. ########## File path: samza-core/src/main/java/org/apache/samza/clustermanager/JobCoordinatorLaunchUtil.java ########## @@ -76,11 +85,37 @@ public static void run(SamzaApplication app, Config config) { CoordinatorStreamUtil.writeConfigToCoordinatorStream(finalConfig, true); DiagnosticsUtil.createDiagnosticsStream(finalConfig); - ClusterBasedJobCoordinator jc = new ClusterBasedJobCoordinator( - metrics, - metadataStore, - finalConfig); - jc.run(); + if (new JobCoordinatorConfig(finalConfig).getUseStaticResourceJobCoordinator()) { + runStaticResourceJobCoordinator(metrics, metadataStore, finalConfig); + } else { + ClusterBasedJobCoordinator jc = new ClusterBasedJobCoordinator(metrics, metadataStore, finalConfig); + jc.run(); + } + } + + private static void runStaticResourceJobCoordinator(MetricsRegistryMap metrics, MetadataStore metadataStore, + Config finalConfig) { + StaticResourceJobCoordinator staticResourceJobCoordinator = + StaticResourceJobCoordinator.build(metrics, metadataStore, finalConfig); + addShutdownHook(staticResourceJobCoordinator); + Map<String, MetricsReporter> metricsReporters = + MetricsReporterLoader.getMetricsReporters(new MetricsConfig(finalConfig), JOB_COORDINATOR_CONTAINER_NAME); + metricsReporters.values() + .forEach(metricsReporter -> metricsReporter.register(JOB_COORDINATOR_CONTAINER_NAME, metrics)); + metricsReporters.values().forEach(MetricsReporter::start); + staticResourceJobCoordinator.run(); + metricsReporters.values().forEach(MetricsReporter::stop); + } + Review comment: Ok, I will try fitting it into the `JobCoordinator` interface. The javadocs seem to tie `JobCoordinator` to `StreamProcessor`, and some of the listener methods seemed to be for more dynamic job model handling, but I see where it could potentially be helpful to start merging. I would prefer to not refactor the `ClusterBasedJobCoordinator` flow here in order to fit it into `JobCoordinator` also, so would it be ok to still keep that one as a separate case? ########## File path: samza-core/src/main/java/org/apache/samza/config/JobCoordinatorConfig.java ########## @@ -33,6 +33,8 @@ public final static String DEFAULT_COORDINATOR_FACTORY = ZkJobCoordinatorFactory.class.getName(); private static final String AZURE_COORDINATION_UTILS_FACTORY = "org.apache.samza.coordinator.AzureCoordinationUtilsFactory"; private static final String AZURE_COORDINATOR_FACTORY = "org.apache.samza.coordinator.AzureJobCoordinatorFactory"; + public static final String USE_STATIC_RESOURCE_JOB_COORDINATOR = Review comment: As mentioned in the other comment, I will try to fit `StaticResourceJobCoordinator` into the `JobCoordinator` interface. Then, we could use a factory method. ########## File path: samza-core/src/main/java/org/apache/samza/clustermanager/JobCoordinatorLaunchUtil.java ########## @@ -19,27 +19,36 @@ package org.apache.samza.clustermanager; import java.util.List; +import java.util.Map; +import com.google.common.annotations.VisibleForTesting; import org.apache.samza.SamzaException; import org.apache.samza.application.SamzaApplication; import org.apache.samza.application.descriptors.ApplicationDescriptor; import org.apache.samza.application.descriptors.ApplicationDescriptorImpl; import org.apache.samza.application.descriptors.ApplicationDescriptorUtil; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; +import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.MapConfig; +import org.apache.samza.config.MetricsConfig; +import org.apache.samza.coordinator.StaticResourceJobCoordinator; import org.apache.samza.coordinator.metadatastore.CoordinatorStreamStore; import org.apache.samza.execution.RemoteJobPlanner; import org.apache.samza.metadatastore.MetadataStore; import org.apache.samza.metrics.MetricsRegistryMap; +import org.apache.samza.metrics.MetricsReporter; import org.apache.samza.util.CoordinatorStreamUtil; import org.apache.samza.util.DiagnosticsUtil; +import org.apache.samza.util.MetricsReporterLoader; /** * Util class to launch and run {@link ClusterBasedJobCoordinator}. * This util is being used by both high/low and beam API Samza jobs. */ public class JobCoordinatorLaunchUtil { + private static final String JOB_COORDINATOR_CONTAINER_NAME = "JobCoordinator"; Review comment: Yes, it is still needs `samza.container.name`. Just to be clear, `samza.container.name` is not associated with this variable. ########## File path: samza-core/src/main/java/org/apache/samza/coordinator/communication/HttpCoordinatorToWorkerCommunicationFactory.java ########## @@ -0,0 +1,41 @@ +/* + * 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.samza.coordinator.communication; + +import org.apache.samza.config.ClusterManagerConfig; +import org.apache.samza.coordinator.server.HttpServer; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.ServletHolder; + + +/** + * Implements HTTP-based communication between the coordinator and workers. + */ +public class HttpCoordinatorToWorkerCommunicationFactory implements CoordinatorToWorkerCommunicationFactory { + @Override + public CoordinatorCommunication coordinatorCommunication(CoordinatorCommunicationContext context) { + ClusterManagerConfig clusterManagerConfig = new ClusterManagerConfig(context.getConfig()); + JobModelHttpServlet jobModelHttpServlet = new JobModelHttpServlet(context.getJobModelProvider(), + new JobModelHttpServlet.Metrics(context.getMetricsRegistry())); + HttpServer httpServer = new HttpServer("/", clusterManagerConfig.getCoordinatorUrlPort(), null, + new ServletHolder(DefaultServlet.class)); + httpServer.addServlet("/", jobModelHttpServlet); + return new HttpCoordinatorCommunication(httpServer); Review comment: The communication between JC and workers shouldn't be coupled with a readiness probe. The functionality that each component provides is different and then access pattern to each component is also different. ########## File path: samza-core/src/main/java/org/apache/samza/clustermanager/JobCoordinatorLaunchUtil.java ########## @@ -76,11 +85,37 @@ public static void run(SamzaApplication app, Config config) { CoordinatorStreamUtil.writeConfigToCoordinatorStream(finalConfig, true); DiagnosticsUtil.createDiagnosticsStream(finalConfig); - ClusterBasedJobCoordinator jc = new ClusterBasedJobCoordinator( - metrics, - metadataStore, - finalConfig); - jc.run(); + if (new JobCoordinatorConfig(finalConfig).getUseStaticResourceJobCoordinator()) { + runStaticResourceJobCoordinator(metrics, metadataStore, finalConfig); + } else { + ClusterBasedJobCoordinator jc = new ClusterBasedJobCoordinator(metrics, metadataStore, finalConfig); + jc.run(); + } + } + + private static void runStaticResourceJobCoordinator(MetricsRegistryMap metrics, MetadataStore metadataStore, + Config finalConfig) { + StaticResourceJobCoordinator staticResourceJobCoordinator = + StaticResourceJobCoordinator.build(metrics, metadataStore, finalConfig); + addShutdownHook(staticResourceJobCoordinator); + Map<String, MetricsReporter> metricsReporters = + MetricsReporterLoader.getMetricsReporters(new MetricsConfig(finalConfig), JOB_COORDINATOR_CONTAINER_NAME); + metricsReporters.values() + .forEach(metricsReporter -> metricsReporter.register(JOB_COORDINATOR_CONTAINER_NAME, metrics)); + metricsReporters.values().forEach(MetricsReporter::start); + staticResourceJobCoordinator.run(); + metricsReporters.values().forEach(MetricsReporter::stop); Review comment: `MetricsReporter`s are added inside `ContainerProcessManager` which is inside `ClusterBasedJobCoordinator`. So we don't need to add metrics reporter around `ClusterBasedJobCoordinator` now, although I feel that it would have been more ideal if it was added around it in the first place. I was trying to avoid changing too much of `ClusterBasedJobCoordinator` in this PR because that would make the PR even larger. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
