From 21f1e17195dd303fb2a9c642f5a27847fdd8967c Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Thu, 2 Feb 2023 18:06:17 +0800 Subject: [PATCH 01/21] Temporary submission --- .../apache/uniffle/common/rpc/GrpcServer.java | 74 ++++++++++++++----- .../coordinator/CoordinatorFactory.java | 6 +- .../test/CoordinatorGrpcServerTest.java | 6 +- proto/src/main/proto/Rss.proto | 21 ++++++ .../apache/uniffle/server/ShuffleServer.java | 39 ++++++++++ .../uniffle/server/ShuffleServerFactory.java | 9 ++- .../ShuffleServerInternalGrpcService.java | 41 ++++++++++ .../uniffle/server/MockedGrpcServer.java | 4 +- 8 files changed, 177 insertions(+), 23 deletions(-) create mode 100644 server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java diff --git a/common/src/main/java/org/apache/uniffle/common/rpc/GrpcServer.java b/common/src/main/java/org/apache/uniffle/common/rpc/GrpcServer.java index be4e916f09..a993541287 100644 --- a/common/src/main/java/org/apache/uniffle/common/rpc/GrpcServer.java +++ b/common/src/main/java/org/apache/uniffle/common/rpc/GrpcServer.java @@ -18,6 +18,8 @@ package org.apache.uniffle.common.rpc; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadFactory; @@ -25,11 +27,14 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import com.google.common.collect.Lists; import com.google.common.collect.Queues; import io.grpc.BindableService; import io.grpc.Server; import io.grpc.ServerBuilder; +import io.grpc.ServerInterceptor; import io.grpc.ServerInterceptors; +import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,7 +51,10 @@ public class GrpcServer implements ServerInterface { private final int port; private final ExecutorService pool; - public GrpcServer(RssBaseConf conf, BindableService service, GRPCMetrics grpcMetrics) { + protected GrpcServer( + RssBaseConf conf, + List>> servicesWithInterceptors, + GRPCMetrics grpcMetrics) { this.port = conf.getInteger(RssBaseConf.RPC_SERVER_PORT); long maxInboundMessageSize = conf.getLong(RssBaseConf.RPC_MESSAGE_MAX_SIZE); int rpcExecutorSize = conf.getInteger(RssBaseConf.RPC_EXECUTOR_SIZE); @@ -61,23 +69,55 @@ public GrpcServer(RssBaseConf conf, BindableService service, GRPCMetrics grpcMet ); boolean isMetricsEnabled = conf.getBoolean(RssBaseConf.RPC_METRICS_ENABLED); + ServerBuilder builder = ServerBuilder + .forPort(port) + .executor(pool) + .maxInboundMessageSize((int)maxInboundMessageSize); if (isMetricsEnabled) { - MonitoringServerInterceptor monitoringInterceptor = - new MonitoringServerInterceptor(grpcMetrics); - this.server = ServerBuilder - .forPort(port) - .addService(ServerInterceptors.intercept(service, monitoringInterceptor)) - .executor(pool) - .addTransportFilter(new MonitoringServerTransportFilter(grpcMetrics)) - .maxInboundMessageSize((int)maxInboundMessageSize) - .build(); - } else { - this.server = ServerBuilder - .forPort(port) - .addService(service) - .executor(pool) - .maxInboundMessageSize((int)maxInboundMessageSize) - .build(); + builder.addTransportFilter(new MonitoringServerTransportFilter(grpcMetrics)); + } + servicesWithInterceptors.forEach((serviceWithInterceptors) -> { + List interceptors = serviceWithInterceptors.getRight(); + if (isMetricsEnabled) { + MonitoringServerInterceptor monitoringInterceptor = + new MonitoringServerInterceptor(grpcMetrics); + List newInterceptors = Lists.newArrayList(interceptors); + newInterceptors.add(monitoringInterceptor); + interceptors = newInterceptors; + } + builder.addService(ServerInterceptors.intercept(serviceWithInterceptors.getLeft(), interceptors)); + }); + this.server = builder.build(); + } + + public static class Builder { + + private RssBaseConf rssBaseConf; + private GRPCMetrics grpcMetrics; + + private List>> servicesWithInterceptors = new ArrayList<>(); + + public static Builder newBuilder() { + return new Builder(); + } + + public Builder conf(RssBaseConf rssBaseConf) { + this.rssBaseConf = rssBaseConf; + return this; + } + + public Builder addService(BindableService bindableService, ServerInterceptor... interceptors) { + this.servicesWithInterceptors.add(Pair.of(bindableService, Lists.newArrayList(interceptors))); + return this; + } + + public Builder grpcMetrics(GRPCMetrics metrics) { + this.grpcMetrics = metrics; + return this; + } + + public GrpcServer build() { + return new GrpcServer(rssBaseConf, servicesWithInterceptors, grpcMetrics); } } diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorFactory.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorFactory.java index b8f566ee0f..1a952d272b 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorFactory.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorFactory.java @@ -33,8 +33,10 @@ public CoordinatorFactory(CoordinatorServer coordinatorServer) { public ServerInterface getServer() { String type = conf.getString(CoordinatorConf.RPC_SERVER_TYPE); if (type.equals(ServerType.GRPC.name())) { - return new GrpcServer(conf, new CoordinatorGrpcService(coordinatorServer), - coordinatorServer.getGrpcMetrics()); + return GrpcServer.Builder.newBuilder() + .conf(conf) + .grpcMetrics(coordinatorServer.getGrpcMetrics()) + .addService(new CoordinatorGrpcService(coordinatorServer)).build(); } else { throw new UnsupportedOperationException("Unsupported server type " + type); } diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/CoordinatorGrpcServerTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/CoordinatorGrpcServerTest.java index 99abcee406..f5ca9b51dc 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/CoordinatorGrpcServerTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/CoordinatorGrpcServerTest.java @@ -61,7 +61,11 @@ public void testGrpcConnectionSize() throws Exception { GRPCMetrics grpcMetrics = new CoordinatorGrpcMetrics(); grpcMetrics.register(new CollectorRegistry(true)); - GrpcServer grpcServer = new GrpcServer(baseConf, new MockedCoordinatorGrpcService(), grpcMetrics); + GrpcServer grpcServer = GrpcServer.Builder.newBuilder() + .conf(baseConf) + .grpcMetrics(grpcMetrics) + .addService(new MockedCoordinatorGrpcService()) + .build(); grpcServer.start(); // case1: test the single one connection metric diff --git a/proto/src/main/proto/Rss.proto b/proto/src/main/proto/Rss.proto index 0bff956e1f..bb397cd9f5 100644 --- a/proto/src/main/proto/Rss.proto +++ b/proto/src/main/proto/Rss.proto @@ -242,7 +242,14 @@ message ShuffleServerHeartBeatRequest { int32 eventNumInFlush = 5; repeated string tags = 6; google.protobuf.BoolValue isHealthy = 7; + ServerStatus status = 8; map storageInfo = 21; // mount point to storage info mapping. + + enum ServerStatus { + NORMAL = 0; + DECOMMISSIONING = 1; + // todo: more status, such as UPGRADING + } } message ShuffleServerHeartBeatResponse { @@ -458,3 +465,17 @@ message FetchRemoteStorageResponse { StatusCode status = 1; RemoteStorage remoteStorage = 2; } + +service ShuffleServerInternal { + rpc decommission(DecommissionRequest) returns (DecommissionResponse); +} + +message DecommissionRequest { + google.protobuf.BoolValue on = 1; +} + +message DecommissionResponse { + StatusCode status = 1; + google.protobuf.BoolValue on = 2; + string retMsg = 3; +} \ No newline at end of file diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java index 796ba3058f..5af5cb9dfa 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java @@ -78,6 +78,8 @@ public class ShuffleServer { private AtomicBoolean isHealthy = new AtomicBoolean(true); private GRPCMetrics grpcMetrics; private MetricReporter metricReporter; + private boolean decommissioned; + private Thread decommissionedThread; public ShuffleServer(ShuffleServerConf shuffleServerConf) throws Exception { this.shuffleServerConf = shuffleServerConf; @@ -332,4 +334,41 @@ public boolean isHealthy() { public GRPCMetrics getGrpcMetrics() { return grpcMetrics; } + + public boolean isDecommissioned() { + return decommissioned; + } + + public synchronized void setDecommissioned(boolean decommissioned) { + if (this.decommissioned == decommissioned) { + return; + } + this.decommissioned = decommissioned; + LOG.info("set decommissioned state to " + decommissioned); + if (this.decommissioned) { + decommissionedThread = new Thread(() -> { + while (decommissioned) { + int remainApplicationNum = shuffleTaskManager.getAppIds().size(); + if (shuffleTaskManager.getAppIds().isEmpty()) { + LOG.info("all applications finished, exit now"); + System.exit(0); + } + LOG.info("shuffle server is in decommissioned state. remain {} applications not finished.", remainApplicationNum); + try { + Thread.sleep(60000); + } catch (InterruptedException e) { + LOG.warn("Ignore the InterruptedException which should be caused by internal killed"); + } + } + }); + decommissionedThread.setName("decommission"); + decommissionedThread.start(); + } else { + if (decommissionedThread != null) { + decommissionedThread.interrupt(); + decommissionedThread = null; + } + } + } + } diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServerFactory.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServerFactory.java index 24e70e39f9..8ba4adca57 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServerFactory.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerFactory.java @@ -33,8 +33,13 @@ public ShuffleServerFactory(ShuffleServer shuffleServer) { public ServerInterface getServer() { String type = conf.getString(ShuffleServerConf.RPC_SERVER_TYPE); if (type.equals(ServerType.GRPC.name())) { - return new GrpcServer(conf, new ShuffleServerGrpcService(shuffleServer), - shuffleServer.getGrpcMetrics()); + return GrpcServer.Builder.newBuilder() + .conf(conf) + .grpcMetrics(shuffleServer.getGrpcMetrics()) + .addService(new ShuffleServerGrpcService(shuffleServer)) + // todo: Add ServerInterceptor for authentication + .addService(new ShuffleServerInternalGrpcService(shuffleServer)) + .build(); } else { throw new UnsupportedOperationException("Unsupported server type " + type); } diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java new file mode 100644 index 0000000000..fdfe662d33 --- /dev/null +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.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.uniffle.server; + +import com.google.protobuf.BoolValue; +import io.grpc.stub.StreamObserver; +import org.apache.uniffle.proto.RssProtos; +import org.apache.uniffle.proto.ShuffleServerInternalGrpc.ShuffleServerInternalImplBase; + +public class ShuffleServerInternalGrpcService extends ShuffleServerInternalImplBase { + private final ShuffleServer shuffleServer; + + public ShuffleServerInternalGrpcService(ShuffleServer shuffleServer) { + this.shuffleServer = shuffleServer; + } + @Override + public void decommission(RssProtos.DecommissionRequest request, StreamObserver responseObserver) { + shuffleServer.setDecommissioned(request.getOn().getValue()); + RssProtos.DecommissionResponse response = RssProtos.DecommissionResponse + .newBuilder() + .setStatus(RssProtos.StatusCode.forNumber(StatusCode.SUCCESS.statusCode())) + .setOn(BoolValue.newBuilder().setValue(shuffleServer.isDecommissioned()).build()) + .build(); + responseObserver.onNext(response); + responseObserver.onCompleted(); + } +} diff --git a/server/src/test/java/org/apache/uniffle/server/MockedGrpcServer.java b/server/src/test/java/org/apache/uniffle/server/MockedGrpcServer.java index fa45034535..0d10935baa 100644 --- a/server/src/test/java/org/apache/uniffle/server/MockedGrpcServer.java +++ b/server/src/test/java/org/apache/uniffle/server/MockedGrpcServer.java @@ -17,6 +17,8 @@ package org.apache.uniffle.server; +import com.google.common.collect.Lists; +import org.apache.commons.lang3.tuple.Pair; import org.apache.uniffle.common.config.RssBaseConf; import org.apache.uniffle.common.metrics.GRPCMetrics; import org.apache.uniffle.common.rpc.GrpcServer; @@ -26,7 +28,7 @@ public class MockedGrpcServer extends GrpcServer { public MockedGrpcServer(RssBaseConf conf, MockedShuffleServerGrpcService service, GRPCMetrics grpcMetrics) { - super(conf, service, grpcMetrics); + super(conf, Lists.newArrayList(Pair.of(service, Lists.newArrayList())), grpcMetrics); this.service = service; } From 6ef0ba68fbcf9878c11023f185c358e2e211543f Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Fri, 3 Feb 2023 14:08:47 +0800 Subject: [PATCH 02/21] Temporary submission --- .../common/exception/RejectException.java | 30 ++++++ .../uniffle/test/IntegrationTestBase.java | 1 + .../test/ShuffleServerInternalGrpcTest.java | 91 +++++++++++++++++++ .../api/ShuffleServerInternalClient.java | 29 ++++++ .../grpc/ShuffleServerInternalGrpcClient.java | 80 ++++++++++++++++ .../request/RssDecommissionRequest.java | 27 ++++++ .../response/RssDecommissionResponse.java | 30 ++++++ proto/src/main/proto/Rss.proto | 1 + .../apache/uniffle/server/ShuffleServer.java | 48 ++++++---- .../ShuffleServerInternalGrpcService.java | 33 +++++-- .../org/apache/uniffle/server/StatusCode.java | 3 +- 11 files changed, 350 insertions(+), 23 deletions(-) create mode 100644 common/src/main/java/org/apache/uniffle/common/exception/RejectException.java create mode 100644 integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java create mode 100644 internal-client/src/main/java/org/apache/uniffle/client/api/ShuffleServerInternalClient.java create mode 100644 internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java create mode 100644 internal-client/src/main/java/org/apache/uniffle/client/request/RssDecommissionRequest.java create mode 100644 internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java diff --git a/common/src/main/java/org/apache/uniffle/common/exception/RejectException.java b/common/src/main/java/org/apache/uniffle/common/exception/RejectException.java new file mode 100644 index 0000000000..a26deb5dce --- /dev/null +++ b/common/src/main/java/org/apache/uniffle/common/exception/RejectException.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. + */ + + +package org.apache.uniffle.common.exception; + +public class RejectException extends RuntimeException { + + public RejectException(String message) { + super(message); + } + + public RejectException(String message, Throwable e) { + super(message, e); + } +} diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/IntegrationTestBase.java b/integration-test/common/src/test/java/org/apache/uniffle/test/IntegrationTestBase.java index d080fb2e9d..e85a47dfd8 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/IntegrationTestBase.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/IntegrationTestBase.java @@ -40,6 +40,7 @@ public abstract class IntegrationTestBase extends HdfsTestBase { + protected static final String EXPECTED_EXCEPTION_MESSAGE = "Exception should be thrown"; protected static final int SHUFFLE_SERVER_PORT = 20001; protected static final String LOCALHOST; diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java new file mode 100644 index 0000000000..d80909ebf9 --- /dev/null +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java @@ -0,0 +1,91 @@ +/* + * 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.uniffle.test; + +import com.google.common.collect.Lists; +import org.apache.uniffle.client.impl.grpc.ShuffleServerGrpcClient; +import org.apache.uniffle.client.impl.grpc.ShuffleServerInternalGrpcClient; +import org.apache.uniffle.client.request.RssDecommissionRequest; +import org.apache.uniffle.client.request.RssRegisterShuffleRequest; +import org.apache.uniffle.client.request.RssUnregisterShuffleRequest; +import org.apache.uniffle.client.response.RssDecommissionResponse; +import org.apache.uniffle.common.PartitionRange; +import org.apache.uniffle.coordinator.CoordinatorConf; +import org.apache.uniffle.server.ShuffleServerConf; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +public class ShuffleServerInternalGrpcTest extends IntegrationTestBase { + + private ShuffleServerGrpcClient shuffleServerClient; + private ShuffleServerInternalGrpcClient shuffleServerInternalClient; + + @BeforeAll + public static void setupServers(@TempDir File tmpDir) throws Exception { + CoordinatorConf coordinatorConf = getCoordinatorConf(); + coordinatorConf.setLong(CoordinatorConf.COORDINATOR_APP_EXPIRED, 2000); + createCoordinatorServer(coordinatorConf); + ShuffleServerConf shuffleServerConf = getShuffleServerConf(); + File dataDir1 = new File(tmpDir, "data1"); + String basePath = dataDir1.getAbsolutePath(); + shuffleServerConf.set(ShuffleServerConf.RSS_STORAGE_BASE_PATH, Arrays.asList(basePath)); + shuffleServerConf.set(ShuffleServerConf.SERVER_APP_EXPIRED_WITHOUT_HEARTBEAT, 2000L); + createShuffleServer(shuffleServerConf); + startServers(); + } + + @BeforeEach + public void createClient() { + shuffleServerClient = new ShuffleServerGrpcClient(LOCALHOST, SHUFFLE_SERVER_PORT); + shuffleServerInternalClient = new ShuffleServerInternalGrpcClient(LOCALHOST, SHUFFLE_SERVER_PORT); + } + + @Test + public void decommissionTest() { + String appId = "decommissionTest"; + int shuffleId = 0; + shuffleServerClient.registerShuffle(new RssRegisterShuffleRequest(appId, shuffleId, + Lists.newArrayList(new PartitionRange(0, 1)), "")); + RssDecommissionResponse response = shuffleServerInternalClient.decommission(new RssDecommissionRequest(true)); + assertTrue(response.isOn()); + response = shuffleServerInternalClient.decommission(new RssDecommissionRequest(false)); + assertTrue(!response.isOn()); + + // Clean all apps, shuffle server will be shutdown right now. + shuffleServerClient.unregisterShuffle(new RssUnregisterShuffleRequest(appId, shuffleId)); + response = shuffleServerInternalClient.decommission(new RssDecommissionRequest(true)); + assertTrue(response.isOn()); + try { + // Server is already shutdown, so exception should be thrown here. + shuffleServerInternalClient.decommission(new RssDecommissionRequest(true)); + fail(EXPECTED_EXCEPTION_MESSAGE); + } catch (Exception e) { + //assertTrue(e.getMessage().contains("must be set by the client or fetched from coordinators")); + } + + } + +} diff --git a/internal-client/src/main/java/org/apache/uniffle/client/api/ShuffleServerInternalClient.java b/internal-client/src/main/java/org/apache/uniffle/client/api/ShuffleServerInternalClient.java new file mode 100644 index 0000000000..5f8befc739 --- /dev/null +++ b/internal-client/src/main/java/org/apache/uniffle/client/api/ShuffleServerInternalClient.java @@ -0,0 +1,29 @@ +/* + * 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.uniffle.client.api; + +import org.apache.uniffle.client.request.RssDecommissionRequest; +import org.apache.uniffle.client.response.RssDecommissionResponse; + +public interface ShuffleServerInternalClient { + + RssDecommissionResponse decommission(RssDecommissionRequest request); + + void close(); + +} diff --git a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java new file mode 100644 index 0000000000..774e8e4a98 --- /dev/null +++ b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java @@ -0,0 +1,80 @@ +/* + * 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.uniffle.client.impl.grpc; + +import com.google.protobuf.BoolValue; +import org.apache.uniffle.client.api.ShuffleServerInternalClient; +import org.apache.uniffle.client.request.RssDecommissionRequest; +import org.apache.uniffle.client.response.ResponseStatusCode; +import org.apache.uniffle.client.response.RssDecommissionResponse; +import org.apache.uniffle.common.exception.RssException; +import org.apache.uniffle.proto.RssProtos; +import org.apache.uniffle.proto.ShuffleServerInternalGrpc; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.TimeUnit; + +public class ShuffleServerInternalGrpcClient extends GrpcClient implements ShuffleServerInternalClient { + + private static final Logger LOG = LoggerFactory.getLogger(ShuffleServerGrpcClient.class); + private static final long FAILED_REQUIRE_ID = -1; + private static final long RPC_TIMEOUT_DEFAULT_MS = 60000; + private long rpcTimeout = RPC_TIMEOUT_DEFAULT_MS; + private ShuffleServerInternalGrpc.ShuffleServerInternalBlockingStub blockingStub; + + public ShuffleServerInternalGrpcClient(String host, int port) { + this(host, port, 3); + } + + public ShuffleServerInternalGrpcClient(String host, int port, int maxRetryAttempts) { + this(host, port, maxRetryAttempts, true); + } + + public ShuffleServerInternalGrpcClient(String host, int port, int maxRetryAttempts, boolean usePlaintext) { + super(host, port, maxRetryAttempts, usePlaintext); + // todo Add ClientInterceptor for authentication + blockingStub = ShuffleServerInternalGrpc.newBlockingStub(channel); + } + + public ShuffleServerInternalGrpc.ShuffleServerInternalBlockingStub getBlockingStub() { + return blockingStub.withDeadlineAfter(rpcTimeout, TimeUnit.MILLISECONDS); + } + @Override + public RssDecommissionResponse decommission(RssDecommissionRequest request) { + RssProtos.DecommissionRequest protoRequest = + RssProtos.DecommissionRequest.newBuilder().setOn(BoolValue.newBuilder().setValue(request.isOn()).build()) + .build(); + RssProtos.DecommissionResponse rpcResponse = + blockingStub.withDeadlineAfter(RPC_TIMEOUT_DEFAULT_MS, TimeUnit.MILLISECONDS).decommission(protoRequest); + RssProtos.StatusCode statusCode = rpcResponse.getStatus(); + + RssDecommissionResponse response; + switch (statusCode) { + case SUCCESS: + response = new RssDecommissionResponse( + ResponseStatusCode.SUCCESS, rpcResponse.getOn().getValue()); + break; + default: + String msg = "Can't " + (request.isOn() ? "enable" : "disable") + " decommission to " + + host + ":" + port + ", errorMsg:" + rpcResponse.getRetMsg(); + LOG.error(msg); + throw new RssException(msg); + } + return response; + } +} diff --git a/internal-client/src/main/java/org/apache/uniffle/client/request/RssDecommissionRequest.java b/internal-client/src/main/java/org/apache/uniffle/client/request/RssDecommissionRequest.java new file mode 100644 index 0000000000..dadde86ab5 --- /dev/null +++ b/internal-client/src/main/java/org/apache/uniffle/client/request/RssDecommissionRequest.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. + */ +package org.apache.uniffle.client.request; + +public class RssDecommissionRequest { + private boolean on; + public boolean isOn() { + return on; + } + public RssDecommissionRequest(boolean on) { + this.on = on; + } +} \ No newline at end of file diff --git a/internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java b/internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java new file mode 100644 index 0000000000..7cba95dacb --- /dev/null +++ b/internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.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. + */ + +package org.apache.uniffle.client.response; + +public class RssDecommissionResponse extends ClientResponse{ + private boolean on; + public RssDecommissionResponse(ResponseStatusCode statusCode, boolean on) { + super(statusCode); + this.on = on; + } + + public boolean isOn() { + return on; + } +} \ No newline at end of file diff --git a/proto/src/main/proto/Rss.proto b/proto/src/main/proto/Rss.proto index bb397cd9f5..f7d5c1dae4 100644 --- a/proto/src/main/proto/Rss.proto +++ b/proto/src/main/proto/Rss.proto @@ -279,6 +279,7 @@ enum StatusCode { INTERNAL_ERROR = 6; TIMEOUT = 7; ACCESS_DENIED = 8; + REJECT = 9; // add more status } diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java index 5af5cb9dfa..d86ee01c4e 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java @@ -27,6 +27,8 @@ import com.google.common.collect.Sets; import io.prometheus.client.CollectorRegistry; import org.apache.commons.collections.CollectionUtils; +import org.apache.uniffle.common.exception.RejectException; +import org.apache.uniffle.proto.RssProtos.ShuffleServerHeartBeatRequest.ServerStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import picocli.CommandLine; @@ -78,8 +80,9 @@ public class ShuffleServer { private AtomicBoolean isHealthy = new AtomicBoolean(true); private GRPCMetrics grpcMetrics; private MetricReporter metricReporter; - private boolean decommissioned; private Thread decommissionedThread; + private ServerStatus serverStatus = ServerStatus.NORMAL; + private Object statusLock = new Object(); public ShuffleServer(ShuffleServerConf shuffleServerConf) throws Exception { this.shuffleServerConf = shuffleServerConf; @@ -335,25 +338,31 @@ public GRPCMetrics getGrpcMetrics() { return grpcMetrics; } - public boolean isDecommissioned() { - return decommissioned; + public boolean isDecommissioning() { + return ServerStatus.DECOMMISSIONING.equals(serverStatus); } - - public synchronized void setDecommissioned(boolean decommissioned) { - if (this.decommissioned == decommissioned) { - return; + public void decommission() { + if (isDecommissioning()) { + throw new RejectException("Shuffle Server is decommissioning. Nothing need to do."); + } + if (!ServerStatus.NORMAL.equals(serverStatus)) { + throw new RejectException("Shuffle Server is processing other procedures, current status:" + serverStatus); } - this.decommissioned = decommissioned; - LOG.info("set decommissioned state to " + decommissioned); - if (this.decommissioned) { + synchronized (statusLock) { decommissionedThread = new Thread(() -> { - while (decommissioned) { + while (isDecommissioning()) { int remainApplicationNum = shuffleTaskManager.getAppIds().size(); - if (shuffleTaskManager.getAppIds().isEmpty()) { + if (remainApplicationNum == 0) { LOG.info("all applications finished, exit now"); - System.exit(0); + try { + stopServer(); + break; + } catch (Exception e) { + LOG.error("Stop server failed!", e); + System.exit(0); + } } - LOG.info("shuffle server is in decommissioned state. remain {} applications not finished.", remainApplicationNum); + LOG.info("Shuffle server is decommissioning. remain {} applications not finished.", remainApplicationNum); try { Thread.sleep(60000); } catch (InterruptedException e) { @@ -363,12 +372,19 @@ public synchronized void setDecommissioned(boolean decommissioned) { }); decommissionedThread.setName("decommission"); decommissionedThread.start(); - } else { + serverStatus = ServerStatus.DECOMMISSIONING; + } + } + public void cancelDecommission() { + if (!isDecommissioning()) { + throw new RejectException("Shuffle server is not decommissioning. Nothing need to do."); + } + synchronized (statusLock) { if (decommissionedThread != null) { decommissionedThread.interrupt(); decommissionedThread = null; } + serverStatus = ServerStatus.NORMAL; } } - } diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java index fdfe662d33..7ebe6e0a77 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java @@ -14,10 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.uniffle.server; import com.google.protobuf.BoolValue; import io.grpc.stub.StreamObserver; +import org.apache.uniffle.common.exception.RejectException; import org.apache.uniffle.proto.RssProtos; import org.apache.uniffle.proto.ShuffleServerInternalGrpc.ShuffleServerInternalImplBase; @@ -29,12 +31,31 @@ public ShuffleServerInternalGrpcService(ShuffleServer shuffleServer) { } @Override public void decommission(RssProtos.DecommissionRequest request, StreamObserver responseObserver) { - shuffleServer.setDecommissioned(request.getOn().getValue()); - RssProtos.DecommissionResponse response = RssProtos.DecommissionResponse - .newBuilder() - .setStatus(RssProtos.StatusCode.forNumber(StatusCode.SUCCESS.statusCode())) - .setOn(BoolValue.newBuilder().setValue(shuffleServer.isDecommissioned()).build()) - .build(); + boolean decommission = request.getOn().getValue(); + RssProtos.DecommissionResponse response; + try { + if (decommission) { + shuffleServer.decommission(); + } else { + shuffleServer.cancelDecommission(); + } + response = RssProtos.DecommissionResponse + .newBuilder() + .setStatus(StatusCode.SUCCESS.toProto()) + .setOn(BoolValue.newBuilder().setValue(shuffleServer.isDecommissioning()).build()) + .build(); + } catch (Exception e) { + StatusCode statusCode = StatusCode.INTERNAL_ERROR; + if (e instanceof RejectException) { + statusCode = StatusCode.REJECT; + } + response = RssProtos.DecommissionResponse + .newBuilder() + .setStatus(statusCode.toProto()) + .setOn(BoolValue.newBuilder().setValue(shuffleServer.isDecommissioning()).build()) + .setRetMsg(e.getMessage()) + .build(); + } responseObserver.onNext(response); responseObserver.onCompleted(); } diff --git a/server/src/main/java/org/apache/uniffle/server/StatusCode.java b/server/src/main/java/org/apache/uniffle/server/StatusCode.java index 98fe3973c7..e3eeae309f 100644 --- a/server/src/main/java/org/apache/uniffle/server/StatusCode.java +++ b/server/src/main/java/org/apache/uniffle/server/StatusCode.java @@ -28,7 +28,8 @@ public enum StatusCode { NO_PARTITION(5), INTERNAL_ERROR(6), TIMEOUT(7), - ACCESS_DENIED(8); + ACCESS_DENIED(8), + REJECT(9); private final int statusCode; From 5663623ce0bf863a2f9ce2b7f0cd9071241e5cd5 Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Fri, 3 Feb 2023 18:08:49 +0800 Subject: [PATCH 03/21] Temporary submission --- .../apache/uniffle/common/ServerStatus.java | 53 +++++++++ .../coordinator/CoordinatorGrpcService.java | 2 + .../uniffle/coordinator/ServerNode.java | 27 ++++- .../coordinator/SimpleClusterManager.java | 4 + .../uniffle/coordinator/ServerNodeTest.java | 4 +- .../test/ShuffleServerInternalGrpcTest.java | 15 +-- .../impl/grpc/CoordinatorGrpcClient.java | 3 + .../grpc/ShuffleServerInternalGrpcClient.java | 11 +- .../request/RssDecommissionRequest.java | 5 +- .../request/RssSendHeartBeatRequest.java | 8 ++ .../response/RssDecommissionResponse.java | 5 +- .../uniffle/server/RegisterHeartBeat.java | 4 + .../apache/uniffle/server/ShuffleServer.java | 102 +++++++++--------- .../ShuffleServerInternalGrpcService.java | 6 +- .../uniffle/server/MockedGrpcServer.java | 1 + 15 files changed, 184 insertions(+), 66 deletions(-) create mode 100644 common/src/main/java/org/apache/uniffle/common/ServerStatus.java diff --git a/common/src/main/java/org/apache/uniffle/common/ServerStatus.java b/common/src/main/java/org/apache/uniffle/common/ServerStatus.java new file mode 100644 index 0000000000..dde14a1a56 --- /dev/null +++ b/common/src/main/java/org/apache/uniffle/common/ServerStatus.java @@ -0,0 +1,53 @@ +/* + * 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.uniffle.common; + +import org.apache.uniffle.proto.RssProtos; + +import java.util.HashMap; +import java.util.Map; + +public enum ServerStatus { + NORMAL(0), + DECOMMISSIONING(1); + private final int statusCode; + + private static Map statusMap; + + static { + statusMap = new HashMap<>(); + for (ServerStatus status : ServerStatus.values()) { + statusMap.put(status.serverStatus(), status); + } + } + ServerStatus(int code) { + this.statusCode = code; + } + + public int serverStatus() { + return statusCode; + } + + public RssProtos.ShuffleServerHeartBeatRequest.ServerStatus toProto() { + return RssProtos.ShuffleServerHeartBeatRequest.ServerStatus.forNumber(this.serverStatus()); + } + + public static ServerStatus fromProto(RssProtos.ShuffleServerHeartBeatRequest.ServerStatus status) { + return statusMap.get(status.getNumber()); + } + +} diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java index 8e2b49d1a1..45f14abbe8 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java @@ -28,6 +28,7 @@ import io.grpc.Context; import io.grpc.Status; import io.grpc.stub.StreamObserver; +import org.apache.uniffle.common.ServerStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -376,6 +377,7 @@ private ServerNode toServerNode(ShuffleServerHeartBeatRequest request) { request.getEventNumInFlush(), Sets.newHashSet(request.getTagsList()), isHealthy, + ServerStatus.fromProto(request.getStatus()), StorageInfoUtils.fromProto(request.getStorageInfoMap())); } } diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/ServerNode.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/ServerNode.java index 069e5c7909..2159a40723 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/ServerNode.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/ServerNode.java @@ -22,6 +22,7 @@ import com.google.common.collect.Maps; +import org.apache.uniffle.common.ServerStatus; import org.apache.uniffle.common.storage.StorageInfo; import org.apache.uniffle.proto.RssProtos.ShuffleServerId; @@ -37,8 +38,10 @@ public class ServerNode implements Comparable { private long timestamp; private Set tags; private boolean isHealthy; + private final ServerStatus status; private Map storageInfo; + // Only for test public ServerNode( String id, String ip, @@ -50,7 +53,7 @@ public ServerNode( Set tags, boolean isHealthy) { this(id, ip, port, usedMemory, preAllocatedMemory, availableMemory, eventNumInFlush, tags, isHealthy, - Maps.newHashMap()); + ServerStatus.NORMAL, Maps.newHashMap()); } public ServerNode( @@ -63,6 +66,22 @@ public ServerNode( int eventNumInFlush, Set tags, boolean isHealthy, + ServerStatus status) { + this(id, ip, port, usedMemory, preAllocatedMemory, availableMemory, eventNumInFlush, tags, isHealthy, + status, Maps.newHashMap()); + } + + public ServerNode( + String id, + String ip, + int port, + long usedMemory, + long preAllocatedMemory, + long availableMemory, + int eventNumInFlush, + Set tags, + boolean isHealthy, + ServerStatus status, Map storageInfoMap) { this.id = id; this.ip = ip; @@ -74,6 +93,7 @@ public ServerNode( this.timestamp = System.currentTimeMillis(); this.tags = tags; this.isHealthy = isHealthy; + this.status = status; this.storageInfo = storageInfoMap; } @@ -120,7 +140,9 @@ public Set getTags() { public boolean isHealthy() { return isHealthy; } - + public ServerStatus getStatus() { + return status; + } public Map getStorageInfo() { return storageInfo; } @@ -137,6 +159,7 @@ public String toString() { + "], timestamp[" + timestamp + "], tags" + tags.toString() + "" + ", healthy[" + isHealthy + + ", status[" + status + "], storages[num=" + storageInfo.size() + "]"; } diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java index ae1f6d2329..21298e7935 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java @@ -41,6 +41,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.uniffle.common.ServerStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -199,6 +200,9 @@ public void add(ServerNode node) { public List getServerList(Set requiredTags) { List availableNodes = Lists.newArrayList(); for (ServerNode node : servers.values()) { + if (!ServerStatus.NORMAL.equals(node.getStatus())) { + continue; + } if (!excludeNodes.contains(node.getId()) && node.getTags().containsAll(requiredTags) && node.isHealthy()) { diff --git a/coordinator/src/test/java/org/apache/uniffle/coordinator/ServerNodeTest.java b/coordinator/src/test/java/org/apache/uniffle/coordinator/ServerNodeTest.java index 9c6e7a3876..73f813ccf3 100644 --- a/coordinator/src/test/java/org/apache/uniffle/coordinator/ServerNodeTest.java +++ b/coordinator/src/test/java/org/apache/uniffle/coordinator/ServerNodeTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import org.apache.uniffle.common.ServerStatus; import org.junit.jupiter.api.Test; import org.apache.uniffle.common.storage.StorageInfo; @@ -65,7 +66,8 @@ public void testStorageInfoOfServerNode() { 60L, StorageStatus.NORMAL); localStorageInfo.put("/mnt", info); - ServerNode sn2 = new ServerNode("sn2", "ip", 0, 100L, 50L, 20, 10, tags, true, localStorageInfo); + ServerNode sn2 = new ServerNode("sn2", "ip", 0, 100L, 50L, 20, 10, tags, + true, ServerStatus.NORMAL, localStorageInfo); assertEquals(1, sn2.getStorageInfo().size()); } } diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java index d80909ebf9..b782c27c29 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java @@ -17,7 +17,15 @@ package org.apache.uniffle.test; +import java.io.File; +import java.util.Arrays; + import com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + import org.apache.uniffle.client.impl.grpc.ShuffleServerGrpcClient; import org.apache.uniffle.client.impl.grpc.ShuffleServerInternalGrpcClient; import org.apache.uniffle.client.request.RssDecommissionRequest; @@ -27,13 +35,6 @@ import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.coordinator.CoordinatorConf; import org.apache.uniffle.server.ShuffleServerConf; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -import java.io.File; -import java.util.Arrays; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; diff --git a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java index 38f90dc1af..adeed4df4a 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java @@ -118,6 +118,7 @@ public ShuffleServerHeartBeatResponse doSendHeartBeat( long timeout, Set tags, boolean isHealthy, + ShuffleServerHeartBeatRequest.ServerStatus serverStatus, Map storageInfo) { ShuffleServerId serverId = ShuffleServerId.newBuilder().setId(id).setIp(ip).setPort(port).build(); @@ -130,6 +131,7 @@ public ShuffleServerHeartBeatResponse doSendHeartBeat( .setEventNumInFlush(eventNumInFlush) .addAllTags(tags) .setIsHealthy(BoolValue.newBuilder().setValue(isHealthy).build()) + .setStatusValue(serverStatus.getNumber()) .putAllStorageInfo(StorageInfoUtils.toProto(storageInfo)) .build(); @@ -195,6 +197,7 @@ public RssSendHeartBeatResponse sendHeartBeat(RssSendHeartBeatRequest request) { request.getTimeout(), request.getTags(), request.isHealthy(), + request.getServerStatus(), request.getStorageInfo()); RssSendHeartBeatResponse response; diff --git a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java index 774e8e4a98..c6633c8946 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java @@ -14,9 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.uniffle.client.impl.grpc; +import java.util.concurrent.TimeUnit; + import com.google.protobuf.BoolValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.uniffle.client.api.ShuffleServerInternalClient; import org.apache.uniffle.client.request.RssDecommissionRequest; import org.apache.uniffle.client.response.ResponseStatusCode; @@ -24,10 +30,6 @@ import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.proto.RssProtos; import org.apache.uniffle.proto.ShuffleServerInternalGrpc; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.TimeUnit; public class ShuffleServerInternalGrpcClient extends GrpcClient implements ShuffleServerInternalClient { @@ -54,6 +56,7 @@ public ShuffleServerInternalGrpcClient(String host, int port, int maxRetryAttemp public ShuffleServerInternalGrpc.ShuffleServerInternalBlockingStub getBlockingStub() { return blockingStub.withDeadlineAfter(rpcTimeout, TimeUnit.MILLISECONDS); } + @Override public RssDecommissionResponse decommission(RssDecommissionRequest request) { RssProtos.DecommissionRequest protoRequest = diff --git a/internal-client/src/main/java/org/apache/uniffle/client/request/RssDecommissionRequest.java b/internal-client/src/main/java/org/apache/uniffle/client/request/RssDecommissionRequest.java index dadde86ab5..f50dedcde1 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/request/RssDecommissionRequest.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/request/RssDecommissionRequest.java @@ -14,14 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.uniffle.client.request; public class RssDecommissionRequest { private boolean on; + public boolean isOn() { return on; } + public RssDecommissionRequest(boolean on) { this.on = on; } -} \ No newline at end of file +} diff --git a/internal-client/src/main/java/org/apache/uniffle/client/request/RssSendHeartBeatRequest.java b/internal-client/src/main/java/org/apache/uniffle/client/request/RssSendHeartBeatRequest.java index 12a20f5e0c..9576075d27 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/request/RssSendHeartBeatRequest.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/request/RssSendHeartBeatRequest.java @@ -21,6 +21,7 @@ import java.util.Map; import java.util.Set; +import org.apache.uniffle.common.ServerStatus; import org.apache.uniffle.common.storage.StorageInfo; public class RssSendHeartBeatRequest { @@ -35,6 +36,7 @@ public class RssSendHeartBeatRequest { private final Set tags; private final long timeout; private final boolean isHealthy; + private final ServerStatus serverStatus; private final Map storageInfo; public RssSendHeartBeatRequest( @@ -48,6 +50,7 @@ public RssSendHeartBeatRequest( long timeout, Set tags, boolean isHealthy, + ServerStatus serverStatus, Map storageInfo) { this.shuffleServerId = shuffleServerId; this.shuffleServerIp = shuffleServerIp; @@ -59,6 +62,7 @@ public RssSendHeartBeatRequest( this.tags = tags; this.timeout = timeout; this.isHealthy = isHealthy; + this.serverStatus = serverStatus; this.storageInfo = storageInfo; } @@ -102,6 +106,10 @@ public boolean isHealthy() { return isHealthy; } + public ServerStatus getServerStatus() { + return serverStatus; + } + public Map getStorageInfo() { return storageInfo; } diff --git a/internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java b/internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java index 7cba95dacb..aa2e6252d8 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java @@ -17,8 +17,9 @@ package org.apache.uniffle.client.response; -public class RssDecommissionResponse extends ClientResponse{ +public class RssDecommissionResponse extends ClientResponse { private boolean on; + public RssDecommissionResponse(ResponseStatusCode statusCode, boolean on) { super(statusCode); this.on = on; @@ -27,4 +28,4 @@ public RssDecommissionResponse(ResponseStatusCode statusCode, boolean on) { public boolean isOn() { return on; } -} \ No newline at end of file +} diff --git a/server/src/main/java/org/apache/uniffle/server/RegisterHeartBeat.java b/server/src/main/java/org/apache/uniffle/server/RegisterHeartBeat.java index 4235256003..dfe5aa0846 100644 --- a/server/src/main/java/org/apache/uniffle/server/RegisterHeartBeat.java +++ b/server/src/main/java/org/apache/uniffle/server/RegisterHeartBeat.java @@ -28,6 +28,7 @@ import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; +import org.apache.uniffle.common.ServerStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,6 +84,7 @@ public void startHeartBeat() { shuffleServer.getEventNumInFlush(), shuffleServer.getTags(), shuffleServer.isHealthy(), + shuffleServer.getServerStatus(), shuffleServer.getStorageManager().getStorageInfo()); } catch (Exception e) { LOG.warn("Error happened when send heart beat to coordinator"); @@ -102,6 +104,7 @@ boolean sendHeartBeat( int eventNumInFlush, Set tags, boolean isHealthy, + ServerStatus serverStatus, Map localStorageInfo) { boolean sendSuccessfully = false; RssSendHeartBeatRequest request = new RssSendHeartBeatRequest( @@ -115,6 +118,7 @@ boolean sendHeartBeat( heartBeatTimeout, tags, isHealthy, + serverStatus, localStorageInfo); List> respFutures = coordinatorClients .stream() diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java index d86ee01c4e..101382ade6 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java @@ -27,13 +27,13 @@ import com.google.common.collect.Sets; import io.prometheus.client.CollectorRegistry; import org.apache.commons.collections.CollectionUtils; -import org.apache.uniffle.common.exception.RejectException; -import org.apache.uniffle.proto.RssProtos.ShuffleServerHeartBeatRequest.ServerStatus; +import org.apache.uniffle.common.ServerStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import picocli.CommandLine; import org.apache.uniffle.common.Arguments; +import org.apache.uniffle.common.exception.RejectException; import org.apache.uniffle.common.metrics.GRPCMetrics; import org.apache.uniffle.common.metrics.JvmMetrics; import org.apache.uniffle.common.metrics.MetricReporter; @@ -264,6 +264,57 @@ private void blockUntilShutdown() throws InterruptedException { server.blockUntilShutdown(); } + public ServerStatus getServerStatus() { + return serverStatus; + } + public void decommission() { + if (isDecommissioning()) { + throw new RejectException("Shuffle Server is decommissioning. Nothing need to do."); + } + if (!ServerStatus.NORMAL.equals(serverStatus)) { + throw new RejectException("Shuffle Server is processing other procedures, current status:" + serverStatus); + } + synchronized (statusLock) { + decommissionedThread = new Thread(() -> { + while (isDecommissioning()) { + int remainApplicationNum = shuffleTaskManager.getAppIds().size(); + if (remainApplicationNum == 0) { + LOG.info("all applications finished, exit now"); + try { + stopServer(); + break; + } catch (Exception e) { + LOG.error("Stop server failed!", e); + System.exit(0); + } + } + LOG.info("Shuffle server is decommissioning. remain {} applications not finished.", remainApplicationNum); + try { + Thread.sleep(60000); + } catch (InterruptedException e) { + LOG.warn("Ignore the InterruptedException which should be caused by internal killed"); + } + } + }); + decommissionedThread.setName("decommission"); + decommissionedThread.start(); + serverStatus = ServerStatus.DECOMMISSIONING; + } + } + + public void cancelDecommission() { + if (!isDecommissioning()) { + throw new RejectException("Shuffle server is not decommissioning. Nothing need to do."); + } + synchronized (statusLock) { + if (decommissionedThread != null) { + decommissionedThread.interrupt(); + decommissionedThread = null; + } + serverStatus = ServerStatus.NORMAL; + } + } + public String getIp() { return this.ip; } @@ -341,50 +392,5 @@ public GRPCMetrics getGrpcMetrics() { public boolean isDecommissioning() { return ServerStatus.DECOMMISSIONING.equals(serverStatus); } - public void decommission() { - if (isDecommissioning()) { - throw new RejectException("Shuffle Server is decommissioning. Nothing need to do."); - } - if (!ServerStatus.NORMAL.equals(serverStatus)) { - throw new RejectException("Shuffle Server is processing other procedures, current status:" + serverStatus); - } - synchronized (statusLock) { - decommissionedThread = new Thread(() -> { - while (isDecommissioning()) { - int remainApplicationNum = shuffleTaskManager.getAppIds().size(); - if (remainApplicationNum == 0) { - LOG.info("all applications finished, exit now"); - try { - stopServer(); - break; - } catch (Exception e) { - LOG.error("Stop server failed!", e); - System.exit(0); - } - } - LOG.info("Shuffle server is decommissioning. remain {} applications not finished.", remainApplicationNum); - try { - Thread.sleep(60000); - } catch (InterruptedException e) { - LOG.warn("Ignore the InterruptedException which should be caused by internal killed"); - } - } - }); - decommissionedThread.setName("decommission"); - decommissionedThread.start(); - serverStatus = ServerStatus.DECOMMISSIONING; - } - } - public void cancelDecommission() { - if (!isDecommissioning()) { - throw new RejectException("Shuffle server is not decommissioning. Nothing need to do."); - } - synchronized (statusLock) { - if (decommissionedThread != null) { - decommissionedThread.interrupt(); - decommissionedThread = null; - } - serverStatus = ServerStatus.NORMAL; - } - } + } diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java index 7ebe6e0a77..2f30ab4b08 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java @@ -19,6 +19,7 @@ import com.google.protobuf.BoolValue; import io.grpc.stub.StreamObserver; + import org.apache.uniffle.common.exception.RejectException; import org.apache.uniffle.proto.RssProtos; import org.apache.uniffle.proto.ShuffleServerInternalGrpc.ShuffleServerInternalImplBase; @@ -29,8 +30,11 @@ public class ShuffleServerInternalGrpcService extends ShuffleServerInternalImplB public ShuffleServerInternalGrpcService(ShuffleServer shuffleServer) { this.shuffleServer = shuffleServer; } + @Override - public void decommission(RssProtos.DecommissionRequest request, StreamObserver responseObserver) { + public void decommission( + RssProtos.DecommissionRequest request, + StreamObserver responseObserver) { boolean decommission = request.getOn().getValue(); RssProtos.DecommissionResponse response; try { diff --git a/server/src/test/java/org/apache/uniffle/server/MockedGrpcServer.java b/server/src/test/java/org/apache/uniffle/server/MockedGrpcServer.java index 0d10935baa..58de5382f0 100644 --- a/server/src/test/java/org/apache/uniffle/server/MockedGrpcServer.java +++ b/server/src/test/java/org/apache/uniffle/server/MockedGrpcServer.java @@ -19,6 +19,7 @@ import com.google.common.collect.Lists; import org.apache.commons.lang3.tuple.Pair; + import org.apache.uniffle.common.config.RssBaseConf; import org.apache.uniffle.common.metrics.GRPCMetrics; import org.apache.uniffle.common.rpc.GrpcServer; From 855911935f30135a92f6207e97e81196c58fae78 Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Mon, 6 Feb 2023 15:24:15 +0800 Subject: [PATCH 04/21] Complete the rpc section --- .../apache/uniffle/common/ServerStatus.java | 14 +++-- .../common/web/DecommissionServlet.java | 35 ++++++++++++ .../common/web/RestAPIBaseServlet.java | 47 ++++++++++++++++ .../uniffle/common/ServerStatusTest.java | 55 +++++++++++++++++++ .../coordinator/CoordinatorGrpcService.java | 2 +- .../uniffle/coordinator/ServerNode.java | 4 +- .../coordinator/SimpleClusterManager.java | 4 +- .../uniffle/coordinator/ServerNodeTest.java | 4 +- .../test/ShuffleServerInternalGrpcTest.java | 20 +++++-- .../impl/grpc/CoordinatorGrpcClient.java | 5 +- .../grpc/ShuffleServerInternalGrpcClient.java | 4 +- .../response/RssDecommissionResponse.java | 4 +- proto/src/main/proto/Rss.proto | 12 ++-- .../uniffle/server/RegisterHeartBeat.java | 2 +- .../apache/uniffle/server/ShuffleServer.java | 22 ++++++-- .../uniffle/server/ShuffleServerConf.java | 6 ++ .../ShuffleServerInternalGrpcService.java | 1 + 17 files changed, 206 insertions(+), 35 deletions(-) create mode 100644 common/src/main/java/org/apache/uniffle/common/web/DecommissionServlet.java create mode 100644 common/src/main/java/org/apache/uniffle/common/web/RestAPIBaseServlet.java create mode 100644 common/src/test/java/org/apache/uniffle/common/ServerStatusTest.java diff --git a/common/src/main/java/org/apache/uniffle/common/ServerStatus.java b/common/src/main/java/org/apache/uniffle/common/ServerStatus.java index dde14a1a56..e53698db7e 100644 --- a/common/src/main/java/org/apache/uniffle/common/ServerStatus.java +++ b/common/src/main/java/org/apache/uniffle/common/ServerStatus.java @@ -14,15 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.uniffle.common; -import org.apache.uniffle.proto.RssProtos; +package org.apache.uniffle.common; import java.util.HashMap; import java.util.Map; +import org.apache.uniffle.proto.RssProtos; + public enum ServerStatus { - NORMAL(0), + NORMAL_STATUS(0), DECOMMISSIONING(1); private final int statusCode; @@ -34,6 +35,7 @@ public enum ServerStatus { statusMap.put(status.serverStatus(), status); } } + ServerStatus(int code) { this.statusCode = code; } @@ -42,11 +44,11 @@ public int serverStatus() { return statusCode; } - public RssProtos.ShuffleServerHeartBeatRequest.ServerStatus toProto() { - return RssProtos.ShuffleServerHeartBeatRequest.ServerStatus.forNumber(this.serverStatus()); + public RssProtos.ServerStatus toProto() { + return RssProtos.ServerStatus.forNumber(this.serverStatus()); } - public static ServerStatus fromProto(RssProtos.ShuffleServerHeartBeatRequest.ServerStatus status) { + public static ServerStatus fromProto(RssProtos.ServerStatus status) { return statusMap.get(status.getNumber()); } diff --git a/common/src/main/java/org/apache/uniffle/common/web/DecommissionServlet.java b/common/src/main/java/org/apache/uniffle/common/web/DecommissionServlet.java new file mode 100644 index 0000000000..3191f4aba4 --- /dev/null +++ b/common/src/main/java/org/apache/uniffle/common/web/DecommissionServlet.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. + */ + +package org.apache.uniffle.common.web; + +import java.io.IOException; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +public class DecommissionServlet extends RestAPIBaseServlet { + @Override + protected void handleGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + + } + + @Override + protected void handlePost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + + } +} diff --git a/common/src/main/java/org/apache/uniffle/common/web/RestAPIBaseServlet.java b/common/src/main/java/org/apache/uniffle/common/web/RestAPIBaseServlet.java new file mode 100644 index 0000000000..34a2b3f829 --- /dev/null +++ b/common/src/main/java/org/apache/uniffle/common/web/RestAPIBaseServlet.java @@ -0,0 +1,47 @@ +/* + * 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.uniffle.common.web; + +import java.io.IOException; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +public abstract class RestAPIBaseServlet extends HttpServlet { + @Override + protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + // todo: Do something for authentication + handleGet(req, resp); + } + + @Override + protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + // todo: Do something for authentication + handlePost(req, resp); + } + + protected abstract void handleGet( + HttpServletRequest req, + HttpServletResponse resp) throws ServletException, IOException; + + + protected abstract void handlePost( + HttpServletRequest req, + HttpServletResponse resp) throws ServletException, IOException; +} diff --git a/common/src/test/java/org/apache/uniffle/common/ServerStatusTest.java b/common/src/test/java/org/apache/uniffle/common/ServerStatusTest.java new file mode 100644 index 0000000000..64d848266e --- /dev/null +++ b/common/src/test/java/org/apache/uniffle/common/ServerStatusTest.java @@ -0,0 +1,55 @@ +/* + * 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.uniffle.common; + +import org.junit.jupiter.api.Test; + +import org.apache.uniffle.common.rpc.StatusCode; +import org.apache.uniffle.proto.RssProtos; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; + +public class ServerStatusTest { + + @Test + public void test() throws Exception { + RssProtos.StatusCode[] protoStatusCode = RssProtos.StatusCode.values(); + for (RssProtos.StatusCode statusCode : protoStatusCode) { + try { + if (RssProtos.StatusCode.UNRECOGNIZED.equals(statusCode)) { + continue; + } + StatusCode.valueOf(statusCode.name()); + } catch (Exception e) { + fail(e.getMessage()); + } + } + StatusCode[] statusCodes = StatusCode.values(); + for (StatusCode statusCode : statusCodes) { + try { + RssProtos.StatusCode.valueOf(statusCode.name()); + } catch (Exception e) { + fail(e.getMessage()); + } + } + for (int i = 0; i < statusCodes.length; i++) { + assertEquals(protoStatusCode[i], statusCodes[i].toProto()); + } + } +} diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java index 45f14abbe8..efde220100 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java @@ -28,12 +28,12 @@ import io.grpc.Context; import io.grpc.Status; import io.grpc.stub.StreamObserver; -import org.apache.uniffle.common.ServerStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.RemoteStorageInfo; +import org.apache.uniffle.common.ServerStatus; import org.apache.uniffle.common.storage.StorageInfoUtils; import org.apache.uniffle.coordinator.access.AccessCheckResult; import org.apache.uniffle.coordinator.access.AccessInfo; diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/ServerNode.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/ServerNode.java index 2159a40723..717766952b 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/ServerNode.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/ServerNode.java @@ -53,7 +53,7 @@ public ServerNode( Set tags, boolean isHealthy) { this(id, ip, port, usedMemory, preAllocatedMemory, availableMemory, eventNumInFlush, tags, isHealthy, - ServerStatus.NORMAL, Maps.newHashMap()); + ServerStatus.NORMAL_STATUS, Maps.newHashMap()); } public ServerNode( @@ -140,9 +140,11 @@ public Set getTags() { public boolean isHealthy() { return isHealthy; } + public ServerStatus getStatus() { return status; } + public Map getStorageInfo() { return storageInfo; } diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java index 21298e7935..f27fc02cd0 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java @@ -41,10 +41,10 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.uniffle.common.ServerStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.uniffle.common.ServerStatus; import org.apache.uniffle.common.filesystem.HadoopFilesystemProvider; import org.apache.uniffle.common.util.ThreadUtils; import org.apache.uniffle.coordinator.metric.CoordinatorMetrics; @@ -200,7 +200,7 @@ public void add(ServerNode node) { public List getServerList(Set requiredTags) { List availableNodes = Lists.newArrayList(); for (ServerNode node : servers.values()) { - if (!ServerStatus.NORMAL.equals(node.getStatus())) { + if (!ServerStatus.NORMAL_STATUS.equals(node.getStatus())) { continue; } if (!excludeNodes.contains(node.getId()) diff --git a/coordinator/src/test/java/org/apache/uniffle/coordinator/ServerNodeTest.java b/coordinator/src/test/java/org/apache/uniffle/coordinator/ServerNodeTest.java index 73f813ccf3..8e03cf3905 100644 --- a/coordinator/src/test/java/org/apache/uniffle/coordinator/ServerNodeTest.java +++ b/coordinator/src/test/java/org/apache/uniffle/coordinator/ServerNodeTest.java @@ -25,9 +25,9 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import org.apache.uniffle.common.ServerStatus; import org.junit.jupiter.api.Test; +import org.apache.uniffle.common.ServerStatus; import org.apache.uniffle.common.storage.StorageInfo; import org.apache.uniffle.common.storage.StorageMedia; import org.apache.uniffle.common.storage.StorageStatus; @@ -67,7 +67,7 @@ public void testStorageInfoOfServerNode() { StorageStatus.NORMAL); localStorageInfo.put("/mnt", info); ServerNode sn2 = new ServerNode("sn2", "ip", 0, 100L, 50L, 20, 10, tags, - true, ServerStatus.NORMAL, localStorageInfo); + true, ServerStatus.NORMAL_STATUS, localStorageInfo); assertEquals(1, sn2.getStorageInfo().size()); } } diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java index b782c27c29..95dc96712e 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerInternalGrpcTest.java @@ -19,8 +19,10 @@ import java.io.File; import java.util.Arrays; +import java.util.concurrent.TimeUnit; import com.google.common.collect.Lists; +import org.awaitility.Awaitility; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -33,9 +35,12 @@ import org.apache.uniffle.client.request.RssUnregisterShuffleRequest; import org.apache.uniffle.client.response.RssDecommissionResponse; import org.apache.uniffle.common.PartitionRange; +import org.apache.uniffle.common.ServerStatus; import org.apache.uniffle.coordinator.CoordinatorConf; +import org.apache.uniffle.server.ShuffleServer; import org.apache.uniffle.server.ShuffleServerConf; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -53,7 +58,8 @@ public static void setupServers(@TempDir File tmpDir) throws Exception { File dataDir1 = new File(tmpDir, "data1"); String basePath = dataDir1.getAbsolutePath(); shuffleServerConf.set(ShuffleServerConf.RSS_STORAGE_BASE_PATH, Arrays.asList(basePath)); - shuffleServerConf.set(ShuffleServerConf.SERVER_APP_EXPIRED_WITHOUT_HEARTBEAT, 2000L); + shuffleServerConf.set(ShuffleServerConf.SERVER_APP_EXPIRED_WITHOUT_HEARTBEAT, 5000L); + shuffleServerConf.set(ShuffleServerConf.SERVER_DECOMMISSION_CHECK_INTERVAL, 500L); createShuffleServer(shuffleServerConf); startServers(); } @@ -74,19 +80,23 @@ public void decommissionTest() { assertTrue(response.isOn()); response = shuffleServerInternalClient.decommission(new RssDecommissionRequest(false)); assertTrue(!response.isOn()); + ShuffleServer shuffleServer = shuffleServers.get(0); + assertEquals(ServerStatus.NORMAL_STATUS, shuffleServer.getServerStatus()); // Clean all apps, shuffle server will be shutdown right now. shuffleServerClient.unregisterShuffle(new RssUnregisterShuffleRequest(appId, shuffleId)); response = shuffleServerInternalClient.decommission(new RssDecommissionRequest(true)); assertTrue(response.isOn()); + assertEquals(ServerStatus.DECOMMISSIONING, shuffleServer.getServerStatus()); try { - // Server is already shutdown, so exception should be thrown here. - shuffleServerInternalClient.decommission(new RssDecommissionRequest(true)); + Awaitility.await().timeout(10, TimeUnit.SECONDS).until(() -> + !shuffleServer.isRunning()); + // Server is already shutdown, so io exception should be thrown here. + shuffleServerInternalClient.decommission(new RssDecommissionRequest(false)); fail(EXPECTED_EXCEPTION_MESSAGE); } catch (Exception e) { - //assertTrue(e.getMessage().contains("must be set by the client or fetched from coordinators")); + assertTrue(e.getMessage().contains("UNAVAILABLE: io exception")); } - } } diff --git a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java index 4892c75b25..93ad702426 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java @@ -50,6 +50,7 @@ import org.apache.uniffle.client.response.RssSendHeartBeatResponse; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.RemoteStorageInfo; +import org.apache.uniffle.common.ServerStatus; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.common.rpc.StatusCode; @@ -117,7 +118,7 @@ public ShuffleServerHeartBeatResponse doSendHeartBeat( long timeout, Set tags, boolean isHealthy, - ShuffleServerHeartBeatRequest.ServerStatus serverStatus, + ServerStatus serverStatus, Map storageInfo) { ShuffleServerId serverId = ShuffleServerId.newBuilder().setId(id).setIp(ip).setPort(port).build(); @@ -130,7 +131,7 @@ public ShuffleServerHeartBeatResponse doSendHeartBeat( .setEventNumInFlush(eventNumInFlush) .addAllTags(tags) .setIsHealthy(BoolValue.newBuilder().setValue(isHealthy).build()) - .setStatusValue(serverStatus.getNumber()) + .setStatusValue(serverStatus.serverStatus()) .putAllStorageInfo(StorageInfoUtils.toProto(storageInfo)) .build(); diff --git a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java index c6633c8946..bdb1a4aa1e 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerInternalGrpcClient.java @@ -25,9 +25,9 @@ import org.apache.uniffle.client.api.ShuffleServerInternalClient; import org.apache.uniffle.client.request.RssDecommissionRequest; -import org.apache.uniffle.client.response.ResponseStatusCode; import org.apache.uniffle.client.response.RssDecommissionResponse; import org.apache.uniffle.common.exception.RssException; +import org.apache.uniffle.common.rpc.StatusCode; import org.apache.uniffle.proto.RssProtos; import org.apache.uniffle.proto.ShuffleServerInternalGrpc; @@ -70,7 +70,7 @@ public RssDecommissionResponse decommission(RssDecommissionRequest request) { switch (statusCode) { case SUCCESS: response = new RssDecommissionResponse( - ResponseStatusCode.SUCCESS, rpcResponse.getOn().getValue()); + StatusCode.SUCCESS, rpcResponse.getOn().getValue()); break; default: String msg = "Can't " + (request.isOn() ? "enable" : "disable") + " decommission to " diff --git a/internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java b/internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java index aa2e6252d8..10f7066569 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java @@ -17,10 +17,12 @@ package org.apache.uniffle.client.response; +import org.apache.uniffle.common.rpc.StatusCode; + public class RssDecommissionResponse extends ClientResponse { private boolean on; - public RssDecommissionResponse(ResponseStatusCode statusCode, boolean on) { + public RssDecommissionResponse(StatusCode statusCode, boolean on) { super(statusCode); this.on = on; } diff --git a/proto/src/main/proto/Rss.proto b/proto/src/main/proto/Rss.proto index f7d5c1dae4..56a96aa3ff 100644 --- a/proto/src/main/proto/Rss.proto +++ b/proto/src/main/proto/Rss.proto @@ -234,6 +234,12 @@ message ShuffleCommitResponse { string retMsg = 3; } +enum ServerStatus { + NORMAL_STATUS = 0; + DECOMMISSIONING = 1; + // todo: more status, such as UPGRADING +} + message ShuffleServerHeartBeatRequest { ShuffleServerId serverId = 1; int64 usedMemory = 2; @@ -244,12 +250,6 @@ message ShuffleServerHeartBeatRequest { google.protobuf.BoolValue isHealthy = 7; ServerStatus status = 8; map storageInfo = 21; // mount point to storage info mapping. - - enum ServerStatus { - NORMAL = 0; - DECOMMISSIONING = 1; - // todo: more status, such as UPGRADING - } } message ShuffleServerHeartBeatResponse { diff --git a/server/src/main/java/org/apache/uniffle/server/RegisterHeartBeat.java b/server/src/main/java/org/apache/uniffle/server/RegisterHeartBeat.java index c784165a67..b7883fd5de 100644 --- a/server/src/main/java/org/apache/uniffle/server/RegisterHeartBeat.java +++ b/server/src/main/java/org/apache/uniffle/server/RegisterHeartBeat.java @@ -28,7 +28,6 @@ import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; -import org.apache.uniffle.common.ServerStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,6 +35,7 @@ import org.apache.uniffle.client.factory.CoordinatorClientFactory; import org.apache.uniffle.client.request.RssSendHeartBeatRequest; import org.apache.uniffle.client.response.RssSendHeartBeatResponse; +import org.apache.uniffle.common.ServerStatus; import org.apache.uniffle.common.rpc.StatusCode; import org.apache.uniffle.common.storage.StorageInfo; import org.apache.uniffle.common.util.ThreadUtils; diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java index 101382ade6..6039bc9855 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java @@ -27,12 +27,12 @@ import com.google.common.collect.Sets; import io.prometheus.client.CollectorRegistry; import org.apache.commons.collections.CollectionUtils; -import org.apache.uniffle.common.ServerStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import picocli.CommandLine; import org.apache.uniffle.common.Arguments; +import org.apache.uniffle.common.ServerStatus; import org.apache.uniffle.common.exception.RejectException; import org.apache.uniffle.common.metrics.GRPCMetrics; import org.apache.uniffle.common.metrics.JvmMetrics; @@ -57,6 +57,7 @@ import static org.apache.uniffle.common.config.RssBaseConf.RSS_SECURITY_HADOOP_KRB5_CONF_FILE; import static org.apache.uniffle.common.config.RssBaseConf.RSS_STORAGE_TYPE; import static org.apache.uniffle.common.config.RssBaseConf.RSS_TEST_MODE_ENABLE; +import static org.apache.uniffle.server.ShuffleServerConf.SERVER_DECOMMISSION_CHECK_INTERVAL; /** * Server that manages startup/shutdown of a {@code Greeter} server. @@ -81,8 +82,9 @@ public class ShuffleServer { private GRPCMetrics grpcMetrics; private MetricReporter metricReporter; private Thread decommissionedThread; - private ServerStatus serverStatus = ServerStatus.NORMAL; + private ServerStatus serverStatus = ServerStatus.NORMAL_STATUS; private Object statusLock = new Object(); + private boolean running; public ShuffleServer(ShuffleServerConf shuffleServerConf) throws Exception { this.shuffleServerConf = shuffleServerConf; @@ -128,6 +130,7 @@ public void run() { LOG.info("*** server shut down"); } }); + running = true; LOG.info("Shuffle server start successfully!"); } @@ -154,6 +157,7 @@ public void stopServer() throws Exception { } SecurityContextFactory.get().getSecurityContext().close(); server.stop(); + running = false; LOG.info("RPC Server Stopped!"); } @@ -267,14 +271,17 @@ private void blockUntilShutdown() throws InterruptedException { public ServerStatus getServerStatus() { return serverStatus; } + public void decommission() { if (isDecommissioning()) { throw new RejectException("Shuffle Server is decommissioning. Nothing need to do."); } - if (!ServerStatus.NORMAL.equals(serverStatus)) { + if (!ServerStatus.NORMAL_STATUS.equals(serverStatus)) { throw new RejectException("Shuffle Server is processing other procedures, current status:" + serverStatus); } synchronized (statusLock) { + serverStatus = ServerStatus.DECOMMISSIONING; + long checkInterval = shuffleServerConf.get(SERVER_DECOMMISSION_CHECK_INTERVAL); decommissionedThread = new Thread(() -> { while (isDecommissioning()) { int remainApplicationNum = shuffleTaskManager.getAppIds().size(); @@ -290,7 +297,7 @@ public void decommission() { } LOG.info("Shuffle server is decommissioning. remain {} applications not finished.", remainApplicationNum); try { - Thread.sleep(60000); + Thread.sleep(checkInterval); } catch (InterruptedException e) { LOG.warn("Ignore the InterruptedException which should be caused by internal killed"); } @@ -298,7 +305,6 @@ public void decommission() { }); decommissionedThread.setName("decommission"); decommissionedThread.start(); - serverStatus = ServerStatus.DECOMMISSIONING; } } @@ -307,11 +313,11 @@ public void cancelDecommission() { throw new RejectException("Shuffle server is not decommissioning. Nothing need to do."); } synchronized (statusLock) { + serverStatus = ServerStatus.NORMAL_STATUS; if (decommissionedThread != null) { decommissionedThread.interrupt(); decommissionedThread = null; } - serverStatus = ServerStatus.NORMAL; } } @@ -393,4 +399,8 @@ public boolean isDecommissioning() { return ServerStatus.DECOMMISSIONING.equals(serverStatus); } + public boolean isRunning() { + return running; + } + } diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServerConf.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServerConf.java index 5b18107551..2c3eac0fea 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServerConf.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerConf.java @@ -356,6 +356,12 @@ public class ShuffleServerConf extends RssBaseConf { .withDescription("The memory usage limit ratio for huge partition, it will only triggered when partition's " + "size exceeds the threshold of '" + HUGE_PARTITION_SIZE_THRESHOLD.key() + "'"); + public static final ConfigOption SERVER_DECOMMISSION_CHECK_INTERVAL = ConfigOptions + .key("rss.server.decommission.check.interval") + .longType() + .defaultValue(60 * 1000L) + .withDescription("The interval to check if all applications have finish when server decommissioning"); + public ShuffleServerConf() { } diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java index 2f30ab4b08..87c3cd769d 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java @@ -21,6 +21,7 @@ import io.grpc.stub.StreamObserver; import org.apache.uniffle.common.exception.RejectException; +import org.apache.uniffle.common.rpc.StatusCode; import org.apache.uniffle.proto.RssProtos; import org.apache.uniffle.proto.ShuffleServerInternalGrpc.ShuffleServerInternalImplBase; From 6d433ae5efb9dba491fa89038df366b34cd4773d Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Mon, 13 Feb 2023 18:54:29 +0800 Subject: [PATCH 05/21] Optimize --- .../apache/uniffle/common/ServerStatus.java | 29 +--- ...tion.java => InvalidRequestException.java} | 6 +- .../common/web/DecommissionServlet.java | 35 ----- .../common/web/RestAPIBaseServlet.java | 47 ------ .../uniffle/common/metrics/TestUtils.java | 21 ++- coordinator/pom.xml | 5 +- .../uniffle/coordinator/ClusterManager.java | 6 + .../coordinator/CoordinatorServer.java | 13 ++ .../coordinator/SimpleClusterManager.java | 50 ++++++ .../uniffle/coordinator/web/Response.java | 71 +++++++++ .../coordinator/web/servlet/BaseServlet.java | 86 +++++++++++ .../web/servlet/DecommissionServlet.java | 54 +++++++ .../coordinator/web/servlet/NodesServlet.java | 54 +++++++ .../metric/CoordinatorMetricsTest.java | 8 +- .../org/apache/uniffle/test/ServletTest.java | 144 ++++++++++++++++++ .../impl/grpc/CoordinatorGrpcClient.java | 2 +- .../apache/uniffle/server/ShuffleServer.java | 9 +- .../ShuffleServerInternalGrpcService.java | 4 +- .../server/ShuffleServerMetricsTest.java | 8 +- 19 files changed, 525 insertions(+), 127 deletions(-) rename common/src/main/java/org/apache/uniffle/common/exception/{RejectException.java => InvalidRequestException.java} (83%) delete mode 100644 common/src/main/java/org/apache/uniffle/common/web/DecommissionServlet.java delete mode 100644 common/src/main/java/org/apache/uniffle/common/web/RestAPIBaseServlet.java create mode 100644 coordinator/src/main/java/org/apache/uniffle/coordinator/web/Response.java create mode 100644 coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java create mode 100644 coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/DecommissionServlet.java create mode 100644 coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/NodesServlet.java create mode 100644 integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java diff --git a/common/src/main/java/org/apache/uniffle/common/ServerStatus.java b/common/src/main/java/org/apache/uniffle/common/ServerStatus.java index e53698db7e..0e18dc6dff 100644 --- a/common/src/main/java/org/apache/uniffle/common/ServerStatus.java +++ b/common/src/main/java/org/apache/uniffle/common/ServerStatus.java @@ -17,39 +17,18 @@ package org.apache.uniffle.common; -import java.util.HashMap; -import java.util.Map; - import org.apache.uniffle.proto.RssProtos; public enum ServerStatus { - NORMAL_STATUS(0), - DECOMMISSIONING(1); - private final int statusCode; - - private static Map statusMap; - - static { - statusMap = new HashMap<>(); - for (ServerStatus status : ServerStatus.values()) { - statusMap.put(status.serverStatus(), status); - } - } - - ServerStatus(int code) { - this.statusCode = code; - } - - public int serverStatus() { - return statusCode; - } + NORMAL_STATUS, + DECOMMISSIONING; public RssProtos.ServerStatus toProto() { - return RssProtos.ServerStatus.forNumber(this.serverStatus()); + return RssProtos.ServerStatus.forNumber(this.ordinal()); } public static ServerStatus fromProto(RssProtos.ServerStatus status) { - return statusMap.get(status.getNumber()); + return values()[status.getNumber()]; } } diff --git a/common/src/main/java/org/apache/uniffle/common/exception/RejectException.java b/common/src/main/java/org/apache/uniffle/common/exception/InvalidRequestException.java similarity index 83% rename from common/src/main/java/org/apache/uniffle/common/exception/RejectException.java rename to common/src/main/java/org/apache/uniffle/common/exception/InvalidRequestException.java index a26deb5dce..d03f25e038 100644 --- a/common/src/main/java/org/apache/uniffle/common/exception/RejectException.java +++ b/common/src/main/java/org/apache/uniffle/common/exception/InvalidRequestException.java @@ -18,13 +18,13 @@ package org.apache.uniffle.common.exception; -public class RejectException extends RuntimeException { +public class InvalidRequestException extends RuntimeException { - public RejectException(String message) { + public InvalidRequestException(String message) { super(message); } - public RejectException(String message, Throwable e) { + public InvalidRequestException(String message, Throwable e) { super(message, e); } } diff --git a/common/src/main/java/org/apache/uniffle/common/web/DecommissionServlet.java b/common/src/main/java/org/apache/uniffle/common/web/DecommissionServlet.java deleted file mode 100644 index 3191f4aba4..0000000000 --- a/common/src/main/java/org/apache/uniffle/common/web/DecommissionServlet.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.uniffle.common.web; - -import java.io.IOException; -import javax.servlet.ServletException; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; - -public class DecommissionServlet extends RestAPIBaseServlet { - @Override - protected void handleGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - - } - - @Override - protected void handlePost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - - } -} diff --git a/common/src/main/java/org/apache/uniffle/common/web/RestAPIBaseServlet.java b/common/src/main/java/org/apache/uniffle/common/web/RestAPIBaseServlet.java deleted file mode 100644 index 34a2b3f829..0000000000 --- a/common/src/main/java/org/apache/uniffle/common/web/RestAPIBaseServlet.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.uniffle.common.web; - -import java.io.IOException; -import javax.servlet.ServletException; -import javax.servlet.http.HttpServlet; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; - -public abstract class RestAPIBaseServlet extends HttpServlet { - @Override - protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - // todo: Do something for authentication - handleGet(req, resp); - } - - @Override - protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - // todo: Do something for authentication - handlePost(req, resp); - } - - protected abstract void handleGet( - HttpServletRequest req, - HttpServletResponse resp) throws ServletException, IOException; - - - protected abstract void handlePost( - HttpServletRequest req, - HttpServletResponse resp) throws ServletException, IOException; -} diff --git a/common/src/test/java/org/apache/uniffle/common/metrics/TestUtils.java b/common/src/test/java/org/apache/uniffle/common/metrics/TestUtils.java index 1233f9b535..8d8c5f7301 100644 --- a/common/src/test/java/org/apache/uniffle/common/metrics/TestUtils.java +++ b/common/src/test/java/org/apache/uniffle/common/metrics/TestUtils.java @@ -20,6 +20,7 @@ import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; +import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.URL; @@ -28,7 +29,7 @@ public class TestUtils { private TestUtils() { } - public static String httpGetMetrics(String urlString) throws IOException { + public static String httpGet(String urlString) throws IOException { URL url = new URL(urlString); HttpURLConnection con = (HttpURLConnection) url.openConnection(); con.setRequestMethod("GET"); @@ -42,4 +43,22 @@ public static String httpGetMetrics(String urlString) throws IOException { in.close(); return content.toString(); } + + public static String httpPost(String urlString, String postData) throws IOException { + URL url = new URL(urlString); + HttpURLConnection con = (HttpURLConnection) url.openConnection(); + con.setDoOutput(true); + con.setRequestMethod("POST"); + OutputStream outputStream = con.getOutputStream(); + outputStream.write(postData.getBytes()); + BufferedReader in = new BufferedReader( + new InputStreamReader(con.getInputStream())); + String inputLine; + StringBuffer content = new StringBuffer(); + while ((inputLine = in.readLine()) != null) { + content.append(inputLine); + } + in.close(); + return content.toString(); + } } diff --git a/coordinator/pom.xml b/coordinator/pom.xml index a09be37070..e0934af0f6 100644 --- a/coordinator/pom.xml +++ b/coordinator/pom.xml @@ -36,7 +36,10 @@ org.apache.uniffle rss-common - + + org.apache.uniffle + rss-internal-client + com.google.protobuf protobuf-java-util diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/ClusterManager.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/ClusterManager.java index 8db4abc59f..d671df1654 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/ClusterManager.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/ClusterManager.java @@ -54,4 +54,10 @@ public interface ClusterManager extends Closeable { * @return whether to be ready for serving */ boolean isReadyForServe(); + + ServerNode getServerNodeById(String serverId); + + void decommission(String serverId); + + void cancelDecommission(String serverId); } diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java index e1a126befc..ebe5ff2663 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java @@ -38,6 +38,8 @@ import org.apache.uniffle.coordinator.metric.CoordinatorMetrics; import org.apache.uniffle.coordinator.strategy.assignment.AssignmentStrategy; import org.apache.uniffle.coordinator.strategy.assignment.AssignmentStrategyFactory; +import org.apache.uniffle.coordinator.web.servlet.DecommissionServlet; +import org.apache.uniffle.coordinator.web.servlet.NodesServlet; import static org.apache.uniffle.common.config.RssBaseConf.RSS_SECURITY_HADOOP_KERBEROS_ENABLE; import static org.apache.uniffle.common.config.RssBaseConf.RSS_SECURITY_HADOOP_KERBEROS_KEYTAB_FILE; @@ -139,6 +141,7 @@ private void initialization() throws Exception { id = ip + "-" + port; LOG.info("Start to initialize coordinator {}", id); jettyServer = new JettyServer(coordinatorConf); + registerRestAPI(); // register metrics first to avoid NPE problem when add dynamic metrics registerMetrics(); this.applicationManager = new ApplicationManager(coordinatorConf); @@ -169,6 +172,16 @@ private void initialization() throws Exception { server = coordinatorFactory.getServer(); } + private void registerRestAPI() throws Exception { + LOG.info("Register REST API"); + jettyServer.addServlet( + new NodesServlet(this), + "/api/server/nodes"); + jettyServer.addServlet( + new DecommissionServlet(this), + "/api/server/decommission"); + } + private void registerMetrics() throws Exception { LOG.info("Register metrics"); CollectorRegistry coordinatorCollectorRegistry = new CollectorRegistry(true); diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java index f27fc02cd0..ab90d5abc8 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java @@ -44,7 +44,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.uniffle.client.impl.grpc.ShuffleServerInternalGrpcClient; +import org.apache.uniffle.client.request.RssDecommissionRequest; import org.apache.uniffle.common.ServerStatus; +import org.apache.uniffle.common.exception.InvalidRequestException; import org.apache.uniffle.common.filesystem.HadoopFilesystemProvider; import org.apache.uniffle.common.util.ThreadUtils; import org.apache.uniffle.coordinator.metric.CoordinatorMetrics; @@ -54,6 +57,7 @@ public class SimpleClusterManager implements ClusterManager { private static final Logger LOG = LoggerFactory.getLogger(SimpleClusterManager.class); private final Map servers = Maps.newConcurrentMap(); + private Map serverClientMap = Maps.newConcurrentMap(); private Set excludeNodes = Sets.newConcurrentHashSet(); // tag -> nodes private Map> tagToNodes = Maps.newConcurrentMap(); @@ -117,6 +121,10 @@ void nodesCheck() { for (String serverId : deleteIds) { ServerNode sn = servers.remove(serverId); if (sn != null) { + ShuffleServerInternalGrpcClient shuffleServerClient = serverClientMap.remove(sn); + if (shuffleServerClient != null) { + shuffleServerClient.close(); + } for (Set nodesWithTag : tagToNodes.values()) { nodesWithTag.remove(sn); } @@ -253,6 +261,42 @@ public boolean isReadyForServe() { return readyForServe; } + @Override + public void decommission(String serverId) { + ServerNode serverNode = getServerNodeById(serverId); + if (!ServerStatus.NORMAL_STATUS.equals(serverNode.getStatus())) { + throw new InvalidRequestException("Server [" + serverId + + "] is processing other procedures, current status:" + serverNode.getStatus()); + } + + ShuffleServerInternalGrpcClient shuffleServerClient = serverClientMap.computeIfAbsent(serverNode, + id -> new ShuffleServerInternalGrpcClient(serverNode.getIp(), serverNode.getPort())); + shuffleServerClient.decommission(new RssDecommissionRequest(true)); + + } + + @Override + public void cancelDecommission(String serverId) { + ServerNode serverNode = getServerNodeById(serverId); + if (!ServerStatus.DECOMMISSIONING.equals(serverNode.getStatus())) { + throw new InvalidRequestException("Server [" + + serverId + "] is not decommissioning. Nothing need to do."); + } + + ShuffleServerInternalGrpcClient shuffleServerClient = serverClientMap.computeIfAbsent(serverNode, + id -> new ShuffleServerInternalGrpcClient(serverNode.getIp(), serverNode.getPort())); + shuffleServerClient.decommission(new RssDecommissionRequest(false)); + } + + @Override + public ServerNode getServerNodeById(String serverId) { + ServerNode serverNode = servers.get(serverId); + if (serverNode == null) { + throw new InvalidRequestException("Server Id [" + serverId + "] not found!"); + } + return serverNode; + } + @Override public void close() throws IOException { if (hadoopFileSystem != null) { @@ -268,6 +312,12 @@ public void close() throws IOException { } } + private ShuffleServerInternalGrpcClient createShuffleServerClient(ServerNode serverNode) { + synchronized (serverNode) { + return new ShuffleServerInternalGrpcClient(serverNode.getIp(), serverNode.getPort()); + } + } + @VisibleForTesting public void setStartTime(long startTime) { this.startTime = startTime; diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/Response.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/Response.java new file mode 100644 index 0000000000..b883d55fde --- /dev/null +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/Response.java @@ -0,0 +1,71 @@ +/* + * 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.uniffle.coordinator.web; + +public class Response { + private static final int SUCCESS_CODE = 0; + private static final int ERROR_CODE = -1; + private int code; + private T data; + private String errMsg; + + public Response() { + } + + public Response(int code, T data, String errMsg) { + this.code = code; + this.data = data; + this.errMsg = errMsg; + } + + public static Response success(T data) { + return new Response<>(SUCCESS_CODE, data, null); + } + + public static Response fail(String msg) { + return new Response<>(ERROR_CODE, null, msg); + } + + public static Response fail(String msg, int code) { + return new Response<>(code, null, msg); + } + + public int getCode() { + return code; + } + + public void setCode(int code) { + this.code = code; + } + + public T getData() { + return data; + } + + public void setData(T data) { + this.data = data; + } + + public String getErrMsg() { + return errMsg; + } + + public void setErrMsg(String errMsg) { + this.errMsg = errMsg; + } +} diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java new file mode 100644 index 0000000000..2b44bcede8 --- /dev/null +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java @@ -0,0 +1,86 @@ +/* + * 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.uniffle.coordinator.web.servlet; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Map; +import java.util.concurrent.Callable; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.uniffle.coordinator.web.Response; + +public abstract class BaseServlet extends HttpServlet { + public static final String JSON_MIME_TYPE = "application/json"; + final ObjectMapper mapper = new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL); + + @Override + protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + writeJSON(resp, handlerRequest(() -> handleGet(req, resp))); + } + + @Override + protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + writeJSON(resp, handlerRequest(() -> handlePost(req, resp))); + } + + private Response handlerRequest( + Callable function) { + Response response; + try { + // todo: Do something for authentication + response = function.call(); + } catch (Exception e) { + response = Response.fail(e.getMessage()); + } + return response; + } + + protected Response handleGet( + HttpServletRequest req, + HttpServletResponse resp) throws ServletException, IOException { + throw new IOException("Method not support!"); + } + + + protected Response handlePost( + HttpServletRequest req, + HttpServletResponse resp) throws ServletException, IOException { + throw new IOException("Method not support!"); + } + + protected void writeJSON(final HttpServletResponse resp, final Object obj) + throws IOException { + if (obj == null) { + return; + } + resp.setContentType(JSON_MIME_TYPE); + final OutputStream stream = resp.getOutputStream(); + mapper.writeValue(stream, obj); + } + + protected Map parseParamsFromJson(HttpServletRequest req) throws IOException { + return mapper.readValue(req.getInputStream(), Map.class); + } +} diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/DecommissionServlet.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/DecommissionServlet.java new file mode 100644 index 0000000000..1fe03d2774 --- /dev/null +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/DecommissionServlet.java @@ -0,0 +1,54 @@ +/* + * 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.uniffle.coordinator.web.servlet; + +import java.io.IOException; +import java.util.Map; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import com.google.common.base.Preconditions; + +import org.apache.uniffle.coordinator.ClusterManager; +import org.apache.uniffle.coordinator.CoordinatorServer; +import org.apache.uniffle.coordinator.web.Response; + +public class DecommissionServlet extends BaseServlet { + private final CoordinatorServer coordinator; + + public DecommissionServlet(CoordinatorServer coordinator) { + this.coordinator = coordinator; + } + + @Override + protected Response handlePost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + Map params = parseParamsFromJson(req); + String serverId = (String) params.get("serverId"); + Preconditions.checkNotNull(serverId, "Parameter[serverId] should not be null!"); + Boolean on = (Boolean) params.get("on"); + Preconditions.checkNotNull(on, "Parameter[on] should not be null!"); + ClusterManager clusterManager = coordinator.getClusterManager(); + if (on) { + clusterManager.decommission(serverId); + } else { + clusterManager.cancelDecommission(serverId); + } + return Response.success(null); + } +} diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/NodesServlet.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/NodesServlet.java new file mode 100644 index 0000000000..1b5e2e2f6c --- /dev/null +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/NodesServlet.java @@ -0,0 +1,54 @@ +/* + * 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.uniffle.coordinator.web.servlet; + +import java.io.IOException; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import org.apache.commons.lang3.StringUtils; + +import org.apache.uniffle.coordinator.CoordinatorServer; +import org.apache.uniffle.coordinator.ServerNode; +import org.apache.uniffle.coordinator.web.Response; + + +public class NodesServlet extends BaseServlet { + private final CoordinatorServer coordinator; + + public NodesServlet(CoordinatorServer coordinator) { + this.coordinator = coordinator; + } + + @Override + protected Response handleGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + List serverList = coordinator.getClusterManager().getServerList(Collections.EMPTY_SET); + String id = req.getParameter("id"); + if (StringUtils.isNotEmpty(id)) { + serverList = serverList.stream().filter((server) -> + id.equals(server.getId())).collect(Collectors.toList()); + } + Collections.sort(serverList, Comparator.comparing(ServerNode::getId)); + return Response.success(serverList); + } +} diff --git a/coordinator/src/test/java/org/apache/uniffle/coordinator/metric/CoordinatorMetricsTest.java b/coordinator/src/test/java/org/apache/uniffle/coordinator/metric/CoordinatorMetricsTest.java index 0fcb3d047b..210bb29d2b 100644 --- a/coordinator/src/test/java/org/apache/uniffle/coordinator/metric/CoordinatorMetricsTest.java +++ b/coordinator/src/test/java/org/apache/uniffle/coordinator/metric/CoordinatorMetricsTest.java @@ -68,7 +68,7 @@ public static void tearDown() throws Exception { @Test public void testDynamicMetrics() throws Exception { - String content = TestUtils.httpGetMetrics(SERVER_METRICS_URL); + String content = TestUtils.httpGet(SERVER_METRICS_URL); ObjectMapper mapper = new ObjectMapper(); JsonNode metricsNode = mapper.readTree(content).get("metrics"); String remoteStorageMetricsName = CoordinatorMetrics.REMOTE_STORAGE_IN_USED_PREFIX + "path1"; @@ -85,7 +85,7 @@ public void testDynamicMetrics() throws Exception { @Test public void testCoordinatorMetrics() throws Exception { - String content = TestUtils.httpGetMetrics(SERVER_METRICS_URL); + String content = TestUtils.httpGet(SERVER_METRICS_URL); ObjectMapper mapper = new ObjectMapper(); JsonNode actualObj = mapper.readTree(content); assertEquals(2, actualObj.size()); @@ -94,7 +94,7 @@ public void testCoordinatorMetrics() throws Exception { @Test public void testJvmMetrics() throws Exception { - String content = TestUtils.httpGetMetrics(SERVER_JVM_URL); + String content = TestUtils.httpGet(SERVER_JVM_URL); ObjectMapper mapper = new ObjectMapper(); JsonNode actualObj = mapper.readTree(content); assertEquals(2, actualObj.size()); @@ -102,7 +102,7 @@ public void testJvmMetrics() throws Exception { @Test public void testGrpcMetrics() throws Exception { - String content = TestUtils.httpGetMetrics(SERVER_GRPC_URL); + String content = TestUtils.httpGet(SERVER_GRPC_URL); ObjectMapper mapper = new ObjectMapper(); JsonNode actualObj = mapper.readTree(content); assertEquals(2, actualObj.size()); diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java new file mode 100644 index 0000000000..46016c1c40 --- /dev/null +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java @@ -0,0 +1,144 @@ +/* + * 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.uniffle.test; + +import java.io.File; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import org.apache.uniffle.client.impl.grpc.ShuffleServerGrpcClient; +import org.apache.uniffle.client.request.RssRegisterShuffleRequest; +import org.apache.uniffle.common.PartitionRange; +import org.apache.uniffle.common.ServerStatus; +import org.apache.uniffle.common.config.RssBaseConf; +import org.apache.uniffle.common.metrics.TestUtils; +import org.apache.uniffle.coordinator.CoordinatorConf; +import org.apache.uniffle.coordinator.CoordinatorServer; +import org.apache.uniffle.coordinator.web.Response; +import org.apache.uniffle.server.ShuffleServer; +import org.apache.uniffle.server.ShuffleServerConf; +import org.apache.uniffle.storage.util.StorageType; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class ServletTest extends IntegrationTestBase { + private static final String URL_PREFIX = "http://127.0.0.1:12345/api/"; + private static final String NODES_URL = URL_PREFIX + "server/nodes"; + private static final String DECOMMISSION_URL = URL_PREFIX + "server/decommission"; + private static CoordinatorServer coordinatorServer; + private ObjectMapper objectMapper = new ObjectMapper(); + + @BeforeAll + public static void setUp(@TempDir File tmpDir) throws Exception { + CoordinatorConf coordinatorConf = new CoordinatorConf(); + coordinatorConf.set(RssBaseConf.JETTY_HTTP_PORT, 12345); + coordinatorConf.set(RssBaseConf.JETTY_CORE_POOL_SIZE, 128); + coordinatorConf.set(RssBaseConf.RPC_SERVER_PORT, 12346); + createCoordinatorServer(coordinatorConf); + + ShuffleServerConf shuffleServerConf = getShuffleServerConf(); + shuffleServerConf.set(RssBaseConf.RSS_COORDINATOR_QUORUM, "127.0.0.1:12346"); + File dataDir1 = new File(tmpDir, "data1"); + File dataDir2 = new File(tmpDir, "data2"); + List basePath = Lists.newArrayList(dataDir1.getAbsolutePath(), dataDir2.getAbsolutePath()); + shuffleServerConf.setString(RssBaseConf.RSS_STORAGE_TYPE, StorageType.LOCALFILE.name()); + shuffleServerConf.set(RssBaseConf.RSS_STORAGE_BASE_PATH, basePath); + createShuffleServer(shuffleServerConf); + File dataDir3 = new File(tmpDir, "data3"); + File dataDir4 = new File(tmpDir, "data4"); + basePath = Lists.newArrayList(dataDir3.getAbsolutePath(), dataDir4.getAbsolutePath()); + shuffleServerConf.set(RssBaseConf.RSS_STORAGE_BASE_PATH, basePath); + shuffleServerConf.set(RssBaseConf.RPC_SERVER_PORT, SHUFFLE_SERVER_PORT + 1); + shuffleServerConf.set(RssBaseConf.JETTY_HTTP_PORT, 18081); + createShuffleServer(shuffleServerConf); + startServers(); + coordinatorServer = coordinators.get(0); + Awaitility.await().timeout(30, TimeUnit.SECONDS).until(() -> + coordinatorServer.getClusterManager().list().size() == 2); + } + + @Test + public void testNodesServlet() throws Exception { + String content = TestUtils.httpGet(NODES_URL); + Response> response = objectMapper.readValue(content, new TypeReference>>() { + }); + List serverList = response.getData(); + assertEquals(0, response.getCode()); + assertEquals(2, serverList.size()); + assertEquals(SHUFFLE_SERVER_PORT, Integer.parseInt(serverList.get(0).get("port").toString())); + assertEquals(ServerStatus.NORMAL_STATUS.toString(), serverList.get(0).get("status")); + assertEquals(SHUFFLE_SERVER_PORT + 1, Integer.parseInt(serverList.get(1).get("port").toString())); + assertEquals(ServerStatus.NORMAL_STATUS.toString(), serverList.get(1).get("status")); + + // Only fetch one server. + ShuffleServer shuffleServer = shuffleServers.get(0); + content = TestUtils.httpGet(NODES_URL + "?id=" + shuffleServer.getId()); + response = objectMapper.readValue(content, new TypeReference>>() { + }); + serverList = response.getData(); + assertEquals(1, serverList.size()); + assertEquals(shuffleServer.getId(), serverList.get(0).get("id")); + } + + @Test + public void testDecommissionServlet() throws Exception { + ShuffleServer shuffleServer = shuffleServers.get(0); + assertEquals(ServerStatus.NORMAL_STATUS, shuffleServer.getServerStatus()); + Map params = new HashMap<>(); + params.put("serverId", shuffleServer.getId()); + params.put("on", false); + String content = TestUtils.httpPost(DECOMMISSION_URL, objectMapper.writeValueAsString(params)); + Response response = objectMapper.readValue(content, Response.class); + assertEquals(-1, response.getCode()); + assertNotNull(response.getErrMsg()); + assertTrue(response.getErrMsg().contains("is not decommissioning. Nothing need to do")); + + // Register shuffle, avoid server exiting immediately. + ShuffleServerGrpcClient shuffleServerClient = new ShuffleServerGrpcClient(LOCALHOST, SHUFFLE_SERVER_PORT); + shuffleServerClient.registerShuffle(new RssRegisterShuffleRequest("testDecommissionServlet_appId", 0, + Lists.newArrayList(new PartitionRange(0, 1)), "")); + params.put("on", true); + content = TestUtils.httpPost(DECOMMISSION_URL, objectMapper.writeValueAsString(params)); + response = objectMapper.readValue(content, Response.class); + assertEquals(0, response.getCode()); + assertEquals(ServerStatus.DECOMMISSIONING, shuffleServer.getServerStatus()); + + // Wait until shuffle server send heartbeat to coordinator. + Awaitility.await().timeout(10, TimeUnit.SECONDS).until(() -> + ServerStatus.DECOMMISSIONING.equals( + coordinatorServer.getClusterManager().getServerNodeById(shuffleServer.getId()).getStatus())); + // Cancel decommission. + params.put("on", false); + content = TestUtils.httpPost(DECOMMISSION_URL, objectMapper.writeValueAsString(params)); + response = objectMapper.readValue(content, Response.class); + assertEquals(0, response.getCode()); + assertEquals(ServerStatus.NORMAL_STATUS, shuffleServer.getServerStatus()); + } +} diff --git a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java index 93ad702426..0f66a83127 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java @@ -131,7 +131,7 @@ public ShuffleServerHeartBeatResponse doSendHeartBeat( .setEventNumInFlush(eventNumInFlush) .addAllTags(tags) .setIsHealthy(BoolValue.newBuilder().setValue(isHealthy).build()) - .setStatusValue(serverStatus.serverStatus()) + .setStatusValue(serverStatus.ordinal()) .putAllStorageInfo(StorageInfoUtils.toProto(storageInfo)) .build(); diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java index 6039bc9855..f49c0b5e20 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java @@ -33,7 +33,7 @@ import org.apache.uniffle.common.Arguments; import org.apache.uniffle.common.ServerStatus; -import org.apache.uniffle.common.exception.RejectException; +import org.apache.uniffle.common.exception.InvalidRequestException; import org.apache.uniffle.common.metrics.GRPCMetrics; import org.apache.uniffle.common.metrics.JvmMetrics; import org.apache.uniffle.common.metrics.MetricReporter; @@ -274,10 +274,11 @@ public ServerStatus getServerStatus() { public void decommission() { if (isDecommissioning()) { - throw new RejectException("Shuffle Server is decommissioning. Nothing need to do."); + throw new InvalidRequestException("Shuffle Server is decommissioning. Nothing need to do."); } if (!ServerStatus.NORMAL_STATUS.equals(serverStatus)) { - throw new RejectException("Shuffle Server is processing other procedures, current status:" + serverStatus); + throw new InvalidRequestException( + "Shuffle Server is processing other procedures, current status:" + serverStatus); } synchronized (statusLock) { serverStatus = ServerStatus.DECOMMISSIONING; @@ -310,7 +311,7 @@ public void decommission() { public void cancelDecommission() { if (!isDecommissioning()) { - throw new RejectException("Shuffle server is not decommissioning. Nothing need to do."); + throw new InvalidRequestException("Shuffle server is not decommissioning. Nothing need to do."); } synchronized (statusLock) { serverStatus = ServerStatus.NORMAL_STATUS; diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java index 87c3cd769d..0ee0761fd3 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java @@ -20,7 +20,7 @@ import com.google.protobuf.BoolValue; import io.grpc.stub.StreamObserver; -import org.apache.uniffle.common.exception.RejectException; +import org.apache.uniffle.common.exception.InvalidRequestException; import org.apache.uniffle.common.rpc.StatusCode; import org.apache.uniffle.proto.RssProtos; import org.apache.uniffle.proto.ShuffleServerInternalGrpc.ShuffleServerInternalImplBase; @@ -51,7 +51,7 @@ public void decommission( .build(); } catch (Exception e) { StatusCode statusCode = StatusCode.INTERNAL_ERROR; - if (e instanceof RejectException) { + if (e instanceof InvalidRequestException) { statusCode = StatusCode.REJECT; } response = RssProtos.DecommissionResponse diff --git a/server/src/test/java/org/apache/uniffle/server/ShuffleServerMetricsTest.java b/server/src/test/java/org/apache/uniffle/server/ShuffleServerMetricsTest.java index b995a90f3b..99993eb489 100644 --- a/server/src/test/java/org/apache/uniffle/server/ShuffleServerMetricsTest.java +++ b/server/src/test/java/org/apache/uniffle/server/ShuffleServerMetricsTest.java @@ -77,7 +77,7 @@ public static void tearDown() throws Exception { @Test public void testJvmMetrics() throws Exception { - String content = TestUtils.httpGetMetrics(SERVER_JVM_URL); + String content = TestUtils.httpGet(SERVER_JVM_URL); ObjectMapper mapper = new ObjectMapper(); JsonNode actualObj = mapper.readTree(content); assertEquals(2, actualObj.size()); @@ -85,7 +85,7 @@ public void testJvmMetrics() throws Exception { @Test public void testServerMetrics() throws Exception { - String content = TestUtils.httpGetMetrics(SERVER_METRICS_URL); + String content = TestUtils.httpGet(SERVER_METRICS_URL); ObjectMapper mapper = new ObjectMapper(); JsonNode actualObj = mapper.readTree(content); assertEquals(2, actualObj.size()); @@ -145,7 +145,7 @@ public void testStorageCounter() { @Test public void testGrpcMetrics() throws Exception { - String content = TestUtils.httpGetMetrics(SERVER_GRPC_URL); + String content = TestUtils.httpGet(SERVER_GRPC_URL); ObjectMapper mapper = new ObjectMapper(); JsonNode actualObj = mapper.readTree(content); assertEquals(2, actualObj.size()); @@ -187,7 +187,7 @@ public Void call() throws Exception { f.get(); } - String content = TestUtils.httpGetMetrics(SERVER_METRICS_URL); + String content = TestUtils.httpGet(SERVER_METRICS_URL); ObjectMapper mapper = new ObjectMapper(); JsonNode actualObj = mapper.readTree(content); From 5e33f3323252957f6bcb2b2abe334c1d92ecec35 Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Mon, 13 Feb 2023 19:08:14 +0800 Subject: [PATCH 06/21] Fix code style. --- .../org/apache/uniffle/coordinator/SimpleClusterManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java index 3801087dcb..6a93222dfd 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java @@ -49,7 +49,6 @@ import org.apache.uniffle.common.ServerStatus; import org.apache.uniffle.common.config.RssConf; import org.apache.uniffle.common.exception.InvalidRequestException; - import org.apache.uniffle.common.filesystem.HadoopFilesystemProvider; import org.apache.uniffle.common.util.ThreadUtils; import org.apache.uniffle.coordinator.metric.CoordinatorMetrics; From 4d1f539ce5660e4f3a388f43a1ffd99404b5b40c Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Tue, 14 Feb 2023 11:12:04 +0800 Subject: [PATCH 07/21] Optimize --- .../org/apache/uniffle/common/ServerStatus.java | 4 +++- .../org/apache/uniffle/common/rpc/StatusCode.java | 2 +- .../uniffle/coordinator/CoordinatorServer.java | 4 ++-- .../uniffle/coordinator/SimpleClusterManager.java | 13 ++++++------- proto/src/main/proto/Rss.proto | 2 +- .../org/apache/uniffle/server/ShuffleServer.java | 4 ++-- .../server/ShuffleServerInternalGrpcService.java | 2 +- 7 files changed, 16 insertions(+), 15 deletions(-) diff --git a/common/src/main/java/org/apache/uniffle/common/ServerStatus.java b/common/src/main/java/org/apache/uniffle/common/ServerStatus.java index 0e18dc6dff..f715f6c260 100644 --- a/common/src/main/java/org/apache/uniffle/common/ServerStatus.java +++ b/common/src/main/java/org/apache/uniffle/common/ServerStatus.java @@ -23,12 +23,14 @@ public enum ServerStatus { NORMAL_STATUS, DECOMMISSIONING; + static final ServerStatus[] VALUES = ServerStatus.values(); + public RssProtos.ServerStatus toProto() { return RssProtos.ServerStatus.forNumber(this.ordinal()); } public static ServerStatus fromProto(RssProtos.ServerStatus status) { - return values()[status.getNumber()]; + return VALUES[status.getNumber()]; } } diff --git a/common/src/main/java/org/apache/uniffle/common/rpc/StatusCode.java b/common/src/main/java/org/apache/uniffle/common/rpc/StatusCode.java index 1fe606acfa..ccd0338db9 100644 --- a/common/src/main/java/org/apache/uniffle/common/rpc/StatusCode.java +++ b/common/src/main/java/org/apache/uniffle/common/rpc/StatusCode.java @@ -29,7 +29,7 @@ public enum StatusCode { INTERNAL_ERROR(6), TIMEOUT(7), ACCESS_DENIED(8), - REJECT(9); + INVALID_REQUEST(9); private final int statusCode; diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java index 3c6b144c41..ad60b0bc11 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java @@ -148,7 +148,7 @@ private void initialization() throws Exception { id = ip + "-" + port; LOG.info("Start to initialize coordinator {}", id); jettyServer = new JettyServer(coordinatorConf); - registerRestAPI(); + registerRESTAPI(); // register metrics first to avoid NPE problem when add dynamic metrics registerMetrics(); this.applicationManager = new ApplicationManager(coordinatorConf); @@ -179,7 +179,7 @@ private void initialization() throws Exception { server = coordinatorFactory.getServer(); } - private void registerRestAPI() throws Exception { + private void registerRESTAPI() throws Exception { LOG.info("Register REST API"); jettyServer.addServlet( new NodesServlet(this), diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java index 6a93222dfd..dcd4ef1f3e 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java @@ -269,10 +269,7 @@ public void decommission(String serverId) { throw new InvalidRequestException("Server [" + serverId + "] is processing other procedures, current status:" + serverNode.getStatus()); } - - ShuffleServerInternalGrpcClient shuffleServerClient = serverClientMap.computeIfAbsent(serverNode, - id -> new ShuffleServerInternalGrpcClient(serverNode.getIp(), serverNode.getPort())); - shuffleServerClient.decommission(new RssDecommissionRequest(true)); + getShuffleServerClient(serverNode).decommission(new RssDecommissionRequest(true)); } @@ -283,10 +280,12 @@ public void cancelDecommission(String serverId) { throw new InvalidRequestException("Server [" + serverId + "] is not decommissioning. Nothing need to do."); } + getShuffleServerClient(serverNode).decommission(new RssDecommissionRequest(false)); + } - ShuffleServerInternalGrpcClient shuffleServerClient = serverClientMap.computeIfAbsent(serverNode, - id -> new ShuffleServerInternalGrpcClient(serverNode.getIp(), serverNode.getPort())); - shuffleServerClient.decommission(new RssDecommissionRequest(false)); + private ShuffleServerInternalGrpcClient getShuffleServerClient(ServerNode serverNode) { + return serverClientMap.computeIfAbsent(serverNode, + key -> new ShuffleServerInternalGrpcClient(serverNode.getIp(), serverNode.getPort())); } @Override diff --git a/proto/src/main/proto/Rss.proto b/proto/src/main/proto/Rss.proto index 56a96aa3ff..f268ff1364 100644 --- a/proto/src/main/proto/Rss.proto +++ b/proto/src/main/proto/Rss.proto @@ -279,7 +279,7 @@ enum StatusCode { INTERNAL_ERROR = 6; TIMEOUT = 7; ACCESS_DENIED = 8; - REJECT = 9; + INVALID_REQUEST = 9; // add more status } diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java index f49c0b5e20..e03bcd7158 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java @@ -42,6 +42,7 @@ import org.apache.uniffle.common.security.SecurityConfig; import org.apache.uniffle.common.security.SecurityContextFactory; import org.apache.uniffle.common.util.Constants; +import org.apache.uniffle.common.util.ExitUtils; import org.apache.uniffle.common.util.RssUtils; import org.apache.uniffle.common.web.CommonMetricsServlet; import org.apache.uniffle.common.web.JettyServer; @@ -292,8 +293,7 @@ public void decommission() { stopServer(); break; } catch (Exception e) { - LOG.error("Stop server failed!", e); - System.exit(0); + ExitUtils.terminate(1, "Stop server failed!", e, LOG); } } LOG.info("Shuffle server is decommissioning. remain {} applications not finished.", remainApplicationNum); diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java index 0ee0761fd3..0d2f661789 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerInternalGrpcService.java @@ -52,7 +52,7 @@ public void decommission( } catch (Exception e) { StatusCode statusCode = StatusCode.INTERNAL_ERROR; if (e instanceof InvalidRequestException) { - statusCode = StatusCode.REJECT; + statusCode = StatusCode.INVALID_REQUEST; } response = RssProtos.DecommissionResponse .newBuilder() From ffa34566bec43269e453b868e5df7e1661a50d84 Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Tue, 14 Feb 2023 11:48:00 +0800 Subject: [PATCH 08/21] Optimize --- .../apache/uniffle/server/ShuffleServer.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java index e03bcd7158..2a833383e6 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java @@ -274,14 +274,14 @@ public ServerStatus getServerStatus() { } public void decommission() { - if (isDecommissioning()) { - throw new InvalidRequestException("Shuffle Server is decommissioning. Nothing need to do."); - } - if (!ServerStatus.NORMAL_STATUS.equals(serverStatus)) { - throw new InvalidRequestException( - "Shuffle Server is processing other procedures, current status:" + serverStatus); - } synchronized (statusLock) { + if (isDecommissioning()) { + throw new InvalidRequestException("Shuffle Server is decommissioning. Nothing need to do."); + } + if (!ServerStatus.NORMAL_STATUS.equals(serverStatus)) { + throw new InvalidRequestException( + "Shuffle Server is processing other procedures, current status:" + serverStatus); + } serverStatus = ServerStatus.DECOMMISSIONING; long checkInterval = shuffleServerConf.get(SERVER_DECOMMISSION_CHECK_INTERVAL); decommissionedThread = new Thread(() -> { @@ -310,10 +310,10 @@ public void decommission() { } public void cancelDecommission() { - if (!isDecommissioning()) { - throw new InvalidRequestException("Shuffle server is not decommissioning. Nothing need to do."); - } synchronized (statusLock) { + if (!isDecommissioning()) { + throw new InvalidRequestException("Shuffle server is not decommissioning. Nothing need to do."); + } serverStatus = ServerStatus.NORMAL_STATUS; if (decommissionedThread != null) { decommissionedThread.interrupt(); From 7b37fa721d7ebe9c1b653e9d6d50f0f8eea5b7b8 Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Wed, 15 Feb 2023 16:22:19 +0800 Subject: [PATCH 09/21] Optimize --- .../apache/uniffle/coordinator/SimpleClusterManager.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java index dcd4ef1f3e..eeeff8c3e4 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/SimpleClusterManager.java @@ -312,12 +312,6 @@ public void close() throws IOException { } } - private ShuffleServerInternalGrpcClient createShuffleServerClient(ServerNode serverNode) { - synchronized (serverNode) { - return new ShuffleServerInternalGrpcClient(serverNode.getIp(), serverNode.getPort()); - } - } - @VisibleForTesting public void setStartTime(long startTime) { this.startTime = startTime; From ea37b61e80eb8a9ecde3f9da192068e067a8c7fc Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Sat, 4 Mar 2023 16:36:27 +0800 Subject: [PATCH 10/21] Fix UT --- .../test/java/org/apache/uniffle/common/ServerStatusTest.java | 2 +- .../test/java/org/apache/uniffle/common/rpc/StatusCodeTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/common/src/test/java/org/apache/uniffle/common/ServerStatusTest.java b/common/src/test/java/org/apache/uniffle/common/ServerStatusTest.java index e449215aad..2546cfa6d1 100644 --- a/common/src/test/java/org/apache/uniffle/common/ServerStatusTest.java +++ b/common/src/test/java/org/apache/uniffle/common/ServerStatusTest.java @@ -55,7 +55,7 @@ public void test() throws Exception { fail(e.getMessage()); } } - for (int i = 0; i < serverStatuses.size() - 1; i++) { + for (int i = 0; i < serverStatuses.size(); i++) { assertEquals(protoServerStatuses.get(i), serverStatuses.get(i).toProto()); assertEquals(ServerStatus.fromProto(protoServerStatuses.get(i)), serverStatuses.get(i)); } diff --git a/common/src/test/java/org/apache/uniffle/common/rpc/StatusCodeTest.java b/common/src/test/java/org/apache/uniffle/common/rpc/StatusCodeTest.java index 5956c1d34d..4e9c5b2f82 100644 --- a/common/src/test/java/org/apache/uniffle/common/rpc/StatusCodeTest.java +++ b/common/src/test/java/org/apache/uniffle/common/rpc/StatusCodeTest.java @@ -55,7 +55,7 @@ public void test() throws Exception { fail(e.getMessage()); } } - for (int i = 0; i < statusCodes.size() - 1; i++) { + for (int i = 0; i < statusCodes.size(); i++) { assertEquals(protoStatusCode.get(i), statusCodes.get(i).toProto()); assertEquals(StatusCode.fromProto(protoStatusCode.get(i)), statusCodes.get(i)); } From bc5f98883440b6121e8d14c043248865db46b32f Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Sat, 4 Mar 2023 18:59:28 +0800 Subject: [PATCH 11/21] test --- integration-test/common/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/integration-test/common/pom.xml b/integration-test/common/pom.xml index 37b7eb6e85..5ce511d640 100644 --- a/integration-test/common/pom.xml +++ b/integration-test/common/pom.xml @@ -86,6 +86,7 @@ com.google.guava guava + 31.0.1-jre test From d4a3fc65b567fa2801f618cf03e31baaabebde57 Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Sat, 4 Mar 2023 19:12:42 +0800 Subject: [PATCH 12/21] test --- coordinator/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/coordinator/pom.xml b/coordinator/pom.xml index ac6b2ec5e8..75f69f33d1 100644 --- a/coordinator/pom.xml +++ b/coordinator/pom.xml @@ -80,6 +80,10 @@ org.mockito mockito-inline + + com.google.guava + guava + From c4a5e2a7df78cf5be6725188795406cdc06a1e6f Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Sat, 4 Mar 2023 19:27:30 +0800 Subject: [PATCH 13/21] Revert "test" This reverts commit d4a3fc65b567fa2801f618cf03e31baaabebde57. --- coordinator/pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/coordinator/pom.xml b/coordinator/pom.xml index 75f69f33d1..ac6b2ec5e8 100644 --- a/coordinator/pom.xml +++ b/coordinator/pom.xml @@ -80,10 +80,6 @@ org.mockito mockito-inline - - com.google.guava - guava - From 5f6b66b3cfcf071b975b1d85ae3f4a0a45564621 Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Sat, 4 Mar 2023 19:27:44 +0800 Subject: [PATCH 14/21] Revert "test" This reverts commit bc5f98883440b6121e8d14c043248865db46b32f. --- integration-test/common/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/integration-test/common/pom.xml b/integration-test/common/pom.xml index 5ce511d640..37b7eb6e85 100644 --- a/integration-test/common/pom.xml +++ b/integration-test/common/pom.xml @@ -86,7 +86,6 @@ com.google.guava guava - 31.0.1-jre test From 840b7409fcc94fe17822fc0833cfc27ab7bc2f98 Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Sat, 4 Mar 2023 19:29:45 +0800 Subject: [PATCH 15/21] test --- coordinator/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/coordinator/pom.xml b/coordinator/pom.xml index ac6b2ec5e8..df147d3ffc 100644 --- a/coordinator/pom.xml +++ b/coordinator/pom.xml @@ -107,6 +107,7 @@ com.google.protobuf:protobuf-java-util com.google.guava:guava + com.google.guava:failureaccess com.fasterxml.jackson.core:jackson-databind com.fasterxml.jackson.core:jackson-core From c8e5ae038b32d6e2bde4ef8f8fd282c600932a85 Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Sat, 4 Mar 2023 20:49:13 +0800 Subject: [PATCH 16/21] testOptimize --- .../request/CancelDecommissionRequest.java | 15 ++++++++++ .../web/request/DecommissionRequest.java | 16 +++++++++++ .../coordinator/web/servlet/BaseServlet.java | 9 +++--- .../servlet/CancelDecommissionServlet.java | 18 ++++++------ .../web/servlet/DecommissionServlet.java | 18 ++++++------ .../coordinator/web/servlet/NodesServlet.java | 20 +++++++------ .../org/apache/uniffle/test/ServletTest.java | 28 +++++++++++++------ 7 files changed, 86 insertions(+), 38 deletions(-) create mode 100644 coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java create mode 100644 coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java new file mode 100644 index 0000000000..55d37a7b47 --- /dev/null +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java @@ -0,0 +1,15 @@ +package org.apache.uniffle.coordinator.web.request; + +import java.util.List; + +public class CancelDecommissionRequest { + private List serverIds; + + public List getServerIds() { + return serverIds; + } + + public void setServerIds(List serverIds) { + this.serverIds = serverIds; + } +} diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java new file mode 100644 index 0000000000..bc7db7727b --- /dev/null +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java @@ -0,0 +1,16 @@ +package org.apache.uniffle.coordinator.web.request; + +import java.util.List; + +public class DecommissionRequest { + private List serverIds; + + public List getServerIds() { + return serverIds; + } + + public void setServerIds(List serverIds) { + this.serverIds = serverIds; + } + +} diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java index 2b44bcede8..2a30233421 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.io.OutputStream; -import java.util.Map; import java.util.concurrent.Callable; import javax.servlet.ServletException; import javax.servlet.http.HttpServlet; @@ -36,12 +35,12 @@ public abstract class BaseServlet extends HttpServlet { final ObjectMapper mapper = new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL); @Override - protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws IOException { writeJSON(resp, handlerRequest(() -> handleGet(req, resp))); } @Override - protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws IOException { writeJSON(resp, handlerRequest(() -> handlePost(req, resp))); } @@ -80,7 +79,7 @@ protected void writeJSON(final HttpServletResponse resp, final Object obj) mapper.writeValue(stream, obj); } - protected Map parseParamsFromJson(HttpServletRequest req) throws IOException { - return mapper.readValue(req.getInputStream(), Map.class); + protected T parseParamsFromJson(HttpServletRequest req, Class clazz) throws IOException { + return mapper.readValue(req.getInputStream(), clazz); } } diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/CancelDecommissionServlet.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/CancelDecommissionServlet.java index 49d6393ec7..b7411d4e22 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/CancelDecommissionServlet.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/CancelDecommissionServlet.java @@ -18,16 +18,15 @@ package org.apache.uniffle.coordinator.web.servlet; import java.io.IOException; -import java.util.Map; -import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import com.google.common.base.Preconditions; +import org.apache.commons.collections.CollectionUtils; import org.apache.uniffle.coordinator.ClusterManager; import org.apache.uniffle.coordinator.CoordinatorServer; import org.apache.uniffle.coordinator.web.Response; +import org.apache.uniffle.coordinator.web.request.CancelDecommissionRequest; public class CancelDecommissionServlet extends BaseServlet { private final CoordinatorServer coordinator; @@ -37,12 +36,15 @@ public CancelDecommissionServlet(CoordinatorServer coordinator) { } @Override - protected Response handlePost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - Map params = parseParamsFromJson(req); - String serverId = (String) params.get("serverId"); - Preconditions.checkNotNull(serverId, "Parameter[serverId] should not be null!"); + protected Response handlePost(HttpServletRequest req, HttpServletResponse resp) throws IOException { + CancelDecommissionRequest params = parseParamsFromJson(req, CancelDecommissionRequest.class); + if (CollectionUtils.isEmpty(params.getServerIds())) { + return Response.fail("Parameter[serverIds] should not be null!"); + } ClusterManager clusterManager = coordinator.getClusterManager(); - clusterManager.cancelDecommission(serverId); + params.getServerIds().forEach((serverId) -> { + clusterManager.cancelDecommission(serverId); + }); return Response.success(null); } } diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/DecommissionServlet.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/DecommissionServlet.java index ed4eaf8f93..96f06dd3c2 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/DecommissionServlet.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/DecommissionServlet.java @@ -18,16 +18,15 @@ package org.apache.uniffle.coordinator.web.servlet; import java.io.IOException; -import java.util.Map; -import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import com.google.common.base.Preconditions; +import org.apache.commons.collections.CollectionUtils; import org.apache.uniffle.coordinator.ClusterManager; import org.apache.uniffle.coordinator.CoordinatorServer; import org.apache.uniffle.coordinator.web.Response; +import org.apache.uniffle.coordinator.web.request.DecommissionRequest; public class DecommissionServlet extends BaseServlet { private final CoordinatorServer coordinator; @@ -37,12 +36,15 @@ public DecommissionServlet(CoordinatorServer coordinator) { } @Override - protected Response handlePost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - Map params = parseParamsFromJson(req); - String serverId = (String) params.get("serverId"); - Preconditions.checkNotNull(serverId, "Parameter[serverId] should not be null!"); + protected Response handlePost(HttpServletRequest req, HttpServletResponse resp) throws IOException { + DecommissionRequest params = parseParamsFromJson(req, DecommissionRequest.class); + if (CollectionUtils.isEmpty(params.getServerIds())) { + return Response.fail("Parameter[serverIds] should not be null!"); + } ClusterManager clusterManager = coordinator.getClusterManager(); - clusterManager.decommission(serverId); + params.getServerIds().forEach((serverId) -> { + clusterManager.decommission(serverId); + }); return Response.success(null); } } diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/NodesServlet.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/NodesServlet.java index 1b5e2e2f6c..048502352a 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/NodesServlet.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/NodesServlet.java @@ -17,17 +17,13 @@ package org.apache.uniffle.coordinator.web.servlet; -import java.io.IOException; import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.stream.Collectors; -import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import org.apache.commons.lang3.StringUtils; - import org.apache.uniffle.coordinator.CoordinatorServer; import org.apache.uniffle.coordinator.ServerNode; import org.apache.uniffle.coordinator.web.Response; @@ -41,13 +37,19 @@ public NodesServlet(CoordinatorServer coordinator) { } @Override - protected Response handleGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + protected Response handleGet(HttpServletRequest req, HttpServletResponse resp) { List serverList = coordinator.getClusterManager().getServerList(Collections.EMPTY_SET); String id = req.getParameter("id"); - if (StringUtils.isNotEmpty(id)) { - serverList = serverList.stream().filter((server) -> - id.equals(server.getId())).collect(Collectors.toList()); - } + String status = req.getParameter("status"); + serverList = serverList.stream().filter((server) -> { + if (id != null && !id.equals(server.getId())) { + return false; + } + if (status != null && !server.getStatus().toString().equals(status)) { + return false; + } + return true; + }).collect(Collectors.toList()); Collections.sort(serverList, Comparator.comparing(ServerNode::getId)); return Response.success(serverList); } diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java index e51c2d2480..842b3b24db 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java @@ -20,7 +20,6 @@ import java.io.File; import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.TimeUnit; import com.fasterxml.jackson.core.type.TypeReference; @@ -40,6 +39,8 @@ import org.apache.uniffle.coordinator.CoordinatorConf; import org.apache.uniffle.coordinator.CoordinatorServer; import org.apache.uniffle.coordinator.web.Response; +import org.apache.uniffle.coordinator.web.request.CancelDecommissionRequest; +import org.apache.uniffle.coordinator.web.request.DecommissionRequest; import org.apache.uniffle.server.ShuffleServer; import org.apache.uniffle.server.ShuffleServerConf; import org.apache.uniffle.storage.util.StorageType; @@ -105,20 +106,30 @@ public void testNodesServlet() throws Exception { serverList = response.getData(); assertEquals(1, serverList.size()); assertEquals(shuffleServer.getId(), serverList.get(0).get("id")); + + content = TestUtils.httpGet(NODES_URL + "?status=DECOMMISSIONED"); + response = objectMapper.readValue(content, new TypeReference>>() {}); + serverList = response.getData(); + assertEquals(0, serverList.size()); + content = TestUtils.httpGet(NODES_URL + "?status=ACTIVE"); + response = objectMapper.readValue(content, new TypeReference>>() {}); + serverList = response.getData(); + assertEquals(2, serverList.size()); } @Test public void testDecommissionServlet() throws Exception { ShuffleServer shuffleServer = shuffleServers.get(0); assertEquals(ServerStatus.ACTIVE, shuffleServer.getServerStatus()); - Map params = new HashMap<>(); - params.put("serverId", "not_exist_serverId"); - String content = TestUtils.httpPost(CANCEL_DECOMMISSION_URL, objectMapper.writeValueAsString(params)); + DecommissionRequest decommissionRequest = new DecommissionRequest(); + decommissionRequest.setServerIds(Lists.newArrayList("not_exist_serverId")); + String content = TestUtils.httpPost(CANCEL_DECOMMISSION_URL, objectMapper.writeValueAsString(decommissionRequest)); Response response = objectMapper.readValue(content, Response.class); assertEquals(-1, response.getCode()); assertNotNull(response.getErrMsg()); - params.put("serverId", shuffleServer.getId()); - content = TestUtils.httpPost(CANCEL_DECOMMISSION_URL, objectMapper.writeValueAsString(params)); + CancelDecommissionRequest cancelDecommissionRequest = new CancelDecommissionRequest(); + cancelDecommissionRequest.setServerIds(Lists.newArrayList(shuffleServer.getId())); + content = TestUtils.httpPost(CANCEL_DECOMMISSION_URL, objectMapper.writeValueAsString(cancelDecommissionRequest)); response = objectMapper.readValue(content, Response.class); assertEquals(0, response.getCode()); @@ -126,7 +137,8 @@ public void testDecommissionServlet() throws Exception { ShuffleServerGrpcClient shuffleServerClient = new ShuffleServerGrpcClient(LOCALHOST, SHUFFLE_SERVER_PORT); shuffleServerClient.registerShuffle(new RssRegisterShuffleRequest("testDecommissionServlet_appId", 0, Lists.newArrayList(new PartitionRange(0, 1)), "")); - content = TestUtils.httpPost(DECOMMISSION_URL, objectMapper.writeValueAsString(params)); + decommissionRequest.setServerIds(Lists.newArrayList(shuffleServer.getId())); + content = TestUtils.httpPost(DECOMMISSION_URL, objectMapper.writeValueAsString(decommissionRequest)); response = objectMapper.readValue(content, Response.class); assertEquals(0, response.getCode()); assertEquals(ServerStatus.DECOMMISSIONING, shuffleServer.getServerStatus()); @@ -136,7 +148,7 @@ public void testDecommissionServlet() throws Exception { ServerStatus.DECOMMISSIONING.equals( coordinatorServer.getClusterManager().getServerNodeById(shuffleServer.getId()).getStatus())); // Cancel decommission. - content = TestUtils.httpPost(CANCEL_DECOMMISSION_URL, objectMapper.writeValueAsString(params)); + content = TestUtils.httpPost(CANCEL_DECOMMISSION_URL, objectMapper.writeValueAsString(cancelDecommissionRequest)); response = objectMapper.readValue(content, Response.class); assertEquals(0, response.getCode()); assertEquals(ServerStatus.ACTIVE, shuffleServer.getServerStatus()); From d61b6ce60d4c8f0bad749320d06c42e3719c3558 Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Sat, 4 Mar 2023 20:58:31 +0800 Subject: [PATCH 17/21] Fix license --- .../web/request/CancelDecommissionRequest.java | 17 +++++++++++++++++ .../web/request/DecommissionRequest.java | 17 +++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java index 55d37a7b47..1fe05e9c61 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java @@ -1,3 +1,20 @@ +/* + * 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.uniffle.coordinator.web.request; import java.util.List; diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java index bc7db7727b..3a5aa9185f 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java @@ -1,3 +1,20 @@ +/* + * 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.uniffle.coordinator.web.request; import java.util.List; From 2381092e498b7fa77ec5d4de42e10039f911bdb3 Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Tue, 7 Mar 2023 21:36:29 +0800 Subject: [PATCH 18/21] Optimize --- .../uniffle/common/metrics/TestUtils.java | 34 ++++++++++--------- .../uniffle/server/ShuffleServerTest.java | 6 ++-- 2 files changed, 21 insertions(+), 19 deletions(-) diff --git a/common/src/test/java/org/apache/uniffle/common/metrics/TestUtils.java b/common/src/test/java/org/apache/uniffle/common/metrics/TestUtils.java index 8d8c5f7301..ea623ea5d4 100644 --- a/common/src/test/java/org/apache/uniffle/common/metrics/TestUtils.java +++ b/common/src/test/java/org/apache/uniffle/common/metrics/TestUtils.java @@ -33,14 +33,14 @@ public static String httpGet(String urlString) throws IOException { URL url = new URL(urlString); HttpURLConnection con = (HttpURLConnection) url.openConnection(); con.setRequestMethod("GET"); - BufferedReader in = new BufferedReader( - new InputStreamReader(con.getInputStream())); - String inputLine; - StringBuffer content = new StringBuffer(); - while ((inputLine = in.readLine()) != null) { - content.append(inputLine); + StringBuilder content = new StringBuilder(); + try (BufferedReader in = new BufferedReader( + new InputStreamReader(con.getInputStream()));) { + String inputLine; + while ((inputLine = in.readLine()) != null) { + content.append(inputLine); + } } - in.close(); return content.toString(); } @@ -49,16 +49,18 @@ public static String httpPost(String urlString, String postData) throws IOExcept HttpURLConnection con = (HttpURLConnection) url.openConnection(); con.setDoOutput(true); con.setRequestMethod("POST"); - OutputStream outputStream = con.getOutputStream(); - outputStream.write(postData.getBytes()); - BufferedReader in = new BufferedReader( - new InputStreamReader(con.getInputStream())); - String inputLine; - StringBuffer content = new StringBuffer(); - while ((inputLine = in.readLine()) != null) { - content.append(inputLine); + StringBuilder content = new StringBuilder(); + try (OutputStream outputStream = con.getOutputStream();) { + outputStream.write(postData.getBytes()); + try (BufferedReader in = new BufferedReader( + new InputStreamReader(con.getInputStream()));) { + String inputLine; + while ((inputLine = in.readLine()) != null) { + content.append(inputLine); + } + } } - in.close(); + return content.toString(); } } diff --git a/server/src/test/java/org/apache/uniffle/server/ShuffleServerTest.java b/server/src/test/java/org/apache/uniffle/server/ShuffleServerTest.java index 19c9513e91..6dc5e56eb0 100644 --- a/server/src/test/java/org/apache/uniffle/server/ShuffleServerTest.java +++ b/server/src/test/java/org/apache/uniffle/server/ShuffleServerTest.java @@ -50,7 +50,6 @@ public void startTest() { ShuffleServerConf serverConf = createShuffleServerConf(); ShuffleServer ss1 = new ShuffleServer(serverConf); ss1.start(); - ss1.stopServer(); ExitUtils.disableSystemExit(); ShuffleServer ss2 = new ShuffleServer(serverConf); String expectMessage = "Fail to start jetty http server"; @@ -61,7 +60,6 @@ public void startTest() { assertEquals(expectMessage, e.getMessage()); assertEquals(expectStatus, ((ExitException) e).getStatus()); } - ss2.stopServer(); serverConf.setInteger("rss.jetty.http.port", 9529); ss2 = new ShuffleServer(serverConf); @@ -72,7 +70,7 @@ public void startTest() { assertEquals(expectMessage, e.getMessage()); assertEquals(expectStatus, ((ExitException) e).getStatus()); } - ss2.stopServer(); + ss1.stopServer(); final Thread t = new Thread(null, () -> { throw new AssertionError("TestUncaughtException"); @@ -92,6 +90,8 @@ public void decommissionTest(boolean shutdown) throws Exception { ShuffleServerConf serverConf = createShuffleServerConf(); serverConf.set(SERVER_DECOMMISSION_CHECK_INTERVAL, 1000L); serverConf.set(SERVER_DECOMMISSION_SHUTDOWN, shutdown); + serverConf.set(ShuffleServerConf.RPC_SERVER_PORT, 19527); + serverConf.set(ShuffleServerConf.JETTY_HTTP_PORT, 19528); ShuffleServer shuffleServer = new ShuffleServer(serverConf); shuffleServer.start(); assertEquals(ServerStatus.ACTIVE, shuffleServer.getServerStatus()); From c64aaa74b9ebf288932744a2afd3e68ae849579a Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Wed, 8 Mar 2023 21:37:12 +0800 Subject: [PATCH 19/21] Add docs --- .../coordinator/CoordinatorServer.java | 4 +- docs/coordinator_guide.md | 58 +++++++++++++++++++ 2 files changed, 60 insertions(+), 2 deletions(-) diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java index 836678f3dc..5b453d68e5 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorServer.java @@ -192,8 +192,8 @@ private void registerRESTAPI() throws Exception { new DecommissionServlet(this), "/api/server/decommission"); jettyServer.addServlet( - new CancelDecommissionServlet(this), - "/api/server/cancelDecommission"); + new CancelDecommissionServlet(this), + "/api/server/cancelDecommission"); } private void registerMetrics() throws Exception { diff --git a/docs/coordinator_guide.md b/docs/coordinator_guide.md index d53a5b37c4..06768d8384 100644 --- a/docs/coordinator_guide.md +++ b/docs/coordinator_guide.md @@ -130,3 +130,61 @@ PrometheusPushGatewayMetricReporter is one of the built-in metrics reporter, whi |rss.metrics.prometheus.pushgateway.groupingkey|-| Specifies the grouping key which is the group and global labels of all metrics. The label name and value are separated by '=', and labels are separated by ';', e.g., k1=v1;k2=v2. Please ensure that your grouping key meets the [Prometheus requirements](https://prometheus.io/docs/concepts/data_model/#metric-names-and-labels). | |rss.metrics.prometheus.pushgateway.jobname|-| The job name under which metrics will be pushed. | |rss.metrics.prometheus.pushgateway.report.interval.seconds|10| The interval in seconds for the reporter to report metrics. | + +## RESTful API + +### Fetch Shuffle servers + +
+ GET /api/server/nodes + +##### Parameters + +> |name|type|data type|description| +> |----|----|---------|-----------| +> |id|required|string|shuffle server id, eg:127.0.0.1:19999| +> |status|optional|string|Shuffle server status, eg:ACTIVE, DECOMMISSIONING, DECOMMISSIONED| + +##### Example cURL + +> ```shell +> curl -X GET http://localhost:19998/api/server/nodes +> ``` +
+ +### Decommission shuffle servers + +
+ POST /api/server/decommission + +##### Parameters + +> |name|type| data type |description| +> |----|-------------------|---------|-----------| +> |serverIds|required| array |Shuffle server array, eg:["127.0.0.1:19999"]| +> +##### Example cURL + +> ```javascript +> curl -X POST -H "Content-Type: application/json" http://localhost:19998/api/server/decommission -d '{"serverIds:": ["127.0.0.1:19999"]}' +> ``` +
+ + +### Cancel decommission shuffle servers + +
+ POST /api/server/cancelDecommission + +##### Parameters + +> |name|type| data type |description| +> |----|-------------------|---------|-----------| +> |serverIds|required| array |Shuffle server array, eg:["127.0.0.1:19999"]| +> +##### Example cURL + +> ```javascript +> curl -X POST -H "Content-Type: application/json" http://localhost:19998/api/server/cancelDecommission -d '{"serverIds:": ["127.0.0.1:19999"]}' +> ``` +
\ No newline at end of file From f0e1ef509fc969e030cae61f38694e91ff099d4a Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Wed, 8 Mar 2023 21:45:31 +0800 Subject: [PATCH 20/21] Add docsOptimize --- docs/coordinator_guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/coordinator_guide.md b/docs/coordinator_guide.md index 06768d8384..26d4f1c5cb 100644 --- a/docs/coordinator_guide.md +++ b/docs/coordinator_guide.md @@ -147,7 +147,7 @@ PrometheusPushGatewayMetricReporter is one of the built-in metrics reporter, whi ##### Example cURL -> ```shell +> ```javascript > curl -X GET http://localhost:19998/api/server/nodes > ``` From fddc0531d786cd1dda69ca363b1a30912cf9539f Mon Sep 17 00:00:00 2001 From: xianjingfeng <583872483@qq.com> Date: Thu, 9 Mar 2023 10:22:52 +0800 Subject: [PATCH 21/21] Optimize --- .../web/request/CancelDecommissionRequest.java | 8 ++++---- .../coordinator/web/request/DecommissionRequest.java | 9 ++++----- .../uniffle/coordinator/web/servlet/BaseServlet.java | 1 - docs/coordinator_guide.md | 8 ++++---- .../test/java/org/apache/uniffle/test/ServletTest.java | 7 ++++--- 5 files changed, 16 insertions(+), 17 deletions(-) diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java index 1fe05e9c61..997a135ea4 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java @@ -17,16 +17,16 @@ package org.apache.uniffle.coordinator.web.request; -import java.util.List; +import java.util.Set; public class CancelDecommissionRequest { - private List serverIds; + private Set serverIds; - public List getServerIds() { + public Set getServerIds() { return serverIds; } - public void setServerIds(List serverIds) { + public void setServerIds(Set serverIds) { this.serverIds = serverIds; } } diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java index 3a5aa9185f..c11a716c43 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java @@ -17,17 +17,16 @@ package org.apache.uniffle.coordinator.web.request; -import java.util.List; +import java.util.Set; public class DecommissionRequest { - private List serverIds; + private Set serverIds; - public List getServerIds() { + public Set getServerIds() { return serverIds; } - public void setServerIds(List serverIds) { + public void setServerIds(Set serverIds) { this.serverIds = serverIds; } - } diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java index 2a30233421..a67701f224 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/web/servlet/BaseServlet.java @@ -62,7 +62,6 @@ protected Response handleGet( throw new IOException("Method not support!"); } - protected Response handlePost( HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { diff --git a/docs/coordinator_guide.md b/docs/coordinator_guide.md index 26d4f1c5cb..cb511b82b0 100644 --- a/docs/coordinator_guide.md +++ b/docs/coordinator_guide.md @@ -131,7 +131,7 @@ PrometheusPushGatewayMetricReporter is one of the built-in metrics reporter, whi |rss.metrics.prometheus.pushgateway.jobname|-| The job name under which metrics will be pushed. | |rss.metrics.prometheus.pushgateway.report.interval.seconds|10| The interval in seconds for the reporter to report metrics. | -## RESTful API +## RESTful API(beta) ### Fetch Shuffle servers @@ -147,7 +147,7 @@ PrometheusPushGatewayMetricReporter is one of the built-in metrics reporter, whi ##### Example cURL -> ```javascript +> ```bash > curl -X GET http://localhost:19998/api/server/nodes > ``` @@ -165,7 +165,7 @@ PrometheusPushGatewayMetricReporter is one of the built-in metrics reporter, whi > ##### Example cURL -> ```javascript +> ```bash > curl -X POST -H "Content-Type: application/json" http://localhost:19998/api/server/decommission -d '{"serverIds:": ["127.0.0.1:19999"]}' > ``` @@ -184,7 +184,7 @@ PrometheusPushGatewayMetricReporter is one of the built-in metrics reporter, whi > ##### Example cURL -> ```javascript +> ```bash > curl -X POST -H "Content-Type: application/json" http://localhost:19998/api/server/cancelDecommission -d '{"serverIds:": ["127.0.0.1:19999"]}' > ``` \ No newline at end of file diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java index 842b3b24db..1963195e24 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ServletTest.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import org.awaitility.Awaitility; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -122,13 +123,13 @@ public void testDecommissionServlet() throws Exception { ShuffleServer shuffleServer = shuffleServers.get(0); assertEquals(ServerStatus.ACTIVE, shuffleServer.getServerStatus()); DecommissionRequest decommissionRequest = new DecommissionRequest(); - decommissionRequest.setServerIds(Lists.newArrayList("not_exist_serverId")); + decommissionRequest.setServerIds(Sets.newHashSet("not_exist_serverId")); String content = TestUtils.httpPost(CANCEL_DECOMMISSION_URL, objectMapper.writeValueAsString(decommissionRequest)); Response response = objectMapper.readValue(content, Response.class); assertEquals(-1, response.getCode()); assertNotNull(response.getErrMsg()); CancelDecommissionRequest cancelDecommissionRequest = new CancelDecommissionRequest(); - cancelDecommissionRequest.setServerIds(Lists.newArrayList(shuffleServer.getId())); + cancelDecommissionRequest.setServerIds(Sets.newHashSet(shuffleServer.getId())); content = TestUtils.httpPost(CANCEL_DECOMMISSION_URL, objectMapper.writeValueAsString(cancelDecommissionRequest)); response = objectMapper.readValue(content, Response.class); assertEquals(0, response.getCode()); @@ -137,7 +138,7 @@ public void testDecommissionServlet() throws Exception { ShuffleServerGrpcClient shuffleServerClient = new ShuffleServerGrpcClient(LOCALHOST, SHUFFLE_SERVER_PORT); shuffleServerClient.registerShuffle(new RssRegisterShuffleRequest("testDecommissionServlet_appId", 0, Lists.newArrayList(new PartitionRange(0, 1)), "")); - decommissionRequest.setServerIds(Lists.newArrayList(shuffleServer.getId())); + decommissionRequest.setServerIds(Sets.newHashSet(shuffleServer.getId())); content = TestUtils.httpPost(DECOMMISSION_URL, objectMapper.writeValueAsString(decommissionRequest)); response = objectMapper.readValue(content, Response.class); assertEquals(0, response.getCode());