Skip to content

Commit

Permalink
[hotfix][tests] Extend TestingMiniCluster to support multiple Dispatc…
Browse files Browse the repository at this point in the history
…hers and ResourceManagers

This commit adds funcationality to the TestingMiniCluster to start multiple Dispatchers and
ResourceManagers. This can be used to test HA scenarios.
  • Loading branch information
tillrohrmann committed Feb 5, 2019
1 parent a9bcc8b commit 58d94ae
Show file tree
Hide file tree
Showing 14 changed files with 362 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ public ResourceManager<RegisteredMesosWorkerNode> createResourceManager(

return new MesosResourceManager(
rpcService,
ResourceManager.RESOURCE_MANAGER_NAME,
getEndpointId(),
resourceId,
highAvailabilityServices,
heartbeatServices,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@

import javax.annotation.Nullable;

import java.util.UUID;

/**
* {@link Dispatcher} factory interface.
*/
Expand All @@ -50,4 +52,12 @@ T createDispatcher(
ArchivedExecutionGraphStore archivedExecutionGraphStore,
FatalErrorHandler fatalErrorHandler,
HistoryServerArchivist historyServerArchivist) throws Exception;

default String generateEndpointIdWithUUID() {
return getEndpointId() + UUID.randomUUID();
}

default String getEndpointId() {
return Dispatcher.DISPATCHER_NAME;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ public MiniDispatcher createDispatcher(

return new MiniDispatcher(
rpcService,
Dispatcher.DISPATCHER_NAME,
getEndpointId(),
configuration,
highAvailabilityServices,
resourceManagerGatewayRetriever,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ public Dispatcher createDispatcher(
// create the default dispatcher
return new StandaloneDispatcher(
rpcService,
Dispatcher.DISPATCHER_NAME,
getEndpointId(),
configuration,
highAvailabilityServices,
resourceManagerGatewayRetriever,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@

package org.apache.flink.runtime.entrypoint.component;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.runtime.dispatcher.Dispatcher;
import org.apache.flink.runtime.dispatcher.DispatcherFactory;
import org.apache.flink.runtime.dispatcher.DispatcherGateway;
import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
Expand All @@ -36,7 +38,14 @@
public class SessionDispatcherResourceManagerComponentFactory extends AbstractDispatcherResourceManagerComponentFactory<Dispatcher, DispatcherGateway> {

public SessionDispatcherResourceManagerComponentFactory(@Nonnull ResourceManagerFactory<?> resourceManagerFactory) {
super(SessionDispatcherFactory.INSTANCE, resourceManagerFactory, SessionRestEndpointFactory.INSTANCE);
this(SessionDispatcherFactory.INSTANCE, resourceManagerFactory);
}

@VisibleForTesting
public SessionDispatcherResourceManagerComponentFactory(
@Nonnull DispatcherFactory<Dispatcher> dispatcherFactory,
@Nonnull ResourceManagerFactory<?> resourceManagerFactory) {
super(dispatcherFactory, resourceManagerFactory, SessionRestEndpointFactory.INSTANCE);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.runtime.minicluster;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobSubmissionResult;
Expand Down Expand Up @@ -354,7 +355,8 @@ public void start() throws Exception {
}
}

private Collection<? extends DispatcherResourceManagerComponent<?>> createDispatcherResourceManagerComponents(
@VisibleForTesting
protected Collection<? extends DispatcherResourceManagerComponent<?>> createDispatcherResourceManagerComponents(
Configuration configuration,
RpcServiceFactory rpcServiceFactory,
HighAvailabilityServices haServices,
Expand All @@ -363,7 +365,7 @@ private Collection<? extends DispatcherResourceManagerComponent<?>> createDispat
MetricRegistry metricRegistry,
MetricQueryServiceRetriever metricQueryServiceRetriever,
FatalErrorHandler fatalErrorHandler) throws Exception {
SessionDispatcherResourceManagerComponentFactory dispatcherResourceManagerComponentFactory = new SessionDispatcherResourceManagerComponentFactory(StandaloneResourceManagerFactory.INSTANCE);
SessionDispatcherResourceManagerComponentFactory dispatcherResourceManagerComponentFactory = createDispatcherResourceManagerComponentFactory();
return Collections.singleton(
dispatcherResourceManagerComponentFactory.create(
configuration,
Expand All @@ -377,6 +379,12 @@ private Collection<? extends DispatcherResourceManagerComponent<?>> createDispat
fatalErrorHandler));
}

@Nonnull
private SessionDispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory() {
return new SessionDispatcherResourceManagerComponentFactory(StandaloneResourceManagerFactory.INSTANCE);
}

@VisibleForTesting
protected HighAvailabilityServices createHighAvailabilityServices(Configuration configuration, Executor executor) throws Exception {
LOG.info("Starting high-availability services");
return HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@

import javax.annotation.Nullable;

import java.util.UUID;

/**
* {@link ResourceManager} factory.
*
Expand All @@ -49,4 +51,12 @@ ResourceManager<T> createResourceManager(
ClusterInformation clusterInformation,
@Nullable String webInterfaceUrl,
JobManagerMetricGroup jobManagerMetricGroup) throws Exception;

default String generateEndpointIdWithUUID() {
return getEndpointId() + UUID.randomUUID();
}

default String getEndpointId() {
return ResourceManager.RESOURCE_MANAGER_NAME;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
package org.apache.flink.runtime.resourcemanager;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.clusterframework.FlinkResourceManager;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.entrypoint.ClusterInformation;
import org.apache.flink.runtime.heartbeat.HeartbeatServices;
Expand Down Expand Up @@ -57,7 +56,7 @@ public ResourceManager<ResourceID> createResourceManager(

return new StandaloneResourceManager(
rpcService,
FlinkResourceManager.RESOURCE_MANAGER_NAME,
getEndpointId(),
resourceId,
highAvailabilityServices,
heartbeatServices,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,8 @@
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.runtime.jobmaster.JobNotFinishedException;
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
import org.apache.flink.runtime.minicluster.TestingMiniCluster;
import org.apache.flink.runtime.minicluster.TestingMiniClusterConfiguration;
import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.runtime.testutils.CommonTestUtils;
import org.apache.flink.runtime.util.LeaderRetrievalUtils;
Expand Down Expand Up @@ -75,7 +75,7 @@ public static void setupClass() throws Exception {
highAvailabilityServices = new TestingEmbeddedHaServices(TestingUtils.defaultExecutor());

miniCluster = new TestingMiniCluster(
new MiniClusterConfiguration.Builder()
new TestingMiniClusterConfiguration.Builder()
.setNumTaskManagers(NUM_TMS)
.setNumSlotsPerTaskManager(SLOTS_PER_TM)
.build(),
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* 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:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.runtime.minicluster;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.blob.BlobServer;
import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore;
import org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory;
import org.apache.flink.runtime.dispatcher.Dispatcher;
import org.apache.flink.runtime.dispatcher.DispatcherFactory;
import org.apache.flink.runtime.dispatcher.HistoryServerArchivist;
import org.apache.flink.runtime.dispatcher.StandaloneDispatcher;
import org.apache.flink.runtime.heartbeat.HeartbeatServices;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
import org.apache.flink.runtime.rpc.FatalErrorHandler;
import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;

import javax.annotation.Nullable;

/**
* {@link DispatcherFactory} which creates a {@link StandaloneDispatcher} which has an
* endpoint id with a random UUID suffix.
*/
public enum SessionDispatcherWithUUIDFactory implements DispatcherFactory<Dispatcher> {
INSTANCE;

@Override
public Dispatcher createDispatcher(
Configuration configuration,
RpcService rpcService,
HighAvailabilityServices highAvailabilityServices,
GatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever,
BlobServer blobServer,
HeartbeatServices heartbeatServices,
JobManagerMetricGroup jobManagerMetricGroup,
@Nullable String metricQueryServicePath,
ArchivedExecutionGraphStore archivedExecutionGraphStore,
FatalErrorHandler fatalErrorHandler,
HistoryServerArchivist historyServerArchivist) throws Exception {
// create the default dispatcher
return new StandaloneDispatcher(
rpcService,
generateEndpointIdWithUUID(),
configuration,
highAvailabilityServices,
resourceManagerGatewayRetriever,
blobServer,
heartbeatServices,
jobManagerMetricGroup,
metricQueryServicePath,
archivedExecutionGraphStore,
DefaultJobManagerRunnerFactory.INSTANCE,
fatalErrorHandler,
historyServerArchivist);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
/*
* 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:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.runtime.minicluster;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.entrypoint.ClusterInformation;
import org.apache.flink.runtime.heartbeat.HeartbeatServices;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
import org.apache.flink.runtime.metrics.MetricRegistry;
import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
import org.apache.flink.runtime.resourcemanager.ResourceManager;
import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory;
import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices;
import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration;
import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
import org.apache.flink.runtime.rpc.FatalErrorHandler;
import org.apache.flink.runtime.rpc.RpcService;

import javax.annotation.Nullable;

/**
* {@link ResourceManagerFactory} which starts the {@link ResourceManager} with an endpoint id which
* has a random UUID suffix.
*/
public enum StandaloneResourceManagerWithUUIDFactory implements ResourceManagerFactory<ResourceID> {
INSTANCE;

@Override
public ResourceManager<ResourceID> createResourceManager(
Configuration configuration,
ResourceID resourceId,
RpcService rpcService,
HighAvailabilityServices highAvailabilityServices,
HeartbeatServices heartbeatServices,
MetricRegistry metricRegistry,
FatalErrorHandler fatalErrorHandler,
ClusterInformation clusterInformation,
@Nullable String webInterfaceUrl,
JobManagerMetricGroup jobManagerMetricGroup) throws Exception {
final ResourceManagerRuntimeServicesConfiguration resourceManagerRuntimeServicesConfiguration = ResourceManagerRuntimeServicesConfiguration.fromConfiguration(configuration);
final ResourceManagerRuntimeServices resourceManagerRuntimeServices = ResourceManagerRuntimeServices.fromConfiguration(
resourceManagerRuntimeServicesConfiguration,
highAvailabilityServices,
rpcService.getScheduledExecutor());

return new StandaloneResourceManager(
rpcService,
generateEndpointIdWithUUID(),
resourceId,
highAvailabilityServices,
heartbeatServices,
resourceManagerRuntimeServices.getSlotManager(),
metricRegistry,
resourceManagerRuntimeServices.getJobLeaderIdService(),
clusterInformation,
fatalErrorHandler,
jobManagerMetricGroup);
}
}
Loading

0 comments on commit 58d94ae

Please sign in to comment.