[FLINK-6379] Remove internal methods from MesosResourceManagerGateway

Some internal methods which are required for the interplay between the TaskMonitor,
LaunchCoordinator and the MesosResourceManager were exposed as RPC methods. In order
to keep the RPC interface as lean as possible, these methods have been removed.

Fix checkstyle violations
上级 4bb488c0
......@@ -20,20 +20,21 @@ package org.apache.flink.mesos.runtime.clusterframework;
import org.apache.flink.mesos.scheduler.LaunchCoordinator;
import org.apache.flink.mesos.scheduler.ReconciliationCoordinator;
import org.apache.flink.mesos.scheduler.SchedulerGateway;
import org.apache.flink.mesos.scheduler.TaskMonitor;
import org.apache.flink.mesos.scheduler.messages.AcceptOffers;
import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
/**
* The {@link MesosResourceManager}'s RPC gateway interface.
* Actions defined by the MesosResourceManager.
*
* <p>These are called by the MesosResourceManager components such
* as {@link LaunchCoordinator}, and {@link TaskMonitor}.
*/
public interface MesosResourceManagerGateway extends ResourceManagerGateway, SchedulerGateway {
public interface MesosResourceManagerActions {
/**
* Accept the given offers as advised by the launch coordinator.
*
* Note: This method is a callback for the {@link LaunchCoordinator}.
* <p>Note: This method is a callback for the {@link LaunchCoordinator}.
*
* @param offersToAccept Offers to accept from Mesos
*/
......@@ -42,7 +43,7 @@ public interface MesosResourceManagerGateway extends ResourceManagerGateway, Sch
/**
* Trigger reconciliation with the Mesos master.
*
* Note: This method is a callback for the {@link TaskMonitor}.
* <p>Note: This method is a callback for the {@link TaskMonitor}.
*
* @param reconciliationRequest Message containing the tasks which shall be reconciled
*/
......@@ -51,7 +52,7 @@ public interface MesosResourceManagerGateway extends ResourceManagerGateway, Sch
/**
* Notify that the given Mesos task has been terminated.
*
* Note: This method is a callback for the {@link TaskMonitor}.
* <p>Note: This method is a callback for the {@link TaskMonitor}.
*
* @param terminatedTask Message containing the terminated task
*/
......
......@@ -19,6 +19,7 @@
package org.apache.flink.mesos.runtime.clusterframework.store;
import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
import org.apache.mesos.Protos;
import java.io.Serializable;
......
/*
* 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.flink.mesos.scheduler;
import org.apache.flink.mesos.scheduler.messages.Disconnected;
import org.apache.flink.mesos.scheduler.messages.Error;
import org.apache.flink.mesos.scheduler.messages.ExecutorLost;
import org.apache.flink.mesos.scheduler.messages.FrameworkMessage;
import org.apache.flink.mesos.scheduler.messages.OfferRescinded;
import org.apache.flink.mesos.scheduler.messages.ReRegistered;
import org.apache.flink.mesos.scheduler.messages.Registered;
import org.apache.flink.mesos.scheduler.messages.ResourceOffers;
import org.apache.flink.mesos.scheduler.messages.SlaveLost;
import org.apache.flink.mesos.scheduler.messages.StatusUpdate;
import org.apache.flink.runtime.rpc.RpcGateway;
/**
* A scheduler's RPC gateway interface.
*
* Implemented by RPC endpoints that accept Mesos scheduler messages.
*/
public interface SchedulerGateway extends RpcGateway {
/**
* Called when connected to Mesos as a new framework.
*/
void registered(Registered message);
/**
* Called when reconnected to Mesos following a failover event.
*/
void reregistered(ReRegistered message);
/**
* Called when disconnected from Mesos.
*/
void disconnected(Disconnected message);
/**
* Called when resource offers are made to the framework.
*/
void resourceOffers(ResourceOffers message);
/**
* Called when resource offers are rescinded.
*/
void offerRescinded(OfferRescinded message);
/**
* Called when a status update arrives from the Mesos master.
*/
void statusUpdate(StatusUpdate message);
/**
* Called when a framework message arrives from a custom Mesos task executor.
*/
void frameworkMessage(FrameworkMessage message);
/**
* Called when a Mesos slave is lost.
*/
void slaveLost(SlaveLost message);
/**
* Called when a custom Mesos task executor is lost.
*/
void executorLost(ExecutorLost message);
/**
* Called when an error is reported by the scheduler callback.
*/
void error(Error message);
}
/*
* 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.flink.mesos.scheduler;
import org.apache.flink.mesos.scheduler.messages.Disconnected;
import org.apache.flink.mesos.scheduler.messages.Error;
import org.apache.flink.mesos.scheduler.messages.ExecutorLost;
import org.apache.flink.mesos.scheduler.messages.FrameworkMessage;
import org.apache.flink.mesos.scheduler.messages.OfferRescinded;
import org.apache.flink.mesos.scheduler.messages.ReRegistered;
import org.apache.flink.mesos.scheduler.messages.Registered;
import org.apache.flink.mesos.scheduler.messages.ResourceOffers;
import org.apache.flink.mesos.scheduler.messages.SlaveLost;
import org.apache.flink.mesos.scheduler.messages.StatusUpdate;
import org.apache.mesos.Protos;
import org.apache.mesos.Scheduler;
import org.apache.flink.mesos.runtime.clusterframework.MesosResourceManager;
import org.apache.mesos.SchedulerDriver;
import java.util.List;
/**
* This class reacts to callbacks from the Mesos scheduler driver.
*
* Forwards incoming messages to the {@link MesosResourceManager} RPC gateway.
*
* See https://mesos.apache.org/api/latest/java/org/apache/mesos/Scheduler.html
*/
public class SchedulerProxyV2 implements Scheduler {
/** The actor to which we report the callbacks */
private final SchedulerGateway gateway;
public SchedulerProxyV2(SchedulerGateway gateway) {
this.gateway = gateway;
}
@Override
public void registered(SchedulerDriver driver, Protos.FrameworkID frameworkId, Protos.MasterInfo masterInfo) {
gateway.registered(new Registered(frameworkId, masterInfo));
}
@Override
public void reregistered(SchedulerDriver driver, Protos.MasterInfo masterInfo) {
gateway.reregistered(new ReRegistered(masterInfo));
}
@Override
public void disconnected(SchedulerDriver driver) {
gateway.disconnected(new Disconnected());
}
@Override
public void resourceOffers(SchedulerDriver driver, List<Protos.Offer> offers) {
gateway.resourceOffers(new ResourceOffers(offers));
}
@Override
public void offerRescinded(SchedulerDriver driver, Protos.OfferID offerId) {
gateway.offerRescinded(new OfferRescinded(offerId));
}
@Override
public void statusUpdate(SchedulerDriver driver, Protos.TaskStatus status) {
gateway.statusUpdate(new StatusUpdate(status));
}
@Override
public void frameworkMessage(SchedulerDriver driver, Protos.ExecutorID executorId, Protos.SlaveID slaveId, byte[] data) {
gateway.frameworkMessage(new FrameworkMessage(executorId, slaveId, data));
}
@Override
public void slaveLost(SchedulerDriver driver, Protos.SlaveID slaveId) {
gateway.slaveLost(new SlaveLost(slaveId));
}
@Override
public void executorLost(SchedulerDriver driver, Protos.ExecutorID executorId, Protos.SlaveID slaveId, int status) {
gateway.executorLost(new ExecutorLost(executorId, slaveId, status));
}
@Override
public void error(SchedulerDriver driver, String message) {
gateway.error(new Error(message));
}
}
......@@ -18,23 +18,20 @@
package org.apache.flink.mesos.runtime.clusterframework;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.testkit.JavaTestKit;
import akka.testkit.TestProbe;
import com.netflix.fenzo.ConstraintEvaluator;
import junit.framework.AssertionFailedError;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore;
import org.apache.flink.mesos.scheduler.ConnectionMonitor;
import org.apache.flink.mesos.scheduler.LaunchCoordinator;
import org.apache.flink.mesos.scheduler.ReconciliationCoordinator;
import org.apache.flink.mesos.scheduler.TaskMonitor;
import org.apache.flink.mesos.scheduler.messages.*;
import org.apache.flink.mesos.scheduler.messages.Error;
import org.apache.flink.mesos.scheduler.messages.AcceptOffers;
import org.apache.flink.mesos.scheduler.messages.Disconnected;
import org.apache.flink.mesos.scheduler.messages.OfferRescinded;
import org.apache.flink.mesos.scheduler.messages.ReRegistered;
import org.apache.flink.mesos.scheduler.messages.Registered;
import org.apache.flink.mesos.scheduler.messages.ResourceOffers;
import org.apache.flink.mesos.scheduler.messages.StatusUpdate;
import org.apache.flink.mesos.util.MesosArtifactResolver;
import org.apache.flink.mesos.util.MesosConfiguration;
import org.apache.flink.runtime.akka.AkkaUtils;
......@@ -55,7 +52,9 @@ import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
import org.apache.flink.runtime.metrics.MetricRegistry;
import org.apache.flink.runtime.registration.RegistrationResponse;
import org.apache.flink.runtime.resourcemanager.*;
import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
import org.apache.flink.runtime.resourcemanager.SlotRequest;
import org.apache.flink.runtime.resourcemanager.slotmanager.ResourceManagerActions;
import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
import org.apache.flink.runtime.rpc.FatalErrorHandler;
......@@ -66,30 +65,51 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
import org.apache.flink.runtime.util.TestingFatalErrorHandler;
import org.apache.flink.util.TestLogger;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.testkit.JavaTestKit;
import akka.testkit.TestProbe;
import com.netflix.fenzo.ConstraintEvaluator;
import junit.framework.AssertionFailedError;
import org.apache.mesos.Protos;
import org.apache.mesos.Scheduler;
import org.apache.mesos.SchedulerDriver;
import org.junit.*;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Option;
import scala.collection.JavaConverters;
import java.util.*;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import scala.Option;
import static java.util.Collections.singletonList;
import static org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager.extractGoalState;
import static org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager.extractResourceID;
import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasEntry;
import static org.hamcrest.Matchers.hasItem;
import static org.hamcrest.Matchers.hasSize;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Mockito.*;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
/**
* General tests for the Mesos resource manager component (v2).
......@@ -151,13 +171,24 @@ public class MesosResourceManagerTest extends TestLogger {
}
@Override
protected ActorRef createConnectionMonitor() { return connectionMonitor.ref(); }
protected ActorRef createConnectionMonitor() {
return connectionMonitor.ref();
}
@Override
protected ActorRef createTaskRouter() { return taskRouter.ref(); }
protected ActorRef createTaskMonitor(SchedulerDriver schedulerDriver) {
return taskRouter.ref();
}
@Override
protected ActorRef createLaunchCoordinator() { return launchCoordinator.ref(); }
protected ActorRef createLaunchCoordinator(SchedulerDriver schedulerDriver, ActorRef selfActorRef) {
return launchCoordinator.ref();
}
@Override
protected ActorRef createReconciliationCoordinator() { return reconciliationCoordinator.ref(); }
protected ActorRef createReconciliationCoordinator(SchedulerDriver schedulerDriver) {
return reconciliationCoordinator.ref();
}
@Override
protected void closeTaskManagerConnection(ResourceID resourceID, Exception cause) {
......@@ -179,7 +210,7 @@ public class MesosResourceManagerTest extends TestLogger {
// RM
ResourceManagerConfiguration rmConfiguration;
ResourceID rmResourceID;
static final String rmAddress = "/resourceManager";
static final String RM_ADDRESS = "/resourceManager";
TestingMesosResourceManager resourceManager;
// domain objects for test purposes
......@@ -227,7 +258,7 @@ public class MesosResourceManagerTest extends TestLogger {
resourceManager =
new TestingMesosResourceManager(
rpcService,
rmAddress,
RM_ADDRESS,
rmResourceID,
rmConfiguration,
rmServices.highAvailabilityServices,
......@@ -252,7 +283,7 @@ public class MesosResourceManagerTest extends TestLogger {
task3Executor = mockTaskExecutor(task3);
// JobMaster
jobMaster1 = mockJobMaster(rmServices, new JobID(1,0));
jobMaster1 = mockJobMaster(rmServices, new JobID(1, 0));
}
/**
......@@ -290,7 +321,7 @@ public class MesosResourceManagerTest extends TestLogger {
rmActions = invocation.getArgumentAt(2, ResourceManagerActions.class);
return null;
}
}).when(slotManager).start(any(UUID.class),any(Executor.class),any(ResourceManagerActions.class));
}).when(slotManager).start(any(UUID.class), any(Executor.class), any(ResourceManagerActions.class));
when(slotManager.registerSlotRequest(any(SlotRequest.class))).thenReturn(true);
}
......@@ -669,41 +700,4 @@ public class MesosResourceManagerTest extends TestLogger {
resourceManager.taskRouter.expectMsgClass(Disconnected.class);
}};
}
/**
* Test Mesos scheduler error.
*/
@Test
public void testError() throws Exception {
new Context() {{
when(rmServices.workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
startResourceManager();
resourceManager.error(new Error("test"));
assertTrue(fatalErrorHandler.hasExceptionOccurred());
}};
}
@Test
public void testAdapter() throws Exception {
Protos.TaskID task1 = Protos.TaskID.newBuilder().setValue("taskmanager-00001").build();
Protos.TaskStatus status1 = Protos.TaskStatus.newBuilder().setTaskId(task1).setState(Protos.TaskState.TASK_KILLED).build();
String host1 = "host1";
MesosResourceManagerGateway gateway = mock(MesosResourceManagerGateway.class);
ActorRef adapter = system.actorOf(MesosResourceManager.AkkaAdapter.createActorProps(gateway));
List<Protos.TaskStatus> tasks = Collections.singletonList(status1);
ReconciliationCoordinator.Reconcile msg1 = new ReconciliationCoordinator.Reconcile(
JavaConverters.asScalaBufferConverter(tasks).asScala(), false);
adapter.tell(msg1, ActorRef.noSender());
verify(gateway, timeout(1000L)).reconcile(eq(msg1));
TaskMonitor.TaskTerminated msg2 = new TaskMonitor.TaskTerminated(task1, status1);
adapter.tell(msg2, ActorRef.noSender());
verify(gateway, timeout(1000L)).taskTerminated(eq(msg2));
AcceptOffers msg3 = new AcceptOffers(host1, Collections.<Protos.OfferID>emptyList(), Collections.<Protos.Offer.Operation>emptyList());
adapter.tell(msg3, ActorRef.noSender());
verify(gateway, timeout(1000L)).acceptOffers(eq(msg3));
}
}
......@@ -83,8 +83,8 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
* <li>{@link #requestSlot(UUID, UUID, SlotRequest)} requests a slot from the resource manager</li>
* </ul>
*/
public abstract class ResourceManager<C extends ResourceManagerGateway, WorkerType extends Serializable>
extends RpcEndpoint<C>
public abstract class ResourceManager<WorkerType extends Serializable>
extends RpcEndpoint<ResourceManagerGateway>
implements LeaderContender {
public static final String RESOURCE_MANAGER_NAME = "resourcemanager";
......@@ -609,8 +609,13 @@ public abstract class ResourceManager<C extends ResourceManagerGateway, WorkerTy
*/
@RpcMethod
public void shutDownCluster(final ApplicationStatus finalStatus, final String optionalDiagnostics) {
log.info("shut down cluster because application is in {}, diagnostics {}", finalStatus, optionalDiagnostics);
shutDownApplication(finalStatus, optionalDiagnostics);
log.info("Shut down cluster because application is in {}, diagnostics {}.", finalStatus, optionalDiagnostics);
try {
shutDownApplication(finalStatus, optionalDiagnostics);
} catch (ResourceManagerException e) {
log.warn("Could not properly shutdown the application.", e);
}
}
@RpcMethod
......@@ -880,8 +885,9 @@ public abstract class ResourceManager<C extends ResourceManagerGateway, WorkerTy
*
* @param finalStatus The application status to report.
* @param optionalDiagnostics An optional diagnostics message.
* @throws ResourceManagerException if the application could not be shut down.
*/
protected abstract void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics);
protected abstract void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) throws ResourceManagerException;
/**
* Allocates a resource using the resource profile.
......
......@@ -42,7 +42,7 @@ public class ResourceManagerRunner implements FatalErrorHandler {
private final ResourceManagerRuntimeServices resourceManagerRuntimeServices;
private final ResourceManager<? extends ResourceManagerGateway, ?> resourceManager;
private final ResourceManager<?> resourceManager;
public ResourceManagerRunner(
final ResourceID resourceId,
......
......@@ -36,7 +36,7 @@ import org.apache.flink.runtime.rpc.RpcService;
*
* This ResourceManager doesn't acquire new resources.
*/
public class StandaloneResourceManager extends ResourceManager<StandaloneResourceManagerGateway, ResourceID> {
public class StandaloneResourceManager extends ResourceManager<ResourceID> {
public StandaloneResourceManager(
RpcService rpcService,
......
/*
* 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.flink.runtime.resourcemanager;
/**
* The {@link StandaloneResourceManager}'s RPC gateway interface.
*/
public interface StandaloneResourceManagerGateway extends ResourceManagerGateway {
}
......@@ -47,7 +47,6 @@ import org.apache.flink.runtime.resourcemanager.ResourceManager;
import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
import org.apache.flink.runtime.resourcemanager.SlotRequest;
import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerGateway;
import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
import org.apache.flink.runtime.rpc.TestingSerialRpcService;
import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
......@@ -127,7 +126,7 @@ public class TaskExecutorITCase extends TestLogger {
TestingUtils.infiniteTime(),
TestingUtils.infiniteTime());
ResourceManager<StandaloneResourceManagerGateway,ResourceID> resourceManager = new StandaloneResourceManager(
ResourceManager<ResourceID> resourceManager = new StandaloneResourceManager(
rpcService,
FlinkResourceManager.RESOURCE_MANAGER_NAME,
rmResourceId,
......
......@@ -193,7 +193,7 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
return new AkkaRpcService(actorSystem, Time.of(duration.length(), duration.unit()));
}
private ResourceManager<?,?> createResourceManager(Configuration config) throws Exception {
private ResourceManager<?> createResourceManager(Configuration config) throws Exception {
final ResourceManagerConfiguration resourceManagerConfiguration = ResourceManagerConfiguration.fromConfiguration(config);
final ResourceManagerRuntimeServicesConfiguration resourceManagerRuntimeServicesConfiguration = ResourceManagerRuntimeServicesConfiguration.fromConfiguration(config);
final ResourceManagerRuntimeServices resourceManagerRuntimeServices = ResourceManagerRuntimeServices.fromConfiguration(
......
......@@ -63,7 +63,7 @@ import scala.concurrent.duration.FiniteDuration;
* The yarn implementation of the resource manager. Used when the system is started
* via the resource framework YARN.
*/
public class YarnResourceManager extends ResourceManager<YarnResourceManagerGateway, ResourceID> implements AMRMClientAsync.CallbackHandler {
public class YarnResourceManager extends ResourceManager<ResourceID> implements AMRMClientAsync.CallbackHandler {
/** The process environment variables. */
private final Map<String, String> env;
......
/*
* 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.flink.yarn;
import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
/**
* The {@link YarnResourceManager}'s RPC gateway interface.
*/
public interface YarnResourceManagerGateway extends ResourceManagerGateway {
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册