From 84285d3eadca2f734157d2970fc2c5cce02331e1 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Fri, 12 Jan 2024 19:00:53 -0800 Subject: [PATCH 01/76] RATIS-1925. Support Zero-Copy in GrpcClientProtocolService (#1007) --- .../ratis/protocol/RaftClientRequest.java | 10 +- .../ratis/util/ReferenceCountedObject.java | 21 +++ .../java/org/apache/ratis/grpc/GrpcUtil.java | 26 ++++ .../ratis/grpc/metrics/ZeroCopyMetrics.java | 58 ++++++++ .../server/GrpcClientProtocolService.java | 134 +++++++++++------- .../apache/ratis/grpc/server/GrpcService.java | 8 +- .../grpc/util/ZeroCopyMessageMarshaller.java | 8 +- .../ratis/server/impl/LeaderStateImpl.java | 12 +- .../server/impl/MessageStreamRequests.java | 45 ++++-- .../ratis/server/impl/RaftServerImpl.java | 9 +- .../impl/SimpleStateMachine4Testing.java | 2 +- .../grpc/util/GrpcZeroCopyTestServer.java | 4 +- 12 files changed, 258 insertions(+), 79 deletions(-) create mode 100644 ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java index ed41f1ea2c..18c157130b 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java @@ -474,7 +474,13 @@ public long getTimeoutMs() { @Override public String toString() { - return super.toString() + ", seq=" + ProtoUtils.toString(slidingWindowEntry) + ", " - + type + ", " + getMessage(); + return toStringShort() + ", " + getMessage(); + } + + /** + * @return a short string which does not include {@link #message}. + */ + public String toStringShort() { + return super.toString() + ", seq=" + ProtoUtils.toString(slidingWindowEntry) + ", " + type; } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java index 3f72f5ffe8..815b90dbc1 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java @@ -19,6 +19,7 @@ import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; +import java.util.Collection; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -101,6 +102,26 @@ static ReferenceCountedObject wrap(V value) { return wrap(value, () -> {}, ignored -> {}); } + static ReferenceCountedObject delegateFrom(Collection> fromRefs, V value) { + return new ReferenceCountedObject() { + @Override + public V get() { + return value; + } + + @Override + public V retain() { + fromRefs.forEach(ReferenceCountedObject::retain); + return value; + } + + @Override + public boolean release() { + return fromRefs.stream().map(ReferenceCountedObject::release).allMatch(r -> r); + } + }; + } + /** * @return a {@link ReferenceCountedObject} of the given value by delegating to this object. */ diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java index 22653b6efb..0baefa2d3f 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java @@ -24,8 +24,12 @@ import org.apache.ratis.security.TlsConf.CertificatesConf; import org.apache.ratis.security.TlsConf.PrivateKeyConf; import org.apache.ratis.security.TlsConf.KeyManagerConf; +import org.apache.ratis.thirdparty.com.google.protobuf.MessageLite; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.Metadata; +import org.apache.ratis.thirdparty.io.grpc.MethodDescriptor; +import org.apache.ratis.thirdparty.io.grpc.ServerCallHandler; +import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; @@ -304,4 +308,26 @@ static void setKeyManager(SslContextBuilder b, KeyManagerConf keyManagerConfig) b.keyManager(privateKey.get(), certificates.get()); } } + + /** + * Used to add a method to Service definition with a custom request marshaller. + * + * @param orig original service definition. + * @param newServiceBuilder builder of the new service definition. + * @param origMethod the original method definition. + * @param customMarshaller custom marshaller to be set for the method. + * @param + * @param + */ + static void addMethodWithCustomMarshaller( + ServerServiceDefinition orig, ServerServiceDefinition.Builder newServiceBuilder, + MethodDescriptor origMethod, MethodDescriptor.PrototypeMarshaller customMarshaller) { + MethodDescriptor newMethod = origMethod.toBuilder() + .setRequestMarshaller(customMarshaller) + .build(); + @SuppressWarnings("unchecked") + ServerCallHandler serverCallHandler = + (ServerCallHandler) orig.getMethod(newMethod.getFullMethodName()).getServerCallHandler(); + newServiceBuilder.addMethod(newMethod, serverCallHandler); + } } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java new file mode 100644 index 0000000000..20da4ee634 --- /dev/null +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java @@ -0,0 +1,58 @@ +/** + * 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.ratis.grpc.metrics; + +import org.apache.ratis.metrics.LongCounter; +import org.apache.ratis.metrics.MetricRegistryInfo; +import org.apache.ratis.metrics.RatisMetricRegistry; +import org.apache.ratis.metrics.RatisMetrics; +import org.apache.ratis.thirdparty.com.google.protobuf.AbstractMessage; + +public class ZeroCopyMetrics extends RatisMetrics { + private static final String RATIS_GRPC_METRICS_APP_NAME = "ratis_grpc"; + private static final String RATIS_GRPC_METRICS_COMP_NAME = "zero_copy"; + private static final String RATIS_GRPC_METRICS_DESC = "Metrics for Ratis Grpc Zero copy"; + + private final LongCounter zeroCopyMessages = getRegistry().counter("num_zero_copy_messages"); + private final LongCounter nonZeroCopyMessages = getRegistry().counter("num_non_zero_copy_messages"); + private final LongCounter releasedMessages = getRegistry().counter("num_released_messages"); + + public ZeroCopyMetrics() { + super(createRegistry()); + } + + private static RatisMetricRegistry createRegistry() { + return create(new MetricRegistryInfo("", + RATIS_GRPC_METRICS_APP_NAME, + RATIS_GRPC_METRICS_COMP_NAME, RATIS_GRPC_METRICS_DESC)); + } + + + public void onZeroCopyMessage(AbstractMessage ignored) { + zeroCopyMessages.inc(); + } + + public void onNonZeroCopyMessage(AbstractMessage ignored) { + nonZeroCopyMessages.inc(); + } + + public void onReleasedMessage(AbstractMessage ignored) { + releasedMessages.inc(); + } + +} \ No newline at end of file diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index 9c19684677..e8de4def04 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -19,10 +19,13 @@ import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.grpc.GrpcUtil; +import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; +import org.apache.ratis.grpc.util.ZeroCopyMessageMarshaller; import org.apache.ratis.protocol.*; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; import org.apache.ratis.protocol.exceptions.GroupMismatchException; import org.apache.ratis.protocol.exceptions.RaftException; +import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; @@ -30,16 +33,15 @@ import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SlidingWindow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.HashMap; import java.util.Iterator; import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; @@ -48,15 +50,21 @@ import java.util.function.Consumer; import java.util.function.Supplier; +import static org.apache.ratis.grpc.GrpcUtil.addMethodWithCustomMarshaller; +import static org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.getOrderedMethod; +import static org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.getUnorderedMethod; + class GrpcClientProtocolService extends RaftClientProtocolServiceImplBase { private static final Logger LOG = LoggerFactory.getLogger(GrpcClientProtocolService.class); private static class PendingOrderedRequest implements SlidingWindow.ServerSideRequest { + private final ReferenceCountedObject requestRef; private final RaftClientRequest request; private final AtomicReference reply = new AtomicReference<>(); - PendingOrderedRequest(RaftClientRequest request) { - this.request = request; + PendingOrderedRequest(ReferenceCountedObject requestRef) { + this.requestRef = requestRef; + this.request = requestRef != null ? requestRef.get() : null; } @Override @@ -76,15 +84,16 @@ public boolean hasReply() { @Override public void setReply(RaftClientReply r) { final boolean set = reply.compareAndSet(null, r); - Preconditions.assertTrue(set, () -> "Reply is already set: request=" + request + ", reply=" + reply); + Preconditions.assertTrue(set, () -> "Reply is already set: request=" + + request.toStringShort() + ", reply=" + reply); } RaftClientReply getReply() { return reply.get(); } - RaftClientRequest getRequest() { - return request; + ReferenceCountedObject getRequestRef() { + return requestRef; } @Override @@ -135,18 +144,31 @@ void closeAllExisting(RaftGroupId groupId) { private final ExecutorService executor; private final OrderedStreamObservers orderedStreamObservers = new OrderedStreamObservers(); + private final ZeroCopyMessageMarshaller zeroCopyRequestMarshaller; GrpcClientProtocolService(Supplier idSupplier, RaftClientAsynchronousProtocol protocol, - ExecutorService executor) { + ExecutorService executor, ZeroCopyMetrics zeroCopyMetrics) { this.idSupplier = idSupplier; this.protocol = protocol; this.executor = executor; + this.zeroCopyRequestMarshaller = new ZeroCopyMessageMarshaller<>(RaftClientRequestProto.getDefaultInstance(), + zeroCopyMetrics::onZeroCopyMessage, zeroCopyMetrics::onNonZeroCopyMessage, zeroCopyMetrics::onReleasedMessage); } RaftPeerId getId() { return idSupplier.get(); } + ServerServiceDefinition bindServiceWithZeroCopy() { + ServerServiceDefinition orig = super.bindService(); + ServerServiceDefinition.Builder builder = ServerServiceDefinition.builder(orig.getServiceDescriptor().getName()); + + addMethodWithCustomMarshaller(orig, builder, getOrderedMethod(), zeroCopyRequestMarshaller); + addMethodWithCustomMarshaller(orig, builder, getUnorderedMethod(), zeroCopyRequestMarshaller); + + return builder.build(); + } + @Override public StreamObserver ordered(StreamObserver responseObserver) { final OrderedRequestStreamObserver so = new OrderedRequestStreamObserver(responseObserver); @@ -220,31 +242,38 @@ boolean isClosed() { return isClosed.get(); } - CompletableFuture processClientRequest(RaftClientRequest request, Consumer replyHandler) { - try { - final String errMsg = LOG.isDebugEnabled() ? "processClientRequest for " + request : ""; - return protocol.submitClientRequestAsync(request - ).thenAcceptAsync(replyHandler, executor - ).exceptionally(exception -> { - // TODO: the exception may be from either raft or state machine. - // Currently we skip all the following responses when getting an - // exception from the state machine. - responseError(exception, () -> errMsg); - return null; - }); - } catch (IOException e) { - throw new CompletionException("Failed processClientRequest for " + request + " in " + name, e); - } + CompletableFuture processClientRequest(ReferenceCountedObject requestRef, + Consumer replyHandler) { + final String errMsg = LOG.isDebugEnabled() ? "processClientRequest for " + requestRef.get() : ""; + return protocol.submitClientRequestAsync(requestRef + ).thenAcceptAsync(replyHandler, executor + ).exceptionally(exception -> { + // TODO: the exception may be from either raft or state machine. + // Currently we skip all the following responses when getting an + // exception from the state machine. + responseError(exception, () -> errMsg); + return null; + }); } - abstract void processClientRequest(RaftClientRequest request); + abstract void processClientRequest(ReferenceCountedObject requestRef); @Override public void onNext(RaftClientRequestProto request) { + ReferenceCountedObject requestRef = null; try { final RaftClientRequest r = ClientProtoUtils.toRaftClientRequest(request); - processClientRequest(r); + requestRef = ReferenceCountedObject.wrap(r, () -> {}, released -> { + if (released) { + zeroCopyRequestMarshaller.release(request); + } + }); + + processClientRequest(requestRef); } catch (Exception e) { + if (requestRef == null) { + zeroCopyRequestMarshaller.release(request); + } responseError(e, () -> "onNext for " + ClientProtoUtils.toString(request) + " in " + name); } } @@ -278,15 +307,18 @@ private class UnorderedRequestStreamObserver extends RequestStreamObserver { } @Override - void processClientRequest(RaftClientRequest request) { - final CompletableFuture f = processClientRequest(request, reply -> { + void processClientRequest(ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.retain(); + final long callId = request.getCallId(); + + final CompletableFuture f = processClientRequest(requestRef, reply -> { if (!reply.isSuccess()) { - LOG.info("Failed " + request + ", reply=" + reply); + LOG.info("Failed {}, reply={}", request, reply); } final RaftClientReplyProto proto = ClientProtoUtils.toRaftClientReplyProto(reply); responseNext(proto); - }); - final long callId = request.getCallId(); + }).whenComplete((r, e) -> requestRef.release()); + put(callId, f); f.thenAccept(dummy -> remove(callId)); } @@ -329,31 +361,35 @@ RaftGroupId getGroupId() { void processClientRequest(PendingOrderedRequest pending) { final long seq = pending.getSeqNum(); - processClientRequest(pending.getRequest(), + processClientRequest(pending.getRequestRef(), reply -> slidingWindow.receiveReply(seq, reply, this::sendReply)); } @Override - void processClientRequest(RaftClientRequest r) { - if (isClosed()) { - final AlreadyClosedException exception = new AlreadyClosedException(getName() + ": the stream is closed"); - responseError(exception, () -> "processClientRequest (stream already closed) for " + r); - } + void processClientRequest(ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.retain(); + try { + if (isClosed()) { + final AlreadyClosedException exception = new AlreadyClosedException(getName() + ": the stream is closed"); + responseError(exception, () -> "processClientRequest (stream already closed) for " + request); + } - final RaftGroupId requestGroupId = r.getRaftGroupId(); - // use the group id in the first request as the group id of this observer - final RaftGroupId updated = groupId.updateAndGet(g -> g != null ? g: requestGroupId); - final PendingOrderedRequest pending = new PendingOrderedRequest(r); - - if (!requestGroupId.equals(updated)) { - final GroupMismatchException exception = new GroupMismatchException(getId() - + ": The group (" + requestGroupId + ") of " + r.getClientId() - + " does not match the group (" + updated + ") of the " + JavaUtils.getClassSimpleName(getClass())); - responseError(exception, () -> "processClientRequest (Group mismatched) for " + r); - return; + final RaftGroupId requestGroupId = request.getRaftGroupId(); + // use the group id in the first request as the group id of this observer + final RaftGroupId updated = groupId.updateAndGet(g -> g != null ? g : requestGroupId); + final PendingOrderedRequest pending = new PendingOrderedRequest(requestRef); + + if (!requestGroupId.equals(updated)) { + final GroupMismatchException exception = new GroupMismatchException(getId() + + ": The group (" + requestGroupId + ") of " + request.getClientId() + + " does not match the group (" + updated + ") of the " + JavaUtils.getClassSimpleName(getClass())); + responseError(exception, () -> "processClientRequest (Group mismatched) for " + request); + return; + } + slidingWindow.receivedRequest(pending, this::processClientRequest); + } finally { + requestRef.release(); } - - slidingWindow.receivedRequest(pending, this::processClientRequest); } private void sendReply(PendingOrderedRequest ready) { diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java index 097900a0fb..d89afd565e 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -21,6 +21,7 @@ import org.apache.ratis.grpc.GrpcConfigKeys; import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.grpc.GrpcUtil; +import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; import org.apache.ratis.grpc.metrics.intercept.server.MetricServerInterceptor; import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftPeerId; @@ -153,6 +154,7 @@ public static Builder newBuilder() { private final GrpcClientProtocolService clientProtocolService; private final MetricServerInterceptor serverInterceptor; + private final ZeroCopyMetrics zeroCopyMetrics; public MetricServerInterceptor getServerInterceptor() { return serverInterceptor; @@ -199,7 +201,8 @@ private GrpcService(RaftServer raftServer, Supplier idSupplier, GrpcConfigKeys.Server.asyncRequestThreadPoolCached(properties), GrpcConfigKeys.Server.asyncRequestThreadPoolSize(properties), getId() + "-request-"); - this.clientProtocolService = new GrpcClientProtocolService(idSupplier, raftServer, executor); + this.zeroCopyMetrics = new ZeroCopyMetrics(); + this.clientProtocolService = new GrpcClientProtocolService(idSupplier, raftServer, executor, zeroCopyMetrics); this.serverInterceptor = new MetricServerInterceptor( idSupplier, @@ -252,7 +255,8 @@ private MemoizedSupplier newAddressSupplier(int port, Server } private void addClientService(NettyServerBuilder builder) { - builder.addService(ServerInterceptors.intercept(clientProtocolService, serverInterceptor)); + builder.addService(ServerInterceptors.intercept(clientProtocolService.bindServiceWithZeroCopy(), + serverInterceptor)); } private void addAdminService(RaftServer raftServer, NettyServerBuilder nettyServerBuilder) { diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java index bb8183a247..057550c137 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java @@ -62,12 +62,14 @@ public class ZeroCopyMessageMarshaller implements Prototy private final Consumer zeroCopyCount; private final Consumer nonZeroCopyCount; + private final Consumer releasedCount; public ZeroCopyMessageMarshaller(T defaultInstance) { - this(defaultInstance, m -> {}, m -> {}); + this(defaultInstance, m -> {}, m -> {}, m -> {}); } - public ZeroCopyMessageMarshaller(T defaultInstance, Consumer zeroCopyCount, Consumer nonZeroCopyCount) { + public ZeroCopyMessageMarshaller(T defaultInstance, Consumer zeroCopyCount, Consumer nonZeroCopyCount, + Consumer releasedCount) { this.name = JavaUtils.getClassSimpleName(defaultInstance.getClass()) + "-Marshaller"; @SuppressWarnings("unchecked") final Parser p = (Parser) defaultInstance.getParserForType(); @@ -76,6 +78,7 @@ public ZeroCopyMessageMarshaller(T defaultInstance, Consumer zeroCopyCount, C this.zeroCopyCount = zeroCopyCount; this.nonZeroCopyCount = nonZeroCopyCount; + this.releasedCount = releasedCount; } @Override @@ -124,6 +127,7 @@ public void release(T message) { } try { stream.close(); + releasedCount.accept(message); } catch (IOException e) { LOG.error(name + ": Failed to close stream.", e); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index b2788918d5..8864c220c1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -56,6 +56,7 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; @@ -534,15 +535,16 @@ PendingRequest addPendingRequest(PendingRequests.Permit permit, RaftClientReques return pendingRequests.add(permit, request, entry); } - CompletableFuture streamAsync(RaftClientRequest request) { - return messageStreamRequests.streamAsync(request) + CompletableFuture streamAsync(ReferenceCountedObject requestRef) { + RaftClientRequest request = requestRef.get(); + return messageStreamRequests.streamAsync(requestRef) .thenApply(dummy -> server.newSuccessReply(request)) .exceptionally(e -> exception2RaftClientReply(request, e)); } - CompletableFuture streamEndOfRequestAsync(RaftClientRequest request) { - return messageStreamRequests.streamEndOfRequestAsync(request) - .thenApply(bytes -> RaftClientRequest.toWriteRequest(request, Message.valueOf(bytes))); + CompletableFuture> streamEndOfRequestAsync( + ReferenceCountedObject requestRef) { + return messageStreamRequests.streamEndOfRequestAsync(requestRef); } CompletableFuture addWatchRequest(RaftClientRequest request) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/MessageStreamRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/MessageStreamRequests.java index ac81b348bb..c00c57b364 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/MessageStreamRequests.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/MessageStreamRequests.java @@ -25,12 +25,15 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; class MessageStreamRequests { public static final Logger LOG = LoggerFactory.getLogger(MessageStreamRequests.class); @@ -39,12 +42,14 @@ private static class PendingStream { private final ClientInvocationId key; private long nextId = -1; private ByteString bytes = ByteString.EMPTY; + private final List> pendingRefs = new LinkedList<>(); PendingStream(ClientInvocationId key) { this.key = key; } - synchronized CompletableFuture append(long messageId, Message message) { + synchronized CompletableFuture append(long messageId, + ReferenceCountedObject requestRef) { if (nextId == -1) { nextId = messageId; } else if (messageId != nextId) { @@ -52,27 +57,38 @@ synchronized CompletableFuture append(long messageId, Message messag "Unexpected message id in " + key + ": messageId = " + messageId + " != nextId = " + nextId)); } nextId++; + final Message message = requestRef.retain().getMessage(); + pendingRefs.add(requestRef); bytes = bytes.concat(message.getContent()); return CompletableFuture.completedFuture(bytes); } - synchronized CompletableFuture getBytes(long messageId, Message message) { - return append(messageId, message); + synchronized CompletableFuture> getWriteRequest(long messageId, + ReferenceCountedObject requestRef) { + return append(messageId, requestRef) + .thenApply(appended -> RaftClientRequest.toWriteRequest(requestRef.get(), () -> appended)) + .thenApply(request -> ReferenceCountedObject.delegateFrom(pendingRefs, request)); + } + + synchronized void clear() { + pendingRefs.forEach(ReferenceCountedObject::release); + pendingRefs.clear(); } } static class StreamMap { - private final ConcurrentMap map = new ConcurrentHashMap<>(); + private final Map map = new HashMap<>(); - PendingStream computeIfAbsent(ClientInvocationId key) { + synchronized PendingStream computeIfAbsent(ClientInvocationId key) { return map.computeIfAbsent(key, PendingStream::new); } - PendingStream remove(ClientInvocationId key) { + synchronized PendingStream remove(ClientInvocationId key) { return map.remove(key); } - void clear() { + synchronized void clear() { + map.values().forEach(PendingStream::clear); map.clear(); } } @@ -84,15 +100,18 @@ void clear() { this.name = name + "-" + JavaUtils.getClassSimpleName(getClass()); } - CompletableFuture streamAsync(RaftClientRequest request) { + CompletableFuture streamAsync(ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.get(); final MessageStreamRequestTypeProto stream = request.getType().getMessageStream(); Preconditions.assertTrue(!stream.getEndOfRequest()); final ClientInvocationId key = ClientInvocationId.valueOf(request.getClientId(), stream.getStreamId()); final PendingStream pending = streams.computeIfAbsent(key); - return pending.append(stream.getMessageId(), request.getMessage()); + return pending.append(stream.getMessageId(), requestRef); } - CompletableFuture streamEndOfRequestAsync(RaftClientRequest request) { + CompletableFuture> streamEndOfRequestAsync( + ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.get(); final MessageStreamRequestTypeProto stream = request.getType().getMessageStream(); Preconditions.assertTrue(stream.getEndOfRequest()); final ClientInvocationId key = ClientInvocationId.valueOf(request.getClientId(), stream.getStreamId()); @@ -101,7 +120,7 @@ CompletableFuture streamEndOfRequestAsync(RaftClientRequest request) if (pending == null) { return JavaUtils.completeExceptionally(new StreamException(name + ": " + key + " not found")); } - return pending.getBytes(stream.getMessageId(), request.getMessage()); + return pending.getWriteRequest(stream.getMessageId(), requestRef); } void clear() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 51067a87a1..64fa520290 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1102,21 +1102,22 @@ private CompletableFuture messageStreamAsync(ReferenceCountedOb } if (request.getType().getMessageStream().getEndOfRequest()) { - final CompletableFuture f = streamEndOfRequestAsync(request); + final CompletableFuture> f = streamEndOfRequestAsync(requestRef); if (f.isCompletedExceptionally()) { return f.thenApply(r -> null); } // the message stream has ended and the request become a WRITE request - return replyFuture(requestRef.delegate(f.join())); + return replyFuture(f.join()); } return role.getLeaderState() - .map(ls -> ls.streamAsync(request)) + .map(ls -> ls.streamAsync(requestRef)) .orElseGet(() -> CompletableFuture.completedFuture( newExceptionReply(request, generateNotLeaderException()))); } - private CompletableFuture streamEndOfRequestAsync(RaftClientRequest request) { + private CompletableFuture> streamEndOfRequestAsync( + ReferenceCountedObject request) { return role.getLeaderState() .map(ls -> ls.streamEndOfRequestAsync(request)) .orElse(null); diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 312c9508d3..07073be520 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -328,7 +328,7 @@ public CompletableFuture query(Message request) { final String string = request.getContent().toStringUtf8(); Exception exception; try { - LOG.info("query " + string); + LOG.info("query {}, all available: {}", string, dataMap.keySet()); final LogEntryProto entry = dataMap.get(string); if (entry != null) { return CompletableFuture.completedFuture(Message.valueOf(entry.toByteString())); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java index e1bfe4e222..21db98d4c4 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java @@ -75,12 +75,14 @@ public synchronized String toString() { private final Count zeroCopyCount = new Count(); private final Count nonZeroCopyCount = new Count(); + private final Count releasedCount = new Count(); private final Server server; private final ZeroCopyMessageMarshaller marshaller = new ZeroCopyMessageMarshaller<>( BinaryRequest.getDefaultInstance(), zeroCopyCount::inc, - nonZeroCopyCount::inc); + nonZeroCopyCount::inc, + releasedCount::inc); GrpcZeroCopyTestServer(int port) { final GreeterImpl greeter = new GreeterImpl(); From 59dfd35491df2346de80a685365b14da52f6bc1f Mon Sep 17 00:00:00 2001 From: Ivan Andika <36403683+ivandika3@users.noreply.github.com> Date: Sun, 14 Jan 2024 07:08:55 +0800 Subject: [PATCH 02/76] RATIS-1998. Add watch request metrics (#1009) --- ratis-docs/src/site/markdown/metrics.md | 53 +++++++++-------- .../apache/ratis/metrics/RatisMetrics.java | 7 +++ .../ratis/server/impl/LeaderStateImpl.java | 3 +- .../ratis/server/impl/WatchRequests.java | 24 +++++++- .../server/metrics/RaftServerMetricsImpl.java | 34 +++++++++++ .../org/apache/ratis/WatchRequestTests.java | 57 +++++++++++++++++++ 6 files changed, 149 insertions(+), 29 deletions(-) diff --git a/ratis-docs/src/site/markdown/metrics.md b/ratis-docs/src/site/markdown/metrics.md index 10c78ccbbb..23486792e6 100644 --- a/ratis-docs/src/site/markdown/metrics.md +++ b/ratis-docs/src/site/markdown/metrics.md @@ -80,31 +80,34 @@ ### Raft Server Metrics -| Application | Component | Name | Type | Description | -|-------------|-----------|----------------------------------|---------|---------------------------------------------------------------------| -| ratis | server | {peer}_lastHeartbeatElapsedTime | Gauge | Time elapsed since last heartbeat rpc response | -| ratis | server | follower_append_entry_latency | Timer | Time taken for followers to append log entries | -| ratis | server | {peer}_peerCommitIndex | Gauge | Commit index of peer | -| ratis | server | clientReadRequest | Timer | Time taken to process read requests from client | -| ratis | server | clientStaleReadRequest | Timer | Time taken to process stale-read requests from client | -| ratis | server | clientWriteRequest | Timer | Time taken to process write requests from client | -| ratis | server | clientWatch{level}Request | Timer | Time taken to process watch(replication_level) requests from client | -| ratis | server | numRequestQueueLimitHits | Counter | Number of (total client requests in queue) limit hits | -| ratis | server | numRequestsByteSizeLimitHits | Counter | Number of (total size of client requests in queue) limit hits | -| ratis | server | numResourceLimitHits | Counter | Sum of numRequestQueueLimitHits and numRequestsByteSizeLimitHits | -| ratis | server | numPendingRequestInQueue | Gauge | Number of pending client requests in queue | -| ratis | server | numPendingRequestMegaByteSize | Gauge | Total size of pending client requests in queue | -| ratis | server | retryCacheEntryCount | Gauge | Number of entries in retry cache | -| ratis | server | retryCacheHitCount | Gauge | Number of retry cache hits | -| ratis | server | retryCacheHitRate | Gauge | Retry cache hit rate | -| ratis | server | retryCacheMissCount | Gauge | Number of retry cache misses | -| ratis | server | retryCacheMissRate | Gauge | Retry cache miss rate | -| ratis | server | numFailedClientStaleReadOnServer | Counter | Number of failed stale-read requests | -| ratis | server | numFailedClientReadOnServer | Counter | Number of failed read requests | -| ratis | server | numFailedClientWriteOnServer | Counter | Number of failed write requests | -| ratis | server | numFailedClientWatchOnServer | Counter | Number of failed watch requests | -| ratis | server | numFailedClientStreamOnServer | Counter | Number of failed stream requests | -| ratis | server | numInstallSnapshot | Counter | Number of install-snapshot requests | +| Application | Component | Name | Type | Description | +|-------------|-----------|--------------------------------------|---------|---------------------------------------------------------------------| +| ratis | server | {peer}_lastHeartbeatElapsedTime | Gauge | Time elapsed since last heartbeat rpc response | +| ratis | server | follower_append_entry_latency | Timer | Time taken for followers to append log entries | +| ratis | server | {peer}_peerCommitIndex | Gauge | Commit index of peer | +| ratis | server | clientReadRequest | Timer | Time taken to process read requests from client | +| ratis | server | clientStaleReadRequest | Timer | Time taken to process stale-read requests from client | +| ratis | server | clientWriteRequest | Timer | Time taken to process write requests from client | +| ratis | server | clientWatch{level}Request | Timer | Time taken to process watch(replication_level) requests from client | +| ratis | server | numRequestQueueLimitHits | Counter | Number of (total client requests in queue) limit hits | +| ratis | server | numRequestsByteSizeLimitHits | Counter | Number of (total size of client requests in queue) limit hits | +| ratis | server | numResourceLimitHits | Counter | Sum of numRequestQueueLimitHits and numRequestsByteSizeLimitHits | +| ratis | server | numPendingRequestInQueue | Gauge | Number of pending client requests in queue | +| ratis | server | numPendingRequestMegaByteSize | Gauge | Total size of pending client requests in queue | +| ratis | server | retryCacheEntryCount | Gauge | Number of entries in retry cache | +| ratis | server | retryCacheHitCount | Gauge | Number of retry cache hits | +| ratis | server | retryCacheHitRate | Gauge | Retry cache hit rate | +| ratis | server | retryCacheMissCount | Gauge | Number of retry cache misses | +| ratis | server | retryCacheMissRate | Gauge | Retry cache miss rate | +| ratis | server | numFailedClientStaleReadOnServer | Counter | Number of failed stale-read requests | +| ratis | server | numFailedClientReadOnServer | Counter | Number of failed read requests | +| ratis | server | numFailedClientWriteOnServer | Counter | Number of failed write requests | +| ratis | server | numFailedClientWatchOnServer | Counter | Number of failed watch requests | +| ratis | server | numFailedClientStreamOnServer | Counter | Number of failed stream requests | +| ratis | server | numInstallSnapshot | Counter | Number of install-snapshot requests | +| ratis | server | numWatch{level}RequestTimeout | Counter | Number of watch(replication_level) request timeout | +| ratis | server | numWatch{level}RequestInQueue | Gauge | Number of watch(replication_level) requests in queue | +| ratis | server | numWatch{level}RequestQueueLimitHits | Counter | Number of (total watch request in queue) limit hits | ## Ratis Netty Metrics diff --git a/ratis-metrics-api/src/main/java/org/apache/ratis/metrics/RatisMetrics.java b/ratis-metrics-api/src/main/java/org/apache/ratis/metrics/RatisMetrics.java index 32bcf52e3c..eafc3849d1 100644 --- a/ratis-metrics-api/src/main/java/org/apache/ratis/metrics/RatisMetrics.java +++ b/ratis-metrics-api/src/main/java/org/apache/ratis/metrics/RatisMetrics.java @@ -49,6 +49,13 @@ protected static > Map> newCounter return Collections.unmodifiableMap(maps); } + protected static > Map newCounterMap( + Class clazz, Function constructor) { + final EnumMap map = new EnumMap<>(clazz); + Arrays.stream(clazz.getEnumConstants()).forEach(t -> map.put(t, constructor.apply(t))); + return Collections.unmodifiableMap(map); + } + protected static > Map newTimerMap( Class clazz, Function constructor) { final EnumMap map = new EnumMap<>(clazz); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 8864c220c1..4175424e89 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -379,7 +379,7 @@ boolean isApplied(LogEntryProto logEntry) { raftServerMetrics = server.getRaftServerMetrics(); logAppenderMetrics = new LogAppenderMetrics(server.getMemberId()); this.pendingRequests = new PendingRequests(server.getMemberId(), properties, raftServerMetrics); - this.watchRequests = new WatchRequests(server.getMemberId(), properties); + this.watchRequests = new WatchRequests(server.getMemberId(), properties, raftServerMetrics); this.messageStreamRequests = new MessageStreamRequests(server.getMemberId()); this.pendingStepDown = new PendingStepDown(this); this.readIndexHeartbeats = new ReadIndexHeartbeats(); @@ -457,6 +457,7 @@ CompletableFuture stop() { logAppenderMetrics.unregister(); raftServerMetrics.unregister(); pendingRequests.close(); + watchRequests.close(); return f; } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/WatchRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/WatchRequests.java index f4c6200b9b..6988bfbae2 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/WatchRequests.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/WatchRequests.java @@ -24,6 +24,7 @@ import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.protocol.exceptions.ResourceUnavailableException; import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.metrics.RaftServerMetricsImpl; import org.apache.ratis.util.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,11 +76,15 @@ private class WatchQueue { private final SortedMap q = new TreeMap<>( Comparator.comparingLong(PendingWatch::getIndex).thenComparing(PendingWatch::getCreationTime)); private final ResourceSemaphore resource; + private final RaftServerMetricsImpl raftServerMetrics; private volatile long index; //Invariant: q.isEmpty() or index < any element q - WatchQueue(ReplicationLevel replication, int elementLimit) { + WatchQueue(ReplicationLevel replication, int elementLimit, RaftServerMetricsImpl raftServerMetrics) { this.replication = replication; this.resource = new ResourceSemaphore(elementLimit); + this.raftServerMetrics = raftServerMetrics; + + raftServerMetrics.addNumPendingWatchRequestsGauge(resource::used, replication); } long getIndex() { @@ -103,6 +108,7 @@ CompletableFuture add(RaftClientRequest request) { if (computed == null) { // failed to acquire + raftServerMetrics.onWatchRequestQueueLimitHit(replication); return JavaUtils.completeExceptionally(new ResourceUnavailableException( "Failed to acquire a pending watch request in " + name + " for " + request)); } @@ -123,6 +129,7 @@ void handleTimeout(RaftClientRequest request, PendingWatch pending) { pending.getFuture().completeExceptionally( new NotReplicatedException(request.getCallId(), replication, pending.getIndex())); LOG.debug("{}: timeout {}, {}", name, pending, request); + raftServerMetrics.onWatchRequestTimeout(replication); } } @@ -162,6 +169,12 @@ synchronized void failAll(Exception e) { q.clear(); resource.close(); } + + void close() { + if (raftServerMetrics != null) { + raftServerMetrics.removeNumPendingWatchRequestsGauge(replication); + } + } } private final String name; @@ -171,7 +184,7 @@ synchronized void failAll(Exception e) { private final TimeDuration watchTimeoutDenominationNanos; private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); - WatchRequests(Object name, RaftProperties properties) { + WatchRequests(Object name, RaftProperties properties, RaftServerMetricsImpl raftServerMetrics) { this.name = name + "-" + JavaUtils.getClassSimpleName(getClass()); final TimeDuration watchTimeout = RaftServerConfigKeys.Watch.timeout(properties); @@ -183,7 +196,8 @@ synchronized void failAll(Exception e) { + watchTimeoutDenomination + ")."); final int elementLimit = RaftServerConfigKeys.Watch.elementLimit(properties); - Arrays.stream(ReplicationLevel.values()).forEach(r -> queues.put(r, new WatchQueue(r, elementLimit))); + Arrays.stream(ReplicationLevel.values()).forEach(r -> queues.put(r, + new WatchQueue(r, elementLimit, raftServerMetrics))); } CompletableFuture add(RaftClientRequest request) { @@ -207,4 +221,8 @@ void update(ReplicationLevel replication, final long newIndex) { void failWatches(Exception e) { queues.values().forEach(q -> q.failAll(e)); } + + void close() { + queues.values().forEach(WatchQueue::close); + } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/metrics/RaftServerMetricsImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/metrics/RaftServerMetricsImpl.java index cdbce6ee22..70711c3e3b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/metrics/RaftServerMetricsImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/metrics/RaftServerMetricsImpl.java @@ -57,9 +57,14 @@ public final class RaftServerMetricsImpl extends RatisMetrics implements RaftSer public static final String REQUEST_QUEUE_LIMIT_HIT_COUNTER = "numRequestQueueLimitHits"; public static final String REQUEST_BYTE_SIZE_LIMIT_HIT_COUNTER = "numRequestsByteSizeLimitHits"; public static final String RESOURCE_LIMIT_HIT_COUNTER = "numResourceLimitHits"; + public static final String WATCH_REQUEST_QUEUE_LIMIT_HIT_COUNTER = "numWatch%sRequestQueueLimitHits"; public static final String REQUEST_QUEUE_SIZE = "numPendingRequestInQueue"; public static final String REQUEST_MEGA_BYTE_SIZE = "numPendingRequestMegaByteSize"; + + public static final String WATCH_REQUEST_QUEUE_SIZE = "numWatch%sRequestInQueue"; + public static final String WATCH_REQUEST_TIMEOUT_COUNTER = "numWatch%sRequestTimeout"; + public static final String RETRY_CACHE_ENTRY_COUNT_METRIC = "retryCacheEntryCount"; public static final String RETRY_CACHE_HIT_COUNT_METRIC = "retryCacheHitCount"; public static final String RETRY_CACHE_HIT_RATE_METRIC = "retryCacheHitRate"; @@ -76,6 +81,11 @@ public final class RaftServerMetricsImpl extends RatisMetrics implements RaftSer private final LongCounter numRequestQueueLimitHits = getRegistry().counter(REQUEST_QUEUE_LIMIT_HIT_COUNTER); private final LongCounter numRequestsByteSizeLimitHits = getRegistry().counter(REQUEST_BYTE_SIZE_LIMIT_HIT_COUNTER); private final LongCounter numResourceLimitHits = getRegistry().counter(RESOURCE_LIMIT_HIT_COUNTER); + private final Map numWatchRequestQueueLimitHits = newCounterMap(ReplicationLevel.class, + replication -> getRegistry().counter( + String.format(WATCH_REQUEST_QUEUE_LIMIT_HIT_COUNTER, Type.toString(replication)))); + private final Map numWatchRequestsTimeout = newCounterMap(ReplicationLevel.class, + replication -> getRegistry().counter(String.format(WATCH_REQUEST_TIMEOUT_COUNTER, Type.toString(replication)))); private final LongCounter numFailedClientStaleRead = getRegistry().counter(RATIS_SERVER_FAILED_CLIENT_STALE_READ_COUNT); @@ -150,6 +160,14 @@ public LongCounter getNumInstallSnapshot() { return numInstallSnapshot; } + public LongCounter getNumWatchRequestQueueLimitHits(ReplicationLevel replication) { + return numWatchRequestQueueLimitHits.get(replication); + } + + public LongCounter getNumWatchRequestsTimeout(ReplicationLevel replication) { + return numWatchRequestsTimeout.get(replication); + } + private static RatisMetricRegistry createRegistry(String serverId) { return create(new MetricRegistryInfo(serverId, RATIS_APPLICATION_NAME_METRICS, RATIS_SERVER_METRICS, @@ -237,6 +255,22 @@ public boolean removeNumPendingRequestsByteSize() { return getRegistry().remove(REQUEST_MEGA_BYTE_SIZE); } + public void onWatchRequestQueueLimitHit(ReplicationLevel replicationLevel) { + numWatchRequestQueueLimitHits.get(replicationLevel).inc(); + } + + public void onWatchRequestTimeout(ReplicationLevel replicationLevel) { + numWatchRequestsTimeout.get(replicationLevel).inc(); + } + + public void addNumPendingWatchRequestsGauge(Supplier queueSize, ReplicationLevel replication) { + getRegistry().gauge(String.format(WATCH_REQUEST_QUEUE_SIZE, Type.toString(replication)), () -> queueSize); + } + + public boolean removeNumPendingWatchRequestsGauge(ReplicationLevel replication) { + return getRegistry().remove(String.format(WATCH_REQUEST_QUEUE_SIZE, Type.toString(replication))); + } + public void onRequestByteSizeLimitHit() { numRequestsByteSizeLimitHits.inc(); } diff --git a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java index 1f19f9d1c2..a9bdd1a3a9 100644 --- a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java @@ -31,8 +31,10 @@ import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.RaftServerConfigKeys.Watch; import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.server.impl.RaftServerTestUtil; +import org.apache.ratis.server.metrics.RaftServerMetricsImpl; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; @@ -47,12 +49,14 @@ import org.slf4j.event.Level; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import java.util.stream.Stream; import static org.junit.Assert.fail; @@ -469,6 +473,59 @@ static void runTestWatchRequestClientTimeout(TestParameters p) throws Exception } } + @Test + public void testWatchMetrics() throws Exception { + final RaftProperties p = getProperties(); + RaftServerConfigKeys.Watch.setElementLimit(p, 10); + RaftServerConfigKeys.Watch.setTimeout(p, TimeDuration.valueOf(2, TimeUnit.SECONDS)); + try { + runWithNewCluster(NUM_SERVERS, + cluster -> runSingleTest(WatchRequestTests::runTestWatchMetrics, cluster, LOG)); + } finally { + RaftServerConfigKeys.Watch.setElementLimit(p, Watch.ELEMENT_LIMIT_DEFAULT); + RaftServerConfigKeys.Watch.setTimeout(p, RaftServerConfigKeys.Watch.TIMEOUT_DEFAULT); + } + } + + static RaftServerMetricsImpl getRaftServerMetrics(RaftServer.Division division) { + return (RaftServerMetricsImpl) division.getRaftServerMetrics(); + } + + static void runTestWatchMetrics(TestParameters p) throws Exception { + final MiniRaftCluster cluster = p.cluster; + + List clients = new ArrayList<>(); + + final ReplicationLevel replicationLevel = ReplicationLevel.MAJORITY; + try { + long initialWatchRequestTimeoutCount = getRaftServerMetrics(cluster.getLeader()) + .getNumWatchRequestsTimeout(replicationLevel).getCount(); + long initialLimitHit = getRaftServerMetrics(cluster.getLeader()) + .getNumWatchRequestQueueLimitHits(replicationLevel).getCount(); + + int uncommittedBaseIndex = 10000; + // Logs with indices 10001 - 10011 will never be committed, so it should fail with NotReplicatedException + for (int i = 1; i <= 11; i++) { + RaftClient client = cluster.createClient(cluster.getLeader().getId(), RetryPolicies.noRetry()); + clients.add(client); + client.async().watch(uncommittedBaseIndex + i, replicationLevel); + } + + // All the watch timeout for each unique index should increment the metric + RaftTestUtil.waitFor(() -> getRaftServerMetrics(cluster.getLeader()) + .getNumWatchRequestsTimeout(replicationLevel).getCount() == initialWatchRequestTimeoutCount + 10, + 300, 5000); + // There are 11 pending watch request, but the pending watch request limit is 10 + RaftTestUtil.waitFor(() -> getRaftServerMetrics(cluster.getLeader()) + .getNumWatchRequestQueueLimitHits(replicationLevel).getCount() == + initialLimitHit + 1, 300, 5000); + } finally { + for(RaftClient client : clients) { + client.close(); + } + } + } + static void checkTimeout(List> replies, List> watches, Logger LOG) throws Exception { for(int i = 0; i < replies.size(); i++) { From 206205810db1cab29987b2331dac205aafd62497 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 15 Jan 2024 18:06:15 +0100 Subject: [PATCH 03/76] RATIS-1989. Intermittent timeout in TestStreamObserverWithTimeout (#1012) --- .../grpc/server/GrpcServerProtocolClient.java | 2 +- .../grpc/util/StreamObserverWithTimeout.java | 20 ++++++++----- .../ratis/grpc/util/GrpcTestClient.java | 6 +++- .../ratis/grpc/util/GrpcTestServer.java | 23 +++++++++++---- .../util/TestStreamObserverWithTimeout.java | 29 ++++++++++++------- 5 files changed, 54 insertions(+), 26 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java index dd4e199b25..970134d02a 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java @@ -139,7 +139,7 @@ CallStreamObserver appendEntries( StreamObserver installSnapshot( String name, TimeDuration timeout, int limit, StreamObserver responseHandler) { return StreamObserverWithTimeout.newInstance(name, ServerStringUtils::toInstallSnapshotRequestString, - timeout, limit, i -> asyncStub.withInterceptors(i).installSnapshot(responseHandler)); + () -> timeout, limit, i -> asyncStub.withInterceptors(i).installSnapshot(responseHandler)); } // short-circuit the backoff timer and make them reconnect immediately. diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/StreamObserverWithTimeout.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/StreamObserverWithTimeout.java index ff89e7d4b7..3cc754e565 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/StreamObserverWithTimeout.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/StreamObserverWithTimeout.java @@ -32,13 +32,14 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.function.IntSupplier; +import java.util.function.Supplier; public final class StreamObserverWithTimeout implements StreamObserver { public static final Logger LOG = LoggerFactory.getLogger(StreamObserverWithTimeout.class); public static StreamObserverWithTimeout newInstance( String name, Function request2String, - TimeDuration timeout, int outstandingLimit, + Supplier timeout, int outstandingLimit, Function> newStreamObserver) { final AtomicInteger responseCount = new AtomicInteger(); final ResourceSemaphore semaphore = outstandingLimit > 0? new ResourceSemaphore(outstandingLimit): null; @@ -55,7 +56,7 @@ public static StreamObserverWithTimeout newInstance( private final String name; private final Function requestToStringFunction; - private final TimeDuration timeout; + private final Supplier timeoutSupplier; private final StreamObserver observer; private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); @@ -65,17 +66,18 @@ public static StreamObserverWithTimeout newInstance( private final ResourceSemaphore semaphore; private StreamObserverWithTimeout(String name, Function requestToStringFunction, - TimeDuration timeout, IntSupplier responseCount, ResourceSemaphore semaphore, StreamObserver observer) { + Supplier timeoutSupplier, IntSupplier responseCount, ResourceSemaphore semaphore, + StreamObserver observer) { this.name = JavaUtils.getClassSimpleName(getClass()) + "-" + name; this.requestToStringFunction = requestToStringFunction; - this.timeout = timeout; + this.timeoutSupplier = timeoutSupplier; this.responseCount = responseCount; this.semaphore = semaphore; this.observer = observer; } - private void acquire(StringSupplier request) { + private void acquire(StringSupplier request, TimeDuration timeout) { if (semaphore == null) { return; } @@ -96,14 +98,16 @@ private void acquire(StringSupplier request) { @Override public void onNext(T request) { final StringSupplier requestString = StringSupplier.get(() -> requestToStringFunction.apply(request)); - acquire(requestString); + final TimeDuration timeout = timeoutSupplier.get(); + acquire(requestString, timeout); observer.onNext(request); final int id = requestCount.incrementAndGet(); - scheduler.onTimeout(timeout, () -> handleTimeout(id, requestString), + LOG.debug("{}: send {} with timeout={}: {}", name, id, timeout, requestString); + scheduler.onTimeout(timeout, () -> handleTimeout(id, timeout, requestString), LOG, () -> name + ": Timeout check failed for request: " + requestString); } - private void handleTimeout(int id, StringSupplier request) { + private void handleTimeout(int id, TimeDuration timeout, StringSupplier request) { if (id > responseCount.getAsInt()) { onError(new TimeoutIOException(name + ": Timed out " + timeout + " for sending request " + request)); } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcTestClient.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcTestClient.java index 130c05eb94..ca8957e2d4 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcTestClient.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcTestClient.java @@ -37,6 +37,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiFunction; /** gRPC client for testing */ @@ -55,8 +56,11 @@ static StreamObserverFactory withDeadline(TimeDuration timeout) { static StreamObserverFactory withTimeout(TimeDuration timeout) { final String className = JavaUtils.getClassSimpleName(HelloRequest.class) + ":"; + final AtomicBoolean initialized = new AtomicBoolean(); return (stub, responseHandler) -> StreamObserverWithTimeout.newInstance("test", - r -> className + r.getName(), timeout, 2, + r -> className + r.getName(), + () -> initialized.getAndSet(true) ? timeout : TimeDuration.ONE_MINUTE.add(timeout), + 2, i -> stub.withInterceptors(i).hello(responseHandler)); } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcTestServer.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcTestServer.java index d497ac1cd0..345c565f30 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcTestServer.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcTestServer.java @@ -23,6 +23,8 @@ import org.apache.ratis.thirdparty.io.grpc.Server; import org.apache.ratis.thirdparty.io.grpc.ServerBuilder; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; +import org.apache.ratis.thirdparty.io.netty.util.concurrent.ThreadPerTaskExecutor; +import org.apache.ratis.util.Daemon; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; @@ -31,16 +33,22 @@ import java.io.Closeable; import java.io.IOException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; /** gRPC server for testing */ class GrpcTestServer implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(GrpcTestServer.class); + private static final AtomicLong COUNTER = new AtomicLong(); private final Server server; - GrpcTestServer(int port, int slow, TimeDuration timeout) { + GrpcTestServer(int port, int warmup, int slow, TimeDuration timeout) { this.server = ServerBuilder.forPort(port) - .addService(new GreeterImpl(slow, timeout)) + .executor(new ThreadPerTaskExecutor(r -> Daemon.newBuilder() + .setName("test-server-" + COUNTER.getAndIncrement()) + .setRunnable(r) + .build())) + .addService(new GreeterImpl(warmup, slow, timeout)) .build(); } @@ -64,14 +72,16 @@ static String toReplySuffix(String request) { return ") Hello " + request; } + private final int warmup; private final int slow; private final TimeDuration shortSleepTime; private final TimeDuration longSleepTime; private int count = 0; - GreeterImpl(int slow, TimeDuration timeout) { + GreeterImpl(int warmup, int slow, TimeDuration timeout) { + this.warmup = warmup; this.slow = slow; - this.shortSleepTime = timeout.multiply(0.1); + this.shortSleepTime = timeout.multiply(0.25); this.longSleepTime = timeout.multiply(2); } @@ -81,7 +91,8 @@ public StreamObserver hello(StreamObserver responseObs @Override public void onNext(HelloRequest helloRequest) { final String reply = count + toReplySuffix(helloRequest.getName()); - final TimeDuration sleepTime = count < slow ? shortSleepTime : longSleepTime; + final TimeDuration sleepTime = count < warmup ? TimeDuration.ZERO : + count < (warmup + slow) ? shortSleepTime : longSleepTime; LOG.info("count = {}, slow = {}, sleep {}", reply, slow, sleepTime); try { sleepTime.sleep(); @@ -105,4 +116,4 @@ public void onCompleted() { }; } } -} +} \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java index 7a32fb96af..d0c936aa40 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java @@ -24,6 +24,7 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.TimeoutTimer; import org.junit.Assert; import org.junit.Test; import org.slf4j.event.Level; @@ -37,6 +38,8 @@ public class TestStreamObserverWithTimeout extends BaseTest { { Slf4jUtils.setLogLevel(ResponseNotifyClientInterceptor.LOG, Level.TRACE); + Slf4jUtils.setLogLevel(StreamObserverWithTimeout.LOG, Level.DEBUG); + Slf4jUtils.setLogLevel(TimeoutTimer.LOG, Level.DEBUG); } enum Type { @@ -57,14 +60,14 @@ StreamObserverFactory createFunction(TimeDuration timeout) { @Test public void testWithDeadline() throws Exception { //the total sleep time is within the deadline - runTestTimeout(8, Type.WithDeadline); + runTestTimeout(2, Type.WithDeadline); } @Test public void testWithDeadlineFailure() { //Expected to have DEADLINE_EXCEEDED testFailureCase("total sleep time is longer than the deadline", - () -> runTestTimeout(12, Type.WithDeadline), + () -> runTestTimeout(5, Type.WithDeadline), ExecutionException.class, StatusRuntimeException.class); } @@ -72,7 +75,7 @@ public void testWithDeadlineFailure() { public void testWithTimeout() throws Exception { //Each sleep time is within the timeout, //Note that the total sleep time is longer than the timeout, but it does not matter. - runTestTimeout(12, Type.WithTimeout); + runTestTimeout(5, Type.WithTimeout); } void runTestTimeout(int slow, Type type) throws Exception { @@ -80,14 +83,20 @@ void runTestTimeout(int slow, Type type) throws Exception { final TimeDuration timeout = ONE_SECOND.multiply(0.5); final StreamObserverFactory function = type.createFunction(timeout); + // first request may take longer due to initialization + final int warmup = type == Type.WithTimeout ? 1 : 0; final List messages = new ArrayList<>(); for (int i = 0; i < 2 * slow; i++) { - messages.add("m" + i); + messages.add("m" + (i + warmup)); } - try (GrpcTestServer server = new GrpcTestServer(NetUtils.getFreePort(), slow, timeout)) { + try (GrpcTestServer server = new GrpcTestServer(NetUtils.getFreePort(), warmup, slow, timeout)) { final int port = server.start(); try (GrpcTestClient client = new GrpcTestClient(NetUtils.LOCALHOST, port, function)) { + if (warmup == 1) { + client.send("warmup").join(); + } + final List> futures = new ArrayList<>(); for (String m : messages) { futures.add(client.send(m)); @@ -95,20 +104,20 @@ void runTestTimeout(int slow, Type type) throws Exception { int i = 0; for (; i < slow; i++) { - final String expected = i + GrpcTestServer.GreeterImpl.toReplySuffix(messages.get(i)); + final String expected = (i + warmup) + GrpcTestServer.GreeterImpl.toReplySuffix(messages.get(i)); final String reply = futures.get(i).get(); - Assert.assertEquals("expected = " + expected + " != reply = " + reply, expected, reply); - LOG.info("{}) passed", i); + Assert.assertEquals(expected, reply); + LOG.info("{}) passed", (i + warmup)); } for (; i < messages.size(); i++) { final CompletableFuture f = futures.get(i); try { final String reply = f.get(); - Assert.fail(i + ") reply = " + reply + ", " + Assert.fail((i + warmup) + ") reply = " + reply + ", " + StringUtils.completableFuture2String(f, false)); } catch (ExecutionException e) { - LOG.info("GOOD! {}) {}, {}", i, StringUtils.completableFuture2String(f, true), e); + LOG.info("GOOD! {}) {}, {}", (i + warmup), StringUtils.completableFuture2String(f, true), e); } } } From 863c2e8a069337f1dd1ae3f2520d118a443fa612 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 15 Jan 2024 13:28:14 -0800 Subject: [PATCH 04/76] RATIS-1987. Intermittent NPE in TestGrpcZeroCopy. (#1013) --- .../grpc/util/GrpcZeroCopyTestClient.java | 43 +++++++++---------- 1 file changed, 21 insertions(+), 22 deletions(-) diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestClient.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestClient.java index 791d5a6d24..ef66b72be2 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestClient.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestClient.java @@ -29,27 +29,24 @@ import org.apache.ratis.thirdparty.io.grpc.ManagedChannelBuilder; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.util.IOUtils; +import org.apache.ratis.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Objects; import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; -import java.util.function.BiFunction; +import java.util.function.Consumer; /** gRPC client for testing */ class GrpcZeroCopyTestClient implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(GrpcZeroCopyTestClient.class); - @FunctionalInterface - interface StreamObserverFactory - extends BiFunction, StreamObserver> { - } - private final ManagedChannel channel; private final StreamObserver helloRequestHandler; @@ -67,7 +64,9 @@ interface StreamObserverFactory final StreamObserver helloResponseHandler = new StreamObserver() { @Override public void onNext(HelloReply helloReply) { - helloReplies.poll().complete(helloReply.getMessage()); + final CompletableFuture polled = helloReplies.poll(); + Objects.requireNonNull(polled, "polled"); + polled.complete(helloReply.getMessage()); } @Override @@ -93,7 +92,9 @@ void completeExceptionally(Throwable throwable) { final StreamObserver binaryResponseHandler = new StreamObserver() { @Override public void onNext(BinaryReply binaryReply) { - binaryReplies.poll().complete(binaryReply.getData()); + final CompletableFuture polled = binaryReplies.poll(); + Objects.requireNonNull(polled, "polled"); + polled.complete(binaryReply.getData()); } @Override @@ -132,29 +133,27 @@ public void close() throws IOException { CompletableFuture send(String name) { LOG.info("send message {}", name); final HelloRequest request = HelloRequest.newBuilder().setName(name).build(); - final CompletableFuture f = new CompletableFuture<>(); - try { - helloRequestHandler.onNext(request); - helloReplies.offer(f); - } catch (IllegalStateException e) { - // already closed - f.completeExceptionally(e); - } - return f; + return send(request, helloReplies, helloRequestHandler::onNext); } CompletableFuture send(ByteBuffer data) { LOG.info("send data: size={}, direct? {}", data.remaining(), data.isDirect()); final BinaryRequest request = BinaryRequest.newBuilder().setData(UnsafeByteOperations.unsafeWrap(data)).build(); - final CompletableFuture f = new CompletableFuture<>(); + return send(request, binaryReplies, binaryRequestHandler::onNext); + } + + static CompletableFuture send(REQUEST request, + Queue> queue, Consumer onNext) { + final CompletableFuture f = new CompletableFuture<>(); + queue.offer(f); try { - binaryRequestHandler.onNext(request); - binaryReplies.offer(f); - } catch (IllegalStateException e) { + onNext.accept(request); + } catch (Exception e) { // already closed f.completeExceptionally(e); + final CompletableFuture polled = queue.poll(); + Preconditions.assertSame(f, polled, "future"); } return f; } - } \ No newline at end of file From 6671ef6839e55469333f2f88935fe2d924e6ae5b Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 16 Jan 2024 08:11:08 -0800 Subject: [PATCH 05/76] RATIS-1999. Move RaftServerJmxAdapter out from RaftServerImpl. (#1010) --- .../org/apache/ratis/util/JmxRegister.java | 7 +- .../ratis/server/impl/LeaderStateImpl.java | 6 +- .../ratis/server/impl/RaftServerImpl.java | 134 ++++-------------- .../server/impl/RaftServerJmxAdapter.java | 97 +++++++++++++ .../ratis/server/impl/TestRaftServerJmx.java | 6 +- 5 files changed, 135 insertions(+), 115 deletions(-) create mode 100644 ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerJmxAdapter.java diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java b/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java index 54f7989245..4554410488 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java @@ -46,15 +46,16 @@ static ObjectName tryRegister(String name, Object mBean) { } /** - * Try registering the mBean with the names one by one. + * Try registering the mxBean with the names one by one. * @return the registered name, or, if it fails, return null. */ - public synchronized String register(Object mBean, Iterable> names) { + public synchronized String register(Object mxBean, Iterable> names) { if (registeredName == null) { for (Supplier supplier : names) { final String name = supplier.get(); - registeredName = tryRegister(name, mBean); + registeredName = tryRegister(name, mxBean); if (registeredName != null) { + LOG.info("register mxBean {} as {}", mxBean.getClass(), name); return name; } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 4175424e89..ce19fdafad 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -643,6 +643,10 @@ private RaftPeer getPeer(RaftPeerId id) { return server.getRaftConf().getPeer(id, RaftPeerRole.FOLLOWER, RaftPeerRole.LISTENER); } + private LogAppender newLogAppender(FollowerInfo f) { + return server.getRaftServer().getFactory().newLogAppender(server, this, f); + } + private Collection addSenders(Collection newPeers, long nextIndex, boolean caughtUp) { final Timestamp t = Timestamp.currentTime().addTimeMs(-server.getMaxTimeoutMs()); final List newAppenders = newPeers.stream().map(peer -> { @@ -650,7 +654,7 @@ private Collection addSenders(Collection newPeers, long n followerInfoMap.put(peer.getId(), f); raftServerMetrics.addFollower(peer.getId()); logAppenderMetrics.addFollowerGauges(peer.getId(), f::getNextIndex, f::getMatchIndex, f::getLastRpcTime); - return server.newLogAppender(this, f); + return newLogAppender(f); }).collect(Collectors.toList()); senders.addAll(newAppenders); return newAppenders; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 64fa520290..37d7f300c9 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.nio.file.NoSuchFileException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -41,12 +40,12 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; -import javax.management.ObjectName; import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.metrics.Timekeeper; import org.apache.ratis.proto.RaftProtos.*; +import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto.TypeCase; import org.apache.ratis.protocol.ClientInvocationId; import org.apache.ratis.protocol.GroupInfoReply; @@ -71,11 +70,9 @@ import org.apache.ratis.server.DivisionProperties; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; -import org.apache.ratis.server.RaftServerMXBean; import org.apache.ratis.server.RaftServerRpc; import org.apache.ratis.server.impl.LeaderElection.Phase; import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry; -import org.apache.ratis.server.leader.FollowerInfo; import org.apache.ratis.server.leader.LeaderState; import org.apache.ratis.server.leader.LogAppender; import org.apache.ratis.server.metrics.LeaderElectionMetrics; @@ -100,8 +97,8 @@ import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; -import org.apache.ratis.util.JmxRegister; import org.apache.ratis.util.LifeCycle; +import org.apache.ratis.util.LifeCycle.State; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; @@ -109,16 +106,6 @@ import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedSupplier; -import static org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.INCONSISTENCY; -import static org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.NOT_LEADER; -import static org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.SUCCESS; -import static org.apache.ratis.server.raftlog.RaftLog.INVALID_LOG_INDEX; -import static org.apache.ratis.util.LifeCycle.State.EXCEPTION; -import static org.apache.ratis.util.LifeCycle.State.NEW; -import static org.apache.ratis.util.LifeCycle.State.PAUSED; -import static org.apache.ratis.util.LifeCycle.State.PAUSING; -import static org.apache.ratis.util.LifeCycle.State.RUNNING; -import static org.apache.ratis.util.LifeCycle.State.STARTING; class RaftServerImpl implements RaftServer.Division, RaftServerProtocol, RaftServerAsynchronousProtocol, @@ -197,7 +184,7 @@ public long[] getFollowerNextIndices() { private final CommitInfoCache commitInfoCache = new CommitInfoCache(); private final WriteIndexCache writeIndexCache; - private final RaftServerJmxAdapter jmxAdapter; + private final RaftServerJmxAdapter jmxAdapter = new RaftServerJmxAdapter(this); private final LeaderElectionMetrics leaderElectionMetrics; private final RaftServerMetricsImpl raftServerMetrics; @@ -239,7 +226,6 @@ public long[] getFollowerNextIndices() { this.readOption = RaftServerConfigKeys.Read.option(properties); this.writeIndexCache = new WriteIndexCache(properties); - this.jmxAdapter = new RaftServerJmxAdapter(); this.leaderElectionMetrics = LeaderElectionMetrics.getLeaderElectionMetrics( getMemberId(), state::getLastLeaderElapsedTimeMs); this.raftServerMetrics = RaftServerMetricsImpl.computeIfAbsentRaftServerMetrics( @@ -271,10 +257,6 @@ public DivisionProperties properties() { return divisionProperties; } - LogAppender newLogAppender(LeaderState leaderState, FollowerInfo f) { - return getRaftServer().getFactory().newLogAppender(this, leaderState, f); - } - int getMaxTimeoutMs() { return properties().maxRpcTimeoutMs(); } @@ -354,7 +336,7 @@ private void setRole(RaftPeerRole newRole, Object reason) { } boolean start() throws IOException { - if (!lifeCycle.compareAndTransition(NEW, STARTING)) { + if (!lifeCycle.compareAndTransition(State.NEW, State.STARTING)) { return false; } state.initialize(stateMachine); @@ -368,24 +350,15 @@ boolean start() throws IOException { startAsPeer(RaftPeerRole.LISTENER); } else { LOG.info("{}: start with initializing state, conf={}", getMemberId(), conf); - startInitializing(); + setRole(RaftPeerRole.FOLLOWER, "start"); } - registerMBean(getId(), getMemberId().getGroupId(), jmxAdapter, jmxAdapter); + jmxAdapter.registerMBean(); state.start(); startComplete.compareAndSet(false, true); return true; } - static boolean registerMBean( - RaftPeerId id, RaftGroupId groupId, RaftServerMXBean mBean, JmxRegister jmx) { - final String prefix = "Ratis:service=RaftServer,group=" + groupId + ",id="; - final String registered = jmx.register(mBean, Arrays.asList( - () -> prefix + id, - () -> prefix + ObjectName.quote(id.toString()))); - return registered != null; - } - /** * The peer belongs to the current configuration, should start as a follower or listener */ @@ -402,17 +375,7 @@ private void startAsPeer(RaftPeerRole newRole) { } role.startFollowerState(this, reason); - lifeCycle.transition(RUNNING); - } - - /** - * The peer does not have any configuration (maybe it will later be included - * in some configuration). Start still as a follower but will not vote or - * start election. - */ - private void startInitializing() { - setRole(RaftPeerRole.FOLLOWER, "startInitializing"); - // do not start FollowerState + lifeCycle.transition(State.RUNNING); } ServerState getState() { @@ -771,7 +734,7 @@ private CompletableFuture checkLeaderState(RaftClientRequest re } NotLeaderException generateNotLeaderException() { - if (lifeCycle.getCurrentState() != RUNNING) { + if (!lifeCycle.getCurrentState().isRunning()) { return new NotLeaderException(getMemberId(), null, null); } RaftPeerId leaderId = state.getLeaderId(); @@ -1255,7 +1218,7 @@ CompletableFuture takeSnapshotAsync(SnapshotManagementRequest r synchronized (this) { final long installSnapshot = snapshotInstallationHandler.getInProgressInstallSnapshotIndex(); // check snapshot install/load - if (installSnapshot != INVALID_LOG_INDEX) { + if (installSnapshot != RaftLog.INVALID_LOG_INDEX) { String msg = String.format("%s: Failed do snapshot as snapshot (%s) installation is in progress", getMemberId(), installSnapshot); LOG.warn(msg); @@ -1531,13 +1494,13 @@ public CompletableFuture readIndexAsync(ReadIndexRequestPro final LeaderStateImpl leader = role.getLeaderState().orElse(null); if (leader == null) { return CompletableFuture.completedFuture( - ServerProtoUtils.toReadIndexReplyProto(peerId, getMemberId(), false, INVALID_LOG_INDEX)); + ServerProtoUtils.toReadIndexReplyProto(peerId, getMemberId(), false, RaftLog.INVALID_LOG_INDEX)); } return getReadIndex(ClientProtoUtils.toRaftClientRequest(request.getClientRequest()), leader) .thenApply(index -> ServerProtoUtils.toReadIndexReplyProto(peerId, getMemberId(), true, index)) .exceptionally(throwable -> - ServerProtoUtils.toReadIndexReplyProto(peerId, getMemberId(), false, INVALID_LOG_INDEX)); + ServerProtoUtils.toReadIndexReplyProto(peerId, getMemberId(), false, RaftLog.INVALID_LOG_INDEX)); } static void logAppendEntries(boolean isHeartbeat, Supplier message) { @@ -1554,7 +1517,7 @@ static void logAppendEntries(boolean isHeartbeat, Supplier message) { Optional updateLastRpcTime(FollowerState.UpdateType updateType) { final Optional fs = role.getFollowerState(); - if (fs.isPresent() && lifeCycle.getCurrentState() == RUNNING) { + if (fs.isPresent() && lifeCycle.getCurrentState().isRunning()) { fs.get().updateLastRpcTime(updateType); return fs; } else { @@ -1611,8 +1574,8 @@ private CompletableFuture appendEntriesAsync( currentTerm = state.getCurrentTerm(); if (!recognized) { final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto( - leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), NOT_LEADER, callId, - INVALID_LOG_INDEX, isHeartbeat); + leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), + AppendResult.NOT_LEADER, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat); if (LOG.isDebugEnabled()) { LOG.debug("{}: Not recognize {} (term={}) as leader, state: {} reply: {}", getMemberId(), leaderId, leaderTerm, state, ServerStringUtils.toAppendEntriesReplyString(reply)); @@ -1626,7 +1589,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), NOT_ } state.setLeader(leaderId, "appendEntries"); - if (!initializing && lifeCycle.compareAndTransition(STARTING, RUNNING)) { + if (!initializing && lifeCycle.compareAndTransition(State.STARTING, State.RUNNING)) { role.startFollowerState(this, Op.APPEND_ENTRIES); } followerState = updateLastRpcTime(FollowerState.UpdateType.APPEND_START); @@ -1678,11 +1641,10 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), NOT_ matchIndex = requestLastEntry.getIndex(); } else { n = state.getLog().getNextIndex(); - matchIndex = INVALID_LOG_INDEX; + matchIndex = RaftLog.INVALID_LOG_INDEX; } reply = ServerProtoUtils.toAppendEntriesReplyProto(leaderId, getMemberId(), currentTerm, - state.getLog().getLastCommittedIndex(), n, SUCCESS, callId, matchIndex, - isHeartbeat); + state.getLog().getLastCommittedIndex(), n, AppendResult.SUCCESS, callId, matchIndex, isHeartbeat); } logAppendEntries(isHeartbeat, () -> getMemberId() + ": succeeded to handle AppendEntries. Reply: " + ServerStringUtils.toAppendEntriesReplyString(reply)); @@ -1699,8 +1661,8 @@ private AppendEntriesReplyProto checkInconsistentAppendEntries(RaftPeerId leader } final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto( - leaderId, getMemberId(), currentTerm, followerCommit, replyNextIndex, INCONSISTENCY, callId, - INVALID_LOG_INDEX, isHeartbeat); + leaderId, getMemberId(), currentTerm, followerCommit, replyNextIndex, + AppendResult.INCONSISTENCY, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat); LOG.info("{}: inconsistency entries. Reply:{}", getMemberId(), ServerStringUtils.toAppendEntriesReplyString(reply)); return reply; } @@ -1708,7 +1670,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, replyNextIndex, INCONSISTE private long checkInconsistentAppendEntries(TermIndex previous, List entries) { // Check if a snapshot installation through state machine is in progress. final long installSnapshot = snapshotInstallationHandler.getInProgressInstallSnapshotIndex(); - if (installSnapshot != INVALID_LOG_INDEX) { + if (installSnapshot != RaftLog.INVALID_LOG_INDEX) { LOG.info("{}: Failed appendEntries as snapshot ({}) installation is in progress", getMemberId(), installSnapshot); return state.getNextIndex(); } @@ -1720,7 +1682,7 @@ private long checkInconsistentAppendEntries(TermIndex previous, List INVALID_LOG_INDEX && nextIndex >= firstEntryIndex) { + if (nextIndex > RaftLog.INVALID_LOG_INDEX && nextIndex >= firstEntryIndex) { LOG.info("{}: Failed appendEntries as the first entry (index {})" + " already exists (snapshotIndex: {}, commitIndex: {})", getMemberId(), firstEntryIndex, snapshotIndex, commitIndex); @@ -1750,19 +1712,19 @@ boolean pause() { // Pause() should pause ongoing operations: // a. call {@link StateMachine#pause()}. synchronized (this) { - if (!lifeCycle.compareAndTransition(RUNNING, PAUSING)) { + if (!lifeCycle.compareAndTransition(State.RUNNING, State.PAUSING)) { return false; } // TODO: any other operations that needs to be paused? stateMachine.pause(); - lifeCycle.compareAndTransition(PAUSING, PAUSED); + lifeCycle.compareAndTransition(State.PAUSING, State.PAUSED); } return true; } boolean resume() throws IOException { synchronized (this) { - if (!lifeCycle.compareAndTransition(PAUSED, STARTING)) { + if (!lifeCycle.compareAndTransition(State.PAUSED, State.STARTING)) { return false; } // TODO: any other operations that needs to be resumed? @@ -1770,10 +1732,10 @@ boolean resume() throws IOException { stateMachine.reinitialize(); } catch (IOException e) { LOG.warn("Failed to reinitialize statemachine: {}", stateMachine); - lifeCycle.compareAndTransition(STARTING, EXCEPTION); + lifeCycle.compareAndTransition(State.STARTING, State.EXCEPTION); throw e; } - lifeCycle.compareAndTransition(STARTING, RUNNING); + lifeCycle.compareAndTransition(State.STARTING, State.RUNNING); } return true; } @@ -1945,50 +1907,6 @@ public RaftServerMetricsImpl getRaftServerMetrics() { return raftServerMetrics; } - private class RaftServerJmxAdapter extends JmxRegister implements RaftServerMXBean { - @Override - public String getId() { - return getMemberId().getPeerId().toString(); - } - - @Override - public String getLeaderId() { - RaftPeerId leaderId = getState().getLeaderId(); - if (leaderId != null) { - return leaderId.toString(); - } else { - return null; - } - } - - @Override - public long getCurrentTerm() { - return getState().getCurrentTerm(); - } - - @Override - public String getGroupId() { - return getMemberId().getGroupId().toString(); - } - - @Override - public String getRole() { - return role.toString(); - } - - @Override - public List getFollowers() { - return role.getLeaderState().map(LeaderStateImpl::getFollowers).orElseGet(Stream::empty) - .map(RaftPeer::toString).collect(Collectors.toList()); - } - - @Override - public List getGroups() { - return proxy.getGroupIds().stream().map(RaftGroupId::toString) - .collect(Collectors.toList()); - } - } - void onGroupLeaderElected() { transferLeadership.complete(TransferLeadership.Result.SUCCESS); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerJmxAdapter.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerJmxAdapter.java new file mode 100644 index 0000000000..54fa80150d --- /dev/null +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerJmxAdapter.java @@ -0,0 +1,97 @@ +/* + * 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.ratis.server.impl; + +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.server.RaftServerMXBean; +import org.apache.ratis.util.JmxRegister; + +import javax.management.ObjectName; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** JMX for {@link RaftServerImpl}. */ +class RaftServerJmxAdapter extends JmxRegister implements RaftServerMXBean { + static boolean registerMBean(String id, String groupId, RaftServerMXBean mBean, JmxRegister jmx) { + final String prefix = "Ratis:service=RaftServer,group=" + groupId + ",id="; + final String registered = jmx.register(mBean, Arrays.asList( + () -> prefix + id, + () -> prefix + ObjectName.quote(id))); + return registered != null; + } + + private final RaftServerImpl server; + + RaftServerJmxAdapter(RaftServerImpl server) { + this.server = server; + } + + boolean registerMBean() { + return registerMBean(getId(), getGroupId(), this, this); + } + + @Override + public String getId() { + return server.getId().toString(); + } + + @Override + public String getLeaderId() { + RaftPeerId leaderId = server.getState().getLeaderId(); + if (leaderId != null) { + return leaderId.toString(); + } else { + return null; + } + } + + @Override + public long getCurrentTerm() { + return server.getState().getCurrentTerm(); + } + + @Override + public String getGroupId() { + return server.getMemberId().getGroupId().toString(); + } + + @Override + public String getRole() { + return server.getRole().toString(); + } + + @Override + public List getFollowers() { + return server.getRole().getLeaderState() + .map(LeaderStateImpl::getFollowers) + .orElseGet(Stream::empty) + .map(RaftPeer::toString) + .collect(Collectors.toList()); + } + + @Override + public List getGroups() { + return server.getRaftServer().getGroupIds().stream() + .map(RaftGroupId::toString) + .collect(Collectors.toList()); + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java index 0bd77e16a7..01d8392303 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java @@ -99,9 +99,9 @@ static void runRegister(boolean expectToSucceed, String name, JmxRegister jmx) { public List getGroups() { return null; } }; - final RaftPeerId id = RaftPeerId.valueOf(name); - final RaftGroupId groupId = RaftGroupId.randomId(); - final boolean succeeded = RaftServerImpl.registerMBean(id, groupId, mBean, jmx); + final String id = RaftPeerId.valueOf(name).toString(); + final String groupId = RaftGroupId.randomId().toString(); + final boolean succeeded = RaftServerJmxAdapter.registerMBean(id, groupId, mBean, jmx); Assert.assertEquals(expectToSucceed, succeeded); } From 24482561f98e9f9b0555dffbddd92111af303724 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 17 Jan 2024 08:05:38 -0800 Subject: [PATCH 06/76] RATIS-2001. TransactionContext can be wrongly reused. (#1015) --- .../org/apache/ratis/util/Preconditions.java | 6 +++ .../ratis/server/impl/LeaderStateImpl.java | 8 ++-- .../ratis/server/impl/PendingRequest.java | 26 +++++------ .../ratis/server/impl/PendingRequests.java | 44 +++++++++---------- .../ratis/server/impl/RaftServerImpl.java | 17 +++---- .../ratis/server/impl/TransactionManager.java | 15 ++++--- 6 files changed, 58 insertions(+), 58 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java b/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java index 36e647f0f0..c757de2990 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java @@ -20,6 +20,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.function.Supplier; @@ -87,6 +88,11 @@ static void assertSame(Object expected, Object computed, String name) { () -> name + ": expected == " + expected + " but computed == " + computed); } + static void assertEquals(Object expected, Object computed, String name) { + assertTrue(Objects.equals(expected, computed), + () -> name + ": expected == " + expected + " but computed == " + computed); + } + static void assertNull(Object object, Supplier message) { assertTrue(object == null, message); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index ce19fdafad..043c7319be 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -1197,12 +1197,12 @@ private boolean checkLeaderLease() { && (server.getRaftConf().isSingleton() || lease.isValid()); } - void replyPendingRequest(long logIndex, RaftClientReply reply) { - pendingRequests.replyPendingRequest(logIndex, reply); + void replyPendingRequest(TermIndex termIndex, RaftClientReply reply) { + pendingRequests.replyPendingRequest(termIndex, reply); } - TransactionContext getTransactionContext(long index) { - return pendingRequests.getTransactionContext(index); + TransactionContext getTransactionContext(TermIndex termIndex) { + return pendingRequests.getTransactionContext(termIndex); } long[] getFollowerNextIndices() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java index a0b96cc3d2..06a3a7b3ce 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java @@ -17,28 +17,28 @@ */ package org.apache.ratis.server.impl; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto.TypeCase; import org.apache.ratis.proto.RaftProtos.CommitInfoProto; import org.apache.ratis.protocol.*; import org.apache.ratis.protocol.exceptions.NotLeaderException; -import org.apache.ratis.server.raftlog.RaftLog; +import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; import java.util.Collection; +import java.util.Objects; import java.util.concurrent.CompletableFuture; -public class PendingRequest implements Comparable { - private final long index; +class PendingRequest { + private final TermIndex termIndex; private final RaftClientRequest request; private final TransactionContext entry; private final CompletableFuture futureToComplete = new CompletableFuture<>(); private final CompletableFuture futureToReturn; - PendingRequest(long index, RaftClientRequest request, TransactionContext entry) { - this.index = index; + PendingRequest(RaftClientRequest request, TransactionContext entry) { + this.termIndex = entry == null? null: TermIndex.valueOf(entry.getLogEntry()); this.request = request; this.entry = entry; if (request.is(TypeCase.FORWARD)) { @@ -49,7 +49,7 @@ public class PendingRequest implements Comparable { } PendingRequest(SetConfigurationRequest request) { - this(RaftLog.INVALID_LOG_INDEX, request, null); + this(request, null); } RaftClientReply convert(RaftClientRequest q, RaftClientReply p) { @@ -63,8 +63,8 @@ RaftClientReply convert(RaftClientRequest q, RaftClientReply p) { .build(); } - long getIndex() { - return index; + TermIndex getTermIndex() { + return Objects.requireNonNull(termIndex, "termIndex"); } RaftClientRequest getRequest() { @@ -101,14 +101,8 @@ TransactionContext setNotLeaderException(NotLeaderException nle, Collection {}", name, index, p); + synchronized PendingRequest put(Permit permit, PendingRequest p) { + LOG.debug("{}: PendingRequests.put {}", name, p); final Permit removed = permits.remove(permit); if (removed == null) { return null; } Preconditions.assertTrue(removed == permit); - final PendingRequest previous = map.put(index, p); + final PendingRequest previous = map.put(p.getTermIndex(), p); Preconditions.assertTrue(previous == null); return p; } - PendingRequest get(long index) { - final PendingRequest r = map.get(index); - LOG.debug("{}: PendingRequests.get {} returns {}", name, index, r); + PendingRequest get(TermIndex termIndex) { + final PendingRequest r = map.get(termIndex); + LOG.debug("{}: PendingRequests.get {} returns {}", name, termIndex, r); return r; } - PendingRequest remove(long index) { - final PendingRequest r = map.remove(index); - LOG.debug("{}: PendingRequests.remove {} returns {}", name, index, r); + PendingRequest remove(TermIndex termIndex) { + final PendingRequest r = map.remove(termIndex); + LOG.debug("{}: PendingRequests.remove {} returns {}", name, termIndex, r); if (r == null) { return null; } @@ -193,7 +194,7 @@ Collection setNotLeaderException(NotLeaderException nle, LOG.debug("{}: PendingRequests.setNotLeaderException", name); final List transactions = new ArrayList<>(map.size()); for(;;) { - final Iterator i = map.keySet().iterator(); + final Iterator i = map.keySet().iterator(); if (!i.hasNext()) { // the map is empty return transactions; } @@ -232,11 +233,8 @@ Permit tryAcquire(Message message) { } PendingRequest add(Permit permit, RaftClientRequest request, TransactionContext entry) { - // externally synced for now - final long index = entry.getLogEntry().getIndex(); - LOG.debug("{}: addPendingRequest at index={}, request={}", name, index, request); - final PendingRequest pending = new PendingRequest(index, request, entry); - return pendingRequests.put(permit, index, pending); + final PendingRequest pending = new PendingRequest(request, entry); + return pendingRequests.put(permit, pending); } PendingRequest addConfRequest(SetConfigurationRequest request) { @@ -265,17 +263,17 @@ void failSetConfiguration(RaftException e) { pendingSetConf = null; } - TransactionContext getTransactionContext(long index) { - PendingRequest pendingRequest = pendingRequests.get(index); + TransactionContext getTransactionContext(TermIndex termIndex) { + final PendingRequest pendingRequest = pendingRequests.get(termIndex); // it is possible that the pendingRequest is null if this peer just becomes // the new leader and commits transactions received by the previous leader return pendingRequest != null ? pendingRequest.getEntry() : null; } - void replyPendingRequest(long index, RaftClientReply reply) { - final PendingRequest pending = pendingRequests.remove(index); + void replyPendingRequest(TermIndex termIndex, RaftClientReply reply) { + final PendingRequest pending = pendingRequests.remove(termIndex); if (pending != null) { - Preconditions.assertTrue(pending.getIndex() == index); + Preconditions.assertEquals(termIndex, pending.getTermIndex(), "termIndex"); pending.setReply(reply); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 37d7f300c9..de6491364c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1800,7 +1800,7 @@ void submitUpdateCommitEvent() { * from which we will get transaction result later */ private CompletableFuture replyPendingRequest( - ClientInvocationId invocationId, long logIndex, CompletableFuture stateMachineFuture) { + ClientInvocationId invocationId, TermIndex termIndex, CompletableFuture stateMachineFuture) { // update the retry cache final CacheEntry cacheEntry = retryCache.getOrCreateEntry(invocationId); Objects.requireNonNull(cacheEntry , "cacheEntry == null"); @@ -1812,8 +1812,8 @@ private CompletableFuture replyPendingRequest( } return stateMachineFuture.whenComplete((reply, exception) -> { - transactionManager.remove(logIndex); - final RaftClientReply.Builder b = newReplyBuilder(invocationId, logIndex); + transactionManager.remove(termIndex); + final RaftClientReply.Builder b = newReplyBuilder(invocationId, termIndex.getIndex()); final RaftClientReply r; if (exception == null) { r = b.setSuccess().setMessage(reply).build(); @@ -1825,7 +1825,7 @@ private CompletableFuture replyPendingRequest( } // update pending request - role.getLeaderState().ifPresent(leader -> leader.replyPendingRequest(logIndex, r)); + role.getLeaderState().ifPresent(leader -> leader.replyPendingRequest(termIndex, r)); cacheEntry.updateResult(r); }); } @@ -1835,18 +1835,19 @@ TransactionContext getTransactionContext(LogEntryProto entry, Boolean createNew) return null; } + final TermIndex termIndex = TermIndex.valueOf(entry); final Optional leader = getRole().getLeaderState(); if (leader.isPresent()) { - final TransactionContext context = leader.get().getTransactionContext(entry.getIndex()); + final TransactionContext context = leader.get().getTransactionContext(termIndex); if (context != null) { return context; } } if (!createNew) { - return transactionManager.get(entry.getIndex()); + return transactionManager.get(termIndex); } - return transactionManager.computeIfAbsent(entry.getIndex(), + return transactionManager.computeIfAbsent(termIndex, // call startTransaction only once MemoizedSupplier.valueOf(() -> stateMachine.startTransaction(entry, getInfo().getCurrentRole()))); } @@ -1872,7 +1873,7 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf trx = stateMachine.applyTransactionSerial(trx); final CompletableFuture stateMachineFuture = stateMachine.applyTransaction(trx); - return replyPendingRequest(invocationId, next.getIndex(), stateMachineFuture); + return replyPendingRequest(invocationId, TermIndex.valueOf(next), stateMachineFuture); } catch (Exception e) { throw new RaftLogIOException(e); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java index aa989cf989..283900fb62 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java @@ -17,6 +17,7 @@ */ package org.apache.ratis.server.impl; +import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.statemachine.TransactionContext; import java.util.Optional; @@ -28,17 +29,17 @@ * Managing {@link TransactionContext}. */ class TransactionManager { - private final ConcurrentMap> contexts = new ConcurrentHashMap<>(); + private final ConcurrentMap> contexts = new ConcurrentHashMap<>(); - TransactionContext get(long index) { - return Optional.ofNullable(contexts.get(index)).map(Supplier::get).orElse(null); + TransactionContext get(TermIndex termIndex) { + return Optional.ofNullable(contexts.get(termIndex)).map(Supplier::get).orElse(null); } - TransactionContext computeIfAbsent(long index, Supplier constructor) { - return contexts.computeIfAbsent(index, i -> constructor).get(); + TransactionContext computeIfAbsent(TermIndex termIndex, Supplier constructor) { + return contexts.computeIfAbsent(termIndex, i -> constructor).get(); } - void remove(long index) { - contexts.remove(index); + void remove(TermIndex termIndex) { + contexts.remove(termIndex); } } \ No newline at end of file From eae0d18eb3a33da5ddf4a20a79cd24c95c8c9e7a Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 18 Jan 2024 01:41:37 +0100 Subject: [PATCH 07/76] RATIS-2005. Fix URF_UNREAD_FIELD in MetricServerCallListener (#1018) --- ratis-grpc/pom.xml | 7 --- .../server/MetricServerCallListener.java | 43 ------------------- .../server/MetricServerInterceptor.java | 3 +- 3 files changed, 1 insertion(+), 52 deletions(-) delete mode 100644 ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/intercept/server/MetricServerCallListener.java diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index 3352ce16fb..71377b5efc 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -80,12 +80,5 @@ mockito-core test - - - com.github.spotbugs - spotbugs-annotations - provided - true - diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/intercept/server/MetricServerCallListener.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/intercept/server/MetricServerCallListener.java deleted file mode 100644 index 77ae4c8f35..0000000000 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/intercept/server/MetricServerCallListener.java +++ /dev/null @@ -1,43 +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.ratis.grpc.metrics.intercept.server; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import org.apache.ratis.thirdparty.io.grpc.ForwardingServerCallListener; -import org.apache.ratis.thirdparty.io.grpc.ServerCall; -import org.apache.ratis.grpc.metrics.MessageMetrics; - -@SuppressFBWarnings("URF_UNREAD_FIELD") -public class MetricServerCallListener extends ForwardingServerCallListener { - private final ServerCall.Listener delegate; - private MessageMetrics metrics; - - MetricServerCallListener( - ServerCall.Listener delegate, - MessageMetrics metrics - ){ - this.delegate = delegate; - this.metrics = metrics; - } - - @Override - protected ServerCall.Listener delegate() { - return delegate; - } -} diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/intercept/server/MetricServerInterceptor.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/intercept/server/MetricServerInterceptor.java index 7e20946add..e7c1dd17ce 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/intercept/server/MetricServerInterceptor.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/intercept/server/MetricServerInterceptor.java @@ -75,8 +75,7 @@ public ServerCall.Listener interceptCall( } String metricNamePrefix = getMethodMetricPrefix(method); ServerCall monitoringCall = new MetricServerCall<>(call, metricNamePrefix, metrics); - return new MetricServerCallListener<>( - next.startCall(monitoringCall, requestHeaders), metrics); + return next.startCall(monitoringCall, requestHeaders); } @Override From 859a41074c9d896563c95d1451a44267276d72be Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 18 Jan 2024 02:01:47 +0100 Subject: [PATCH 08/76] RATIS-2006. Fix DM_EXIT in filestore Client (#1019) --- ratis-examples/pom.xml | 7 ------- .../org/apache/ratis/examples/filestore/cli/Client.java | 5 +---- .../apache/ratis/examples/filestore/cli/DataStream.java | 5 +++-- .../org/apache/ratis/examples/filestore/cli/LoadGen.java | 2 +- 4 files changed, 5 insertions(+), 14 deletions(-) diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index 5e899f8fff..4078f6260f 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -122,13 +122,6 @@ junit test - - - com.github.spotbugs - spotbugs-annotations - provided - true - diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java index 086b791e49..1856fc9b5d 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java @@ -18,7 +18,6 @@ package org.apache.ratis.examples.filestore.cli; import com.beust.jcommander.Parameter; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.ratis.RaftConfigKeys; import org.apache.ratis.client.RaftClient; import org.apache.ratis.client.RaftClientConfigKeys; @@ -143,12 +142,10 @@ public List getClients(RaftProperties raftProperties) { } - @SuppressFBWarnings("DM_EXIT") - protected void stop(List clients) throws IOException { + protected void close(List clients) throws IOException { for (FileStoreClient client : clients) { client.close(); } - System.exit(0); } public String getPath(String fileName) { diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/DataStream.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/DataStream.java index 4e4b4dbc64..b15b860570 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/DataStream.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/DataStream.java @@ -116,7 +116,8 @@ private boolean checkParam() { @Override protected void operation(List clients) throws IOException, ExecutionException, InterruptedException { if (!checkParam()) { - stop(clients); + close(clients); + return; } final ExecutorService executor = Executors.newFixedThreadPool(getNumThread()); @@ -136,7 +137,7 @@ protected void operation(List clients) throws IOException, Exec System.out.println("Total data written: " + totalWrittenBytes + " bytes"); System.out.println("Total time taken: " + (endTime - startTime) + " millis"); - stop(clients); + close(clients); } private Map>>> streamWrite( diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/LoadGen.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/LoadGen.java index 8225df9eef..42ffac4e1c 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/LoadGen.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/LoadGen.java @@ -64,7 +64,7 @@ protected void operation(List clients) throws IOException, Exec System.out.println("Total data written: " + totalWrittenBytes + " bytes"); System.out.println("Total time taken: " + (endTime - startTime) + " millis"); - stop(clients); + close(clients); } long write(FileChannel in, long offset, FileStoreClient fileStoreClient, String path, From 14eb0bf057c89954a05e7d5c9915f71074591d47 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 18 Jan 2024 02:27:33 +0100 Subject: [PATCH 09/76] RATIS-2002. Fix findbugs warnings in LogSegmentPath (#1017) --- .../raftlog/segmented/LogSegmentPath.java | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegmentPath.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegmentPath.java index e1de7a5431..45dc9a3351 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegmentPath.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegmentPath.java @@ -17,7 +17,6 @@ */ package org.apache.ratis.server.raftlog.segmented; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.util.Preconditions; import org.slf4j.Logger; @@ -30,6 +29,7 @@ import java.util.ArrayList; import java.util.Comparator; import java.util.List; +import java.util.Objects; import java.util.Optional; import java.util.regex.Matcher; @@ -38,9 +38,12 @@ * * This is a value-based class. */ -public final class LogSegmentPath implements Comparable { +public final class LogSegmentPath { static final Logger LOG = LoggerFactory.getLogger(LogSegmentPath.class); + private static final Comparator COMPARATOR = + Comparator.comparing(LogSegmentPath::getStartEnd); + private final Path path; private final LogSegmentStartEnd startEnd; @@ -57,12 +60,6 @@ public LogSegmentStartEnd getStartEnd() { return startEnd; } - @Override - @SuppressFBWarnings("EQ_COMPARETO_USE_OBJECT_EQUALS") - public int compareTo(LogSegmentPath that) { - return Comparator.comparing(LogSegmentPath::getStartEnd).compare(this, that); - } - @Override public String toString() { return path+ "(" + startEnd + ")"; @@ -84,7 +81,7 @@ private static List getLogSegmentPaths(Path dir) throws IOExcept Optional.ofNullable(matchLogSegment(path)).ifPresent(list::add); } } - list.sort(Comparator.naturalOrder()); + list.sort(COMPARATOR); return list; } @@ -100,9 +97,9 @@ public static LogSegmentPath matchLogSegment(Path path) { return Optional.ofNullable(matchCloseSegment(path)).orElseGet(() -> matchOpenSegment(path)); } - @SuppressFBWarnings("NP_NULL_ON_SOME_PATH") private static LogSegmentPath matchCloseSegment(Path path) { - final Matcher matcher = LogSegmentStartEnd.getClosedSegmentPattern().matcher(path.getFileName().toString()); + final String fileName = String.valueOf(Objects.requireNonNull(path).getFileName()); + final Matcher matcher = LogSegmentStartEnd.getClosedSegmentPattern().matcher(fileName); if (matcher.matches()) { Preconditions.assertTrue(matcher.groupCount() == 2); return newInstance(path, matcher.group(1), matcher.group(2)); @@ -110,9 +107,9 @@ private static LogSegmentPath matchCloseSegment(Path path) { return null; } - @SuppressFBWarnings("NP_NULL_ON_SOME_PATH") private static LogSegmentPath matchOpenSegment(Path path) { - final Matcher matcher = LogSegmentStartEnd.getOpenSegmentPattern().matcher(path.getFileName().toString()); + final String fileName = String.valueOf(Objects.requireNonNull(path).getFileName()); + final Matcher matcher = LogSegmentStartEnd.getOpenSegmentPattern().matcher(fileName); if (matcher.matches()) { if (path.toFile().length() > 0L) { return newInstance(path, matcher.group(1), null); From 5d4ec6933f538d0bf2c40483ad3d37173c82a1ca Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Thu, 18 Jan 2024 00:23:22 -0800 Subject: [PATCH 10/76] RATIS-1990. Refactor appendEntries processing to support reference count (#1011) --- .../server/GrpcServerProtocolService.java | 3 +- .../RaftServerAsynchronousProtocol.java | 31 ++++++++++++++-- .../server/raftlog/RaftLogSequentialOps.java | 21 +++++++++-- .../ratis/server/impl/RaftServerImpl.java | 21 ++++++----- .../ratis/server/impl/RaftServerProxy.java | 14 +++++--- .../ratis/server/raftlog/RaftLogBase.java | 14 ++++++-- .../server/raftlog/memory/MemoryRaftLog.java | 35 +++++++++++++------ .../raftlog/segmented/SegmentedRaftLog.java | 15 ++++++-- .../raftlog/memory/MemoryRaftLogTest.java | 9 ++--- .../raftlog/segmented/TestCacheEviction.java | 5 +-- .../segmented/TestSegmentedRaftLog.java | 8 +++-- 11 files changed, 135 insertions(+), 41 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index 766e14321a..dde01c39a3 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -30,6 +30,7 @@ import org.apache.ratis.proto.RaftProtos.*; import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceImplBase; import org.apache.ratis.util.ProtoUtils; +import org.apache.ratis.util.ReferenceCountedObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -226,7 +227,7 @@ public StreamObserver appendEntries( RaftServerProtocol.Op.APPEND_ENTRIES, responseObserver) { @Override CompletableFuture process(AppendEntriesRequestProto request) throws IOException { - return server.appendEntriesAsync(request); + return server.appendEntriesAsync(ReferenceCountedObject.wrap(request)); } @Override diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java index 8a904069ba..1244e72545 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java @@ -22,14 +22,41 @@ import org.apache.ratis.proto.RaftProtos.ReadIndexReplyProto; import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto; import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; +import org.apache.ratis.util.ReferenceCountedObject; import java.io.IOException; import java.util.concurrent.CompletableFuture; public interface RaftServerAsynchronousProtocol { - CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) - throws IOException; + /** + * It is recommended to override {@link #appendEntriesAsync(ReferenceCountedObject)} instead. + * Then, it does not have to override this method. + */ + default CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) + throws IOException { + throw new UnsupportedOperationException(); + } + + /** + * A referenced counted request is submitted from a client for processing. + * Implementations of this method should retain the request, process it and then release it. + * The request may be retained even after the future returned by this method has completed. + * + * @return a future of the reply + * @see ReferenceCountedObject + */ + default CompletableFuture appendEntriesAsync( + ReferenceCountedObject requestRef) throws IOException { + // Default implementation for backward compatibility. + try { + return appendEntriesAsync(requestRef.retain()) + .whenComplete((r, e) -> requestRef.release()); + } catch (Exception e) { + requestRef.release(); + throw e; + } + } CompletableFuture readIndexAsync(ReadIndexRequestProto request) throws IOException; diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java index 7b9f42b6bd..e4523cd935 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java @@ -22,8 +22,10 @@ import org.apache.ratis.server.RaftConfiguration; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.function.CheckedSupplier; +import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; import java.util.Arrays; import java.util.List; @@ -132,21 +134,36 @@ default CompletableFuture appendEntry(LogEntryProto entry, TransactionCont /** * The same as append(Arrays.asList(entries)). * - * @deprecated use {@link #append(List)} + * @deprecated use {@link #append(ReferenceCountedObject)}. */ @Deprecated default List> append(LogEntryProto... entries) { return append(Arrays.asList(entries)); } + /** + * @deprecated use {@link #append(ReferenceCountedObject)}. + */ + @Deprecated + default List> append(List entries) { + throw new UnsupportedOperationException(); + } + /** * Append asynchronously all the given log entries. * Used by the followers. * * If an existing entry conflicts with a new one (same index but different terms), * delete the existing entry and all entries that follow it (§5.3). + * + * A reference counter is also submitted. + * For each entry, implementations of this method should retain the counter, process it and then release. */ - List> append(List entries); + default List> append(ReferenceCountedObject> entriesRef) { + try(UncheckedAutoCloseableSupplier> entries = entriesRef.retainAndReleaseOnClose()) { + return append(entries.get()); + } + } /** * Truncate asynchronously the log entries till the given index (inclusively). diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index de6491364c..8ad835474c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1460,27 +1460,30 @@ private void validateEntries(long expectedTerm, TermIndex previous, public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto r) throws IOException { try { - return appendEntriesAsync(r).join(); + return appendEntriesAsync(ReferenceCountedObject.wrap(r)).join(); } catch (CompletionException e) { throw IOUtils.asIOException(JavaUtils.unwrapCompletionException(e)); } } @Override - public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto r) - throws IOException { + public CompletableFuture appendEntriesAsync( + ReferenceCountedObject requestRef) throws IOException { + final AppendEntriesRequestProto r = requestRef.retain(); final RaftRpcRequestProto request = r.getServerRequest(); final List entries = r.getEntriesList(); final TermIndex previous = r.hasPreviousLog()? TermIndex.valueOf(r.getPreviousLog()) : null; final RaftPeerId requestorId = RaftPeerId.valueOf(request.getRequestorId()); - preAppendEntriesAsync(requestorId, ProtoUtils.toRaftGroupId(request.getRaftGroupId()), r.getLeaderTerm(), - previous, r.getLeaderCommit(), r.getInitializing(), entries); try { + preAppendEntriesAsync(requestorId, ProtoUtils.toRaftGroupId(request.getRaftGroupId()), r.getLeaderTerm(), + previous, r.getLeaderCommit(), r.getInitializing(), entries); return appendEntriesAsync(requestorId, r.getLeaderTerm(), previous, r.getLeaderCommit(), - request.getCallId(), r.getInitializing(), r.getCommitInfosList(), entries); + request.getCallId(), r.getInitializing(), r.getCommitInfosList(), entries, requestRef) + .whenComplete((reply, e) -> requestRef.release()); } catch(Exception t) { LOG.error("{}: Failed appendEntriesAsync {}", getMemberId(), r, t); + requestRef.release(); throw t; } } @@ -1554,7 +1557,8 @@ ExecutorService getServerExecutor() { @SuppressWarnings("checkstyle:parameternumber") private CompletableFuture appendEntriesAsync( RaftPeerId leaderId, long leaderTerm, TermIndex previous, long leaderCommit, long callId, boolean initializing, - List commitInfos, List entries) throws IOException { + List commitInfos, List entries, + ReferenceCountedObject requestRef) throws IOException { final boolean isHeartbeat = entries.isEmpty(); logAppendEntries(isHeartbeat, () -> getMemberId() + ": receive appendEntries(" + leaderId + ", " + leaderTerm + ", " @@ -1612,8 +1616,9 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), state.updateConfiguration(entries); } + final List> futures = entries.isEmpty() ? Collections.emptyList() - : state.getLog().append(entries); + : state.getLog().append(requestRef.delegate(entries)); commitInfos.forEach(commitInfoCache::update); CodeInjectionForTesting.execute(LOG_SYNC, getId(), null); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java index cb7918e51d..589c7eeb14 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java @@ -645,10 +645,16 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ } @Override - public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) { - final RaftGroupId groupId = ProtoUtils.toRaftGroupId(request.getServerRequest().getRaftGroupId()); - return getImplFuture(groupId) - .thenCompose(impl -> impl.executeSubmitServerRequestAsync(() -> impl.appendEntriesAsync(request))); + public CompletableFuture appendEntriesAsync( + ReferenceCountedObject requestRef) { + AppendEntriesRequestProto request = requestRef.retain(); + try { + final RaftGroupId groupId = ProtoUtils.toRaftGroupId(request.getServerRequest().getRaftGroupId()); + return getImplFuture(groupId) + .thenCompose(impl -> impl.executeSubmitServerRequestAsync(() -> impl.appendEntriesAsync(requestRef))); + } finally { + requestRef.release(); + } } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 708b499cd3..0b6fa15b6f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -31,7 +31,9 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.OpenCloseState; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; import java.io.IOException; import java.util.List; @@ -354,11 +356,19 @@ public final CompletableFuture appendEntry(LogEntryProto entry, Transactio protected abstract CompletableFuture appendEntryImpl(LogEntryProto entry, TransactionContext context); @Override - public final List> append(List entries) { + public final List> append(ReferenceCountedObject> entries) { return runner.runSequentially(() -> appendImpl(entries)); } - protected abstract List> appendImpl(List entries); + protected List> appendImpl(List entries) { + throw new UnsupportedOperationException(); + } + + protected List> appendImpl(ReferenceCountedObject> entriesRef) { + try(UncheckedAutoCloseableSupplier> entries = entriesRef.retainAndReleaseOnClose()) { + return appendImpl(entries.get()); + } + } @Override public String toString() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index ebb1e27d77..7435bb178d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -28,6 +28,7 @@ import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.AutoCloseableLock; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import java.io.IOException; import java.util.ArrayList; @@ -35,6 +36,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import java.util.function.LongSupplier; /** @@ -42,10 +44,10 @@ */ public class MemoryRaftLog extends RaftLogBase { static class EntryList { - private final List entries = new ArrayList<>(); + private final List> entries = new ArrayList<>(); LogEntryProto get(int i) { - return i >= 0 && i < entries.size() ? entries.get(i) : null; + return i >= 0 && i < entries.size() ? entries.get(i).get() : null; } TermIndex getTermIndex(int i) { @@ -62,18 +64,25 @@ int size() { void truncate(int index) { if (entries.size() > index) { - entries.subList(index, entries.size()).clear(); + clear(index, entries.size()); } } void purge(int index) { if (entries.size() > index) { - entries.subList(0, index).clear(); + clear(0, index); } } - void add(LogEntryProto entry) { - entries.add(entry); + void clear(int from, int to) { + List> subList = entries.subList(from, to); + subList.forEach(ReferenceCountedObject::release); + subList.clear(); + } + + void add(ReferenceCountedObject entryRef) { + entryRef.retain(); + entries.add(entryRef); } } @@ -170,7 +179,9 @@ protected CompletableFuture appendEntryImpl(LogEntryProto entry, Transacti checkLogState(); try(AutoCloseableLock writeLock = writeLock()) { validateLogEntry(entry); - entries.add(entry); + Function> wrap = context != null ? + context::wrap : ReferenceCountedObject::wrap; + entries.add(wrap.apply(entry)); } return CompletableFuture.completedFuture(entry.getIndex()); } @@ -181,12 +192,14 @@ public long getStartIndex() { } @Override - public List> appendImpl(List logEntryProtos) { + public List> appendImpl(ReferenceCountedObject> entriesRef) { checkLogState(); + final List logEntryProtos = entriesRef.retain(); if (logEntryProtos == null || logEntryProtos.isEmpty()) { + entriesRef.release(); return Collections.emptyList(); } - try(AutoCloseableLock writeLock = writeLock()) { + try (AutoCloseableLock writeLock = writeLock()) { // Before truncating the entries, we first need to check if some // entries are duplicated. If the leader sends entry 6, entry 7, then // entry 6 again, without this check the follower may truncate entry 7 @@ -214,10 +227,12 @@ public List> appendImpl(List logEntryProt } for (int i = index; i < logEntryProtos.size(); i++) { LogEntryProto logEntryProto = logEntryProtos.get(i); - this.entries.add(logEntryProto); + entries.add(entriesRef.delegate(logEntryProto)); futures.add(CompletableFuture.completedFuture(logEntryProto.getIndex())); } return futures; + } finally { + entriesRef.release(); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 1cfb5933f8..b4e9589658 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -36,6 +36,7 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; +import org.apache.ratis.statemachine.impl.TransactionContextImpl; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.AutoCloseableLock; import org.apache.ratis.util.AwaitToRun; @@ -454,12 +455,14 @@ private boolean isSegmentFull(LogSegment segment, LogEntryProto entry) { } @Override - public List> appendImpl(List entries) { + protected List> appendImpl(ReferenceCountedObject> entriesRef) { checkLogState(); + final List entries = entriesRef.retain(); if (entries == null || entries.isEmpty()) { + entriesRef.release(); return Collections.emptyList(); } - try(AutoCloseableLock writeLock = writeLock()) { + try (AutoCloseableLock writeLock = writeLock()) { final TruncateIndices ti = cache.computeTruncateIndices(server::notifyTruncatedLogEntry, entries); final long truncateIndex = ti.getTruncateIndex(); final int index = ti.getArrayIndex(); @@ -474,9 +477,15 @@ public List> appendImpl(List entries) { } for (int i = index; i < entries.size(); i++) { final LogEntryProto entry = entries.get(i); - futures.add(appendEntry(entry, server.getTransactionContext(entry, true))); + TransactionContextImpl transactionContext = (TransactionContextImpl) server.getTransactionContext(entry, true); + if (transactionContext != null) { + transactionContext.setDelegatedRef(entriesRef); + } + futures.add(appendEntry(entry, transactionContext)); } return futures; + } finally { + entriesRef.release(); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java index 5d8d090a38..503de34536 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java @@ -33,6 +33,7 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.Slf4jUtils; import org.junit.Test; import org.slf4j.event.Level; @@ -56,11 +57,11 @@ public void testEntryDoNotPerformTruncation() throws Exception { List entries1 = new ArrayList<>(); entries1.add(LogEntryProto.newBuilder().setIndex(0).setTerm(0).build()); entries1.add(LogEntryProto.newBuilder().setIndex(1).setTerm(0).build()); - raftLog.append(entries1).forEach(CompletableFuture::join); + raftLog.append(ReferenceCountedObject.wrap(entries1)).forEach(CompletableFuture::join); List entries2 = new ArrayList<>(); entries2.add(LogEntryProto.newBuilder().setIndex(0).setTerm(0).build()); - raftLog.append(entries2).forEach(CompletableFuture::join); + raftLog.append(ReferenceCountedObject.wrap(entries2)).forEach(CompletableFuture::join); final LogEntryHeader[] termIndices = raftLog.getEntries(0, 10); assertEquals(2, termIndices.length); @@ -84,11 +85,11 @@ public void testEntryPerformTruncation() throws Exception { List entries1 = new ArrayList<>(); entries1.add(LogEntryProto.newBuilder().setIndex(0).setTerm(0).build()); entries1.add(LogEntryProto.newBuilder().setIndex(1).setTerm(0).build()); - raftLog.append(entries1).forEach(CompletableFuture::join); + raftLog.append(ReferenceCountedObject.wrap(entries1)).forEach(CompletableFuture::join); List entries2 = new ArrayList<>(); entries2.add(LogEntryProto.newBuilder().setIndex(0).setTerm(2).build()); - raftLog.append(entries2).forEach(CompletableFuture::join); + raftLog.append(ReferenceCountedObject.wrap(entries2)).forEach(CompletableFuture::join); final LogEntryHeader[] termIndices = raftLog.getEntries(0, 10); assertEquals(1, termIndices.length); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java index 996f7ef527..675df51d6b 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java @@ -38,6 +38,7 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.junit.Assert; import org.junit.Test; @@ -174,7 +175,7 @@ public void testEvictionInSegmentedLog() throws Exception { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); List slist = TestSegmentedRaftLog.prepareRanges(0, maxCachedNum, 7, 0); List entries = generateEntries(slist); - raftLog.append(entries).forEach(CompletableFuture::join); + raftLog.append(ReferenceCountedObject.wrap(entries)).forEach(CompletableFuture::join); // check the current cached segment number: the last segment is still open Assert.assertEquals(maxCachedNum - 1, @@ -184,7 +185,7 @@ public void testEvictionInSegmentedLog() throws Exception { Mockito.when(info.getFollowerNextIndices()).thenReturn(new long[]{21, 40, 40}); slist = TestSegmentedRaftLog.prepareRanges(maxCachedNum, maxCachedNum + 2, 7, 7 * maxCachedNum); entries = generateEntries(slist); - raftLog.append(entries).forEach(CompletableFuture::join); + raftLog.append(ReferenceCountedObject.wrap(entries)).forEach(CompletableFuture::join); // check the cached segment number again. since the slowest follower is on // index 21, the eviction should happen and evict 3 segments diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 38fa45e6fa..3d5d5f87d0 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -43,6 +43,7 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.util.LifeCycle; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; @@ -551,7 +552,7 @@ public void testAppendEntriesWithInconsistency() throws Exception { LOG.info("newEntries[0] = {}", newEntries.get(0)); final int last = newEntries.size() - 1; LOG.info("newEntries[{}] = {}", last, newEntries.get(last)); - raftLog.append(newEntries).forEach(CompletableFuture::join); + raftLog.append(ReferenceCountedObject.wrap(newEntries)).forEach(CompletableFuture::join); checkFailedEntries(entries, 650, retryCache); checkEntries(raftLog, entries, 0, 650); @@ -710,8 +711,9 @@ public void testAsyncFlushPerf1() throws Exception { long start = System.nanoTime(); for (int i = 0; i < entries.size(); i += 5) { // call append API - futures.add(raftLog.append(Arrays.asList( - entries.get(i), entries.get(i + 1), entries.get(i + 2), entries.get(i + 3), entries.get(i + 4)))); + List entries1 = Arrays.asList( + entries.get(i), entries.get(i + 1), entries.get(i + 2), entries.get(i + 3), entries.get(i + 4)); + futures.add(raftLog.append(ReferenceCountedObject.wrap(entries1))); } for (List> futureList: futures) { futureList.forEach(CompletableFuture::join); From 82f7b3b851211e35056ebf592a90829649568cdb Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 18 Jan 2024 17:18:45 +0100 Subject: [PATCH 11/76] RATIS-2003. Fix IT_NO_SUCH_ELEMENT in InstallSnapshotRequests (#1016) --- .../leader/InstallSnapshotRequests.java | 102 +++++++++--------- 1 file changed, 50 insertions(+), 52 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java index cdb6603c29..6300ea483c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java @@ -17,7 +17,6 @@ */ package org.apache.ratis.server.leader; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.ratis.proto.RaftProtos.FileChunkProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto.SnapshotChunkProto; @@ -33,6 +32,7 @@ import java.io.IOException; import java.nio.file.Path; import java.util.Iterator; +import java.util.List; import java.util.NoSuchElementException; import java.util.Optional; import java.util.function.Function; @@ -59,14 +59,9 @@ class InstallSnapshotRequests implements Iterable { private final int snapshotChunkMaxSize; /** The total size of snapshot files. */ private final long totalSize; + /** The total number of snapshot files. */ + private final int numFiles; - /** The index of the current request. */ - private int requestIndex = 0; - - /** The index of the current file. */ - private int fileIndex = 0; - /** The current file. */ - private FileChunkReader current; InstallSnapshotRequests(RaftServer.Division server, RaftPeerId followerId, String requestId, SnapshotInfo snapshot, int snapshotChunkMaxSize) { @@ -75,8 +70,10 @@ class InstallSnapshotRequests implements Iterable { this.requestId = requestId; this.snapshot = snapshot; this.snapshotChunkMaxSize = snapshotChunkMaxSize; - this.totalSize = snapshot.getFiles().stream().mapToLong(FileInfo::getFileSize).reduce(Long::sum).orElseThrow( + final List files = snapshot.getFiles(); + this.totalSize = files.stream().mapToLong(FileInfo::getFileSize).reduce(Long::sum).orElseThrow( () -> new IllegalStateException("Failed to compute total size for snapshot " + snapshot)); + this.numFiles = files.size(); final File snapshotDir = server.getStateMachine().getStateMachineStorage().getSnapshotDir(); final Function relativize; @@ -96,67 +93,68 @@ class InstallSnapshotRequests implements Iterable { @Override public Iterator iterator() { - return new Iterator() { - @Override - public boolean hasNext() { - return fileIndex < snapshot.getFiles().size(); - } - - @Override - @SuppressFBWarnings("IT_NO_SUCH_ELEMENT") - public InstallSnapshotRequestProto next() { - return nextInstallSnapshotRequestProto(); - } - }; + return new Iter(); } - private InstallSnapshotRequestProto nextInstallSnapshotRequestProto() { - final int numFiles = snapshot.getFiles().size(); - if (fileIndex >= numFiles) { - throw new NoSuchElementException("fileIndex = " + fileIndex + " >= numFiles = " + numFiles); + private class Iter implements Iterator { + + /** The index of the current request. */ + private int requestIndex = 0; + /** The index of the current file. */ + private int fileIndex = 0; + /** The current file. */ + private FileChunkReader current; + + @Override + public boolean hasNext() { + return fileIndex < numFiles; } - final FileInfo info = snapshot.getFiles().get(fileIndex); - try { - if (current == null) { - current = new FileChunkReader(info, getRelativePath.apply(info)); - } - final FileChunkProto chunk = current.readFileChunk(snapshotChunkMaxSize); - if (chunk.getDone()) { - current.close(); - current = null; - fileIndex++; + + @Override + public InstallSnapshotRequestProto next() { + if (!hasNext()) { + throw new NoSuchElementException("fileIndex = " + fileIndex + " >= numFiles = " + numFiles); } - final boolean done = fileIndex == numFiles && chunk.getDone(); - return newInstallSnapshotRequest(chunk, done); - } catch (IOException e) { - if (current != null) { - try { + final FileInfo info = snapshot.getFiles().get(fileIndex); + try { + if (current == null) { + current = new FileChunkReader(info, getRelativePath.apply(info)); + } + final FileChunkProto chunk = current.readFileChunk(snapshotChunkMaxSize); + if (chunk.getDone()) { current.close(); current = null; - } catch (IOException ignored) { + fileIndex++; + } + + final boolean done = fileIndex == numFiles && chunk.getDone(); + return newInstallSnapshotRequest(chunk, done); + } catch (IOException e) { + if (current != null) { + try { + current.close(); + current = null; + } catch (IOException ignored) { + } } + throw new IllegalStateException("Failed to iterate installSnapshot requests: " + this, e); } - throw new IllegalStateException("Failed to iterate installSnapshot requests: " + this, e); } - } - private InstallSnapshotRequestProto newInstallSnapshotRequest(FileChunkProto chunk, boolean done) { - synchronized (server) { - final SnapshotChunkProto.Builder b = LeaderProtoUtils.toSnapshotChunkProtoBuilder( - requestId, requestIndex++, snapshot.getTermIndex(), chunk, totalSize, done); - return LeaderProtoUtils.toInstallSnapshotRequestProto(server, followerId, b); + private InstallSnapshotRequestProto newInstallSnapshotRequest(FileChunkProto chunk, boolean done) { + synchronized (server) { + final SnapshotChunkProto.Builder b = LeaderProtoUtils.toSnapshotChunkProtoBuilder( + requestId, requestIndex++, snapshot.getTermIndex(), chunk, totalSize, done); + return LeaderProtoUtils.toInstallSnapshotRequestProto(server, followerId, b); + } } } - @Override public String toString() { return server.getId() + "->" + followerId + JavaUtils.getClassSimpleName(getClass()) + ": requestId=" + requestId - + ", requestIndex=" + requestIndex - + ", fileIndex=" + fileIndex - + ", currentFile=" + current + ", snapshot=" + snapshot; } } From fc5c7256096c6f5b85308b47a1b2e5242ded4d03 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 18 Jan 2024 21:31:57 +0100 Subject: [PATCH 12/76] RATIS-2004. Fix EQ_COMPARETO_USE_OBJECT_EQUALS in LogSegment (#1020) --- .../server/raftlog/segmented/LogSegment.java | 19 ++++++++++++++----- .../segmented/SegmentedRaftLogCache.java | 6 +++--- 2 files changed, 17 insertions(+), 8 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 1e0ef666e9..0750d2cc8c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -17,7 +17,6 @@ */ package org.apache.ratis.server.raftlog.segmented; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.server.RaftServerConfigKeys.Log.CorruptionPolicy; import org.apache.ratis.server.metrics.SegmentedRaftLogMetrics; @@ -39,6 +38,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -56,7 +56,8 @@ * * This class will be protected by the {@link SegmentedRaftLog}'s read-write lock. */ -public final class LogSegment implements Comparable { +public final class LogSegment { + static final Logger LOG = LoggerFactory.getLogger(LogSegment.class); enum Op { @@ -452,9 +453,17 @@ public String toString() { "log-" + startIndex + "_" + endIndex; } - @Override - @SuppressFBWarnings("EQ_COMPARETO_USE_OBJECT_EQUALS") - public int compareTo(Long l) { + /** Comparator to find index in list of LogSegments. */ + static final Comparator SEGMENT_TO_INDEX_COMPARATOR = (o1, o2) -> { + if (o1 instanceof LogSegment && o2 instanceof Long) { + return ((LogSegment) o1).compareTo((Long) o2); + } else if (o1 instanceof Long && o2 instanceof LogSegment) { + return Integer.compare(0, ((LogSegment) o2).compareTo((Long) o1)); + } + throw new IllegalStateException("Unexpected objects to compare(" + o1 + "," + o2 + ")"); + }; + + private int compareTo(Long l) { return (l >= getStartIndex() && l <= getEndIndex()) ? 0 : (this.getEndIndex() < l ? -1 : 1); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 81f467726f..1d08316fda 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -244,13 +244,13 @@ LogSegment get(int i) { int binarySearch(long index) { try(AutoCloseableLock readLock = readLock()) { - return Collections.binarySearch(segments, index); + return Collections.binarySearch(segments, index, LogSegment.SEGMENT_TO_INDEX_COMPARATOR); } } LogSegment search(long index) { try(AutoCloseableLock readLock = readLock()) { - final int i = Collections.binarySearch(segments, index); + final int i = Collections.binarySearch(segments, index, LogSegment.SEGMENT_TO_INDEX_COMPARATOR); return i < 0? null: segments.get(i); } } @@ -261,7 +261,7 @@ LogEntryHeader[] getTermIndex(long startIndex, long realEnd, LogSegment openSegm long index = startIndex; try(AutoCloseableLock readLock = readLock()) { - searchIndex = Collections.binarySearch(segments, startIndex); + searchIndex = Collections.binarySearch(segments, startIndex, LogSegment.SEGMENT_TO_INDEX_COMPARATOR); if (searchIndex >= 0) { for(int i = searchIndex; i < segments.size() && index < realEnd; i++) { final LogSegment s = segments.get(i); From 82c31eaf39b24ffb31452ec58b3d96dcee9bab7e Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Thu, 18 Jan 2024 22:40:40 -0800 Subject: [PATCH 13/76] RATIS-2009. ReferenceCount should work for all LogEntry types. (#1021) --- .../server/GrpcClientProtocolService.java | 3 ++- .../server/raftlog/RaftLogSequentialOps.java | 16 +++++++++++++--- .../ratis/server/raftlog/RaftLogBase.java | 11 +++++++---- .../server/raftlog/memory/MemoryRaftLog.java | 13 +++++++------ .../raftlog/segmented/SegmentedRaftLog.java | 19 +++++++++---------- .../java/org/apache/ratis/RaftTestUtil.java | 3 ++- 6 files changed, 40 insertions(+), 25 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index e8de4def04..67e75d6063 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -64,7 +64,7 @@ private static class PendingOrderedRequest implements SlidingWindow.ServerSideRe PendingOrderedRequest(ReferenceCountedObject requestRef) { this.requestRef = requestRef; - this.request = requestRef != null ? requestRef.get() : null; + this.request = requestRef != null ? requestRef.retain() : null; } @Override @@ -363,6 +363,7 @@ void processClientRequest(PendingOrderedRequest pending) { final long seq = pending.getSeqNum(); processClientRequest(pending.getRequestRef(), reply -> slidingWindow.receiveReply(seq, reply, this::sendReply)); + pending.getRequestRef().release(); } @Override diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java index e4523cd935..5e274a6959 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java @@ -124,11 +124,21 @@ OUTPUT runSequentially( CompletableFuture appendEntry(LogEntryProto entry); /** - * Append asynchronously an entry. - * Used by the leader. + * @deprecated use {@link #appendEntry(ReferenceCountedObject, TransactionContext)}}. */ + @Deprecated default CompletableFuture appendEntry(LogEntryProto entry, TransactionContext context) { - return appendEntry(entry); + throw new UnsupportedOperationException(); + } + + /** + * Append asynchronously an entry. + * Used for scenarios that there is a ReferenceCountedObject context for resource cleanup when the given entry + * is no longer used/referenced by this log. + */ + default CompletableFuture appendEntry(ReferenceCountedObject entryRef, + TransactionContext context) { + return appendEntry(entryRef.get(), context); } /** diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 0b6fa15b6f..b56b343bd8 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -187,7 +187,8 @@ private long appendImpl(long term, TransactionContext operation) throws StateMac throw new StateMachineException(memberId, new RaftLogIOException( "Log entry size " + entrySize + " exceeds the max buffer limit of " + maxBufferSize)); } - appendEntry(e, operation).whenComplete((returned, t) -> { + + appendEntry(operation.wrap(e), operation).whenComplete((returned, t) -> { if (t != null) { LOG.error(name + ": Failed to write log entry " + LogProtoUtils.toLogEntryString(e), t); } else if (returned != nextIndex) { @@ -345,15 +346,17 @@ public final CompletableFuture purge(long suggestedIndex) { @Override public final CompletableFuture appendEntry(LogEntryProto entry) { - return appendEntry(entry, null); + return appendEntry(ReferenceCountedObject.wrap(entry), null); } @Override - public final CompletableFuture appendEntry(LogEntryProto entry, TransactionContext context) { + public final CompletableFuture appendEntry(ReferenceCountedObject entry, + TransactionContext context) { return runner.runSequentially(() -> appendEntryImpl(entry, context)); } - protected abstract CompletableFuture appendEntryImpl(LogEntryProto entry, TransactionContext context); + protected abstract CompletableFuture appendEntryImpl(ReferenceCountedObject entry, + TransactionContext context); @Override public final List> append(ReferenceCountedObject> entries) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index 7435bb178d..fc7973aab6 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -36,7 +36,6 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; import java.util.function.LongSupplier; /** @@ -175,13 +174,15 @@ public TermIndex getLastEntryTermIndex() { } @Override - protected CompletableFuture appendEntryImpl(LogEntryProto entry, TransactionContext context) { + protected CompletableFuture appendEntryImpl(ReferenceCountedObject entryRef, + TransactionContext context) { checkLogState(); - try(AutoCloseableLock writeLock = writeLock()) { + LogEntryProto entry = entryRef.retain(); + try (AutoCloseableLock writeLock = writeLock()) { validateLogEntry(entry); - Function> wrap = context != null ? - context::wrap : ReferenceCountedObject::wrap; - entries.add(wrap.apply(entry)); + entries.add(entryRef); + } finally { + entryRef.release(); } return CompletableFuture.completedFuture(entry.getIndex()); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index b4e9589658..92b2d42337 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -55,7 +55,6 @@ import java.util.concurrent.CompletionException; import java.util.function.BiFunction; import java.util.function.Consumer; -import java.util.function.Function; import java.util.function.LongSupplier; import org.apache.ratis.util.UncheckedAutoCloseable; @@ -389,8 +388,10 @@ protected CompletableFuture purgeImpl(long index) { } @Override - protected CompletableFuture appendEntryImpl(LogEntryProto entry, TransactionContext context) { + protected CompletableFuture appendEntryImpl(ReferenceCountedObject entryRef, + TransactionContext context) { checkLogState(); + LogEntryProto entry = entryRef.retain(); if (LOG.isTraceEnabled()) { LOG.trace("{}: appendEntry {}", getName(), LogProtoUtils.toLogEntryString(entry)); } @@ -426,19 +427,20 @@ protected CompletableFuture appendEntryImpl(LogEntryProto entry, Transacti // to statemachine first and then to the cache. Not following the order // will leave a spurious entry in the cache. final Task write = fileLogWorker.writeLogEntry(entry, removedStateMachineData, context); - final Function> wrap = context != null ? - context::wrap : ReferenceCountedObject::wrap; if (stateMachineCachingEnabled) { // The stateMachineData will be cached inside the StateMachine itself. - cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE, wrap.apply(removedStateMachineData)); + cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE, + entryRef.delegate(removedStateMachineData)); } else { - cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, wrap.apply(entry) + cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, entryRef ); } return write.getFuture().whenComplete((clientReply, exception) -> appendEntryTimerContext.stop()); } catch (Exception e) { LOG.error("{}: Failed to append {}", getName(), LogProtoUtils.toLogEntryString(entry), e); throw e; + } finally { + entryRef.release(); } } @@ -478,10 +480,7 @@ protected List> appendImpl(ReferenceCountedObject Date: Thu, 18 Jan 2024 23:11:52 -0800 Subject: [PATCH 14/76] RATIS-1934. Support Zero-Copy in GrpcServerProtocolService (#1014) --- .../server/GrpcServerProtocolService.java | 71 ++++++++++++++++--- .../apache/ratis/grpc/server/GrpcService.java | 4 +- 2 files changed, 65 insertions(+), 10 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index dde01c39a3..ebe764fac0 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -20,10 +20,13 @@ import java.util.function.Consumer; import java.util.function.Function; import org.apache.ratis.grpc.GrpcUtil; +import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; +import org.apache.ratis.grpc.util.ZeroCopyMessageMarshaller; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.protocol.RaftServerProtocol; import org.apache.ratis.server.util.ServerStringUtils; +import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; @@ -41,6 +44,9 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; +import static org.apache.ratis.grpc.GrpcUtil.addMethodWithCustomMarshaller; +import static org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.getAppendEntriesMethod; + class GrpcServerProtocolService extends RaftServerProtocolServiceImplBase { public static final Logger LOG = LoggerFactory.getLogger(GrpcServerProtocolService.class); @@ -48,8 +54,9 @@ static class PendingServerRequest { private final REQUEST request; private final CompletableFuture future = new CompletableFuture<>(); - PendingServerRequest(REQUEST request) { - this.request = request; + PendingServerRequest(ReferenceCountedObject requestRef) { + this.request = requestRef.retain(); + this.future.whenComplete((r, e) -> requestRef.release()); } REQUEST getRequest() { @@ -83,7 +90,21 @@ private String getPreviousRequestString() { .orElse(null); } - abstract CompletableFuture process(REQUEST request) throws IOException; + CompletableFuture process(REQUEST request) throws IOException { + throw new UnsupportedOperationException("This method is not supported."); + } + + CompletableFuture process(ReferenceCountedObject requestRef) + throws IOException { + try { + return process(requestRef.retain()); + } finally { + requestRef.release(); + } + } + + void release(REQUEST req) { + } abstract long getCallId(REQUEST request); @@ -120,22 +141,29 @@ void composeRequest(CompletableFuture current) { @Override public void onNext(REQUEST request) { + ReferenceCountedObject requestRef = ReferenceCountedObject.wrap(request, () -> {}, released -> { + if (released) { + release(request); + } + }); + if (!replyInOrder(request)) { try { - composeRequest(process(request).thenApply(this::handleReply)); + composeRequest(process(requestRef).thenApply(this::handleReply)); } catch (Exception e) { handleError(e, request); + release(request); } return; } - final PendingServerRequest current = new PendingServerRequest<>(request); + final PendingServerRequest current = new PendingServerRequest<>(requestRef); final PendingServerRequest previous = previousOnNext.getAndSet(current); final CompletableFuture previousFuture = Optional.ofNullable(previous) .map(PendingServerRequest::getFuture) .orElse(CompletableFuture.completedFuture(null)); try { - final CompletableFuture f = process(request).exceptionally(e -> { + final CompletableFuture f = process(requestRef).exceptionally(e -> { // Handle cases, such as RaftServer is paused handleError(e, request); current.getFuture().completeExceptionally(e); @@ -176,16 +204,35 @@ public void onError(Throwable t) { private final Supplier idSupplier; private final RaftServer server; + private final ZeroCopyMessageMarshaller zeroCopyRequestMarshaller; - GrpcServerProtocolService(Supplier idSupplier, RaftServer server) { + GrpcServerProtocolService(Supplier idSupplier, RaftServer server, ZeroCopyMetrics zeroCopyMetrics) { this.idSupplier = idSupplier; this.server = server; + this.zeroCopyRequestMarshaller = new ZeroCopyMessageMarshaller<>(AppendEntriesRequestProto.getDefaultInstance(), + zeroCopyMetrics::onZeroCopyMessage, zeroCopyMetrics::onNonZeroCopyMessage, zeroCopyMetrics::onReleasedMessage); } RaftPeerId getId() { return idSupplier.get(); } + ServerServiceDefinition bindServiceWithZeroCopy() { + ServerServiceDefinition orig = super.bindService(); + ServerServiceDefinition.Builder builder = ServerServiceDefinition.builder(orig.getServiceDescriptor().getName()); + + // Add appendEntries with zero copy marshaller. + addMethodWithCustomMarshaller(orig, builder, getAppendEntriesMethod(), zeroCopyRequestMarshaller); + // Add remaining methods as is. + orig.getMethods().stream().filter( + x -> !x.getMethodDescriptor().getFullMethodName().equals(getAppendEntriesMethod().getFullMethodName()) + ).forEach( + builder::addMethod + ); + + return builder.build(); + } + @Override public void requestVote(RequestVoteRequestProto request, StreamObserver responseObserver) { @@ -226,8 +273,14 @@ public StreamObserver appendEntries( return new ServerRequestStreamObserver( RaftServerProtocol.Op.APPEND_ENTRIES, responseObserver) { @Override - CompletableFuture process(AppendEntriesRequestProto request) throws IOException { - return server.appendEntriesAsync(ReferenceCountedObject.wrap(request)); + CompletableFuture process(ReferenceCountedObject requestRef) + throws IOException { + return server.appendEntriesAsync(requestRef); + } + + @Override + void release(AppendEntriesRequestProto req) { + zeroCopyRequestMarshaller.release(req); } @Override diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java index d89afd565e..d2dadcd90a 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -214,8 +214,10 @@ private GrpcService(RaftServer raftServer, Supplier idSupplier, final NettyServerBuilder serverBuilder = startBuildingNettyServer(serverHost, serverPort, serverTlsConfig, grpcMessageSizeMax, flowControlWindow); + GrpcServerProtocolService serverProtocolService = new GrpcServerProtocolService(idSupplier, raftServer, + zeroCopyMetrics); serverBuilder.addService(ServerInterceptors.intercept( - new GrpcServerProtocolService(idSupplier, raftServer), serverInterceptor)); + serverProtocolService.bindServiceWithZeroCopy(), serverInterceptor)); if (!separateAdminServer) { addAdminService(raftServer, serverBuilder); } From 561cd6d48ab95ff72bbfd4a89c9074581d2ad1bf Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 19 Jan 2024 08:15:10 +0100 Subject: [PATCH 15/76] RATIS-2010. Remove dependency on spotbugs-annotations (#1022) --- pom.xml | 8 ------- ratis-assembly/src/main/resources/NOTICE | 24 +------------------ ratis-common/pom.xml | 7 ------ .../org/apache/ratis/protocol/RaftPeerId.java | 2 -- ratis-netty/pom.xml | 6 ----- ratis-server-api/pom.xml | 7 ------ ratis-server/pom.xml | 8 ------- .../segmented/BufferedWriteChannel.java | 2 -- .../statemachine/impl/BaseStateMachine.java | 2 -- 9 files changed, 1 insertion(+), 65 deletions(-) diff --git a/pom.xml b/pom.xml index 68488077d4..0b3eedee12 100644 --- a/pom.xml +++ b/pom.xml @@ -440,14 +440,6 @@ 6.0.53 provided - - - com.github.spotbugs - spotbugs-annotations - ${spotbugs.version} - provided - true - diff --git a/ratis-assembly/src/main/resources/NOTICE b/ratis-assembly/src/main/resources/NOTICE index 9bc9242c6a..0e3c94434b 100644 --- a/ratis-assembly/src/main/resources/NOTICE +++ b/ratis-assembly/src/main/resources/NOTICE @@ -292,27 +292,5 @@ networking library, which can be obtained at: * HOMEPAGE: * https://netty.io * LOCATION_IN_GRPC: -* netty/third_party/netty ------------------------------------------------------------------------ -The JSR-305 reference implementation (jsr305.jar) is distributed under the terms of the New BSD: - -Redistribution and use in source and binary forms, with or without modification, -are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, -this list of conditions and the following disclaimer. - -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation and/or -other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS -OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY -AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR -CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL -DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) -ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - + * netty/third_party/netty ----------------------------------------------------------------------- diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index 9bb36bcce5..9205e81c2c 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -69,12 +69,5 @@ junit-jupiter-params test - - - com.github.spotbugs - spotbugs-annotations - provided - true - diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java index 8098039d1b..8db842d734 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java @@ -17,7 +17,6 @@ */ package org.apache.ratis.protocol; -import javax.annotation.concurrent.Immutable; import org.apache.ratis.proto.RaftProtos.RaftPeerIdProto; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.JavaUtils; @@ -34,7 +33,6 @@ *

* This is a value-based class. */ -@Immutable public final class RaftPeerId { private static final Map BYTE_STRING_MAP = new ConcurrentHashMap<>(); private static final Map STRING_MAP = new ConcurrentHashMap<>(); diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml index c29f35171f..1966db8ac4 100644 --- a/ratis-netty/pom.xml +++ b/ratis-netty/pom.xml @@ -73,11 +73,5 @@ slf4j-api - - com.github.spotbugs - spotbugs-annotations - provided - true - diff --git a/ratis-server-api/pom.xml b/ratis-server-api/pom.xml index 52f5eba361..f3c4b617a0 100644 --- a/ratis-server-api/pom.xml +++ b/ratis-server-api/pom.xml @@ -47,12 +47,5 @@ org.slf4j slf4j-api - - - com.github.spotbugs - spotbugs-annotations - provided - true - diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index eb2920323d..3756e3e57d 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -84,13 +84,5 @@ ratis-metrics-default test - - - - com.github.spotbugs - spotbugs-annotations - provided - true - diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/BufferedWriteChannel.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/BufferedWriteChannel.java index 7ad1a48eef..634fca4ce4 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/BufferedWriteChannel.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/BufferedWriteChannel.java @@ -17,7 +17,6 @@ */ package org.apache.ratis.server.raftlog.segmented; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.function.CheckedBiFunction; import org.apache.ratis.util.function.CheckedConsumer; @@ -177,7 +176,6 @@ boolean isOpen() { } @Override - @SuppressFBWarnings("RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT") public void close() throws IOException { if (!isOpen()) { return; diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java index 3c20bc5714..c987c53ddb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java @@ -18,7 +18,6 @@ package org.apache.ratis.statemachine.impl; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftClientRequest; @@ -197,7 +196,6 @@ public CompletableFuture queryStale(Message request, long minIndex) { } @Override - @SuppressFBWarnings("NP_NULL_PARAM_DEREF") public CompletableFuture query(Message request) { return CompletableFuture.completedFuture(null); } From c1e0ec284aa88078db63a8b3cc1a90231144eff3 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 22 Jan 2024 14:30:30 -0800 Subject: [PATCH 16/76] RATIS-2013. OrderedAsync retry results an IllegalStateException in GrpcClientProtocolService. (#1026) --- .../ratis/client/impl/RaftClientImpl.java | 6 ++++-- .../org/apache/ratis/util/SlidingWindow.java | 18 +++++++++++++++--- .../retry/TestExceptionDependentRetry.java | 9 ++++++--- 3 files changed, 25 insertions(+), 8 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java index ec16763c2c..f423919475 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java @@ -87,8 +87,10 @@ public abstract static class PendingClientRequest { public abstract RaftClientRequest newRequestImpl(); final RaftClientRequest newRequest() { - attemptCount.incrementAndGet(); - return newRequestImpl(); + final int attempt = attemptCount.incrementAndGet(); + final RaftClientRequest request = newRequestImpl(); + LOG.debug("attempt #{}, newRequest {}", attempt, request); + return request; } CompletableFuture getReplyFuture() { diff --git a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java index 316604db07..732e3d890e 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java @@ -122,6 +122,13 @@ public Iterator iterator() { return requests.values().iterator(); } + /** @return true iff the request already exists. */ + boolean putIfAbsent(REQUEST request) { + final long seqNum = request.getSeqNum(); + final REQUEST previous = requests.putIfAbsent(seqNum, request); + return previous != null; + } + void putNewRequest(REQUEST request) { final long seqNum = request.getSeqNum(); CollectionUtils.putNew(seqNum, request, requests, () -> getName() + ":requests"); @@ -443,11 +450,16 @@ public synchronized void receivedRequest(REQUEST request, Consumer proc final long seqNum = request.getSeqNum(); if (nextToProcess == -1 && (request.isFirstRequest() || seqNum == 0)) { nextToProcess = seqNum; - LOG.debug("{}: got seq={} (first request), set nextToProcess in {}", requests.getName(), seqNum, this); + requests.putNewRequest(request); + LOG.debug("Received seq={} (first request), {}", seqNum, this); } else { - LOG.debug("{}: got seq={} in {}", requests.getName(), seqNum, this); + final boolean isRetry = requests.putIfAbsent(request); + LOG.debug("Received seq={}, isRetry? {}, {}", seqNum, isRetry, this); + if (isRetry) { + return; + } } - requests.putNewRequest(request); + processRequestsFromHead(processingMethod); } diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java index 7eba6a82b9..264db89464 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java @@ -178,12 +178,14 @@ public void testExceptionRetryAttempts() throws Exception { } void runTestExceptionRetryAttempts(MiniRaftClusterWithGrpc cluster) throws Exception { - final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); + final int retryCount = 5; + final RetryPolicy timeoutPolicy = MultipleLinearRandomRetry.parseCommaSeparated("1ms, " + retryCount); final ExceptionDependentRetry policy = ExceptionDependentRetry.newBuilder() - .setExceptionToPolicy(TimeoutIOException.class, MultipleLinearRandomRetry.parseCommaSeparated("1ms, 5")) + .setExceptionToPolicy(TimeoutIOException.class, timeoutPolicy) .setDefaultPolicy(RetryPolicies.retryForeverNoSleep()) .build(); + final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); // create a client with the exception dependent policy try (final RaftClient client = cluster.createClient(policy)) { client.async().send(new RaftTestUtil.SimpleMessage("1")).get(); @@ -196,7 +198,8 @@ void runTestExceptionRetryAttempts(MiniRaftClusterWithGrpc cluster) throws Excep Assert.fail("Test should have failed."); } catch (ExecutionException e) { RaftRetryFailureException rrfe = (RaftRetryFailureException) e.getCause(); - Assert.assertEquals(16, rrfe.getAttemptCount()); + final int expectedCount = 1 + retryCount; // new request attempt + retry attempts + Assert.assertEquals(expectedCount, rrfe.getAttemptCount()); } finally { SimpleStateMachine4Testing.get(leader).unblockWriteStateMachineData(); cluster.shutdown(); From 0e247578bf46cd0ffdc9863da3ff77ba21753a37 Mon Sep 17 00:00:00 2001 From: Nandakumar Vadivelu Date: Tue, 23 Jan 2024 07:27:50 +0530 Subject: [PATCH 17/76] RATIS-1972. Add junit 5 dependencies in ratis-server. (#1003) --- .../test/java/org/apache/ratis/BaseTest.java | 38 ++++++++++++++++--- ratis-examples/pom.xml | 5 +++ ratis-server/pom.xml | 25 ++++++++++++ .../server/impl/TestLogAppenderMetrics.java | 20 +++++----- .../server/impl/TestRetryCacheMetrics.java | 12 +++--- .../metrics/TestLeaderElectionMetrics.java | 14 +++---- ratis-test/pom.xml | 5 +++ 7 files changed, 90 insertions(+), 29 deletions(-) diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java index 8213694614..fb34d64f1e 100644 --- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java +++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java @@ -27,10 +27,13 @@ import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedRunnable; import org.junit.After; -import org.junit.Assert; import org.junit.Rule; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestInfo; +import org.junit.jupiter.api.Timeout; import org.junit.rules.TestName; -import org.junit.rules.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -47,6 +50,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; +@Timeout(value = 100) public abstract class BaseTest { public final Logger LOG = LoggerFactory.getLogger(getClass()); @@ -80,7 +84,24 @@ public List getPeersWithPriority(List peers, RaftPeer sugges return peersWithPriority; } + + /* + * Junit 4 reference will be removed and the code will be refactored once + * all the unit tests are migrated to Junit 5. + */ + + private String testCaseName; + + @BeforeEach + public void setup(TestInfo testInfo) { + testCaseName = testInfo.getTestMethod() + .orElseThrow(() -> new RuntimeException("Exception while getting test name.")) + .getName(); + } + + // @After annotation is retained to support junit 4 tests. @After + @AfterEach public void assertNoFailures() { final Throwable e = firstException.get(); if (e != null) { @@ -90,9 +111,12 @@ public void assertNoFailures() { ExitUtils.assertNotTerminated(); } + // Retained to support junit 4 tests. @Rule - public final Timeout globalTimeout = new Timeout(getGlobalTimeoutSeconds(), TimeUnit.SECONDS ); + public final org.junit.rules.Timeout globalTimeout = new org.junit.rules.Timeout( + getGlobalTimeoutSeconds(), TimeUnit.SECONDS ); + // Retained to support junit 4 tests. @Rule public final TestName testName = new TestName(); @@ -122,7 +146,9 @@ public File getClassTestDir() { } public File getTestDir() { - return new File(getClassTestDir(), testName.getMethodName()); + // This will work for both junit 4 and 5. + final String name = testCaseName != null ? testCaseName : testName.getMethodName(); + return new File(getClassTestDir(), name); } @SafeVarargs @@ -135,13 +161,13 @@ public static void assertThrowable( description, expectedThrowableClass.getSimpleName(), StringUtils.array2String(expectedCauseClasses, Class::getSimpleName)); } - Assert.assertEquals(expectedThrowableClass, t.getClass()); + Assertions.assertEquals(expectedThrowableClass, t.getClass()); for (Class expectedCause : expectedCauseClasses) { final Throwable previous = t; t = Objects.requireNonNull(previous.getCause(), () -> "previous.getCause() == null for previous=" + previous); - Assert.assertEquals(expectedCause, t.getClass()); + Assertions.assertEquals(expectedCause, t.getClass()); } } diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index 4078f6260f..54691d6f14 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -122,6 +122,11 @@ junit test + + org.junit.jupiter + junit-jupiter-api + test + diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index 3756e3e57d..38af72d9db 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -70,6 +70,31 @@ junit test + + org.junit.jupiter + junit-jupiter-api + test + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.vintage + junit-vintage-engine + test + + + org.junit.platform + junit-platform-launcher + test + + + org.junit.jupiter + junit-jupiter-params + test + org.mockito mockito-core diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java index 5c78db42f6..4bd075ef66 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java @@ -27,11 +27,11 @@ import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.metrics.LogAppenderMetrics; import org.apache.ratis.util.Timestamp; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; import org.apache.ratis.thirdparty.com.codahale.metrics.Gauge; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestLogAppenderMetrics { @@ -39,7 +39,7 @@ public class TestLogAppenderMetrics { private RaftPeerId raftPeerId; private MyFollowerInfo followerInfo; - @Before + @BeforeEach public void setup() { RaftGroupId raftGroupId = RaftGroupId.randomId(); raftPeerId = RaftPeerId.valueOf("TestId"); @@ -55,19 +55,19 @@ public void setup() { public void testLogAppenderGauges() { Gauge nextIndex = ratisMetricRegistry.getGauges((s, metric) -> s.contains(String.format(FOLLOWER_NEXT_INDEX, raftPeerId.toString()))).values().iterator().next(); - Assert.assertEquals(100L, nextIndex.getValue()); + Assertions.assertEquals(100L, nextIndex.getValue()); Gauge matchIndex = ratisMetricRegistry.getGauges((s, metric) -> s.contains(String.format(FOLLOWER_MATCH_INDEX, raftPeerId.toString()))).values().iterator().next(); - Assert.assertEquals(0L, matchIndex.getValue()); + Assertions.assertEquals(0L, matchIndex.getValue()); Gauge rpcTime = ratisMetricRegistry.getGauges((s, metric) -> s.contains(String.format(FOLLOWER_RPC_RESP_TIME, raftPeerId.toString()))).values().iterator().next(); - Assert.assertTrue(((Long) rpcTime.getValue()) > 0); + Assertions.assertTrue(((Long) rpcTime.getValue()) > 0); followerInfo.updateNextIndex(200L); followerInfo.updateMatchIndex(100L); followerInfo.updateLastRpcResponseTime(); - Assert.assertEquals(200L, nextIndex.getValue()); - Assert.assertEquals(100L, matchIndex.getValue()); - Assert.assertNotNull(rpcTime.getValue()); + Assertions.assertEquals(200L, nextIndex.getValue()); + Assertions.assertEquals(100L, matchIndex.getValue()); + Assertions.assertNotNull(rpcTime.getValue()); } private static class MyFollowerInfo { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRetryCacheMetrics.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRetryCacheMetrics.java index b25a50bf97..0d779a2dc0 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRetryCacheMetrics.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRetryCacheMetrics.java @@ -19,7 +19,7 @@ package org.apache.ratis.server.impl; import static org.apache.ratis.server.metrics.RaftServerMetricsImpl.*; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import org.apache.ratis.metrics.impl.RatisMetricRegistryImpl; import org.apache.ratis.thirdparty.com.codahale.metrics.Gauge; @@ -30,9 +30,9 @@ import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; -import org.junit.After; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import java.util.Map; @@ -43,7 +43,7 @@ public class TestRetryCacheMetrics { private static RatisMetricRegistryImpl ratisMetricRegistry; private static RetryCacheImpl retryCache; - @BeforeClass + @BeforeAll public static void setUp() { RaftGroupId raftGroupId = RaftGroupId.randomId(); RaftPeerId raftPeerId = RaftPeerId.valueOf("TestId"); @@ -56,7 +56,7 @@ public static void setUp() { ratisMetricRegistry = (RatisMetricRegistryImpl) raftServerMetrics.getRegistry(); } - @After + @AfterEach public void tearDown() { retryCache.close(); checkEntryCount(0); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/metrics/TestLeaderElectionMetrics.java b/ratis-server/src/test/java/org/apache/ratis/server/metrics/TestLeaderElectionMetrics.java index a39612c57e..bba5d92316 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/metrics/TestLeaderElectionMetrics.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/metrics/TestLeaderElectionMetrics.java @@ -20,8 +20,8 @@ import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LAST_LEADER_ELECTION_ELAPSED_TIME; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_TIMEOUT_COUNT_METRIC; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.apache.ratis.metrics.impl.RatisMetricRegistryImpl; import org.apache.ratis.thirdparty.com.codahale.metrics.Gauge; @@ -29,8 +29,8 @@ import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.protocol.RaftPeerId; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import java.util.SortedMap; @@ -42,7 +42,7 @@ public class TestLeaderElectionMetrics extends BaseTest { private static LeaderElectionMetrics leaderElectionMetrics; private static RatisMetricRegistryImpl ratisMetricRegistry; - @BeforeClass + @BeforeAll public static void setUp() { RaftGroupId raftGroupId = RaftGroupId.randomId(); RaftPeerId raftPeerId = RaftPeerId.valueOf("TestId"); @@ -58,14 +58,14 @@ public void testOnLeaderElectionCompletion() throws Exception { (s, metric) -> s.contains(LAST_LEADER_ELECTION_ELAPSED_TIME)); LOG.info("{} gauges: {}", LAST_LEADER_ELECTION_ELAPSED_TIME, gauges); final Long leaderElectionLatency = (Long)gauges.values().iterator().next().getValue(); - assertTrue("leaderElectionLatency = " + leaderElectionLatency, leaderElectionLatency >= 0L); + assertTrue(leaderElectionLatency >= 0L, "leaderElectionLatency = " + leaderElectionLatency); } @Test public void testOnLeaderElectionTimeout() throws Exception { long numLeaderElectionTimeouts = ratisMetricRegistry.counter( LEADER_ELECTION_TIMEOUT_COUNT_METRIC).getCount(); - assertTrue(numLeaderElectionTimeouts == 0); + assertEquals(0, numLeaderElectionTimeouts); leaderElectionMetrics.onLeaderElectionTimeout(); numLeaderElectionTimeouts = ratisMetricRegistry.counter(LEADER_ELECTION_TIMEOUT_COUNT_METRIC).getCount(); assertEquals(1, numLeaderElectionTimeouts); diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 40b6b2ec27..2021e06ad4 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -136,6 +136,11 @@ junit test + + org.junit.jupiter + junit-jupiter-api + test + org.mockito mockito-core From ea1341b05dbaedb7b9775fa870522bf1de052a0e Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 23 Jan 2024 10:01:01 -0800 Subject: [PATCH 18/76] RATIS-2012. Client should not retry after close. (#1025) --- .../ratis/client/impl/BlockingImpl.java | 12 +- .../ratis/client/impl/OrderedAsync.java | 101 ++++---- .../ratis/client/impl/RaftClientImpl.java | 42 ++-- .../ratis/client/impl/UnorderedAsync.java | 14 +- .../ratis/client/retry/ClientRetryEvent.java | 24 +- .../org/apache/ratis/util/PeerProxyMap.java | 17 +- .../java/org/apache/ratis/RaftAsyncTests.java | 2 +- .../java/org/apache/ratis/RaftBasicTests.java | 14 +- .../java/org/apache/ratis/RaftTestUtil.java | 30 ++- .../apache/ratis/RaftLogTruncateTests.java | 216 ++++++++++++++++++ .../grpc/TestRaftLogTruncateWithGrpc.java | 24 ++ .../apache/ratis/grpc/TestRaftWithGrpc.java | 6 - .../retry/TestExceptionDependentRetry.java | 2 +- .../apache/ratis/retry/TestRetryPolicy.java | 20 +- .../ratis/server/ServerRestartTests.java | 2 +- 15 files changed, 380 insertions(+), 146 deletions(-) create mode 100644 ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftLogTruncateWithGrpc.java diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/BlockingImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/BlockingImpl.java index 4be9fa3275..76987801ba 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/BlockingImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/BlockingImpl.java @@ -119,16 +119,18 @@ public RaftClientRequest newRequestImpl() { ioe = e; } - pending.incrementExceptionCount(ioe); - ClientRetryEvent event = new ClientRetryEvent(request, ioe, pending); + if (client.isClosed()) { + throw new AlreadyClosedException(this + " is closed."); + } + + final ClientRetryEvent event = pending.newClientRetryEvent(request, ioe); final RetryPolicy retryPolicy = client.getRetryPolicy(); final RetryPolicy.Action action = retryPolicy.handleAttemptFailure(event); - TimeDuration sleepTime = client.getEffectiveSleepTime(ioe, action.getSleepTime()); - if (!action.shouldRetry()) { - throw (IOException)client.noMoreRetries(event); + throw client.noMoreRetries(event); } + final TimeDuration sleepTime = client.getEffectiveSleepTime(ioe, action.getSleepTime()); try { sleepTime.sleep(); } catch (InterruptedException e) { diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java index a1aa58681c..34dc3be113 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java @@ -149,10 +149,6 @@ private void failAllAsyncRequests(RaftClientRequest request, Throwable t) { getSlidingWindow(request).fail(request.getSlidingWindowEntry().getSeqNum(), t); } - private void handleAsyncRetryFailure(ClientRetryEvent event) { - failAllAsyncRequests(event.getRequest(), client.noMoreRetries(event)); - } - CompletableFuture send(RaftClientRequest.Type type, Message message, RaftPeerId server) { if (!type.is(TypeCase.WATCH) && !type.is(TypeCase.MESSAGESTREAM)) { Objects.requireNonNull(message, "message == null"); @@ -187,85 +183,68 @@ private void sendRequestWithRetry(PendingOrderedRequest pending) { if (pending == null) { return; } - - final CompletableFuture f = pending.getReplyFuture(); - if (f.isDone()) { + if (pending.getReplyFuture().isDone()) { return; } - final RaftClientRequest request = pending.newRequestImpl(); + final RaftClientRequest request = pending.newRequest(); if (request == null) { // already done - LOG.debug("{} newRequestImpl returns null", pending); + LOG.debug("{} newRequest returns null", pending); return; } - final RetryPolicy retryPolicy = client.getRetryPolicy(); - sendRequest(pending).exceptionally(e -> { - if (e instanceof CompletionException) { - e = JavaUtils.unwrapCompletionException(e); - scheduleWithTimeout(pending, request, retryPolicy, e); - return null; - } - f.completeExceptionally(e); - return null; - }); - } - - private void scheduleWithTimeout(PendingOrderedRequest pending, - RaftClientRequest request, RetryPolicy retryPolicy, Throwable e) { - final int attempt = pending.getAttemptCount(); - final ClientRetryEvent event = new ClientRetryEvent(request, e, pending); - final TimeDuration sleepTime = client.getEffectiveSleepTime(e, - retryPolicy.handleAttemptFailure(event).getSleepTime()); - LOG.debug("schedule* attempt #{} with sleep {} and policy {} for {}", attempt, sleepTime, retryPolicy, request); - scheduleWithTimeout(pending, sleepTime, getSlidingWindow(request)); - } - - private void scheduleWithTimeout(PendingOrderedRequest pending, TimeDuration sleepTime, - SlidingWindow.Client slidingWindow) { - client.getScheduler().onTimeout(sleepTime, - () -> slidingWindow.retry(pending, this::sendRequestWithRetry), - LOG, () -> "Failed* to retry " + pending); - } - - private CompletableFuture sendRequest(PendingOrderedRequest pending) { - final RetryPolicy retryPolicy = client.getRetryPolicy(); - final RaftClientRequest request; if (getSlidingWindow((RaftPeerId) null).isFirst(pending.getSeqNum())) { pending.setFirstRequest(); } - request = pending.newRequest(); LOG.debug("{}: send* {}", client.getId(), request); - return client.getClientRpc().sendRequestAsync(request).thenApply(reply -> { + client.getClientRpc().sendRequestAsync(request).thenAccept(reply -> { LOG.debug("{}: receive* {}", client.getId(), reply); Objects.requireNonNull(reply, "reply == null"); client.handleReply(request, reply); getSlidingWindow(request).receiveReply( request.getSlidingWindowEntry().getSeqNum(), reply, this::sendRequestWithRetry); - return reply; }).exceptionally(e -> { LOG.error(client.getId() + ": Failed* " + request, e); - e = JavaUtils.unwrapCompletionException(e); - if (e instanceof IOException && !(e instanceof GroupMismatchException)) { - pending.incrementExceptionCount(e); - final ClientRetryEvent event = new ClientRetryEvent(request, e, pending); - if (!retryPolicy.handleAttemptFailure(event).shouldRetry()) { - handleAsyncRetryFailure(event); - } else { - if (e instanceof NotLeaderException) { - NotLeaderException nle = (NotLeaderException)e; - client.handleNotLeaderException(request, nle, this::resetSlidingWindow); - } else { - client.handleIOException(request, (IOException) e, null, this::resetSlidingWindow); - } - } - throw new CompletionException(e); - } - failAllAsyncRequests(request, e); + handleException(pending, request, e); return null; }); } + private void handleException(PendingOrderedRequest pending, RaftClientRequest request, Throwable e) { + final RetryPolicy retryPolicy = client.getRetryPolicy(); + if (client.isClosed()) { + failAllAsyncRequests(request, new AlreadyClosedException(client + " is closed.")); + return; + } + + e = JavaUtils.unwrapCompletionException(e); + if (!(e instanceof IOException) || e instanceof GroupMismatchException) { + // non-retryable exceptions + failAllAsyncRequests(request, e); + return; + } + + final ClientRetryEvent event = pending.newClientRetryEvent(request, e); + final RetryPolicy.Action action = retryPolicy.handleAttemptFailure(event); + if (!action.shouldRetry()) { + failAllAsyncRequests(request, client.noMoreRetries(event)); + return; + } + + if (e instanceof NotLeaderException) { + client.handleNotLeaderException(request, (NotLeaderException) e, this::resetSlidingWindow); + } else { + client.handleIOException(request, (IOException) e, null, this::resetSlidingWindow); + } + final TimeDuration sleepTime = client.getEffectiveSleepTime(e, action.getSleepTime()); + LOG.debug("schedule* retry with sleep {} for attempt #{} of {}, {}", + sleepTime, event.getAttemptCount(), request, retryPolicy); + final SlidingWindow.Client slidingWindow = getSlidingWindow(request); + client.getScheduler().onTimeout(sleepTime, + () -> slidingWindow.retry(pending, this::sendRequestWithRetry), + LOG, () -> "Failed* to retry " + pending); + } + void assertRequestSemaphore(int expectedAvailablePermits, int expectedQueueLength) { Preconditions.assertSame(expectedAvailablePermits, requestSemaphore.availablePermits(), "availablePermits"); Preconditions.assertSame(expectedQueueLength, requestSemaphore.getQueueLength(), "queueLength"); diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java index f423919475..1b82709daf 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java @@ -44,11 +44,13 @@ import org.apache.ratis.thirdparty.com.google.common.cache.Cache; import org.apache.ratis.thirdparty.com.google.common.cache.CacheBuilder; import org.apache.ratis.util.CollectionUtils; +import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.TimeoutExecutor; +import org.apache.ratis.util.Timestamp; import java.io.IOException; import java.util.ArrayList; @@ -65,6 +67,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -79,10 +82,10 @@ public final class RaftClientImpl implements RaftClient { .build(); public abstract static class PendingClientRequest { - private final long creationTimeInMs = System.currentTimeMillis(); + private final Timestamp creationTime = Timestamp.currentTime(); private final CompletableFuture replyFuture = new CompletableFuture<>(); private final AtomicInteger attemptCount = new AtomicInteger(); - private final Map, Integer> exceptionCount = new ConcurrentHashMap<>(); + private final Map, Integer> exceptionCounts = new ConcurrentHashMap<>(); public abstract RaftClientRequest newRequestImpl(); @@ -101,19 +104,10 @@ public int getAttemptCount() { return attemptCount.get(); } - int incrementExceptionCount(Throwable t) { - return t != null ? exceptionCount.compute(t.getClass(), (k, v) -> v != null ? v + 1 : 1) : 0; - } - - public int getExceptionCount(Throwable t) { - return t != null ? Optional.ofNullable(exceptionCount.get(t.getClass())).orElse(0) : 0; - } - - public boolean isRequestTimeout(TimeDuration timeout) { - if (timeout == null) { - return false; - } - return System.currentTimeMillis() - creationTimeInMs > timeout.toLong(TimeUnit.MILLISECONDS); + public ClientRetryEvent newClientRetryEvent(RaftClientRequest request, Throwable throwable) { + final int exceptionCount = throwable == null? 0 + : exceptionCounts.compute(throwable.getClass(), (k, v) -> v == null? 1: v+1); + return new ClientRetryEvent(getAttemptCount(), request, exceptionCount, throwable, creationTime); } } @@ -196,6 +190,8 @@ private synchronized Set getAndReset() { private final ConcurrentMap leaderElectionManagement = new ConcurrentHashMap<>(); + private final AtomicBoolean closed = new AtomicBoolean(); + @SuppressWarnings("checkstyle:ParameterNumber") RaftClientImpl(ClientId clientId, RaftGroup group, RaftPeerId leaderId, RaftPeer primaryDataStreamServer, RaftClientRpc clientRpc, RetryPolicy retryPolicy, RaftProperties properties, Parameters parameters) { @@ -346,11 +342,11 @@ public DataStreamApi getDataStreamApi() { return dataStreamApi.get(); } - Throwable noMoreRetries(ClientRetryEvent event) { + IOException noMoreRetries(ClientRetryEvent event) { final int attemptCount = event.getAttemptCount(); final Throwable throwable = event.getCause(); if (attemptCount == 1 && throwable != null) { - return throwable; + return IOUtils.asIOException(throwable); } return new RaftRetryFailureException(event.getRequest(), attemptCount, retryPolicy, throwable); } @@ -418,8 +414,7 @@ void handleIOException(RaftClientRequest request, IOException ioe) { void handleIOException(RaftClientRequest request, IOException ioe, RaftPeerId newLeader, Consumer handler) { - LOG.debug("{}: suggested new leader: {}. Failed {} with {}", - clientId, newLeader, request, ioe); + LOG.debug("{}: suggested new leader: {}. Failed {}", clientId, newLeader, request, ioe); if (LOG.isTraceEnabled()) { LOG.trace("Stack trace", new Throwable("TRACE")); } @@ -456,8 +451,17 @@ public RaftClientRpc getClientRpc() { return clientRpc; } + boolean isClosed() { + return closed.get(); + } + @Override public void close() throws IOException { + if (!closed.compareAndSet(false, true)) { + return; + } + + LOG.debug("close {}", getId()); clientRpc.close(); if (dataStreamApi.isInitialized()) { dataStreamApi.get().close(); diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/UnorderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/UnorderedAsync.java index 84b817b581..eccda4dbdd 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/UnorderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/UnorderedAsync.java @@ -22,6 +22,7 @@ import org.apache.ratis.protocol.ClientId; import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.exceptions.AlreadyClosedException; import org.apache.ratis.protocol.exceptions.GroupMismatchException; import org.apache.ratis.protocol.exceptions.NotLeaderException; import org.apache.ratis.protocol.RaftClientReply; @@ -89,11 +90,14 @@ static void sendRequestWithRetry(PendingClientRequest pending, RaftClientImpl cl } final Throwable cause = replyException != null ? replyException : e; - pending.incrementExceptionCount(cause); - final ClientRetryEvent event = new ClientRetryEvent(request, cause, pending); + if (client.isClosed()) { + f.completeExceptionally(new AlreadyClosedException(client + " is closed")); + return; + } + + final ClientRetryEvent event = pending.newClientRetryEvent(request, cause); RetryPolicy retryPolicy = client.getRetryPolicy(); final RetryPolicy.Action action = retryPolicy.handleAttemptFailure(event); - TimeDuration sleepTime = client.getEffectiveSleepTime(cause, action.getSleepTime()); if (!action.shouldRetry()) { f.completeExceptionally(client.noMoreRetries(event)); return; @@ -124,7 +128,9 @@ static void sendRequestWithRetry(PendingClientRequest pending, RaftClientImpl cl } } - LOG.debug("schedule retry for attempt #{}, policy={}, request={}", attemptCount, retryPolicy, request); + final TimeDuration sleepTime = client.getEffectiveSleepTime(cause, action.getSleepTime()); + LOG.debug("schedule~ attempt #{} with sleep {} and policy {} for {}", + attemptCount, sleepTime, retryPolicy, request); client.getScheduler().onTimeout(sleepTime, () -> sendRequestWithRetry(pending, client), LOG, () -> clientId + ": Failed~ to retry " + request); } catch (Exception ex) { diff --git a/ratis-client/src/main/java/org/apache/ratis/client/retry/ClientRetryEvent.java b/ratis-client/src/main/java/org/apache/ratis/client/retry/ClientRetryEvent.java index f0c38efb96..c6a8beb06f 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/retry/ClientRetryEvent.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/retry/ClientRetryEvent.java @@ -17,12 +17,11 @@ */ package org.apache.ratis.client.retry; -import org.apache.ratis.client.impl.RaftClientImpl.PendingClientRequest; import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.retry.RetryPolicy; -import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.Timestamp; /** An {@link RetryPolicy.Event} specific to client request failure. */ public class ClientRetryEvent implements RetryPolicy.Event { @@ -30,23 +29,15 @@ public class ClientRetryEvent implements RetryPolicy.Event { private final int causeCount; private final RaftClientRequest request; private final Throwable cause; - private PendingClientRequest pending; + private final Timestamp pendingRequestCreationTime; - @VisibleForTesting - public ClientRetryEvent(int attemptCount, RaftClientRequest request, Throwable cause) { - this(attemptCount, request, attemptCount, cause); - } - - public ClientRetryEvent(RaftClientRequest request, Throwable t, PendingClientRequest pending) { - this(pending.getAttemptCount(), request, pending.getExceptionCount(t), t); - this.pending = pending; - } - - private ClientRetryEvent(int attemptCount, RaftClientRequest request, int causeCount, Throwable cause) { + public ClientRetryEvent(int attemptCount, RaftClientRequest request, int causeCount, Throwable cause, + Timestamp pendingRequestCreationTime) { this.attemptCount = attemptCount; this.causeCount = causeCount; this.request = request; this.cause = cause; + this.pendingRequestCreationTime = pendingRequestCreationTime; } @Override @@ -69,7 +60,7 @@ public Throwable getCause() { } boolean isRequestTimeout(TimeDuration timeout) { - return pending != null && pending.isRequestTimeout(timeout); + return timeout != null && pendingRequestCreationTime.elapsedTime().compareTo(timeout) >= 0; } @Override @@ -77,6 +68,7 @@ public String toString() { return JavaUtils.getClassSimpleName(getClass()) + ":attempt=" + attemptCount + ",request=" + request - + ",cause=" + cause; + + ",cause=" + cause + + ",causeCount=" + causeCount; } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java index 105ecbfb43..0ce0595fa9 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java @@ -36,6 +36,7 @@ import java.util.Objects; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; /** A map from peer id to peer and its proxy. */ public class PeerProxyMap implements RaftPeer.Add, Closeable { @@ -65,7 +66,7 @@ PROXY getProxy() throws IOException { throw new AlreadyClosedException(name + " is already " + current); } lifeCycle.startAndTransition( - () -> proxy = createProxy.apply(peer), IOException.class); + () -> proxy = createProxyImpl(peer), IOException.class); } } } @@ -92,6 +93,7 @@ public String toString() { private final Object resetLock = new Object(); private final CheckedFunction createProxy; + private final AtomicBoolean closed = new AtomicBoolean(); public PeerProxyMap(String name, CheckedFunction createProxy) { this.name = name; @@ -102,6 +104,13 @@ public String getName() { return name; } + private PROXY createProxyImpl(RaftPeer peer) throws IOException { + if (closed.get()) { + throw new AlreadyClosedException(name + ": Failed to create proxy for " + peer); + } + return createProxy.apply(peer); + } + public PROXY getProxy(RaftPeerId id) throws IOException { Objects.requireNonNull(id, "id == null"); PeerAndProxy p = peers.get(id); @@ -161,6 +170,10 @@ public boolean handleException(RaftPeerId serverId, Throwable e, boolean reconne @Override public void close() { + if (!closed.compareAndSet(false, true)) { + return; + } + final List exceptions = Collections.synchronizedList(new ArrayList<>()); ConcurrentUtils.parallelForEachAsync(peers.values(), pp -> pp.setNullProxyAndClose().map(proxy -> closeProxy(proxy, pp)).ifPresent(exceptions::add), @@ -180,7 +193,7 @@ public void close() { private IOException closeProxy(PROXY proxy, PeerAndProxy pp) { try { - LOG.debug("{}: Closing proxy for peer {}", name, pp); + LOG.debug("{}: Closing proxy {} {} for peer {}", name, proxy.getClass().getSimpleName(), proxy, pp); proxy.close(); return null; } catch (IOException e) { diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java index 260f6013e8..71c5c5ef06 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java @@ -372,7 +372,7 @@ public void testRequestTimeout() throws Exception { @Test public void testStateMachineMetrics() throws Exception { runWithNewCluster(NUM_SERVERS, cluster -> - RaftBasicTests.testStateMachineMetrics(true, cluster, LOG)); + RaftBasicTests.runTestStateMachineMetrics(true, cluster)); } @Test diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index 47c9b0e085..4ff9681f0a 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -168,12 +168,10 @@ static void runTestBasicAppendEntries( final List divisions = cluster.getServerAliveStream().collect(Collectors.toList()); for(RaftServer.Division impl: divisions) { - JavaUtils.attempt(() -> RaftTestUtil.assertLogEntries(impl, term, messages), - 50, TimeDuration.valueOf(1, TimeUnit.SECONDS), impl.getId() + " assertLogEntries", LOG); + RaftTestUtil.assertLogEntries(impl, term, messages, 50, LOG); } } - @Test public void testOldLeaderCommit() throws Exception { runWithNewCluster(NUM_SERVERS, this::runTestOldLeaderCommit); @@ -218,7 +216,7 @@ void runTestOldLeaderCommit(CLUSTER cluster) throws Exception { cluster.getServerAliveStream() .map(RaftServer.Division::getRaftLog) - .forEach(log -> RaftTestUtil.assertLogEntries(log, term, messages)); + .forEach(log -> RaftTestUtil.assertLogEntries(log, term, messages, System.out::println)); } @Test @@ -453,8 +451,12 @@ public static void testRequestTimeout(boolean async, MiniRaftCluster cluster, Lo } } - public static void testStateMachineMetrics(boolean async, - MiniRaftCluster cluster, Logger LOG) throws Exception { + @Test + public void testStateMachineMetrics() throws Exception { + runWithNewCluster(NUM_SERVERS, cluster -> runTestStateMachineMetrics(false, cluster)); + } + + static void runTestStateMachineMetrics(boolean async, MiniRaftCluster cluster) throws Exception { RaftServer.Division leader = waitForLeader(cluster); try (final RaftClient client = cluster.createClient()) { Gauge appliedIndexGauge = getStatemachineGaugeWithName(leader, diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index fa4188716c..41a431149e 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -249,19 +249,16 @@ static void assertLogEntries(MiniRaftCluster cluster, SimpleMessage expectedMess } } - static void assertLogEntries(RaftServer.Division server, long expectedTerm, SimpleMessage... expectedMessages) { - LOG.info("checking raft log for {}", server.getMemberId()); - final RaftLog log = server.getRaftLog(); - try { - RaftTestUtil.assertLogEntries(log, expectedTerm, expectedMessages); - } catch (AssertionError e) { - LOG.error("Unexpected raft log in {}", server.getMemberId(), e); - throw e; - } + static void assertLogEntries(RaftServer.Division server, long expectedTerm, SimpleMessage[] expectedMessages, + int numAttempts, Logger log) throws Exception { + final String name = server.getId() + " assertLogEntries"; + final Function> print = i -> i < numAttempts? s -> {}: System.out::println; + JavaUtils.attempt(i -> assertLogEntries(server.getRaftLog(), expectedTerm, expectedMessages, print.apply(i)), + numAttempts, TimeDuration.ONE_SECOND, () -> name, log); } static Iterable getLogEntryProtos(RaftLog log) { - return CollectionUtils.as(log.getEntries(0, Long.MAX_VALUE), ti -> { + return CollectionUtils.as(log.getEntries(0, log.getLastEntryTermIndex().getIndex() + 1), ti -> { try { return log.get(ti.getIndex()); } catch (IOException exception) { @@ -270,17 +267,17 @@ static Iterable getLogEntryProtos(RaftLog log) { }); } - static List getStateMachineLogEntries(RaftLog log) { + static List getStateMachineLogEntries(RaftLog log, Consumer print) { final List entries = new ArrayList<>(); for (LogEntryProto e : getLogEntryProtos(log)) { final String s = LogProtoUtils.toLogEntryString(e); if (e.hasStateMachineLogEntry()) { - LOG.info(s + ", " + e.getStateMachineLogEntry().toString().trim().replace("\n", ", ")); + print.accept(entries.size() + ") " + s); entries.add(e); } else if (e.hasConfigurationEntry()) { - LOG.info("Found {}, ignoring it.", s); + print.accept("Ignoring " + s); } else if (e.hasMetadataEntry()) { - LOG.info("Found {}, ignoring it.", s); + print.accept("Ignoring " + s); } else { throw new AssertionError("Unexpected LogEntryBodyCase " + e.getLogEntryBodyCase() + " at " + s); } @@ -288,13 +285,14 @@ static List getStateMachineLogEntries(RaftLog log) { return entries; } - static void assertLogEntries(RaftLog log, long expectedTerm, SimpleMessage... expectedMessages) { - final List entries = getStateMachineLogEntries(log); + static Void assertLogEntries(RaftLog log, long expectedTerm, SimpleMessage[] expectedMessages, Consumer print) { + final List entries = getStateMachineLogEntries(log, print); try { assertLogEntries(entries, expectedTerm, expectedMessages); } catch(Exception t) { throw new AssertionError("entries: " + entries, t); } + return null; } static void assertLogEntries(List entries, long expectedTerm, SimpleMessage... expectedMessages) { diff --git a/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java b/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java new file mode 100644 index 0000000000..80c57741ce --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java @@ -0,0 +1,216 @@ +/* + * 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.ratis; + +import org.apache.ratis.RaftTestUtil.SimpleMessage; +import org.apache.ratis.client.RaftClient; +import org.apache.ratis.client.RaftClientConfigKeys; +import org.apache.ratis.client.impl.OrderedAsync; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.protocol.RaftClientReply; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.server.RaftServer; +import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog; +import org.apache.ratis.statemachine.StateMachine; +import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; +import org.apache.ratis.util.Slf4jUtils; +import org.apache.ratis.util.TimeDuration; +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.event.Level; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.apache.ratis.RaftTestUtil.waitForLeader; + +public abstract class RaftLogTruncateTests extends BaseTest + implements MiniRaftCluster.Factory.Get { + public static final int NUM_SERVERS = 5; + final TimeDuration MIN_TIMEOUT = TimeDuration.valueOf(3, TimeUnit.SECONDS); + + static SimpleMessage[] arraycopy(SimpleMessage[] src1, SimpleMessage[] src2) { + final SimpleMessage[] dst = new SimpleMessage[src1.length + src2.length]; + System.arraycopy(src1, 0, dst, 0, src1.length); + System.arraycopy(src2, 0, dst, src1.length, src2.length); + return dst; + } + + { + Slf4jUtils.setLogLevel(OrderedAsync.LOG, Level.ERROR); + Slf4jUtils.setLogLevel(RaftServerConfigKeys.LOG, Level.ERROR); + Slf4jUtils.setLogLevel(RaftClientConfigKeys.LOG, Level.ERROR); + + final RaftProperties p = getProperties(); + p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, SimpleStateMachine4Testing.class, StateMachine.class); + + // set a long rpc timeout so, when the leader does not have the majority, it won't step down fast. + RaftServerConfigKeys.Rpc.setTimeoutMin(p, MIN_TIMEOUT); + RaftServerConfigKeys.Rpc.setTimeoutMax(p, MIN_TIMEOUT.multiply(2)); + RaftServerConfigKeys.Rpc.setFirstElectionTimeoutMin(p, TimeDuration.ONE_SECOND); + RaftServerConfigKeys.Rpc.setFirstElectionTimeoutMax(p, TimeDuration.ONE_SECOND.multiply(2)); + } + + @Override + public int getGlobalTimeoutSeconds() { + return 200; + } + + @Test + public void testLogTruncate() throws Exception { + runWithNewCluster(NUM_SERVERS, this::runTestLogTruncate); + } + + void runTestLogTruncate(MiniRaftCluster cluster) throws Exception { + final RaftServer.Division oldLeader = waitForLeader(cluster); + final List oldFollowers = cluster.getFollowers(); + final List killedPeers = new ArrayList<>(); + final List remainingPeers = new ArrayList<>(); + + final int majorityIndex = NUM_SERVERS / 2 + 1; + Assert.assertEquals(NUM_SERVERS - 1, oldFollowers.size()); + Assert.assertTrue(majorityIndex < oldFollowers.size()); + + for (int i = 0; i < majorityIndex; i++) { + killedPeers.add(oldFollowers.get(i).getId()); + } + remainingPeers.add(oldLeader.getId()); + for (int i = majorityIndex; i < oldFollowers.size(); i++) { + remainingPeers.add(oldFollowers.get(i).getId()); + } + + try { + runTestLogTruncate(cluster, oldLeader, killedPeers, remainingPeers); + } catch (Throwable e) { + LOG.info("killedPeers : {}", killedPeers); + LOG.info("remainingPeers: {}", remainingPeers); + throw e; + } + } + + void runTestLogTruncate(MiniRaftCluster cluster, RaftServer.Division oldLeader, + List killedPeers, List remainingPeers) throws Exception { + final List exceptions = Collections.synchronizedList(new ArrayList<>()); + final long oldLeaderTerm = oldLeader.getInfo().getCurrentTerm(); + LOG.info("oldLeader: {}, term={}", oldLeader.getId(), oldLeaderTerm); + + final SimpleMessage[] firstBatch = SimpleMessage.create(5, "first"); + final SimpleMessage[] secondBatch = SimpleMessage.create(4, "second"); + + try (final RaftClient client = cluster.createClient(oldLeader.getId())) { + // send some messages + for (SimpleMessage batch : firstBatch) { + final RaftClientReply reply = client.io().send(batch); + Assert.assertTrue(reply.isSuccess()); + } + for (RaftServer.Division f : cluster.getFollowers()) { + assertLogEntries(f, oldLeaderTerm, firstBatch); + } + + // kill a majority of followers + LOG.info("Before killServer {}: {}", killedPeers, cluster.printServers()); + for (RaftPeerId f : killedPeers) { + cluster.killServer(f); + } + LOG.info("After killServer {}: {}", killedPeers, cluster.printServers()); + + // send more messages, but they won't be committed due to not enough followers + final SimpleMessage[] messagesToBeTruncated = SimpleMessage.create(3, "messagesToBeTruncated"); + final AtomicBoolean done = new AtomicBoolean(); + for (SimpleMessage message : messagesToBeTruncated) { + client.async().send(message).whenComplete((r, e) -> { + if (!done.get()) { + exceptions.add(new IllegalStateException(message + " is completed: reply=" + r, e)); + } + }); + } + + // check log messages + final SimpleMessage[] expectedMessages = arraycopy(firstBatch, messagesToBeTruncated); + for (RaftPeerId f : remainingPeers) { + assertLogEntries(cluster.getDivision(f), oldLeaderTerm, expectedMessages); + } + done.set(true); + LOG.info("done"); + } + + // kill the remaining servers + LOG.info("Before killServer {}: {}", remainingPeers, cluster.printServers()); + for (RaftPeerId f : remainingPeers) { + cluster.killServer(f); + } + LOG.info("After killServer {}: {}", remainingPeers, cluster.printServers()); + + // restart the earlier followers + for (RaftPeerId f : killedPeers) { + cluster.restartServer(f, false); + } + + // The new leader should be one of the earlier followers + final RaftServer.Division newLeader = waitForLeader(cluster); + LOG.info("After restartServer {}: {}", killedPeers, cluster.printServers()); + final long newLeaderTerm = newLeader.getInfo().getCurrentTerm(); + + final SegmentedRaftLog newLeaderLog = (SegmentedRaftLog) newLeader.getRaftLog(); + LOG.info("newLeader: {}, term {}, last={}", newLeader.getId(), newLeaderTerm, + newLeaderLog.getLastEntryTermIndex()); + Assert.assertTrue(killedPeers.contains(newLeader.getId())); + + // restart the remaining servers + for (RaftPeerId f : remainingPeers) { + cluster.restartServer(f, false); + } + + // check RaftLog truncate + for (RaftPeerId f : remainingPeers) { + assertLogEntries(cluster.getDivision(f), oldLeaderTerm, firstBatch); + } + + try (final RaftClient client = cluster.createClient(newLeader.getId())) { + // send more messages + for (SimpleMessage batch : secondBatch) { + final RaftClientReply reply = client.io().send(batch); + Assert.assertTrue(reply.isSuccess()); + } + } + + // check log messages -- it should be truncated and then append the new messages + final SimpleMessage[] expectedMessages = arraycopy(firstBatch, secondBatch); + for (RaftPeerId f : killedPeers) { + assertLogEntries(cluster.getDivision(f), oldLeaderTerm, expectedMessages); + } + + if (!exceptions.isEmpty()) { + LOG.info("{} exceptions", exceptions.size()); + for(int i = 0 ; i < exceptions.size(); i++) { + LOG.info("exception {})", i, exceptions.get(i)); + } + Assert.fail(); + } + } + + private void assertLogEntries(RaftServer.Division server, long term, SimpleMessage[] expectedMessages) + throws Exception { + RaftTestUtil.assertLogEntries(server, term, expectedMessages, 30, LOG); + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftLogTruncateWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftLogTruncateWithGrpc.java new file mode 100644 index 0000000000..dc28463747 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftLogTruncateWithGrpc.java @@ -0,0 +1,24 @@ +/* + * 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.ratis.grpc; + +import org.apache.ratis.RaftLogTruncateTests; + +public class TestRaftLogTruncateWithGrpc extends RaftLogTruncateTests + implements MiniRaftClusterWithGrpc.FactoryGet { +} diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java index bc0061f5f8..046453d582 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java @@ -73,12 +73,6 @@ public void testRequestTimeout() throws Exception { runWithNewCluster(NUM_SERVERS, cluster -> testRequestTimeout(false, cluster, LOG)); } - @Test - public void testStateMachineMetrics() throws Exception { - runWithNewCluster(NUM_SERVERS, cluster -> - testStateMachineMetrics(false, cluster, LOG)); - } - @Test public void testUpdateViaHeartbeat() throws Exception { runWithNewCluster(NUM_SERVERS, this::runTestUpdateViaHeartbeat); diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java index 264db89464..36e6dfbccf 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java @@ -154,7 +154,7 @@ private void testException(int retries, int maxAttempts, long sleepTime) { for (int i = 0; i < retries + 1; i++) { RetryPolicy.Action action = exceptionDependentRetry - .handleAttemptFailure(new ClientRetryEvent(i, null, exception)); + .handleAttemptFailure(TestRetryPolicy.newClientRetryEvent(i, null, exception)); final boolean expected = i < retries && i < maxAttempts; Assert.assertEquals(expected, action.shouldRetry()); diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java index d69cd1a2e6..1b9536b4b6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java @@ -33,6 +33,7 @@ import org.apache.ratis.protocol.exceptions.TimeoutIOException; import org.apache.ratis.protocol.exceptions.ResourceUnavailableException; import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.Timestamp; import org.junit.Assert; import org.junit.Test; @@ -70,6 +71,10 @@ public void testRetryMultipleTimesWithFixedSleep() { } } + static ClientRetryEvent newClientRetryEvent(int attemptCount, RaftClientRequest request, Throwable cause) { + return new ClientRetryEvent(attemptCount, request, attemptCount, cause, Timestamp.currentTime()); + } + @Test public void testRequestTypeDependentRetry() { final RequestTypeDependentRetryPolicy.Builder b = RequestTypeDependentRetryPolicy.newBuilder(); @@ -88,7 +93,7 @@ public void testRequestTypeDependentRetry() { RaftClientRequest.watchRequestType(1, ReplicationLevel.MAJORITY)); for(int i = 1; i < 2*n; i++) { { //write - final ClientRetryEvent event = new ClientRetryEvent(i, writeRequest, null); + final ClientRetryEvent event = newClientRetryEvent(i, writeRequest, null); final RetryPolicy.Action action = policy.handleAttemptFailure(event); final boolean expected = i < n; @@ -101,21 +106,21 @@ public void testRequestTypeDependentRetry() { } { //read and stale read are using default - final ClientRetryEvent event = new ClientRetryEvent(i, readRequest, null); + final ClientRetryEvent event = newClientRetryEvent(i, readRequest, null); final RetryPolicy.Action action = policy.handleAttemptFailure(event); Assert.assertTrue(action.shouldRetry()); Assert.assertEquals(0L, action.getSleepTime().getDuration()); } { - final ClientRetryEvent event = new ClientRetryEvent(i, staleReadRequest, null); + final ClientRetryEvent event = newClientRetryEvent(i, staleReadRequest, null); final RetryPolicy.Action action = policy.handleAttemptFailure(event); Assert.assertTrue(action.shouldRetry()); Assert.assertEquals(0L, action.getSleepTime().getDuration()); } { //watch has no retry - final ClientRetryEvent event = new ClientRetryEvent(i, watchRequest, null); + final ClientRetryEvent event = newClientRetryEvent(i, watchRequest, null); final RetryPolicy.Action action = policy.handleAttemptFailure(event); Assert.assertFalse(action.shouldRetry()); Assert.assertEquals(0L, action.getSleepTime().getDuration()); @@ -148,7 +153,7 @@ public RaftClientRequest newRequestImpl() { }; for (RaftClientRequest request : requests) { - final ClientRetryEvent event = new ClientRetryEvent(request, new Exception(), pending); + final ClientRetryEvent event = pending.newClientRetryEvent(request, new Exception()); final RetryPolicy.Action action = policy.handleAttemptFailure(event); Assert.assertTrue(action.shouldRetry()); Assert.assertEquals(0L, action.getSleepTime().getDuration()); @@ -156,7 +161,7 @@ public RaftClientRequest newRequestImpl() { timeout.sleep(); for (RaftClientRequest request : requests) { - final ClientRetryEvent event = new ClientRetryEvent(request, new Exception(), pending); + final ClientRetryEvent event = pending.newClientRetryEvent(request, new Exception()); final RetryPolicy.Action action = policy.handleAttemptFailure(event); Assert.assertFalse(action.shouldRetry()); } @@ -218,8 +223,7 @@ public void testRequestTypeDependentRetryWithExceptionDependentPolicy() throws E */ private void checkEvent(int exceptionAttemptCount, RetryPolicy retryPolicy, RaftClientRequest raftClientRequest, Throwable exception, Pair exceptionPolicyPair) { - final ClientRetryEvent event = - new ClientRetryEvent(exceptionAttemptCount, raftClientRequest, exception); + final ClientRetryEvent event = newClientRetryEvent(exceptionAttemptCount, raftClientRequest, exception); final RetryPolicy.Action action = retryPolicy.handleAttemptFailure(event); final boolean expected = exceptionAttemptCount < exceptionPolicyPair.retries; diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java index 73ff1eb53e..2f3edf7815 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java @@ -261,7 +261,7 @@ void runTestRestartCommitIndex(MiniRaftCluster cluster) throws Exception { final RaftPeerId leaderId = leader.getId(); ids.add(leaderId); - RaftTestUtil.getStateMachineLogEntries(leaderLog); + RaftTestUtil.getStateMachineLogEntries(leaderLog, LOG::info); // check that the last metadata entry is written to the log JavaUtils.attempt(() -> assertLastLogEntry(leader), 20, HUNDRED_MILLIS, "leader last metadata entry", LOG); From 79923c2d79c70dcf642c2c4bd24cfbf2501d0090 Mon Sep 17 00:00:00 2001 From: Ivan Andika <36403683+ivandika3@users.noreply.github.com> Date: Thu, 25 Jan 2024 01:14:49 +0800 Subject: [PATCH 19/76] RATIS-2008. Follower should recognize candidate if the candidate is the same peer as the current valid leader (#1024) --- .../main/java/org/apache/ratis/server/impl/VoteContext.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/VoteContext.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/VoteContext.java index 6375d652c0..3ba0c0e01e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/VoteContext.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/VoteContext.java @@ -102,8 +102,10 @@ private boolean checkLeader() { if (info.isFollower()) { final RaftPeerId leader = impl.getState().getLeaderId(); if (leader != null + && !leader.equals(candidateId) && impl.getRole().getFollowerState().map(FollowerState::isCurrentLeaderValid).orElse(false)) { - return reject("this server is a follower and still has a valid leader " + leader); + return reject("this server is a follower and still has a valid leader " + leader + + " different than the candidate " + candidateId); } } return true; From e2aaddca9b850e978ee7a1634cadac76e81559c4 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Wed, 24 Jan 2024 22:54:32 -0800 Subject: [PATCH 20/76] RATIS-2007. Zero-copy buffers are not released (#1027) --- .../RaftClientAsynchronousProtocol.java | 6 ++--- .../org/apache/ratis/util/SlidingWindow.java | 12 +++++++-- .../server/GrpcClientProtocolService.java | 17 +++++++++--- .../ratis/server/impl/RaftServerImpl.java | 26 +++++++++++-------- .../server/raftlog/segmented/LogSegment.java | 1 + 5 files changed, 43 insertions(+), 19 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java index 1985bbe667..222ccff057 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java @@ -46,11 +46,11 @@ default CompletableFuture submitClientRequestAsync( ReferenceCountedObject requestRef) { try { // for backward compatibility - return submitClientRequestAsync(requestRef.retain()) - .whenComplete((r, e) -> requestRef.release()); + return submitClientRequestAsync(requestRef.retain()); } catch (Exception e) { - requestRef.release(); return JavaUtils.completeExceptionally(e); + } finally { + requestRef.release(); } } } \ No newline at end of file diff --git a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java index 732e3d890e..7adc7a7210 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java @@ -52,6 +52,9 @@ interface Request { boolean hasReply(); void fail(Throwable e); + + default void release() { + } } interface ClientSideRequest extends Request { @@ -170,8 +173,12 @@ void endOfRequests(long nextToProcess, REQUEST end, Consumer replyMetho putNewRequest(end); } - void clear() { + void clear(long nextToProcess) { LOG.debug("close {}", this); + final SortedMap tail = requests.tailMap(nextToProcess); + for (REQUEST r : tail.values()) { + r.release(); + } requests.clear(); } @@ -469,6 +476,7 @@ private void processRequestsFromHead(Consumer processingMethod) { return; } else if (r.getSeqNum() == nextToProcess) { processingMethod.accept(r); + r.release(); nextToProcess++; } } @@ -514,7 +522,7 @@ public synchronized boolean endOfRequests(Consumer replyMethod) { @Override public void close() { - requests.clear(); + requests.clear(nextToProcess); } } } \ No newline at end of file diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index 67e75d6063..0671a1841d 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -96,6 +96,13 @@ ReferenceCountedObject getRequestRef() { return requestRef; } + @Override + public void release() { + if (requestRef != null) { + requestRef.release(); + } + } + @Override public long getSeqNum() { return request != null? request.getSlidingWindowEntry().getSeqNum(): Long.MAX_VALUE; @@ -363,7 +370,6 @@ void processClientRequest(PendingOrderedRequest pending) { final long seq = pending.getSeqNum(); processClientRequest(pending.getRequestRef(), reply -> slidingWindow.receiveReply(seq, reply, this::sendReply)); - pending.getRequestRef().release(); } @Override @@ -378,7 +384,6 @@ void processClientRequest(ReferenceCountedObject requestRef) final RaftGroupId requestGroupId = request.getRaftGroupId(); // use the group id in the first request as the group id of this observer final RaftGroupId updated = groupId.updateAndGet(g -> g != null ? g : requestGroupId); - final PendingOrderedRequest pending = new PendingOrderedRequest(requestRef); if (!requestGroupId.equals(updated)) { final GroupMismatchException exception = new GroupMismatchException(getId() @@ -387,7 +392,13 @@ void processClientRequest(ReferenceCountedObject requestRef) responseError(exception, () -> "processClientRequest (Group mismatched) for " + request); return; } - slidingWindow.receivedRequest(pending, this::processClientRequest); + final PendingOrderedRequest pending = new PendingOrderedRequest(requestRef); + try { + slidingWindow.receivedRequest(pending, this::processClientRequest); + } catch (Exception e) { + pending.release(); + throw e; + } } finally { requestRef.release(); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 8ad835474c..396243b1c0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -887,15 +887,19 @@ public CompletableFuture submitClientRequestAsync( return CompletableFuture.completedFuture(reply); } - final Timekeeper timer = raftServerMetrics.getClientRequestTimer(request.getType()); - final Optional timerContext = Optional.ofNullable(timer).map(Timekeeper::time); - return replyFuture(requestRef).whenComplete((clientReply, exception) -> { + try { + RaftClientRequest.Type type = request.getType(); + final Timekeeper timer = raftServerMetrics.getClientRequestTimer(type); + final Optional timerContext = Optional.ofNullable(timer).map(Timekeeper::time); + return replyFuture(requestRef).whenComplete((clientReply, exception) -> { + timerContext.ifPresent(Timekeeper.Context::stop); + if (exception != null || clientReply.getException() != null) { + raftServerMetrics.incFailedRequestCount(type); + } + }); + } finally { requestRef.release(); - timerContext.ifPresent(Timekeeper.Context::stop); - if (exception != null || clientReply.getException() != null) { - raftServerMetrics.incFailedRequestCount(request.getType()); - } - }); + } } private CompletableFuture replyFuture(ReferenceCountedObject requestRef) { @@ -1479,12 +1483,12 @@ public CompletableFuture appendEntriesAsync( preAppendEntriesAsync(requestorId, ProtoUtils.toRaftGroupId(request.getRaftGroupId()), r.getLeaderTerm(), previous, r.getLeaderCommit(), r.getInitializing(), entries); return appendEntriesAsync(requestorId, r.getLeaderTerm(), previous, r.getLeaderCommit(), - request.getCallId(), r.getInitializing(), r.getCommitInfosList(), entries, requestRef) - .whenComplete((reply, e) -> requestRef.release()); + request.getCallId(), r.getInitializing(), r.getCommitInfosList(), entries, requestRef); } catch(Exception t) { LOG.error("{}: Failed appendEntriesAsync {}", getMemberId(), r, t); - requestRef.release(); throw t; + } finally { + requestRef.release(); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 0750d2cc8c..68da35014f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -245,6 +245,7 @@ public LogEntryProto load(LogRecord key) throws IOException { if (ti.equals(key.getTermIndex())) { toReturn.set(entry); } + entryRef.release(); }); loadingTimes.incrementAndGet(); return Objects.requireNonNull(toReturn.get()); From a18d08c89cd92e52a6c5c075fed3df59012ce4b6 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 25 Jan 2024 08:42:03 +0100 Subject: [PATCH 21/76] RATIS-2015. Node.js 16 actions are deprecated (#1028) --- .github/workflows/post-commit.yml | 58 +++++++++++++++---------------- .github/workflows/repeat-test.yml | 10 +++--- 2 files changed, 34 insertions(+), 34 deletions(-) diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml index 9978caacf8..2d3258ccc5 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yml @@ -23,9 +23,9 @@ jobs: runs-on: ubuntu-20.04 steps: - name: Checkout project - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Cache for maven dependencies - uses: actions/cache@v3 + uses: actions/cache@v4 with: path: | ~/.m2/repository @@ -34,14 +34,14 @@ jobs: restore-keys: | maven-repo- - name: Setup java - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 8 - name: Run a full build run: ./dev-support/checks/build.sh -Prelease assembly:single - name: Store binaries for tests - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: ratis-bin path: | @@ -49,7 +49,7 @@ jobs: !ratis-assembly/target/apache-ratis-*-src.tar.gz retention-days: 1 - name: Store source tarball for compilation - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: ratis-src path: ratis-assembly/target/apache-ratis-*-src.tar.gz @@ -64,14 +64,14 @@ jobs: fail-fast: false steps: - name: Download source tarball - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: ratis-src - name: Untar sources run: | tar --strip-components 1 -xzvf apache-ratis-*-src.tar.gz - name: Cache for maven dependencies - uses: actions/cache/restore@v3 + uses: actions/cache/restore@v4 with: path: | ~/.m2/repository @@ -80,7 +80,7 @@ jobs: restore-keys: | maven-repo- - name: Setup java - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: ${{ matrix.java }} @@ -91,9 +91,9 @@ jobs: runs-on: ubuntu-20.04 steps: - name: Checkout project - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Cache for maven dependencies - uses: actions/cache/restore@v3 + uses: actions/cache/restore@v4 with: path: | ~/.m2/repository @@ -104,7 +104,7 @@ jobs: - name: Run tests run: ./dev-support/checks/rat.sh - name: Upload results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: rat @@ -114,11 +114,11 @@ jobs: runs-on: ubuntu-20.04 steps: - name: Checkout project - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Run tests run: ./dev-support/checks/author.sh - name: Upload results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: author @@ -140,9 +140,9 @@ jobs: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts # REMOVE CODE ABOVE WHEN ISSUE IS ADDRESSED! - name: Checkout project - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Cache for maven dependencies - uses: actions/cache/restore@v3 + uses: actions/cache/restore@v4 with: path: | ~/.m2/repository @@ -151,7 +151,7 @@ jobs: restore-keys: | maven-repo- - name: Setup java - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 8 @@ -161,7 +161,7 @@ jobs: run: cat target/${{ github.job }}/summary.txt if: ${{ !cancelled() }} - name: Upload results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !cancelled() }} with: name: unit-${{ matrix.profile }} @@ -171,9 +171,9 @@ jobs: runs-on: ubuntu-20.04 steps: - name: Checkout project - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Cache for maven dependencies - uses: actions/cache/restore@v3 + uses: actions/cache/restore@v4 with: path: | ~/.m2/repository @@ -184,7 +184,7 @@ jobs: - name: Run tests run: ./dev-support/checks/checkstyle.sh - name: Upload results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: checkstyle @@ -194,14 +194,14 @@ jobs: runs-on: ubuntu-20.04 steps: - name: Setup java - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 8 - name: Checkout project - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Cache for maven dependencies - uses: actions/cache/restore@v3 + uses: actions/cache/restore@v4 with: path: | ~/.m2/repository @@ -212,7 +212,7 @@ jobs: - name: Run tests run: ./dev-support/checks/findbugs.sh - name: Upload results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: findbugs @@ -225,11 +225,11 @@ jobs: if: (github.repository == 'apache/ratis' || github.repository == 'apache/incubator-ratis') && github.event_name != 'pull_request' steps: - name: Checkout project - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 - name: Cache for maven dependencies - uses: actions/cache/restore@v3 + uses: actions/cache/restore@v4 with: path: | ~/.m2/repository @@ -238,12 +238,12 @@ jobs: restore-keys: | maven-repo- - name: Setup java 17 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 17 - name: Download artifacts - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: path: target/artifacts - name: Untar binaries @@ -258,7 +258,7 @@ jobs: SONAR_TOKEN: ${{ secrets.SONARCLOUD_TOKEN }} GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - name: Archive build results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: ${{ github.job }} diff --git a/.github/workflows/repeat-test.yml b/.github/workflows/repeat-test.yml index 86150d2598..e3c05bec6a 100644 --- a/.github/workflows/repeat-test.yml +++ b/.github/workflows/repeat-test.yml @@ -87,11 +87,11 @@ jobs: split: ${{ fromJson(needs.prepare.outputs.matrix) }} fail-fast: ${{ fromJson(github.event.inputs.fail-fast) }} steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 with: ref: ${{ github.event.inputs.ref }} - name: Cache for maven dependencies - uses: actions/cache@v3 + uses: actions/cache@v4 with: path: | ~/.m2/repository @@ -100,7 +100,7 @@ jobs: restore-keys: | maven-repo- - name: Setup java - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 8 @@ -113,7 +113,7 @@ jobs: run: dev-support/checks/_summary.sh target/unit/summary.txt if: ${{ !cancelled() }} - name: Archive build results - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: always() with: name: result-${{ env.TEST_CLASS }}-split-${{ matrix.split }} @@ -124,7 +124,7 @@ jobs: runs-on: ubuntu-20.04 steps: - name: Download build results - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 - name: Count failures run: | failures=$(find . -name 'summary.txt' | grep -v 'iteration' | xargs grep -v 'exit code: 0' | wc -l) From 5560718aba3ec9cb1447f7305ba4e23b2567effa Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Thu, 25 Jan 2024 08:07:23 -0800 Subject: [PATCH 22/76] RATIS-2011. When a log entry is truncated, remove TransactionContext. (#1029) --- .../ratis/client/impl/OrderedAsync.java | 11 ++- .../org/apache/ratis/util/BatchLogger.java | 12 ++- .../java/org/apache/ratis/util/JavaUtils.java | 16 +--- .../ratis/server/impl/RaftServerImpl.java | 21 +++- .../ratis/server/impl/TransactionManager.java | 49 ++++++++-- .../ratis/server/raftlog/LogProtoUtils.java | 18 +++- .../ratis/server/raftlog/RaftLogBase.java | 3 +- .../java/org/apache/ratis/RaftTestUtil.java | 45 ++++++++- .../ratis/server/impl/RaftServerTestUtil.java | 13 +++ .../ratis/server/impl/RetryCacheTestUtil.java | 4 + .../apache/ratis/RaftLogTruncateTests.java | 95 ++++++++++++++----- 11 files changed, 231 insertions(+), 56 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java index 34dc3be113..09c6cd4ac9 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java @@ -32,6 +32,7 @@ import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.rpc.CallId; +import org.apache.ratis.util.BatchLogger; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; @@ -50,6 +51,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.LongFunction; @@ -57,6 +59,10 @@ public final class OrderedAsync { public static final Logger LOG = LoggerFactory.getLogger(OrderedAsync.class); + private enum BatchLogKey implements BatchLogger.Key { + SEND_REQUEST_EXCEPTION + } + static class PendingOrderedRequest extends PendingClientRequest implements SlidingWindow.ClientSideRequest { private final long callId; @@ -204,7 +210,10 @@ private void sendRequestWithRetry(PendingOrderedRequest pending) { getSlidingWindow(request).receiveReply( request.getSlidingWindowEntry().getSeqNum(), reply, this::sendRequestWithRetry); }).exceptionally(e -> { - LOG.error(client.getId() + ": Failed* " + request, e); + final Throwable exception = e; + final String key = client.getId() + "-" + request.getCallId() + "-" + exception; + final Consumer op = suffix -> LOG.error("{} {}: Failed* {}", suffix, client.getId(), request, exception); + BatchLogger.warn(BatchLogKey.SEND_REQUEST_EXCEPTION, key, op); handleException(pending, request, e); return null; }); diff --git a/ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java b/ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java index 9ccd66ad71..38dad5c499 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java @@ -35,7 +35,13 @@ public final class BatchLogger { private BatchLogger() { } - public interface Key {} + public interface Key { + TimeDuration DEFAULT_DURATION = TimeDuration.valueOf(5, TimeUnit.SECONDS); + + default TimeDuration getBatchDuration() { + return DEFAULT_DURATION; + } + } private static final class UniqueId { private final Key key; @@ -93,6 +99,10 @@ private synchronized boolean tryStartBatch(Consumer op) { private static final TimeoutExecutor SCHEDULER = TimeoutExecutor.getInstance(); private static final ConcurrentMap LOG_CACHE = new ConcurrentHashMap<>(); + public static void warn(Key key, String name, Consumer op) { + warn(key, name, op, key.getBatchDuration(), true); + } + public static void warn(Key key, String name, Consumer op, TimeDuration batchDuration) { warn(key, name, op, batchDuration, true); } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java index 00725903a7..d3f899a7fd 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java @@ -41,7 +41,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; -import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -229,7 +228,7 @@ static RETURN attempt( } if (log != null && log.isWarnEnabled()) { log.warn("FAILED \"" + name.get() + "\", attempt #" + i + "/" + numAttempts - + ": " + t + ", sleep " + sleepTime + " and then retry.", t); + + ", sleep " + sleepTime + " and then retry: " + t); } } @@ -245,19 +244,6 @@ static void attempt( attemptRepeatedly(CheckedRunnable.asCheckedSupplier(runnable), numAttempts, sleepTime, name, log); } - /** Attempt to wait the given condition to return true multiple times. */ - static void attemptUntilTrue( - BooleanSupplier condition, int numAttempts, TimeDuration sleepTime, String name, Logger log) - throws InterruptedException { - Objects.requireNonNull(condition, "condition == null"); - attempt(() -> { - if (!condition.getAsBoolean()) { - throw new IllegalStateException("Condition " + name + " is false."); - } - }, numAttempts, sleepTime, name, log); - } - - static Timer runRepeatedly(Runnable runnable, long delay, long period, TimeUnit unit) { final Timer timer = new Timer(true); timer.schedule(new TimerTask() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 396243b1c0..8f6c9273ed 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -90,6 +90,7 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.TransactionContextImpl; +import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.CollectionUtils; @@ -179,7 +180,7 @@ public long[] getFollowerNextIndices() { private final DataStreamMap dataStreamMap; private final RaftServerConfigKeys.Read.Option readOption; - private final TransactionManager transactionManager = new TransactionManager(); + private final TransactionManager transactionManager; private final RetryCacheImpl retryCache; private final CommitInfoCache commitInfoCache = new CommitInfoCache(); private final WriteIndexCache writeIndexCache; @@ -225,6 +226,7 @@ public long[] getFollowerNextIndices() { this.dataStreamMap = new DataStreamMapImpl(id); this.readOption = RaftServerConfigKeys.Read.option(properties); this.writeIndexCache = new WriteIndexCache(properties); + this.transactionManager = new TransactionManager(id); this.leaderElectionMetrics = LeaderElectionMetrics.getLeaderElectionMetrics( getMemberId(), state::getLastLeaderElapsedTimeMs); @@ -1821,7 +1823,7 @@ private CompletableFuture replyPendingRequest( } return stateMachineFuture.whenComplete((reply, exception) -> { - transactionManager.remove(termIndex); + getTransactionManager().remove(termIndex); final RaftClientReply.Builder b = newReplyBuilder(invocationId, termIndex.getIndex()); final RaftClientReply r; if (exception == null) { @@ -1839,6 +1841,15 @@ private CompletableFuture replyPendingRequest( }); } + TransactionManager getTransactionManager() { + return transactionManager; + } + + @VisibleForTesting + Map> getTransactionContextMapForTesting() { + return getTransactionManager().getMap(); + } + TransactionContext getTransactionContext(LogEntryProto entry, Boolean createNew) { if (!entry.hasStateMachineLogEntry()) { return null; @@ -1854,9 +1865,9 @@ TransactionContext getTransactionContext(LogEntryProto entry, Boolean createNew) } if (!createNew) { - return transactionManager.get(termIndex); + return getTransactionManager().get(termIndex); } - return transactionManager.computeIfAbsent(termIndex, + return getTransactionManager().computeIfAbsent(termIndex, // call startTransaction only once MemoizedSupplier.valueOf(() -> stateMachine.startTransaction(entry, getInfo().getCurrentRole()))); } @@ -1898,6 +1909,8 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf */ void notifyTruncatedLogEntry(LogEntryProto logEntry) { if (logEntry.hasStateMachineLogEntry()) { + getTransactionManager().remove(TermIndex.valueOf(logEntry)); + final ClientInvocationId invocationId = ClientInvocationId.valueOf(logEntry.getStateMachineLogEntry()); final CacheEntry cacheEntry = getRetryCache().getIfPresent(invocationId); if (cacheEntry != null) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java index 283900fb62..c33bc26bce 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java @@ -19,27 +19,64 @@ import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.statemachine.TransactionContext; +import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.MemoizedSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.Collections; +import java.util.Map; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.function.Supplier; /** * Managing {@link TransactionContext}. */ class TransactionManager { - private final ConcurrentMap> contexts = new ConcurrentHashMap<>(); + static final Logger LOG = LoggerFactory.getLogger(TransactionManager.class); + + private final String name; + private final ConcurrentMap> contexts = new ConcurrentHashMap<>(); + + TransactionManager(Object name) { + this.name = name + "-" + JavaUtils.getClassSimpleName(getClass()); + } + + @VisibleForTesting + Map> getMap() { + LOG.debug("{}", this); + return Collections.unmodifiableMap(contexts); + } TransactionContext get(TermIndex termIndex) { - return Optional.ofNullable(contexts.get(termIndex)).map(Supplier::get).orElse(null); + return Optional.ofNullable(contexts.get(termIndex)).map(MemoizedSupplier::get).orElse(null); } - TransactionContext computeIfAbsent(TermIndex termIndex, Supplier constructor) { - return contexts.computeIfAbsent(termIndex, i -> constructor).get(); + TransactionContext computeIfAbsent(TermIndex termIndex, MemoizedSupplier constructor) { + final MemoizedSupplier m = contexts.computeIfAbsent(termIndex, i -> constructor); + if (!m.isInitialized()) { + LOG.debug("{}: {}", termIndex, this); + } + return m.get(); } void remove(TermIndex termIndex) { - contexts.remove(termIndex); + final MemoizedSupplier removed = contexts.remove(termIndex); + if (removed != null) { + LOG.debug("{}: {}", termIndex, this); + } + } + + @Override + public String toString() { + if (contexts.isEmpty()) { + return name + " "; + } + + final StringBuilder b = new StringBuilder(name); + contexts.forEach((k, v) -> b.append("\n ").append(k).append(": initialized? ").append(v.isInitialized())); + return b.toString(); } } \ No newline at end of file diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java index b75777c9e7..de06faf63e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java @@ -53,17 +53,27 @@ public static String toLogEntryString(LogEntryProto entry, Function peers) { + return peers.stream().map(AbstractMessage::toString) + .map(s -> s.replace("\n", "")) + .map(s -> s.replace(" ", "")) + .collect(Collectors.joining(", ")); + } + + static String stateMachineLogEntryProtoToString(StateMachineLogEntryProto p) { + return "logData:" + p.getLogData() + ", stateMachineEntry:" + p.getType() + ":" + p.getStateMachineEntry(); + } + public static String toLogEntryString(LogEntryProto entry) { - return toLogEntryString(entry, null); + return toLogEntryString(entry, LogProtoUtils::stateMachineLogEntryProtoToString); } public static String toLogEntriesString(List entries) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index b56b343bd8..49e66e253d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -375,7 +375,8 @@ protected List> appendImpl(ReferenceCountedObject getLogEntryProtos(RaftLog log) { }); } + Comparator LOG_ENTRY_PROTO_COMPARATOR = Comparator.comparing( + e -> e.getStateMachineLogEntry().getLogData().asReadOnlyByteBuffer()); + Comparator SIMPLE_MESSAGE_COMPARATOR = Comparator.comparing( + m -> m.getContent().asReadOnlyByteBuffer()); + + /** @return a map of message-array-index to {@link LogEntryProto} for the entries found in the given log. */ + static Map getStateMachineLogEntries(RaftLog log, SimpleMessage[] messages) { + if (messages.length == 0) { + return Collections.emptyMap(); + } + final List entries = getStateMachineLogEntries(log, s -> {}); + if (entries.isEmpty()) { + return Collections.emptyMap(); + } + + entries.sort(LOG_ENTRY_PROTO_COMPARATOR); + Arrays.sort(messages, SIMPLE_MESSAGE_COMPARATOR); + + final Map found = new HashMap<>(); + for (int e = 0, m = 0; e < entries.size() && m < messages.length; ) { + final int diff = messages[m].getContent().asReadOnlyByteBuffer().compareTo( + entries.get(e).getStateMachineLogEntry().getLogData().asReadOnlyByteBuffer()); + if (diff == 0) { + found.put(m, entries.get(e)); + m++; + e++; + } else if (diff < 0) { + m++; // message < entry + } else { + e++; // message > entry + } + } + + Assertions.assertEquals(messages.length, found.size()); + return found; + } + static List getStateMachineLogEntries(RaftLog log, Consumer print) { final List entries = new ArrayList<>(); for (LogEntryProto e : getLogEntryProtos(log)) { @@ -306,8 +348,7 @@ static void assertLogEntries(List entries, long expectedTerm, Sim } Assert.assertTrue(e.getIndex() > logIndex); logIndex = e.getIndex(); - Assert.assertArrayEquals(expectedMessages[i].getContent().toByteArray(), - e.getStateMachineLogEntry().getLogData().toByteArray()); + Assert.assertEquals(expectedMessages[i].getContent(), e.getStateMachineLogEntry().getLogData()); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java index 2927ec3495..58a51e0514 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java @@ -31,9 +31,12 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerRpc; import org.apache.ratis.server.leader.LogAppender; +import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog; import org.apache.ratis.server.storage.RaftStorage; +import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedConsumer; @@ -49,6 +52,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -138,6 +142,15 @@ public static ConfigurationManager getConfigurationManager(RaftServer.Division s return (ConfigurationManager) RaftTestUtil.getDeclaredField(getState(server), "configurationManager"); } + public static Logger getTransactionContextLog() { + return TransactionManager.LOG; + } + + public static Map> getTransactionContextMap( + RaftServer.Division server) { + return ((RaftServerImpl)server).getTransactionContextMapForTesting(); + } + public static RaftConfiguration newRaftConfiguration(Collection peers) { return RaftConfigurationImpl.newBuilder().setConf(peers).build(); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java index f59958a943..e5a55e49ca 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java @@ -80,7 +80,11 @@ public static SegmentedRaftLog newSegmentedRaftLog(RaftGroupMemberId memberId, R final RaftServerImpl server = mock(RaftServerImpl.class); when(server.getRetryCache()).thenReturn((RetryCacheImpl) retryCache); when(server.getMemberId()).thenReturn(memberId); + + final TransactionManager transactionManager = new TransactionManager(memberId.getPeerId()); + when(server.getTransactionManager()).thenReturn(transactionManager); doCallRealMethod().when(server).notifyTruncatedLogEntry(any(LogEntryProto.class)); + return SegmentedRaftLog.newBuilder() .setMemberId(memberId) .setServer(server) diff --git a/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java b/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java index 80c57741ce..c21110ea0e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java @@ -22,23 +22,32 @@ import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.client.impl.OrderedAsync; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.protocol.RaftClientReply; +import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.impl.RaftServerTestUtil; +import org.apache.ratis.server.protocol.TermIndex; +import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog; import org.apache.ratis.statemachine.StateMachine; +import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; +import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -57,9 +66,12 @@ static SimpleMessage[] arraycopy(SimpleMessage[] src1, SimpleMessage[] src2) { } { + Slf4jUtils.setLogLevel(RaftServerTestUtil.getTransactionContextLog(), Level.TRACE); + Slf4jUtils.setLogLevel(OrderedAsync.LOG, Level.ERROR); Slf4jUtils.setLogLevel(RaftServerConfigKeys.LOG, Level.ERROR); Slf4jUtils.setLogLevel(RaftClientConfigKeys.LOG, Level.ERROR); + Slf4jUtils.setLogLevel(RaftClientConfigKeys.LOG, Level.ERROR); final RaftProperties p = getProperties(); p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, SimpleStateMachine4Testing.class, StateMachine.class); @@ -88,8 +100,8 @@ void runTestLogTruncate(MiniRaftCluster cluster) throws Exception { final List remainingPeers = new ArrayList<>(); final int majorityIndex = NUM_SERVERS / 2 + 1; - Assert.assertEquals(NUM_SERVERS - 1, oldFollowers.size()); - Assert.assertTrue(majorityIndex < oldFollowers.size()); + Assertions.assertEquals(NUM_SERVERS - 1, oldFollowers.size()); + Assertions.assertTrue(majorityIndex < oldFollowers.size()); for (int i = 0; i < majorityIndex; i++) { killedPeers.add(oldFollowers.get(i).getId()); @@ -113,18 +125,26 @@ void runTestLogTruncate(MiniRaftCluster cluster, RaftServer.Division oldLeader, final List exceptions = Collections.synchronizedList(new ArrayList<>()); final long oldLeaderTerm = oldLeader.getInfo().getCurrentTerm(); LOG.info("oldLeader: {}, term={}", oldLeader.getId(), oldLeaderTerm); + LOG.info("killedPeers : {}", killedPeers); + LOG.info("remainingPeers: {}", remainingPeers); final SimpleMessage[] firstBatch = SimpleMessage.create(5, "first"); final SimpleMessage[] secondBatch = SimpleMessage.create(4, "second"); + for (RaftPeer peer : cluster.getGroup().getPeers()) { + assertEmptyTransactionContextMap(cluster.getDivision(peer.getId())); + } + try (final RaftClient client = cluster.createClient(oldLeader.getId())) { // send some messages for (SimpleMessage batch : firstBatch) { final RaftClientReply reply = client.io().send(batch); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } - for (RaftServer.Division f : cluster.getFollowers()) { - assertLogEntries(f, oldLeaderTerm, firstBatch); + for (RaftPeer peer : cluster.getGroup().getPeers()) { + final RaftServer.Division division = cluster.getDivision(peer.getId()); + assertLogEntries(division, oldLeaderTerm, firstBatch); + assertEmptyTransactionContextMap(division); } // kill a majority of followers @@ -148,17 +168,19 @@ void runTestLogTruncate(MiniRaftCluster cluster, RaftServer.Division oldLeader, // check log messages final SimpleMessage[] expectedMessages = arraycopy(firstBatch, messagesToBeTruncated); for (RaftPeerId f : remainingPeers) { - assertLogEntries(cluster.getDivision(f), oldLeaderTerm, expectedMessages); + final RaftServer.Division division = cluster.getDivision(f); + assertLogEntries(division, oldLeaderTerm, expectedMessages); + if (!division.getId().equals(oldLeader.getId())) { + assertEntriesInTransactionContextMap(division, messagesToBeTruncated, firstBatch); + } } done.set(true); LOG.info("done"); } - // kill the remaining servers - LOG.info("Before killServer {}: {}", remainingPeers, cluster.printServers()); - for (RaftPeerId f : remainingPeers) { - cluster.killServer(f); - } + // kill the old leader + LOG.info("Before killServer {}: {}", oldLeader.getId(), cluster.printServers()); + cluster.killServer(oldLeader.getId()); LOG.info("After killServer {}: {}", remainingPeers, cluster.printServers()); // restart the earlier followers @@ -174,12 +196,10 @@ void runTestLogTruncate(MiniRaftCluster cluster, RaftServer.Division oldLeader, final SegmentedRaftLog newLeaderLog = (SegmentedRaftLog) newLeader.getRaftLog(); LOG.info("newLeader: {}, term {}, last={}", newLeader.getId(), newLeaderTerm, newLeaderLog.getLastEntryTermIndex()); - Assert.assertTrue(killedPeers.contains(newLeader.getId())); + Assertions.assertTrue(killedPeers.contains(newLeader.getId())); - // restart the remaining servers - for (RaftPeerId f : remainingPeers) { - cluster.restartServer(f, false); - } + // restart the old leader + cluster.restartServer(oldLeader.getId(), false); // check RaftLog truncate for (RaftPeerId f : remainingPeers) { @@ -190,14 +210,16 @@ void runTestLogTruncate(MiniRaftCluster cluster, RaftServer.Division oldLeader, // send more messages for (SimpleMessage batch : secondBatch) { final RaftClientReply reply = client.io().send(batch); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } // check log messages -- it should be truncated and then append the new messages final SimpleMessage[] expectedMessages = arraycopy(firstBatch, secondBatch); - for (RaftPeerId f : killedPeers) { - assertLogEntries(cluster.getDivision(f), oldLeaderTerm, expectedMessages); + for (RaftPeer peer : cluster.getGroup().getPeers()) { + final RaftServer.Division division = cluster.getDivision(peer.getId()); + assertLogEntries(division, oldLeaderTerm, expectedMessages); + assertEmptyTransactionContextMap(division); } if (!exceptions.isEmpty()) { @@ -205,7 +227,36 @@ void runTestLogTruncate(MiniRaftCluster cluster, RaftServer.Division oldLeader, for(int i = 0 ; i < exceptions.size(); i++) { LOG.info("exception {})", i, exceptions.get(i)); } - Assert.fail(); + Assertions.fail(); + } + } + + static void assertEmptyTransactionContextMap(RaftServer.Division division) { + Assertions.assertTrue(RaftServerTestUtil.getTransactionContextMap(division).isEmpty(), + () -> division.getId() + " TransactionContextMap is non-empty"); + } + + static void assertEntriesInTransactionContextMap(RaftServer.Division division, + SimpleMessage[] existing, SimpleMessage[] nonExisting) { + final RaftLog log = division.getRaftLog(); + assertEntriesInTransactionContextMap(division, + RaftTestUtil.getStateMachineLogEntries(log, existing).values(), + RaftTestUtil.getStateMachineLogEntries(log, nonExisting).values()); + } + + static void assertEntriesInTransactionContextMap(RaftServer.Division division, + Collection existing, Collection nonExisting) { + final Map> map + = RaftServerTestUtil.getTransactionContextMap(division); + for(LogEntryProto e : existing) { + final TermIndex termIndex = TermIndex.valueOf(e); + Assertions.assertTrue(map.containsKey(termIndex), + () -> termIndex + " not found in " + division.getId()); + } + for(LogEntryProto e : nonExisting) { + final TermIndex termIndex = TermIndex.valueOf(e); + Assertions.assertFalse(map.containsKey(termIndex), + () -> termIndex + " found in " + division.getId()); } } From 7d587ec03c3d6e592a1fd544d93094fdeb229c45 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Sat, 27 Jan 2024 13:10:03 -0800 Subject: [PATCH 23/76] RATIS-2017: checkAndEvictCache must be done in writeLock. (#1031) --- .../server/raftlog/segmented/SegmentedRaftLog.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 92b2d42337..4e057c07b2 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -326,11 +326,13 @@ public EntryWithData getEntryWithData(long index) throws RaftLogIOException { private void checkAndEvictCache() { if (cache.shouldEvict()) { - // TODO if the cache is hitting the maximum size and we cannot evict any - // segment's cache, should block the new entry appending or new segment - // allocation. - cache.evictCache(server.getFollowerNextIndices(), fileLogWorker.getSafeCacheEvictIndex(), - server.getLastAppliedIndex()); + try (AutoCloseableLock ignored = writeLock()){ + // TODO if the cache is hitting the maximum size and we cannot evict any + // segment's cache, should block the new entry appending or new segment + // allocation. + cache.evictCache(server.getFollowerNextIndices(), fileLogWorker.getSafeCacheEvictIndex(), + server.getLastAppliedIndex()); + } } } From 035cd968f934f4a8f9d9ccec082f86b0e6437aab Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Mon, 29 Jan 2024 00:07:22 +0800 Subject: [PATCH 24/76] RATIS-2016. Correct NotificationInstallSnapshot's index (#1030) --- .../ratis/grpc/server/GrpcLogAppender.java | 32 ++++++++++++++++--- 1 file changed, 28 insertions(+), 4 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 9a2e84f21c..ec44d8c485 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -41,7 +41,9 @@ import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult; import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto; +import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto.InstallSnapshotReplyBodyCase; import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; +import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto.InstallSnapshotRequestBodyCase; import org.apache.ratis.statemachine.SnapshotInfo; import org.apache.ratis.util.*; import org.slf4j.Logger; @@ -67,6 +69,8 @@ private enum BatchLogKey implements BatchLogger.Key { APPEND_LOG_RESPONSE_HANDLER_ON_ERROR } + public static final int INSTALL_SNAPSHOT_NOTIFICATION_INDEX = 0; + private static final Comparator CALL_ID_COMPARATOR = (left, right) -> { // calculate diff in order to take care the possibility of numerical overflow final long diff = left - right; @@ -577,15 +581,35 @@ private class InstallSnapshotResponseHandler implements StreamObserver Date: Mon, 29 Jan 2024 21:58:26 +0530 Subject: [PATCH 25/76] RATIS-1974. Migrate ratis-metrics-default tests to Junit 5 (#1034) --- ratis-metrics-default/pom.xml | 11 +++++++++++ .../metrics/TestMetricRegistriesLoader.java | 18 +++++++++--------- .../ratis/metrics/impl/TestRefCountingMap.java | 18 +++++++++--------- 3 files changed, 29 insertions(+), 18 deletions(-) diff --git a/ratis-metrics-default/pom.xml b/ratis-metrics-default/pom.xml index 7caccf634b..4fe8073922 100644 --- a/ratis-metrics-default/pom.xml +++ b/ratis-metrics-default/pom.xml @@ -54,6 +54,17 @@ junit test + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.jupiter + junit-jupiter-api + test + + org.mockito mockito-core diff --git a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java b/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java index 0093fa759f..c7f26740d6 100644 --- a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java +++ b/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java @@ -17,15 +17,15 @@ */ package org.apache.ratis.metrics; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.mockito.Mockito.mock; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import org.apache.ratis.metrics.impl.MetricRegistriesImpl; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collections; @@ -57,7 +57,7 @@ public void testLoadMultipleInstances() { @Test public void testLoadDefault() { final MetricRegistries r = MetricRegistriesLoader.load(); - Assert.assertSame(MetricRegistriesImpl.class, r.getClass()); + Assertions.assertSame(MetricRegistriesImpl.class, r.getClass()); } @Test @@ -71,16 +71,16 @@ public void testAddRemoveReporter() { // check if add and remove of metric do reporting counter increase MetricRegistryInfo info = new MetricRegistryInfo("t1", "t1", "t1", "t1"); r.create(info); - Assert.assertTrue(cntr.get() == 1); + assertEquals(1, cntr.get()); r.remove(info); - Assert.assertTrue(cntr.get() == 2); + assertEquals(2, cntr.get()); // after removal, add and remove of metric must not do any increase r.removeReporterRegistration(reporter, stopReporter); r.create(info); - Assert.assertTrue(cntr.get() == 2); + assertEquals(2, cntr.get()); r.remove(info); - Assert.assertTrue(cntr.get() == 2); + assertEquals(2, cntr.get()); } } diff --git a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl/TestRefCountingMap.java b/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl/TestRefCountingMap.java index a8c54d7b15..be6eb3037c 100644 --- a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl/TestRefCountingMap.java +++ b/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl/TestRefCountingMap.java @@ -17,23 +17,23 @@ */ package org.apache.ratis.metrics.impl; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Collection; import java.util.Set; import org.apache.ratis.thirdparty.com.google.common.collect.Lists; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestRefCountingMap { private RefCountingMap map; - @Before + @BeforeEach public void setUp() { map = new RefCountingMap<>(); } @@ -128,7 +128,7 @@ public void testKeySet() { Set keys = map.keySet(); assertEquals(3, keys.size()); - Lists.newArrayList("foo", "bar", "baz").stream().forEach(v -> assertTrue(keys.contains(v))); + Lists.newArrayList("foo", "bar", "baz").forEach(v -> assertTrue(keys.contains(v))); } @Test @@ -141,7 +141,7 @@ public void testValues() { Collection values = map.values(); assertEquals(3, values.size()); - Lists.newArrayList("foovalue", "foovalue3", "foovalue4").stream() + Lists.newArrayList("foovalue", "foovalue3", "foovalue4") .forEach(v -> assertTrue(values.contains(v))); } } From c4b10faa8a82085bf8ba801fd46e4b71f81297fb Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Mon, 29 Jan 2024 18:30:00 -0800 Subject: [PATCH 26/76] RATIS-2018. Zero-copy buffers are not released - 2nd chunk (#1032) --- .../ratis/util/ReferenceCountedObject.java | 8 ++- .../org/apache/ratis/util/SlidingWindow.java | 16 +++-- .../java/org/apache/ratis/grpc/GrpcUtil.java | 8 +-- .../server/GrpcClientProtocolService.java | 9 ++- .../server/GrpcServerProtocolService.java | 67 ++++++++++++------- .../RaftServerAsynchronousProtocol.java | 6 +- .../ratis/server/impl/RaftServerImpl.java | 8 ++- .../segmented/SegmentedRaftLogCache.java | 14 ++-- .../impl/SimpleStateMachine4Testing.java | 14 +++- 9 files changed, 100 insertions(+), 50 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java index 815b90dbc1..4cca3a9313 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java @@ -117,7 +117,13 @@ public V retain() { @Override public boolean release() { - return fromRefs.stream().map(ReferenceCountedObject::release).allMatch(r -> r); + boolean allReleased = true; + for (ReferenceCountedObject ref : fromRefs) { + if (!ref.release()) { + allReleased = false; + } + } + return allReleased; } }; } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java index 7adc7a7210..22285a72c0 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java @@ -167,6 +167,7 @@ void endOfRequests(long nextToProcess, REQUEST end, Consumer replyMetho + " will NEVER be processed; request = " + r); r.fail(e); replyMethod.accept(r); + r.release(); } tail.clear(); @@ -455,19 +456,26 @@ public synchronized String toString() { /** A request (or a retry) arrives (may be out-of-order except for the first request). */ public synchronized void receivedRequest(REQUEST request, Consumer processingMethod) { final long seqNum = request.getSeqNum(); + final boolean accepted; if (nextToProcess == -1 && (request.isFirstRequest() || seqNum == 0)) { nextToProcess = seqNum; requests.putNewRequest(request); LOG.debug("Received seq={} (first request), {}", seqNum, this); + accepted = true; + } else if (request.getSeqNum() < nextToProcess) { + LOG.debug("Received seq={} < nextToProcess {}, {}", seqNum, nextToProcess, this); + accepted = false; } else { final boolean isRetry = requests.putIfAbsent(request); LOG.debug("Received seq={}, isRetry? {}, {}", seqNum, isRetry, this); - if (isRetry) { - return; - } + accepted = !isRetry; } - processRequestsFromHead(processingMethod); + if (accepted) { + processRequestsFromHead(processingMethod); + } else { + request.release(); + } } private void processRequestsFromHead(Consumer processingMethod) { diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java index 0baefa2d3f..5a7f003183 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java @@ -63,14 +63,8 @@ public interface GrpcUtil { Metadata.Key.of("heartbeat", Metadata.ASCII_STRING_MARSHALLER); static StatusRuntimeException wrapException(Throwable t) { - return wrapException(t, -1); - } - - static StatusRuntimeException wrapException(Throwable t, long callId) { t = JavaUtils.unwrapCompletionException(t); - Metadata trailers = new StatusRuntimeExceptionMetadataBuilder(t) - .addCallId(callId) - .build(); + Metadata trailers = new StatusRuntimeExceptionMetadataBuilder(t).build(); return wrapException(t, trailers); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index 0671a1841d..97b0118909 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -29,6 +29,7 @@ import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; +import org.apache.ratis.proto.RaftProtos.SlidingWindowEntry; import org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceImplBase; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.JavaUtils; @@ -317,14 +318,16 @@ private class UnorderedRequestStreamObserver extends RequestStreamObserver { void processClientRequest(ReferenceCountedObject requestRef) { final RaftClientRequest request = requestRef.retain(); final long callId = request.getCallId(); - + final SlidingWindowEntry slidingWindowEntry = request.getSlidingWindowEntry(); final CompletableFuture f = processClientRequest(requestRef, reply -> { if (!reply.isSuccess()) { - LOG.info("Failed {}, reply={}", request, reply); + LOG.info("Failed request cid={}, {}, reply={}", callId, slidingWindowEntry, reply); } final RaftClientReplyProto proto = ClientProtoUtils.toRaftClientReplyProto(reply); responseNext(proto); - }).whenComplete((r, e) -> requestRef.release()); + }); + + requestRef.release(); put(callId, f); f.thenAccept(dummy -> remove(callId)); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index ebe764fac0..9c426f322a 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -28,7 +28,6 @@ import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; import org.apache.ratis.thirdparty.io.grpc.Status; -import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.proto.RaftProtos.*; import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceImplBase; @@ -51,21 +50,28 @@ class GrpcServerProtocolService extends RaftServerProtocolServiceImplBase { public static final Logger LOG = LoggerFactory.getLogger(GrpcServerProtocolService.class); static class PendingServerRequest { - private final REQUEST request; + private final AtomicReference> requestRef; private final CompletableFuture future = new CompletableFuture<>(); PendingServerRequest(ReferenceCountedObject requestRef) { - this.request = requestRef.retain(); - this.future.whenComplete((r, e) -> requestRef.release()); + requestRef.retain(); + this.requestRef = new AtomicReference<>(requestRef); } REQUEST getRequest() { - return request; + return Optional.ofNullable(requestRef.get()) + .map(ReferenceCountedObject::get) + .orElse(null); } CompletableFuture getFuture() { return future; } + + void release() { + Optional.ofNullable(requestRef.getAndSet(null)) + .ifPresent(ReferenceCountedObject::release); + } } abstract class ServerRequestStreamObserver implements StreamObserver { @@ -108,20 +114,21 @@ void release(REQUEST req) { abstract long getCallId(REQUEST request); + boolean isHeartbeat(REQUEST request) { + return false; + } + abstract String requestToString(REQUEST request); abstract String replyToString(REPLY reply); abstract boolean replyInOrder(REQUEST request); - StatusRuntimeException wrapException(Throwable e, REQUEST request) { - return GrpcUtil.wrapException(e, getCallId(request)); - } - - private void handleError(Throwable e, REQUEST request) { - GrpcUtil.warn(LOG, () -> getId() + ": Failed " + op + " request " + requestToString(request), e); + private synchronized void handleError(Throwable e, long callId, boolean isHeartbeat) { + GrpcUtil.warn(LOG, () -> getId() + ": Failed " + op + " request cid=" + callId + ", isHeartbeat? " + + isHeartbeat, e); if (isClosed.compareAndSet(false, true)) { - responseObserver.onError(wrapException(e, request)); + responseObserver.onError(GrpcUtil.wrapException(e, callId, isHeartbeat)); } } @@ -151,21 +158,22 @@ public void onNext(REQUEST request) { try { composeRequest(process(requestRef).thenApply(this::handleReply)); } catch (Exception e) { - handleError(e, request); + handleError(e, getCallId(request), isHeartbeat(request)); release(request); } return; } final PendingServerRequest current = new PendingServerRequest<>(requestRef); - final PendingServerRequest previous = previousOnNext.getAndSet(current); - final CompletableFuture previousFuture = Optional.ofNullable(previous) - .map(PendingServerRequest::getFuture) + final long callId = getCallId(current.getRequest()); + final boolean isHeartbeat = isHeartbeat(current.getRequest()); + final Optional> previous = Optional.ofNullable(previousOnNext.getAndSet(current)); + final CompletableFuture previousFuture = previous.map(PendingServerRequest::getFuture) .orElse(CompletableFuture.completedFuture(null)); try { final CompletableFuture f = process(requestRef).exceptionally(e -> { // Handle cases, such as RaftServer is paused - handleError(e, request); + handleError(e, callId, isHeartbeat); current.getFuture().completeExceptionally(e); return null; }).thenCombine(previousFuture, (reply, v) -> { @@ -175,8 +183,14 @@ public void onNext(REQUEST request) { }); composeRequest(f); } catch (Exception e) { - handleError(e, request); + handleError(e, callId, isHeartbeat); current.getFuture().completeExceptionally(e); + } finally { + previous.ifPresent(PendingServerRequest::release); + if (isClosed.get()) { + // Some requests may come after onCompleted or onError, ensure they're released. + releaseLast(); + } } } @@ -188,8 +202,10 @@ public void onCompleted() { LOG.info("{}: Completed {}, lastReply: {}", getId(), op, reply); responseObserver.onCompleted(); }); + releaseLast(); } } + @Override public void onError(Throwable t) { GrpcUtil.warn(LOG, () -> getId() + ": "+ op + " onError, lastRequest: " + getPreviousRequestString(), t); @@ -198,8 +214,13 @@ public void onError(Throwable t) { if (status != null && status.getCode() != Status.Code.CANCELLED) { responseObserver.onCompleted(); } + releaseLast(); } } + + private void releaseLast() { + Optional.ofNullable(previousOnNext.get()).ifPresent(PendingServerRequest::release); + } } private final Supplier idSupplier; @@ -288,6 +309,11 @@ long getCallId(AppendEntriesRequestProto request) { return request.getServerRequest().getCallId(); } + @Override + boolean isHeartbeat(AppendEntriesRequestProto request) { + return request.getEntriesCount() == 0; + } + @Override String requestToString(AppendEntriesRequestProto request) { return ServerStringUtils.toAppendEntriesRequestString(request); @@ -302,11 +328,6 @@ String replyToString(AppendEntriesReplyProto reply) { boolean replyInOrder(AppendEntriesRequestProto request) { return request.getEntriesCount() != 0; } - - @Override - StatusRuntimeException wrapException(Throwable e, AppendEntriesRequestProto request) { - return GrpcUtil.wrapException(e, getCallId(request), request.getEntriesCount() == 0); - } }; } diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java index 1244e72545..035e0a815f 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java @@ -50,11 +50,9 @@ default CompletableFuture appendEntriesAsync( ReferenceCountedObject requestRef) throws IOException { // Default implementation for backward compatibility. try { - return appendEntriesAsync(requestRef.retain()) - .whenComplete((r, e) -> requestRef.release()); - } catch (Exception e) { + return appendEntriesAsync(requestRef.retain()); + } finally { requestRef.release(); - throw e; } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 8f6c9273ed..c47db14d66 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1076,7 +1076,13 @@ private CompletableFuture messageStreamAsync(ReferenceCountedOb return f.thenApply(r -> null); } // the message stream has ended and the request become a WRITE request - return replyFuture(f.join()); + ReferenceCountedObject joinedRequest = f.join(); + try { + return replyFuture(joinedRequest); + } finally { + // Released pending streaming requests. + joinedRequest.release(); + } } return role.getLeaderState() diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 1d08316fda..d222482012 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -42,6 +42,7 @@ import java.util.*; import java.util.concurrent.CopyOnWriteArrayList; import java.util.function.Consumer; +import java.util.stream.Collectors; /** * In-memory RaftLog Cache. Currently we provide a simple implementation that @@ -350,12 +351,10 @@ TruncationSegments truncate(long index, LogSegment openSegment, Runnable clearOp TruncationSegments purge(long index) { try (AutoCloseableLock writeLock = writeLock()) { int segmentIndex = binarySearch(index); - List list = new ArrayList<>(); + List list = new LinkedList<>(); if (segmentIndex == -segments.size() - 1) { - for (LogSegment ls : segments) { - list.add(SegmentFileInfo.newClosedSegmentFileInfo(ls)); - } + list.addAll(segments); segments.clear(); sizeInBytes = 0; } else if (segmentIndex >= 0) { @@ -368,13 +367,16 @@ TruncationSegments purge(long index) { for (int i = 0; i <= startIndex; i++) { LogSegment segment = segments.remove(0); // must remove the first segment to avoid gaps. sizeInBytes -= segment.getTotalFileSize(); - list.add(SegmentFileInfo.newClosedSegmentFileInfo(segment)); + list.add(segment); } } else { throw new IllegalStateException("Unexpected gap in segments: binarySearch(" + index + ") returns " + segmentIndex + ", segments=" + segments); } - return list.isEmpty() ? null : new TruncationSegments(null, list); + list.forEach(LogSegment::evictCache); + List toDelete = list.stream().map(SegmentFileInfo::newClosedSegmentFileInfo) + .collect(Collectors.toList()); + return list.isEmpty() ? null : new TruncationSegments(null, toDelete); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 07073be520..7c40ec2518 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -42,6 +42,7 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.Daemon; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; @@ -247,13 +248,24 @@ public CompletableFuture applyTransaction(TransactionContext trx) { LogEntryProto entry = Objects.requireNonNull(trx.getLogEntry()); LOG.info("applyTransaction for log index {}", entry.getIndex()); - put(entry); + // TODO: Logs kept in StateMachine's cache may be corrupted. Copy for now to have the test pass. + // Use ReferenceCount per RATIS-1997. + LogEntryProto copied = copy(entry); + put(copied); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); final SimpleMessage m = new SimpleMessage(entry.getIndex() + " OK"); return collecting.collect(Collecting.Type.APPLY_TRANSACTION, m); } + private LogEntryProto copy(LogEntryProto log) { + try { + return LogEntryProto.parseFrom(log.toByteString()); + } catch (InvalidProtocolBufferException e) { + throw new IllegalStateException("Error copying log entry", e); + } + } + @Override public long takeSnapshot() { final TermIndex termIndex = getLastAppliedTermIndex(); From cf897f02957bce7ab587a895143405c914adbf91 Mon Sep 17 00:00:00 2001 From: Nandakumar Vadivelu Date: Tue, 30 Jan 2024 21:40:12 +0530 Subject: [PATCH 27/76] RATIS-1973. Migrate ratis-examples tests to Junit 5. (#1035) --- ratis-examples/pom.xml | 10 +++++++ .../org/apache/ratis/TestMultiRaftGroup.java | 27 +++++++---------- .../ratis/examples/ParameterizedBaseTest.java | 13 ++++---- .../examples/arithmetic/TestArithmetic.java | 30 +++++++++---------- .../arithmetic/TestArithmeticLogDump.java | 16 +++++----- .../arithmetic/cli/TestAssignCli.java | 30 +++++++++---------- .../arithmetic/expression/TestExpression.java | 26 ++++++++-------- .../ratis/examples/common/TestSubCommand.java | 16 +++++----- .../ratis/examples/counter/TestCounter.java | 21 ++++++------- 9 files changed, 93 insertions(+), 96 deletions(-) diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index 54691d6f14..7454e928eb 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -122,11 +122,21 @@ junit test + + org.junit.jupiter + junit-jupiter-engine + test + org.junit.jupiter junit-jupiter-api test + + org.junit.jupiter + junit-jupiter-params + test + diff --git a/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java b/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java index 49a9102940..190f758589 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java +++ b/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java @@ -27,45 +27,40 @@ import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.function.CheckedBiConsumer; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; import java.io.IOException; import java.util.Collection; import java.util.concurrent.atomic.AtomicInteger; -@RunWith(Parameterized.class) public class TestMultiRaftGroup extends BaseTest { static { Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); } - @Parameterized.Parameters public static Collection data() throws IOException { return ParameterizedBaseTest.getMiniRaftClusters(ArithmeticStateMachine.class, 0); } - @Parameterized.Parameter - public MiniRaftCluster cluster; - - @Test - public void testMultiRaftGroup() throws Exception { - runTestMultiRaftGroup(3, 6, 9, 12, 15); + @ParameterizedTest + @MethodSource("data") + public void testMultiRaftGroup(MiniRaftCluster cluster) throws Exception { + runTestMultiRaftGroup(cluster, 3, 6, 9, 12, 15); } - private void runTestMultiRaftGroup(int... idIndex) throws Exception { - runTestMultiRaftGroup(idIndex, -1); + private void runTestMultiRaftGroup(MiniRaftCluster cluster, int... idIndex) throws Exception { + runTestMultiRaftGroup(cluster, idIndex, -1); } private final AtomicInteger start = new AtomicInteger(3); private final int count = 10; - private void runTestMultiRaftGroup(int[] idIndex, int chosen) throws Exception { + private void runTestMultiRaftGroup(MiniRaftCluster cluster, int[] idIndex, int chosen) throws Exception { - final CheckedBiConsumer checker = (cluster, group) -> { - try (final RaftClient client = cluster.createClient(group)) { + final CheckedBiConsumer checker = (c, group) -> { + try (final RaftClient client = c.createClient(group)) { TestArithmetic.runTestPythagorean(client, start.getAndAdd(2*count), count); } }; diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java b/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java index 9352a24e80..df2fce13ee 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java @@ -28,10 +28,9 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.AfterClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,17 +39,15 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -@RunWith(Parameterized.class) public abstract class ParameterizedBaseTest extends BaseTest { public static final Logger LOG = LoggerFactory.getLogger(ParameterizedBaseTest.class); /** Subclasses should override this method to provide real data parameters. */ - @Parameterized.Parameters public static Collection data() { return Collections.emptyList(); } - /** For {@link Parameterized} test so that a cluster can be shared by multiple {@link Test} */ + /** For {@link ParameterizedTest} so that a cluster can be shared by multiple {@link Test} */ private static final AtomicReference currentCluster = new AtomicReference<>(); /** Set {@link #currentCluster} to the given cluster and start it if {@link #currentCluster} is changed. */ @@ -66,7 +63,7 @@ public static void setAndStart(MiniRaftCluster cluster) throws InterruptedExcept } } - @AfterClass + @AfterAll public static void shutdownCurrentCluster() { final MiniRaftCluster cluster = currentCluster.getAndSet(null); if (cluster != null) { diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java index 0fa295c544..0c56898f0c 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java @@ -27,9 +27,9 @@ import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.Preconditions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; import java.io.IOException; @@ -44,16 +44,13 @@ public class TestArithmetic extends ParameterizedBaseTest { Slf4jUtils.setLogLevel(ArithmeticStateMachine.LOG, Level.DEBUG); } - @Parameterized.Parameters public static Collection data() { return getMiniRaftClusters(ArithmeticStateMachine.class, 3); } - @Parameterized.Parameter - public MiniRaftCluster cluster; - - @Test - public void testPythagorean() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testPythagorean(MiniRaftCluster cluster) throws Exception { setAndStart(cluster); try (final RaftClient client = cluster.createClient()) { runTestPythagorean(client, 3, 10); @@ -85,8 +82,9 @@ public static void runTestPythagorean( } } - @Test - public void testGaussLegendre() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testGaussLegendre(MiniRaftCluster cluster) throws Exception { setAndStart(cluster); try (final RaftClient client = cluster.createClient()) { runGaussLegendre(client); @@ -117,14 +115,14 @@ void runGaussLegendre(RaftClient client) throws IOException { final double pi = e.evaluate(null); if (converged) { - Assert.assertTrue(pi == previous); + Assertions.assertEquals(pi, previous); } else if (pi == previous) { converged = true; } LOG.info("{} = {}, converged? {}", pi_i, pi, converged); previous = pi; } - Assert.assertTrue(converged); + Assertions.assertTrue(converged); } static Variable defineVariable(RaftClient client, String name, double value) throws IOException { @@ -145,7 +143,7 @@ static Expression assign(RaftClient client, Variable x, double value) throws IOE static void assignNull(RaftClient client, Variable x) throws IOException { final Expression e = assign(client, x, NullValue.getInstance()); - Assert.assertEquals(NullValue.getInstance(), e); + Assertions.assertEquals(NullValue.getInstance(), e); } static Expression assign(RaftClient client, Variable x, Expression e) throws IOException { @@ -158,11 +156,11 @@ static Expression assign(RaftClient client, Variable x, Expression e, Double exp } static Expression assertRaftClientReply(RaftClientReply reply, Double expected) { - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final Expression e = Expression.Utils.bytes2Expression( reply.getMessage().getContent().toByteArray(), 0); if (expected != null) { - Assert.assertEquals(expected, e.evaluate(null)); + Assertions.assertEquals(expected, e.evaluate(null)); } return e; } diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java index 564afaac2c..c39335308f 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java @@ -34,10 +34,10 @@ import org.apache.ratis.tools.ParseRatisLog; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; public class TestArithmeticLogDump extends BaseTest { @@ -60,13 +60,13 @@ public RaftProperties getProperties() { return properties; } - @Before + @BeforeEach public void setup() throws IOException { - Assert.assertNull(cluster.getLeader()); + Assertions.assertNull(cluster.getLeader()); cluster.start(); } - @After + @AfterEach public void tearDown() { if (cluster != null) { cluster.shutdown(); @@ -77,7 +77,7 @@ public void tearDown() { public void testLogDump() throws Exception { final RaftServer.Division leaderServer = RaftTestUtil.waitForLeader(cluster); final List files = LogSegmentPath.getLogSegmentPaths(leaderServer.getRaftStorage()); - Assert.assertEquals(1, files.size()); + Assertions.assertEquals(1, files.size()); cluster.shutdown(); ParseRatisLog.Builder builder = new ParseRatisLog.Builder(); diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/cli/TestAssignCli.java b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/cli/TestAssignCli.java index 3052adba10..c28b4d2ddf 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/cli/TestAssignCli.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/cli/TestAssignCli.java @@ -19,8 +19,8 @@ import org.apache.ratis.examples.arithmetic.expression.DoubleValue; import org.apache.ratis.examples.arithmetic.expression.Variable; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import static org.apache.ratis.examples.arithmetic.expression.BinaryExpression.Op.ADD; import static org.apache.ratis.examples.arithmetic.expression.BinaryExpression.Op.MULT; @@ -32,55 +32,55 @@ public class TestAssignCli { @Test public void createExpression() { - Assert.assertEquals( + Assertions.assertEquals( new DoubleValue(2.0), new Assign().createExpression("2.0")); - Assert.assertEquals( + Assertions.assertEquals( new DoubleValue(42.0), new Assign().createExpression("42")); - Assert.assertEquals( + Assertions.assertEquals( MULT.apply(2.0, new Variable("a")), new Assign().createExpression("2*a")); - Assert.assertEquals( + Assertions.assertEquals( MULT.apply(new Variable("v1"), 2.0), new Assign().createExpression("v1 * 2")); - Assert.assertEquals( + Assertions.assertEquals( ADD.apply(2.0, 1.0), new Assign().createExpression("2+1")); - Assert.assertEquals( + Assertions.assertEquals( SUBTRACT.apply(1.0, 6.0), new Assign().createExpression("1 - 6")); - Assert.assertEquals( + Assertions.assertEquals( ADD.apply(new Variable("a"), new Variable("v2")), new Assign().createExpression("a+v2")); - Assert.assertEquals( + Assertions.assertEquals( ADD.apply(new Variable("v1"), new Variable("b")), new Assign().createExpression("v1 + b")); - Assert.assertEquals( + Assertions.assertEquals( SQRT.apply(new Variable("a")), new Assign().createExpression("√a")); - Assert.assertEquals( + Assertions.assertEquals( SQRT.apply(new Variable("ABC")), new Assign().createExpression("√ABC")); - Assert.assertEquals( + Assertions.assertEquals( SQRT.apply(2.0), new Assign().createExpression("√2")); - Assert.assertEquals( + Assertions.assertEquals( NEG.apply(2.0), new Assign().createExpression("~2.0")); - Assert.assertEquals( + Assertions.assertEquals( MINUS.apply(6.0), new Assign().createExpression("-6.0")); } diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java index 6996fe9786..4cc81c6ed1 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java @@ -19,8 +19,8 @@ import org.apache.ratis.BaseTest; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.Random; import java.util.concurrent.ThreadLocalRandom; @@ -42,21 +42,21 @@ public void testArithmeticUtils() throws Exception { final int n = ran.nextInt(); Expression.Utils.int2bytes(n, buf, offset); final int m = Expression.Utils.bytes2int(buf, offset); - Assert.assertEquals(n, m); + Assertions.assertEquals(n, m); offset += 4; } { final long n = ran.nextLong(); Expression.Utils.long2bytes(n, buf, offset); final long m = Expression.Utils.bytes2long(buf, offset); - Assert.assertEquals(n, m); + Assertions.assertEquals(n, m); offset += 8; } { final double n = ran.nextDouble(); Expression.Utils.double2bytes(n, buf, offset); final double m = Expression.Utils.bytes2double(buf, offset); - Assert.assertTrue(n == m); + Assertions.assertEquals(n, m); offset += 8; } } @@ -65,11 +65,11 @@ public void testArithmeticUtils() throws Exception { public void testOp() throws Exception { for(BinaryExpression.Op op : BinaryExpression.Op.values()) { final byte b = op.byteValue(); - Assert.assertEquals(op, BinaryExpression.Op.valueOf(b)); + Assertions.assertEquals(op, BinaryExpression.Op.valueOf(b)); } for(UnaryExpression.Op op : UnaryExpression.Op.values()) { final byte b = op.byteValue(); - Assert.assertEquals(op, UnaryExpression.Op.valueOf(b)); + Assertions.assertEquals(op, UnaryExpression.Op.valueOf(b)); } } @@ -84,9 +84,9 @@ public void testExpression() throws Exception { final int len = a.toBytes(buf, offset); final Variable a2 = new Variable(buf, offset); LOG.info("var a2: " + a2); - Assert.assertEquals(a.getName(), a2.getName()); - Assert.assertEquals(len, a.length()); - Assert.assertEquals(len, a2.length()); + Assertions.assertEquals(a.getName(), a2.getName()); + Assertions.assertEquals(len, a.length()); + Assertions.assertEquals(len, a2.length()); offset += len; } @@ -96,9 +96,9 @@ public void testExpression() throws Exception { final int len = three.toBytes(buf, offset); final DoubleValue three2 = new DoubleValue(buf, offset); LOG.info("double three2: " + three2.evaluate(null)); - Assert.assertTrue(three.evaluate(null).equals(three2.evaluate(null))); - Assert.assertEquals(len, three.length()); - Assert.assertEquals(len, three2.length()); + Assertions.assertEquals(three.evaluate(null), three2.evaluate(null)); + Assertions.assertEquals(len, three.length()); + Assertions.assertEquals(len, three2.length()); } } } diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/common/TestSubCommand.java b/ratis-examples/src/test/java/org/apache/ratis/examples/common/TestSubCommand.java index 71fe7e6a4a..5ef034809a 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/common/TestSubCommand.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/common/TestSubCommand.java @@ -22,23 +22,23 @@ import java.util.Collection; import java.util.Collections; import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -@RunWith(Parameterized.class) public class TestSubCommand { - @Parameterized.Parameters public static Collection data() { return Collections.singleton("127.0.0.1:6667"); } - @Parameterized.Parameter - public String peers; - - @Test(expected = IllegalArgumentException.class) - public void testParsePeers() { - SubCommandBase.parsePeers(peers); + @ParameterizedTest + @MethodSource("data") + public void testParsePeers(String peers) { + Assertions.assertThrows(IllegalArgumentException.class, + () -> SubCommandBase.parsePeers(peers)); } } diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/counter/TestCounter.java b/ratis-examples/src/test/java/org/apache/ratis/examples/counter/TestCounter.java index 6fbe8770e3..3d188010d6 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/counter/TestCounter.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/counter/TestCounter.java @@ -24,42 +24,39 @@ import org.apache.ratis.examples.counter.server.CounterStateMachine; import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftClientReply; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import java.io.IOException; import java.util.Collection; public class TestCounter extends ParameterizedBaseTest { - @Parameterized.Parameters public static Collection data() { return getMiniRaftClusters(CounterStateMachine.class, 3); } - @Parameterized.Parameter - public MiniRaftCluster cluster; - - @Test - public void testSeveralCounter() throws IOException, InterruptedException { + @ParameterizedTest + @MethodSource("data") + public void testSeveralCounter(MiniRaftCluster cluster) throws IOException, InterruptedException { setAndStart(cluster); try (final RaftClient client = cluster.createClient()) { for (int i = 0; i < 10; i++) { client.io().send(Message.valueOf("INCREMENT")); } RaftClientReply reply1 = client.io().sendReadOnly(Message.valueOf("GET")); - Assert.assertEquals(10, reply1.getMessage().getContent().asReadOnlyByteBuffer().getInt()); + Assertions.assertEquals(10, reply1.getMessage().getContent().asReadOnlyByteBuffer().getInt()); for (int i = 0; i < 10; i++) { client.io().send(Message.valueOf("INCREMENT")); } RaftClientReply reply2 = client.io().sendReadOnly(Message.valueOf("GET")); - Assert.assertEquals(20, reply2.getMessage().getContent().asReadOnlyByteBuffer().getInt()); + Assertions.assertEquals(20, reply2.getMessage().getContent().asReadOnlyByteBuffer().getInt()); for (int i = 0; i < 10; i++) { client.io().send(Message.valueOf("INCREMENT")); } RaftClientReply reply3 = client.io().sendReadOnly(Message.valueOf("GET")); - Assert.assertEquals(30, reply3.getMessage().getContent().asReadOnlyByteBuffer().getInt()); + Assertions.assertEquals(30, reply3.getMessage().getContent().asReadOnlyByteBuffer().getInt()); } } } From d2f1c57fe60e3abe1bbfa1820297f9d60736bb42 Mon Sep 17 00:00:00 2001 From: Potato Date: Wed, 31 Jan 2024 00:12:23 +0800 Subject: [PATCH 28/76] RATIS-2019 Fixed abnormal exit of StateMachineUpdater (#1033) --- .../java/org/apache/ratis/util/JavaUtils.java | 4 +++ .../ratis/server/impl/LeaderStateImpl.java | 26 ++++++++----------- .../server/impl/ReadIndexHeartbeats.java | 9 +++---- .../ratis/server/impl/RetryCacheImpl.java | 2 +- 4 files changed, 19 insertions(+), 22 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java index d3f899a7fd..f689006db2 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java @@ -269,6 +269,10 @@ static CompletableFuture completeExceptionally(Throwable t) { return future; } + static boolean isCompletedNormally(CompletableFuture future) { + return future.isDone() && !future.isCancelled() && !future.isCompletedExceptionally(); + } + static Throwable unwrapCompletionException(Throwable t) { Objects.requireNonNull(t, "t == null"); return t instanceof CompletionException && t.getCause() != null? t.getCause(): t; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 043c7319be..4f313a4373 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -311,19 +311,16 @@ CompletableFuture getAppliedIndexFuture() { return appliedIndexFuture; } - boolean isApplied(LogEntryProto logEntry) { - if (appliedIndexFuture.isDone()) { - return true; - } - final long appliedIndex = logEntry != null? logEntry.getIndex(): server.getState().getLastAppliedIndex(); - if (appliedIndex >= startIndex) { - appliedIndexFuture.complete(appliedIndex); - LOG.info("leader is ready since appliedIndex == {} >= startIndex == {}", - appliedIndex, startIndex); - return true; - } else { - return false; + boolean checkStartIndex(LogEntryProto logEntry) { + final boolean completed = logEntry.getIndex() == startIndex && appliedIndexFuture.complete(startIndex); + if (completed) { + LOG.info("Leader {} is ready since appliedIndex == startIndex == {}", LeaderStateImpl.this, startIndex); } + return completed; + } + + boolean isApplied() { + return JavaUtils.isCompletedNormally(appliedIndexFuture); } } @@ -422,12 +419,11 @@ void start() { } boolean isReady() { - return startupLogEntry.isInitialized() && startupLogEntry.get().isApplied(null); + return startupLogEntry.isInitialized() && startupLogEntry.get().isApplied(); } void checkReady(LogEntryProto entry) { - Preconditions.assertTrue(startupLogEntry.isInitialized()); - if (entry.getTerm() == getCurrentTerm() && startupLogEntry.get().isApplied(entry)) { + if (entry.getTerm() == server.getState().getCurrentTerm() && startupLogEntry.get().checkStartIndex(entry)) { server.getStateMachine().leaderEvent().notifyLeaderReady(); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java index 3f31a25308..d08a1ea406 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java @@ -22,6 +22,7 @@ import org.apache.ratis.server.leader.LogAppender; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.RaftLogIndex; +import org.apache.ratis.util.JavaUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,7 +100,7 @@ CompletableFuture getFuture() { boolean receive(LogAppender logAppender, AppendEntriesReplyProto proto, Predicate> hasMajority) { - if (isCompletedNormally()) { + if (JavaUtils.isCompletedNormally(future)) { return true; } @@ -112,16 +113,12 @@ boolean receive(LogAppender logAppender, AppendEntriesReplyProto proto, } } - return isCompletedNormally(); + return JavaUtils.isCompletedNormally(future); } boolean isAcknowledged(RaftPeerId id) { return Optional.ofNullable(replies.get(id)).filter(HeartbeatAck::isAcknowledged).isPresent(); } - - boolean isCompletedNormally() { - return future.isDone() && !future.isCancelled() && !future.isCompletedExceptionally(); - } } class AppendEntriesListeners { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java index a8bac4e5e8..50d238b07a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java @@ -67,7 +67,7 @@ boolean isDone() { } boolean isCompletedNormally() { - return !failed && replyFuture.isDone() && !replyFuture.isCompletedExceptionally() && !replyFuture.isCancelled(); + return !failed && JavaUtils.isCompletedNormally(replyFuture); } void updateResult(RaftClientReply reply) { From c46da2213327ea89e9d91b351aee654a50c70da9 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Wed, 31 Jan 2024 09:23:29 -0800 Subject: [PATCH 29/76] RATIS-1978. Add tests assertions to verify all zero-copy messages are released properly (#1023) --- .../org/apache/ratis/util/LeakDetector.java | 139 +++++++++++ .../util/ReferenceCountedLeakDetector.java | 222 ++++++++++++++++++ .../ratis/util/ReferenceCountedObject.java | 54 +---- .../ratis/grpc/metrics/ZeroCopyMetrics.java | 15 ++ .../apache/ratis/grpc/server/GrpcService.java | 6 + .../ratis/grpc/MiniRaftClusterWithGrpc.java | 30 ++- .../org/apache/ratis/LogAppenderTests.java | 1 + .../apache/ratis/RaftExceptionBaseTest.java | 6 +- .../java/org/apache/ratis/RaftTestUtil.java | 15 ++ .../server/impl/LeaderElectionTests.java | 2 + .../ratis/server/impl/MiniRaftCluster.java | 9 + 11 files changed, 452 insertions(+), 47 deletions(-) create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java new file mode 100644 index 0000000000..d801868163 --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java @@ -0,0 +1,139 @@ +/* + * 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.ratis.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.ref.ReferenceQueue; +import java.lang.ref.WeakReference; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Simple general resource leak detector using {@link ReferenceQueue} and {@link java.lang.ref.WeakReference} to + * observe resource object life-cycle and assert proper resource closure before they are GCed. + * + *

+ * Example usage: + * + *

 {@code
+ * class MyResource implements AutoClosable {
+ *   static final LeakDetector LEAK_DETECTOR = new LeakDetector("MyResource");
+ *
+ *   private final UncheckedAutoCloseable leakTracker = LEAK_DETECTOR.track(this, () -> {
+ *      // report leaks, don't refer to the original object (MyResource) here.
+ *      System.out.println("MyResource is not closed before being discarded.");
+ *   });
+ *
+ *   @Override
+ *   public void close() {
+ *     // proper resources cleanup...
+ *     // inform tracker that this object is closed properly.
+ *     leakTracker.close();
+ *   }
+ * }
+ *
+ * }
+ */ +public class LeakDetector { + private static final Logger LOG = LoggerFactory.getLogger(LeakDetector.class); + private static final AtomicLong COUNTER = new AtomicLong(); + + private final ReferenceQueue queue = new ReferenceQueue<>(); + private final Set allLeaks = Collections.newSetFromMap(new ConcurrentHashMap<>()); + private final String name; + + public LeakDetector(String name) { + this.name = name + COUNTER.getAndIncrement(); + } + + LeakDetector start() { + Thread t = new Thread(this::run); + t.setName(LeakDetector.class.getSimpleName() + "-" + name); + t.setDaemon(true); + LOG.info("Starting leak detector thread {}.", name); + t.start(); + return this; + } + + private void run() { + while (true) { + try { + LeakTracker tracker = (LeakTracker) queue.remove(); + // Original resource already been GCed, if tracker is not closed yet, + // report a leak. + if (allLeaks.remove(tracker)) { + tracker.reportLeak(); + } + } catch (InterruptedException e) { + LOG.warn("Thread interrupted, exiting.", e); + break; + } + } + + LOG.warn("Exiting leak detector {}.", name); + } + + public UncheckedAutoCloseable track(Object leakable, Runnable reportLeak) { + // A rate filter can be put here to only track a subset of all objects, e.g. 5%, 10%, + // if we have proofs that leak tracking impacts performance, or a single LeakDetector + // thread can't keep up with the pace of object allocation. + // For now, it looks effective enough and let keep it simple. + LeakTracker tracker = new LeakTracker(leakable, queue, allLeaks, reportLeak); + allLeaks.add(tracker); + return tracker; + } + + public void assertNoLeaks() { + Preconditions.assertTrue(allLeaks.isEmpty(), this::allLeaksString); + } + + String allLeaksString() { + if (allLeaks.isEmpty()) { + return "allLeaks = "; + } + allLeaks.forEach(LeakTracker::reportLeak); + return "allLeaks.size = " + allLeaks.size(); + } + + private static final class LeakTracker extends WeakReference implements UncheckedAutoCloseable { + private final Set allLeaks; + private final Runnable leakReporter; + LeakTracker(Object referent, ReferenceQueue referenceQueue, + Set allLeaks, Runnable leakReporter) { + super(referent, referenceQueue); + this.allLeaks = allLeaks; + this.leakReporter = leakReporter; + } + + /** + * Called by the tracked resource when closing. + */ + @Override + public void close() { + allLeaks.remove(this); + } + + void reportLeak() { + leakReporter.run(); + } + } +} diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java new file mode 100644 index 0000000000..32abe805f1 --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java @@ -0,0 +1,222 @@ +/* + * 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.ratis.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.function.Supplier; + +/** + * A utility to detect leaks from @{@link ReferenceCountedObject}. + */ +public final class ReferenceCountedLeakDetector { + private static final Logger LOG = LoggerFactory.getLogger(ReferenceCountedLeakDetector.class); + // Leak detection is turned off by default. + + private static final AtomicReference FACTORY = new AtomicReference<>(Mode.NONE); + private static final Supplier SUPPLIER + = MemoizedSupplier.valueOf(() -> new LeakDetector(FACTORY.get().name()).start()); + + static Factory getFactory() { + return FACTORY.get(); + } + + public static LeakDetector getLeakDetector() { + return SUPPLIER.get(); + } + + private ReferenceCountedLeakDetector() { + } + + static synchronized void enable(boolean advanced) { + FACTORY.set(advanced ? Mode.ADVANCED : Mode.SIMPLE); + } + + interface Factory { + ReferenceCountedObject create(V value, Runnable retainMethod, Consumer releaseMethod); + } + + private enum Mode implements Factory { + /** Leak detector is not enable in production to avoid performance impacts. */ + NONE { + @Override + public ReferenceCountedObject create(V value, Runnable retainMethod, Consumer releaseMethod) { + return new Impl<>(value, retainMethod, releaseMethod); + } + }, + /** Leak detector is enabled to detect leaks. This is intended to use in every tests. */ + SIMPLE { + @Override + public ReferenceCountedObject create(V value, Runnable retainMethod, Consumer releaseMethod) { + return new SimpleTracing<>(value, retainMethod, releaseMethod, getLeakDetector()); + } + }, + /** + * Leak detector is enabled to detect leaks and report object creation stacktrace as well as every retain and + * release stacktraces. This has severe impact in performance and only used to debug specific test cases. + */ + ADVANCED { + @Override + public ReferenceCountedObject create(V value, Runnable retainMethod, Consumer releaseMethod) { + return new AdvancedTracing<>(value, retainMethod, releaseMethod, getLeakDetector()); + } + } + } + + private static class Impl implements ReferenceCountedObject { + private final AtomicInteger count; + private final V value; + private final Runnable retainMethod; + private final Consumer releaseMethod; + + Impl(V value, Runnable retainMethod, Consumer releaseMethod) { + this.value = value; + this.retainMethod = retainMethod; + this.releaseMethod = releaseMethod; + count = new AtomicInteger(); + } + + @Override + public V get() { + final int previous = count.get(); + if (previous < 0) { + throw new IllegalStateException("Failed to get: object has already been completely released."); + } else if (previous == 0) { + throw new IllegalStateException("Failed to get: object has not yet been retained."); + } + return value; + } + + @Override + public V retain() { + // n < 0: exception + // n >= 0: n++ + if (count.getAndUpdate(n -> n < 0? n : n + 1) < 0) { + throw new IllegalStateException("Failed to retain: object has already been completely released."); + } + + retainMethod.run(); + return value; + } + + @Override + public boolean release() { + // n <= 0: exception + // n > 1: n-- + // n == 1: n = -1 + final int previous = count.getAndUpdate(n -> n <= 1? -1: n - 1); + if (previous < 0) { + throw new IllegalStateException("Failed to release: object has already been completely released."); + } else if (previous == 0) { + throw new IllegalStateException("Failed to release: object has not yet been retained."); + } + final boolean completedReleased = previous == 1; + releaseMethod.accept(completedReleased); + return completedReleased; + } + } + + private static class SimpleTracing extends Impl { + private final UncheckedAutoCloseable leakTracker; + + SimpleTracing(T value, Runnable retainMethod, Consumer releaseMethod, LeakDetector leakDetector) { + super(value, retainMethod, releaseMethod); + final Class clazz = value.getClass(); + this.leakTracker = leakDetector.track(this, + () -> LOG.warn("LEAK: A {} is not released properly", clazz.getName())); + } + + @Override + public boolean release() { + boolean released = super.release(); + if (released) { + leakTracker.close(); + } + return released; + } + } + + private static class AdvancedTracing extends Impl { + private final UncheckedAutoCloseable leakTracker; + private final List retainsTraces; + private final List releaseTraces; + + AdvancedTracing(T value, Runnable retainMethod, Consumer releaseMethod, LeakDetector leakDetector) { + super(value, retainMethod, releaseMethod); + + StackTraceElement[] createStrace = Thread.currentThread().getStackTrace(); + final Class clazz = value.getClass(); + final List localRetainsTraces = new LinkedList<>(); + final List localReleaseTraces = new LinkedList<>(); + + this.leakTracker = leakDetector.track(this, () -> + LOG.warn("LEAK: A {} is not released properly.\nCreation trace:\n{}\n" + + "Retain traces({}):\n{}\nRelease traces({}):\n{}", + clazz.getName(), formatStackTrace(createStrace, 3), + localRetainsTraces.size(), formatStackTraces(localRetainsTraces, 2), + localReleaseTraces.size(), formatStackTraces(localReleaseTraces, 2))); + + this.retainsTraces = localRetainsTraces; + this.releaseTraces = localReleaseTraces; + } + + @Override + public T retain() { + T retain = super.retain(); + retainsTraces.add(Thread.currentThread().getStackTrace()); + return retain; + } + + @Override + public boolean release() { + boolean released = super.release(); + if (released) { + leakTracker.close(); + } + releaseTraces.add(Thread.currentThread().getStackTrace()); + return released; + } + } + + private static String formatStackTrace(StackTraceElement[] stackTrace, int startIdx) { + final StringBuilder sb = new StringBuilder(); + for (int line = startIdx; line < stackTrace.length; line++) { + sb.append(stackTrace[line]).append("\n"); + } + return sb.toString(); + } + + private static String formatStackTraces(List stackTraces, int startIdx) { + final StringBuilder sb = new StringBuilder(); + stackTraces.forEach(stackTrace -> { + if (sb.length() > 0) { + sb.append("\n"); + } + for (int line = startIdx; line < stackTrace.length; line++) { + sb.append(stackTrace[line]).append("\n"); + } + }); + return sb.toString(); + } +} diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java index 4cca3a9313..eb5ff30ae8 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java @@ -22,7 +22,6 @@ import java.util.Collection; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; /** @@ -45,6 +44,7 @@ * @param The object type. */ public interface ReferenceCountedObject { + /** @return the object. */ T get(); @@ -167,52 +167,20 @@ static ReferenceCountedObject wrap(V value, Runnable retainMethod, Consum Objects.requireNonNull(retainMethod, "retainMethod == null"); Objects.requireNonNull(releaseMethod, "releaseMethod == null"); - return new ReferenceCountedObject() { - private final AtomicInteger count = new AtomicInteger(); - - @Override - public V get() { - final int previous = count.get(); - if (previous < 0) { - throw new IllegalStateException("Failed to get: object has already been completely released."); - } else if (previous == 0) { - throw new IllegalStateException("Failed to get: object has not yet been retained."); - } - return value; - } - - @Override - public V retain() { - // n < 0: exception - // n >= 0: n++ - if (count.getAndUpdate(n -> n < 0? n : n + 1) < 0) { - throw new IllegalStateException("Failed to retain: object has already been completely released."); - } - - retainMethod.run(); - return value; - } - - @Override - public boolean release() { - // n <= 0: exception - // n > 1: n-- - // n == 1: n = -1 - final int previous = count.getAndUpdate(n -> n <= 1? -1: n - 1); - if (previous < 0) { - throw new IllegalStateException("Failed to release: object has already been completely released."); - } else if (previous == 0) { - throw new IllegalStateException("Failed to release: object has not yet been retained."); - } - final boolean completedReleased = previous == 1; - releaseMethod.accept(completedReleased); - return completedReleased; - } - }; + return ReferenceCountedLeakDetector.getFactory().create(value, retainMethod, releaseMethod); } /** The same as wrap(value, retainMethod, ignored -> releaseMethod.run()). */ static ReferenceCountedObject wrap(V value, Runnable retainMethod, Runnable releaseMethod) { return wrap(value, retainMethod, ignored -> releaseMethod.run()); } + + static void enableLeakDetection() { + ReferenceCountedLeakDetector.enable(false); + } + + static void enableAdvancedLeakDetection() { + ReferenceCountedLeakDetector.enable(true); + } + } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java index 20da4ee634..fec2135a98 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java @@ -21,6 +21,7 @@ import org.apache.ratis.metrics.MetricRegistryInfo; import org.apache.ratis.metrics.RatisMetricRegistry; import org.apache.ratis.metrics.RatisMetrics; +import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.com.google.protobuf.AbstractMessage; public class ZeroCopyMetrics extends RatisMetrics { @@ -55,4 +56,18 @@ public void onReleasedMessage(AbstractMessage ignored) { releasedMessages.inc(); } + @VisibleForTesting + public long zeroCopyMessages() { + return zeroCopyMessages.getCount(); + } + + @VisibleForTesting + public long nonZeroCopyMessages() { + return nonZeroCopyMessages.getCount(); + } + + @VisibleForTesting + public long releasedMessages() { + return releasedMessages.getCount(); + } } \ No newline at end of file diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java index d2dadcd90a..fa93586040 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -30,6 +30,7 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.RaftServerRpcWithProxy; import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; +import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.io.grpc.ServerInterceptors; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder; @@ -328,6 +329,7 @@ public void closeImpl() throws IOException { serverInterceptor.close(); ConcurrentUtils.shutdownAndWait(executor); + zeroCopyMetrics.unregister(); } @Override @@ -385,4 +387,8 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ return getProxies().getProxy(target).startLeaderElection(request); } + @VisibleForTesting + public ZeroCopyMetrics getZeroCopyMetrics() { + return zeroCopyMetrics; + } } diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java index 18c65c5be4..68556afb89 100644 --- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java +++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java @@ -21,14 +21,19 @@ import org.apache.ratis.RaftTestUtil; import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; import org.apache.ratis.grpc.server.GrpcService; import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.rpc.SupportedRpcType; +import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.impl.DelayLocalExecutionInjection; import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.util.NetUtils; +import org.apache.ratis.util.ReferenceCountedObject; +import org.junit.Assert; import java.util.Optional; @@ -45,6 +50,10 @@ public MiniRaftClusterWithGrpc newCluster(String[] ids, String[] listenerIds, Ra } }; + static { + ReferenceCountedObject.enableLeakDetection(); + } + public interface FactoryGet extends Factory.Get { @Override default Factory getFactory() { @@ -55,7 +64,8 @@ default Factory getFactory() { public static final DelayLocalExecutionInjection sendServerRequestInjection = new DelayLocalExecutionInjection(GrpcService.GRPC_SEND_SERVER_REQUEST); - protected MiniRaftClusterWithGrpc(String[] ids, String[] listenerIds, RaftProperties properties, Parameters parameters) { + protected MiniRaftClusterWithGrpc(String[] ids, String[] listenerIds, RaftProperties properties, + Parameters parameters) { super(ids, listenerIds, properties, parameters); } @@ -75,4 +85,22 @@ protected void blockQueueAndSetDelay(String leaderId, int delayMs) RaftTestUtil.blockQueueAndSetDelay(getServers(), sendServerRequestInjection, leaderId, delayMs, getTimeoutMax()); } + + @Override + public void shutdown() { + super.shutdown(); + assertZeroCopyMetrics(); + } + + public void assertZeroCopyMetrics() { + getServers().forEach(server -> server.getGroupIds().forEach(id -> { + LOG.info("Checking {}-{}", server.getId(), id); + RaftServer.Division division = RaftServerTestUtil.getDivision(server, id); + GrpcService service = (GrpcService) RaftServerTestUtil.getServerRpc(division); + ZeroCopyMetrics zeroCopyMetrics = service.getZeroCopyMetrics(); + Assert.assertEquals(0, zeroCopyMetrics.nonZeroCopyMessages()); + Assert.assertEquals("Zero copy messages are not released, please check logs to find leaks. ", + zeroCopyMetrics.zeroCopyMessages(), zeroCopyMetrics.releasedMessages()); + })); + } } diff --git a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java index ce08e4a74f..2f1492d433 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java @@ -168,6 +168,7 @@ public void testFollowerHeartbeatMetric() throws IOException, InterruptedExcepti assertTrue(t.getTimer().getMeanRate() > 0.0d); assertTrue(t.getTimer().getCount() > 0L); } + cluster.shutdown(); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java index 9b6d811211..5fa03328e4 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java @@ -138,7 +138,7 @@ void runTestNotLeaderExceptionWithReconf(CLUSTER cluster) throws Exception { @Test public void testGroupMismatchException() throws Exception { - runWithSameCluster(NUM_PEERS, this::runTestGroupMismatchException); + runWithNewCluster(NUM_PEERS, this::runTestGroupMismatchException); } void runTestGroupMismatchException(CLUSTER cluster) throws Exception { @@ -171,7 +171,7 @@ void runTestGroupMismatchException(CLUSTER cluster) throws Exception { @Test public void testStaleReadException() throws Exception { - runWithSameCluster(NUM_PEERS, this::runTestStaleReadException); + runWithNewCluster(NUM_PEERS, this::runTestStaleReadException); } void runTestStaleReadException(CLUSTER cluster) throws Exception { @@ -186,7 +186,7 @@ void runTestStaleReadException(CLUSTER cluster) throws Exception { @Test public void testLogAppenderBufferCapacity() throws Exception { - runWithSameCluster(NUM_PEERS, this::runTestLogAppenderBufferCapacity); + runWithNewCluster(NUM_PEERS, this::runTestLogAppenderBufferCapacity); } void runTestLogAppenderBufferCapacity(CLUSTER cluster) throws Exception { diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index 8cd15aac15..024a60d3ae 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -50,6 +50,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.lang.ref.WeakReference; import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Arrays; @@ -565,4 +566,18 @@ static void assertSuccessReply(RaftClientReply reply) { Assert.assertNotNull("reply == null", reply); Assert.assertTrue("reply is not success: " + reply, reply.isSuccess()); } + + static void gc() throws InterruptedException { + // use WeakReference to detect gc + Object obj = new Object(); + final WeakReference weakRef = new WeakReference<>(obj); + obj = null; + + // loop until gc has completed. + for (int i = 0; weakRef.get() != null; i++) { + LOG.info("gc {}", i); + System.gc(); + Thread.sleep(100); + } + } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 6453e8e944..6def81c478 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -353,6 +353,7 @@ public void testLateServerStart() throws Exception { 10, ONE_SECOND, "getLeaderId", LOG); LOG.info(cluster.printServers()); Assert.assertEquals(leader.getId(), lastServerLeaderId); + cluster.shutdown(); } protected void testDisconnectLeader() throws Exception { @@ -523,6 +524,7 @@ public void testLeaderElectionMetrics() throws IOException, InterruptedException Long leaderElectionLatency = (Long) ratisMetricRegistry.getGauges((s, metric) -> s.contains(LAST_LEADER_ELECTION_ELAPSED_TIME)).values().iterator().next().getValue(); assertTrue(leaderElectionLatency > 0L); + cluster.shutdown(); } @Test diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index f6dd6121cb..933a995cc9 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -50,6 +50,7 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.NetUtils; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedLeakDetector; import org.apache.ratis.util.ReflectionUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedConsumer; @@ -854,6 +855,14 @@ public void shutdown() { Optional.ofNullable(timer.get()).ifPresent(Timer::cancel); ExitUtils.assertNotTerminated(); LOG.info("{} shutdown completed", JavaUtils.getClassSimpleName(getClass())); + + // GC to ensure leak detection work. + try { + RaftTestUtil.gc(); + } catch (InterruptedException e) { + LOG.info("gc interrupted."); + } + ReferenceCountedLeakDetector.getLeakDetector().assertNoLeaks(); } /** From a2cb49e8cb64c30c1f7f7adb82f87477501f522d Mon Sep 17 00:00:00 2001 From: Nandakumar Vadivelu Date: Thu, 1 Feb 2024 21:34:03 +0530 Subject: [PATCH 30/76] RATIS-1975. Migrate ratis-metrics-dropwizard3 tests to Junit 5. (#1037) --- ratis-metrics-dropwizard3/pom.xml | 10 ++++++++++ .../TestLoadDm3MetricRegistries.java | 14 +++++++------- .../dropwizard3/TestRefCountingMap.java | 18 +++++++++--------- 3 files changed, 26 insertions(+), 16 deletions(-) diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml index 04b86cbaa8..42fff0445b 100644 --- a/ratis-metrics-dropwizard3/pom.xml +++ b/ratis-metrics-dropwizard3/pom.xml @@ -58,6 +58,16 @@ junit test + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.jupiter + junit-jupiter-api + test + org.mockito mockito-core diff --git a/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestLoadDm3MetricRegistries.java b/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestLoadDm3MetricRegistries.java index 92a36f5c86..5b3897ffd4 100644 --- a/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestLoadDm3MetricRegistries.java +++ b/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestLoadDm3MetricRegistries.java @@ -23,8 +23,8 @@ import org.apache.ratis.metrics.MetricRegistriesLoader; import org.apache.ratis.metrics.MetricRegistryInfo; import org.apache.ratis.metrics.RatisMetricRegistry; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; /** * Test class for {@link MetricRegistriesLoader}. @@ -33,7 +33,7 @@ public class TestLoadDm3MetricRegistries { @Test public void testLoadDm3() { final MetricRegistries r = MetricRegistriesLoader.load(); - Assert.assertSame(Dm3MetricRegistriesImpl.class, r.getClass()); + Assertions.assertSame(Dm3MetricRegistriesImpl.class, r.getClass()); } @Test @@ -47,15 +47,15 @@ public void testAddRemoveReporter() { // check if add and remove of metric do reporting counter increase MetricRegistryInfo info = new MetricRegistryInfo("t1", "t1", "t1", "t1"); r.create(info); - Assert.assertTrue(cntr.get() == 1); + Assertions.assertEquals(1, cntr.get()); r.remove(info); - Assert.assertTrue(cntr.get() == 2); + Assertions.assertEquals(2, cntr.get()); // after removal, add and remove of metric must not do any increase r.removeReporterRegistration(reporter, stopReporter); r.create(info); - Assert.assertTrue(cntr.get() == 2); + Assertions.assertEquals(2, cntr.get()); r.remove(info); - Assert.assertTrue(cntr.get() == 2); + Assertions.assertEquals(2, cntr.get()); } } diff --git a/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java b/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java index 1aa15d37c7..87b8bf0123 100644 --- a/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java +++ b/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java @@ -17,23 +17,23 @@ */ package org.apache.ratis.metrics.dropwizard3; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Collection; import java.util.Set; import org.apache.ratis.thirdparty.com.google.common.collect.Lists; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestRefCountingMap { private RefCountingMap map; - @Before + @BeforeEach public void setUp() { map = new RefCountingMap<>(); } @@ -128,7 +128,7 @@ public void testKeySet() { Set keys = map.keySet(); assertEquals(3, keys.size()); - Lists.newArrayList("foo", "bar", "baz").stream().forEach(v -> assertTrue(keys.contains(v))); + Lists.newArrayList("foo", "bar", "baz").forEach(v -> assertTrue(keys.contains(v))); } @Test @@ -141,7 +141,7 @@ public void testValues() { Collection values = map.values(); assertEquals(3, values.size()); - Lists.newArrayList("foovalue", "foovalue3", "foovalue4").stream() + Lists.newArrayList("foovalue", "foovalue3", "foovalue4") .forEach(v -> assertTrue(values.contains(v))); } } From e44de26623a6b65d9d66eb782df70ffd98fab364 Mon Sep 17 00:00:00 2001 From: Nandakumar Vadivelu Date: Thu, 1 Feb 2024 22:27:19 +0530 Subject: [PATCH 31/76] RATIS-1976. Migrate ratis-test tests to Junit 5 - Part 1 (#1038) --- .../ratis/InstallSnapshotFromLeaderTests.java | 31 ++-- .../org/apache/ratis/RetryCacheTests.java | 22 +-- .../server/impl/LeaderElectionTests.java | 132 +++++++++--------- ratis-test/pom.xml | 10 ++ .../ratis/client/TestClientProtoUtils.java | 6 +- .../org/apache/ratis/conf/TestConfUtils.java | 2 +- .../apache/ratis/conf/TestRaftProperties.java | 45 +++--- .../DataStreamAsyncClusterTests.java | 12 +- .../ratis/datastream/DataStreamBaseTest.java | 12 +- .../datastream/DataStreamClusterTests.java | 14 +- .../ratis/datastream/DataStreamTestUtils.java | 48 +++---- ...WithRpcTypeGrpcAndDataStreamTypeNetty.java | 1 - .../datastream/TestDataStreamDisabled.java | 10 +- ...WithRpcTypeGrpcAndDataStreamTypeNetty.java | 10 +- ...ataStreamChainTopologyWithGrpcCluster.java | 4 +- ...DataStreamStarTopologyWithGrpcCluster.java | 4 +- .../TestNettyDataStreamWithMock.java | 10 +- .../TestNettyDataStreamWithNettyCluster.java | 5 +- .../apache/ratis/grpc/TestGrpcFactory.java | 8 +- .../ratis/grpc/TestGrpcMessageMetrics.java | 6 +- .../grpc/TestLeaderElectionWithGrpc.java | 2 +- .../ratis/grpc/TestLeaderInstallSnapshot.java | 23 ++- .../ratis/grpc/TestLogAppenderWithGrpc.java | 35 +++-- .../grpc/TestRaftOutputStreamWithGrpc.java | 6 +- .../ratis/grpc/TestRaftServerWithGrpc.java | 94 +++++++------ .../ratis/grpc/TestRaftSnapshotWithGrpc.java | 8 +- .../apache/ratis/grpc/TestRaftWithGrpc.java | 42 +++--- .../ratis/grpc/TestRetryCacheWithGrpc.java | 14 +- .../grpc/server/TestGrpcServerMetrics.java | 26 ++-- .../grpc/util/GrpcZeroCopyTestServer.java | 10 +- .../ratis/grpc/util/TestGrpcZeroCopy.java | 16 +-- .../util/TestStreamObserverWithTimeout.java | 10 +- .../netty/TestLeaderElectionWithNetty.java | 2 +- 33 files changed, 348 insertions(+), 332 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java index 15dafb88c8..9d6a2f1834 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java @@ -39,8 +39,7 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,14 +73,12 @@ public abstract class InstallSnapshotFromLeaderTests { for (RaftServer.Division follower : cluster.getFollowers()) { final SnapshotInfo info = follower.getStateMachine().getLatestSnapshot(); - Assert.assertNotNull(info); - Assert.assertEquals(3, info.getFiles().size()); + Assertions.assertNotNull(info); + Assertions.assertEquals(3, info.getFiles().size()); } }, 10, ONE_SECOND, "check snapshot", LOG); } finally { @@ -182,8 +179,8 @@ public long takeSnapshot() { return RaftLog.INVALID_LOG_INDEX; } - Assert.assertTrue(file1.exists()); - Assert.assertTrue(file2.exists()); + Assertions.assertTrue(file1.exists()); + Assertions.assertTrue(file2.exists()); return super.takeSnapshot(); } @@ -199,7 +196,7 @@ public SnapshotInfo getLatestSnapshot() { files.add(new FileInfo( file2.toPath(), null)); - Assert.assertEquals(2, files.size()); + Assertions.assertEquals(2, files.size()); SnapshotInfo info = super.getLatestSnapshot(); if (info == null) { @@ -224,8 +221,8 @@ public synchronized void initialize(RaftServer server, RaftGroupId groupId, Raft this.snapshotDir = new File(root, "snapshot"); this.tmpDir = new File(root, "tmp"); FileUtils.deleteFully(root); - Assert.assertTrue(this.snapshotDir.mkdirs()); - Assert.assertTrue(this.tmpDir.mkdirs()); + Assertions.assertTrue(this.snapshotDir.mkdirs()); + Assertions.assertTrue(this.tmpDir.mkdirs()); this.root.deleteOnExit(); } @@ -246,13 +243,13 @@ public long takeSnapshot() { try { FileUtils.deleteFully(snapshotRealDir); FileUtils.deleteFully(snapshotTmpDir); - Assert.assertTrue(snapshotTmpDir.mkdirs()); + Assertions.assertTrue(snapshotTmpDir.mkdirs()); final File snapshotFile1 = new File(snapshotTmpDir, "deer"); final File snapshotFile2 = new File(snapshotTmpDir, "loves"); final File snapshotFile3 = new File(snapshotTmpDir, "vegetable"); - Assert.assertTrue(snapshotFile1.createNewFile()); - Assert.assertTrue(snapshotFile2.createNewFile()); - Assert.assertTrue(snapshotFile3.createNewFile()); + Assertions.assertTrue(snapshotFile1.createNewFile()); + Assertions.assertTrue(snapshotFile2.createNewFile()); + Assertions.assertTrue(snapshotFile3.createNewFile()); FileUtils.move(snapshotTmpDir, snapshotRealDir); } catch (IOException ioe) { LOG.error("create snapshot data file failed", ioe); diff --git a/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java b/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java index 18561ee65c..f81ac9556b 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java @@ -35,8 +35,8 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.Arrays; @@ -84,9 +84,9 @@ void runTestBasicRetry(CLUSTER cluster) throws Exception { } public static void assertReply(RaftClientReply reply, RaftClient client, long callId) { - Assert.assertEquals(client.getId(), reply.getClientId()); - Assert.assertEquals(callId, reply.getCallId()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertEquals(client.getId(), reply.getClientId()); + Assertions.assertEquals(callId, reply.getCallId()); + Assertions.assertTrue(reply.isSuccess()); } public void assertServer(MiniRaftCluster cluster, ClientId clientId, long callId, long oldLastApplied) throws Exception { @@ -97,10 +97,10 @@ public void assertServer(MiniRaftCluster cluster, ClientId clientId, long callId if (server.getInfo().getLastAppliedIndex() < leaderApplied) { Thread.sleep(1000); } - Assert.assertEquals(2, server.getRetryCache().getStatistics().size()); - Assert.assertNotNull(RetryCacheTestUtil.get(server, clientId, callId)); + Assertions.assertEquals(2, server.getRetryCache().getStatistics().size()); + Assertions.assertNotNull(RetryCacheTestUtil.get(server, clientId, callId)); // make sure there is only one log entry committed - Assert.assertEquals(1, count(server.getRaftLog(), oldLastApplied + 1)); + Assertions.assertEquals(1, count(server.getRaftLog(), oldLastApplied + 1)); } } @@ -145,10 +145,10 @@ void runTestRetryOnNewLeader(CLUSTER cluster) throws Exception { final RaftPeerId newLeaderId = JavaUtils.attemptRepeatedly(() -> { final RaftPeerId id = RaftTestUtil.waitForLeader(cluster).getId(); - Assert.assertNotEquals(leaderId, id); + Assertions.assertNotEquals(leaderId, id); return id; }, 10, TimeDuration.valueOf(100, TimeUnit.MILLISECONDS), "wait for a leader different than " + leaderId, LOG); - Assert.assertNotEquals(leaderId, newLeaderId); + Assertions.assertNotEquals(leaderId, newLeaderId); // same clientId and callId in the request r = cluster.newRaftClientRequest(client.getId(), newLeaderId, callId, new SimpleMessage("message")); @@ -164,7 +164,7 @@ void runTestRetryOnNewLeader(CLUSTER cluster) throws Exception { } // check the new leader and make sure the retry did not get committed - Assert.assertEquals(0, count(cluster.getLeader().getRaftLog(), oldLastApplied + 1)); + Assertions.assertEquals(0, count(cluster.getLeader().getRaftLog(), oldLastApplied + 1)); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 6def81c478..fbb909ac93 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -44,8 +44,8 @@ import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedBiConsumer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import java.io.IOException; @@ -138,18 +138,18 @@ public void testLeaderNotCountListenerForMajority() throws Exception { void runTestLeaderNotCountListenerForMajority(CLUSTER cluster) throws Exception { final RaftServer.Division leader = waitForLeader(cluster); - Assert.assertEquals(2, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); + Assertions.assertEquals(2, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List listeners = cluster.getListeners() .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); - Assert.assertEquals(2, listeners.size()); + Assertions.assertEquals(2, listeners.size()); RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assert.assertEquals(0, peer.size()); + Assertions.assertEquals(0, peer.size()); } - Assert.assertEquals(3, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); + Assertions.assertEquals(3, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); } @Test @@ -167,7 +167,7 @@ void runTestListenerNotStartLeaderElection(CLUSTER cluster) throws Exception { isolate(cluster, listenerId); maxTimeout.sleep(); maxTimeout.sleep(); - Assert.assertEquals(RaftProtos.RaftPeerRole.LISTENER, listener.getInfo().getCurrentRole()); + Assertions.assertEquals(RaftProtos.RaftPeerRole.LISTENER, listener.getInfo().getCurrentRole()); } finally { deIsolate(cluster, listener.getId()); } @@ -183,18 +183,18 @@ public void testTransferLeader() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers(); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); RaftServer.Division newLeader = followers.get(0); RaftClientReply reply = client.admin().transferLeadership(newLeader.getId(), 20000); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final RaftServer.Division currLeader = waitForLeader(cluster); - Assert.assertEquals(newLeader.getId(), currLeader.getId()); + Assertions.assertEquals(newLeader.getId(), currLeader.getId()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertEquals(newLeader.getId().toString(), reply.getReplierId()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); + Assertions.assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -211,24 +211,24 @@ public void testYieldLeaderToHigherPriority() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers(); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); RaftServer.Division newLeader = followers.get(0); List peers = cluster.getPeers(); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); // Wait the old leader to step down. // TODO: make it more deterministic. TimeDuration.valueOf(1, TimeUnit.SECONDS).sleep(); final RaftServer.Division currLeader = waitForLeader(cluster); - Assert.assertEquals(newLeader.getId(), currLeader.getId()); + Assertions.assertEquals(newLeader.getId(), currLeader.getId()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertEquals(newLeader.getId().toString(), reply.getReplierId()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); + Assertions.assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -243,7 +243,7 @@ public void testTransferLeaderTimeout() throws Exception { final RaftServer.Division leader = waitForLeader(cluster); try (RaftClient client = cluster.createClient(leader.getId())) { List followers = cluster.getFollowers(); - Assert.assertEquals(followers.size(), 2); + Assertions.assertEquals(followers.size(), 2); RaftServer.Division newLeader = followers.get(0); // isolate new leader, so that transfer leadership will timeout @@ -259,9 +259,9 @@ public void testTransferLeaderTimeout() throws Exception { client.admin().transferLeadership(newLeader.getId(), timeoutMs); } catch (TransferLeadershipException e) { long cost = System.currentTimeMillis() - start; - Assert.assertTrue(cost > timeoutMs); - Assert.assertTrue(e.getMessage().contains("Failed to transfer leadership to")); - Assert.assertTrue(e.getMessage().contains(TransferLeadership.Result.Type.TIMED_OUT.toString())); + Assertions.assertTrue(cost > timeoutMs); + Assertions.assertTrue(e.getMessage().contains("Failed to transfer leadership to")); + Assertions.assertTrue(e.getMessage().contains(TransferLeadership.Result.Type.TIMED_OUT.toString())); } return true; @@ -275,17 +275,17 @@ public void testTransferLeaderTimeout() throws Exception { try { client.io().send(new RaftTestUtil.SimpleMessage("message")); } catch (LeaderSteppingDownException e) { - Assert.assertTrue(e.getMessage().contains("is stepping down")); + Assertions.assertTrue(e.getMessage().contains("is stepping down")); } return null; }, 5, TimeDuration.ONE_SECOND, "check leader steppingDown", RaftServer.LOG); - Assert.assertTrue(transferTimeoutFuture.get()); + Assertions.assertTrue(transferTimeoutFuture.get()); // after transfer timeout, leader should accept request RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertEquals(leader.getId().toString(), reply.getReplierId()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertEquals(leader.getId().toString(), reply.getReplierId()); + Assertions.assertTrue(reply.isSuccess()); deIsolate(cluster, newLeader.getId()); } @@ -323,7 +323,7 @@ static void enforceLeader(MiniRaftCluster cluster, final String newLeader, Logge LOG.info(cluster.printServers()); final RaftServer.Division leader = cluster.getLeader(); - Assert.assertEquals(newLeader, leader.getId().toString()); + Assertions.assertEquals(newLeader, leader.getId().toString()); } @Test @@ -352,7 +352,7 @@ public void testLateServerStart() throws Exception { .orElseThrow(() -> new IllegalStateException("No leader yet")), 10, ONE_SECOND, "getLeaderId", LOG); LOG.info(cluster.printServers()); - Assert.assertEquals(leader.getId(), lastServerLeaderId); + Assertions.assertEquals(leader.getId(), lastServerLeaderId); cluster.shutdown(); } @@ -366,8 +366,8 @@ protected void testDisconnectLeader() throws Exception { Thread.sleep(1000); isolate(cluster, leader.getId()); RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertNotEquals(reply.getReplierId(), leader.getId().toString()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertNotEquals(reply.getReplierId(), leader.getId().toString()); + Assertions.assertTrue(reply.isSuccess()); } finally { deIsolate(cluster, leader.getId()); } @@ -398,15 +398,15 @@ public void testAddListener() throws Exception { try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); - Assert.assertEquals(servers.size(), 3); + Assertions.assertEquals(servers.size(), 3); MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, true, false, RaftProtos.RaftPeerRole.LISTENER); RaftClientReply reply = client.admin().setConfiguration(servers, Arrays.asList(changes.newPeers)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); Collection listener = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assert.assertEquals(1, listener.size()); - Assert.assertEquals(changes.newPeers[0].getId(), new ArrayList<>(listener).get(0).getId()); + Assertions.assertEquals(1, listener.size()); + Assertions.assertEquals(changes.newPeers[0].getId(), new ArrayList<>(listener).get(0).getId()); } cluster.shutdown(); } @@ -420,18 +420,18 @@ public void testAddFollowerWhenExistsListener() throws Exception { try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); - Assert.assertEquals(4, servers.size()); + Assertions.assertEquals(4, servers.size()); List listener = new ArrayList<>( leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER)); - Assert.assertEquals(1, listener.size()); + Assertions.assertEquals(1, listener.size()); MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, true, false); ArrayList newPeers = new ArrayList<>(Arrays.asList(changes.newPeers)); newPeers.addAll(leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER)); RaftClientReply reply = client.admin().setConfiguration(newPeers, listener); - Assert.assertTrue(reply.isSuccess()); - Assert.assertEquals(4, + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertEquals(4, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER).size()); - Assert.assertEquals(1, + Assertions.assertEquals(1, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } cluster.shutdown(); @@ -445,13 +445,13 @@ public void testRemoveListener() throws Exception { final RaftServer.Division leader = waitForLeader(cluster); try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertEquals(1, cluster.getListeners().size()); + Assertions.assertEquals(1, cluster.getListeners().size()); List servers = cluster.getFollowers().stream().map(RaftServer.Division::getPeer).collect( Collectors.toList()); servers.add(leader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(servers); - Assert.assertTrue(reply.isSuccess()); - Assert.assertEquals(0, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertEquals(0, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } cluster.shutdown(); } @@ -466,15 +466,15 @@ public void testChangeFollowerToListener() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers().stream().map( RaftServer.Division::getPeer).collect(Collectors.toList()); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); List listeners = new ArrayList<>(); listeners.add(followers.get(1)); followers.remove(1); RaftClientReply reply = client.admin().setConfiguration(followers, listeners); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assert.assertEquals(1, peer.size()); - Assert.assertEquals(listeners.get(0).getId(), new ArrayList<>(peer).get(0).getId()); + Assertions.assertEquals(1, peer.size()); + Assertions.assertEquals(listeners.get(0).getId(), new ArrayList<>(peer).get(0).getId()); } cluster.shutdown(); } @@ -489,11 +489,11 @@ public void testChangeListenerToFollower() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List listeners = cluster.getListeners() .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); - Assert.assertEquals(listeners.size(), 1); + Assertions.assertEquals(listeners.size(), 1); RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assert.assertEquals(0, peer.size()); + Assertions.assertEquals(0, peer.size()); } cluster.shutdown(); } @@ -573,7 +573,7 @@ public void testPreVote() { isolate(cluster, follower.getId()); // send message so that the isolated follower's log lag the others RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final long savedTerm = leader.getInfo().getCurrentTerm(); LOG.info("Wait follower {} timeout and trigger pre-vote", follower.getId()); @@ -588,7 +588,7 @@ public void testPreVote() { assertEquals(savedTerm, leader.getInfo().getCurrentTerm()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -608,7 +608,7 @@ void runTestListenerRejectRequestVote(CLUSTER cluster) throws IOException, Inter final RaftProtos.RequestVoteRequestProto r = ServerProtoUtils.toRequestVoteRequestProto( leader.getMemberId(), listener.getId(), leader.getRaftLog().getLastEntryTermIndex().getTerm() + 1, lastEntry, true); RaftProtos.RequestVoteReplyProto listenerReply = listener.getRaftServer().requestVote(r); - Assert.assertFalse(listenerReply.getServerReply().getSuccess()); + Assertions.assertFalse(listenerReply.getServerReply().getSuccess()); } @@ -622,23 +622,23 @@ void runTestPauseResumeLeaderElection(CLUSTER cluster) throws IOException, Inter final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); final RaftPeerId leaderId = leader.getId(); final List followers = cluster.getFollowers(); - Assert.assertTrue(followers.size() >= 1); + Assertions.assertTrue(followers.size() >= 1); final RaftServerImpl f1 = (RaftServerImpl)followers.get(0); try (final RaftClient client = cluster.createClient()) { pauseLeaderReply = client.getLeaderElectionManagementApi(f1.getId()).pause(); - Assert.assertTrue(pauseLeaderReply.isSuccess()); + Assertions.assertTrue(pauseLeaderReply.isSuccess()); client.io().send(new RaftTestUtil.SimpleMessage("message")); RaftServer.Division newLeader = followers.get(0); List peers = cluster.getPeers(); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - Assert.assertTrue(reply.isSuccess()); - JavaUtils.attempt(() -> Assert.assertEquals(leaderId, leader.getId()), + Assertions.assertTrue(reply.isSuccess()); + JavaUtils.attempt(() -> Assertions.assertEquals(leaderId, leader.getId()), 20, HUNDRED_MILLIS, "check leader id", LOG); final RaftClientReply resumeLeaderReply = client.getLeaderElectionManagementApi(f1.getId()).resume(); - Assert.assertTrue(resumeLeaderReply.isSuccess()); - JavaUtils.attempt(() -> Assert.assertEquals(f1.getId(), cluster.getLeader().getId()), + Assertions.assertTrue(resumeLeaderReply.isSuccess()); + JavaUtils.attempt(() -> Assertions.assertEquals(f1.getId(), cluster.getLeader().getId()), 20, HUNDRED_MILLIS, "check new leader", LOG); } } @@ -664,15 +664,15 @@ void runTestLeaderLease(CLUSTER cluster, long leaseTimeoutMs) throws Exception { try (final RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertTrue(leader.getInfo().isLeader()); - Assert.assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, true); isolate(cluster, leader.getId()); Thread.sleep(leaseTimeoutMs); - Assert.assertTrue(leader.getInfo().isLeader()); - Assert.assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, false); } finally { deIsolate(cluster, leader.getId()); @@ -692,8 +692,8 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM try (final RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertTrue(leader.getInfo().isLeader()); - Assert.assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, true); final List followers = cluster.getFollowers(); @@ -714,8 +714,8 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM Thread.sleep(leaseTimeoutMs); - Assert.assertTrue(leader.getInfo().isLeader()); - Assert.assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, false); } finally { diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 2021e06ad4..5990ae057f 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -136,11 +136,21 @@ junit test + + org.junit.jupiter + junit-jupiter-engine + test + org.junit.jupiter junit-jupiter-api test + + org.junit.jupiter + junit-jupiter-params + test + org.mockito mockito-core diff --git a/ratis-test/src/test/java/org/apache/ratis/client/TestClientProtoUtils.java b/ratis-test/src/test/java/org/apache/ratis/client/TestClientProtoUtils.java index 9d85320539..56bf94a84e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/client/TestClientProtoUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/client/TestClientProtoUtils.java @@ -29,8 +29,8 @@ import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.concurrent.TimeUnit; @@ -71,7 +71,7 @@ void runTestToRaftClientRequestProto(int n, SizeInBytes messageSize) final RaftClientRequest computed = ClientProtoUtils.toRaftClientRequest(proto); final TimeDuration r = startTime.elapsedTime().subtract(p); - Assert.assertEquals(request.getMessage().getContent(), computed.getMessage().getContent()); + Assertions.assertEquals(request.getMessage().getContent(), computed.getMessage().getContent()); toProto = toProto.add(p); toRequest = toRequest.add(r); diff --git a/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java b/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java index 1600da18dc..67c02cd5f6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java @@ -23,7 +23,7 @@ import org.apache.ratis.grpc.GrpcConfigKeys; import org.apache.ratis.netty.NettyConfigKeys; import org.apache.ratis.server.RaftServerConfigKeys; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestConfUtils extends BaseTest { @Test diff --git a/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java b/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java index c65d00f43e..29bfc321df 100644 --- a/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java +++ b/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java @@ -17,8 +17,10 @@ */ package org.apache.ratis.conf; -import org.junit.Assert; -import org.junit.Test; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestRaftProperties { enum Type {APPEND_ENTRIES} @@ -29,71 +31,72 @@ static class Request_Vote { static final String KEY = "key"; static void setUnderscoreValue(RaftProperties p, String valueWithUnderscore) { - Assert.assertTrue(valueWithUnderscore.contains("_")); + Assertions.assertTrue(valueWithUnderscore.contains("_")); p.set(KEY, valueWithUnderscore); } static void setNonUnderscoreValue(RaftProperties p, String valueWithoutUnderscore) { - Assert.assertFalse(valueWithoutUnderscore.contains("_")); + Assertions.assertFalse(valueWithoutUnderscore.contains("_")); p.set(KEY, valueWithoutUnderscore); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testUnderscore() { final RaftProperties p = new RaftProperties(); { // boolean - Assert.assertNull(p.getBoolean(KEY, null)); + Assertions.assertNull(p.getBoolean(KEY, null)); setNonUnderscoreValue(p, "true"); - Assert.assertTrue(p.getBoolean(KEY, null)); + Assertions.assertTrue(p.getBoolean(KEY, null)); setNonUnderscoreValue(p, "false"); - Assert.assertFalse(p.getBoolean(KEY, null)); + Assertions.assertFalse(p.getBoolean(KEY, null)); setUnderscoreValue(p, "fa_lse"); - Assert.assertNull(p.getBoolean(KEY, null)); + Assertions.assertNull(p.getBoolean(KEY, null)); p.unset(KEY); } { //int final Integer expected = 1000000; - Assert.assertNull(p.getInt(KEY, null)); + Assertions.assertNull(p.getInt(KEY, null)); setUnderscoreValue(p, "1_000_000"); - Assert.assertEquals(expected, p.getInt(KEY, null)); + Assertions.assertEquals(expected, p.getInt(KEY, null)); setNonUnderscoreValue(p, "1000000"); - Assert.assertEquals(expected, p.getInt(KEY, null)); + Assertions.assertEquals(expected, p.getInt(KEY, null)); p.unset(KEY); } { // long final Long expected = 1_000_000_000_000L; - Assert.assertNull(p.getLong(KEY, null)); + Assertions.assertNull(p.getLong(KEY, null)); setUnderscoreValue(p, "1_000_000_000_000"); - Assert.assertEquals(expected, p.getLong(KEY, null)); + Assertions.assertEquals(expected, p.getLong(KEY, null)); setNonUnderscoreValue(p, "1000000000000"); - Assert.assertEquals(expected, p.getLong(KEY, null)); + Assertions.assertEquals(expected, p.getLong(KEY, null)); p.unset(KEY); } { // File final String expected = "1_000_000"; - Assert.assertNull(p.getFile(KEY, null)); + Assertions.assertNull(p.getFile(KEY, null)); setUnderscoreValue(p, expected); - Assert.assertEquals(expected, p.getFile(KEY, null).getName()); + Assertions.assertEquals(expected, p.getFile(KEY, null).getName()); p.unset(KEY); } { // class final Type expected = Type.APPEND_ENTRIES; - Assert.assertNull(p.getEnum(KEY, Type.class, null)); + Assertions.assertNull(p.getEnum(KEY, Type.class, null)); setUnderscoreValue(p, expected.name()); - Assert.assertEquals(expected, p.getEnum(KEY, Type.class, null)); + Assertions.assertEquals(expected, p.getEnum(KEY, Type.class, null)); p.unset(KEY); } { // enum final Class expected = Request_Vote.class; - Assert.assertNull(p.getClass(KEY, null)); + Assertions.assertNull(p.getClass(KEY, null)); setUnderscoreValue(p, expected.getName()); - Assert.assertEquals(expected, p.getClass(KEY, null)); + Assertions.assertEquals(expected, p.getClass(KEY, null)); p.unset(KEY); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java index 8c315070e5..2fcf500e2c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java @@ -34,8 +34,8 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedBiFunction; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.IOException; @@ -130,14 +130,14 @@ void runTestDataStream(CLUSTER cluster, boolean stepDownLeader, CheckedBiFunctio // wait for all servers to catch up try (RaftClient client = cluster.createClient()) { RaftClientReply reply = client.async().watch(maxIndex, ReplicationLevel.ALL).join(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } // assert all streams are linked for (RaftServer proxy : cluster.getServers()) { final RaftServer.Division impl = proxy.getDivision(cluster.getGroupId()); final MultiDataStreamStateMachine stateMachine = (MultiDataStreamStateMachine) impl.getStateMachine(); for (SingleDataStream s : stateMachine.getStreams()) { - Assert.assertFalse(s.getDataChannel().isOpen()); + Assertions.assertFalse(s.getDataChannel().isOpen()); DataStreamTestUtils.assertLogEntry(impl, s); } } @@ -150,7 +150,7 @@ Long runTestDataStream( futures.add(CompletableFuture.supplyAsync( () -> runTestDataStream(cluster, numStreams, bufferSize, bufferNum, stepDownLeader), executor)); } - Assert.assertEquals(numClients, futures.size()); + Assertions.assertEquals(numClients, futures.size()); return futures.stream() .map(CompletableFuture::join) .max(Long::compareTo) @@ -174,7 +174,7 @@ long runTestDataStream(CLUSTER cluster, int numStreams, int bufferSize, int buff futures.add(CompletableFuture.supplyAsync(() -> DataStreamTestUtils.writeAndCloseAndAssertReplies( servers, leader, out, bufferSize, bufferNum, client.getId(), stepDownLeader).join(), executor)); } - Assert.assertEquals(numStreams, futures.size()); + Assertions.assertEquals(numStreams, futures.size()); return futures.stream() .map(CompletableFuture::join) .map(RaftClientReply::getLogIndex) diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java index 3f241a8b9c..70e26af249 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java @@ -34,7 +34,7 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.util.CollectionUtils; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import java.io.IOException; import java.util.ArrayList; @@ -132,10 +132,10 @@ void runTestMockCluster(ClientId clientId, int bufferSize, int bufferNum, .stream(null, DataStreamTestUtils.getRoutingTableChainTopology(peers, getPrimaryServer().getPeer())); if (headerException != null) { final DataStreamReply headerReply = out.getHeaderFuture().join(); - Assert.assertFalse(headerReply.isSuccess()); + Assertions.assertFalse(headerReply.isSuccess()); final RaftClientReply clientReply = ClientProtoUtils.toRaftClientReply( ((DataStreamReplyByteBuffer)headerReply).slice()); - Assert.assertTrue(clientReply.getException().getMessage().contains(headerException.getMessage())); + Assertions.assertTrue(clientReply.getException().getMessage().contains(headerException.getMessage())); return; } @@ -143,11 +143,11 @@ void runTestMockCluster(ClientId clientId, int bufferSize, int bufferNum, CollectionUtils.as(servers, Server::getRaftServer), null, out, bufferSize, bufferNum, client.getId(), false).join(); if (expectedException != null) { - Assert.assertFalse(clientReply.isSuccess()); - Assert.assertTrue(clientReply.getException().getMessage().contains( + Assertions.assertFalse(clientReply.isSuccess()); + Assertions.assertTrue(clientReply.getException().getMessage().contains( expectedException.getMessage())); } else { - Assert.assertTrue(clientReply.isSuccess()); + Assertions.assertTrue(clientReply.isSuccess()); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamClusterTests.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamClusterTests.java index 352d98e650..f99ff56236 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamClusterTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamClusterTests.java @@ -36,8 +36,8 @@ import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedConsumer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.File; import java.nio.channels.FileChannel; @@ -115,9 +115,9 @@ void runTestInvalidPrimaryInRoutingTable(CLUSTER cluster) throws Exception { } } - Assert.assertNotNull( - "Cannot find peer other than the primary", notPrimary); - Assert.assertNotEquals(primaryServer, notPrimary); + Assertions.assertNotNull(notPrimary, + "Cannot find peer other than the primary"); + Assertions.assertNotEquals(primaryServer, notPrimary); try (RaftClient client = cluster.createClient(primaryServer)) { RoutingTable routingTableWithWrongPrimary = @@ -156,7 +156,7 @@ static CheckedConsumer transferToWritableByteCh public void accept(DataStreamOutputImpl out) throws Exception { try (FileChannel in = FileUtils.newFileChannel(f, StandardOpenOption.READ)) { final long transferred = in.transferTo(0, size, out.getWritableByteChannel()); - Assert.assertEquals(size, transferred); + Assertions.assertEquals(size, transferred); } } @@ -196,7 +196,7 @@ void assertLogEntry(CLUSTER cluster, RaftClientRequest request) throws Exception final RaftServer.Division impl = proxy.getDivision(cluster.getGroupId()); final MultiDataStreamStateMachine stateMachine = (MultiDataStreamStateMachine) impl.getStateMachine(); final SingleDataStream s = stateMachine.getSingleDataStream(request); - Assert.assertFalse(s.getDataChannel().isOpen()); + Assertions.assertFalse(s.getDataChannel().isOpen()); DataStreamTestUtils.assertLogEntry(impl, s); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index 738cb0359c..e4a930f1d1 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -49,7 +49,7 @@ import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -87,7 +87,7 @@ static ByteBuffer initBuffer(int offset, int size) { buffer.put(pos2byte(offset + j)); } buffer.flip(); - Assert.assertEquals(length, buffer.remaining()); + Assertions.assertEquals(length, buffer.remaining()); return buffer; } @@ -117,7 +117,7 @@ public int read(ByteBuffer dst) { FileUtils.createDirectories(f.getParentFile()); try(FileChannel out = FileUtils.newFileChannel(f, StandardOpenOption.CREATE, StandardOpenOption.WRITE)) { final long transferred = out.transferFrom(source, 0, size); - Assert.assertEquals(size, transferred); + Assertions.assertEquals(size, transferred); } } @@ -253,7 +253,7 @@ public int write(ByteBuffer src) { } final int remaining = src.remaining(); for (; src.remaining() > 0; ) { - Assert.assertEquals(pos2byte(bytesWritten), src.get()); + Assertions.assertEquals(pos2byte(bytesWritten), src.get()); bytesWritten += 1; } return remaining; @@ -302,9 +302,9 @@ static int writeAndAssertReplies(DataStreamOutputImpl out, int bufferSize, int b } static void assertSuccessReply(Type expectedType, long expectedBytesWritten, DataStreamReply reply) { - Assert.assertTrue(reply.isSuccess()); - Assert.assertEquals(expectedBytesWritten, reply.getBytesWritten()); - Assert.assertEquals(expectedType, reply.getType()); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertEquals(expectedBytesWritten, reply.getBytesWritten()); + Assertions.assertEquals(expectedType, reply.getType()); } static CompletableFuture writeAndCloseAndAssertReplies( @@ -328,26 +328,26 @@ static CompletableFuture writeAndCloseAndAssertReplies( static void assertHeader(RaftServer server, RaftClientRequest header, int dataSize, boolean stepDownLeader) throws Exception { // check header - Assert.assertEquals(RaftClientRequest.dataStreamRequestType(), header.getType()); + Assertions.assertEquals(RaftClientRequest.dataStreamRequestType(), header.getType()); // check stream final MultiDataStreamStateMachine stateMachine = (MultiDataStreamStateMachine) server.getDivision(header.getRaftGroupId()).getStateMachine(); final SingleDataStream stream = stateMachine.getSingleDataStream(header); final MyDataChannel channel = stream.getDataChannel(); - Assert.assertEquals(dataSize, channel.getBytesWritten()); - Assert.assertEquals(dataSize, channel.getForcedPosition()); + Assertions.assertEquals(dataSize, channel.getBytesWritten()); + Assertions.assertEquals(dataSize, channel.getForcedPosition()); // check writeRequest final RaftClientRequest writeRequest = stream.getWriteRequest(); - Assert.assertEquals(RaftClientRequest.dataStreamRequestType(), writeRequest.getType()); + Assertions.assertEquals(RaftClientRequest.dataStreamRequestType(), writeRequest.getType()); assertRaftClientMessage(header, null, writeRequest, header.getClientId(), stepDownLeader); } static CompletableFuture assertCloseReply(DataStreamOutputImpl out, DataStreamReply dataStreamReply, long bytesWritten, RaftPeerId leader, ClientId clientId, boolean stepDownLeader) { // Test close idempotent - Assert.assertSame(dataStreamReply, out.closeAsync().join()); - Assert.assertEquals(dataStreamReply.getClientId(), clientId); + Assertions.assertSame(dataStreamReply, out.closeAsync().join()); + Assertions.assertEquals(dataStreamReply.getClientId(), clientId); BaseTest.testFailureCase("writeAsync should fail", () -> out.writeAsync(DataStreamRequestByteBuffer.EMPTY_BYTE_BUFFER).join(), CompletionException.class, (Logger) null, AlreadyClosedException.class); @@ -359,7 +359,7 @@ static CompletableFuture assertCloseReply(DataStreamOutputImpl if (reply.isSuccess()) { final ByteString bytes = reply.getMessage().getContent(); if (!bytes.equals(MOCK)) { - Assert.assertEquals(bytesWritten2ByteString(bytesWritten), bytes); + Assertions.assertEquals(bytesWritten2ByteString(bytesWritten), bytes); } } @@ -372,13 +372,13 @@ static CompletableFuture assertCloseReply(DataStreamOutputImpl static void assertRaftClientMessage( RaftClientMessage expected, RaftPeerId expectedServerId, RaftClientMessage computed, ClientId expectedClientId, boolean stepDownLeader) { - Assert.assertNotNull(computed); - Assert.assertEquals(expectedClientId, computed.getClientId()); + Assertions.assertNotNull(computed); + Assertions.assertEquals(expectedClientId, computed.getClientId()); if (!stepDownLeader) { - Assert.assertEquals( + Assertions.assertEquals( Optional.ofNullable(expectedServerId).orElseGet(expected::getServerId), computed.getServerId()); } - Assert.assertEquals(expected.getRaftGroupId(), computed.getRaftGroupId()); + Assertions.assertEquals(expected.getRaftGroupId(), computed.getRaftGroupId()); } static LogEntryProto searchLogEntry(ClientInvocationId invocationId, RaftLog log) throws Exception { @@ -394,12 +394,12 @@ static LogEntryProto searchLogEntry(ClientInvocationId invocationId, RaftLog log } static void assertLogEntry(LogEntryProto logEntry, RaftClientRequest request) { - Assert.assertNotNull(logEntry); - Assert.assertTrue(logEntry.hasStateMachineLogEntry()); + Assertions.assertNotNull(logEntry); + Assertions.assertTrue(logEntry.hasStateMachineLogEntry()); final StateMachineLogEntryProto s = logEntry.getStateMachineLogEntry(); - Assert.assertEquals(StateMachineLogEntryProto.Type.DATASTREAM, s.getType()); - Assert.assertEquals(request.getCallId(), s.getCallId()); - Assert.assertEquals(request.getClientId().toByteString(), s.getClientId()); + Assertions.assertEquals(StateMachineLogEntryProto.Type.DATASTREAM, s.getType()); + Assertions.assertEquals(request.getCallId(), s.getCallId()); + Assertions.assertEquals(request.getClientId().toByteString(), s.getClientId()); } static void assertLogEntry(RaftServer.Division division, SingleDataStream stream) throws Exception { @@ -408,6 +408,6 @@ static void assertLogEntry(RaftServer.Division division, SingleDataStream stream assertLogEntry(entryFromStream, request); final LogEntryProto entryFromLog = searchLogEntry(ClientInvocationId.valueOf(request), division.getRaftLog()); - Assert.assertEquals(entryFromStream, entryFromLog); + Assertions.assertEquals(entryFromStream, entryFromLog); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java b/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java index 3396ada9b3..b9e20fb82e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java @@ -17,7 +17,6 @@ */ package org.apache.ratis.datastream; -import org.apache.ratis.security.TlsConf; import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.RaftConfigKeys; import org.apache.ratis.conf.Parameters; diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java index 168a1b02dc..697e746877 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java @@ -26,14 +26,14 @@ import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftPeer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestDataStreamDisabled extends BaseTest { @Test public void testDataStreamDisabled() throws Exception { final RaftProperties properties = new RaftProperties(); - Assert.assertEquals(SupportedDataStreamType.DISABLED, RaftConfigKeys.DataStream.type(properties, LOG::info)); + Assertions.assertEquals(SupportedDataStreamType.DISABLED, RaftConfigKeys.DataStream.type(properties, LOG::info)); final RaftPeer server = RaftPeer.newBuilder().setId("s0").build(); @@ -44,9 +44,9 @@ public void testDataStreamDisabled() throws Exception { .setProperties(properties) .build(); DataStreamOutput out = client.getDataStreamApi().stream()) { - Assert.fail("Unexpected object: " + out); + Assertions.fail("Unexpected object: " + out); } catch (UnsupportedOperationException e) { - Assert.assertTrue(e.getMessage().contains( + Assertions.assertTrue(e.getMessage().contains( DisabledDataStreamClientFactory.class.getName() + "$1 does not support streamAsync")); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java index 8e423ab293..8e6d892c83 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java @@ -24,7 +24,7 @@ import org.apache.ratis.security.TlsConf; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Ignore; +import org.junit.jupiter.api.Disabled; import org.slf4j.event.Level; import java.util.function.Supplier; @@ -55,22 +55,22 @@ public MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.Factory getFactory() return new MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.Factory(getParameters()); } - @Ignore + @Disabled @Override public void testStreamWrites() { } - @Ignore + @Disabled @Override public void testStreamWithInvalidRoutingTable() { } - @Ignore + @Disabled @Override public void testMultipleStreamsMultipleServers() { } - @Ignore + @Disabled @Override public void testMultipleStreamsMultipleServersStepDownLeader() { } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamChainTopologyWithGrpcCluster.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamChainTopologyWithGrpcCluster.java index 31b28b4c2d..778ee8225c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamChainTopologyWithGrpcCluster.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamChainTopologyWithGrpcCluster.java @@ -22,13 +22,13 @@ import org.apache.ratis.netty.NettyConfigKeys; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; public class TestNettyDataStreamChainTopologyWithGrpcCluster extends DataStreamAsyncClusterTests implements MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.FactoryGet { - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); RaftClientConfigKeys.DataStream.setRequestTimeout(p, TimeDuration.ONE_MINUTE); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamStarTopologyWithGrpcCluster.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamStarTopologyWithGrpcCluster.java index 45247d489a..bd80d6b6b5 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamStarTopologyWithGrpcCluster.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamStarTopologyWithGrpcCluster.java @@ -25,7 +25,7 @@ import org.apache.ratis.protocol.RoutingTable; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; import java.util.Collection; import java.util.List; @@ -35,7 +35,7 @@ public class TestNettyDataStreamStarTopologyWithGrpcCluster extends DataStreamAsyncClusterTests implements MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.FactoryGet { - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); RaftClientConfigKeys.DataStream.setRequestTimeout(p, TimeDuration.ONE_MINUTE); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java index 27a1ee102f..503f8cf66e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java @@ -36,14 +36,14 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.util.NetUtils; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -@Ignore +@Disabled public class TestNettyDataStreamWithMock extends DataStreamBaseTest { static RaftPeer newRaftPeer(RaftServer server) { return RaftPeer.newBuilder() @@ -53,7 +53,7 @@ static RaftPeer newRaftPeer(RaftServer server) { .build(); } - @Before + @BeforeEach public void setup() { properties = new RaftProperties(); RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.NETTY); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithNettyCluster.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithNettyCluster.java index 90af31425f..1a29d014e4 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithNettyCluster.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithNettyCluster.java @@ -17,9 +17,10 @@ */ package org.apache.ratis.datastream; -import org.junit.Ignore; -@Ignore("Ignored by runzhiwang, because NettyClientRpc does not support sendRequestAsync") +import org.junit.jupiter.api.Disabled; + +@Disabled("Ignored by runzhiwang, because NettyClientRpc does not support sendRequestAsync") public class TestNettyDataStreamWithNettyCluster extends DataStreamClusterTests implements MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty.FactoryGet { diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcFactory.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcFactory.java index 76fbcee5eb..99a395d8a1 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcFactory.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcFactory.java @@ -18,16 +18,14 @@ package org.apache.ratis.grpc; import org.apache.ratis.BaseTest; -import org.apache.ratis.util.JavaUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestGrpcFactory extends BaseTest { @Test public void testUseCacheForAllThreads() { // trigger GrpcFactory static initializer final boolean value = GrpcFactory.checkPooledByteBufAllocatorUseCacheForAllThreads(LOG::info); - Assert.assertFalse(value); - LOG.info("value is {}", value); + Assertions.assertFalse(value); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java index 2abba79300..aee13223b8 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java @@ -29,8 +29,8 @@ import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -68,6 +68,6 @@ static void assertMessageCount(RaftServer.Division server) { GrpcService service = (GrpcService) RaftServerTestUtil.getServerRpc(server); RatisMetricRegistry registry = service.getServerInterceptor().getMetrics().getRegistry(); String counter_prefix = serverId + "_" + "ratis.grpc.RaftServerProtocolService"; - Assert.assertTrue(registry.counter(counter_prefix + "_" + "requestVote" + "_OK_completed_total").getCount() > 0); + Assertions.assertTrue(registry.counter(counter_prefix + "_" + "requestVote" + "_OK_completed_total").getCount() > 0); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java index 7730cb1166..ef6bc2a866 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java @@ -19,7 +19,7 @@ import org.apache.ratis.server.impl.BlockRequestHandlingInjection; import org.apache.ratis.server.impl.LeaderElectionTests; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestLeaderElectionWithGrpc extends LeaderElectionTests diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java index 5f7a40f0f4..22c590c9dd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java @@ -18,23 +18,32 @@ package org.apache.ratis.grpc; import org.apache.ratis.InstallSnapshotFromLeaderTests; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Arrays; import java.util.Collection; -@RunWith(Parameterized.class) public class TestLeaderInstallSnapshot extends InstallSnapshotFromLeaderTests implements MiniRaftClusterWithGrpc.FactoryGet { - public TestLeaderInstallSnapshot(Boolean separateHeartbeat) { + public static Collection data() { + return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); + } + + @ParameterizedTest + @MethodSource("data") + public void testMultiFileInstallSnapshot(Boolean separateHeartbeat) throws Exception { GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); + super.testMultiFileInstallSnapshot(); } - @Parameterized.Parameters - public static Collection data() { - return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); + @ParameterizedTest + @MethodSource("data") + public void testSeparateSnapshotInstallPath(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); + super.testSeparateSnapshotInstallPath(); } + } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java index c0d102f957..107cd7ba9a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java @@ -33,10 +33,9 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; import java.io.IOException; @@ -47,7 +46,6 @@ import static org.apache.ratis.RaftTestUtil.waitForLeader; -@RunWith(Parameterized.class) public class TestLogAppenderWithGrpc extends LogAppenderTests implements MiniRaftClusterWithGrpc.FactoryGet { @@ -55,17 +53,14 @@ public class TestLogAppenderWithGrpc Slf4jUtils.setLogLevel(FollowerInfo.LOG, Level.DEBUG); } - public TestLogAppenderWithGrpc(Boolean separateHeartbeat) { - GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); - } - - @Parameterized.Parameters public static Collection data() { return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); } - @Test - public void testPendingLimits() throws IOException, InterruptedException { + @ParameterizedTest + @MethodSource("data") + public void testPendingLimits(Boolean separateHeartbeat) throws IOException, InterruptedException { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); int maxAppends = 10; RaftProperties properties = new RaftProperties(); properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, @@ -94,7 +89,7 @@ public void testPendingLimits() throws IOException, InterruptedException { JavaUtils.attempt(() -> { for (long nextIndex : leader.getInfo().getFollowerNextIndices()) { // Verify nextIndex does not progress due to pendingRequests limit - Assert.assertEquals(initialNextIndex + maxAppends, nextIndex); + Assertions.assertEquals(initialNextIndex + maxAppends, nextIndex); } }, 10, ONE_SECOND, "matching nextIndex", LOG); for (RaftServer.Division server : cluster.getFollowers()) { @@ -107,8 +102,10 @@ public void testPendingLimits() throws IOException, InterruptedException { } } - @Test - public void testRestartLogAppender() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testRestartLogAppender(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(2, this::runTestRestartLogAppender); } @@ -120,7 +117,7 @@ private void runTestRestartLogAppender(MiniRaftClusterWithGrpc cluster) throws E try(RaftClient client = cluster.createClient(leader.getId())) { for(int i = 0; i < 10; i++) { final RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + ++messageCount)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -128,7 +125,7 @@ private void runTestRestartLogAppender(MiniRaftClusterWithGrpc cluster) throws E final GrpcServerMetrics leaderMetrics = new GrpcServerMetrics(leader.getMemberId().toString()); final String counter = String.format(GrpcServerMetrics.RATIS_GRPC_METRICS_LOG_APPENDER_INCONSISTENCY, cluster.getFollowers().iterator().next().getMemberId().getPeerId()); - Assert.assertEquals(0L, leaderMetrics.getRegistry().counter(counter).getCount()); + Assertions.assertEquals(0L, leaderMetrics.getRegistry().counter(counter).getCount()); // restart LogAppender RaftServerTestUtil.restartLogAppenders(leader); @@ -137,7 +134,7 @@ private void runTestRestartLogAppender(MiniRaftClusterWithGrpc cluster) throws E try(RaftClient client = cluster.createClient(leader.getId())) { for(int i = 0; i < 10; i++) { final RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + ++messageCount)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -148,7 +145,7 @@ private void runTestRestartLogAppender(MiniRaftClusterWithGrpc cluster) throws E // assert INCONSISTENCY counter >= 1 // If old LogAppender die before new LogAppender start, INCONSISTENCY equal to 1, // else INCONSISTENCY greater than 1 - Assert.assertTrue(newleaderMetrics.getRegistry().counter(counter).getCount() >= 1L); + Assertions.assertTrue(newleaderMetrics.getRegistry().counter(counter).getCount() >= 1L); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftOutputStreamWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftOutputStreamWithGrpc.java index fb35d958ab..2f1ef3f124 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftOutputStreamWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftOutputStreamWithGrpc.java @@ -18,12 +18,10 @@ package org.apache.ratis.grpc; import org.apache.ratis.OutputStreamBaseTest; +import org.junit.jupiter.api.Timeout; +@Timeout(value = 100) public class TestRaftOutputStreamWithGrpc extends OutputStreamBaseTest implements MiniRaftClusterWithGrpc.FactoryGet { - @Override - public int getGlobalTimeoutSeconds() { - return 100; - } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java index 0af1d87cce..05d772c17e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java @@ -26,7 +26,6 @@ import org.apache.ratis.protocol.ClientInvocationId; import org.apache.ratis.server.RetryCache; import org.apache.ratis.util.JavaUtils; -import org.slf4j.event.Level; import org.apache.ratis.conf.Parameters; import org.apache.ratis.security.SecurityTestUtils; import org.apache.ratis.server.storage.RaftStorage; @@ -62,12 +61,11 @@ import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.event.Level; import javax.net.ssl.KeyManager; import javax.net.ssl.TrustManager; import java.io.IOException; @@ -80,30 +78,26 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -@RunWith(Parameterized.class) public class TestRaftServerWithGrpc extends BaseTest implements MiniRaftClusterWithGrpc.FactoryGet { { Slf4jUtils.setLogLevel(GrpcClientProtocolClient.LOG, Level.TRACE); } - public TestRaftServerWithGrpc(Boolean separateHeartbeat) { - GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); - } - - @Parameterized.Parameters public static Collection data() { return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); } - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, SimpleStateMachine4Testing.class, StateMachine.class); RaftClientConfigKeys.Rpc.setRequestTimeout(p, TimeDuration.valueOf(1, TimeUnit.SECONDS)); } - @Test - public void testServerRestartOnException() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testServerRestartOnException(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(1, this::runTestServerRestartOnException); } @@ -149,8 +143,10 @@ void runTestServerRestartOnException(MiniRaftClusterWithGrpc cluster) throws Exc cluster.getServerFactory(leaderId).newRaftServerRpc(cluster.getServer(leaderId)); } - @Test - public void testUnsupportedMethods() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testUnsupportedMethods(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(1, this::runTestUnsupportedMethods); } @@ -167,8 +163,10 @@ void runTestUnsupportedMethods(MiniRaftClusterWithGrpc cluster) throws Exception UnsupportedOperationException.class); } - @Test - public void testLeaderRestart() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testLeaderRestart(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(3, this::runTestLeaderRestart); } @@ -178,7 +176,7 @@ void runTestLeaderRestart(MiniRaftClusterWithGrpc cluster) throws Exception { try (final RaftClient client = cluster.createClient()) { // send a request to make sure leader is ready final CompletableFuture f = client.async().send(new SimpleMessage("testing")); - Assert.assertTrue(f.get().isSuccess()); + Assertions.assertTrue(f.get().isSuccess()); } try (final RaftClient client = cluster.createClient()) { @@ -189,14 +187,14 @@ void runTestLeaderRestart(MiniRaftClusterWithGrpc cluster) throws Exception { { // send a request using rpc directly final RaftClientRequest request = newRaftClientRequest(client, seqNum.incrementAndGet()); - Assert.assertEquals(client.getId(), request.getClientId()); + Assertions.assertEquals(client.getId(), request.getClientId()); final CompletableFuture f = rpc.sendRequestAsync(request); final RaftClientReply reply = f.get(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); RaftClientTestUtil.handleReply(request, reply, client); invocationId = ClientInvocationId.valueOf(request.getClientId(), request.getCallId()); final RetryCache.Entry entry = leader.getRetryCache().getIfPresent(invocationId); - Assert.assertNotNull(entry); + Assertions.assertNotNull(entry); LOG.info("cache entry {}", entry); } @@ -206,13 +204,13 @@ void runTestLeaderRestart(MiniRaftClusterWithGrpc cluster) throws Exception { final RaftClientRequest requestBlocked = newRaftClientRequest(client, seqNum.incrementAndGet()); final CompletableFuture futureBlocked = rpc.sendRequestAsync(requestBlocked); - JavaUtils.attempt(() -> Assert.assertNull(leader.getRetryCache().getIfPresent(invocationId)), + JavaUtils.attempt(() -> Assertions.assertNull(leader.getRetryCache().getIfPresent(invocationId)), 10, HUNDRED_MILLIS, "invalidate cache entry", LOG); LOG.info("cache entry not found for {}", invocationId); // change leader RaftTestUtil.changeLeader(cluster, leader.getId()); - Assert.assertNotEquals(RaftPeerRole.LEADER, leader.getInfo().getCurrentRole()); + Assertions.assertNotEquals(RaftPeerRole.LEADER, leader.getInfo().getCurrentRole()); // the blocked request should fail testFailureCase("request should fail", futureBlocked::get, @@ -229,13 +227,17 @@ void runTestLeaderRestart(MiniRaftClusterWithGrpc cluster) throws Exception { } - @Test - public void testRaftClientMetrics() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testRaftClientMetrics(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(3, this::testRaftClientRequestMetrics); } - @Test - public void testRaftServerMetrics() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testRaftServerMetrics(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); final RaftProperties p = getProperties(); RaftServerConfigKeys.Write.setElementLimit(p, 10); RaftServerConfigKeys.Write.setByteLimit(p, SizeInBytes.valueOf("1MB")); @@ -252,7 +254,7 @@ void testRequestMetrics(MiniRaftClusterWithGrpc cluster) throws Exception { try (RaftClient client = cluster.createClient()) { // send a request to make sure leader is ready final CompletableFuture< RaftClientReply > f = client.async().send(new SimpleMessage("testing")); - Assert.assertTrue(f.get().isSuccess()); + Assertions.assertTrue(f.get().isSuccess()); } SimpleStateMachine4Testing stateMachine = SimpleStateMachine4Testing.get(cluster.getLeader()); @@ -285,14 +287,14 @@ void testRequestMetrics(MiniRaftClusterWithGrpc cluster) throws Exception { client = cluster.createClient(cluster.getLeader().getId(), RetryPolicies.noRetry()); final SizeInBytes size = SizeInBytes.valueOf("1025kb"); final ByteString bytes = randomByteString(size.getSizeInt()); - Assert.assertEquals(size.getSizeInt(), bytes.size()); + Assertions.assertEquals(size.getSizeInt(), bytes.size()); client.async().send(new SimpleMessage(size + "-message", bytes)); clients.add(client); RaftTestUtil.waitFor(() -> getRaftServerMetrics(cluster.getLeader()) .getNumRequestsByteSizeLimitHits().getCount() == 1, 300, 5000); - Assert.assertEquals(2, getRaftServerMetrics(cluster.getLeader()) + Assertions.assertEquals(2, getRaftServerMetrics(cluster.getLeader()) .getNumResourceLimitHits().getCount()); } finally { for (RaftClient client : clients) { @@ -326,36 +328,36 @@ void testRaftClientRequestMetrics(MiniRaftClusterWithGrpc cluster) throws IOExce try (final RaftClient client = cluster.createClient()) { final CompletableFuture f1 = client.async().send(new SimpleMessage("testing")); - Assert.assertTrue(f1.get().isSuccess()); + Assertions.assertTrue(f1.get().isSuccess()); final DefaultTimekeeperImpl write = (DefaultTimekeeperImpl) registry.timer(RAFT_CLIENT_WRITE_REQUEST); - JavaUtils.attempt(() -> Assert.assertTrue(write.getTimer().getCount() > 0), + JavaUtils.attempt(() -> Assertions.assertTrue(write.getTimer().getCount() > 0), 3, TimeDuration.ONE_SECOND, "writeTimer metrics", LOG); final CompletableFuture f2 = client.async().sendReadOnly(new SimpleMessage("testing")); - Assert.assertTrue(f2.get().isSuccess()); + Assertions.assertTrue(f2.get().isSuccess()); final DefaultTimekeeperImpl read = (DefaultTimekeeperImpl) registry.timer(RAFT_CLIENT_READ_REQUEST); - JavaUtils.attempt(() -> Assert.assertTrue(read.getTimer().getCount() > 0), + JavaUtils.attempt(() -> Assertions.assertTrue(read.getTimer().getCount() > 0), 3, TimeDuration.ONE_SECOND, "readTimer metrics", LOG); final CompletableFuture f3 = client.async().sendStaleRead(new SimpleMessage("testing"), 0, leader.getId()); - Assert.assertTrue(f3.get().isSuccess()); + Assertions.assertTrue(f3.get().isSuccess()); final DefaultTimekeeperImpl staleRead = (DefaultTimekeeperImpl) registry.timer(RAFT_CLIENT_STALE_READ_REQUEST); - JavaUtils.attempt(() -> Assert.assertTrue(staleRead.getTimer().getCount() > 0), + JavaUtils.attempt(() -> Assertions.assertTrue(staleRead.getTimer().getCount() > 0), 3, TimeDuration.ONE_SECOND, "staleReadTimer metrics", LOG); final CompletableFuture f4 = client.async().watch(0, RaftProtos.ReplicationLevel.ALL); - Assert.assertTrue(f4.get().isSuccess()); + Assertions.assertTrue(f4.get().isSuccess()); final DefaultTimekeeperImpl watchAll = (DefaultTimekeeperImpl) registry.timer( String.format(RAFT_CLIENT_WATCH_REQUEST, "-ALL")); - JavaUtils.attempt(() -> Assert.assertTrue(watchAll.getTimer().getCount() > 0), + JavaUtils.attempt(() -> Assertions.assertTrue(watchAll.getTimer().getCount() > 0), 3, TimeDuration.ONE_SECOND, "watchAllTimer metrics", LOG); final CompletableFuture f5 = client.async().watch(0, RaftProtos.ReplicationLevel.MAJORITY); - Assert.assertTrue(f5.get().isSuccess()); + Assertions.assertTrue(f5.get().isSuccess()); final DefaultTimekeeperImpl watch = (DefaultTimekeeperImpl) registry.timer( String.format(RAFT_CLIENT_WATCH_REQUEST, "")); - JavaUtils.attempt(() -> Assert.assertTrue(watch.getTimer().getCount() > 0), + JavaUtils.attempt(() -> Assertions.assertTrue(watch.getTimer().getCount() > 0), 3, TimeDuration.ONE_SECOND, "watchTimer metrics", LOG); } } @@ -366,8 +368,10 @@ static RaftClientRequest newRaftClientRequest(RaftClient client, long seqNum) { RaftClientRequest.writeRequestType(), ProtoUtils.toSlidingWindowEntry(seqNum, seqNum == 1L)); } - @Test - public void testTlsWithKeyAndTrustManager() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testTlsWithKeyAndTrustManager(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); final RaftProperties p = getProperties(); RaftServerConfigKeys.Write.setElementLimit(p, 10); RaftServerConfigKeys.Write.setByteLimit(p, SizeInBytes.valueOf("1MB")); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java index 2d8524f26c..82318c43ee 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java @@ -26,7 +26,7 @@ import org.apache.ratis.metrics.RatisMetricRegistry; import org.apache.ratis.server.RaftServer; import org.apache.ratis.statemachine.RaftSnapshotBaseTest; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; public class TestRaftSnapshotWithGrpc extends RaftSnapshotBaseTest { @Override @@ -39,10 +39,10 @@ protected void verifyInstallSnapshotMetric(RaftServer.Division leader) { MetricRegistryInfo info = new MetricRegistryInfo(leader.getMemberId().toString(), "ratis_grpc", "log_appender", "Metrics for Ratis Grpc Log Appender"); Optional metricRegistry = MetricRegistries.global().get(info); - Assert.assertTrue(metricRegistry.isPresent()); + Assertions.assertTrue(metricRegistry.isPresent()); final LongCounter installSnapshotCounter = metricRegistry.get().counter("num_install_snapshot"); - Assert.assertNotNull(installSnapshotCounter); - Assert.assertTrue(installSnapshotCounter.getCount() >= 1); + Assertions.assertNotNull(installSnapshotCounter); + Assertions.assertTrue(installSnapshotCounter.getCount() >= 1); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java index 046453d582..b93621137d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java @@ -30,10 +30,9 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Arrays; import java.util.Collection; @@ -42,7 +41,6 @@ import static org.apache.ratis.RaftTestUtil.waitForLeader; -@RunWith(Parameterized.class) public class TestRaftWithGrpc extends RaftBasicTests implements MiniRaftClusterWithGrpc.FactoryGet { @@ -52,29 +50,29 @@ public class TestRaftWithGrpc SimpleStateMachine4Testing.class, StateMachine.class); } - public TestRaftWithGrpc(Boolean separateHeartbeat) { - GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); - } - - @Parameterized.Parameters public static Collection data() { return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); } - @Override - @Test - public void testWithLoad() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testWithLoad(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); super.testWithLoad(); BlockRequestHandlingInjection.getInstance().unblockAll(); } - @Test - public void testRequestTimeout() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testRequestTimeout(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(NUM_SERVERS, cluster -> testRequestTimeout(false, cluster, LOG)); } - @Test - public void testUpdateViaHeartbeat() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testUpdateViaHeartbeat(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(NUM_SERVERS, this::runTestUpdateViaHeartbeat); } @@ -91,7 +89,7 @@ void runTestUpdateViaHeartbeat(MiniRaftClusterWithGrpc cluster) throws Exception replyFuture = client.async().send(new RaftTestUtil.SimpleMessage("abc")); TimeDuration.valueOf(5 , TimeUnit.SECONDS).sleep(); // replyFuture should not be completed until append request is unblocked. - Assert.assertFalse(replyFuture.isDone()); + Assertions.assertFalse(replyFuture.isDone()); // unblock append request. cluster.getServerAliveStream() .filter(impl -> !impl.getInfo().isLeader()) @@ -107,9 +105,9 @@ void runTestUpdateViaHeartbeat(MiniRaftClusterWithGrpc cluster) throws Exception final LogEntryHeader[] leaderEntries = leaderLog.getEntries(0, Long.MAX_VALUE); final RaftLog followerLog = raftServer.getRaftLog(); - Assert.assertEquals(leaderNextIndex, followerLog.getNextIndex()); + Assertions.assertEquals(leaderNextIndex, followerLog.getNextIndex()); final LogEntryHeader[] serverEntries = followerLog.getEntries(0, Long.MAX_VALUE); - Assert.assertArrayEquals(serverEntries, leaderEntries); + Assertions.assertArrayEquals(serverEntries, leaderEntries); }, 10, HUNDRED_MILLIS, "assertRaftLog-" + raftServer.getId(), LOG))); // Wait for heartbeats from leader to be received by followers @@ -119,8 +117,8 @@ void runTestUpdateViaHeartbeat(MiniRaftClusterWithGrpc cluster) throws Exception final long leaderNextIndex = leaderLog.getNextIndex(); // FollowerInfo in the leader state should have updated next and match index. final long followerMatchIndex = logAppender.getFollower().getMatchIndex(); - Assert.assertTrue(followerMatchIndex >= leaderNextIndex - 1); - Assert.assertEquals(followerMatchIndex + 1, logAppender.getFollower().getNextIndex()); + Assertions.assertTrue(followerMatchIndex >= leaderNextIndex - 1); + Assertions.assertEquals(followerMatchIndex + 1, logAppender.getFollower().getNextIndex()); }, 10, HUNDRED_MILLIS, "assertRaftLog-" + logAppender.getFollower(), LOG))); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java index a39a4d1ef4..879b9eabdb 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java @@ -37,8 +37,9 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.io.IOException; @@ -66,7 +67,7 @@ public void testInvalidateRepliedCalls() throws Exception { } static long assertReply(RaftClientReply reply) { - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); return reply.getCallId(); } @@ -90,7 +91,7 @@ void assertRetryCacheEntry(RaftClient client, long callId, boolean exist) throws void assertRetryCacheEntry(RaftClient client, long callId, boolean exist, boolean eventually) throws InterruptedException { Supplier lookup = () -> RetryCacheTestUtil.get(leader, client.getId(), callId); - Consumer assertion = exist ? Assert::assertNotNull : Assert::assertNull; + Consumer assertion = exist ? Assertions::assertNotNull : Assertions::assertNull; if (eventually) { JavaUtils.attempt(() -> assertion.accept(lookup.get()), 100, TimeDuration.ONE_MILLISECOND, "retry cache entry", null); @@ -144,7 +145,7 @@ void run() throws Exception { ONE_SECOND.sleep(); // No calls can be completed. for (CompletableFuture f : asyncCalls) { - Assert.assertFalse(f.isDone()); + Assertions.assertFalse(f.isDone()); } stateMachine.unblockApplyTransaction(); // No calls can be invalidated. @@ -170,7 +171,8 @@ void run() throws Exception { } } - @Test(timeout = 10000) + @Test + @Timeout(value = 10000) public void testRetryOnResourceUnavailableException() throws InterruptedException, IOException { RaftProperties properties = new RaftProperties(); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcServerMetrics.java b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcServerMetrics.java index 04f8ded95d..1b57730594 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcServerMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcServerMetrics.java @@ -38,9 +38,9 @@ import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.thirdparty.com.codahale.metrics.Gauge; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; public class TestGrpcServerMetrics { @@ -50,7 +50,7 @@ public class TestGrpcServerMetrics { private static RaftPeerId raftPeerId; private static RaftPeerId followerId; - @BeforeClass + @BeforeAll public static void setUp() throws Exception { raftGroupId = RaftGroupId.randomId(); raftPeerId = RaftPeerId.valueOf("TestId"); @@ -74,11 +74,11 @@ public void testGrpcLogAppenderLatencyTimer() throws Exception { final String format = RATIS_GRPC_METRICS_LOG_APPENDER_LATENCY + GrpcServerMetrics.getHeartbeatSuffix(heartbeat); final String name = String.format(format, followerId); final DefaultTimekeeperImpl t = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(name); - Assert.assertEquals(0L, t.getTimer().getSnapshot().getMax()); + Assertions.assertEquals(0L, t.getTimer().getSnapshot().getMax()); req.startRequestTimer(); Thread.sleep(1000L); req.stopRequestTimer(); - Assert.assertTrue(t.getTimer().getSnapshot().getMax() > 1000L); + Assertions.assertTrue(t.getTimer().getSnapshot().getMax() > 1000L); } } @@ -89,7 +89,7 @@ public void testGrpcLogRequestTotal() { RATIS_GRPC_METRICS_REQUESTS_COUNT + GrpcServerMetrics .getHeartbeatSuffix(heartbeat)).getCount(); grpcServerMetrics.onRequestCreate(heartbeat); - Assert.assertEquals(reqTotal + 1, ratisMetricRegistry.counter( + Assertions.assertEquals(reqTotal + 1, ratisMetricRegistry.counter( RATIS_GRPC_METRICS_REQUESTS_COUNT + GrpcServerMetrics .getHeartbeatSuffix(heartbeat)).getCount()); } @@ -97,9 +97,9 @@ public void testGrpcLogRequestTotal() { @Test public void testGrpcLogRequestRetry() { - Assert.assertEquals(0L, ratisMetricRegistry.counter(RATIS_GRPC_METRICS_REQUEST_RETRY_COUNT).getCount()); + Assertions.assertEquals(0L, ratisMetricRegistry.counter(RATIS_GRPC_METRICS_REQUEST_RETRY_COUNT).getCount()); grpcServerMetrics.onRequestRetry(); - Assert.assertEquals(1L, ratisMetricRegistry.counter(RATIS_GRPC_METRICS_REQUEST_RETRY_COUNT).getCount()); + Assertions.assertEquals(1L, ratisMetricRegistry.counter(RATIS_GRPC_METRICS_REQUEST_RETRY_COUNT).getCount()); } @Test @@ -110,9 +110,9 @@ public void testGrpcLogPendingRequestCount() { pendingRequest::logRequestsSize); final String name = String.format(RATIS_GRPC_METRICS_LOG_APPENDER_PENDING_COUNT, raftPeerId); final Gauge gauge = ServerMetricsTestUtils.getGaugeWithName(name, grpcServerMetrics::getRegistry); - Assert.assertEquals(0, gauge.getValue()); + Assertions.assertEquals(0, gauge.getValue()); when(pendingRequest.logRequestsSize()).thenReturn(10); - Assert.assertEquals(10, gauge.getValue()); + Assertions.assertEquals(10, gauge.getValue()); } @Test @@ -133,8 +133,8 @@ public void testGrpcLogAppenderRequestCounters() { private void assertCounterIncremented(String counterVar, Consumer incFunction) { String counter = String.format(counterVar, raftPeerId.toString()); - Assert.assertEquals(0L, ratisMetricRegistry.counter(counter).getCount()); + Assertions.assertEquals(0L, ratisMetricRegistry.counter(counter).getCount()); incFunction.accept(raftPeerId.toString()); - Assert.assertEquals(1L, ratisMetricRegistry.counter(counter).getCount()); + Assertions.assertEquals(1L, ratisMetricRegistry.counter(counter).getCount()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java index 21db98d4c4..77e234b3ed 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java @@ -32,7 +32,7 @@ import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.TraditionalBinaryPrefix; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -117,10 +117,10 @@ Count getNonZeroCopyCount() { void assertCounts(int expectNumElements, long expectNumBytes) { LOG.info("ZeroCopyCount = {}", zeroCopyCount); LOG.info("nonZeroCopyCount = {}", nonZeroCopyCount); - Assert.assertEquals("zeroCopyCount.getNumElements()", expectNumElements, zeroCopyCount.getNumElements()); - Assert.assertEquals("zeroCopyCount.getNumBytes()", expectNumBytes, zeroCopyCount.getNumBytes()); - Assert.assertEquals("nonZeroCopyCount.getNumElements()", 0, nonZeroCopyCount.getNumElements()); - Assert.assertEquals("nonZeroCopyCount.getNumBytes()", 0, nonZeroCopyCount.getNumBytes()); + Assertions.assertEquals(expectNumElements, zeroCopyCount.getNumElements(), "zeroCopyCount.getNumElements()"); + Assertions.assertEquals(expectNumBytes, zeroCopyCount.getNumBytes()," zeroCopyCount.getNumBytes()"); + Assertions.assertEquals(0, nonZeroCopyCount.getNumElements(), "nonZeroCopyCount.getNumElements()"); + Assertions.assertEquals(0, nonZeroCopyCount.getNumBytes(), "nonZeroCopyCount.getNumBytes()"); } int start() throws IOException { diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java index 032a9c1db5..a5f61ee269 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java @@ -26,8 +26,8 @@ import org.apache.ratis.thirdparty.io.netty.buffer.PooledByteBufAllocator; import org.apache.ratis.util.NetUtils; import org.apache.ratis.util.TraditionalBinaryPrefix; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.List; @@ -60,8 +60,8 @@ static void verify(long seed, ByteString b) { random.nextBytes(array); final ByteString expected = UnsafeByteOperations.unsafeWrap(array, 0, remaining); final ByteString computed = b.substring(offset, offset + remaining); - Assert.assertEquals(expected.size(), computed.size()); - Assert.assertEquals(expected, computed); + Assertions.assertEquals(expected.size(), computed.size()); + Assertions.assertEquals(expected, computed); offset += remaining; } } @@ -99,7 +99,7 @@ public static boolean isReady() { /** Test a zero-copy marshaller is available from the versions of gRPC and Protobuf. */ @Test public void testReadiness() { - Assert.assertTrue(isReady()); + Assertions.assertTrue(isReady()); } @@ -134,7 +134,7 @@ void sendMessages(int n, GrpcZeroCopyTestClient client, GrpcZeroCopyTestServer s for (int i = 0; i < futures.size(); i++) { final String expected = GrpcZeroCopyTestServer.toReply(i, messages.get(i)); final String reply = futures.get(i).get(); - Assert.assertEquals("expected = " + expected + " != reply = " + reply, expected, reply); + Assertions.assertEquals(expected, reply, "expected = " + expected + " != reply = " + reply); server.assertCounts(numElements, numBytes); } } @@ -159,8 +159,8 @@ void sendBinaries(int n, GrpcZeroCopyTestClient client, GrpcZeroCopyTestServer s } final ByteString reply = future.get(); - Assert.assertEquals(4, reply.size()); - Assert.assertEquals(size, reply.asReadOnlyByteBuffer().getInt()); + Assertions.assertEquals(4, reply.size()); + Assertions.assertEquals(size, reply.asReadOnlyByteBuffer().getInt()); numElements++; numBytes += size; diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java index d0c936aa40..b279736f39 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java @@ -25,8 +25,8 @@ import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.TimeoutTimer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.ArrayList; @@ -106,7 +106,7 @@ void runTestTimeout(int slow, Type type) throws Exception { for (; i < slow; i++) { final String expected = (i + warmup) + GrpcTestServer.GreeterImpl.toReplySuffix(messages.get(i)); final String reply = futures.get(i).get(); - Assert.assertEquals(expected, reply); + Assertions.assertEquals(expected, reply); LOG.info("{}) passed", (i + warmup)); } @@ -114,10 +114,10 @@ void runTestTimeout(int slow, Type type) throws Exception { final CompletableFuture f = futures.get(i); try { final String reply = f.get(); - Assert.fail((i + warmup) + ") reply = " + reply + ", " + Assertions.fail((i + warmup) + ") reply = " + reply + ", " + StringUtils.completableFuture2String(f, false)); } catch (ExecutionException e) { - LOG.info("GOOD! {}) {}, {}", (i + warmup), StringUtils.completableFuture2String(f, true), e); + LOG.info("GOOD! {}) {}", (i + warmup), StringUtils.completableFuture2String(f, true), e); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java index f84bbb7360..f3b760bf22 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java @@ -19,7 +19,7 @@ import org.apache.ratis.server.impl.BlockRequestHandlingInjection; import org.apache.ratis.server.impl.LeaderElectionTests; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestLeaderElectionWithNetty extends LeaderElectionTests From 4f5612fd89f4ff63a8f529d1517edc663cf84ec2 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 1 Feb 2024 19:11:13 +0100 Subject: [PATCH 32/76] RATIS-2023. Remove duplicate RefCountingMap (#1039) --- .../apache/ratis/util}/RefCountingMap.java | 20 +-- .../ratis/util}/TestRefCountingMap.java | 3 +- .../metrics/impl/MetricRegistriesImpl.java | 1 + .../dropwizard3/Dm3MetricRegistriesImpl.java | 1 + .../metrics/dropwizard3/RefCountingMap.java | 94 ----------- .../dropwizard3/TestRefCountingMap.java | 147 ------------------ 6 files changed, 14 insertions(+), 252 deletions(-) rename {ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl => ratis-common/src/main/java/org/apache/ratis/util}/RefCountingMap.java (88%) rename {ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl => ratis-common/src/test/java/org/apache/ratis/util}/TestRefCountingMap.java (98%) delete mode 100644 ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/RefCountingMap.java delete mode 100644 ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java diff --git a/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/RefCountingMap.java b/ratis-common/src/main/java/org/apache/ratis/util/RefCountingMap.java similarity index 88% rename from ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/RefCountingMap.java rename to ratis-common/src/main/java/org/apache/ratis/util/RefCountingMap.java index 49759781f8..a6a9eb81ab 100644 --- a/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/RefCountingMap.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/RefCountingMap.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.ratis.metrics.impl; +package org.apache.ratis.util; import java.util.Collection; import java.util.Set; @@ -31,7 +31,7 @@ * call will increment the ref count, and each remove() will decrement it. The values are removed * from the map iff ref count == 0. */ -class RefCountingMap { +public final class RefCountingMap { private static class Payload { private final V value; private final AtomicInteger refCount = new AtomicInteger(); @@ -55,15 +55,15 @@ Payload decrement() { private final ConcurrentMap> map = new ConcurrentHashMap<>(); - V put(K k, Supplier supplier) { + public V put(K k, Supplier supplier) { return map.compute(k, (k1, old) -> old != null? old: new Payload<>(supplier.get())).increment(); } - static V get(Payload p) { + public static V get(Payload p) { return p == null ? null : p.get(); } - V get(K k) { + public V get(K k) { return get(map.get(k)); } @@ -72,23 +72,23 @@ V get(K k) { * @param k the key to remove * @return the value associated with the specified key or null if key is removed from map. */ - V remove(K k) { + public V remove(K k) { return get(map.computeIfPresent(k, (k1, v) -> v.decrement())); } - void clear() { + public void clear() { map.clear(); } - Set keySet() { + public Set keySet() { return map.keySet(); } - Collection values() { + public Collection values() { return map.values().stream().map(Payload::get).collect(Collectors.toList()); } - int size() { + public int size() { return map.size(); } } diff --git a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl/TestRefCountingMap.java b/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java similarity index 98% rename from ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl/TestRefCountingMap.java rename to ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java index be6eb3037c..db6e9239ae 100644 --- a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl/TestRefCountingMap.java +++ b/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.ratis.metrics.impl; +package org.apache.ratis.util; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -26,6 +26,7 @@ import java.util.Set; import org.apache.ratis.thirdparty.com.google.common.collect.Lists; +import org.apache.ratis.util.RefCountingMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/MetricRegistriesImpl.java b/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/MetricRegistriesImpl.java index 17968ae9f8..088508fab9 100644 --- a/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/MetricRegistriesImpl.java +++ b/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/MetricRegistriesImpl.java @@ -28,6 +28,7 @@ import org.apache.ratis.metrics.MetricRegistries; import org.apache.ratis.metrics.MetricRegistryInfo; import org.apache.ratis.metrics.RatisMetricRegistry; +import org.apache.ratis.util.RefCountingMap; import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistriesImpl.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistriesImpl.java index b26f2e27a4..a90c5a0cee 100644 --- a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistriesImpl.java +++ b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistriesImpl.java @@ -28,6 +28,7 @@ import org.apache.ratis.metrics.MetricRegistries; import org.apache.ratis.metrics.MetricRegistryInfo; import org.apache.ratis.metrics.RatisMetricRegistry; +import org.apache.ratis.util.RefCountingMap; import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/RefCountingMap.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/RefCountingMap.java deleted file mode 100644 index 6c3ad6f8cd..0000000000 --- a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/RefCountingMap.java +++ /dev/null @@ -1,94 +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.ratis.metrics.dropwizard3; - -import java.util.Collection; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; -import java.util.stream.Collectors; - -/** - * A map of K to V, but does ref counting for added and removed values. The values are - * not added directly, but instead requested from the given Supplier if ref count == 0. Each put() - * call will increment the ref count, and each remove() will decrement it. The values are removed - * from the map iff ref count == 0. - */ -class RefCountingMap { - private static class Payload { - private final V value; - private final AtomicInteger refCount = new AtomicInteger(); - - Payload(V v) { - this.value = v; - } - - V get() { - return value; - } - - V increment() { - return refCount.incrementAndGet() > 0? value: null; - } - - Payload decrement() { - return refCount.decrementAndGet() > 0? this: null; - } - } - - private final ConcurrentMap> map = new ConcurrentHashMap<>(); - - V put(K k, Supplier supplier) { - return map.compute(k, (k1, old) -> old != null? old: new Payload<>(supplier.get())).increment(); - } - - static V get(Payload p) { - return p == null ? null : p.get(); - } - - V get(K k) { - return get(map.get(k)); - } - - /** - * Decrements the ref count of k, and removes from map if ref count == 0. - * @param k the key to remove - * @return the value associated with the specified key or null if key is removed from map. - */ - V remove(K k) { - return get(map.computeIfPresent(k, (k1, v) -> v.decrement())); - } - - void clear() { - map.clear(); - } - - Set keySet() { - return map.keySet(); - } - - Collection values() { - return map.values().stream().map(Payload::get).collect(Collectors.toList()); - } - - int size() { - return map.size(); - } -} diff --git a/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java b/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java deleted file mode 100644 index 87b8bf0123..0000000000 --- a/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java +++ /dev/null @@ -1,147 +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.ratis.metrics.dropwizard3; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertTrue; - -import java.util.Collection; -import java.util.Set; - -import org.apache.ratis.thirdparty.com.google.common.collect.Lists; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -public class TestRefCountingMap { - - private RefCountingMap map; - - @BeforeEach - public void setUp() { - map = new RefCountingMap<>(); - } - - @Test - public void testPutGet() { - map.put("foo", () -> "foovalue"); - - String v = map.get("foo"); - assertNotNull(v); - assertEquals("foovalue", v); - } - - @Test - public void testPutMulti() { - String v1 = map.put("foo", () -> "foovalue"); - String v2 = map.put("foo", () -> "foovalue2"); - String v3 = map.put("foo", () -> "foovalue3"); - - String v = map.get("foo"); - assertEquals("foovalue", v); - assertEquals(v, v1); - assertEquals(v, v2); - assertEquals(v, v3); - } - - @Test - public void testPutRemove() { - map.put("foo", () -> "foovalue"); - String v = map.remove("foo"); - assertNull(v); - v = map.get("foo"); - assertNull(v); - } - - @Test - public void testPutRemoveMulti() { - map.put("foo", () -> "foovalue"); - map.put("foo", () -> "foovalue2"); - map.put("foo", () -> "foovalue3"); - - // remove 1 - String v = map.remove("foo"); - assertEquals("foovalue", v); - - // remove 2 - v = map.remove("foo"); - assertEquals("foovalue", v); - - // remove 3 - v = map.remove("foo"); - assertNull(v); - v = map.get("foo"); - assertNull(v); - } - - @Test - public void testSize() { - assertEquals(0, map.size()); - - // put a key - map.put("foo", () -> "foovalue"); - assertEquals(1, map.size()); - - // put a different key - map.put("bar", () -> "foovalue2"); - assertEquals(2, map.size()); - - // put the same key again - map.put("bar", () -> "foovalue3"); - assertEquals(2, map.size()); // map should be same size - } - - @Test - public void testClear() { - map.put("foo", () -> "foovalue"); - map.put("bar", () -> "foovalue2"); - map.put("baz", () -> "foovalue3"); - - map.clear(); - - assertEquals(0, map.size()); - } - - - @Test - public void testKeySet() { - map.put("foo", () -> "foovalue"); - map.put("bar", () -> "foovalue2"); - map.put("baz", () -> "foovalue3"); - - Set keys = map.keySet(); - assertEquals(3, keys.size()); - - Lists.newArrayList("foo", "bar", "baz").forEach(v -> assertTrue(keys.contains(v))); - } - - @Test - public void testValues() { - map.put("foo", () -> "foovalue"); - map.put("foo", () -> "foovalue2"); - map.put("bar", () -> "foovalue3"); - map.put("baz", () -> "foovalue4"); - - Collection values = map.values(); - assertEquals(3, values.size()); - - Lists.newArrayList("foovalue", "foovalue3", "foovalue4") - .forEach(v -> assertTrue(values.contains(v))); - } -} From 775b286c4540057e5f81419806d9e7737e5f568e Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Thu, 1 Feb 2024 15:39:48 -0800 Subject: [PATCH 33/76] RATIS-1997. Refactor StateMachine interface to use ReferenceCountedObject (#1036) --- .../filestore/FileStoreStateMachine.java | 9 ++++--- .../ratis/statemachine/StateMachine.java | 27 +++++++++++++++++++ .../raftlog/segmented/SegmentedRaftLog.java | 2 +- .../segmented/SegmentedRaftLogWorker.java | 24 ++++++++++------- .../impl/SimpleStateMachine4Testing.java | 4 ++- .../segmented/TestSegmentedRaftLog.java | 3 ++- 6 files changed, 53 insertions(+), 16 deletions(-) diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java index 858e300ec9..0ee7a60ac7 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java @@ -42,6 +42,7 @@ import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.ReferenceCountedObject; import java.io.IOException; import java.nio.file.Path; @@ -123,7 +124,8 @@ public TransactionContext startTransaction(LogEntryProto entry, RaftProtos.RaftP } @Override - public CompletableFuture write(LogEntryProto entry, TransactionContext context) { + public CompletableFuture write(ReferenceCountedObject entryRef, TransactionContext context) { + LogEntryProto entry = entryRef.retain(); final FileStoreRequestProto proto = getProto(context, entry); if (proto.getRequestCase() != FileStoreRequestProto.RequestCase.WRITEHEADER) { return null; @@ -132,9 +134,10 @@ public CompletableFuture write(LogEntryProto entry, TransactionContext final WriteRequestHeaderProto h = proto.getWriteHeader(); final CompletableFuture f = files.write(entry.getIndex(), h.getPath().toStringUtf8(), h.getClose(), h.getSync(), h.getOffset(), - entry.getStateMachineLogEntry().getStateMachineEntry().getStateMachineData()); + entry.getStateMachineLogEntry().getStateMachineEntry().getStateMachineData() + ).whenComplete((r, e) -> entryRef.release()); // sync only if closing the file - return h.getClose()? f: null; + return h.getClose() ? f: null; } static FileStoreRequestProto getProto(TransactionContext context, LogEntryProto entry) { diff --git a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java index b1fc5addae..915b70bb81 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java @@ -92,7 +92,9 @@ default CompletableFuture read(LogEntryProto entry, TransactionConte * Write asynchronously the state machine data in the given log entry to this state machine. * * @return a future for the write task + * @deprecated Applications should implement {@link #write(ReferenceCountedObject, TransactionContext)} instead. */ + @Deprecated default CompletableFuture write(LogEntryProto entry) { return CompletableFuture.completedFuture(null); } @@ -101,11 +103,36 @@ default CompletableFuture write(LogEntryProto entry) { * Write asynchronously the state machine data in the given log entry to this state machine. * * @return a future for the write task + * @deprecated Applications should implement {@link #write(ReferenceCountedObject, TransactionContext)} instead. */ + @Deprecated default CompletableFuture write(LogEntryProto entry, TransactionContext context) { return write(entry); } + /** + * Write asynchronously the state machine data in the given log entry to this state machine. + * + * @param entryRef Reference to a log entry. + * Implementations of this method may call {@link ReferenceCountedObject#get()} + * to access the log entry before this method returns. + * If the log entry is needed after this method returns, + * e.g. for asynchronous computation or caching, + * the implementation must invoke {@link ReferenceCountedObject#retain()} + * and {@link ReferenceCountedObject#release()}. + * @return a future for the write task + */ + default CompletableFuture write(ReferenceCountedObject entryRef, TransactionContext context) { + final LogEntryProto entry = entryRef.get(); + try { + final LogEntryProto copy = LogEntryProto.parseFrom(entry.toByteString()); + return write(copy, context); + } catch (InvalidProtocolBufferException e) { + return JavaUtils.completeExceptionally(new IllegalStateException( + "Failed to copy log entry " + TermIndex.valueOf(entry), e)); + } + } + /** * Create asynchronously a {@link DataStream} to stream state machine data. * The state machine may use the first message (i.e. request.getMessage()) as the header to create the stream. diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 4e057c07b2..baac0c6c7f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -428,7 +428,7 @@ protected CompletableFuture appendEntryImpl(ReferenceCountedObject entry, + LogEntryProto removedStateMachineData, TransactionContext context) { return addIOTask(new WriteLog(entry, removedStateMachineData, context)); } @@ -486,25 +487,28 @@ private class WriteLog extends Task { private final CompletableFuture stateMachineFuture; private final CompletableFuture combined; - WriteLog(LogEntryProto entry, LogEntryProto removedStateMachineData, TransactionContext context) { + WriteLog(ReferenceCountedObject entryRef, LogEntryProto removedStateMachineData, + TransactionContext context) { + LogEntryProto origEntry = entryRef.get(); this.entry = removedStateMachineData; - if (this.entry == entry) { - final StateMachineLogEntryProto proto = entry.hasStateMachineLogEntry()? entry.getStateMachineLogEntry(): null; + if (this.entry == origEntry) { + final StateMachineLogEntryProto proto = origEntry.hasStateMachineLogEntry() ? + origEntry.getStateMachineLogEntry(): null; if (stateMachine != null && proto != null && proto.getType() == StateMachineLogEntryProto.Type.DATASTREAM) { final ClientInvocationId invocationId = ClientInvocationId.valueOf(proto); final CompletableFuture removed = server.getDataStreamMap().remove(invocationId); - this.stateMachineFuture = removed == null? stateMachine.data().link(null, entry) - : removed.thenApply(stream -> stateMachine.data().link(stream, entry)); + this.stateMachineFuture = removed == null? stateMachine.data().link(null, origEntry) + : removed.thenApply(stream -> stateMachine.data().link(stream, origEntry)); } else { this.stateMachineFuture = null; } } else { try { - // this.entry != entry iff the entry has state machine data - this.stateMachineFuture = stateMachine.data().write(entry, context); + // this.entry != origEntry if it has state machine data + this.stateMachineFuture = stateMachine.data().write(entryRef, context); } catch (Exception e) { - LOG.error(name + ": writeStateMachineData failed for index " + entry.getIndex() - + ", entry=" + LogProtoUtils.toLogEntryString(entry, stateMachine::toStateMachineLogEntryString), e); + LOG.error(name + ": writeStateMachineData failed for index " + origEntry.getIndex() + + ", entry=" + LogProtoUtils.toLogEntryString(origEntry, stateMachine::toStateMachineLogEntryString), e); throw e; } } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 7c40ec2518..17d5a607a9 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -48,6 +48,7 @@ import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.MD5FileUtil; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -367,7 +368,8 @@ public TransactionContext startTransaction(RaftClientRequest request) { } @Override - public CompletableFuture write(LogEntryProto entry) { + public CompletableFuture write(ReferenceCountedObject entry, TransactionContext context) { + Preconditions.assertTrue(entry.get() != null); return blocking.getFuture(Blocking.Type.WRITE_STATE_MACHINE_DATA); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 3d5d5f87d0..38341e0258 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -39,6 +39,7 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.server.storage.RaftStorageTestUtils; +import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.impl.BaseStateMachine; @@ -634,7 +635,7 @@ public void testServerShutdownOnTimeoutIOException() throws Throwable { final LogEntryProto entry = prepareLogEntry(0, 0, null, true); final StateMachine sm = new BaseStateMachine() { @Override - public CompletableFuture write(LogEntryProto entry) { + public CompletableFuture write(ReferenceCountedObject entry, TransactionContext context) { getLifeCycle().transition(LifeCycle.State.STARTING); getLifeCycle().transition(LifeCycle.State.RUNNING); From 1d99921ef4bf676029cf231faf2ca79eff59f44b Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 6 Feb 2024 08:22:52 -0800 Subject: [PATCH 34/76] RATIS-2024. Refactor appendEntries code. (#1040) --- .../ratis/server/impl/RaftServerImpl.java | 169 +++++++----------- .../apache/ratis/server/impl/ServerState.java | 23 ++- .../impl/SnapshotInstallationHandler.java | 29 ++- .../ratis/server/util/ServerStringUtils.java | 8 +- 4 files changed, 94 insertions(+), 135 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index c47db14d66..7390093c33 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -108,6 +108,11 @@ import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedSupplier; +import static org.apache.ratis.server.impl.ServerImplUtils.effectiveCommitIndex; +import static org.apache.ratis.server.impl.ServerProtoUtils.toAppendEntriesReplyProto; +import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesReplyString; +import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesRequestString; + class RaftServerImpl implements RaftServer.Division, RaftServerProtocol, RaftServerAsynchronousProtocol, RaftClientProtocol, RaftClientAsynchronousProtocol { @@ -1483,18 +1488,24 @@ public CompletableFuture appendEntriesAsync( ReferenceCountedObject requestRef) throws IOException { final AppendEntriesRequestProto r = requestRef.retain(); final RaftRpcRequestProto request = r.getServerRequest(); - final List entries = r.getEntriesList(); final TermIndex previous = r.hasPreviousLog()? TermIndex.valueOf(r.getPreviousLog()) : null; - final RaftPeerId requestorId = RaftPeerId.valueOf(request.getRequestorId()); - try { - preAppendEntriesAsync(requestorId, ProtoUtils.toRaftGroupId(request.getRaftGroupId()), r.getLeaderTerm(), - previous, r.getLeaderCommit(), r.getInitializing(), entries); - return appendEntriesAsync(requestorId, r.getLeaderTerm(), previous, r.getLeaderCommit(), - request.getCallId(), r.getInitializing(), r.getCommitInfosList(), entries, requestRef); + final RaftPeerId leaderId = RaftPeerId.valueOf(request.getRequestorId()); + final RaftGroupId leaderGroupId = ProtoUtils.toRaftGroupId(request.getRaftGroupId()); + + CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(), leaderId, previous, r); + + assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); + if (!startComplete.get()) { + throw new ServerNotReadyException(getMemberId() + ": The server role is not yet initialized."); + } + assertGroup(leaderId, leaderGroupId); + validateEntries(r.getLeaderTerm(), previous, r.getEntriesList()); + + return appendEntriesAsync(leaderId, request.getCallId(), previous, requestRef); } catch(Exception t) { - LOG.error("{}: Failed appendEntriesAsync {}", getMemberId(), r, t); - throw t; + LOG.error("{}: Failed appendEntries* {}", getMemberId(), toAppendEntriesRequestString(r), t); + throw IOUtils.asIOException(t); } finally { requestRef.release(); } @@ -1540,24 +1551,6 @@ Optional updateLastRpcTime(FollowerState.UpdateType updateType) { } } - private void preAppendEntriesAsync(RaftPeerId leaderId, RaftGroupId leaderGroupId, long leaderTerm, - TermIndex previous, long leaderCommit, boolean initializing, List entries) throws IOException { - CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(), - leaderId, leaderTerm, previous, leaderCommit, initializing, entries); - - assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); - if (!startComplete.get()) { - throw new ServerNotReadyException(getMemberId() + ": The server role is not yet initialized."); - } - assertGroup(leaderId, leaderGroupId); - - try { - validateEntries(leaderTerm, previous, entries); - } catch (IllegalArgumentException e) { - throw new IOException(e); - } - } - private long updateCommitInfoCache() { return commitInfoCache.update(getId(), state.getLog().getLastCommittedIndex()); } @@ -1566,19 +1559,15 @@ ExecutorService getServerExecutor() { return serverExecutor; } - @SuppressWarnings("checkstyle:parameternumber") - private CompletableFuture appendEntriesAsync( - RaftPeerId leaderId, long leaderTerm, TermIndex previous, long leaderCommit, long callId, boolean initializing, - List commitInfos, List entries, - ReferenceCountedObject requestRef) throws IOException { + private CompletableFuture appendEntriesAsync(RaftPeerId leaderId, long callId, + TermIndex previous, ReferenceCountedObject requestRef) throws IOException { + final AppendEntriesRequestProto proto = requestRef.get(); + final List entries = proto.getEntriesList(); final boolean isHeartbeat = entries.isEmpty(); - logAppendEntries(isHeartbeat, - () -> getMemberId() + ": receive appendEntries(" + leaderId + ", " + leaderTerm + ", " - + previous + ", " + leaderCommit + ", " + initializing - + ", commits:" + ProtoUtils.toString(commitInfos) - + ", cId:" + callId - + ", entries: " + LogProtoUtils.toLogEntriesString(entries)); + logAppendEntries(isHeartbeat, () -> getMemberId() + ": appendEntries* " + + toAppendEntriesRequestString(proto)); + final long leaderTerm = proto.getLeaderTerm(); final long currentTerm; final long followerCommit = state.getLog().getLastCommittedIndex(); final Optional followerState; @@ -1586,17 +1575,12 @@ private CompletableFuture appendEntriesAsync( synchronized (this) { // Check life cycle state again to avoid the PAUSING/PAUSED state. assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); - final boolean recognized = state.recognizeLeader(leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); + final boolean recognized = state.recognizeLeader(Op.APPEND_ENTRIES, leaderId, leaderTerm); if (!recognized) { - final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto( + return CompletableFuture.completedFuture(toAppendEntriesReplyProto( leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), - AppendResult.NOT_LEADER, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat); - if (LOG.isDebugEnabled()) { - LOG.debug("{}: Not recognize {} (term={}) as leader, state: {} reply: {}", - getMemberId(), leaderId, leaderTerm, state, ServerStringUtils.toAppendEntriesReplyString(reply)); - } - return CompletableFuture.completedFuture(reply); + AppendResult.NOT_LEADER, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat)); } try { changeToFollowerAndPersistMetadata(leaderTerm, true, "appendEntries"); @@ -1605,7 +1589,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), } state.setLeader(leaderId, "appendEntries"); - if (!initializing && lifeCycle.compareAndTransition(State.STARTING, State.RUNNING)) { + if (!proto.getInitializing() && lifeCycle.compareAndTransition(State.STARTING, State.RUNNING)) { role.startFollowerState(this, Op.APPEND_ENTRIES); } followerState = updateLastRpcTime(FollowerState.UpdateType.APPEND_START); @@ -1617,12 +1601,14 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), // 3. There is a gap between the local log and the entries // In any of these scenarios, we should return an INCONSISTENCY reply // back to leader so that the leader can update this follower's next index. - - AppendEntriesReplyProto inconsistencyReply = checkInconsistentAppendEntries( - leaderId, currentTerm, followerCommit, previous, callId, isHeartbeat, entries); - if (inconsistencyReply != null) { + final long inconsistencyReplyNextIndex = checkInconsistentAppendEntries(previous, entries); + if (inconsistencyReplyNextIndex > RaftLog.INVALID_LOG_INDEX) { + final AppendEntriesReplyProto reply = toAppendEntriesReplyProto( + leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextIndex, + AppendResult.INCONSISTENCY, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat); + LOG.info("{}: appendEntries* reply {}", getMemberId(), toAppendEntriesReplyString(reply)); followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); - return CompletableFuture.completedFuture(inconsistencyReply); + return CompletableFuture.completedFuture(reply); } state.updateConfiguration(entries); @@ -1631,7 +1617,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), final List> futures = entries.isEmpty() ? Collections.emptyList() : state.getLog().append(requestRef.delegate(entries)); - commitInfos.forEach(commitInfoCache::update); + proto.getCommitInfosList().forEach(commitInfoCache::update); CodeInjectionForTesting.execute(LOG_SYNC, getId(), null); if (!isHeartbeat) { @@ -1641,49 +1627,27 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), stateMachine.event().notifySnapshotInstalled(InstallSnapshotResult.SUCCESS, installedIndex, getPeer()); } } - return JavaUtils.allOf(futures).whenCompleteAsync( - (r, t) -> followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)), - serverExecutor - ).thenApply(v -> { - final AppendEntriesReplyProto reply; - synchronized(this) { - final long commitIndex = ServerImplUtils.effectiveCommitIndex(leaderCommit, previous, entries.size()); - state.updateCommitIndex(commitIndex, currentTerm, false); + + final long commitIndex = effectiveCommitIndex(proto.getLeaderCommit(), previous, entries.size()); + final long matchIndex = isHeartbeat? RaftLog.INVALID_LOG_INDEX: entries.get(entries.size() - 1).getIndex(); + return JavaUtils.allOf(futures).whenCompleteAsync((r, t) -> { + followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); + timer.stop(); + }, getServerExecutor()).thenApply(v -> { + final boolean updated = state.updateCommitIndex(commitIndex, currentTerm, false); + if (updated) { updateCommitInfoCache(); - final long n; - final long matchIndex; - if (!isHeartbeat) { - LogEntryProto requestLastEntry = entries.get(entries.size() - 1); - n = requestLastEntry.getIndex() + 1; - matchIndex = requestLastEntry.getIndex(); - } else { - n = state.getLog().getNextIndex(); - matchIndex = RaftLog.INVALID_LOG_INDEX; - } - reply = ServerProtoUtils.toAppendEntriesReplyProto(leaderId, getMemberId(), currentTerm, - state.getLog().getLastCommittedIndex(), n, AppendResult.SUCCESS, callId, matchIndex, isHeartbeat); } - logAppendEntries(isHeartbeat, () -> getMemberId() + ": succeeded to handle AppendEntries. Reply: " - + ServerStringUtils.toAppendEntriesReplyString(reply)); - timer.stop(); // TODO: future never completes exceptionally? + final long nextIndex = isHeartbeat? state.getNextIndex(): matchIndex + 1; + final AppendEntriesReplyProto reply = toAppendEntriesReplyProto(leaderId, getMemberId(), + currentTerm, updated? commitIndex : state.getLog().getLastCommittedIndex(), + nextIndex, AppendResult.SUCCESS, callId, matchIndex, isHeartbeat); + logAppendEntries(isHeartbeat, () -> getMemberId() + + ": appendEntries* reply " + toAppendEntriesReplyString(reply)); return reply; }); } - private AppendEntriesReplyProto checkInconsistentAppendEntries(RaftPeerId leaderId, long currentTerm, - long followerCommit, TermIndex previous, long callId, boolean isHeartbeat, List entries) { - final long replyNextIndex = checkInconsistentAppendEntries(previous, entries); - if (replyNextIndex == -1) { - return null; - } - - final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto( - leaderId, getMemberId(), currentTerm, followerCommit, replyNextIndex, - AppendResult.INCONSISTENCY, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat); - LOG.info("{}: inconsistency entries. Reply:{}", getMemberId(), ServerStringUtils.toAppendEntriesReplyString(reply)); - return reply; - } - private long checkInconsistentAppendEntries(TermIndex previous, List entries) { // Check if a snapshot installation through state machine is in progress. final long installSnapshot = snapshotInstallationHandler.getInProgressInstallSnapshotIndex(); @@ -1714,7 +1678,7 @@ private long checkInconsistentAppendEntries(TermIndex previous, List current || curLeaderId == null) { - // If the request indicates a term that is greater than the current term - // or no leader has been set for the current term, make sure to update - // leader and term later - return true; + if (peerTerm == current && curLeaderId != null && !curLeaderId.equals(peerId)) { + LOG.warn("{}: Failed to recognize {} as leader for {} since current leader is {} (peerTerm = currentTerm = {})", + getMemberId(), peerId, op, curLeaderId, current); + return false; } - return curLeaderId.equals(peerLeaderId); + return true; } static int compareLog(TermIndex lastEntry, TermIndex candidateLastEntry) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 9794314b83..7aae944a43 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -32,6 +32,7 @@ import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.protocol.RaftServerProtocol; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.util.ServerStringUtils; @@ -49,6 +50,8 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.ratis.server.impl.ServerProtoUtils.toInstallSnapshotReplyProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toServerRpcProto; import static org.apache.ratis.server.raftlog.RaftLog.INVALID_LOG_INDEX; class SnapshotInstallationHandler { @@ -142,7 +145,7 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt } // There is a mismatch between configurations on leader and follower. - final InstallSnapshotReplyProto failedReply = ServerProtoUtils.toInstallSnapshotReplyProto( + final InstallSnapshotReplyProto failedReply = toInstallSnapshotReplyProto( leaderId, getMemberId(), state.getCurrentTerm(), InstallSnapshotResult.CONF_MISMATCH); LOG.error("{}: Configuration Mismatch ({}): Leader {} has it set to {} but follower {} has it set to {}", getMemberId(), RaftServerConfigKeys.Log.Appender.INSTALL_SNAPSHOT_ENABLED_KEY, @@ -158,13 +161,11 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest final TermIndex lastIncluded = TermIndex.valueOf(snapshotChunkRequest.getTermIndex()); final long lastIncludedIndex = lastIncluded.getIndex(); synchronized (server) { - final boolean recognized = state.recognizeLeader(leaderId, leaderTerm); + final boolean recognized = state.recognizeLeader(RaftServerProtocol.Op.INSTALL_SNAPSHOT, leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { - final InstallSnapshotReplyProto reply = ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER); - LOG.warn("{}: Failed to recognize leader for installSnapshot chunk.", getMemberId()); - return reply; } server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); state.setLeader(leaderId, "installSnapshot"); @@ -193,7 +194,7 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest if (snapshotChunkRequest.getDone()) { LOG.info("{}: successfully install the entire snapshot-{}", getMemberId(), lastIncludedIndex); } - return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.SUCCESS); } @@ -205,13 +206,11 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( request.getNotification().getFirstAvailableTermIndex()); final long firstAvailableLogIndex = firstAvailableLogTermIndex.getIndex(); synchronized (server) { - final boolean recognized = state.recognizeLeader(leaderId, leaderTerm); + final boolean recognized = state.recognizeLeader("notifyInstallSnapshot", leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { - final InstallSnapshotReplyProto reply = ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.NOT_LEADER); - LOG.warn("{}: Failed to recognize leader for installSnapshot notification.", getMemberId()); - return reply; } server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); state.setLeader(leaderId, "installSnapshot"); @@ -229,7 +228,7 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( inProgressInstallSnapshotIndex.compareAndSet(firstAvailableLogIndex, INVALID_LOG_INDEX); LOG.info("{}: InstallSnapshot notification result: {}, current snapshot index: {}", getMemberId(), InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); - return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); } @@ -307,7 +306,7 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( inProgressInstallSnapshotIndex.set(INVALID_LOG_INDEX); server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_UNAVAILABLE, INVALID_LOG_INDEX, server.getPeer()); - return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_UNAVAILABLE); } @@ -325,7 +324,7 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledIndex, server.getPeer()); installedIndex.set(latestInstalledIndex); - return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledSnapshotTermIndex.getIndex()); } @@ -334,7 +333,7 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( LOG.debug("{}: InstallSnapshot notification result: {}", getMemberId(), InstallSnapshotResult.IN_PROGRESS); } - return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.IN_PROGRESS); } } @@ -342,7 +341,7 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( private RoleInfoProto getRoleInfoProto(RaftPeer leader) { final RoleInfo role = server.getRole(); final Optional fs = role.getFollowerState(); - final ServerRpcProto leaderInfo = ServerProtoUtils.toServerRpcProto(leader, + final ServerRpcProto leaderInfo = toServerRpcProto(leader, fs.map(FollowerState::getLastRpcTime).map(Timestamp::elapsedTimeMs).orElse(0L)); final FollowerInfoProto.Builder followerInfo = FollowerInfoProto.newBuilder() .setLeaderInfo(leaderInfo) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java index 25223c0f4d..284664d012 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java @@ -21,16 +21,19 @@ import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.util.ProtoUtils; +import java.util.List; + /** * This class provides convenient utilities for converting Protocol Buffers messages to strings. * The output strings are for information purpose only. * They are concise and compact compared to the Protocol Buffers implementations of {@link Object#toString()}. - * + *

* The output messages or the output formats may be changed without notice. * Callers of this class should not try to parse the output strings for any purposes. * Instead, they should use the public APIs provided by Protocol Buffers. @@ -42,12 +45,13 @@ public static String toAppendEntriesRequestString(AppendEntriesRequestProto requ if (request == null) { return null; } + final List entries = request.getEntriesList(); return ProtoUtils.toString(request.getServerRequest()) + "-t" + request.getLeaderTerm() + ",previous=" + TermIndex.valueOf(request.getPreviousLog()) + ",leaderCommit=" + request.getLeaderCommit() + ",initializing? " + request.getInitializing() - + ",entries: " + LogProtoUtils.toLogEntriesShortString(request.getEntriesList()); + + "," + (entries.isEmpty()? "HEARTBEAT" : "entries: " + LogProtoUtils.toLogEntriesShortString(entries)); } public static String toAppendEntriesReplyString(AppendEntriesReplyProto reply) { From 16b6536e220ccbc8f9228b4fccfab8eb823ee83e Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 7 Feb 2024 09:20:06 -0800 Subject: [PATCH 35/76] RATIS-2025. Move out assert and proto methods from RaftServerImpl. (#1041) --- .../ratis/server/impl/RaftServerImpl.java | 233 +++++++----------- .../apache/ratis/server/impl/RoleInfo.java | 57 +++++ .../ratis/server/impl/ServerImplUtils.java | 55 ++++- .../ratis/server/impl/ServerProtoUtils.java | 4 + .../impl/SnapshotInstallationHandler.java | 2 +- 5 files changed, 205 insertions(+), 146 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 7390093c33..133cfebdc6 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -17,36 +17,30 @@ */ package org.apache.ratis.server.impl; -import java.io.File; -import java.io.IOException; -import java.nio.file.NoSuchFileException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Function; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.Stream; - import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.metrics.Timekeeper; -import org.apache.ratis.proto.RaftProtos.*; +import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto; import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult; +import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; +import org.apache.ratis.proto.RaftProtos.CommitInfoProto; +import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto; +import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; +import org.apache.ratis.proto.RaftProtos.InstallSnapshotResult; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto.TypeCase; +import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto; +import org.apache.ratis.proto.RaftProtos.RaftPeerRole; +import org.apache.ratis.proto.RaftProtos.RaftRpcRequestProto; +import org.apache.ratis.proto.RaftProtos.ReadIndexReplyProto; +import org.apache.ratis.proto.RaftProtos.ReadIndexRequestProto; +import org.apache.ratis.proto.RaftProtos.ReplicationLevel; +import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; +import org.apache.ratis.proto.RaftProtos.RequestVoteRequestProto; +import org.apache.ratis.proto.RaftProtos.RoleInfoProto; +import org.apache.ratis.proto.RaftProtos.StartLeaderElectionReplyProto; +import org.apache.ratis.proto.RaftProtos.StartLeaderElectionRequestProto; import org.apache.ratis.protocol.ClientInvocationId; import org.apache.ratis.protocol.GroupInfoReply; import org.apache.ratis.protocol.GroupInfoRequest; @@ -64,7 +58,20 @@ import org.apache.ratis.protocol.SetConfigurationRequest; import org.apache.ratis.protocol.SnapshotManagementRequest; import org.apache.ratis.protocol.TransferLeadershipRequest; -import org.apache.ratis.protocol.exceptions.*; +import org.apache.ratis.protocol.exceptions.GroupMismatchException; +import org.apache.ratis.protocol.exceptions.LeaderNotReadyException; +import org.apache.ratis.protocol.exceptions.LeaderSteppingDownException; +import org.apache.ratis.protocol.exceptions.NotLeaderException; +import org.apache.ratis.protocol.exceptions.RaftException; +import org.apache.ratis.protocol.exceptions.ReadException; +import org.apache.ratis.protocol.exceptions.ReadIndexException; +import org.apache.ratis.protocol.exceptions.ReconfigurationInProgressException; +import org.apache.ratis.protocol.exceptions.ResourceUnavailableException; +import org.apache.ratis.protocol.exceptions.ServerNotReadyException; +import org.apache.ratis.protocol.exceptions.SetConfigurationException; +import org.apache.ratis.protocol.exceptions.StaleReadException; +import org.apache.ratis.protocol.exceptions.StateMachineException; +import org.apache.ratis.protocol.exceptions.TransferLeadershipException; import org.apache.ratis.server.DataStreamMap; import org.apache.ratis.server.DivisionInfo; import org.apache.ratis.server.DivisionProperties; @@ -74,7 +81,6 @@ import org.apache.ratis.server.impl.LeaderElection.Phase; import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry; import org.apache.ratis.server.leader.LeaderState; -import org.apache.ratis.server.leader.LogAppender; import org.apache.ratis.server.metrics.LeaderElectionMetrics; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; @@ -85,7 +91,6 @@ import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.server.storage.RaftStorageDirectory; -import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.statemachine.SnapshotInfo; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; @@ -105,13 +110,43 @@ import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; -import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedSupplier; +import java.io.File; +import java.io.IOException; +import java.nio.file.NoSuchFileException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.ratis.server.impl.ServerImplUtils.assertEntries; +import static org.apache.ratis.server.impl.ServerImplUtils.assertGroup; import static org.apache.ratis.server.impl.ServerImplUtils.effectiveCommitIndex; import static org.apache.ratis.server.impl.ServerProtoUtils.toAppendEntriesReplyProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toReadIndexReplyProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toReadIndexRequestProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toRequestVoteReplyProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toStartLeaderElectionReplyProto; import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesReplyString; import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesRequestString; +import static org.apache.ratis.server.util.ServerStringUtils.toRequestVoteReplyString; class RaftServerImpl implements RaftServer.Division, RaftServerProtocol, RaftServerAsynchronousProtocol, @@ -611,46 +646,7 @@ GroupInfoReply getGroupInfo(GroupInfoRequest request) { } RoleInfoProto getRoleInfoProto() { - RaftPeerRole currentRole = role.getCurrentRole(); - RoleInfoProto.Builder roleInfo = RoleInfoProto.newBuilder() - .setSelf(getPeer().getRaftPeerProto()) - .setRole(currentRole) - .setRoleElapsedTimeMs(role.getRoleElapsedTimeMs()); - switch (currentRole) { - case CANDIDATE: - CandidateInfoProto.Builder candidate = CandidateInfoProto.newBuilder() - .setLastLeaderElapsedTimeMs(state.getLastLeaderElapsedTimeMs()); - roleInfo.setCandidateInfo(candidate); - break; - - case LISTENER: - case FOLLOWER: - final Optional fs = role.getFollowerState(); - final ServerRpcProto leaderInfo = ServerProtoUtils.toServerRpcProto( - getRaftConf().getPeer(state.getLeaderId()), - fs.map(FollowerState::getLastRpcTime).map(Timestamp::elapsedTimeMs).orElse(0L)); - // FollowerState can be null while adding a new peer as it is not - // a voting member yet - roleInfo.setFollowerInfo(FollowerInfoProto.newBuilder() - .setLeaderInfo(leaderInfo) - .setOutstandingOp(fs.map(FollowerState::getOutstandingOp).orElse(0))); - break; - - case LEADER: - role.getLeaderState().ifPresent(ls -> { - final LeaderInfoProto.Builder leader = LeaderInfoProto.newBuilder(); - ls.getLogAppenders().map(LogAppender::getFollower).forEach(f -> - leader.addFollowerInfo(ServerProtoUtils.toServerRpcProto( - f.getPeer(), f.getLastRpcResponseTime().elapsedTimeMs()))); - leader.setTerm(ls.getCurrentTerm()); - roleInfo.setLeaderInfo(leader); - }); - break; - - default: - throw new IllegalStateException("incorrect role of server " + currentRole); - } - return roleInfo.build(); + return role.buildRoleInfoProto(this); } synchronized void changeToCandidate(boolean forceStartLeaderElection) { @@ -711,7 +707,7 @@ private CompletableFuture checkLeaderState(RaftClientRequest re */ private CompletableFuture checkLeaderState(RaftClientRequest request, CacheEntry entry) { try { - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); } catch (GroupMismatchException e) { return RetryCacheImpl.failWithException(e, entry); } @@ -760,15 +756,6 @@ void assertLifeCycleState(Set expected) throws ServerNotReadyEx getMemberId() + " is not in " + expected + ": current state is " + c), expected); } - void assertGroup(Object requestorId, RaftGroupId requestorGroupId) throws GroupMismatchException { - final RaftGroupId groupId = getMemberId().getGroupId(); - if (!groupId.equals(requestorGroupId)) { - throw new GroupMismatchException(getMemberId() - + ": The group (" + requestorGroupId + ") of " + requestorId - + " does not match the group (" + groupId + ") of the server " + getId()); - } - } - /** * Append a transaction to the log for processing a client request. * Note that the given request could be different from {@link TransactionContext#getClientRequest()} @@ -1002,8 +989,7 @@ private CompletableFuture sendReadIndexAsync(RaftClientRequ if (leaderId == null) { return JavaUtils.completeExceptionally(new ReadIndexException(getMemberId() + ": Leader is unknown.")); } - final ReadIndexRequestProto request = - ServerProtoUtils.toReadIndexRequestProto(clientRequest, getMemberId(), leaderId); + final ReadIndexRequestProto request = toReadIndexRequestProto(clientRequest, getMemberId(), leaderId); try { return getServerRpc().async().readIndexAsync(request); } catch (IOException e) { @@ -1180,7 +1166,7 @@ CompletableFuture transferLeadershipAsync(TransferLeadershipReq LOG.info("{}: receive transferLeadership {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); synchronized (this) { CompletableFuture reply = checkLeaderState(request); @@ -1221,7 +1207,7 @@ CompletableFuture transferLeadershipAsync(TransferLeadershipReq CompletableFuture takeSnapshotAsync(SnapshotManagementRequest request) throws IOException { LOG.info("{}: takeSnapshotAsync {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); //TODO(liuyaolong): get the gap value from shell command long minGapValue = RaftServerConfigKeys.Snapshot.creationGap(proxy.getProperties()); @@ -1253,7 +1239,7 @@ CompletableFuture leaderElectionManagementAsync(LeaderElectionM throws IOException { LOG.info("{} receive leaderElectionManagement request {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); final LeaderElectionManagementRequest.Pause pause = request.getPause(); if (pause != null) { @@ -1272,7 +1258,7 @@ CompletableFuture leaderElectionManagementAsync(LeaderElectionM CompletableFuture stepDownLeaderAsync(TransferLeadershipRequest request) throws IOException { LOG.info("{} receive stepDown leader request {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); return role.getLeaderState().map(leader -> leader.submitStepDownRequestAsync(request)) .orElseGet(() -> CompletableFuture.completedFuture( @@ -1289,7 +1275,7 @@ public RaftClientReply setConfiguration(SetConfigurationRequest request) throws public CompletableFuture setConfigurationAsync(SetConfigurationRequest request) throws IOException { LOG.info("{}: receive setConfiguration {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); CompletableFuture reply = checkLeaderState(request); if (reply != null) { @@ -1368,15 +1354,13 @@ static List add(RaftPeerRole role, RaftConfigurationImpl conf, SetConf } /** - * check if the remote peer is not included in the current conf - * and should shutdown. should shutdown if all the following stands: - * 1. this is a leader + * The remote peer should shut down if all the following are true. + * 1. this is the current leader * 2. current conf is stable and has been committed - * 3. candidate id is not included in conf - * 4. candidate's last entry's index < conf's index + * 3. candidate is not in the current conf + * 4. candidate last entry index < conf index (the candidate was removed) */ - private boolean shouldSendShutdown(RaftPeerId candidateId, - TermIndex candidateLastEntry) { + private boolean shouldSendShutdown(RaftPeerId candidateId, TermIndex candidateLastEntry) { return getInfo().isLeader() && getRaftConf().isStable() && getState().isConfCommitted() @@ -1403,7 +1387,7 @@ private RequestVoteReplyProto requestVote(Phase phase, LOG.info("{}: receive requestVote({}, {}, {}, {}, {})", getMemberId(), phase, candidateId, candidateGroupId, candidateTerm, candidateLastEntry); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(candidateId, candidateGroupId); + assertGroup(getMemberId(), candidateId, candidateGroupId); boolean shouldShutdown = false; final RequestVoteReplyProto reply; @@ -1430,49 +1414,16 @@ private RequestVoteReplyProto requestVote(Phase phase, } else if(shouldSendShutdown(candidateId, candidateLastEntry)) { shouldShutdown = true; } - reply = ServerProtoUtils.toRequestVoteReplyProto(candidateId, getMemberId(), + reply = toRequestVoteReplyProto(candidateId, getMemberId(), voteGranted, state.getCurrentTerm(), shouldShutdown); if (LOG.isInfoEnabled()) { LOG.info("{} replies to {} vote request: {}. Peer's state: {}", - getMemberId(), phase, ServerStringUtils.toRequestVoteReplyString(reply), state); + getMemberId(), phase, toRequestVoteReplyString(reply), state); } } return reply; } - private void validateEntries(long expectedTerm, TermIndex previous, - List entries) { - if (entries != null && !entries.isEmpty()) { - final long index0 = entries.get(0).getIndex(); - // Check if next entry's index is 1 greater than the snapshotIndex. If yes, then - // we do not have to check for the existence of previous. - if (index0 != state.getSnapshotIndex() + 1) { - if (previous == null || previous.getTerm() == 0) { - Preconditions.assertTrue(index0 == 0, - "Unexpected Index: previous is null but entries[%s].getIndex()=%s", - 0, index0); - } else { - Preconditions.assertTrue(previous.getIndex() == index0 - 1, - "Unexpected Index: previous is %s but entries[%s].getIndex()=%s", - previous, 0, index0); - } - } - - for (int i = 0; i < entries.size(); i++) { - LogEntryProto entry = entries.get(i); - final long t = entry.getTerm(); - Preconditions.assertTrue(expectedTerm >= t, - "Unexpected Term: entries[%s].getTerm()=%s but expectedTerm=%s", - i, t, expectedTerm); - - final long indexi = entry.getIndex(); - Preconditions.assertTrue(indexi == index0 + i, - "Unexpected Index: entries[%s].getIndex()=%s but entries[0].getIndex()=%s", - i, indexi, index0); - } - } - } - @Override public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto r) throws IOException { @@ -1499,8 +1450,8 @@ public CompletableFuture appendEntriesAsync( if (!startComplete.get()) { throw new ServerNotReadyException(getMemberId() + ": The server role is not yet initialized."); } - assertGroup(leaderId, leaderGroupId); - validateEntries(r.getLeaderTerm(), previous, r.getEntriesList()); + assertGroup(getMemberId(), leaderId, leaderGroupId); + assertEntries(r, previous, state); return appendEntriesAsync(leaderId, request.getCallId(), previous, requestRef); } catch(Exception t) { @@ -1519,14 +1470,12 @@ public CompletableFuture readIndexAsync(ReadIndexRequestPro final LeaderStateImpl leader = role.getLeaderState().orElse(null); if (leader == null) { - return CompletableFuture.completedFuture( - ServerProtoUtils.toReadIndexReplyProto(peerId, getMemberId(), false, RaftLog.INVALID_LOG_INDEX)); + return CompletableFuture.completedFuture(toReadIndexReplyProto(peerId, getMemberId())); } return getReadIndex(ClientProtoUtils.toRaftClientRequest(request.getClientRequest()), leader) - .thenApply(index -> ServerProtoUtils.toReadIndexReplyProto(peerId, getMemberId(), true, index)) - .exceptionally(throwable -> - ServerProtoUtils.toReadIndexReplyProto(peerId, getMemberId(), false, RaftLog.INVALID_LOG_INDEX)); + .thenApply(index -> toReadIndexReplyProto(peerId, getMemberId(), true, index)) + .exceptionally(throwable -> toReadIndexReplyProto(peerId, getMemberId())); } static void logAppendEntries(boolean isHeartbeat, Supplier message) { @@ -1731,37 +1680,37 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ if (!request.hasLeaderLastEntry()) { // It should have a leaderLastEntry since there is a placeHolder entry. LOG.warn("{}: leaderLastEntry is missing in {}", getMemberId(), request); - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } final TermIndex leaderLastEntry = TermIndex.valueOf(request.getLeaderLastEntry()); LOG.debug("{}: receive startLeaderElection from {} with lastEntry {}", getMemberId(), leaderId, leaderLastEntry); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(leaderId, leaderGroupId); + assertGroup(getMemberId(), leaderId, leaderGroupId); synchronized (this) { // Check life cycle state again to avoid the PAUSING/PAUSED state. assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); final boolean recognized = state.recognizeLeader("startLeaderElection", leaderId, leaderLastEntry.getTerm()); if (!recognized) { - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } if (!getInfo().isFollower()) { LOG.warn("{} refused StartLeaderElectionRequest from {}, because role is:{}", getMemberId(), leaderId, role.getCurrentRole()); - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } if (ServerState.compareLog(state.getLastEntry(), leaderLastEntry) < 0) { LOG.warn("{} refused StartLeaderElectionRequest from {}, because lastEntry:{} less than leaderEntry:{}", getMemberId(), leaderId, leaderLastEntry, state.getLastEntry()); - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } changeToCandidate(true); - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), true); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), true); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java index fe2bc963b1..5eb01a9d6b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java @@ -18,8 +18,14 @@ package org.apache.ratis.server.impl; +import org.apache.ratis.proto.RaftProtos.CandidateInfoProto; +import org.apache.ratis.proto.RaftProtos.FollowerInfoProto; +import org.apache.ratis.proto.RaftProtos.LeaderInfoProto; import org.apache.ratis.proto.RaftProtos.RaftPeerRole; +import org.apache.ratis.proto.RaftProtos.RoleInfoProto; +import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.server.leader.LogAppender; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.Timestamp; @@ -32,6 +38,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.ratis.server.impl.ServerProtoUtils.toServerRpcProto; + /** * Maintain the Role of a Raft Peer. */ @@ -141,6 +149,55 @@ private T updateAndGet(AtomicReference ref, T current) { return updated; } + RoleInfoProto buildRoleInfoProto(RaftServerImpl server) { + final RaftPeerRole currentRole = getCurrentRole(); + final RoleInfoProto.Builder proto = RoleInfoProto.newBuilder() + .setSelf(server.getPeer().getRaftPeerProto()) + .setRole(currentRole) + .setRoleElapsedTimeMs(getRoleElapsedTimeMs()); + + switch (currentRole) { + case LEADER: + getLeaderState().ifPresent(leader -> { + final LeaderInfoProto.Builder b = LeaderInfoProto.newBuilder() + .setTerm(leader.getCurrentTerm()); + leader.getLogAppenders() + .map(LogAppender::getFollower) + .map(f -> toServerRpcProto(f.getPeer(), f.getLastRpcResponseTime().elapsedTimeMs())) + .forEach(b::addFollowerInfo); + proto.setLeaderInfo(b); + }); + return proto.build(); + + case CANDIDATE: + return proto.setCandidateInfo(CandidateInfoProto.newBuilder() + .setLastLeaderElapsedTimeMs(server.getState().getLastLeaderElapsedTimeMs())) + .build(); + + case LISTENER: + case FOLLOWER: + // FollowerState can be null while adding a new peer as it is not a voting member yet + final FollowerState follower = getFollowerState().orElse(null); + final long rpcElapsed; + final int outstandingOp; + if (follower != null) { + rpcElapsed = follower.getLastRpcTime().elapsedTimeMs(); + outstandingOp = follower.getOutstandingOp(); + } else { + rpcElapsed = 0; + outstandingOp = 0; + } + final RaftPeer leader = server.getRaftConf().getPeer(server.getState().getLeaderId()); + return proto.setFollowerInfo(FollowerInfoProto.newBuilder() + .setLeaderInfo(toServerRpcProto(leader, rpcElapsed)) + .setOutstandingOp(outstandingOp)) + .build(); + + default: + throw new IllegalStateException("Unexpected role " + currentRole); + } + } + @Override public String toString() { return String.format("%9s", role); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index e4fe8f232f..e26c6e0ab1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -19,9 +19,15 @@ import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.protocol.RaftGroup; +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.exceptions.GroupMismatchException; import org.apache.ratis.server.RaftConfiguration; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.protocol.TermIndex; @@ -35,7 +41,6 @@ import java.io.IOException; import java.util.List; -import java.util.Optional; import java.util.concurrent.TimeUnit; /** Server utilities for internal use. */ @@ -88,7 +93,51 @@ public static RaftConfiguration newRaftConfiguration(List conf, List entries = proto.getEntriesList(); + if (entries != null && !entries.isEmpty()) { + final long index0 = entries.get(0).getIndex(); + // Check if next entry's index is 1 greater than the snapshotIndex. If yes, then + // we do not have to check for the existence of previous. + if (index0 != state.getSnapshotIndex() + 1) { + final long expected = previous == null || previous.getTerm() == 0 ? 0 : previous.getIndex() + 1; + Preconditions.assertTrue(index0 == expected, + "Unexpected Index: previous is %s but entries[%s].getIndex() == %s != %s", + previous, 0, index0, expected); + } + + final long leaderTerm = proto.getLeaderTerm(); + for (int i = 0; i < entries.size(); i++) { + final LogEntryProto entry = entries.get(i); + final long entryTerm = entry.getTerm(); + Preconditions.assertTrue(entryTerm <= leaderTerm , + "Unexpected Term: entries[%s].getTerm() == %s > leaderTerm == %s", + i, entryTerm, leaderTerm); + + final long indexI = entry.getIndex(); + final long expected = index0 + i; + Preconditions.assertTrue(indexI == expected, + "Unexpected Index: entries[0].getIndex() == %s but entries[%s].getIndex() == %s != %s", + index0, i, indexI, expected); + } + } } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java index f2be8c61c6..e35cb23867 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java @@ -126,6 +126,10 @@ static ReadIndexReplyProto toReadIndexReplyProto( .build(); } + static ReadIndexReplyProto toReadIndexReplyProto(RaftPeerId requestorId, RaftGroupMemberId replyId) { + return toReadIndexReplyProto(requestorId, replyId, false, RaftLog.INVALID_LOG_INDEX); + } + @SuppressWarnings("parameternumber") static AppendEntriesReplyProto toAppendEntriesReplyProto( RaftPeerId requestorId, RaftGroupMemberId replyId, long term, diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 7aae944a43..3e5ac2b671 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -113,7 +113,7 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt CodeInjectionForTesting.execute(RaftServerImpl.INSTALL_SNAPSHOT, server.getId(), leaderId, request); server.assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); - server.assertGroup(leaderId, leaderGroupId); + ServerImplUtils.assertGroup(getMemberId(), leaderId, leaderGroupId); InstallSnapshotReplyProto reply = null; // Check if install snapshot from Leader is enabled From 67b76509d1d5b0037b278f1808bb0f5ecd7aceed Mon Sep 17 00:00:00 2001 From: Nandakumar Vadivelu Date: Tue, 20 Feb 2024 22:41:53 +0530 Subject: [PATCH 36/76] RATIS-2021. Migrate ratis-test tests to Junit 5 - Part 2. (#1043) --- .../java/org/apache/ratis/RaftBasicTests.java | 41 ++--- .../ratis/TestRaftServerNoLeaderTimeout.java | 20 +-- .../TestRaftServerSlownessDetection.java | 27 ++- .../apache/ratis/netty/TestRaftWithNetty.java | 2 +- .../ratis/netty/TestTlsConfWithNetty.java | 6 +- .../apache/ratis/protocol/TestRaftGroup.java | 16 +- .../org/apache/ratis/protocol/TestRaftId.java | 28 ++- .../ratis/protocol/TestRoutingTable.java | 14 +- .../retry/TestExceptionDependentRetry.java | 33 ++-- .../retry/TestExponentialBackoffRetry.java | 14 +- .../retry/TestMultipleLinearRandomRetry.java | 18 +- .../apache/ratis/retry/TestRetryPolicy.java | 40 ++--- .../ratis/security/SecurityTestUtils.java | 12 +- .../ratis/server/ServerBuilderTest.java | 8 +- .../ratis/server/ServerRestartTests.java | 38 ++-- .../server/TestRaftServerConfigKeys.java | 16 +- .../server/impl/TestPeerConfiguration.java | 6 +- .../server/impl/TestRaftConfiguration.java | 28 +-- .../ratis/server/impl/TestRaftServerJmx.java | 19 +- .../server/raftlog/TestRaftLogIndex.java | 12 +- .../server/raftlog/TestRaftLogMetrics.java | 44 ++--- .../raftlog/memory/MemoryRaftLogTest.java | 4 +- .../segmented/TestBufferedWriteChannel.java | 15 +- .../raftlog/segmented/TestCacheEviction.java | 58 +++--- .../raftlog/segmented/TestLogSegment.java | 80 ++++----- .../segmented/TestRaftLogReadWrite.java | 36 ++-- .../segmented/TestSegmentedRaftLog.java | 167 ++++++++++-------- .../segmented/TestSegmentedRaftLogCache.java | 106 +++++------ .../ratis/server/storage/TestRaftStorage.java | 86 ++++----- .../server/storage/TestStorageImplUtils.java | 22 +-- .../sh/ElectionCommandIntegrationTest.java | 44 ++--- .../cli/sh/GroupCommandIntegrationTest.java | 12 +- .../cli/sh/PeerCommandIntegrationTest.java | 16 +- .../sh/SnapshotCommandIntegrationTest.java | 18 +- .../ratis/shell/cli/sh/TestRatisShell.java | 12 +- .../ratis/statemachine/TestStateMachine.java | 14 +- .../ratis/util/TestDataBlockingQueue.java | 33 ++-- .../org/apache/ratis/util/TestDataQueue.java | 73 ++++---- .../org/apache/ratis/util/TestExitUtils.java | 31 ++-- .../org/apache/ratis/util/TestLifeCycle.java | 19 +- .../org/apache/ratis/util/TestMinMax.java | 16 +- .../apache/ratis/util/TestPeerProxyMap.java | 15 +- .../apache/ratis/util/TestPreconditions.java | 9 +- .../util/TestReferenceCountedObject.java | 45 ++--- .../ratis/util/TestResourceSemaphore.java | 14 +- .../apache/ratis/util/TestTimeDuration.java | 60 ++++--- .../ratis/util/TestTimeoutScheduler.java | 134 +++++++------- .../util/TestTraditionalBinaryPrefix.java | 20 ++- 48 files changed, 832 insertions(+), 769 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index 4ff9681f0a..13ee08c83b 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -42,8 +42,9 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Assumptions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -152,12 +153,12 @@ static void runTestBasicAppendEntries( }); } else { final RaftClientReply reply = client.io().send(message); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } if (async) { f.join(); - Assert.assertEquals(messages.length, asyncReplyCount.get()); + Assertions.assertEquals(messages.length, asyncReplyCount.get()); } } Thread.sleep(cluster.getTimeoutMax().toIntExact(TimeUnit.MILLISECONDS) + 100); @@ -194,7 +195,7 @@ void runTestOldLeaderCommit(CLUSTER cluster) throws Exception { Thread.sleep(cluster.getTimeoutMax().toLong(TimeUnit.MILLISECONDS) + 100); for (RaftServer.Division followerToSendLog : followersToSendLog) { RaftLog followerLog = followerToSendLog.getRaftLog(); - Assert.assertTrue(RaftTestUtil.logEntriesContains(followerLog, messages)); + Assertions.assertTrue(RaftTestUtil.logEntriesContains(followerLog, messages)); } LOG.info(String.format("killing old leader: %s", leaderId.toString())); @@ -212,7 +213,7 @@ void runTestOldLeaderCommit(CLUSTER cluster) throws Exception { Set followersToSendLogIds = followersToSendLog.stream().map(f -> f.getId()).collect(Collectors.toSet()); - Assert.assertTrue(followersToSendLogIds.contains(newLeaderId)); + Assertions.assertTrue(followersToSendLogIds.contains(newLeaderId)); cluster.getServerAliveStream() .map(RaftServer.Division::getRaftLog) @@ -234,8 +235,8 @@ void runTestOldLeaderNotCommit(CLUSTER cluster) throws Exception { cluster.killServer(followers.get(i).getId()); } } catch (IndexOutOfBoundsException e) { - throw new org.junit.AssumptionViolatedException("The assumption is follower.size() = NUM_SERVERS - 1, " - + "actual NUM_SERVERS is " + NUM_SERVERS + ", and actual follower.size() is " + followers.size(), e); + Assumptions.abort("The assumption is follower.size() = NUM_SERVERS - 1, " + + "actual NUM_SERVERS is " + NUM_SERVERS + ", and actual follower.size() is " + followers.size()); } SimpleMessage[] messages = SimpleMessage.create(1); @@ -293,7 +294,7 @@ public void run() { if (!useAsync) { final RaftClientReply reply = client.io().send(messages[step.getAndIncrement()]); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } else { final CompletableFuture replyFuture = client.async().send(messages[i]); @@ -305,13 +306,13 @@ public void run() { if (step.incrementAndGet() == messages.length) { f.complete(null); } - Assert.assertTrue(r.isSuccess()); + Assertions.assertTrue(r.isSuccess()); }); } } if (useAsync) { f.join(); - Assert.assertTrue(step.get() == messages.length); + Assertions.assertEquals(step.get(), messages.length); } } catch(Exception t) { if (exceptionInClientThread.compareAndSet(null, t)) { @@ -393,7 +394,7 @@ public void run() { } final int n = clients.stream().mapToInt(c -> c.step.get()).sum(); - Assert.assertTrue(n >= lastStep.get()); + Assertions.assertTrue(n >= lastStep.get()); if (n - lastStep.get() < 50 * numClients) { // Change leader at least 50 steps. Thread.sleep(10); @@ -447,7 +448,7 @@ public static void testRequestTimeout(boolean async, MiniRaftCluster cluster, Lo // The duration for which the client waits should be more than the retryCacheExpiryDuration. final TimeDuration duration = startTime.elapsedTime(); TimeDuration retryCacheExpiryDuration = RaftServerConfigKeys.RetryCache.expiryTime(cluster.getProperties()); - Assert.assertTrue(duration.compareTo(retryCacheExpiryDuration) >= 0); + Assertions.assertTrue(duration.compareTo(retryCacheExpiryDuration) >= 0); } } @@ -479,10 +480,10 @@ static void runTestStateMachineMetrics(boolean async, MiniRaftCluster cluster) t long smAppliedIndexAfter = (Long) smAppliedIndexGauge.getValue(); checkFollowerCommitLagsLeader(cluster); - Assert.assertTrue("StateMachine Applied Index not incremented", - appliedIndexAfter > appliedIndexBefore); - Assert.assertTrue("StateMachine Apply completed Index not incremented", - smAppliedIndexAfter > smAppliedIndexBefore); + Assertions.assertTrue(appliedIndexAfter > appliedIndexBefore, + "StateMachine Applied Index not incremented"); + Assertions.assertTrue(smAppliedIndexAfter > smAppliedIndexBefore, + "StateMachine Apply completed Index not incremented"); } } @@ -495,12 +496,12 @@ private static void checkFollowerCommitLagsLeader(MiniRaftCluster cluster) { for (RaftServer.Division f : followers) { final RaftGroupMemberId follower = f.getMemberId(); Gauge followerCommitGauge = ServerMetricsTestUtils.getPeerCommitIndexGauge(leader, follower.getPeerId()); - Assert.assertTrue((Long)leaderCommitGauge.getValue() >= + Assertions.assertTrue((Long)leaderCommitGauge.getValue() >= (Long)followerCommitGauge.getValue()); Gauge followerMetric = ServerMetricsTestUtils.getPeerCommitIndexGauge(follower, follower.getPeerId()); System.out.println(followerCommitGauge.getValue()); System.out.println(followerMetric.getValue()); - Assert.assertTrue((Long)followerCommitGauge.getValue() <= (Long)followerMetric.getValue()); + Assertions.assertTrue((Long)followerCommitGauge.getValue() <= (Long)followerMetric.getValue()); } } @@ -511,7 +512,7 @@ private static Gauge getStatemachineGaugeWithName(RaftServer.Division server, St RATIS_STATEMACHINE_METRICS, RATIS_STATEMACHINE_METRICS_DESC); Optional metricRegistry = MetricRegistries.global().get(info); - Assert.assertTrue(metricRegistry.isPresent()); + Assertions.assertTrue(metricRegistry.isPresent()); return ServerMetricsTestUtils.getGaugeWithName(gaugeName, metricRegistry::get); } diff --git a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java index 7f4da0e0fb..c10dc105a5 100644 --- a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java +++ b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java @@ -28,10 +28,10 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.IOException; @@ -60,13 +60,13 @@ private static RaftProperties getProperties() { return properties; } - @Before + @BeforeEach public void setup() throws IOException { - Assert.assertNull(cluster.getLeader()); + Assertions.assertNull(cluster.getLeader()); cluster.start(); } - @After + @AfterEach public void tearDown() { if (cluster != null) { cluster.shutdown(); @@ -91,10 +91,10 @@ public void testLeaderElectionDetection() throws Exception { RaftProtos.RoleInfoProto roleInfoProto = SimpleStateMachine4Testing.get(healthyFollower).getLeaderElectionTimeoutInfo(); - Assert.assertNotNull(roleInfoProto); + Assertions.assertNotNull(roleInfoProto); - Assert.assertEquals(roleInfoProto.getRole(), RaftProtos.RaftPeerRole.CANDIDATE); + Assertions.assertEquals(roleInfoProto.getRole(), RaftProtos.RaftPeerRole.CANDIDATE); final long noLeaderTimeoutMs = noLeaderTimeout.toLong(TimeUnit.MILLISECONDS); - Assert.assertTrue(roleInfoProto.getCandidateInfo().getLastLeaderElapsedTimeMs() > noLeaderTimeoutMs); + Assertions.assertTrue(roleInfoProto.getCandidateInfo().getLastLeaderElapsedTimeMs() > noLeaderTimeoutMs); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java index 0ea0fe9cef..b150a1aa05 100644 --- a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java +++ b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java @@ -30,12 +30,6 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; - import java.io.IOException; import java.util.List; @@ -43,13 +37,18 @@ import java.util.concurrent.TimeUnit; import org.apache.ratis.thirdparty.com.codahale.metrics.Gauge; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; /** * Test Raft Server Slowness detection and notification to Leader's statemachine. */ //TODO: fix StateMachine.notifySlowness(..); see RATIS-370 -@Ignore +@Disabled public class TestRaftServerSlownessDetection extends BaseTest { static { Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); @@ -70,13 +69,13 @@ public RaftProperties getProperties() { return properties; } - @Before + @BeforeEach public void setup() throws IOException { - Assert.assertNull(cluster.getLeader()); + Assertions.assertNull(cluster.getLeader()); cluster.start(); } - @After + @AfterEach public void tearDown() { if (cluster != null) { cluster.shutdown(); @@ -107,23 +106,23 @@ public void testSlownessDetection() throws Exception { Thread.sleep( slownessTimeout * 2); long followerHeartBeatElapsedMetricNew = (long) metric.getValue(); - Assert.assertTrue(followerHeartBeatElapsedMetricNew > followerHeartBeatElapsedMetric); + Assertions.assertTrue(followerHeartBeatElapsedMetricNew > followerHeartBeatElapsedMetric); // Followers should not get any failed not notification for (RaftServer.Division followerServer : cluster.getFollowers()) { - Assert.assertNull(SimpleStateMachine4Testing.get(followerServer).getSlownessInfo()); + Assertions.assertNull(SimpleStateMachine4Testing.get(followerServer).getSlownessInfo()); } // the leader should get notification that the follower has failed now RaftProtos.RoleInfoProto roleInfoProto = SimpleStateMachine4Testing.get(cluster.getLeader()).getSlownessInfo(); - Assert.assertNotNull(roleInfoProto); + Assertions.assertNotNull(roleInfoProto); List followers = roleInfoProto.getLeaderInfo().getFollowerInfoList(); //Assert that the node shutdown is lagging behind for (RaftProtos.ServerRpcProto serverProto : followers) { if (RaftPeerId.valueOf(serverProto.getId().getId()).equals(failedFollower.getId())) { - Assert.assertTrue(serverProto.getLastRpcElapsedTimeMs() > slownessTimeout); + Assertions.assertTrue(serverProto.getLastRpcElapsedTimeMs() > slownessTimeout); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java index 28815d76ce..55b9c37286 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java @@ -19,7 +19,7 @@ import org.apache.ratis.RaftBasicTests; import org.apache.ratis.server.impl.BlockRequestHandlingInjection; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestRaftWithNetty extends RaftBasicTests diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java index db967e391f..77910b2489 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java @@ -38,8 +38,8 @@ import org.apache.ratis.thirdparty.io.netty.handler.logging.LoggingHandler; import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.util.JavaUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -112,7 +112,7 @@ static void runTest(int port, TlsConf serverSslConf, TlsConf clientSslConf) thro final CompletableFuture future = replyFutures.get(i); final String reply = future.get(3, TimeUnit.SECONDS); LOG.info(reply); - Assert.assertEquals(NettyTestServer.toReply(words[i]), reply); + Assertions.assertEquals(NettyTestServer.toReply(words[i]), reply); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftGroup.java b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftGroup.java index 5267b22385..f4a88e40fe 100644 --- a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftGroup.java +++ b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftGroup.java @@ -18,27 +18,25 @@ package org.apache.ratis.protocol; import org.apache.ratis.BaseTest; -import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.LinkedList; import java.util.List; import java.util.UUID; +@Timeout(value = 1) public class TestRaftGroup extends BaseTest { - @Override - public int getGlobalTimeoutSeconds() { - return 1; - } - @Test(expected = IllegalStateException.class) + @Test public void testDuplicatePeerId() throws Exception { UUID groupId = UUID.fromString("02511d47-d67c-49a3-9011-abb3109a44c1"); List peers = new LinkedList<>(); peers.add(RaftPeer.newBuilder().setId("n0").build()); peers.add(RaftPeer.newBuilder().setId("n0").build()); - RaftGroup.valueOf(RaftGroupId.valueOf(groupId), peers); + Assertions.assertThrows(IllegalStateException.class, + () -> RaftGroup.valueOf(RaftGroupId.valueOf(groupId), peers)); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java index 6610b3d049..b0e31ce72e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java +++ b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java @@ -19,16 +19,14 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.UUID; +@Timeout(value = 1) public class TestRaftId extends BaseTest { - @Override - public int getGlobalTimeoutSeconds() { - return 1; - } @Test public void testRaftId() { @@ -39,35 +37,35 @@ public void testRaftId() { static void assertRaftId(UUID original, ByteString expected) { final ByteString bytes = RaftId.toByteString(original); if (expected != null) { - Assert.assertEquals(expected, bytes); + Assertions.assertEquals(expected, bytes); } final UUID computed = RaftId.toUuid(bytes); - Assert.assertEquals(original, computed); - Assert.assertEquals(bytes, RaftId.toByteString(computed)); + Assertions.assertEquals(original, computed); + Assertions.assertEquals(bytes, RaftId.toByteString(computed)); } @Test public void testClientId() { final ClientId id = ClientId.randomId(); final ByteString bytes = id.toByteString(); - Assert.assertEquals(bytes, id.toByteString()); - Assert.assertEquals(id, ClientId.valueOf(bytes)); + Assertions.assertEquals(bytes, id.toByteString()); + Assertions.assertEquals(id, ClientId.valueOf(bytes)); } @Test public void testRaftGroupId() { final RaftGroupId id = RaftGroupId.randomId(); final ByteString bytes = id.toByteString(); - Assert.assertEquals(bytes, id.toByteString()); - Assert.assertEquals(id, RaftGroupId.valueOf(bytes)); + Assertions.assertEquals(bytes, id.toByteString()); + Assertions.assertEquals(id, RaftGroupId.valueOf(bytes)); } @Test public void testRaftPeerId() { final RaftPeerId id = RaftPeerId.valueOf("abc"); final ByteString bytes = id.toByteString(); - Assert.assertEquals(bytes, id.toByteString()); - Assert.assertEquals(id, RaftPeerId.valueOf(bytes)); + Assertions.assertEquals(bytes, id.toByteString()); + Assertions.assertEquals(id, RaftPeerId.valueOf(bytes)); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRoutingTable.java b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRoutingTable.java index 58bdf07fac..f3e08ece7a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRoutingTable.java +++ b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRoutingTable.java @@ -18,14 +18,12 @@ package org.apache.ratis.protocol; import org.apache.ratis.BaseTest; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +@Timeout(value = 1) public class TestRoutingTable extends BaseTest { - @Override - public int getGlobalTimeoutSeconds() { - return 1; - } private final RaftPeerId[] peers = new RaftPeerId[10]; @@ -69,7 +67,7 @@ RoutingTable newRoutingTable(int... peerIndices) { } void testFailureCase(String name, int... peerIndices) { - Assert.assertEquals(0, peerIndices.length % 2); + Assertions.assertEquals(0, peerIndices.length % 2); testFailureCase(name + ": " + toString(peerIndices), () -> newRoutingTable(peerIndices), @@ -77,7 +75,7 @@ void testFailureCase(String name, int... peerIndices) { } String toString(int... peerIndices) { - Assert.assertEquals(0, peerIndices.length % 2); + Assertions.assertEquals(0, peerIndices.length % 2); if (peerIndices.length == 0) { return ""; } diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java index 36e6dfbccf..560c0f9fdd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java @@ -21,7 +21,6 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.RaftTestUtil; -import org.apache.ratis.client.retry.ClientRetryEvent; import org.apache.ratis.client.RaftClient; import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.conf.RaftProperties; @@ -33,16 +32,14 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static org.junit.Assert.fail; - /** * Class to test {@link ExceptionDependentRetry}. */ @@ -99,9 +96,9 @@ public void testExceptionDependentRetryFailureWithExceptionDuplicate() { builder.setExceptionToPolicy(IOException.class, RetryPolicies.retryUpToMaximumCountWithFixedSleep(1, TimeDuration.valueOf(1, TimeUnit.SECONDS))); - fail("testExceptionDependentRetryFailure failed"); + Assertions.fail("testExceptionDependentRetryFailure failed"); } catch (Exception ex) { - Assert.assertEquals(IllegalStateException.class, ex.getClass()); + Assertions.assertEquals(IllegalStateException.class, ex.getClass()); } } @@ -115,9 +112,9 @@ public void testExceptionDependentRetryFailureWithExceptionMappedToNull() { RetryPolicies.retryUpToMaximumCountWithFixedSleep(1, TimeDuration.valueOf(1, TimeUnit.SECONDS))); builder.setExceptionToPolicy(IOException.class, null); - fail("testExceptionDependentRetryFailure failed"); + Assertions.fail("testExceptionDependentRetryFailure failed"); } catch (Exception ex) { - Assert.assertEquals(IllegalStateException.class, ex.getClass()); + Assertions.assertEquals(IllegalStateException.class, ex.getClass()); } } @@ -131,9 +128,9 @@ public void testExceptionDependentRetryFailureWithNoDefault() { RetryPolicies.retryUpToMaximumCountWithFixedSleep(1, TimeDuration.valueOf(1, TimeUnit.SECONDS))); builder.build(); - fail("testExceptionDependentRetryFailureWithNoDefault failed"); + Assertions.fail("testExceptionDependentRetryFailureWithNoDefault failed"); } catch (Exception ex) { - Assert.assertEquals(IllegalStateException.class, ex.getClass()); + Assertions.assertEquals(IllegalStateException.class, ex.getClass()); } try { @@ -143,9 +140,9 @@ public void testExceptionDependentRetryFailureWithNoDefault() { RetryPolicies.retryUpToMaximumCountWithFixedSleep(1, TimeDuration.valueOf(1, TimeUnit.SECONDS))); builder.setDefaultPolicy(null); - fail("testExceptionDependentRetryFailureWithNoDefault failed"); + Assertions.fail("testExceptionDependentRetryFailureWithNoDefault failed"); } catch (Exception ex) { - Assert.assertEquals(IllegalStateException.class, ex.getClass()); + Assertions.assertEquals(IllegalStateException.class, ex.getClass()); } } @@ -157,11 +154,11 @@ private void testException(int retries, int maxAttempts, .handleAttemptFailure(TestRetryPolicy.newClientRetryEvent(i, null, exception)); final boolean expected = i < retries && i < maxAttempts; - Assert.assertEquals(expected, action.shouldRetry()); + Assertions.assertEquals(expected, action.shouldRetry()); if (expected) { - Assert.assertEquals(sleepTime, action.getSleepTime().getDuration()); + Assertions.assertEquals(sleepTime, action.getSleepTime().getDuration()); } else { - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } } } @@ -195,11 +192,11 @@ void runTestExceptionRetryAttempts(MiniRaftClusterWithGrpc cluster) throws Excep SimpleStateMachine4Testing.get(leader).blockWriteStateMachineData(); client.async().send(new RaftTestUtil.SimpleMessage("2")).get(); - Assert.fail("Test should have failed."); + Assertions.fail("Test should have failed."); } catch (ExecutionException e) { RaftRetryFailureException rrfe = (RaftRetryFailureException) e.getCause(); final int expectedCount = 1 + retryCount; // new request attempt + retry attempts - Assert.assertEquals(expectedCount, rrfe.getAttemptCount()); + Assertions.assertEquals(expectedCount, rrfe.getAttemptCount()); } finally { SimpleStateMachine4Testing.get(leader).unblockWriteStateMachineData(); cluster.shutdown(); diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestExponentialBackoffRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestExponentialBackoffRetry.java index 74f1807640..c4d1d3a9c0 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestExponentialBackoffRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestExponentialBackoffRetry.java @@ -19,8 +19,8 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.concurrent.TimeUnit; @@ -36,12 +36,12 @@ public void testExponentialBackoffRetry() { // Test maxAttempts ExponentialBackoffRetry retryPolicy = createPolicy(baseSleep, null, 1); - Assert.assertFalse(retryPolicy.handleAttemptFailure(() -> 1).shouldRetry()); + Assertions.assertFalse(retryPolicy.handleAttemptFailure(() -> 1).shouldRetry()); try { // baseSleep should not be null createPolicy(null, null, 1); - Assert.fail("Policy creation should have failed"); + Assertions.fail("Policy creation should have failed"); } catch (Exception e) { } @@ -67,9 +67,9 @@ private void assertSleep(ExponentialBackoffRetry retryPolicy, // sleep time with randomness added long randomizedDuration = action.getSleepTime().toLong(TimeUnit.MILLISECONDS); - Assert.assertTrue(action.shouldRetry()); - Assert.assertTrue(randomizedDuration >= d * 0.5); - Assert.assertTrue(randomizedDuration <= d * 1.5); + Assertions.assertTrue(action.shouldRetry()); + Assertions.assertTrue(randomizedDuration >= d * 0.5); + Assertions.assertTrue(randomizedDuration <= d * 1.5); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java index 09ae55f33d..eb4b7967cd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java @@ -19,8 +19,8 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestMultipleLinearRandomRetry extends BaseTest { @Override @@ -49,12 +49,12 @@ public void testParseCommaSeparated() { private static void assertIllegalInput(String input) { final MultipleLinearRandomRetry computed = MultipleLinearRandomRetry.parseCommaSeparated(input); - Assert.assertNull(computed); + Assertions.assertNull(computed); } private static MultipleLinearRandomRetry assertLegalInput(String expected, String input) { final MultipleLinearRandomRetry computed = MultipleLinearRandomRetry.parseCommaSeparated(input); - Assert.assertNotNull(computed); - Assert.assertTrue(computed.toString().endsWith(expected)); + Assertions.assertNotNull(computed); + Assertions.assertTrue(computed.toString().endsWith(expected)); return computed; } @@ -69,18 +69,18 @@ public void testMultipleLinearRandomRetry() { for (int j = 1; j <= counts[i]; j++) { final int attempt = ++k; final RetryPolicy.Action action = r.handleAttemptFailure(() -> attempt); - Assert.assertTrue(action.shouldRetry()); + Assertions.assertTrue(action.shouldRetry()); final TimeDuration randomized = action.getSleepTime(); final TimeDuration expected = times[i].to(randomized.getUnit()); final long d = expected.getDuration(); LOG.info("times[{},{}] = {}, randomized={}", i, j, times[i], randomized); - Assert.assertTrue(randomized.getDuration() >= d*0.5); - Assert.assertTrue(randomized.getDuration() < (d*1.5 + precision)); + Assertions.assertTrue(randomized.getDuration() >= d*0.5); + Assertions.assertTrue(randomized.getDuration() < (d*1.5 + precision)); } } final int attempt = ++k; final RetryPolicy.Action action = r.handleAttemptFailure(() -> attempt); - Assert.assertFalse(action.shouldRetry()); + Assertions.assertFalse(action.shouldRetry()); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java index 1b9536b4b6..cee8ee3385 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java @@ -34,8 +34,8 @@ import org.apache.ratis.protocol.exceptions.ResourceUnavailableException; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.ArrayList; @@ -62,11 +62,11 @@ public void testRetryMultipleTimesWithFixedSleep() { final RetryPolicy.Action action = policy.handleAttemptFailure(event); final boolean expected = i < n; - Assert.assertEquals(expected, action.shouldRetry()); + Assertions.assertEquals(expected, action.shouldRetry()); if (expected) { - Assert.assertEquals(sleepTime, action.getSleepTime()); + Assertions.assertEquals(sleepTime, action.getSleepTime()); } else { - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } } } @@ -97,33 +97,33 @@ public void testRequestTypeDependentRetry() { final RetryPolicy.Action action = policy.handleAttemptFailure(event); final boolean expected = i < n; - Assert.assertEquals(expected, action.shouldRetry()); + Assertions.assertEquals(expected, action.shouldRetry()); if (expected) { - Assert.assertEquals(writeSleep, action.getSleepTime()); + Assertions.assertEquals(writeSleep, action.getSleepTime()); } else { - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } } { //read and stale read are using default final ClientRetryEvent event = newClientRetryEvent(i, readRequest, null); final RetryPolicy.Action action = policy.handleAttemptFailure(event); - Assert.assertTrue(action.shouldRetry()); - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertTrue(action.shouldRetry()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } { final ClientRetryEvent event = newClientRetryEvent(i, staleReadRequest, null); final RetryPolicy.Action action = policy.handleAttemptFailure(event); - Assert.assertTrue(action.shouldRetry()); - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertTrue(action.shouldRetry()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } { //watch has no retry final ClientRetryEvent event = newClientRetryEvent(i, watchRequest, null); final RetryPolicy.Action action = policy.handleAttemptFailure(event); - Assert.assertFalse(action.shouldRetry()); - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertFalse(action.shouldRetry()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } } @@ -155,15 +155,15 @@ public RaftClientRequest newRequestImpl() { for (RaftClientRequest request : requests) { final ClientRetryEvent event = pending.newClientRetryEvent(request, new Exception()); final RetryPolicy.Action action = policy.handleAttemptFailure(event); - Assert.assertTrue(action.shouldRetry()); - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertTrue(action.shouldRetry()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } timeout.sleep(); for (RaftClientRequest request : requests) { final ClientRetryEvent event = pending.newClientRetryEvent(request, new Exception()); final RetryPolicy.Action action = policy.handleAttemptFailure(event); - Assert.assertFalse(action.shouldRetry()); + Assertions.assertFalse(action.shouldRetry()); } } @@ -227,11 +227,11 @@ private void checkEvent(int exceptionAttemptCount, RetryPolicy retryPolicy, Raft final RetryPolicy.Action action = retryPolicy.handleAttemptFailure(event); final boolean expected = exceptionAttemptCount < exceptionPolicyPair.retries; - Assert.assertEquals(expected, action.shouldRetry()); + Assertions.assertEquals(expected, action.shouldRetry()); if (expected) { - Assert.assertEquals(exceptionPolicyPair.sleepTime, action.getSleepTime().getDuration()); + Assertions.assertEquals(exceptionPolicyPair.sleepTime, action.getSleepTime().getDuration()); } else { - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java index d6222b2271..c390f10f4a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java @@ -23,7 +23,7 @@ import org.apache.ratis.util.FileUtils; import org.bouncycastle.util.io.pem.PemObject; import org.bouncycastle.util.io.pem.PemReader; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -100,7 +100,7 @@ static PrivateKey getPrivateKey(String keyPath) { KeyFactory keyFactory = KeyFactory.getInstance("RSA"); return keyFactory.generatePrivate(privKeySpec); } catch (Exception e) { - Assert.fail("Failed to get private key from " + keyPath + ". Error: " + + Assertions.fail("Failed to get private key from " + keyPath + ". Error: " + e.getMessage()); } return null; @@ -116,7 +116,7 @@ static X509Certificate[] getCertificate(String certPath) { } return certificate; } catch (Exception e) { - Assert.fail("Failed to get certificate from " + certPath + ". Error: " + + Assertions.fail("Failed to get certificate from " + certPath + ". Error: " + e.getMessage()); } return null; @@ -133,7 +133,7 @@ static KeyStore getServerKeyStore() { keyStore.setKeyEntry("ratis-server-key", privateKey, new char[0], certificate); return keyStore; } catch (Exception e) { - Assert.fail("Failed to get sever key store " + e.getMessage()); + Assertions.fail("Failed to get sever key store " + e.getMessage()); } return null; } @@ -149,7 +149,7 @@ static KeyStore getClientKeyStore() { keyStore.setKeyEntry("ratis-client-key", privateKey, new char[0], certificate); return keyStore; } catch (Exception e) { - Assert.fail("Failed to get client key store " + e.getMessage()); + Assertions.fail("Failed to get client key store " + e.getMessage()); } return null; } @@ -167,7 +167,7 @@ static KeyStore getTrustStore() { } return trustStore; } catch (Exception e) { - Assert.fail("Failed to get sever key store " + e.getMessage()); + Assertions.fail("Failed to get sever key store " + e.getMessage()); } return null; } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java index dd76a2ec1e..9360b9b847 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java @@ -26,8 +26,8 @@ import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.util.Preconditions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; /** * Test {@link RaftServer.Builder}. @@ -58,7 +58,7 @@ public void testPeerIdNotInRaftGroup() { .setStateMachine(new BaseStateMachine()) .setProperties(new RaftProperties()) .build(); - Assert.fail("did not get expected exception"); + Assertions.fail("did not get expected exception"); } catch (IOException e) { Preconditions.assertInstanceOf(e.getCause(), IllegalStateException.class); } @@ -74,7 +74,7 @@ public void testNullPeerIdWithRaftGroup() { .setStateMachine(new BaseStateMachine()) .setProperties(new RaftProperties()) .build(); - Assert.fail("did not get expected exception"); + Assertions.fail("did not get expected exception"); } catch (IOException e) { Preconditions.assertInstanceOf(e.getCause(), IllegalStateException.class); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java index 2f3edf7815..db4e92b7ce 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java @@ -46,8 +46,8 @@ import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -111,7 +111,7 @@ void runTestRestartFollower(MiniRaftCluster cluster) throws Exception { // make sure the restarted follower can catchup final RaftServer.Division followerState = cluster.getDivision(followerId); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(followerState.getInfo().getLastAppliedIndex() >= leaderLastIndex); + Assertions.assertTrue(followerState.getInfo().getLastAppliedIndex() >= leaderLastIndex); return null; }, 10, ONE_SECOND, "follower catchup", LOG); @@ -119,9 +119,9 @@ void runTestRestartFollower(MiniRaftCluster cluster) throws Exception { final RaftServer.Division follower = cluster.restartServer(followerId, false); final RaftLog followerLog = follower.getRaftLog(); final long followerLastIndex = followerLog.getLastEntryTermIndex().getIndex(); - Assert.assertTrue(followerLastIndex >= leaderLastIndex); + Assertions.assertTrue(followerLastIndex >= leaderLastIndex); final long leaderFinalIndex = cluster.getLeader().getRaftLog().getLastEntryTermIndex().getIndex(); - Assert.assertEquals(leaderFinalIndex, followerLastIndex); + Assertions.assertEquals(leaderFinalIndex, followerLastIndex); final File followerOpenLogFile = getOpenLogFile(follower); final File leaderOpenLogFile = getOpenLogFile(cluster.getDivision(leaderId)); @@ -148,7 +148,7 @@ void runTestRestartFollower(MiniRaftCluster cluster) throws Exception { if (i != truncatedMessageIndex) { final Message m = new SimpleMessage("m" + i); final RaftClientReply reply = client.io().sendReadOnly(m); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); LOG.info("query {}: {} {}", m, reply, LogEntryProto.parseFrom(reply.getMessage().getContent())); } } @@ -159,7 +159,7 @@ static void writeSomething(Supplier newMessage, MiniRaftCluster cluster try(final RaftClient client = cluster.createClient()) { // write some messages for(int i = 0; i < 10; i++) { - Assert.assertTrue(client.io().send(newMessage.get()).isSuccess()); + Assertions.assertTrue(client.io().send(newMessage.get()).isSuccess()); } } } @@ -171,7 +171,7 @@ static void assertTruncatedLog(RaftPeerId id, File openLogFile, long lastIndex, } final RaftServer.Division server = cluster.restartServer(id, false); // the last index should be one less than before - Assert.assertEquals(lastIndex - 1, server.getRaftLog().getLastEntryTermIndex().getIndex()); + Assertions.assertEquals(lastIndex - 1, server.getRaftLog().getLastEntryTermIndex().getIndex()); server.getRaftServer().close(); } @@ -184,7 +184,7 @@ static List getOpenLogFiles(RaftServer.Division server) throws Exception { static File getOpenLogFile(RaftServer.Division server) throws Exception { final List openLogs = getOpenLogFiles(server); - Assert.assertEquals(1, openLogs.size()); + Assertions.assertEquals(1, openLogs.size()); return openLogs.get(0).toFile(); } @@ -210,7 +210,7 @@ void runTestRestartWithCorruptedLogHeader(MiniRaftCluster cluster) throws Except 10, HUNDRED_MILLIS, impl.getId() + "-getOpenLogFile", LOG); for(int i = 0; i < SegmentedRaftLogFormat.getHeaderLength(); i++) { assertCorruptedLogHeader(impl.getId(), openLogFile, i, cluster, LOG); - Assert.assertTrue(getOpenLogFiles(impl).isEmpty()); + Assertions.assertTrue(getOpenLogFiles(impl).isEmpty()); } } } @@ -245,7 +245,7 @@ void runTestRestartCommitIndex(MiniRaftCluster cluster) throws Exception { final SimpleMessage m = messages[i]; new Thread(() -> { try (final RaftClient client = cluster.createClient()) { - Assert.assertTrue(client.io().send(m).isSuccess()); + Assertions.assertTrue(client.io().send(m).isSuccess()); } catch (IOException e) { throw new IllegalStateException("Failed to send " + m, e); } @@ -301,11 +301,11 @@ void runTestRestartCommitIndex(MiniRaftCluster cluster) throws Exception { final RaftServer.Division server = cluster.getDivision(id); final RaftLog raftLog = server.getRaftLog(); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(raftLog.getLastCommittedIndex() >= loggedCommitIndex); + Assertions.assertTrue(raftLog.getLastCommittedIndex() >= loggedCommitIndex); return null; }, 10, HUNDRED_MILLIS, id + "(commitIndex >= loggedCommitIndex)", LOG); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(server.getInfo().getLastAppliedIndex() >= loggedCommitIndex); + Assertions.assertTrue(server.getInfo().getLastAppliedIndex() >= loggedCommitIndex); return null; }, 10, HUNDRED_MILLIS, id + "(lastAppliedIndex >= loggedCommitIndex)", LOG); LOG.info("{}: commitIndex={}, lastAppliedIndex={}", @@ -318,16 +318,16 @@ static void assertLastLogEntry(RaftServer.Division server) throws RaftLogIOExcep final RaftLog raftLog = server.getRaftLog(); final long lastIndex = raftLog.getLastEntryTermIndex().getIndex(); final LogEntryProto lastEntry = raftLog.get(lastIndex); - Assert.assertTrue(lastEntry.hasMetadataEntry()); + Assertions.assertTrue(lastEntry.hasMetadataEntry()); final long loggedCommitIndex = lastEntry.getMetadataEntry().getCommitIndex(); final LogEntryProto lastCommittedEntry = raftLog.get(loggedCommitIndex); - Assert.assertTrue(lastCommittedEntry.hasStateMachineLogEntry()); + Assertions.assertTrue(lastCommittedEntry.hasStateMachineLogEntry()); final SimpleStateMachine4Testing leaderStateMachine = SimpleStateMachine4Testing.get(server); final TermIndex lastAppliedTermIndex = leaderStateMachine.getLastAppliedTermIndex(); - Assert.assertEquals(lastCommittedEntry.getTerm(), lastAppliedTermIndex.getTerm()); - Assert.assertTrue(lastCommittedEntry.getIndex() <= lastAppliedTermIndex.getIndex()); + Assertions.assertEquals(lastCommittedEntry.getTerm(), lastAppliedTermIndex.getTerm()); + Assertions.assertTrue(lastCommittedEntry.getIndex() <= lastAppliedTermIndex.getIndex()); } @Test @@ -364,11 +364,11 @@ private void runTestRestartWithCorruptedLogEntry(CLUSTER cluster) throws Excepti final SimpleMessage lastMessage = messages[messages.length - 1]; try (final RaftClient client = cluster.createClient()) { for (SimpleMessage m : messages) { - Assert.assertTrue(client.io().send(m).isSuccess()); + Assertions.assertTrue(client.io().send(m).isSuccess()); } // assert that the last message exists - Assert.assertTrue(client.io().sendReadOnly(lastMessage).isSuccess()); + Assertions.assertTrue(client.io().sendReadOnly(lastMessage).isSuccess()); } final RaftLog log = leader.getRaftLog(); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java b/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java index bb386e8309..2026398e33 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java @@ -23,9 +23,9 @@ import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -51,7 +51,7 @@ public class TestRaftServerConfigKeys { JavaUtils.getClassSimpleName(TestRaftServerConfigKeys.class) + Integer.toHexString(ThreadLocalRandom.current().nextInt()))); - @AfterClass + @AfterAll public static void tearDown() throws IOException { FileUtils.deleteFully(rootTestDir.get()); } @@ -74,7 +74,7 @@ public void testStorageDirProperty() { final String expected = directories.stream().map(File::getAbsolutePath) .collect(Collectors.joining(",")); final String actual = properties.get(RaftServerConfigKeys.STORAGE_DIR_KEY); - Assert.assertEquals(expected, actual); + Assertions.assertEquals(expected, actual); } /** @@ -97,8 +97,8 @@ public void testStorageDir() { final List actualDirs = storageDirs.stream() .map(File::getAbsolutePath).collect(Collectors.toList()); actualDirs.removeAll(expectedDirs); - Assert.assertEquals(directories.size(), storageDirs.size()); - Assert.assertEquals(0, actualDirs.size()); + Assertions.assertEquals(directories.size(), storageDirs.size()); + Assertions.assertEquals(0, actualDirs.size()); } /** @@ -113,6 +113,6 @@ public void testStorageDir() { int pendingRequestMegabyteLimit = Math.toIntExact( RaftServerConfigKeys.Write.byteLimit(properties).getSize() / SizeInBytes.ONE_MB.getSize()); - Assert.assertEquals(4096, pendingRequestMegabyteLimit); + Assertions.assertEquals(4096, pendingRequestMegabyteLimit); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestPeerConfiguration.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestPeerConfiguration.java index e1adcfa442..df4d53cf0b 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestPeerConfiguration.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestPeerConfiguration.java @@ -20,14 +20,14 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collection; import java.util.stream.Collectors; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestPeerConfiguration extends BaseTest { @Test diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java index 14e0030e6d..77ec0ed29a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java @@ -22,14 +22,14 @@ import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftConfiguration; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collection; import java.util.stream.Collectors; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestRaftConfiguration extends BaseTest { @Test @@ -65,29 +65,29 @@ public void testSingleMode() { RaftConfigurationImpl config = RaftConfigurationImpl.newBuilder() .setConf(new PeerConfiguration(raftPeersWithPriority(1))) .build(); - assertTrue("Peer is in single mode.", config.isSingleMode(RaftPeerId.valueOf("1"))); + assertTrue(config.isSingleMode(RaftPeerId.valueOf("1")), "Peer is in single mode."); config = RaftConfigurationImpl.newBuilder() .setConf(new PeerConfiguration(raftPeersWithPriority(0, 1))) .setOldConf(new PeerConfiguration(raftPeersWithPriority(0))) .build(); - assertTrue("Peer is in single mode.", config.isSingleMode(RaftPeerId.valueOf("0"))); - assertFalse("Peer is a new peer.", config.isSingleMode(RaftPeerId.valueOf("1"))); + assertTrue(config.isSingleMode(RaftPeerId.valueOf("0")), "Peer is in single mode."); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("1")), "Peer is a new peer"); config = RaftConfigurationImpl.newBuilder() .setConf(new PeerConfiguration(raftPeersWithPriority(0, 1))) .build(); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("0"))); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("1"))); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("0")), "Peer is in ha mode."); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("1")), "Peer is in ha mode."); config = RaftConfigurationImpl.newBuilder() .setConf(new PeerConfiguration(raftPeersWithPriority(0, 1))) .setOldConf(new PeerConfiguration(raftPeersWithPriority(2, 3))) .build(); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("0"))); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("1"))); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("3"))); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("4"))); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("0")), "Peer is in ha mode."); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("1")), "Peer is in ha mode."); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("3")), "Peer is in ha mode."); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("4")), "Peer is in ha mode."); } @Test @@ -96,8 +96,8 @@ public void testChangeMajority() { RaftConfigurationImpl config = RaftConfigurationImpl.newBuilder() .setConf(new PeerConfiguration(raftPeersWithPriority(1))) .build(); - assertFalse("Change from single mode to ha mode is not considered as changing majority.", - config.changeMajority(raftPeersWithPriority(1, 2))); + assertFalse(config.changeMajority(raftPeersWithPriority(1, 2)), + "Change from single mode to ha mode is not considered as changing majority."); // Case 2: {1} --> {2}. assertTrue(config.changeMajority(raftPeersWithPriority(2))); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java index 01d8392303..39a1f8855b 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java @@ -24,8 +24,9 @@ import org.apache.ratis.server.RaftServerMXBean; import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc; import org.apache.ratis.util.JmxRegister; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import javax.management.JMException; import javax.management.MBeanServer; @@ -38,7 +39,8 @@ import static org.apache.ratis.RaftTestUtil.waitForLeader; public class TestRaftServerJmx extends BaseTest { - @Test(timeout = 30000) + @Test + @Timeout(value = 30000) public void testJmxBeans() throws Exception { final int NUM_SERVERS = 3; final MiniRaftClusterWithSimulatedRpc cluster @@ -48,16 +50,17 @@ public void testJmxBeans() throws Exception { MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer(); Set objectInstances = platformMBeanServer.queryMBeans(new ObjectName("Ratis:*"), null); - Assert.assertEquals(NUM_SERVERS, objectInstances.size()); + Assertions.assertEquals(NUM_SERVERS, objectInstances.size()); for (ObjectInstance instance : objectInstances) { Object groupId = platformMBeanServer.getAttribute(instance.getObjectName(), "GroupId"); - Assert.assertEquals(cluster.getGroupId().toString(), groupId); + Assertions.assertEquals(cluster.getGroupId().toString(), groupId); } cluster.shutdown(); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30000) public void testRegister() throws JMException { { final JmxRegister jmx = new JmxRegister(); @@ -102,11 +105,11 @@ static void runRegister(boolean expectToSucceed, String name, JmxRegister jmx) { final String id = RaftPeerId.valueOf(name).toString(); final String groupId = RaftGroupId.randomId().toString(); final boolean succeeded = RaftServerJmxAdapter.registerMBean(id, groupId, mBean, jmx); - Assert.assertEquals(expectToSucceed, succeeded); + Assertions.assertEquals(expectToSucceed, succeeded); } static void runUnregister(boolean expectToSucceed, JmxRegister jmx) throws JMException { final boolean succeeded = jmx.unregister(); - Assert.assertEquals(expectToSucceed, succeeded); + Assertions.assertEquals(expectToSucceed, succeeded); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogIndex.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogIndex.java index ac655222ba..d3b4b5ccb0 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogIndex.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogIndex.java @@ -18,8 +18,8 @@ package org.apache.ratis.server.raftlog; import org.apache.ratis.BaseTest; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.function.BiFunction; import java.util.function.Consumer; @@ -34,10 +34,10 @@ static void assertUpdate(RaftLogIndex index, BiFunction update, long oldValue, LongUnaryOperator op, boolean expectUpdate) { - Assert.assertEquals(oldValue, index.get()); + Assertions.assertEquals(oldValue, index.get()); final boolean updated = update.apply(index, op); - Assert.assertEquals(expectUpdate, updated); - Assert.assertEquals(expectUpdate? op.applyAsLong(oldValue): oldValue, index.get()); + Assertions.assertEquals(expectUpdate, updated); + Assertions.assertEquals(expectUpdate? op.applyAsLong(oldValue): oldValue, index.get()); } @@ -45,7 +45,7 @@ static void assertUpdate(RaftLogIndex index, BiFunction log = System.out::println; { // test updateIncreasingly diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java index 70b185e554..65493d7a37 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java @@ -36,8 +36,8 @@ import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import javax.management.ObjectName; import java.lang.management.ManagementFactory; @@ -116,7 +116,7 @@ static void runTestRaftLogMetrics(MiniRaftCluster cluster) throws Exception { static void assertCommitCount(RaftServer.Division server, int expectedMsgs) { final RatisMetricRegistry rlm = ((RatisMetrics)server.getRaftLog().getRaftLogMetrics()).getRegistry(); long stmCount = rlm.counter(STATE_MACHINE_LOG_ENTRY_COUNT).getCount(); - Assert.assertEquals(expectedMsgs, stmCount); + Assertions.assertEquals(expectedMsgs, stmCount); } static RatisMetricRegistryImpl getRegistry(RaftGroupMemberId memberId) { @@ -127,21 +127,21 @@ static void assertFlushCount(RaftServer.Division server) throws Exception { final String flushTimeMetric = RaftStorageTestUtils.getLogFlushTimeMetric(server.getMemberId().toString()); final RatisMetricRegistryImpl ratisMetricRegistry = getRegistry(server.getMemberId()); Timer tm = (Timer) ratisMetricRegistry.get(RAFT_LOG_FLUSH_TIME); - Assert.assertNotNull(tm); + Assertions.assertNotNull(tm); final MetricsStateMachine stateMachine = MetricsStateMachine.get(server); final int expectedFlush = stateMachine.getFlushCount(); JavaUtils.attemptRepeatedly(() -> { - Assert.assertEquals(expectedFlush, tm.getCount()); + Assertions.assertEquals(expectedFlush, tm.getCount()); return null; }, 50, HUNDRED_MILLIS, "expectedFlush == tm.getCount()", null); - Assert.assertTrue(tm.getMeanRate() > 0); + Assertions.assertTrue(tm.getMeanRate() > 0); // Test jmx ObjectName oname = new ObjectName(RATIS_APPLICATION_NAME_METRICS, "name", flushTimeMetric); - Assert.assertEquals(expectedFlush, + Assertions.assertEquals(expectedFlush, ((Long) ManagementFactory.getPlatformMBeanServer().getAttribute(oname, "Count")) .intValue()); } @@ -152,42 +152,42 @@ static void assertRaftLogWritePathMetrics(RaftServer.Division server) throws Exc //Test sync count Timer tm = (Timer) ratisMetricRegistry.get(RAFT_LOG_SYNC_TIME); - Assert.assertNotNull(tm); + Assertions.assertNotNull(tm); final MetricsStateMachine stateMachine = MetricsStateMachine.get(server); final int expectedFlush = stateMachine.getFlushCount(); - Assert.assertEquals(expectedFlush, tm.getCount()); // Ideally, flushCount should be same as syncCount. - Assert.assertTrue(tm.getMeanRate() > 0); + Assertions.assertEquals(expectedFlush, tm.getCount()); // Ideally, flushCount should be same as syncCount. + Assertions.assertTrue(tm.getMeanRate() > 0); // Test jmx. Just testing one metric's JMX is good enough. ObjectName oname = new ObjectName(RATIS_APPLICATION_NAME_METRICS, "name", syncTimeMetric); - Assert.assertEquals(expectedFlush, + Assertions.assertEquals(expectedFlush, ((Long) ManagementFactory.getPlatformMBeanServer().getAttribute(oname, "Count")) .intValue()); long cacheMissCount = ratisMetricRegistry.counter(RAFT_LOG_CACHE_MISS_COUNT).getCount(); - Assert.assertEquals(0, cacheMissCount); + Assertions.assertEquals(0, cacheMissCount); long cacheHitsCount = ratisMetricRegistry.counter(RAFT_LOG_CACHE_HIT_COUNT).getCount(); - Assert.assertTrue(cacheHitsCount > 0); + Assertions.assertTrue(cacheHitsCount > 0); - Assert.assertTrue(ratisMetricRegistry.counter(RAFT_LOG_FLUSH_COUNT).getCount() > 0); - Assert.assertTrue(ratisMetricRegistry.counter(RAFT_LOG_APPEND_ENTRY_COUNT).getCount() > 0); + Assertions.assertTrue(ratisMetricRegistry.counter(RAFT_LOG_FLUSH_COUNT).getCount() > 0); + Assertions.assertTrue(ratisMetricRegistry.counter(RAFT_LOG_APPEND_ENTRY_COUNT).getCount() > 0); final DefaultTimekeeperImpl appendEntry = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_APPEND_ENTRY_LATENCY); - Assert.assertTrue(appendEntry.getTimer().getMeanRate() > 0); + Assertions.assertTrue(appendEntry.getTimer().getMeanRate() > 0); final DefaultTimekeeperImpl taskQueue = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_TASK_QUEUE_TIME); - Assert.assertTrue(taskQueue.getTimer().getMeanRate() > 0); + Assertions.assertTrue(taskQueue.getTimer().getMeanRate() > 0); final DefaultTimekeeperImpl enqueueDelay = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_TASK_ENQUEUE_DELAY); - Assert.assertTrue(enqueueDelay.getTimer().getMeanRate() > 0); + Assertions.assertTrue(enqueueDelay.getTimer().getMeanRate() > 0); final DefaultTimekeeperImpl write = (DefaultTimekeeperImpl) ratisMetricRegistry.timer( String.format(RAFT_LOG_TASK_EXECUTION_TIME, "writelog")); - Assert.assertTrue(write.getTimer().getMeanRate() > 0); + Assertions.assertTrue(write.getTimer().getMeanRate() > 0); - Assert.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_DATA_QUEUE_SIZE)); - Assert.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_WORKER_QUEUE_SIZE)); - Assert.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_SYNC_BATCH_SIZE)); + Assertions.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_DATA_QUEUE_SIZE)); + Assertions.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_WORKER_QUEUE_SIZE)); + Assertions.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_SYNC_BATCH_SIZE)); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java index 503de34536..e88699b052 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java @@ -17,7 +17,7 @@ */ package org.apache.ratis.server.raftlog.memory; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.util.ArrayList; import java.util.List; @@ -35,7 +35,7 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; public class MemoryRaftLogTest extends BaseTest { diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java index c9d7928555..cefd5f6747 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java @@ -18,9 +18,8 @@ package org.apache.ratis.server.raftlog.segmented; import org.apache.ratis.BaseTest; -import org.apache.ratis.util.StringUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.ByteBuffer; @@ -40,8 +39,8 @@ class FakeFileChannel extends FileChannel { private long forcedPosition = 0; void assertValues(long expectedPosition, long expectedForcedPosition) { - Assert.assertEquals(expectedPosition, position); - Assert.assertEquals(expectedForcedPosition, forcedPosition); + Assertions.assertEquals(expectedPosition, position); + Assertions.assertEquals(expectedForcedPosition, forcedPosition); } @Override @@ -200,7 +199,7 @@ static void flush(BufferedWriteChannel out, FakeFileChannel fake, AtomicInteger pos, AtomicInteger force) throws IOException { final int existing = out.writeBufferPosition(); out.flush(); - Assert.assertEquals(0, out.writeBufferPosition()); + Assertions.assertEquals(0, out.writeBufferPosition()); pos.addAndGet(existing); force.set(pos.get()); fake.assertValues(pos.get(), force.get()); @@ -213,9 +212,9 @@ static void writeToBuffer(BufferedWriteChannel out, FakeFileChannel fake, Atomic out.writeToBuffer(n, b -> b.put(buffer)); if (existing + n > bufferCapacity) { pos.addAndGet(existing); - Assert.assertEquals(n, out.writeBufferPosition()); + Assertions.assertEquals(n, out.writeBufferPosition()); } else { - Assert.assertEquals(existing + n, out.writeBufferPosition()); + Assertions.assertEquals(existing + n, out.writeBufferPosition()); } fake.assertValues(pos.get(), force.get()); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java index 675df51d6b..ee73a7079b 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java @@ -40,8 +40,8 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import java.io.File; @@ -56,7 +56,7 @@ public class TestCacheEviction extends BaseTest { private static final CacheInvalidationPolicy policy = new CacheInvalidationPolicyDefault(); static LogSegmentList prepareSegments(int numSegments, boolean[] cached, long start, long size) { - Assert.assertEquals(numSegments, cached.length); + Assertions.assertEquals(numSegments, cached.length); final LogSegmentList segments = new LogSegmentList(JavaUtils.getClassSimpleName(TestCacheEviction.class)); for (int i = 0; i < numSegments; i++) { LogSegment s = LogSegment.newCloseSegment(null, start, start + size - 1, MAX_OP_SIZE, null); @@ -78,35 +78,35 @@ public void testBasicEviction() throws Exception { // case 1, make sure we do not evict cache for segments behind local flushed index List evicted = policy.evict(null, 5, 15, segments, maxCached); - Assert.assertEquals(0, evicted.size()); + Assertions.assertEquals(0, evicted.size()); // case 2, suppose the local flushed index is in the 3rd segment, then we // can evict the first two segment evicted = policy.evict(null, 25, 30, segments, maxCached); - Assert.assertEquals(2, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(0)); - Assert.assertSame(evicted.get(1), segments.get(1)); + Assertions.assertEquals(2, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(0)); + Assertions.assertSame(evicted.get(1), segments.get(1)); // case 3, similar with case 2, but the local applied index is less than // the local flushed index. evicted = policy.evict(null, 25, 15, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(0)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(0)); // case 4, the local applied index is very small, then evict cache behind it // first and let the state machine load the segments later evicted = policy.evict(null, 35, 5, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(2)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(2)); Mockito.when(segments.get(2).hasCache()).thenReturn(false); evicted = policy.evict(null, 35, 5, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(1)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(1)); Mockito.when(segments.get(1).hasCache()).thenReturn(false); evicted = policy.evict(null, 35, 5, segments, maxCached); - Assert.assertEquals(0, evicted.size()); + Assertions.assertEquals(0, evicted.size()); } @Test @@ -119,37 +119,37 @@ public void testEvictionWithFollowerIndices() throws Exception { // flushed index List evicted = policy.evict(new long[]{20, 40, 40}, 5, 15, segments, maxCached); - Assert.assertEquals(0, evicted.size()); + Assertions.assertEquals(0, evicted.size()); // case 2, the follower indices are behind the local flushed index evicted = policy.evict(new long[]{30, 40, 45}, 25, 30, segments, maxCached); - Assert.assertEquals(2, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(0)); - Assert.assertSame(evicted.get(1), segments.get(1)); + Assertions.assertEquals(2, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(0)); + Assertions.assertSame(evicted.get(1), segments.get(1)); // case 3, similar with case 3 in basic eviction test evicted = policy.evict(new long[]{30, 40, 45}, 25, 15, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(0)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(0)); // case 4, the followers are slower than local flush evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(0)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(0)); Mockito.when(segments.get(0).hasCache()).thenReturn(false); evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(2)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(2)); Mockito.when(segments.get(2).hasCache()).thenReturn(false); evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(3)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(3)); Mockito.when(segments.get(3).hasCache()).thenReturn(false); evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); - Assert.assertEquals(0, evicted.size()); + Assertions.assertEquals(0, evicted.size()); } @Test @@ -178,7 +178,7 @@ public void testEvictionInSegmentedLog() throws Exception { raftLog.append(ReferenceCountedObject.wrap(entries)).forEach(CompletableFuture::join); // check the current cached segment number: the last segment is still open - Assert.assertEquals(maxCachedNum - 1, + Assertions.assertEquals(maxCachedNum - 1, raftLog.getRaftLogCache().getCachedSegmentNum()); Mockito.when(info.getLastAppliedIndex()).thenReturn(35L); @@ -189,7 +189,7 @@ public void testEvictionInSegmentedLog() throws Exception { // check the cached segment number again. since the slowest follower is on // index 21, the eviction should happen and evict 3 segments - Assert.assertEquals(maxCachedNum + 1 - 3, + Assertions.assertEquals(maxCachedNum + 1 - 3, raftLog.getRaftLogCache().getCachedSegmentNum()); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java index ece17a0524..8355c67333 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java @@ -37,10 +37,10 @@ import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TraditionalBinaryPrefix; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -65,7 +65,7 @@ public class TestLogSegment extends BaseTest { private long preallocatedSize; private int bufferSize; - @Before + @BeforeEach public void setup() { RaftProperties properties = new RaftProperties(); storageDir = getTestDir(); @@ -78,7 +78,7 @@ public void setup() { RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt(); } - @After + @AfterEach public void tearDown() throws Exception { if (storageDir != null) { FileUtils.deleteFully(storageDir.getParentFile()); @@ -110,11 +110,11 @@ File prepareLog(boolean isOpen, long startIndex, int numEntries, long term, bool // 0 < truncatedEntrySize < entrySize final long fileLength = file.length(); final long truncatedFileLength = fileLength - (entrySize - truncatedEntrySize); - Assert.assertTrue(truncatedFileLength < fileLength); + Assertions.assertTrue(truncatedFileLength < fileLength); LOG.info("truncate last entry: entry(size={}, truncated={}), file(length={}, truncated={})", entrySize, truncatedEntrySize, fileLength, truncatedFileLength); FileUtils.truncateFile(file, truncatedFileLength); - Assert.assertEquals(truncatedFileLength, file.length()); + Assertions.assertEquals(truncatedFileLength, file.length()); } storage.close(); @@ -128,18 +128,18 @@ static int size(LogEntryProto entry) { static void checkLogSegment(LogSegment segment, long start, long end, boolean isOpen, long totalSize, long term) throws Exception { - Assert.assertEquals(start, segment.getStartIndex()); - Assert.assertEquals(end, segment.getEndIndex()); - Assert.assertEquals(isOpen, segment.isOpen()); - Assert.assertEquals(totalSize, segment.getTotalFileSize()); + Assertions.assertEquals(start, segment.getStartIndex()); + Assertions.assertEquals(end, segment.getEndIndex()); + Assertions.assertEquals(isOpen, segment.isOpen()); + Assertions.assertEquals(totalSize, segment.getTotalFileSize()); long offset = SegmentedRaftLogFormat.getHeaderLength(); for (long i = start; i <= end; i++) { LogSegment.LogRecord record = segment.getLogRecord(i); final TermIndex ti = record.getTermIndex(); - Assert.assertEquals(i, ti.getIndex()); - Assert.assertEquals(term, ti.getTerm()); - Assert.assertEquals(offset, record.getOffset()); + Assertions.assertEquals(i, ti.getIndex()); + Assertions.assertEquals(term, ti.getTerm()); + Assertions.assertEquals(offset, record.getOffset()); LogEntryProto entry = segment.getEntryFromCache(ti); if (entry == null) { @@ -179,7 +179,7 @@ private void testLoadSegment(boolean loadInitial, boolean isLastEntryPartiallyWr checkLogSegment(openSegment, 0, 99 - delta, true, openSegmentFile.length(), 0); storage.close(); // for open segment we currently always keep log entries in the memory - Assert.assertEquals(0, openSegment.getLoadingTimes()); + Assertions.assertEquals(0, openSegment.getLoadingTimes()); // load a closed segment (1000-1099) final File closedSegmentFile = prepareLog(false, 1000, 100, 1, false); @@ -187,7 +187,7 @@ private void testLoadSegment(boolean loadInitial, boolean isLastEntryPartiallyWr LogSegmentStartEnd.valueOf(1000, 1099L), MAX_OP_SIZE, loadInitial, null, null); checkLogSegment(closedSegment, 1000, 1099, false, closedSegment.getTotalFileSize(), 1); - Assert.assertEquals(loadInitial ? 0 : 1, closedSegment.getLoadingTimes()); + Assertions.assertEquals(loadInitial ? 0 : 1, closedSegment.getLoadingTimes()); } @Test @@ -208,7 +208,7 @@ public void testAppendEntries() throws Exception { segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); } - Assert.assertTrue(segment.getTotalFileSize() >= max); + Assertions.assertTrue(segment.getTotalFileSize() >= max); checkLogSegment(segment, start, i - 1 + start, true, size, term); } @@ -224,9 +224,9 @@ public void testAppendEntryMetric() throws Exception { storage.close(); final DefaultTimekeeperImpl readEntryTimer = (DefaultTimekeeperImpl) raftLogMetrics.getReadEntryTimer(); - Assert.assertNotNull(readEntryTimer); - Assert.assertEquals(100, readEntryTimer.getTimer().getCount()); - Assert.assertTrue(readEntryTimer.getTimer().getMeanRate() > 0); + Assertions.assertNotNull(readEntryTimer); + Assertions.assertEquals(100, readEntryTimer.getTimer().getCount()); + Assertions.assertTrue(readEntryTimer.getTimer().getMeanRate() > 0); } @@ -238,7 +238,7 @@ public void testAppendWithGap() throws Exception { try { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m, 0, 1001); segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); - Assert.fail("should fail since the entry's index needs to be 1000"); + Assertions.fail("should fail since the entry's index needs to be 1000"); } catch (IllegalStateException e) { // the exception is expected. } @@ -249,7 +249,7 @@ public void testAppendWithGap() throws Exception { try { entry = LogProtoUtils.toLogEntryProto(m, 0, 1002); segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); - Assert.fail("should fail since the entry's index needs to be 1001"); + Assertions.fail("should fail since the entry's index needs to be 1001"); } catch (IllegalStateException e) { // the exception is expected. } @@ -269,18 +269,18 @@ public void testTruncate() throws Exception { // truncate an open segment (remove 1080~1099) long newSize = segment.getLogRecord(start + 80).getOffset(); segment.truncate(start + 80); - Assert.assertEquals(80, segment.numOfEntries()); + Assertions.assertEquals(80, segment.numOfEntries()); checkLogSegment(segment, start, start + 79, false, newSize, term); // truncate a closed segment (remove 1050~1079) newSize = segment.getLogRecord(start + 50).getOffset(); segment.truncate(start + 50); - Assert.assertEquals(50, segment.numOfEntries()); + Assertions.assertEquals(50, segment.numOfEntries()); checkLogSegment(segment, start, start + 49, false, newSize, term); // truncate all the remaining entries segment.truncate(start); - Assert.assertEquals(0, segment.numOfEntries()); + Assertions.assertEquals(0, segment.numOfEntries()); checkLogSegment(segment, start, start - 1, false, SegmentedRaftLogFormat.getHeaderLength(), term); } @@ -300,12 +300,12 @@ public void testPreallocateSegment() throws Exception { for (int a : preallocated) { try(SegmentedRaftLogOutputStream ignored = new SegmentedRaftLogOutputStream(file, false, max, a, ByteBuffer.allocateDirect(bufferSize))) { - Assert.assertEquals("max=" + max + ", a=" + a, file.length(), Math.min(max, a)); + Assertions.assertEquals(file.length(), Math.min(max, a), "max=" + max + ", a=" + a); } try(SegmentedRaftLogInputStream in = SegmentedRaftLogTestUtils.newSegmentedRaftLogInputStream( file, 0, INVALID_LOG_INDEX, true)) { LogEntryProto entry = in.nextEntry(); - Assert.assertNull(entry); + Assertions.assertNull(entry); } } } @@ -321,14 +321,14 @@ public void testPreallocateSegment() throws Exception { size = LogSegment.getEntrySize(entry, Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); out.write(entry); } - Assert.assertEquals(file.length(), + Assertions.assertEquals(file.length(), size + SegmentedRaftLogFormat.getHeaderLength()); try (SegmentedRaftLogInputStream in = SegmentedRaftLogTestUtils.newSegmentedRaftLogInputStream( file, 0, INVALID_LOG_INDEX, true)) { LogEntryProto entry = in.nextEntry(); - Assert.assertArrayEquals(content, + Assertions.assertArrayEquals(content, entry.getStateMachineLogEntry().getLogData().toByteArray()); - Assert.assertNull(in.nextEntry()); + Assertions.assertNull(in.nextEntry()); } } @@ -351,19 +351,19 @@ public void testPreallocationAndAppend() throws Exception { long preallocated = 16 * 1024; try (SegmentedRaftLogOutputStream out = new SegmentedRaftLogOutputStream(file, false, max.getSize(), 16 * 1024, ByteBuffer.allocateDirect(10 * 1024))) { - Assert.assertEquals(preallocated, file.length()); + Assertions.assertEquals(preallocated, file.length()); while (totalSize + entrySize < max.getSize()) { totalSize += entrySize; out.write(entry); if (totalSize > preallocated) { - Assert.assertEquals("totalSize==" + totalSize, - preallocated + 16 * 1024, file.length()); + Assertions.assertEquals(preallocated + 16 * 1024, file.length(), + "totalSize==" + totalSize); preallocated += 16 * 1024; } } } - Assert.assertEquals(totalSize, file.length()); + Assertions.assertEquals(totalSize, file.length()); } @Test @@ -374,14 +374,14 @@ public void testZeroSizeInProgressFile() throws Exception { // create zero size in-progress file LOG.info("file: " + file); - Assert.assertTrue(file.createNewFile()); + Assertions.assertTrue(file.createNewFile()); final Path path = file.toPath(); - Assert.assertTrue(Files.exists(path)); - Assert.assertEquals(0, Files.size(path)); + Assertions.assertTrue(Files.exists(path)); + Assertions.assertEquals(0, Files.size(path)); // getLogSegmentPaths should remove it. final List logs = LogSegmentPath.getLogSegmentPaths(storage); - Assert.assertEquals(0, logs.size()); - Assert.assertFalse(Files.exists(path)); + Assertions.assertEquals(0, logs.size()); + Assertions.assertFalse(Files.exists(path)); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java index a020b43bd8..4f151c733f 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java @@ -29,10 +29,10 @@ import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream; import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.util.FileUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -53,7 +53,7 @@ public class TestRaftLogReadWrite extends BaseTest { private long preallocatedSize; private int bufferSize; - @Before + @BeforeEach public void setup() { storageDir = getTestDir(); RaftProperties properties = new RaftProperties(); @@ -66,7 +66,7 @@ public void setup() { RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt(); } - @After + @AfterEach public void tearDown() throws Exception { if (storageDir != null) { FileUtils.deleteFully(storageDir.getParentFile()); @@ -116,10 +116,10 @@ public void testReadWriteLog() throws IOException { storage.close(); } - Assert.assertEquals(size, openSegment.length()); + Assertions.assertEquals(size, openSegment.length()); final LogEntryProto[] readEntries = readLog(openSegment, 0, RaftLog.INVALID_LOG_INDEX, true); - Assert.assertArrayEquals(entries, readEntries); + Assertions.assertArrayEquals(entries, readEntries); } @Test @@ -146,7 +146,7 @@ public void testAppendLog() throws IOException { } final LogEntryProto[] readEntries = readLog(openSegment, 0, RaftLog.INVALID_LOG_INDEX, true); - Assert.assertArrayEquals(entries, readEntries); + Assertions.assertArrayEquals(entries, readEntries); storage.close(); } @@ -168,16 +168,16 @@ public void testReadWithPadding() throws IOException { out.flush(); // make sure the file contains padding - Assert.assertEquals( + Assertions.assertEquals( RaftServerConfigKeys.Log.PREALLOCATED_SIZE_DEFAULT.getSize(), openSegment.length()); // check if the reader can correctly read the log file final LogEntryProto[] readEntries = readLog(openSegment, 0, RaftLog.INVALID_LOG_INDEX, true); - Assert.assertArrayEquals(entries, readEntries); + Assertions.assertArrayEquals(entries, readEntries); out.close(); - Assert.assertEquals(size, openSegment.length()); + Assertions.assertEquals(size, openSegment.length()); } /** @@ -200,12 +200,12 @@ public void testReadWithCorruptPadding() throws IOException { out.flush(); // make sure the file contains padding - Assert.assertEquals(4 * 1024 * 1024, openSegment.length()); + Assertions.assertEquals(4 * 1024 * 1024, openSegment.length()); try (FileChannel fout = FileUtils.newFileChannel(openSegment, StandardOpenOption.WRITE)) { final byte[] array = {-1, 1}; final int written = fout.write(ByteBuffer.wrap(array), 16 * 1024 * 1024 - 10); - Assert.assertEquals(array.length, written); + Assertions.assertEquals(array.length, written); } List list = new ArrayList<>(); @@ -215,7 +215,7 @@ public void testReadWithCorruptPadding() throws IOException { while ((entry = in.nextEntry()) != null) { list.add(entry); } - Assert.fail("should fail since we corrupt the padding"); + Assertions.fail("should fail since we corrupt the padding"); } catch (IOException e) { boolean findVerifyTerminator = false; for (StackTraceElement s : e.getStackTrace()) { @@ -224,9 +224,9 @@ public void testReadWithCorruptPadding() throws IOException { break; } } - Assert.assertTrue(findVerifyTerminator); + Assertions.assertTrue(findVerifyTerminator); } - Assert.assertArrayEquals(entries, + Assertions.assertArrayEquals(entries, list.toArray(new LogEntryProto[list.size()])); } @@ -259,7 +259,7 @@ public void testReadWithEntryCorruption() throws IOException { try { readLog(openSegment, 0, RaftLog.INVALID_LOG_INDEX, true); - Assert.fail("The read of corrupted log file should fail"); + Assertions.fail("The read of corrupted log file should fail"); } catch (ChecksumException e) { LOG.info("Caught ChecksumException as expected", e); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 38341e0258..55fd6fbdf7 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -17,8 +17,6 @@ */ package org.apache.ratis.server.raftlog.segmented; -import static org.junit.Assert.assertTrue; - import org.apache.ratis.BaseTest; import org.apache.ratis.RaftTestUtil.SimpleOperation; import org.apache.ratis.conf.RaftProperties; @@ -50,18 +48,12 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -69,12 +61,22 @@ import java.util.concurrent.TimeUnit; import java.util.function.LongSupplier; import java.util.function.Supplier; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import java.util.stream.Stream; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; -@RunWith(Parameterized.class) +import static java.lang.Boolean.FALSE; +import static java.lang.Boolean.TRUE; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.params.provider.Arguments.arguments; + public class TestSegmentedRaftLog extends BaseTest { static { Slf4jUtils.setLogLevel(SegmentedRaftLogWorker.LOG, Level.INFO); @@ -82,18 +84,12 @@ public class TestSegmentedRaftLog extends BaseTest { Slf4jUtils.setLogLevel(SegmentedRaftLog.LOG, Level.INFO); } - private final Boolean smSyncFlush; - private final Boolean useAsyncFlush; - - public TestSegmentedRaftLog(Boolean raftLogAsync, Boolean smSync) { - this.useAsyncFlush = raftLogAsync; - this.smSyncFlush = smSync; - } - - @Parameterized.Parameters - public static Collection data() { - return Arrays.asList((new Boolean[][] {{Boolean.FALSE, Boolean.FALSE}, {Boolean.FALSE, Boolean.TRUE}, - {Boolean.TRUE, Boolean.FALSE}, {Boolean.TRUE, Boolean.TRUE}})); + public static Stream data() { + return Stream.of( + arguments(FALSE, FALSE), + arguments(FALSE, TRUE), + arguments(TRUE, FALSE), + arguments(TRUE, TRUE)); } public static long getOpenSegmentSize(RaftLog raftLog) { @@ -155,13 +151,11 @@ private SegmentedRaftLog newSegmentedRaftLogWithSnapshotIndex(RaftStorage storag .build(); } - @Before + @BeforeEach public void setup() throws Exception { storageDir = getTestDir(); properties = new RaftProperties(); RaftServerConfigKeys.setStorageDir(properties, Collections.singletonList(storageDir)); - RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); - RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); storage = RaftStorageTestUtils.newRaftStorage(storageDir); this.segmentMaxSize = RaftServerConfigKeys.Log.segmentSizeMax(properties).getSize(); @@ -171,7 +165,7 @@ public void setup() throws Exception { RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt(); } - @After + @AfterEach public void tearDown() throws Exception { if (storageDir != null) { FileUtils.deleteFully(storageDir.getParentFile()); @@ -213,8 +207,11 @@ private LogEntryProto getLastEntry(SegmentedRaftLog raftLog) return raftLog.get(raftLog.getLastEntryTermIndex().getIndex()); } - @Test - public void testLoadLogSegments() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testLoadLogSegments(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); // first generate log files List ranges = prepareRanges(0, 5, 100, 0); LogEntryProto[] entries = prepareLog(ranges); @@ -225,7 +222,7 @@ public void testLoadLogSegments() throws Exception { // check if log entries are loaded correctly for (LogEntryProto e : entries) { LogEntryProto entry = raftLog.get(e.getIndex()); - Assert.assertEquals(e, entry); + Assertions.assertEquals(e, entry); } final LogEntryHeader[] termIndices = raftLog.getEntries(0, 500); @@ -238,8 +235,8 @@ public void testLoadLogSegments() throws Exception { } }) .toArray(LogEntryProto[]::new); - Assert.assertArrayEquals(entries, entriesFromLog); - Assert.assertEquals(entries[entries.length - 1], getLastEntry(raftLog)); + Assertions.assertArrayEquals(entries, entriesFromLog); + Assertions.assertEquals(entries[entries.length - 1], getLastEntry(raftLog)); final RatisMetricRegistry metricRegistryForLogWorker = RaftLogMetricsBase.createRegistry(memberId); @@ -278,8 +275,11 @@ static LogEntryProto prepareLogEntry(long term, long index, Supplier str /** * Append entry one by one and check if log state is correct. */ - @Test - public void testAppendEntry() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAppendEntry(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); List ranges = prepareRanges(0, 5, 200, 0); List entries = prepareLogEntries(ranges, null); @@ -333,8 +333,11 @@ public void testAppendEntry() throws Exception { } } - @Test - public void testAppendEntryAfterPurge() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAppendEntryAfterPurge(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); List ranges = prepareRanges(0, 5, 200, 0); List entries = prepareLogEntries(ranges, null); @@ -364,8 +367,11 @@ public long getAsLong() { /** * Keep appending entries, make sure the rolling is correct. */ - @Test - public void testAppendAndRoll() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAppendAndRoll(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); RaftServerConfigKeys.Log.setPreallocatedSize(properties, SizeInBytes.valueOf("16KB")); RaftServerConfigKeys.Log.setSegmentSizeMax(properties, SizeInBytes.valueOf("128KB")); @@ -384,12 +390,15 @@ public void testAppendAndRoll() throws Exception { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); // check if the raft log is correct checkEntries(raftLog, entries, 0, entries.size()); - Assert.assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments()); + Assertions.assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments()); } } - @Test - public void testTruncate() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testTruncate(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); // prepare the log for truncation List ranges = prepareRanges(0, 5, 200, 0); List entries = prepareLogEntries(ranges, null); @@ -420,10 +429,10 @@ private void testTruncate(List entries, long fromIndex) raftLog.open(RaftLog.INVALID_LOG_INDEX, null); // check if the raft log is correct if (fromIndex > 0) { - Assert.assertEquals(entries.get((int) (fromIndex - 1)), + Assertions.assertEquals(entries.get((int) (fromIndex - 1)), getLastEntry(raftLog)); } else { - Assert.assertNull(raftLog.getLastEntryTermIndex()); + Assertions.assertNull(raftLog.getLastEntryTermIndex()); } checkEntries(raftLog, entries, 0, (int) fromIndex); } @@ -434,7 +443,7 @@ private void checkEntries(RaftLog raftLog, List expected, if (size > 0) { for (int i = offset; i < size + offset; i++) { LogEntryProto entry = raftLog.get(expected.get(i).getIndex()); - Assert.assertEquals(expected.get(i), entry); + Assertions.assertEquals(expected.get(i), entry); } final LogEntryHeader[] termIndices = raftLog.getEntries( expected.get(offset).getIndex(), @@ -450,7 +459,7 @@ private void checkEntries(RaftLog raftLog, List expected, .toArray(LogEntryProto[]::new); LogEntryProto[] expectedArray = expected.subList(offset, offset + size) .stream().toArray(LogEntryProto[]::new); - Assert.assertArrayEquals(expectedArray, entriesFromLog); + Assertions.assertArrayEquals(expectedArray, entriesFromLog); } } @@ -519,15 +528,18 @@ private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int pur final CompletableFuture f = raftLog.purge(purgeIndex); final Long purged = f.get(); LOG.info("purgeIndex = {}, purged = {}", purgeIndex, purged); - Assert.assertEquals(expectedIndex, raftLog.getRaftLogCache().getStartIndex()); + Assertions.assertEquals(expectedIndex, raftLog.getRaftLogCache().getStartIndex()); } } /** * Test append with inconsistent entries */ - @Test - public void testAppendEntriesWithInconsistency() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); // prepare the log for truncation List ranges = prepareRanges(0, 5, 200, 0); List entries = prepareLogEntries(ranges, null); @@ -558,9 +570,9 @@ public void testAppendEntriesWithInconsistency() throws Exception { checkFailedEntries(entries, 650, retryCache); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); - Assert.assertEquals(newEntries.get(newEntries.size() - 1), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1), getLastEntry(raftLog)); - Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), raftLog.getFlushIndex()); } @@ -569,18 +581,21 @@ public void testAppendEntriesWithInconsistency() throws Exception { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); - Assert.assertEquals(newEntries.get(newEntries.size() - 1), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1), getLastEntry(raftLog)); - Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), raftLog.getFlushIndex()); SegmentedRaftLogCache cache = raftLog.getRaftLogCache(); - Assert.assertEquals(5, cache.getNumOfSegments()); + Assertions.assertEquals(5, cache.getNumOfSegments()); } } - @Test - public void testSegmentedRaftLogStateMachineData() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testSegmentedRaftLogStateMachineData(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); final SegmentRange range = new SegmentRange(0, 10, 1, true); final List entries = prepareLogEntries(range, null, true, new ArrayList<>()); @@ -624,8 +639,11 @@ public void testSegmentedRaftLogStateMachineData() throws Exception { } } - @Test(expected = TimeoutIOException.class) - public void testServerShutdownOnTimeoutIOException() throws Throwable { + @ParameterizedTest + @MethodSource("data") + public void testServerShutdownOnTimeoutIOException(Boolean useAsyncFlush, Boolean smSyncFlush) throws Throwable { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); RaftServerConfigKeys.Log.StateMachineData.setSync(properties, true); final TimeDuration syncTimeout = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); RaftServerConfigKeys.Log.StateMachineData.setSyncTimeout(properties, syncTimeout); @@ -647,14 +665,14 @@ public void notifyLogFailed(Throwable cause, LogEntryProto entry) { LOG.info("Test StateMachine: Ratis log failed notification received as expected.", cause); LOG.info("Test StateMachine: Transition to PAUSED state."); - Assert.assertNotNull(entry); + Assertions.assertNotNull(entry); getLifeCycle().transition(LifeCycle.State.PAUSING); getLifeCycle().transition(LifeCycle.State.PAUSED); } }; - Throwable ex = null; // TimeoutIOException + ExecutionException ex; try (SegmentedRaftLog raftLog = SegmentedRaftLog.newBuilder() .setMemberId(memberId) .setStateMachine(sm) @@ -665,15 +683,10 @@ public void notifyLogFailed(Throwable cause, LogEntryProto entry) { // SegmentedRaftLogWorker should catch TimeoutIOException CompletableFuture f = raftLog.appendEntry(entry); // Wait for async writeStateMachineData to finish - try { - f.get(); - } catch (ExecutionException e) { - ex = e.getCause(); - } + ex = Assertions.assertThrows(ExecutionException.class, f::get); } - Assert.assertNotNull(ex); - Assert.assertSame(LifeCycle.State.PAUSED, sm.getLifeCycleState()); - throw ex; + Assertions.assertSame(LifeCycle.State.PAUSED, sm.getLifeCycleState()); + Assertions.assertInstanceOf(TimeoutIOException.class, ex.getCause()); } static Thread startAppendEntryThread(RaftLog raftLog, LogEntryProto entry) { @@ -690,9 +703,9 @@ static Thread startAppendEntryThread(RaftLog raftLog, LogEntryProto entry) { void assertIndices(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) { LOG.info("assert expectedFlushIndex={}", expectedFlushIndex); - Assert.assertEquals(expectedFlushIndex, raftLog.getFlushIndex()); + Assertions.assertEquals(expectedFlushIndex, raftLog.getFlushIndex()); LOG.info("assert expectedNextIndex={}", expectedNextIndex); - Assert.assertEquals(expectedNextIndex, raftLog.getNextIndex()); + Assertions.assertEquals(expectedNextIndex, raftLog.getNextIndex()); } void assertIndicesMultipleAttempts(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) throws Exception { @@ -700,8 +713,11 @@ void assertIndicesMultipleAttempts(RaftLog raftLog, long expectedFlushIndex, lon 10, HUNDRED_MILLIS, "assertIndices", LOG); } - @Test - public void testAsyncFlushPerf1() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAsyncFlushPerf1(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); List ranges = prepareRanges(0, 50, 20000, 0); List entries = prepareLogEntries(ranges, null); @@ -724,8 +740,11 @@ public void testAsyncFlushPerf1() throws Exception { } } - @Test - public void testAsyncFlushPerf2() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAsyncFlushPerf2(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); List ranges = prepareRanges(0, 50, 20000, 0); List entries = prepareLogEntries(ranges, null); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index 5be3c36578..fa892b2003 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -37,10 +37,10 @@ import org.apache.ratis.server.raftlog.segmented.LogSegment.Op; import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.util.ReferenceCountedObject; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestSegmentedRaftLogCache { private static final RaftProperties prop = new RaftProperties(); @@ -49,14 +49,14 @@ public class TestSegmentedRaftLogCache { private SegmentedRaftLogMetrics raftLogMetrics; private RatisMetricRegistryImpl ratisMetricRegistry; - @Before + @BeforeEach public void setup() { raftLogMetrics = new SegmentedRaftLogMetrics(RaftServerTestUtil.TEST_MEMBER_ID); ratisMetricRegistry = (RatisMetricRegistryImpl) raftLogMetrics.getRegistry(); cache = new SegmentedRaftLogCache(null, null, prop, raftLogMetrics); } - @After + @AfterEach public void clear() { raftLogMetrics.unregister(); } @@ -75,14 +75,14 @@ private LogSegment prepareLogSegment(long start, long end, boolean isOpen) { } private void checkCache(long start, long end, int segmentSize) { - Assert.assertEquals(start, cache.getStartIndex()); - Assert.assertEquals(end, cache.getEndIndex()); + Assertions.assertEquals(start, cache.getStartIndex()); + Assertions.assertEquals(end, cache.getEndIndex()); for (long index = start; index <= end; index++) { final LogSegment segment = cache.getSegment(index); final LogRecord record = segment.getLogRecord(index); final LogEntryProto entry = segment.getEntryFromCache(record.getTermIndex()); - Assert.assertEquals(index, entry.getIndex()); + Assertions.assertEquals(index, entry.getIndex()); } long[] offsets = new long[]{start, start + 1, start + (end - start) / 2, @@ -99,9 +99,9 @@ private void checkCache(long start, long end, int segmentSize) { private void checkCacheEntries(long offset, int size, long end) { final LogEntryHeader[] entries = cache.getTermIndices(offset, offset + size); long realEnd = offset + size > end + 1 ? end + 1 : offset + size; - Assert.assertEquals(realEnd - offset, entries.length); + Assertions.assertEquals(realEnd - offset, entries.length); for (long i = offset; i < realEnd; i++) { - Assert.assertEquals(i, entries[(int) (i - offset)].getIndex()); + Assertions.assertEquals(i, entries[(int) (i - offset)].getIndex()); } } @@ -114,7 +114,7 @@ public void testAddSegments() throws Exception { try { LogSegment s = prepareLogSegment(102, 103, true); cache.addSegment(s); - Assert.fail("should fail since there is gap between two segments"); + Assertions.fail("should fail since there is gap between two segments"); } catch (IllegalStateException ignored) { } @@ -125,7 +125,7 @@ public void testAddSegments() throws Exception { try { LogSegment s = prepareLogSegment(201, 202, true); cache.addSegment(s); - Assert.fail("should fail since there is still an open segment in cache"); + Assertions.fail("should fail since there is still an open segment in cache"); } catch (IllegalStateException ignored) { } @@ -135,17 +135,17 @@ public void testAddSegments() throws Exception { try { LogSegment s = prepareLogSegment(202, 203, true); cache.addSegment(s); - Assert.fail("should fail since there is gap between two segments"); + Assertions.fail("should fail since there is gap between two segments"); } catch (IllegalStateException ignored) { } LogSegment s3 = prepareLogSegment(201, 300, true); cache.addSegment(s3); - Assert.assertNotNull(cache.getOpenSegment()); + Assertions.assertNotNull(cache.getOpenSegment()); checkCache(1, 300, 100); cache.rollOpenSegment(true); - Assert.assertNotNull(cache.getOpenSegment()); + Assertions.assertNotNull(cache.getOpenSegment()); checkCache(1, 300, 100); } @@ -159,7 +159,7 @@ public void testAppendEntry() { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, 0); cache.appendEntry(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry) ); - Assert.fail("the open segment is null"); + Assertions.fail("the open segment is null"); } catch (IllegalStateException ignored) { } @@ -171,7 +171,7 @@ public void testAppendEntry() { ); } - Assert.assertNotNull(cache.getOpenSegment()); + Assertions.assertNotNull(cache.getOpenSegment()); checkCache(0, 199, 100); } @@ -188,7 +188,7 @@ public void testTruncate() throws Exception { cache.addSegment(s); long end = cache.getEndIndex(); - Assert.assertEquals(599, end); + Assertions.assertEquals(599, end); int numOfSegments = 6; // start truncation for (int i = 0; i < 10; i++) { // truncate 10 times @@ -200,46 +200,46 @@ public void testTruncate() throws Exception { // check TruncationSegments int currentNum= (int) (end / 100 + 1); if (currentNum < numOfSegments) { - Assert.assertEquals(1, ts.getToDelete().length); + Assertions.assertEquals(1, ts.getToDelete().length); numOfSegments = currentNum; } else { - Assert.assertEquals(0, ts.getToDelete().length); + Assertions.assertEquals(0, ts.getToDelete().length); } } // 230 entries remaining. truncate at the segment boundary TruncationSegments ts = cache.truncate(200); checkCache(0, 199, 100); - Assert.assertEquals(1, ts.getToDelete().length); - Assert.assertEquals(200, ts.getToDelete()[0].getStartIndex()); - Assert.assertEquals(229, ts.getToDelete()[0].getEndIndex()); - Assert.assertEquals(0, ts.getToDelete()[0].getTargetLength()); - Assert.assertFalse(ts.getToDelete()[0].isOpen()); - Assert.assertNull(ts.getToTruncate()); + Assertions.assertEquals(1, ts.getToDelete().length); + Assertions.assertEquals(200, ts.getToDelete()[0].getStartIndex()); + Assertions.assertEquals(229, ts.getToDelete()[0].getEndIndex()); + Assertions.assertEquals(0, ts.getToDelete()[0].getTargetLength()); + Assertions.assertFalse(ts.getToDelete()[0].isOpen()); + Assertions.assertNull(ts.getToTruncate()); // add another open segment and truncate it as a whole LogSegment newOpen = prepareLogSegment(200, 249, true); cache.addSegment(newOpen); ts = cache.truncate(200); checkCache(0, 199, 100); - Assert.assertEquals(1, ts.getToDelete().length); - Assert.assertEquals(200, ts.getToDelete()[0].getStartIndex()); - Assert.assertEquals(249, ts.getToDelete()[0].getEndIndex()); - Assert.assertEquals(0, ts.getToDelete()[0].getTargetLength()); - Assert.assertTrue(ts.getToDelete()[0].isOpen()); - Assert.assertNull(ts.getToTruncate()); + Assertions.assertEquals(1, ts.getToDelete().length); + Assertions.assertEquals(200, ts.getToDelete()[0].getStartIndex()); + Assertions.assertEquals(249, ts.getToDelete()[0].getEndIndex()); + Assertions.assertEquals(0, ts.getToDelete()[0].getTargetLength()); + Assertions.assertTrue(ts.getToDelete()[0].isOpen()); + Assertions.assertNull(ts.getToTruncate()); // add another open segment and truncate part of it newOpen = prepareLogSegment(200, 249, true); cache.addSegment(newOpen); ts = cache.truncate(220); checkCache(0, 219, 100); - Assert.assertNull(cache.getOpenSegment()); - Assert.assertEquals(0, ts.getToDelete().length); - Assert.assertTrue(ts.getToTruncate().isOpen()); - Assert.assertEquals(219, ts.getToTruncate().getNewEndIndex()); - Assert.assertEquals(200, ts.getToTruncate().getStartIndex()); - Assert.assertEquals(249, ts.getToTruncate().getEndIndex()); + Assertions.assertNull(cache.getOpenSegment()); + Assertions.assertEquals(0, ts.getToDelete().length); + Assertions.assertTrue(ts.getToTruncate().isOpen()); + Assertions.assertEquals(219, ts.getToTruncate().getNewEndIndex()); + Assertions.assertEquals(200, ts.getToTruncate().getStartIndex()); + Assertions.assertEquals(249, ts.getToTruncate().getEndIndex()); } @Test @@ -255,9 +255,9 @@ public void testOpenSegmentPurge() { int purgeIndex = sIndex; // open segment should never be purged TruncationSegments ts = cache.purge(purgeIndex); - Assert.assertNull(ts.getToTruncate()); - Assert.assertEquals(end - start, ts.getToDelete().length); - Assert.assertEquals(sIndex, cache.getStartIndex()); + Assertions.assertNull(ts.getToTruncate()); + Assertions.assertEquals(end - start, ts.getToDelete().length); + Assertions.assertEquals(sIndex, cache.getStartIndex()); } @Test @@ -272,9 +272,9 @@ public void testCloseSegmentPurge() { // overlapped close segment will not purged. Passing in index - 1 since // we purge a closed segment when end index == passed in purge index. TruncationSegments ts = cache.purge(purgeIndex - 1); - Assert.assertNull(ts.getToTruncate()); - Assert.assertEquals(end - start - 1, ts.getToDelete().length); - Assert.assertEquals(1, cache.getNumOfSegments()); + Assertions.assertNull(ts.getToTruncate()); + Assertions.assertEquals(end - start - 1, ts.getToDelete().length); + Assertions.assertEquals(1, cache.getNumOfSegments()); } private void populatedSegment(int start, int end, int segmentSize, boolean isOpen) { @@ -291,15 +291,15 @@ private void testIterator(long startIndex) throws IOException { TermIndex prev = null; while (iterator.hasNext()) { TermIndex termIndex = iterator.next(); - Assert.assertEquals(cache.getLogRecord(termIndex.getIndex()).getTermIndex(), termIndex); + Assertions.assertEquals(cache.getLogRecord(termIndex.getIndex()).getTermIndex(), termIndex); if (prev != null) { - Assert.assertEquals(prev.getIndex() + 1, termIndex.getIndex()); + Assertions.assertEquals(prev.getIndex() + 1, termIndex.getIndex()); } prev = termIndex; } if (startIndex <= cache.getEndIndex()) { - Assert.assertNotNull(prev); - Assert.assertEquals(cache.getEndIndex(), prev.getIndex()); + Assertions.assertNotNull(prev); + Assertions.assertEquals(cache.getEndIndex(), prev.getIndex()); } } @@ -321,7 +321,7 @@ public void testIterator() throws Exception { testIterator(299); Iterator iterator = cache.iterator(300); - Assert.assertFalse(iterator.hasNext()); + Assertions.assertFalse(iterator.hasNext()); } @Test @@ -332,14 +332,14 @@ public void testCacheMetric() { Long closedSegmentsNum = (Long) ratisMetricRegistry.getGauges((s, metric) -> s.contains(RAFT_LOG_CACHE_CLOSED_SEGMENTS_NUM)).values().iterator().next().getValue(); - Assert.assertEquals(2L, closedSegmentsNum.longValue()); + Assertions.assertEquals(2L, closedSegmentsNum.longValue()); Long closedSegmentsSizeInBytes = (Long) ratisMetricRegistry.getGauges((s, metric) -> s.contains(RAFT_LOG_CACHE_CLOSED_SEGMENTS_SIZE_IN_BYTES)).values().iterator().next().getValue(); - Assert.assertEquals(closedSegmentsSizeInBytes.longValue(), cache.getClosedSegmentsSizeInBytes()); + Assertions.assertEquals(closedSegmentsSizeInBytes.longValue(), cache.getClosedSegmentsSizeInBytes()); Long openSegmentSizeInBytes = (Long) ratisMetricRegistry.getGauges((s, metric) -> s.contains(RAFT_LOG_CACHE_OPEN_SEGMENT_SIZE_IN_BYTES)).values().iterator().next().getValue(); - Assert.assertEquals(openSegmentSizeInBytes.longValue(), cache.getOpenSegmentSizeInBytes()); + Assertions.assertEquals(openSegmentSizeInBytes.longValue(), cache.getOpenSegmentSizeInBytes()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java index 6e7555703f..eb65f342e3 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java @@ -32,10 +32,10 @@ import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.SizeInBytes; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import java.io.File; @@ -65,12 +65,12 @@ static RaftStorageImpl newRaftStorage(File dir) throws IOException { private File storageDir; - @Before + @BeforeEach public void setup() { storageDir = getTestDir(); } - @After + @AfterEach public void tearDown() throws Exception { if (storageDir != null) { FileUtils.deleteFully(storageDir.getParentFile()); @@ -93,22 +93,22 @@ public void testNotExistent() throws IOException { // we will format the empty directory final RaftStorageImpl storage = newRaftStorage(storageDir); - Assert.assertEquals(StorageState.NORMAL, storage.getState()); + Assertions.assertEquals(StorageState.NORMAL, storage.getState()); try { formatRaftStorage(storageDir).close(); - Assert.fail("the format should fail since the storage is still locked"); + Assertions.fail("the format should fail since the storage is still locked"); } catch (IOException e) { - Assert.assertTrue(e.getMessage().contains("directory is already locked")); + Assertions.assertTrue(e.getMessage().contains("directory is already locked")); } storage.close(); FileUtils.deleteFully(storageDir); - Assert.assertTrue(storageDir.createNewFile()); + Assertions.assertTrue(storageDir.createNewFile()); try (RaftStorage ignored = newRaftStorage(storageDir)) { - Assert.fail(); + Assertions.fail(); } catch (IOException e) { - Assert.assertTrue( + Assertions.assertTrue( e.getMessage().contains(StorageState.NON_EXISTENT.name())); } } @@ -121,42 +121,42 @@ public void testStorage() throws Exception { final RaftStorageDirectoryImpl sd = new RaftStorageDirectoryImpl(storageDir, SizeInBytes.ZERO); try { StorageState state = sd.analyzeStorage(true); - Assert.assertEquals(StorageState.NOT_FORMATTED, state); - Assert.assertTrue(sd.isCurrentEmpty()); + Assertions.assertEquals(StorageState.NOT_FORMATTED, state); + Assertions.assertTrue(sd.isCurrentEmpty()); } finally { sd.unlock(); } RaftStorageImpl storage = newRaftStorage(storageDir); - Assert.assertEquals(StorageState.NORMAL, storage.getState()); + Assertions.assertEquals(StorageState.NORMAL, storage.getState()); storage.close(); - Assert.assertEquals(StorageState.NORMAL, sd.analyzeStorage(false)); + Assertions.assertEquals(StorageState.NORMAL, sd.analyzeStorage(false)); assertMetadataFile(sd.getMetaFile()); // test format storage = formatRaftStorage(storageDir); - Assert.assertEquals(StorageState.NORMAL, storage.getState()); + Assertions.assertEquals(StorageState.NORMAL, storage.getState()); final RaftStorageMetadataFile metaFile = new RaftStorageMetadataFileImpl(sd.getMetaFile()); - Assert.assertEquals(RaftStorageMetadata.getDefault(), metaFile.getMetadata()); + Assertions.assertEquals(RaftStorageMetadata.getDefault(), metaFile.getMetadata()); storage.close(); } static void assertMetadataFile(File m) throws Exception { - Assert.assertTrue(m.exists()); + Assertions.assertTrue(m.exists()); final RaftStorageMetadataFile metaFile = new RaftStorageMetadataFileImpl(m); - Assert.assertEquals(RaftStorageMetadata.getDefault(), metaFile.getMetadata()); + Assertions.assertEquals(RaftStorageMetadata.getDefault(), metaFile.getMetadata()); final RaftPeerId peer1 = RaftPeerId.valueOf("peer1"); final RaftStorageMetadata metadata = RaftStorageMetadata.valueOf(123, peer1); metaFile.persist(metadata); - Assert.assertEquals(metadata.getTerm(), 123); - Assert.assertEquals(metadata.getVotedFor(), peer1); - Assert.assertEquals(metadata, metaFile.getMetadata()); + Assertions.assertEquals(metadata.getTerm(), 123); + Assertions.assertEquals(metadata.getVotedFor(), peer1); + Assertions.assertEquals(metadata, metaFile.getMetadata()); final RaftStorageMetadataFile metaFile2 = new RaftStorageMetadataFileImpl(m); - Assert.assertNull(((AtomicReference) RaftTestUtil.getDeclaredField(metaFile2, "metadata")).get()); - Assert.assertEquals(metadata, metaFile2.getMetadata()); + Assertions.assertNull(((AtomicReference) RaftTestUtil.getDeclaredField(metaFile2, "metadata")).get()); + Assertions.assertEquals(metadata, metaFile2.getMetadata()); } @Test @@ -172,27 +172,27 @@ public void testMetaFile() throws Exception { @Test public void testCleanMetaTmpFile() throws Exception { RaftStorageImpl storage = newRaftStorage(storageDir); - Assert.assertEquals(StorageState.NORMAL, storage.getState()); + Assertions.assertEquals(StorageState.NORMAL, storage.getState()); storage.close(); final RaftStorageDirectoryImpl sd = new RaftStorageDirectoryImpl(storageDir, SizeInBytes.ZERO); File metaFile = sd.getMetaFile(); FileUtils.move(metaFile, sd.getMetaTmpFile()); - Assert.assertEquals(StorageState.NOT_FORMATTED, sd.analyzeStorage(false)); + Assertions.assertEquals(StorageState.NOT_FORMATTED, sd.analyzeStorage(false)); // RaftStorage initialization should succeed as the raft-meta.tmp is // always cleaned. newRaftStorage(storageDir).close(); - Assert.assertTrue(sd.getMetaFile().exists()); - Assert.assertTrue(sd.getMetaTmpFile().createNewFile()); - Assert.assertTrue(sd.getMetaTmpFile().exists()); + Assertions.assertTrue(sd.getMetaFile().exists()); + Assertions.assertTrue(sd.getMetaTmpFile().createNewFile()); + Assertions.assertTrue(sd.getMetaTmpFile().exists()); try { storage = newRaftStorage(storageDir); - Assert.assertEquals(StorageState.NORMAL, storage.getState()); - Assert.assertFalse(sd.getMetaTmpFile().exists()); - Assert.assertTrue(sd.getMetaFile().exists()); + Assertions.assertEquals(StorageState.NORMAL, storage.getState()); + Assertions.assertFalse(sd.getMetaTmpFile().exists()); + Assertions.assertTrue(sd.getMetaFile().exists()); } finally { storage.close(); } @@ -207,14 +207,14 @@ public void testSnapshotFileName() { final File file = new File(storageDir, name); final TermIndex ti = SimpleStateMachineStorage.getTermIndexFromSnapshotFile(file); System.out.println("file = " + file); - Assert.assertEquals(term, ti.getTerm()); - Assert.assertEquals(index, ti.getIndex()); + Assertions.assertEquals(term, ti.getTerm()); + Assertions.assertEquals(index, ti.getIndex()); System.out.println("ti = " + ti); final File foo = new File(storageDir, "foo"); try { SimpleStateMachineStorage.getTermIndexFromSnapshotFile(foo); - Assert.fail(); + Assertions.fail(); } catch(IllegalArgumentException iae) { System.out.println("Good " + iae); } @@ -244,7 +244,7 @@ public int getNumSnapshotsRetained() { final long index = ThreadLocalRandom.current().nextLong(100, 1000L); if (termIndexSet.add(TermIndex.valueOf(term, index))) { File file = simpleStateMachineStorage.getSnapshotFile(term, index); - Assert.assertTrue(file.createNewFile()); + Assertions.assertTrue(file.createNewFile()); } } // create MD5 files that will not be deleted in older version @@ -254,7 +254,7 @@ public int getNumSnapshotsRetained() { if (termIndexSet.add(TermIndex.valueOf(term, index))) { File file = simpleStateMachineStorage.getSnapshotFile(term, index); File snapshotFile = new File(file.getParent(), file.getName() + MD5_SUFFIX); - Assert.assertTrue(snapshotFile.createNewFile()); + Assertions.assertTrue(snapshotFile.createNewFile()); } } @@ -274,7 +274,7 @@ public int getNumSnapshotsRetained() { System.out.println(file.getName()); Matcher matcher = SNAPSHOT_REGEX.matcher(file.getName()); if (matcher.matches()) { - Assert.assertTrue(remainingIndices.contains(Long.parseLong(matcher.group(2)))); + Assertions.assertTrue(remainingIndices.contains(Long.parseLong(matcher.group(2)))); } } @@ -288,7 +288,7 @@ public int getNumSnapshotsRetained() { final long term = ThreadLocalRandom.current().nextLong(1, 10L); final long index = ThreadLocalRandom.current().nextLong(1000L); File file = simpleStateMachineStorage.getSnapshotFile(term, index); - Assert.assertTrue(file.createNewFile()); + Assertions.assertTrue(file.createNewFile()); } simpleStateMachineStorage.cleanupOldSnapshots(new SnapshotRetentionPolicy() { }); @@ -297,8 +297,8 @@ public int getNumSnapshotsRetained() { private static File[] assertFileCount(File dir, int expected) { File[] files = dir.listFiles(); - Assert.assertNotNull(files); - Assert.assertEquals(Arrays.toString(files), expected, files.length); + Assertions.assertNotNull(files); + Assertions.assertEquals(expected, files.length, Arrays.toString(files)); return files; } @@ -309,6 +309,6 @@ public void testNotEnoughSpace() throws IOException { final RaftStorageDirectoryImpl sd = new RaftStorageDirectoryImpl(mockStorageDir, SizeInBytes.valueOf("100M")); StorageState state = sd.analyzeStorage(false); - Assert.assertEquals(StorageState.NO_SPACE, state); + Assertions.assertEquals(StorageState.NO_SPACE, state); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java index e8661e2689..4f75f793fc 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java @@ -22,9 +22,9 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -55,7 +55,7 @@ static File chooseNewStorageDir(List volumes, String sub) throws IOExcepti return new File(vol, sub); } - @AfterClass + @AfterAll public static void tearDown() throws IOException { FileUtils.deleteFully(rootTestDir.get()); } @@ -75,9 +75,9 @@ public void testChooseStorageDirWithOneVolume() throws IOException { final File storageDirTwo = chooseNewStorageDir(directories, subDirTwo); File expectedOne = new File(testDir, subDirOne); File expectedTwo = new File(testDir, subDirTwo); - Assert.assertEquals(expectedOne.getCanonicalPath(), + Assertions.assertEquals(expectedOne.getCanonicalPath(), storageDirOne.getCanonicalPath()); - Assert.assertEquals(expectedTwo.getCanonicalPath(), + Assertions.assertEquals(expectedTwo.getCanonicalPath(), storageDirTwo.getCanonicalPath()); } @@ -113,7 +113,7 @@ public void testChooseStorageDirWithMultipleVolumes() throws IOException { String subDir = UUID.randomUUID().toString(); final File storageDirectory = chooseNewStorageDir(directories, subDir); File expected = new File(directories.get(6), subDir); - Assert.assertEquals(expected.getCanonicalPath(), + Assertions.assertEquals(expected.getCanonicalPath(), storageDirectory.getCanonicalPath()); } @@ -124,10 +124,10 @@ public void testChooseStorageDirWithMultipleVolumes() throws IOException { public void testChooseStorageDirWithNoVolume() { try { chooseNewStorageDir(Collections.emptyList(), UUID.randomUUID().toString()); - Assert.fail(); + Assertions.fail(); } catch (IOException ex) { String expectedErrMsg = "No storage directory found."; - Assert.assertEquals(expectedErrMsg, ex.getMessage()); + Assertions.assertEquals(expectedErrMsg, ex.getMessage()); } } @@ -144,7 +144,7 @@ public void testAutoFormatSingleDirectory() throws Exception { final RaftStorageImpl storage = StorageImplUtils.initRaftStorage( "group-1", RaftStorage.StartupOption.RECOVER, properties); - Assert.assertNotNull(storage); + Assertions.assertNotNull(storage); storage.close(); } @@ -172,6 +172,6 @@ public void testAutoFormatMultiDirectories() throws Exception { () -> StorageImplUtils.initRaftStorage( "group-1", RaftStorage.StartupOption.RECOVER, properties), IOException.class); - Assert.assertTrue(ioe.getMessage().contains("Failed to RECOVER: Storage directory not found")); + Assertions.assertTrue(ioe.getMessage().contains("Failed to RECOVER: Storage directory not found")); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java index e708153aa5..a61bb19f73 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java @@ -30,8 +30,8 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.concurrent.TimeUnit; @@ -63,13 +63,13 @@ void runTestElectionTransferCommand(MiniRaftCluster cluster) throws Exception { RaftServer.Division newLeader = cluster.getFollowers().get(0); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); int ret = shell.run("election", "transfer", "-peers", address, "-address", newLeader.getPeer().getAddress()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); JavaUtils.attempt(() -> { - Assert.assertEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertEquals(cluster.getLeader().getId(), newLeader.getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testElectionTransferCommand", LOG); } @@ -85,28 +85,28 @@ void runTestElectionTransferCommandToHigherPriority(MiniRaftCluster cluster) thr RaftServer.Division newLeader = cluster.getFollowers().get(0); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertTrue(cluster.getFollowers().contains(newLeader)); + Assertions.assertTrue(cluster.getFollowers().contains(newLeader)); // set current leader's priority to 2 int ret = shell.run("peer", "setPriority", "-peers", address, "-addressPriority", leader.getPeer().getAddress()+ "|" + 2); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); // transfer to new leader will set its priority to 2 (with timeout 1s) ret = shell.run("election", "transfer", "-peers", address, "-address", newLeader.getPeer().getAddress(), "-timeout", "1"); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); - JavaUtils.attempt(() -> Assert.assertEquals(cluster.getLeader().getId(), newLeader.getId()), + JavaUtils.attempt(() -> Assertions.assertEquals(cluster.getLeader().getId(), newLeader.getId()), 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testElectionTransferLeaderCommand", LOG); // verify that priorities of new leader and old leader are both 2 ret = shell.run("group", "info", "-peers", address); - Assert.assertEquals(0 , ret); + Assertions.assertEquals(0 , ret); String expected = String.format("\"%s\"%n priority: %d", newLeader.getPeer().getAddress(), 2); String expected2 = String.format("\"%s\"%n priority: %d", leader.getPeer().getAddress(), 2); - Assert.assertTrue(out.toString().contains(expected)); - Assert.assertTrue(out.toString().contains(expected2)); + Assertions.assertTrue(out.toString().contains(expected)); + Assertions.assertTrue(out.toString().contains(expected2)); } @Test @@ -120,25 +120,25 @@ void runTestElectionPauseResumeCommand(MiniRaftCluster cluster) throws Exception RaftServer.Division newLeader = cluster.getFollowers().get(0); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); int ret = shell.run("election", "pause", "-peers", sb.toString(), "-address", newLeader.getPeer().getAddress()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); ret = shell.run("peer", "setPriority", "-peers", sb.toString(), "-addressPriority", newLeader.getPeer().getAddress() + "|" + 2); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); JavaUtils.attempt(() -> { - Assert.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testElectionPauseResumeCommand", LOG); ret = shell.run("election", "resume", "-peers", sb.toString(), "-address", newLeader.getPeer().getAddress()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); JavaUtils.attempt(() -> { - Assert.assertEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertEquals(cluster.getLeader().getId(), newLeader.getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testElectionPauseResumeCommand", LOG); } @@ -153,10 +153,10 @@ void runTestElectionStepDownCommand(MiniRaftCluster cluster) throws Exception { RaftServer.Division newLeader = cluster.getFollowers().get(0); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); - Assert.assertEquals(2, cluster.getFollowers().size()); + Assertions.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertEquals(2, cluster.getFollowers().size()); int ret = shell.run("election", "stepDown", "-peers", sb.toString()); - Assert.assertEquals(0, ret); - Assert.assertEquals(3, cluster.getFollowers().size()); + Assertions.assertEquals(0, ret); + Assertions.assertEquals(3, cluster.getFollowers().size()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java index c745307d2d..83c05bfddd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java @@ -29,8 +29,8 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; public abstract class GroupCommandIntegrationTest @@ -64,11 +64,11 @@ void runTestGroupListCommand(MiniRaftCluster cluster) throws Exception { RatisShell shell = new RatisShell(out.getPrintStream()); int ret = shell.run("group", "list", "-peers", address, "-peerId", leader.getPeer().getId().toString()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); String info = out.toString().trim(); String expected = String.format("The peerId %s (server %s) is in 1 groups, and the groupIds is: [%s]", leader.getId(), leader.getPeer().getAddress(), leader.getGroup().getGroupId()); - Assert.assertEquals(expected, info); + Assertions.assertEquals(expected, info); } @Test @@ -82,12 +82,12 @@ void runTestGroupInfoCommand(MiniRaftCluster cluster) throws Exception { final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); int ret = shell.run("group", "info", "-peers", address); - Assert.assertEquals(0 , ret); + Assertions.assertEquals(0 , ret); String result = out.toString().trim(); String hearder = String.format("group id: %s%sleader info: %s(%s)%s%s", cluster.getGroupId().getUuid(), NEW_LINE, leader.getId(), cluster.getLeader().getPeer().getAddress(), NEW_LINE, NEW_LINE); String info = result.substring(0, hearder.length()); - Assert.assertEquals(hearder, info); + Assertions.assertEquals(hearder, info); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java index 80400309ec..b3c35147a9 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java @@ -33,8 +33,8 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.concurrent.TimeUnit; @@ -68,11 +68,11 @@ void runTestPeerRemoveCommand(MiniRaftCluster cluster) throws Exception { RaftPeer[] peers = new RaftPeer[]{cluster.getFollowers().get(1).getPeer(), leader.getPeer()}; final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertTrue(cluster.getFollowers().contains(toRemove)); + Assertions.assertTrue(cluster.getFollowers().contains(toRemove)); int ret = shell.run("peer", "remove", "-peers", address, "-peerId", toRemove.getPeer().getId().toString()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); RaftServerTestUtil.waitAndCheckNewConf(cluster, peers,1, null); } @@ -95,7 +95,7 @@ void runTestPeerAddCommand(MiniRaftCluster cluster) throws Exception { int ret = shell.run("peer", "add", "-peers", sb.toString(), "-address", newPeers[0].getAdminAddress(), "-peerId", newPeers[0].getId().toString()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); RaftServerTestUtil.waitAndCheckNewConf(cluster, ObjectArrays.concat(peers, newPeers[0]), 0, null); } @@ -112,12 +112,12 @@ void runTestPeerSetPriorityCommand(MiniRaftCluster cluster) throws Exception { RaftServer.Division newLeader = cluster.getFollowers().get(0); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertTrue(cluster.getFollowers().contains(newLeader)); + Assertions.assertTrue(cluster.getFollowers().contains(newLeader)); int ret = shell.run("peer", "setPriority", "-peers", address, "-addressPriority", newLeader.getPeer().getAddress()+ "|" + 2); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); JavaUtils.attempt(() -> { - Assert.assertEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertEquals(cluster.getLeader().getId(), newLeader.getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testPeerSetPriorityCommand", LOG); } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java index 3c0ee9772c..f3f4e43077 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java @@ -30,8 +30,8 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.File; @@ -67,7 +67,7 @@ void runTestSnapshotCreateCommand(MiniRaftCluster cluster) throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } final String address = getClusterAddress(cluster); @@ -75,14 +75,14 @@ void runTestSnapshotCreateCommand(MiniRaftCluster cluster) throws Exception { RatisShell shell = new RatisShell(out.getPrintStream()); int ret = shell.run("snapshot", "create", "-peers", address, "-peerId", leader.getPeer().getId().toString()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); String[] str = out.toString().trim().split(" "); int snapshotIndex = Integer.parseInt(str[str.length-1]); LOG.info("snapshotIndex = {}", snapshotIndex); final File snapshotFile = SimpleStateMachine4Testing.get(leader) .getStateMachineStorage().getSnapshotFile(leader.getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } void runTestSnapshotCreateCommandOnSpecificServer(MiniRaftCluster cluster) throws Exception { @@ -91,16 +91,16 @@ void runTestSnapshotCreateCommandOnSpecificServer(MiniRaftCluster cluster) throw try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } final String address = getClusterAddress(cluster); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertEquals(2, cluster.getFollowers().size()); + Assertions.assertEquals(2, cluster.getFollowers().size()); int ret = shell.run("snapshot", "create", "-peers", address, "-peerId", cluster.getFollowers().get(0).getId().toString()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); String[] str = out.toString().trim().split(" "); int snapshotIndex = Integer.parseInt(str[str.length-1]); LOG.info("snapshotIndex = {}", snapshotIndex); @@ -108,7 +108,7 @@ void runTestSnapshotCreateCommandOnSpecificServer(MiniRaftCluster cluster) throw final File snapshotFile = SimpleStateMachine4Testing.get(cluster.getFollowers().get(0)) .getStateMachineStorage() .getSnapshotFile(cluster.getFollowers().get(0).getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java index 6e22276475..21f4685121 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java @@ -27,8 +27,8 @@ import org.apache.ratis.shell.cli.sh.command.PeerCommand; import org.apache.ratis.shell.cli.sh.command.SnapshotCommand; import org.apache.ratis.util.ReflectionUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.reflections.Reflections; import java.io.PrintStream; @@ -49,12 +49,12 @@ public class TestRatisShell extends BaseTest { static final Class[] ARG_CLASSES = new Class[] {Context.class}; static void assertCommand(String message, Command expected, Command computed) { - Assert.assertEquals(message, expected.getClass(), computed.getClass()); - Assert.assertEquals(message, expected.getCommandName(), computed.getCommandName()); + Assertions.assertEquals(expected.getClass(), computed.getClass(), message); + Assertions.assertEquals(expected.getCommandName(), computed.getCommandName(), message); } static void assertCommands(List expected, List computed) { - Assert.assertEquals(expected.size(), computed.size()); + Assertions.assertEquals(expected.size(), computed.size()); for(int i = 0; i < expected.size(); i++) { assertCommand("Command " + i, expected.get(i), computed.get(i)); } @@ -103,7 +103,7 @@ static void runTestFullCommandList(Function pare final List computed = new ArrayList<>(parent.getSubCommands().values()); Collections.sort(computed); - Assert.assertFalse(computed.isEmpty()); + Assertions.assertFalse(computed.isEmpty()); final Package pkg = computed.iterator().next().getClass().getPackage(); final List expected = new ArrayList<>(loadCommands(pkg)); diff --git a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java index 8e37947d3b..094189827d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java +++ b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java @@ -35,7 +35,8 @@ import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.util.Slf4jUtils; -import org.junit.*; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.Collections; @@ -48,7 +49,10 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Test StateMachine related functionality @@ -162,9 +166,9 @@ static void runTestTransactionContextIsPassedBack(MiniRaftCluster cluster) throw final SMTransactionContext sm = SMTransactionContext.get(raftServer); final List ll = new ArrayList<>(sm.applied); Collections.sort(ll); - assertEquals(ll.toString(), ll.size(), numTrx); + assertEquals(ll.size(), numTrx, ll.toString()); for (int i=0; i < numTrx; i++) { - assertEquals(ll.toString(), Long.valueOf(i+1), ll.get(i)); + assertEquals(Long.valueOf(i+1), ll.get(i), ll.toString()); } } @@ -193,7 +197,7 @@ public void testStateMachineRegistry() throws Throwable { final RaftServer server = cluster.getServer(id); for(Map.Entry e: registry.entrySet()) { - Assert.assertSame(e.getValue(), server.getDivision(e.getKey()).getStateMachine()); + Assertions.assertSame(e.getValue(), server.getDivision(e.getKey()).getStateMachine()); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java b/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java index df0dd2f8ed..e9dcf61d09 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java @@ -17,8 +17,9 @@ */ package org.apache.ratis.util; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,34 +37,38 @@ public class TestDataBlockingQueue { final TimeDuration slow = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); final TimeDuration fast = TimeDuration.valueOf(10, TimeUnit.MILLISECONDS); - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testElementLimit() { TestDataQueue.runTestElementLimit(q); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testByteLimit() { TestDataQueue.runTestByteLimit(q); } - @Test(timeout = 10_000) + @Test + @Timeout(value = 1000) public void testSlowOfferFastPoll() throws Exception { runTestBlockingCalls(slow, fast, q); } - @Test(timeout = 10_000) + @Test + @Timeout(value = 1000) public void testFastOfferSlowPoll() throws Exception { runTestBlockingCalls(fast, slow, q); } static void assertOfferPull(long offering, long polled, long elementLimit) { - Assert.assertTrue(offering >= polled); - Assert.assertTrue(offering - polled <= elementLimit + 1); + Assertions.assertTrue(offering >= polled); + Assertions.assertTrue(offering - polled <= elementLimit + 1); } static void runTestBlockingCalls(TimeDuration offerSleepTime, TimeDuration pollSleepTime, DataBlockingQueue q) throws Exception { - Assert.assertTrue(q.isEmpty()); + Assertions.assertTrue(q.isEmpty()); ExitUtils.disableSystemExit(); final int elementLimit = q.getElementLimit(); final TimeDuration timeout = CollectionUtils.min(offerSleepTime, pollSleepTime); @@ -74,11 +79,11 @@ static void runTestBlockingCalls(TimeDuration offerSleepTime, TimeDuration pollS final Thread pollThread = new Thread(() -> { try { - for(; polledValue.get() < endValue;) { + while (polledValue.get() < endValue) { pollSleepTime.sleep(); final Long polled = q.poll(timeout); if (polled != null) { - Assert.assertEquals(polledValue.incrementAndGet(), polled.intValue()); + Assertions.assertEquals(polledValue.incrementAndGet(), polled.intValue()); LOG.info("polled {}", polled); } assertOfferPull(offeringValue.get(), polledValue.get(), elementLimit); @@ -109,10 +114,10 @@ static void runTestBlockingCalls(TimeDuration offerSleepTime, TimeDuration pollS offerThread.join(); pollThread.join(); - Assert.assertEquals(endValue + 1, offeringValue.get()); - Assert.assertEquals(endValue, polledValue.get()); + Assertions.assertEquals(endValue + 1, offeringValue.get()); + Assertions.assertEquals(endValue, polledValue.get()); - Assert.assertTrue(q.isEmpty()); + Assertions.assertTrue(q.isEmpty()); ExitUtils.assertNotTerminated(); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java b/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java index fe8a7ac246..2dafa7e91c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java @@ -18,8 +18,9 @@ package org.apache.ratis.util; import org.apache.ratis.util.function.TriConsumer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.Collections; @@ -37,15 +38,16 @@ static TriConsumer getTimeoutHandler(bool } static void assertSizes(long expectedNumElements, long expectedNumBytes, DataQueue q) { - Assert.assertEquals(expectedNumElements, q.getNumElements()); - Assert.assertEquals(expectedNumBytes, q.getNumBytes()); + Assertions.assertEquals(expectedNumElements, q.getNumElements()); + Assertions.assertEquals(expectedNumBytes, q.getNumBytes()); } final SizeInBytes byteLimit = SizeInBytes.valueOf(100); final int elementLimit = 5; final DataQueue q = new DataQueue<>(null, byteLimit, elementLimit, Long::longValue); - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testElementLimit() { runTestElementLimit(q); } @@ -56,30 +58,31 @@ static void runTestElementLimit(DataQueue q) { final int elementLimit = q.getElementLimit(); long numBytes = 0; for (long i = 0; i < elementLimit; i++) { - Assert.assertEquals(i, q.getNumElements()); - Assert.assertEquals(numBytes, q.getNumBytes()); + Assertions.assertEquals(i, q.getNumElements()); + Assertions.assertEquals(numBytes, q.getNumBytes()); final boolean offered = q.offer(i); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); numBytes += i; assertSizes(i+1, numBytes, q); } { final boolean offered = q.offer(0L); - Assert.assertFalse(offered); + Assertions.assertFalse(offered); assertSizes(elementLimit, numBytes, q); } { // poll all elements final List polled = q.pollList(100, (i, timeout) -> i, getTimeoutHandler(false)); - Assert.assertEquals(elementLimit, polled.size()); + Assertions.assertEquals(elementLimit, polled.size()); for (int i = 0; i < polled.size(); i++) { - Assert.assertEquals(i, polled.get(i).intValue()); + Assertions.assertEquals(i, polled.get(i).intValue()); } } assertSizes(0, 0, q); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testByteLimit() { runTestByteLimit(q); } @@ -90,53 +93,54 @@ static void runTestByteLimit(DataQueue q) { final long byteLimit = q.getByteLimit(); try { q.offer(byteLimit + 1); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException ignored) { } final long halfBytes = byteLimit / 2; { final boolean offered = q.offer(halfBytes); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); assertSizes(1, halfBytes, q); } { final boolean offered = q.offer(halfBytes + 1); - Assert.assertFalse(offered); + Assertions.assertFalse(offered); assertSizes(1, halfBytes, q); } { final boolean offered = q.offer(halfBytes); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); assertSizes(2, byteLimit, q); } { final boolean offered = q.offer(1L); - Assert.assertFalse(offered); + Assertions.assertFalse(offered); assertSizes(2, byteLimit, q); } { final boolean offered = q.offer(0L); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); assertSizes(3, byteLimit, q); } { // poll all elements final List polled = q.pollList(100, (i, timeout) -> i, getTimeoutHandler(false)); - Assert.assertEquals(3, polled.size()); - Assert.assertEquals(halfBytes, polled.get(0).intValue()); - Assert.assertEquals(halfBytes, polled.get(1).intValue()); - Assert.assertEquals(0, polled.get(2).intValue()); + Assertions.assertEquals(3, polled.size()); + Assertions.assertEquals(halfBytes, polled.get(0).intValue()); + Assertions.assertEquals(halfBytes, polled.get(1).intValue()); + Assertions.assertEquals(0, polled.get(2).intValue()); } assertSizes(0, 0, q); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testIteratorAndRemove() { runTestIteratorAndRemove(q); } @@ -149,7 +153,7 @@ static void runTestIteratorAndRemove(DataQueue q) { long numBytes = 0; for(long i = 0; i < elementLimit; i++) { final boolean offered = q.offer(i); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); numElements++; numBytes += i; assertSizes(numElements, numBytes, q); @@ -158,7 +162,7 @@ static void runTestIteratorAndRemove(DataQueue q) { { // test iterator() final Iterator i = q.iterator(); for (long expected = 0; expected < elementLimit; expected++) { - Assert.assertEquals(expected, i.next().longValue()); + Assertions.assertEquals(expected, i.next().longValue()); } } @@ -180,23 +184,24 @@ static void runTestIteratorAndRemove(DataQueue q) { assertSizes(0, 0, q); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testTimeout() { assertSizes(0, 0, q); long numBytes = 0; for (long i = 0; i < elementLimit; i++) { - Assert.assertEquals(i, q.getNumElements()); - Assert.assertEquals(numBytes, q.getNumBytes()); + Assertions.assertEquals(i, q.getNumElements()); + Assertions.assertEquals(numBytes, q.getNumBytes()); final boolean offered = q.offer(i); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); numBytes += i; assertSizes(i+1, numBytes, q); } { // poll with zero time final List polled = q.pollList(0, (i, timeout) -> i, getTimeoutHandler(false)); - Assert.assertTrue(polled.isEmpty()); + Assertions.assertTrue(polled.isEmpty()); assertSizes(elementLimit, numBytes, q); } @@ -209,9 +214,9 @@ public void testTimeout() { } return i; }, getTimeoutHandler(true)); - Assert.assertEquals(halfElements, polled.size()); + Assertions.assertEquals(halfElements, polled.size()); for (int i = 0; i < polled.size(); i++) { - Assert.assertEquals(i, polled.get(i).intValue()); + Assertions.assertEquals(i, polled.get(i).intValue()); numBytes -= i; } assertSizes(elementLimit - halfElements, numBytes, q); @@ -219,9 +224,9 @@ public void testTimeout() { { // poll the remaining elements final List polled = q.pollList(100, (i, timeout) -> i, getTimeoutHandler(false)); - Assert.assertEquals(elementLimit - halfElements, polled.size()); + Assertions.assertEquals(elementLimit - halfElements, polled.size()); for (int i = 0; i < polled.size(); i++) { - Assert.assertEquals(halfElements + i, polled.get(i).intValue()); + Assertions.assertEquals(halfElements + i, polled.get(i).intValue()); } } assertSizes(0, 0, q); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java b/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java index 25b05fd0df..1dd5b270bb 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java @@ -19,15 +19,17 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.util.ExitUtils.ExitException; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestExitUtils extends BaseTest { /** Test if {@link BaseTest} can handle uncaught exception. */ - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testUncaughtException() throws Exception { - Assert.assertFalse(ExitUtils.isTerminated()); - Assert.assertFalse(ExitUtils.clear()); + Assertions.assertFalse(ExitUtils.isTerminated()); + Assertions.assertFalse(ExitUtils.clear()); final Thread t = new Thread(null, () -> { throw new AssertionError("Testing"); @@ -35,25 +37,26 @@ public void testUncaughtException() throws Exception { t.start(); t.join(); - Assert.assertTrue(ExitUtils.isTerminated()); - Assert.assertTrue(ExitUtils.clear()); + Assertions.assertTrue(ExitUtils.isTerminated()); + Assertions.assertTrue(ExitUtils.clear()); } /** Test if {@link BaseTest} can handle ExitUtils.terminate(..). */ - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testExitStatus() { - Assert.assertFalse(ExitUtils.isTerminated()); - Assert.assertFalse(ExitUtils.clear()); + Assertions.assertFalse(ExitUtils.isTerminated()); + Assertions.assertFalse(ExitUtils.clear()); final int status = -1; try { ExitUtils.terminate(status, "testExitStatus", LOG); - Assert.fail(); + Assertions.fail(); } catch (ExitException e) { - Assert.assertEquals(status, e.getStatus()); + Assertions.assertEquals(status, e.getStatus()); } - Assert.assertTrue(ExitUtils.isTerminated()); - Assert.assertTrue(ExitUtils.clear()); + Assertions.assertTrue(ExitUtils.isTerminated()); + Assertions.assertTrue(ExitUtils.clear()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java index 3faf2a4959..f817449995 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java @@ -18,12 +18,13 @@ package org.apache.ratis.util; import org.apache.ratis.util.function.TriConsumer; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import static org.apache.ratis.util.LifeCycle.State.*; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.*; @@ -33,7 +34,8 @@ public class TestLifeCycle { * {@link LifeCycle} uses predecessors to validate transitions * while this test uses successors. */ - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testIsValid() { final Map> successors = new EnumMap<>(LifeCycle.State.class); @@ -49,9 +51,8 @@ public void testIsValid() { final List states = Arrays.asList(LifeCycle.State.values()); states.forEach( from -> states.forEach( - to -> assertEquals(from + " -> " + to, - successors.get(from).contains(to), - isValid(from, to)))); + to -> assertEquals(successors.get(from).contains(to), + isValid(from, to), from + " -> " + to))); } @Test @@ -90,7 +91,7 @@ private static void testInvalidTransition(TriConsumer n).limit(10)); @@ -43,11 +45,11 @@ static void runTestMinMax(LongStream stream) { final List list = stream.collect(ArrayList::new, List::add, List::addAll); final LongMinMax longMinMax = toLongStream(list).collect(LongMinMax::new, LongMinMax::accumulate, LongMinMax::combine); if (longMinMax.isInitialized()) { - Assert.assertEquals(toLongStream(list).min().getAsLong(), longMinMax.getMin()); - Assert.assertEquals(toLongStream(list).max().getAsLong(), longMinMax.getMax()); + Assertions.assertEquals(toLongStream(list).min().getAsLong(), longMinMax.getMin()); + Assertions.assertEquals(toLongStream(list).max().getAsLong(), longMinMax.getMax()); } else { - Assert.assertEquals(OptionalLong.empty(), toLongStream(list).min()); - Assert.assertEquals(OptionalLong.empty(), toLongStream(list).max()); + Assertions.assertEquals(OptionalLong.empty(), toLongStream(list).min()); + Assertions.assertEquals(OptionalLong.empty(), toLongStream(list).max()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java b/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java index 549fbc53f8..118830bda8 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java @@ -20,8 +20,9 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.Closeable; import java.io.IOException; @@ -51,7 +52,8 @@ public String toString() { } } - @Test(timeout = 10_000) + @Test + @Timeout(value = 10_000) public void testCloseDeadLock() throws Exception { final PeerProxyMap map = new PeerProxyMap<>("test", DummyProxy::new); final RaftPeerId id = RaftPeerId.valueOf("s0"); @@ -68,7 +70,7 @@ public void testCloseDeadLock() throws Exception { HUNDRED_MILLIS.sleep(); LOG.info("Try getProxy"); final DummyProxy newProxy = map.getProxy(id); - Assert.assertNotSame(proxy, newProxy); + Assertions.assertNotSame(proxy, newProxy); } catch (Exception e) { setFirstException(e); } @@ -115,7 +117,8 @@ public String toString() { } } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testStackTrace() { final RaftPeerId id = RaftPeerId.valueOf("s0"); final RaftPeer peer = RaftPeer.newBuilder().setId(id).build(); @@ -123,7 +126,7 @@ public void testStackTrace() { final ExceptionProxy ignored = map.computeIfAbsent(peer).get()) { } catch (IOException e) { assertThrowable("closeProxy", e, AnnotatedConnectException.class, LOG, ConnectException.class); - Assert.assertEquals(0, e.getStackTrace().length); + Assertions.assertEquals(0, e.getStackTrace().length); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java b/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java index 884c1e5d59..e99c11fdde 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java @@ -18,7 +18,8 @@ package org.apache.ratis.util; import org.apache.ratis.BaseTest; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.Arrays; import java.util.Collections; @@ -28,7 +29,8 @@ import java.util.Set; public class TestPreconditions extends BaseTest { - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testAssertUnique() { final Set empty = Collections.emptySet(); Preconditions.assertUnique(empty); @@ -53,7 +55,8 @@ public void testAssertUnique() { Preconditions.assertUnique(three, Arrays.asList(4, 5, 6)); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testAssertNull() { final Map map = new HashMap<>(); final String key = "abc1234"; diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java b/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java index fe58b92da6..502261ec23 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java @@ -18,8 +18,9 @@ package org.apache.ratis.util; import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.concurrent.atomic.AtomicInteger; @@ -27,8 +28,8 @@ public class TestReferenceCountedObject { static void assertValues( AtomicInteger retained, int expectedRetained, AtomicInteger released, int expectedReleased) { - Assert.assertEquals("retained", expectedRetained, retained.get()); - Assert.assertEquals("released", expectedReleased, released.get()); + Assertions.assertEquals(expectedRetained, retained.get(), "retained"); + Assertions.assertEquals(expectedReleased, released.get(), "retained"); } static void assertRelease(ReferenceCountedObject ref, @@ -36,10 +37,11 @@ static void assertRelease(ReferenceCountedObject ref, AtomicInteger released, int expectedReleased) { final boolean returned = ref.release(); assertValues(retained, expectedRetained, released, expectedReleased); - Assert.assertEquals(expectedRetained == expectedReleased, returned); + Assertions.assertEquals(expectedRetained == expectedReleased, returned); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testWrap() { final String value = "testWrap"; final AtomicInteger retained = new AtomicInteger(); @@ -50,19 +52,19 @@ public void testWrap() { assertValues(retained, 0, released, 0); try { ref.get(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } assertValues(retained, 0, released, 0); - Assert.assertEquals(value, ref.retain()); + Assertions.assertEquals(value, ref.retain()); assertValues(retained, 1, released, 0); try(UncheckedAutoCloseableSupplier auto = ref.retainAndReleaseOnClose()) { final String got = auto.get(); - Assert.assertEquals(value, got); - Assert.assertSame(got, auto.get()); // it should return the same object. + Assertions.assertEquals(value, got); + Assertions.assertSame(got, auto.get()); // it should return the same object. assertValues(retained, 2, released, 0); } catch (IllegalStateException e) { e.printStackTrace(System.out); @@ -70,12 +72,12 @@ public void testWrap() { assertValues(retained, 2, released, 1); final UncheckedAutoCloseableSupplier notClosing = ref.retainAndReleaseOnClose(); - Assert.assertEquals(value, notClosing.get()); + Assertions.assertEquals(value, notClosing.get()); assertValues(retained, 3, released, 1); assertRelease(ref, retained, 3, released, 2); final UncheckedAutoCloseableSupplier auto = ref.retainAndReleaseOnClose(); - Assert.assertEquals(value, auto.get()); + Assertions.assertEquals(value, auto.get()); assertValues(retained, 4, released, 2); auto.close(); assertValues(retained, 4, released, 3); @@ -87,59 +89,60 @@ public void testWrap() { try { ref.get(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try { ref.retain(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try(UncheckedAutoCloseable ignore = ref.retainAndReleaseOnClose()) { - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try { ref.release(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testReleaseWithoutRetaining() { final ReferenceCountedObject ref = ReferenceCountedObject.wrap(""); try { ref.release(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try { ref.get(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try { ref.retain(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try(UncheckedAutoCloseable ignore = ref.retainAndReleaseOnClose()) { - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java b/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java index 6fe1aed7ef..0f3612dce4 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java @@ -19,15 +19,17 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.RaftTestUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.concurrent.TimeoutException; import static org.apache.ratis.util.ResourceSemaphore.Group.SUCCESS; public class TestResourceSemaphore extends BaseTest { - @Test(timeout = 5000) + @Test + @Timeout(value = 5000) public void testGroup() throws InterruptedException, TimeoutException { final int FAILED_IN_ELEMENT_LIMIT = 0; final int FAILED_IN_BYTE_SIZE_LIMIT = 1; @@ -80,15 +82,15 @@ public void testGroup() throws InterruptedException, TimeoutException { } static void assertUsed(ResourceSemaphore.Group g, int... expected) { - Assert.assertEquals(expected.length, g.resourceSize()); + Assertions.assertEquals(expected.length, g.resourceSize()); for(int i = 0; i < expected.length; i++) { - Assert.assertEquals(expected[i], g.get(i).used()); + Assertions.assertEquals(expected[i], g.get(i).used()); } } static void assertAcquire(ResourceSemaphore.Group g, int expected, int... permits) { final int computed = g.tryAcquire(permits); - Assert.assertEquals(expected, computed); + Assertions.assertEquals(expected, computed); } static Runnable acquire(ResourceSemaphore.Group g, int... permits) { diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java index a2c180d922..f2d58f48db 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java @@ -17,8 +17,9 @@ */ package org.apache.ratis.util; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.util.Arrays; @@ -28,17 +29,18 @@ import static org.apache.ratis.util.TimeDuration.Abbreviation; import static org.apache.ratis.util.TimeDuration.parse; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestTimeDuration { { Slf4jUtils.setLogLevel(TimeDuration.LOG, Level.DEBUG); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testAbbreviation() { Arrays.asList(TimeUnit.values()) .forEach(a -> assertNotNull(Abbreviation.valueOf(a.name()))); @@ -51,10 +53,11 @@ public void testAbbreviation() { Arrays.asList(TimeUnit.values()).forEach(unit -> allSymbols.stream() .map(s -> "0" + s) - .forEach(s -> assertEquals(s, 0L, parse(s, unit)))); + .forEach(s -> assertEquals(0L, parse(s, unit), s))); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testParse() { assertEquals(1L, parse("1_000_000 ns", TimeUnit.MILLISECONDS)); assertEquals(10L, parse("10_000_000 nanos", TimeUnit.MILLISECONDS)); @@ -93,7 +96,8 @@ public void testParse() { assertEquals(2400, parse("100 days", TimeUnit.HOURS)); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testRoundUp() { final long nanosPerSecond = 1_000_000_000L; final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); @@ -108,7 +112,8 @@ public void testRoundUp() { assertEquals(2*nanosPerSecond, oneSecond.roundUpNanos(nanosPerSecond + 1)); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testTo() { final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); assertTo(1000, "1000ms", oneSecond, TimeUnit.MILLISECONDS); @@ -131,7 +136,8 @@ static TimeDuration assertTo(long expected, String expectedString, TimeDuration return computed; } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testAddAndSubtract() { final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); final TimeDuration tenSecond = TimeDuration.valueOf(10, TimeUnit.SECONDS); @@ -184,28 +190,30 @@ public void testAddAndSubtract() { } } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testNegate() { assertNegate(0); assertNegate(1); assertNegate(-1); assertNegate(Long.MAX_VALUE); - Assert.assertEquals( + Assertions.assertEquals( TimeDuration.valueOf(Long.MAX_VALUE, TimeUnit.SECONDS), TimeDuration.valueOf(Long.MIN_VALUE, TimeUnit.SECONDS).negate()); } private static void assertNegate(long n) { - Assert.assertEquals( + Assertions.assertEquals( TimeDuration.valueOf(-n, TimeUnit.SECONDS), TimeDuration.valueOf(n, TimeUnit.SECONDS).negate()); - Assert.assertEquals( + Assertions.assertEquals( TimeDuration.valueOf(n, TimeUnit.SECONDS), TimeDuration.valueOf(-n, TimeUnit.SECONDS).negate()); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testMultiply() { assertMultiply(0, TimeDuration.ONE_SECOND, TimeDuration.valueOf(0, TimeUnit.SECONDS)); assertMultiply(0.001, TimeDuration.ONE_SECOND, TimeDuration.ONE_MILLISECOND); @@ -230,12 +238,13 @@ private static void assertMultiply(double multiplier, TimeDuration t, TimeDurati private static void assertMultiply(TimeDuration t, double multiplier, TimeDuration expected) { final TimeDuration computed = t.multiply(multiplier); TimeDuration.LOG.info("assertMultiply: {} x {} = {} ?= {}\n\n", t, multiplier, computed, expected); - Assert.assertEquals(expected.getUnit(), computed.getUnit()); + Assertions.assertEquals(expected.getUnit(), computed.getUnit()); final long d = Math.abs(computed.getDuration() - expected.getDuration()); - Assert.assertTrue(d <= Math.abs(expected.getDuration()) * TimeDuration.ERROR_THRESHOLD); + Assertions.assertTrue(d <= Math.abs(expected.getDuration()) * TimeDuration.ERROR_THRESHOLD); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testHigherLower() { final TimeUnit[] units = {TimeUnit.NANOSECONDS, TimeUnit.MICROSECONDS, TimeUnit.MILLISECONDS, TimeUnit.SECONDS, TimeUnit.MINUTES, TimeUnit.HOURS, TimeUnit.DAYS}; @@ -243,11 +252,12 @@ public void testHigherLower() { assertHigherLower(units[i-1], units[i]); } - Assert.assertSame(TimeUnit.NANOSECONDS, TimeDuration.lowerUnit(TimeUnit.NANOSECONDS)); - Assert.assertSame(TimeUnit.DAYS, TimeDuration.higherUnit(TimeUnit.DAYS)); + Assertions.assertSame(TimeUnit.NANOSECONDS, TimeDuration.lowerUnit(TimeUnit.NANOSECONDS)); + Assertions.assertSame(TimeUnit.DAYS, TimeDuration.higherUnit(TimeUnit.DAYS)); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testCompareTo() { assertTimeDurationCompareTo(TimeDuration.ONE_MINUTE, TimeDuration.ONE_SECOND); @@ -281,7 +291,7 @@ static void assertTimeDurationCompareTo(TimeDuration larger, TimeDuration smalle } private static void assertHigherLower(TimeUnit lower, TimeUnit higher) { - Assert.assertSame(lower, TimeDuration.lowerUnit(higher)); - Assert.assertSame(higher, TimeDuration.higherUnit(lower)); + Assertions.assertSame(lower, TimeDuration.lowerUnit(higher)); + Assertions.assertSame(higher, TimeDuration.higherUnit(lower)); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java index 848428f5b8..79c97a6ede 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java @@ -18,8 +18,9 @@ package org.apache.ratis.util; import org.apache.ratis.BaseTest; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.util.concurrent.TimeUnit; @@ -41,52 +42,54 @@ public void accept(RuntimeException e) { } void assertNoError() { - Assert.assertFalse(hasError.get()); + Assertions.assertFalse(hasError.get()); } } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testSingleTask() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); scheduler.setGracePeriod(grace); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertFalse(scheduler.hasScheduler()); final ErrorHandler errorHandler = new ErrorHandler(); final AtomicBoolean fired = new AtomicBoolean(false); scheduler.onTimeout(TimeDuration.valueOf(250, TimeUnit.MILLISECONDS), () -> { - Assert.assertFalse(fired.get()); + Assertions.assertFalse(fired.get()); fired.set(true); }, errorHandler); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertFalse(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertFalse(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertFalse(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertFalse(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired.get()); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertTrue(fired.get()); + Assertions.assertFalse(scheduler.hasScheduler()); errorHandler.assertNoError(); scheduler.setGracePeriod(grace); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testMultipleTasks() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); scheduler.setGracePeriod(grace); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertFalse(scheduler.hasScheduler()); final ErrorHandler errorHandler = new ErrorHandler(); @@ -94,126 +97,129 @@ public void testMultipleTasks() throws Exception { for(int i = 0; i < fired.length; i++) { final AtomicBoolean f = fired[i] = new AtomicBoolean(false); scheduler.onTimeout(TimeDuration.valueOf(100*i + 50, TimeUnit.MILLISECONDS), () -> { - Assert.assertFalse(f.get()); + Assertions.assertFalse(f.get()); f.set(true); }, errorHandler); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(scheduler.hasScheduler()); } Thread.sleep(100); - Assert.assertTrue(fired[0].get()); - Assert.assertFalse(fired[1].get()); - Assert.assertFalse(fired[2].get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired[0].get()); + Assertions.assertFalse(fired[1].get()); + Assertions.assertFalse(fired[2].get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired[0].get()); - Assert.assertTrue(fired[1].get()); - Assert.assertFalse(fired[2].get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired[0].get()); + Assertions.assertTrue(fired[1].get()); + Assertions.assertFalse(fired[2].get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired[0].get()); - Assert.assertTrue(fired[1].get()); - Assert.assertTrue(fired[2].get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired[0].get()); + Assertions.assertTrue(fired[1].get()); + Assertions.assertTrue(fired[2].get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired[0].get()); - Assert.assertTrue(fired[1].get()); - Assert.assertTrue(fired[2].get()); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertTrue(fired[0].get()); + Assertions.assertTrue(fired[1].get()); + Assertions.assertTrue(fired[2].get()); + Assertions.assertFalse(scheduler.hasScheduler()); errorHandler.assertNoError(); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testExtendingGracePeriod() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); scheduler.setGracePeriod(grace); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertFalse(scheduler.hasScheduler()); final ErrorHandler errorHandler = new ErrorHandler(); { final AtomicBoolean fired = new AtomicBoolean(false); scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> { - Assert.assertFalse(fired.get()); + Assertions.assertFalse(fired.get()); fired.set(true); }, errorHandler); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertFalse(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertFalse(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); } { // submit another task during grace period final AtomicBoolean fired2 = new AtomicBoolean(false); scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> { - Assert.assertFalse(fired2.get()); + Assertions.assertFalse(fired2.get()); fired2.set(true); }, errorHandler); Thread.sleep(100); - Assert.assertFalse(fired2.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertFalse(fired2.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired2.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired2.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired2.get()); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertTrue(fired2.get()); + Assertions.assertFalse(scheduler.hasScheduler()); } errorHandler.assertNoError(); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testRestartingScheduler() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); scheduler.setGracePeriod(grace); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertFalse(scheduler.hasScheduler()); final ErrorHandler errorHandler = new ErrorHandler(); for(int i = 0; i < 2; i++) { final AtomicBoolean fired = new AtomicBoolean(false); scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> { - Assert.assertFalse(fired.get()); + Assertions.assertFalse(fired.get()); fired.set(true); }, errorHandler); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertFalse(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertFalse(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired.get()); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertTrue(fired.get()); + Assertions.assertFalse(scheduler.hasScheduler()); } errorHandler.assertNoError(); } - @Test(timeout = 10_000) + @Test + @Timeout(value = 10_000) public void testShutdown() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); - Assert.assertEquals(TimeoutScheduler.DEFAULT_GRACE_PERIOD, scheduler.getGracePeriod()); + Assertions.assertEquals(TimeoutScheduler.DEFAULT_GRACE_PERIOD, scheduler.getGracePeriod()); final ErrorHandler errorHandler = new ErrorHandler(); final int numTasks = 100; @@ -223,7 +229,7 @@ public void testShutdown() throws Exception { } HUNDRED_MILLIS.sleep(); HUNDRED_MILLIS.sleep(); - JavaUtils.attempt(() -> Assert.assertEquals(1, scheduler.getTaskCount()), + JavaUtils.attempt(() -> Assertions.assertEquals(1, scheduler.getTaskCount()), 10, HUNDRED_MILLIS, "only 1 shutdown task is scheduled", LOG); final TimeDuration oneMillis = TimeDuration.valueOf(1, TimeUnit.MILLISECONDS); @@ -234,7 +240,7 @@ public void testShutdown() throws Exception { oneMillis.sleep(); } HUNDRED_MILLIS.sleep(); - JavaUtils.attempt(() -> Assert.assertEquals(1, scheduler.getTaskCount()), + JavaUtils.attempt(() -> Assertions.assertEquals(1, scheduler.getTaskCount()), 10, HUNDRED_MILLIS, "only 1 shutdown task is scheduled", LOG); errorHandler.assertNoError(); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java index 8b6b7a378f..988cc5c710 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java @@ -17,15 +17,18 @@ */ package org.apache.ratis.util; -import org.junit.Test; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import static org.apache.ratis.util.TraditionalBinaryPrefix.long2String; import static org.apache.ratis.util.TraditionalBinaryPrefix.string2long; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; public class TestTraditionalBinaryPrefix { - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testTraditionalBinaryPrefix() { //test string2long(..) String[] symbol = {"k", "m", "g", "t", "p", "e"}; @@ -108,19 +111,19 @@ public void testTraditionalBinaryPrefix() { { // n = 2^e final long n = 1L << e; final String expected = (n/p.getValue()) + " " + p.getSymbol(); - assertEquals("n=" + n, expected, long2String(n, null, 2)); + assertEquals(expected, long2String(n, null, 2), "n=" + n); } { // n = 2^e + 1 final long n = (1L << e) + 1; final String expected = (n/p.getValue()) + trailingZeros + p.getSymbol(); - assertEquals("n=" + n, expected, long2String(n, null, decimalPlace)); + assertEquals(expected, long2String(n, null, decimalPlace), "n=" + n); } { // n = 2^e - 1 final long n = (1L << e) - 1; final String expected = ((n+1)/p.getValue()) + trailingZeros + p.getSymbol(); - assertEquals("n=" + n, expected, long2String(n, null, decimalPlace)); + assertEquals(expected, long2String(n, null, decimalPlace), "n=" + n); } } } @@ -143,7 +146,8 @@ private static String byteDescription(long len) { return long2String(len, "B", 2); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testUnderscore() { final SizeInBytes value = SizeInBytes.valueOf("1_000_000_000_000_000"); assertEquals(1_000_000_000_000_000L, value.getSize()); From 044b111d991a619a7c2edcef235e28a7c623f090 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 21 Feb 2024 03:32:07 -0800 Subject: [PATCH 37/76] RATIS-2027. Ratis Streaming: Remote Stream copy data to heap. (#1044) * RATIS-2027. Ratis Streaming: Remote Stream copy data to heap. --- .../client/impl/DataStreamClientImpl.java | 7 ++++++- .../ratis/client/impl/OrderedStreamAsync.java | 4 ++++ .../impl}/DataStreamRequestByteBuf.java | 3 +-- .../ratis/netty/NettyDataStreamUtils.java | 16 +++++++++++++++- .../netty/client/NettyClientStreamRpc.java | 12 ++++++++++++ .../netty/server/DataStreamManagement.java | 19 ++++++++++--------- .../netty/server/NettyServerStreamRpc.java | 12 +++++++----- 7 files changed, 55 insertions(+), 18 deletions(-) rename {ratis-netty/src/main/java/org/apache/ratis/netty/server => ratis-common/src/main/java/org/apache/ratis/datastream/impl}/DataStreamRequestByteBuf.java (96%) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java index 26d01c356f..ba91866d71 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java @@ -40,6 +40,7 @@ import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; import org.apache.ratis.rpc.CallId; +import org.apache.ratis.thirdparty.io.netty.buffer.ByteBuf; import org.apache.ratis.util.IOUtils; import org.apache.ratis.protocol.*; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; @@ -169,6 +170,10 @@ private CompletableFuture writeAsyncImpl(Object data, long leng return f; } + public CompletableFuture writeAsync(ByteBuf src, Iterable options) { + return writeAsyncImpl(src, src.readableBytes(), options); + } + @Override public CompletableFuture writeAsync(ByteBuffer src, Iterable options) { return writeAsyncImpl(src, src.remaining(), options); @@ -235,7 +240,7 @@ public DataStreamClientRpc getClientRpc() { } @Override - public DataStreamOutputRpc stream(RaftClientRequest request) { + public DataStreamOutputImpl stream(RaftClientRequest request) { return new DataStreamOutputImpl(request); } diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedStreamAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedStreamAsync.java index 989c00cbbc..275755514f 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedStreamAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedStreamAsync.java @@ -21,12 +21,14 @@ import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.datastream.impl.DataStreamPacketByteBuffer; +import org.apache.ratis.datastream.impl.DataStreamRequestByteBuf; import org.apache.ratis.datastream.impl.DataStreamRequestByteBuffer; import org.apache.ratis.datastream.impl.DataStreamRequestFilePositionCount; import org.apache.ratis.io.FilePositionCount; import org.apache.ratis.protocol.DataStreamReply; import org.apache.ratis.protocol.DataStreamRequest; import org.apache.ratis.protocol.DataStreamRequestHeader; +import org.apache.ratis.thirdparty.io.netty.buffer.ByteBuf; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.SlidingWindow; @@ -56,6 +58,8 @@ static class DataStreamWindowRequest implements SlidingWindow.ClientSideRequest< DataStreamRequest getDataStreamRequest() { if (header.getDataLength() == 0) { return new DataStreamRequestByteBuffer(header, DataStreamPacketByteBuffer.EMPTY_BYTE_BUFFER); + } else if (data instanceof ByteBuf) { + return new DataStreamRequestByteBuf(header, (ByteBuf)data); } else if (data instanceof ByteBuffer) { return new DataStreamRequestByteBuffer(header, (ByteBuffer)data); } else if (data instanceof FilePositionCount) { diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamRequestByteBuf.java b/ratis-common/src/main/java/org/apache/ratis/datastream/impl/DataStreamRequestByteBuf.java similarity index 96% rename from ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamRequestByteBuf.java rename to ratis-common/src/main/java/org/apache/ratis/datastream/impl/DataStreamRequestByteBuf.java index 2542b1ec6f..1873bec9b4 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamRequestByteBuf.java +++ b/ratis-common/src/main/java/org/apache/ratis/datastream/impl/DataStreamRequestByteBuf.java @@ -16,9 +16,8 @@ * limitations under the License. */ -package org.apache.ratis.netty.server; +package org.apache.ratis.datastream.impl; -import org.apache.ratis.datastream.impl.DataStreamPacketImpl; import org.apache.ratis.io.WriteOption; import org.apache.ratis.proto.RaftProtos.DataStreamPacketHeaderProto.Type; import org.apache.ratis.protocol.ClientId; diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java index bd03fefcc3..aa46cba53f 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java @@ -23,7 +23,7 @@ import org.apache.ratis.io.FilePositionCount; import org.apache.ratis.io.StandardWriteOption; import org.apache.ratis.io.WriteOption; -import org.apache.ratis.netty.server.DataStreamRequestByteBuf; +import org.apache.ratis.datastream.impl.DataStreamRequestByteBuf; import org.apache.ratis.proto.RaftProtos.DataStreamReplyHeaderProto; import org.apache.ratis.proto.RaftProtos.DataStreamRequestHeaderProto; import org.apache.ratis.proto.RaftProtos.DataStreamPacketHeaderProto; @@ -128,6 +128,20 @@ static void encodeByteBuffer(ByteBuffer buffer, Consumer out) { out.accept(Unpooled.wrappedBuffer(buffer)); } + static void encodeDataStreamRequestByteBuf(DataStreamRequestByteBuf request, Consumer out, + ByteBufAllocator allocator) { + encodeDataStreamRequestHeader(request, out, allocator); + encodeByteBuf(request.slice(), out); + } + + static void encodeByteBuf(ByteBuf buffer, Consumer out) { + if (buffer.readableBytes() == 0) { + out.accept(Unpooled.EMPTY_BUFFER); // to avoid EncoderException: must produce at least one message + return; + } + out.accept(buffer); + } + static void encodeDataStreamRequestFilePositionCount( DataStreamRequestFilePositionCount request, Consumer out, ByteBufAllocator allocator) { encodeDataStreamRequestHeader(request, out, allocator); diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java index 020acc2fdb..b2dc3812f2 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java @@ -21,6 +21,7 @@ import org.apache.ratis.client.DataStreamClientRpc; import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.datastream.impl.DataStreamRequestByteBuf; import org.apache.ratis.datastream.impl.DataStreamRequestByteBuffer; import org.apache.ratis.datastream.impl.DataStreamRequestFilePositionCount; import org.apache.ratis.io.StandardWriteOption; @@ -370,6 +371,7 @@ public void initChannel(SocketChannel ch) { p.addLast(ENCODER); p.addLast(ENCODER_FILE_POSITION_COUNT); p.addLast(ENCODER_BYTE_BUFFER); + p.addLast(ENCODER_BYTE_BUF); p.addLast(newDecoder()); p.addLast(handler); } @@ -386,6 +388,16 @@ protected void encode(ChannelHandlerContext context, DataStreamRequestByteBuffer } } + static final MessageToMessageEncoder ENCODER_BYTE_BUF = new EncoderByteBuf(); + + @ChannelHandler.Sharable + static class EncoderByteBuf extends MessageToMessageEncoder { + @Override + protected void encode(ChannelHandlerContext context, DataStreamRequestByteBuf request, List out) { + NettyDataStreamUtils.encodeDataStreamRequestByteBuf(request, out::add, context.alloc()); + } + } + static final MessageToMessageEncoder ENCODER_FILE_POSITION_COUNT = new EncoderFilePositionCount(); diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java index 276a365cea..302aed998e 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java @@ -18,10 +18,11 @@ package org.apache.ratis.netty.server; -import org.apache.ratis.client.DataStreamOutputRpc; import org.apache.ratis.client.impl.ClientProtoUtils; +import org.apache.ratis.client.impl.DataStreamClientImpl.DataStreamOutputImpl; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.datastream.impl.DataStreamReplyByteBuffer; +import org.apache.ratis.datastream.impl.DataStreamRequestByteBuf; import org.apache.ratis.io.StandardWriteOption; import org.apache.ratis.io.WriteOption; import org.apache.ratis.metrics.Timekeeper; @@ -111,12 +112,12 @@ void cleanUp() { } static class RemoteStream { - private final DataStreamOutputRpc out; + private final DataStreamOutputImpl out; private final AtomicReference> sendFuture = new AtomicReference<>(CompletableFuture.completedFuture(null)); private final RequestMetrics metrics; - RemoteStream(DataStreamOutputRpc out, RequestMetrics metrics) { + RemoteStream(DataStreamOutputImpl out, RequestMetrics metrics) { this.metrics = metrics; this.out = out; } @@ -132,7 +133,7 @@ static Iterable addFlush(List original) { CompletableFuture write(DataStreamRequestByteBuf request, Executor executor) { final Timekeeper.Context context = metrics.start(); return composeAsync(sendFuture, executor, - n -> out.writeAsync(request.slice().nioBuffer(), addFlush(request.getWriteOptionList())) + n -> out.writeAsync(request.slice().retain(), addFlush(request.getWriteOptionList())) .whenComplete((l, e) -> metrics.stop(context, e == null))); } } @@ -147,7 +148,7 @@ static class StreamInfo { = new AtomicReference<>(CompletableFuture.completedFuture(null)); StreamInfo(RaftClientRequest request, boolean primary, CompletableFuture stream, RaftServer server, - CheckedBiFunction, Set, IOException> getStreams, + CheckedBiFunction, Set, IOException> getStreams, Function metricsConstructor) throws IOException { this.request = request; @@ -155,7 +156,7 @@ static class StreamInfo { this.local = new LocalStream(stream, metricsConstructor.apply(RequestType.LOCAL_WRITE)); this.server = server; final Set successors = getSuccessors(server.getId()); - final Set outs = getStreams.apply(request, successors); + final Set outs = getStreams.apply(request, successors); this.remotes = outs.stream() .map(o -> new RemoteStream(o, metricsConstructor.apply(RequestType.REMOTE_WRITE))) .collect(Collectors.toSet()); @@ -315,7 +316,7 @@ private CompletableFuture computeDataStreamIfAbsent(RaftClientReques } private StreamInfo newStreamInfo(ByteBuf buf, - CheckedBiFunction, Set, IOException> getStreams) { + CheckedBiFunction, Set, IOException> getStreams) { try { final RaftClientRequest request = ClientProtoUtils.toRaftClientRequest( RaftClientRequestProto.parseFrom(buf.nioBuffer())); @@ -449,7 +450,7 @@ void cleanUpOnChannelInactive(ChannelId channelId, TimeDuration channelInactiveG } void read(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, - CheckedBiFunction, Set, IOException> getStreams) { + CheckedBiFunction, Set, IOException> getStreams) { LOG.debug("{}: read {}", this, request); try { readImpl(request, ctx, getStreams); @@ -459,7 +460,7 @@ void read(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, } private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, - CheckedBiFunction, Set, IOException> getStreams) { + CheckedBiFunction, Set, IOException> getStreams) { final boolean close = request.getWriteOptionList().contains(StandardWriteOption.CLOSE); ClientInvocationId key = ClientInvocationId.valueOf(request.getClientId(), request.getStreamId()); diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyServerStreamRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyServerStreamRpc.java index c5f24b0587..451040bb62 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyServerStreamRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyServerStreamRpc.java @@ -20,9 +20,11 @@ import org.apache.ratis.client.DataStreamClient; import org.apache.ratis.client.DataStreamOutputRpc; +import org.apache.ratis.client.impl.DataStreamClientImpl.DataStreamOutputImpl; import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.datastream.impl.DataStreamReplyByteBuffer; +import org.apache.ratis.datastream.impl.DataStreamRequestByteBuf; import org.apache.ratis.netty.NettyConfigKeys; import org.apache.ratis.netty.NettyDataStreamUtils; import org.apache.ratis.netty.NettyUtils; @@ -90,8 +92,8 @@ void addPeers(Collection newPeers) { map.addRaftPeers(newPeers); } - Set getDataStreamOutput(RaftClientRequest request, Set peers) throws IOException { - final Set outs = new HashSet<>(); + Set getDataStreamOutput(RaftClientRequest request, Set peers) throws IOException { + final Set outs = new HashSet<>(); try { getDataStreamOutput(request, peers, outs); } catch (IOException e) { @@ -101,11 +103,11 @@ Set getDataStreamOutput(RaftClientRequest request, Set peers, Set outs) + private void getDataStreamOutput(RaftClientRequest request, Set peers, Set outs) throws IOException { for (RaftPeer peer : peers) { try { - outs.add((DataStreamOutputRpc) map.computeIfAbsent(peer).get().stream(request)); + outs.add((DataStreamOutputImpl) map.computeIfAbsent(peer).get().stream(request)); } catch (IOException e) { map.handleException(peer.getId(), e, true); throw new IOException(map.getName() + ": Failed to getDataStreamOutput for " + peer, e); @@ -238,7 +240,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { } @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { + public void channelInactive(ChannelHandlerContext ctx) { requests.cleanUpOnChannelInactive(ctx.channel().id(), channelInactiveGracePeriod); } From 53d35c01e7bffeb596585cb981a2a89f685e9bef Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Wed, 21 Feb 2024 10:27:25 -0800 Subject: [PATCH 38/76] RATIS-2020. Refactor TransactionContext to supply LogEntryProto via a ReferenceCountedObject (#1042) --- .../arithmetic/ArithmeticStateMachine.java | 2 +- .../counter/server/CounterStateMachine.java | 2 +- .../filestore/FileStoreStateMachine.java | 2 +- .../statemachine/TransactionContext.java | 34 +++++++++++++++++-- .../ratis/server/impl/LeaderStateImpl.java | 2 +- .../ratis/server/impl/PendingRequest.java | 2 +- .../ratis/server/impl/RaftServerImpl.java | 6 ++++ .../apache/ratis/server/impl/ServerState.java | 2 +- .../ratis/server/raftlog/LogProtoUtils.java | 18 ++++++++++ .../statemachine/impl/BaseStateMachine.java | 6 ++-- .../impl/TransactionContextImpl.java | 32 ++++++++++++++--- .../apache/ratis/ReadOnlyRequestTests.java | 8 +++-- .../impl/StateMachineShutdownTests.java | 2 +- .../impl/SimpleStateMachine4Testing.java | 31 +++++++---------- .../ratis/datastream/DataStreamTestUtils.java | 2 +- .../ratis/statemachine/TestStateMachine.java | 2 +- 16 files changed, 113 insertions(+), 40 deletions(-) diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java index 28e3fb1c7d..e8b142f5dc 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java @@ -164,7 +164,7 @@ public void close() { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final LogEntryProto entry = trx.getLogEntry(); + final LogEntryProto entry = trx.getLogEntryUnsafe(); final AssignmentMessage assignment = new AssignmentMessage(entry.getStateMachineLogEntry().getLogData()); final long index = entry.getIndex(); diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java index b88a763e0f..47880af553 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java @@ -247,7 +247,7 @@ public TransactionContext startTransaction(RaftClientRequest request) throws IOE */ @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final LogEntryProto entry = trx.getLogEntry(); + final LogEntryProto entry = trx.getLogEntryUnsafe(); //increment the counter and update term-index final TermIndex termIndex = TermIndex.valueOf(entry); final int incremented = incrementCounter(termIndex); diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java index 0ee7a60ac7..f870cbacfe 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java @@ -227,7 +227,7 @@ public CompletableFuture link(DataStream stream, LogEntryProto entry) { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final LogEntryProto entry = trx.getLogEntry(); + final LogEntryProto entry = trx.getLogEntryUnsafe(); final long index = entry.getIndex(); updateLastAppliedTermIndex(entry.getTerm(), index); diff --git a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java index e0190747fc..2ec87e37ad 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.Objects; +import java.util.Optional; /** * Context for a transaction. @@ -94,11 +95,40 @@ public interface TransactionContext { LogEntryProto initLogEntry(long term, long index); /** - * Returns the committed log entry - * @return the committed log entry + * @return a copy of the committed log entry if it exists; otherwise, returns null + * + * @deprecated Use {@link #getLogEntryRef()} or {@link #getLogEntryUnsafe()} to avoid copying. */ + @Deprecated LogEntryProto getLogEntry(); + /** + * @return the committed log entry if it exists; otherwise, returns null. + * The returned value is safe to use only before {@link StateMachine#applyTransaction} returns. + * Once {@link StateMachine#applyTransaction} has returned, it is unsafe to use the log entry + * since the underlying buffers can possiby be released. + */ + default LogEntryProto getLogEntryUnsafe() { + return getLogEntryRef().get(); + } + + /** + * Get a {@link ReferenceCountedObject} to the committed log entry. + * + * It is safe to access the log entry by calling {@link ReferenceCountedObject#get()} + * (without {@link ReferenceCountedObject#retain()}) + * inside the scope of {@link StateMachine#applyTransaction}. + * + * If the log entry is needed after {@link StateMachine#applyTransaction} returns, + * e.g. for asynchronous computation or caching, + * the caller must invoke {@link ReferenceCountedObject#retain()} and {@link ReferenceCountedObject#release()}. + * + * @return a reference to the committed log entry if it exists; otherwise, returns null. + */ + default ReferenceCountedObject getLogEntryRef() { + return Optional.ofNullable(getLogEntryUnsafe()).map(this::wrap).orElse(null); + } + /** Wrap the given log entry as a {@link ReferenceCountedObject} for retaining it for later use. */ default ReferenceCountedObject wrap(LogEntryProto entry) { Preconditions.assertSame(getLogEntry().getTerm(), entry.getTerm(), "entry.term"); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 4f313a4373..e8a4adc9ca 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -527,7 +527,7 @@ PendingRequests.Permit tryAcquirePendingRequest(Message message) { PendingRequest addPendingRequest(PendingRequests.Permit permit, RaftClientRequest request, TransactionContext entry) { if (LOG.isDebugEnabled()) { LOG.debug("{}: addPendingRequest at {}, entry={}", this, request, - LogProtoUtils.toLogEntryString(entry.getLogEntry())); + LogProtoUtils.toLogEntryString(entry.getLogEntryUnsafe())); } return pendingRequests.add(permit, request, entry); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java index 06a3a7b3ce..4271d76162 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java @@ -38,7 +38,7 @@ class PendingRequest { private final CompletableFuture futureToReturn; PendingRequest(RaftClientRequest request, TransactionContext entry) { - this.termIndex = entry == null? null: TermIndex.valueOf(entry.getLogEntry()); + this.termIndex = entry == null? null: TermIndex.valueOf(entry.getLogEntryUnsafe()); this.request = request; this.entry = entry; if (request.is(TypeCase.FORWARD)) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 133cfebdc6..0885fb807b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1802,6 +1802,10 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf final ClientInvocationId invocationId = ClientInvocationId.valueOf(next.getStateMachineLogEntry()); writeIndexCache.add(invocationId.getClientId(), ((TransactionContextImpl) trx).getLogIndexFuture()); + // TODO: RaftLog to provide the log entry as a ReferenceCountedObject as per RATIS-2028. + ReferenceCountedObject ref = ReferenceCountedObject.wrap(next); + ((TransactionContextImpl) trx).setDelegatedRef(ref); + ref.retain(); try { // Let the StateMachine inject logic for committed transactions in sequential order. trx = stateMachine.applyTransactionSerial(trx); @@ -1810,6 +1814,8 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf return replyPendingRequest(invocationId, TermIndex.valueOf(next), stateMachineFuture); } catch (Exception e) { throw new RaftLogIOException(e); + } finally { + ref.release(); } } return null; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index d02994ec54..27eaf317ec 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -318,7 +318,7 @@ TermIndex getLastEntry() { void appendLog(TransactionContext operation) throws StateMachineException { getLog().append(currentTerm.get(), operation); - Objects.requireNonNull(operation.getLogEntry()); + Objects.requireNonNull(operation.getLogEntryUnsafe(), "transaction-logEntry"); } /** @return true iff the given peer id is recognized as the leader. */ diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java index de06faf63e..b177f0e14c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java @@ -27,6 +27,7 @@ import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.thirdparty.com.google.protobuf.AbstractMessage; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; @@ -221,4 +222,21 @@ public static RaftConfiguration toRaftConfiguration(LogEntryProto entry) { final List oldListener = ProtoUtils.toRaftPeers(proto.getOldListenersList()); return ServerImplUtils.newRaftConfiguration(conf, listener, entry.getIndex(), oldConf, oldListener); } + + public static LogEntryProto copy(LogEntryProto proto) { + if (proto == null) { + return null; + } + + if (!proto.hasStateMachineLogEntry() && !proto.hasMetadataEntry() && !proto.hasConfigurationEntry()) { + // empty entry, just return as is. + return proto; + } + + try { + return LogEntryProto.parseFrom(proto.toByteString()); + } catch (InvalidProtocolBufferException e) { + throw new IllegalArgumentException("Failed to copy log entry " + TermIndex.valueOf(proto), e); + } + } } diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java index c987c53ddb..98f270d2d9 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java @@ -18,7 +18,7 @@ package org.apache.ratis.statemachine.impl; -import org.apache.ratis.proto.RaftProtos; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.protocol.RaftGroupId; @@ -110,10 +110,10 @@ public TransactionContext applyTransactionSerial(TransactionContext trx) throws @Override public CompletableFuture applyTransaction(TransactionContext trx) { // return the same message contained in the entry - RaftProtos.LogEntryProto entry = Objects.requireNonNull(trx.getLogEntry()); + final LogEntryProto entry = Objects.requireNonNull(trx.getLogEntryUnsafe()); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); return CompletableFuture.completedFuture( - Message.valueOf(trx.getLogEntry().getStateMachineLogEntry().getLogData())); + Message.valueOf(entry.getStateMachineLogEntry().getLogData())); } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java index 7c4f1782de..44bd32c674 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java @@ -25,12 +25,14 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ReferenceCountedObject; import java.io.IOException; import java.util.Objects; import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; /** * Implementation of {@link TransactionContext} @@ -69,6 +71,9 @@ public class TransactionContextImpl implements TransactionContext { /** Committed LogEntry. */ private volatile LogEntryProto logEntry; + /** Committed LogEntry copy. */ + private volatile Supplier logEntryCopy; + /** For wrapping {@link #logEntry} in order to release the underlying buffer. */ private volatile ReferenceCountedObject delegatedRef; @@ -112,7 +117,7 @@ private static StateMachineLogEntryProto get(StateMachineLogEntryProto stateMach */ TransactionContextImpl(RaftPeerRole serverRole, StateMachine stateMachine, LogEntryProto logEntry) { this(serverRole, null, stateMachine, logEntry.getStateMachineLogEntry()); - this.logEntry = logEntry; + setLogEntry(logEntry); this.logIndexFuture.complete(logEntry.getIndex()); } @@ -135,8 +140,10 @@ public ReferenceCountedObject wrap(LogEntryProto entry) { if (delegatedRef == null) { return TransactionContext.super.wrap(entry); } - Preconditions.assertSame(getLogEntry().getTerm(), entry.getTerm(), "entry.term"); - Preconditions.assertSame(getLogEntry().getIndex(), entry.getIndex(), "entry.index"); + final LogEntryProto expected = getLogEntryUnsafe(); + Objects.requireNonNull(expected, "logEntry == null"); + Preconditions.assertSame(expected.getTerm(), entry.getTerm(), "entry.term"); + Preconditions.assertSame(expected.getIndex(), entry.getIndex(), "entry.index"); return delegatedRef.delegate(entry); } @@ -168,18 +175,31 @@ public LogEntryProto initLogEntry(long term, long index) { Objects.requireNonNull(stateMachineLogEntry, "stateMachineLogEntry == null"); logIndexFuture.complete(index); - return logEntry = LogProtoUtils.toLogEntryProto(stateMachineLogEntry, term, index); + return setLogEntry(LogProtoUtils.toLogEntryProto(stateMachineLogEntry, term, index)); } public CompletableFuture getLogIndexFuture() { return logIndexFuture; } + private LogEntryProto setLogEntry(LogEntryProto entry) { + this.logEntry = entry; + this.logEntryCopy = MemoizedSupplier.valueOf(() -> LogProtoUtils.copy(entry)); + return entry; + } + + @Override public LogEntryProto getLogEntry() { + return logEntryCopy == null ? null : logEntryCopy.get(); + } + + @Override + public LogEntryProto getLogEntryUnsafe() { return logEntry; } + @Override public TransactionContext setException(Exception ioe) { this.exception = ioe; @@ -209,4 +229,8 @@ public TransactionContext cancelTransaction() throws IOException { // call this to let the SM know that Transaction cannot be synced return stateMachine.cancelTransaction(this); } + + public static LogEntryProto getLogEntry(TransactionContext context) { + return ((TransactionContextImpl) context).logEntry; + } } diff --git a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java index eea75592ef..ead2a8be8e 100644 --- a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java @@ -19,6 +19,7 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeerId; @@ -366,10 +367,11 @@ private void timeoutIncrement() { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - LOG.debug("apply trx with index=" + trx.getLogEntry().getIndex()); - updateLastAppliedTermIndex(trx.getLogEntry().getTerm(), trx.getLogEntry().getIndex()); + final RaftProtos.LogEntryProto logEntry = trx.getLogEntryUnsafe(); + LOG.debug("apply trx with index=" + logEntry.getIndex()); + updateLastAppliedTermIndex(logEntry.getTerm(), logEntry.getIndex()); - String command = trx.getLogEntry().getStateMachineLogEntry() + String command = logEntry.getStateMachineLogEntry() .getLogData().toString(StandardCharsets.UTF_8); LOG.info("receive command: {}", command); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java index 28f8e6ace9..246abb99f0 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java @@ -56,7 +56,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) { } } } - RaftProtos.LogEntryProto entry = trx.getLogEntry(); + final RaftProtos.LogEntryProto entry = trx.getLogEntryUnsafe(); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); return CompletableFuture.completedFuture(new RaftTestUtil.SimpleMessage("done")); } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 17d5a607a9..7dd1db3c71 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -84,7 +84,7 @@ public static SimpleStateMachine4Testing get(RaftServer.Division s) { return (SimpleStateMachine4Testing)s.getStateMachine(); } - private final SortedMap indexMap = Collections.synchronizedSortedMap(new TreeMap<>()); + private final SortedMap> indexMap = Collections.synchronizedSortedMap(new TreeMap<>()); private final SortedMap dataMap = Collections.synchronizedSortedMap(new TreeMap<>()); private final Daemon checkpointer; private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage(); @@ -199,8 +199,9 @@ public RoleInfoProto getLeaderElectionTimeoutInfo() { return leaderElectionTimeoutInfo; } - private void put(LogEntryProto entry) { - final LogEntryProto previous = indexMap.put(entry.getIndex(), entry); + private void put(ReferenceCountedObject entryRef) { + LogEntryProto entry = entryRef.retain(); + final ReferenceCountedObject previous = indexMap.put(entry.getIndex(), entryRef); Preconditions.assertNull(previous, "previous"); final String s = entry.getStateMachineLogEntry().getLogData().toStringUtf8(); dataMap.put(s, entry); @@ -246,27 +247,17 @@ public synchronized void reinitialize() throws IOException { @Override public CompletableFuture applyTransaction(TransactionContext trx) { blocking.await(Blocking.Type.APPLY_TRANSACTION); - LogEntryProto entry = Objects.requireNonNull(trx.getLogEntry()); + ReferenceCountedObject entryRef = Objects.requireNonNull(trx.getLogEntryRef()); + LogEntryProto entry = entryRef.get(); LOG.info("applyTransaction for log index {}", entry.getIndex()); - // TODO: Logs kept in StateMachine's cache may be corrupted. Copy for now to have the test pass. - // Use ReferenceCount per RATIS-1997. - LogEntryProto copied = copy(entry); - put(copied); + put(entryRef); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); final SimpleMessage m = new SimpleMessage(entry.getIndex() + " OK"); return collecting.collect(Collecting.Type.APPLY_TRANSACTION, m); } - private LogEntryProto copy(LogEntryProto log) { - try { - return LogEntryProto.parseFrom(log.toByteString()); - } catch (InvalidProtocolBufferException e) { - throw new IllegalStateException("Error copying log entry", e); - } - } - @Override public long takeSnapshot() { final TermIndex termIndex = getLastAppliedTermIndex(); @@ -280,7 +271,8 @@ public long takeSnapshot() { LOG.debug("Taking a snapshot with {}, file:{}", termIndex, snapshotFile); try (SegmentedRaftLogOutputStream out = new SegmentedRaftLogOutputStream(snapshotFile, false, segmentMaxSize, preallocatedSize, ByteBuffer.allocateDirect(bufferSize))) { - for (final LogEntryProto entry : indexMap.values()) { + for (final ReferenceCountedObject entryRef : indexMap.values()) { + LogEntryProto entry = entryRef.get(); if (entry.getIndex() > endIndex) { break; } else { @@ -315,7 +307,7 @@ private synchronized void loadSnapshot(SingleFileSnapshotInfo snapshot) throws I snapshot.getFile().getPath().toFile(), 0, endIndex, false)) { LogEntryProto entry; while ((entry = in.nextEntry()) != null) { - put(entry); + put(ReferenceCountedObject.wrap(entry)); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); } } @@ -390,10 +382,11 @@ public void close() { running = false; checkpointer.interrupt(); }); + indexMap.values().forEach(ReferenceCountedObject::release); } public LogEntryProto[] getContent() { - return indexMap.values().toArray(new LogEntryProto[0]); + return indexMap.values().stream().map(ReferenceCountedObject::get).toArray(LogEntryProto[]::new); } public void blockStartTransaction() { diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index e4a930f1d1..2970bbef84 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -165,7 +165,7 @@ public CompletableFuture link(DataStream stream, LogEntryProto entry) { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final LogEntryProto entry = Objects.requireNonNull(trx.getLogEntry()); + final LogEntryProto entry = Objects.requireNonNull(trx.getLogEntryUnsafe()); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); final SingleDataStream s = getSingleDataStream(ClientInvocationId.valueOf(entry.getStateMachineLogEntry())); final ByteString bytesWritten = bytesWritten2ByteString(s.getDataChannel().getBytesWritten()); diff --git a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java index 094189827d..07ea4edbcb 100644 --- a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java +++ b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java @@ -91,7 +91,7 @@ public TransactionContext startTransaction(RaftClientRequest request) { @Override public CompletableFuture applyTransaction(TransactionContext trx) { try { - assertNotNull(trx.getLogEntry()); + assertNotNull(trx.getLogEntryUnsafe()); assertNotNull(trx.getStateMachineLogEntry()); Object context = trx.getStateMachineContext(); if (isLeader.get()) { From 1bd451046b2d33fe063712d7e811262bef580a4e Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 1 Mar 2024 10:52:17 -0800 Subject: [PATCH 39/76] RATIS-2038. IllegalStateException when ServerState is closed before start. (#1048) --- .../src/main/java/org/apache/ratis/server/impl/ServerState.java | 2 +- .../java/org/apache/ratis/server/impl/StateMachineUpdater.java | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index 27eaf317ec..af241a6331 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -158,6 +158,7 @@ void writeRaftConfiguration(LogEntryProto conf) { } void start() { + // initialize stateMachineUpdater stateMachineUpdater.get().start(); } @@ -421,7 +422,6 @@ void close() { } LOG.warn(getMemberId() + ": Failed to join " + getStateMachineUpdater(), e); } - LOG.info("{}: applyIndex: {}", getMemberId(), getLastAppliedIndex()); try { if (log.isInitialized()) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index 43fbdd8843..5f6e972e2c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -133,6 +133,8 @@ void start() { private void stop() { state = State.STOP; try { + LOG.info("{}: closing {}, lastApplied={}", name, + JavaUtils.getClassSimpleName(stateMachine.getClass()), stateMachine.getLastAppliedTermIndex()); stateMachine.close(); if (stateMachineMetrics.isInitialized()) { stateMachineMetrics.get().unregister(); From df436c73f66f192a2fb9f8e43500196e4e3a7cec Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 6 Mar 2024 22:29:50 -0800 Subject: [PATCH 40/76] RATIS-2035. Refactor streaming code for Read. (#1046) --- .../apache/ratis/netty/server/ChannelMap.java | 48 ++++++++++++++++ .../netty/server/DataStreamManagement.java | 47 +--------------- .../apache/ratis/netty/server/StreamMap.java | 55 +++++++++++++++++++ .../ratis/server/DataStreamServerRpc.java | 10 +--- .../apache/ratis/server/RaftServerRpc.java | 9 +-- .../org/apache/ratis/server/ServerRpc.java | 33 +++++++++++ .../ratis/datastream/DataStreamBaseTest.java | 4 +- 7 files changed, 141 insertions(+), 65 deletions(-) create mode 100644 ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java create mode 100644 ratis-netty/src/main/java/org/apache/ratis/netty/server/StreamMap.java create mode 100644 ratis-server-api/src/main/java/org/apache/ratis/server/ServerRpc.java diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java new file mode 100644 index 0000000000..7b0d761840 --- /dev/null +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java @@ -0,0 +1,48 @@ +/* + * 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.ratis.netty.server; + +import org.apache.ratis.protocol.ClientInvocationId; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelId; + +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +/** Map: {@link ChannelId} -> {@link ClientInvocationId}s. */ +class ChannelMap { + private final Map> map = new ConcurrentHashMap<>(); + + void add(ChannelId channelId, ClientInvocationId clientInvocationId) { + map.computeIfAbsent(channelId, (e) -> new ConcurrentHashMap<>()) + .put(clientInvocationId, clientInvocationId); + } + + void remove(ChannelId channelId, ClientInvocationId clientInvocationId) { + Optional.ofNullable(map.get(channelId)) + .ifPresent((ids) -> ids.remove(clientInvocationId)); + } + + Set remove(ChannelId channelId) { + return Optional.ofNullable(map.remove(channelId)) + .map(Map::keySet) + .orElse(Collections.emptySet()); + } +} diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java index 302aed998e..a6e9b815ee 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java @@ -70,13 +70,10 @@ import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicReference; @@ -219,52 +216,10 @@ private Set getSuccessors(RaftPeerId peerId) throws IOException { } } - static class StreamMap { - private final ConcurrentMap map = new ConcurrentHashMap<>(); - - StreamInfo computeIfAbsent(ClientInvocationId key, Function function) { - final StreamInfo info = map.computeIfAbsent(key, function); - LOG.debug("computeIfAbsent({}) returns {}", key, info); - return info; - } - - StreamInfo get(ClientInvocationId key) { - final StreamInfo info = map.get(key); - LOG.debug("get({}) returns {}", key, info); - return info; - } - - StreamInfo remove(ClientInvocationId key) { - final StreamInfo info = map.remove(key); - LOG.debug("remove({}) returns {}", key, info); - return info; - } - } - - public static class ChannelMap { - private final Map> map = new ConcurrentHashMap<>(); - - public void add(ChannelId channelId, - ClientInvocationId clientInvocationId) { - map.computeIfAbsent(channelId, (e) -> new ConcurrentHashMap<>()).put(clientInvocationId, clientInvocationId); - } - - public void remove(ChannelId channelId, - ClientInvocationId clientInvocationId) { - Optional.ofNullable(map.get(channelId)).ifPresent((ids) -> ids.remove(clientInvocationId)); - } - - public Set remove(ChannelId channelId) { - return Optional.ofNullable(map.remove(channelId)) - .map(Map::keySet) - .orElse(Collections.emptySet()); - } - } - private final RaftServer server; private final String name; - private final StreamMap streams = new StreamMap(); + private final StreamMap streams = new StreamMap<>(); private final ChannelMap channels; private final ExecutorService requestExecutor; private final ExecutorService writeExecutor; diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/StreamMap.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/StreamMap.java new file mode 100644 index 0000000000..073698cb80 --- /dev/null +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/StreamMap.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.ratis.netty.server; + +import org.apache.ratis.protocol.ClientInvocationId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Function; + +/** + * Map: {@link ClientInvocationId} -> {@link STREAM}. + * + * @param the stream type. + */ +class StreamMap { + public static final Logger LOG = LoggerFactory.getLogger(StreamMap.class); + + private final ConcurrentMap map = new ConcurrentHashMap<>(); + + STREAM computeIfAbsent(ClientInvocationId key, Function function) { + final STREAM info = map.computeIfAbsent(key, function); + LOG.debug("computeIfAbsent({}) returns {}", key, info); + return info; + } + + STREAM get(ClientInvocationId key) { + final STREAM info = map.get(key); + LOG.debug("get({}) returns {}", key, info); + return info; + } + + STREAM remove(ClientInvocationId key) { + final STREAM info = map.remove(key); + LOG.debug("remove({}) returns {}", key, info); + return info; + } +} diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/DataStreamServerRpc.java b/ratis-server-api/src/main/java/org/apache/ratis/server/DataStreamServerRpc.java index 4e948c6f81..6316ef6073 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/DataStreamServerRpc.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/DataStreamServerRpc.java @@ -20,18 +20,10 @@ import org.apache.ratis.protocol.RaftPeer; import java.io.Closeable; -import java.net.InetSocketAddress; /** * A server interface handling incoming streams * Relays those streams to other servers after persisting */ -public interface DataStreamServerRpc extends RaftPeer.Add, Closeable { - /** - * start server - */ - void start(); - - /** @return the address where this RPC server is listening to. */ - InetSocketAddress getInetSocketAddress(); +public interface DataStreamServerRpc extends ServerRpc, RaftPeer.Add, Closeable { } diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerRpc.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerRpc.java index d81f9cc8bc..76bd817f5e 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerRpc.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerRpc.java @@ -26,20 +26,13 @@ import org.apache.ratis.util.JavaUtils; import java.io.Closeable; -import java.io.IOException; import java.net.InetSocketAddress; /** * An server-side interface for supporting different RPC implementations * such as Netty, gRPC and Hadoop. */ -public interface RaftServerRpc extends RaftServerProtocol, RpcType.Get, RaftPeer.Add, Closeable { - /** Start the RPC service. */ - void start() throws IOException; - - /** @return the address where this RPC server is listening */ - InetSocketAddress getInetSocketAddress(); - +public interface RaftServerRpc extends RaftServerProtocol, ServerRpc, RpcType.Get, RaftPeer.Add, Closeable { /** @return the address where this RPC server is listening for client requests */ default InetSocketAddress getClientServerAddress() { return getInetSocketAddress(); diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/ServerRpc.java b/ratis-server-api/src/main/java/org/apache/ratis/server/ServerRpc.java new file mode 100644 index 0000000000..6ad5eacf14 --- /dev/null +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/ServerRpc.java @@ -0,0 +1,33 @@ +/* + * 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.ratis.server; + +import java.io.Closeable; +import java.io.IOException; +import java.net.InetSocketAddress; + +/** + * A general server interface. + */ +public interface ServerRpc extends Closeable { + /** Start the RPC service. */ + void start() throws IOException; + + /** @return the address where this RPC server is listening to. */ + InetSocketAddress getInetSocketAddress(); +} diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java index 70e26af249..2ac01ac1f5 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java @@ -67,7 +67,7 @@ RaftServer getRaftServer() { return raftServer; } - void start() { + void start() throws IOException { dataStreamServer.getServerRpc().start(); } @@ -90,7 +90,7 @@ Server getPrimaryServer() { return servers.get(0); } - void setup(RaftGroupId groupId, List peers, List raftServers) { + void setup(RaftGroupId groupId, List peers, List raftServers) throws Exception { raftGroup = RaftGroup.valueOf(groupId, peers); this.peers = peers; servers = new ArrayList<>(peers.size()); From 99a833c29eb28203cc6107ede26318ff86e636e1 Mon Sep 17 00:00:00 2001 From: DaveTeng0 <109315747+DaveTeng0@users.noreply.github.com> Date: Thu, 7 Mar 2024 16:37:16 -0800 Subject: [PATCH 41/76] Ratis-2031. Add peer info to response of GroupInfoCommand CLI (#1047) --- .../ratis/client/impl/ClientProtoUtils.java | 4 +- .../apache/ratis/protocol/GroupInfoReply.java | 13 +++-- ratis-proto/src/main/proto/Raft.proto | 10 ++++ .../ratis/server/impl/RaftServerImpl.java | 16 +++++- .../shell/cli/sh/group/GroupInfoCommand.java | 1 + .../cli/sh/GroupCommandIntegrationTest.java | 49 +++++++++++++++++++ 6 files changed, 88 insertions(+), 5 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java index db19831955..003f202bd9 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java @@ -364,6 +364,7 @@ static GroupInfoReplyProto toGroupInfoReplyProto(GroupInfoReply reply) { b.setIsRaftStorageHealthy(reply.isRaftStorageHealthy()); b.setRole(reply.getRoleInfoProto()); b.addAllCommitInfos(reply.getCommitInfos()); + b.setLogInfo(reply.getLogInfoProto()); } } return b.build(); @@ -506,7 +507,8 @@ static GroupInfoReply toGroupInfoReply(GroupInfoReplyProto replyProto) { ProtoUtils.toRaftGroup(replyProto.getGroup()), replyProto.getRole(), replyProto.getIsRaftStorageHealthy(), - replyProto.hasConf()? replyProto.getConf(): null); + replyProto.hasConf()? replyProto.getConf(): null, + replyProto.getLogInfo()); } static Message toMessage(final ClientMessageEntryProto p) { diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/GroupInfoReply.java b/ratis-common/src/main/java/org/apache/ratis/protocol/GroupInfoReply.java index 632fa65293..bfac81a2b0 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/GroupInfoReply.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/GroupInfoReply.java @@ -19,6 +19,7 @@ import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto; import org.apache.ratis.proto.RaftProtos.CommitInfoProto; +import org.apache.ratis.proto.RaftProtos.LogInfoProto; import org.apache.ratis.proto.RaftProtos.RoleInfoProto; import java.util.Collection; @@ -33,25 +34,27 @@ public class GroupInfoReply extends RaftClientReply { private final RoleInfoProto roleInfoProto; private final boolean isRaftStorageHealthy; private final RaftConfigurationProto conf; + private final LogInfoProto logInfoProto; public GroupInfoReply(RaftClientRequest request, Collection commitInfos, RaftGroup group, RoleInfoProto roleInfoProto, boolean isRaftStorageHealthy, - RaftConfigurationProto conf) { + RaftConfigurationProto conf, LogInfoProto logInfoProto) { this(request.getClientId(), request.getServerId(), request.getRaftGroupId(), request.getCallId(), commitInfos, - group, roleInfoProto, isRaftStorageHealthy, conf); + group, roleInfoProto, isRaftStorageHealthy, conf, logInfoProto); } @SuppressWarnings("parameternumber") public GroupInfoReply(ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId, Collection commitInfos, RaftGroup group, RoleInfoProto roleInfoProto, boolean isRaftStorageHealthy, - RaftConfigurationProto conf) { + RaftConfigurationProto conf, LogInfoProto logInfoProto) { super(clientId, serverId, groupId, callId, true, null, null, 0L, commitInfos); this.group = group; this.roleInfoProto = roleInfoProto; this.isRaftStorageHealthy = isRaftStorageHealthy; this.conf = conf; + this.logInfoProto = logInfoProto; } public RaftGroup getGroup() { @@ -69,4 +72,8 @@ public boolean isRaftStorageHealthy() { public Optional getConf() { return Optional.ofNullable(conf); } + + public LogInfoProto getLogInfoProto() { + return logInfoProto; + } } diff --git a/ratis-proto/src/main/proto/Raft.proto b/ratis-proto/src/main/proto/Raft.proto index 586ec1b052..edc57ec65e 100644 --- a/ratis-proto/src/main/proto/Raft.proto +++ b/ratis-proto/src/main/proto/Raft.proto @@ -556,4 +556,14 @@ message GroupInfoReplyProto { bool isRaftStorageHealthy = 4; repeated CommitInfoProto commitInfos = 5; RaftConfigurationProto conf = 6; + LogInfoProto logInfo = 7; +} + +/** Add new LogInfoProto for RATIS-2030, allow GroupInfoCommand to show each server's last committed log, + last applied log, last snapshot log, last entry log.*/ +message LogInfoProto { + TermIndexProto lastSnapshot = 1; + TermIndexProto applied = 2; + TermIndexProto committed = 3; + TermIndexProto lastEntry = 4; } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 0885fb807b..17a741e703 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -24,6 +24,7 @@ import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult; import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; import org.apache.ratis.proto.RaftProtos.CommitInfoProto; +import org.apache.ratis.proto.RaftProtos.LogInfoProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotResult; @@ -642,7 +643,20 @@ GroupInfoReply getGroupInfo(GroupInfoRequest request) { final RaftConfigurationProto conf = LogProtoUtils.toRaftConfigurationProtoBuilder(getRaftConf()).build(); return new GroupInfoReply(request, getCommitInfos(), getGroup(), getRoleInfoProto(), - dir.isHealthy(), conf); + dir.isHealthy(), conf, getLogInfo()); + } + + LogInfoProto getLogInfo(){ + final RaftLog log = getRaftLog(); + LogInfoProto.Builder logInfoBuilder = LogInfoProto.newBuilder() + .setApplied(getStateMachine().getLastAppliedTermIndex().toProto()) + .setCommitted(log.getTermIndex(log.getLastCommittedIndex()).toProto()) + .setLastEntry(log.getLastEntryTermIndex().toProto()); + final SnapshotInfo snapshot = getStateMachine().getLatestSnapshot(); + if (snapshot != null) { + logInfoBuilder.setLastSnapshot(snapshot.getTermIndex().toProto()); + } + return logInfoBuilder.build(); } RoleInfoProto getRoleInfoProto() { diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupInfoCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupInfoCommand.java index d2c4e65991..0125440e90 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupInfoCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupInfoCommand.java @@ -54,6 +54,7 @@ public int run(CommandLine cl) throws IOException { printf("leader info: %s(%s)%n%n", leader.getId().toStringUtf8(), leader.getAddress()); } println(reply.getCommitInfos()); + println(reply.getLogInfoProto()); return 0; } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java index 83c05bfddd..6900d4804a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java @@ -20,6 +20,7 @@ import org.apache.ratis.RaftTestUtil; import org.apache.ratis.client.RaftClient; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.MiniRaftCluster; @@ -90,4 +91,52 @@ void runTestGroupInfoCommand(MiniRaftCluster cluster) throws Exception { String info = result.substring(0, hearder.length()); Assertions.assertEquals(hearder, info); } + + @Test + public void testGroupInfoCommandIncludesCorrectLogInfo() throws Exception { + // set number of server to 1 so that we can make sure which server returns the LogInfoProto + // since information of applied index, snapshot index, and last entry index are not shared between servers + runWithNewCluster(1, this::runTestGroupInfoCommandWithLogInfoVerification); + } + + void runTestGroupInfoCommandWithLogInfoVerification(MiniRaftCluster cluster) throws Exception { + RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); + + try (final RaftClient client = cluster.createClient(leader.getId())) { + for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { + RaftClientReply + reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); + Assertions.assertTrue(reply.isSuccess()); + } + } + + leader.getStateMachine().takeSnapshot(); + + final String address = getClusterAddress(cluster); + final StringPrintStream out = new StringPrintStream(); + RatisShell shell = new RatisShell(out.getPrintStream()); + int ret = shell.run("group", "info", "-peers", address); + Assertions.assertEquals(0 , ret); + String result = out.toString().trim(); + String hearder = String.format("group id: %s%sleader info: %s(%s)%s%s", + cluster.getGroupId().getUuid(), NEW_LINE, leader.getId(), + cluster.getLeader().getPeer().getAddress(), NEW_LINE, NEW_LINE); + String info = result.substring(0, hearder.length()); + Assertions.assertEquals(hearder, info); + long currentTerm = leader.getInfo().getCurrentTerm(); + String LogInfoProtoFormat = "%s {" + NEW_LINE + " term: " + currentTerm + NEW_LINE + " index: %s"; + Assertions.assertTrue(result.contains( + String.format(LogInfoProtoFormat, "applied", + leader.getStateMachine().getLastAppliedTermIndex().getIndex()))); + Assertions.assertTrue(result.contains( + String.format(LogInfoProtoFormat, "committed", + leader.getRaftLog().getLastCommittedIndex()))); + Assertions.assertTrue(result.contains( + String.format(LogInfoProtoFormat, "lastSnapshot", + leader.getStateMachine().getLatestSnapshot().getIndex()))); + Assertions.assertTrue(result.contains( + String.format(LogInfoProtoFormat, "lastEntry", + leader.getRaftLog().getLastCommittedIndex()))); + } + } From e199daaf2a288508bd3e119e9c252c70ee90937d Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Mon, 11 Mar 2024 17:58:00 -0700 Subject: [PATCH 42/76] RATIS-2028. Refactor RaftLog to supply log as ReferenceCountedObject (#1045) --- .../apache/ratis/server/raftlog/RaftLog.java | 17 ++++++++- .../ratis/server/impl/RaftServerImpl.java | 12 ++---- .../server/impl/StateMachineUpdater.java | 19 ++++++---- .../ratis/server/raftlog/RaftLogBase.java | 8 +++- .../server/raftlog/memory/MemoryRaftLog.java | 35 +++++++++++++++--- .../server/raftlog/segmented/LogSegment.java | 21 +++++------ .../raftlog/segmented/SegmentedRaftLog.java | 37 +++++++++++++++++-- .../server/storage/RaftStorageTestUtils.java | 16 +++++++- .../statemachine/RaftSnapshotBaseTest.java | 4 +- .../ratis/datastream/DataStreamTestUtils.java | 4 +- .../ratis/server/ServerRestartTests.java | 10 +++-- .../raftlog/segmented/TestLogSegment.java | 4 +- .../segmented/TestSegmentedRaftLog.java | 7 ++-- .../segmented/TestSegmentedRaftLogCache.java | 4 +- 14 files changed, 147 insertions(+), 51 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java index e504462b80..e4fbd664ea 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java @@ -21,6 +21,7 @@ import org.apache.ratis.server.metrics.RaftLogMetrics; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.storage.RaftStorageMetadata; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,10 +58,24 @@ default boolean contains(TermIndex ti) { /** * @return null if the log entry is not found in this log; - * otherwise, return the log entry corresponding to the given index. + * otherwise, return a copy of the log entry corresponding to the given index. + * @deprecated use {@link RaftLog#retainLog(long)} instead in order to avoid copying. */ + @Deprecated LogEntryProto get(long index) throws RaftLogIOException; + /** + * @return a retained {@link ReferenceCountedObject} to the log entry corresponding to the given index if it exists; + * otherwise, return null. + * Since the returned reference is retained, the caller must call {@link ReferenceCountedObject#release()}} + * after use. + */ + default ReferenceCountedObject retainLog(long index) throws RaftLogIOException { + ReferenceCountedObject wrap = ReferenceCountedObject.wrap(get(index)); + wrap.retain(); + return wrap; + } + /** * @return null if the log entry is not found in this log; * otherwise, return the {@link EntryWithData} corresponding to the given index. diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 17a741e703..e8aeb66784 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1800,7 +1800,9 @@ TransactionContext getTransactionContext(LogEntryProto entry, Boolean createNew) MemoizedSupplier.valueOf(() -> stateMachine.startTransaction(entry, getInfo().getCurrentRole()))); } - CompletableFuture applyLogToStateMachine(LogEntryProto next) throws RaftLogIOException { + CompletableFuture applyLogToStateMachine(ReferenceCountedObject nextRef) + throws RaftLogIOException { + LogEntryProto next = nextRef.get(); if (!next.hasStateMachineLogEntry()) { stateMachine.event().notifyTermIndexUpdated(next.getTerm(), next.getIndex()); } @@ -1815,11 +1817,7 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf TransactionContext trx = getTransactionContext(next, true); final ClientInvocationId invocationId = ClientInvocationId.valueOf(next.getStateMachineLogEntry()); writeIndexCache.add(invocationId.getClientId(), ((TransactionContextImpl) trx).getLogIndexFuture()); - - // TODO: RaftLog to provide the log entry as a ReferenceCountedObject as per RATIS-2028. - ReferenceCountedObject ref = ReferenceCountedObject.wrap(next); - ((TransactionContextImpl) trx).setDelegatedRef(ref); - ref.retain(); + ((TransactionContextImpl) trx).setDelegatedRef(nextRef); try { // Let the StateMachine inject logic for committed transactions in sequential order. trx = stateMachine.applyTransactionSerial(trx); @@ -1828,8 +1826,6 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf return replyPendingRequest(invocationId, TermIndex.valueOf(next), stateMachineFuture); } catch (Exception e) { throw new RaftLogIOException(e); - } finally { - ref.release(); } } return null; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index 5f6e972e2c..b01270dc0b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -235,10 +235,17 @@ private MemoizedSupplier>> applyLog() throws Raf final long committed = raftLog.getLastCommittedIndex(); for(long applied; (applied = getLastAppliedIndex()) < committed && state == State.RUNNING && !shouldStop(); ) { final long nextIndex = applied + 1; - final LogEntryProto next = raftLog.get(nextIndex); - if (next != null) { + final ReferenceCountedObject next = raftLog.retainLog(nextIndex); + if (next == null) { + LOG.debug("{}: logEntry {} is null. There may be snapshot to load. state:{}", + this, nextIndex, state); + break; + } + + try { + final LogEntryProto entry = next.get(); if (LOG.isTraceEnabled()) { - LOG.trace("{}: applying nextIndex={}, nextLog={}", this, nextIndex, LogProtoUtils.toLogEntryString(next)); + LOG.trace("{}: applying nextIndex={}, nextLog={}", this, nextIndex, LogProtoUtils.toLogEntryString(entry)); } else { LOG.debug("{}: applying nextIndex={}", this, nextIndex); } @@ -252,10 +259,8 @@ private MemoizedSupplier>> applyLog() throws Raf } else { notifyAppliedIndex(incremented); } - } else { - LOG.debug("{}: logEntry {} is null. There may be snapshot to load. state:{}", - this, nextIndex, state); - break; + } finally { + next.release(); } } return futures; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 49e66e253d..0a9a1c93ca 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -240,13 +240,19 @@ private boolean shouldAppendMetadata(long newCommitIndex) { //log neither lastMetadataEntry, nor entries with a smaller commit index. return false; } + ReferenceCountedObject ref = null; try { - if (get(newCommitIndex).hasMetadataEntry()) { + ref = retainLog(newCommitIndex); + if (ref.get().hasMetadataEntry()) { // do not log the metadata entry return false; } } catch(RaftLogIOException e) { LOG.error("Failed to get log entry for index " + newCommitIndex, e); + } finally { + if (ref != null) { + ref.release(); + } } return true; } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index fc7973aab6..feedaeee4a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -22,8 +22,10 @@ import org.apache.ratis.server.metrics.RaftLogMetricsBase; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.raftlog.RaftLogBase; import org.apache.ratis.server.raftlog.LogEntryHeader; +import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.server.storage.RaftStorageMetadata; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.AutoCloseableLock; @@ -45,8 +47,13 @@ public class MemoryRaftLog extends RaftLogBase { static class EntryList { private final List> entries = new ArrayList<>(); + ReferenceCountedObject getRef(int i) { + return i >= 0 && i < entries.size() ? entries.get(i) : null; + } + LogEntryProto get(int i) { - return i >= 0 && i < entries.size() ? entries.get(i).get() : null; + final ReferenceCountedObject ref = getRef(i); + return ref != null ? ref.get() : null; } TermIndex getTermIndex(int i) { @@ -108,16 +115,34 @@ public RaftLogMetricsBase getRaftLogMetrics() { } @Override - public LogEntryProto get(long index) { + public LogEntryProto get(long index) throws RaftLogIOException { + final ReferenceCountedObject ref = retainLog(index); + try { + return LogProtoUtils.copy(ref.get()); + } finally { + ref.release(); + } + } + + @Override + public ReferenceCountedObject retainLog(long index) { checkLogState(); - try(AutoCloseableLock readLock = readLock()) { - return entries.get(Math.toIntExact(index)); + try (AutoCloseableLock readLock = readLock()) { + ReferenceCountedObject ref = entries.getRef(Math.toIntExact(index)); + ref.retain(); + return ref; } } @Override public EntryWithData getEntryWithData(long index) { - return newEntryWithData(get(index), null); + // TODO. The reference counted object should be passed to LogAppender RATIS-2026. + ReferenceCountedObject ref = retainLog(index); + try { + return newEntryWithData(ref.get(), null); + } finally { + ref.release(); + } } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 68da35014f..2fcd7914e5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -224,7 +224,7 @@ private void assertSegment(long expectedStart, int expectedEntryCount, boolean c * * In the future we can make the cache loader configurable if necessary. */ - class LogEntryLoader extends CacheLoader { + class LogEntryLoader extends CacheLoader> { private final SegmentedRaftLogMetrics raftLogMetrics; LogEntryLoader(SegmentedRaftLogMetrics raftLogMetrics) { @@ -232,18 +232,19 @@ class LogEntryLoader extends CacheLoader { } @Override - public LogEntryProto load(LogRecord key) throws IOException { + public ReferenceCountedObject load(LogRecord key) throws IOException { final File file = getFile(); // note the loading should not exceed the endIndex: it is possible that // the on-disk log file should be truncated but has not been done yet. - final AtomicReference toReturn = new AtomicReference<>(); + final AtomicReference> toReturn = new AtomicReference<>(); final LogSegmentStartEnd startEnd = LogSegmentStartEnd.valueOf(startIndex, endIndex, isOpen); readSegmentFile(file, startEnd, maxOpSize, getLogCorruptionPolicy(), raftLogMetrics, entryRef -> { final LogEntryProto entry = entryRef.retain(); final TermIndex ti = TermIndex.valueOf(entry); putEntryCache(ti, entryRef, Op.LOAD_SEGMENT_FILE); if (ti.equals(key.getTermIndex())) { - toReturn.set(entry); + entryRef.retain(); + toReturn.set(entryRef); } entryRef.release(); }); @@ -260,10 +261,8 @@ long size() { return size.get(); } - LogEntryProto get(TermIndex ti) { - return Optional.ofNullable(map.get(ti)) - .map(ReferenceCountedObject::get) - .orElse(null); + ReferenceCountedObject get(TermIndex ti) { + return map.get(ti); } void clear() { @@ -386,15 +385,15 @@ private LogRecord appendLogRecord(Op op, LogEntryProto entry) { return record; } - LogEntryProto getEntryFromCache(TermIndex ti) { + ReferenceCountedObject getEntryFromCache(TermIndex ti) { return entryCache.get(ti); } /** * Acquire LogSegment's monitor so that there is no concurrent loading. */ - synchronized LogEntryProto loadCache(LogRecord record) throws RaftLogIOException { - LogEntryProto entry = entryCache.get(record.getTermIndex()); + synchronized ReferenceCountedObject loadCache(LogRecord record) throws RaftLogIOException { + ReferenceCountedObject entry = entryCache.get(record.getTermIndex()); if (entry != null) { return entry; } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index baac0c6c7f..bb0793abe3 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -180,11 +180,17 @@ public long getLastAppliedIndex() { @Override public void notifyTruncatedLogEntry(TermIndex ti) { + ReferenceCountedObject ref = null; try { - final LogEntryProto entry = get(ti.getIndex()); + ref = retainLog(ti.getIndex()); + final LogEntryProto entry = ref != null ? ref.get() : null; notifyTruncatedLogEntry.accept(entry); } catch (RaftLogIOException e) { LOG.error("{}: Failed to read log {}", getName(), ti, e); + } finally { + if (ref != null) { + ref.release(); + } } } @@ -272,6 +278,19 @@ private void loadLogSegments(long lastIndexInSnapshot, @Override public LogEntryProto get(long index) throws RaftLogIOException { + final ReferenceCountedObject ref = retainLog(index); + if (ref == null) { + return null; + } + try { + return LogProtoUtils.copy(ref.get()); + } finally { + ref.release(); + } + } + + @Override + public ReferenceCountedObject retainLog(long index) throws RaftLogIOException { checkLogState(); final LogSegment segment; final LogRecord record; @@ -284,9 +303,10 @@ record = segment.getLogRecord(index); if (record == null) { return null; } - final LogEntryProto entry = segment.getEntryFromCache(record.getTermIndex()); + final ReferenceCountedObject entry = segment.getEntryFromCache(record.getTermIndex()); if (entry != null) { getRaftLogMetrics().onRaftLogCacheHit(); + entry.retain(); return entry; } } @@ -299,10 +319,19 @@ record = segment.getLogRecord(index); @Override public EntryWithData getEntryWithData(long index) throws RaftLogIOException { - final LogEntryProto entry = get(index); - if (entry == null) { + final ReferenceCountedObject entryRef = retainLog(index); + if (entryRef == null) { throw new RaftLogIOException("Log entry not found: index = " + index); } + try { + // TODO. The reference counted object should be passed to LogAppender RATIS-2026. + return getEntryWithData(entryRef.get()); + } finally { + entryRef.release(); + } + } + + private EntryWithData getEntryWithData(LogEntryProto entry) throws RaftLogIOException { if (!LogProtoUtils.isStateMachineDataEmpty(entry)) { return newEntryWithData(entry, null); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java index bb4f6a076c..ee30bd2c79 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java @@ -21,12 +21,15 @@ import static org.apache.ratis.server.metrics.SegmentedRaftLogMetrics.RATIS_LOG_WORKER_METRICS; import org.apache.ratis.metrics.RatisMetrics; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; +import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.RaftLogBase; import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.util.AutoCloseableLock; +import org.apache.ratis.util.ReferenceCountedObject; import java.io.File; import java.io.IOException; @@ -72,11 +75,22 @@ static void printLog(RaftLogBase log, Consumer println) { b.append(i == committed? 'c': ' '); b.append(String.format("%3d: ", i)); try { - b.append(LogProtoUtils.toLogEntryString(log.get(i))); + b.append(LogProtoUtils.toLogEntryString(getLogUnsafe(log, i))); } catch (RaftLogIOException e) { b.append(e); } println.accept(b.toString()); } } + + static LogEntryProto getLogUnsafe(RaftLog log, long index) throws RaftLogIOException { + ReferenceCountedObject ref = log.retainLog(index); + try { + return ref != null ? ref.get() : null; + } finally { + if (ref != null) { + ref.release(); + } + } + } } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index fe1a97ddca..9a716cad39 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -21,6 +21,7 @@ import static org.apache.ratis.server.impl.StateMachineMetrics.RATIS_STATEMACHINE_METRICS_DESC; import static org.apache.ratis.server.impl.StateMachineMetrics.STATEMACHINE_TAKE_SNAPSHOT_TIMER; import static org.apache.ratis.metrics.RatisMetrics.RATIS_APPLICATION_NAME_METRICS; +import static org.apache.ratis.server.storage.RaftStorageTestUtils.getLogUnsafe; import org.apache.ratis.BaseTest; import org.apache.ratis.metrics.LongCounter; @@ -43,6 +44,7 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.segmented.LogSegmentPath; import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.server.storage.RaftStorageTestUtils; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage; import org.apache.ratis.util.FileUtils; @@ -95,7 +97,7 @@ public static void assertLeaderContent(MiniRaftCluster cluster) throws Exception public static void assertLogContent(RaftServer.Division server, boolean isLeader) throws Exception { final RaftLog log = server.getRaftLog(); final long lastIndex = log.getLastEntryTermIndex().getIndex(); - final LogEntryProto e = log.get(lastIndex); + final LogEntryProto e = getLogUnsafe(log, lastIndex); Assert.assertTrue(e.hasMetadataEntry()); JavaUtils.attemptRepeatedly(() -> { diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index 2970bbef84..47138919df 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -69,6 +69,8 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ThreadLocalRandom; +import static org.apache.ratis.server.storage.RaftStorageTestUtils.getLogUnsafe; + public interface DataStreamTestUtils { Logger LOG = LoggerFactory.getLogger(DataStreamTestUtils.class); @@ -383,7 +385,7 @@ static void assertRaftClientMessage( static LogEntryProto searchLogEntry(ClientInvocationId invocationId, RaftLog log) throws Exception { for (LogEntryHeader termIndex : log.getEntries(0, Long.MAX_VALUE)) { - final LogEntryProto entry = log.get(termIndex.getIndex()); + final LogEntryProto entry = getLogUnsafe(log, termIndex.getIndex()); if (entry.hasStateMachineLogEntry()) { if (invocationId.match(entry.getStateMachineLogEntry())) { return entry; diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java index db4e92b7ce..11311f3602 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java @@ -65,6 +65,8 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static org.apache.ratis.server.storage.RaftStorageTestUtils.getLogUnsafe; + /** * Test restarting raft peers. */ @@ -268,10 +270,10 @@ void runTestRestartCommitIndex(MiniRaftCluster cluster) throws Exception { final long lastIndex = leaderLog.getLastEntryTermIndex().getIndex(); LOG.info("{}: leader lastIndex={}", leaderId, lastIndex); - final LogEntryProto lastEntry = leaderLog.get(lastIndex); + final LogEntryProto lastEntry = getLogUnsafe(leaderLog, lastIndex); LOG.info("{}: leader lastEntry entry[{}] = {}", leaderId, lastIndex, LogProtoUtils.toLogEntryString(lastEntry)); final long loggedCommitIndex = lastEntry.getMetadataEntry().getCommitIndex(); - final LogEntryProto lastCommittedEntry = leaderLog.get(loggedCommitIndex); + final LogEntryProto lastCommittedEntry = getLogUnsafe(leaderLog, loggedCommitIndex); LOG.info("{}: leader lastCommittedEntry = entry[{}] = {}", leaderId, loggedCommitIndex, LogProtoUtils.toLogEntryString(lastCommittedEntry)); @@ -317,11 +319,11 @@ void runTestRestartCommitIndex(MiniRaftCluster cluster) throws Exception { static void assertLastLogEntry(RaftServer.Division server) throws RaftLogIOException { final RaftLog raftLog = server.getRaftLog(); final long lastIndex = raftLog.getLastEntryTermIndex().getIndex(); - final LogEntryProto lastEntry = raftLog.get(lastIndex); + final LogEntryProto lastEntry = getLogUnsafe(raftLog, lastIndex); Assertions.assertTrue(lastEntry.hasMetadataEntry()); final long loggedCommitIndex = lastEntry.getMetadataEntry().getCommitIndex(); - final LogEntryProto lastCommittedEntry = raftLog.get(loggedCommitIndex); + final LogEntryProto lastCommittedEntry = getLogUnsafe(raftLog, loggedCommitIndex); Assertions.assertTrue(lastCommittedEntry.hasStateMachineLogEntry()); final SimpleStateMachine4Testing leaderStateMachine = SimpleStateMachine4Testing.get(server); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java index 8355c67333..7692ad06b6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java @@ -141,11 +141,11 @@ static void checkLogSegment(LogSegment segment, long start, long end, Assertions.assertEquals(term, ti.getTerm()); Assertions.assertEquals(offset, record.getOffset()); - LogEntryProto entry = segment.getEntryFromCache(ti); + ReferenceCountedObject entry = segment.getEntryFromCache(ti); if (entry == null) { entry = segment.loadCache(record); } - offset += getEntrySize(entry, Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + offset += getEntrySize(entry.get(), Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 55fd6fbdf7..7b20babf4d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -74,6 +74,7 @@ import static java.lang.Boolean.FALSE; import static java.lang.Boolean.TRUE; +import static org.apache.ratis.server.storage.RaftStorageTestUtils.getLogUnsafe; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.params.provider.Arguments.arguments; @@ -204,7 +205,7 @@ static List prepareRanges(int startTerm, int endTerm, int segmentS private LogEntryProto getLastEntry(SegmentedRaftLog raftLog) throws IOException { - return raftLog.get(raftLog.getLastEntryTermIndex().getIndex()); + return getLogUnsafe(raftLog, raftLog.getLastEntryTermIndex().getIndex()); } @ParameterizedTest @@ -229,7 +230,7 @@ public void testLoadLogSegments(Boolean useAsyncFlush, Boolean smSyncFlush) thro LogEntryProto[] entriesFromLog = Arrays.stream(termIndices) .map(ti -> { try { - return raftLog.get(ti.getIndex()); + return getLogUnsafe(raftLog, ti.getIndex()); } catch (IOException e) { throw new RuntimeException(e); } @@ -451,7 +452,7 @@ private void checkEntries(RaftLog raftLog, List expected, LogEntryProto[] entriesFromLog = Arrays.stream(termIndices) .map(ti -> { try { - return raftLog.get(ti.getIndex()); + return getLogUnsafe(raftLog, ti.getIndex()); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index fa892b2003..87172323d6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -81,8 +81,8 @@ private void checkCache(long start, long end, int segmentSize) { for (long index = start; index <= end; index++) { final LogSegment segment = cache.getSegment(index); final LogRecord record = segment.getLogRecord(index); - final LogEntryProto entry = segment.getEntryFromCache(record.getTermIndex()); - Assertions.assertEquals(index, entry.getIndex()); + final ReferenceCountedObject entry = segment.getEntryFromCache(record.getTermIndex()); + Assertions.assertEquals(index, entry.get().getIndex()); } long[] offsets = new long[]{start, start + 1, start + (end - start) / 2, From 422cb9d4cb3f6be8e3169797d11a3c2e666c138f Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Fri, 15 Mar 2024 09:22:03 +0800 Subject: [PATCH 43/76] RATIS-2043. Change toString() of RaftConfigurationImpl (#1051) --- .../org/apache/ratis/server/impl/RaftConfigurationImpl.java | 2 +- .../main/java/org/apache/ratis/server/impl/RaftServerImpl.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java index d609264af5..98f41020e0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java @@ -298,7 +298,7 @@ boolean isSingleton() { @Override public String toString() { - return logEntryIndex + ": " + conf + ", old=" + oldConf; + return "conf: {index: " + logEntryIndex + ", cur=" + conf + ", old=" + oldConf + "}"; } boolean hasNoChange(Collection newMembers, Collection newListeners) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index e8aeb66784..23cab70d3b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1210,7 +1210,7 @@ CompletableFuture transferLeadershipAsync(TransferLeadershipReq if (!conf.isHighestPriority(request.getNewLeader())) { String msg = getMemberId() + " refused to transfer leadership to peer " + request.getNewLeader() + - " as it does not has highest priority " + conf; + " as it does not has highest priority in " + conf; return logAndReturnTransferLeadershipFail(request, msg); } From 44a4f1fb73a073589a4563d492f6acfc68ce4a5d Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Mon, 18 Mar 2024 23:15:14 +0800 Subject: [PATCH 44/76] RATIS-2044. Fix ReadIndex loss caused by data race in AppendEntriesListeners (#1052) --- .../ratis/server/impl/ReadIndexHeartbeats.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java index d08a1ea406..4ff1460d7d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java @@ -23,6 +23,7 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.RaftLogIndex; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -123,8 +124,15 @@ boolean isAcknowledged(RaftPeerId id) { class AppendEntriesListeners { private final NavigableMap sorted = new TreeMap<>(); + private Exception exception = null; synchronized AppendEntriesListener add(long commitIndex, Function constructor) { + if (exception != null) { + Preconditions.assertTrue(sorted.isEmpty()); + final AppendEntriesListener listener = constructor.apply(commitIndex); + listener.getFuture().completeExceptionally(exception); + return listener; + } return sorted.computeIfAbsent(commitIndex, constructor); } @@ -152,6 +160,10 @@ synchronized void onAppendEntriesReply(LogAppender appender, AppendEntriesReplyP } synchronized void failAll(Exception e) { + if (exception != null) { + return; + } + exception = e; sorted.forEach((index, listener) -> listener.getFuture().completeExceptionally(e)); sorted.clear(); } From 72c060d9cc524045ce55e830db070c5514f23ed4 Mon Sep 17 00:00:00 2001 From: Hemant Kumar Date: Wed, 20 Mar 2024 16:26:43 -0700 Subject: [PATCH 45/76] RATIS-2045. SnapshotInstallationHandler doesn't notify follower when snapshotIndex is -1 and firstAvailableLogIndex is 0 (#1053) --- .../apache/ratis/server/impl/SnapshotInstallationHandler.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 3e5ac2b671..f03e2d883d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -221,7 +221,8 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( // Check if snapshot index is already at par or ahead of the first // available log index of the Leader. final long snapshotIndex = state.getLog().getSnapshotIndex(); - if (snapshotIndex + 1 >= firstAvailableLogIndex && firstAvailableLogIndex > INVALID_LOG_INDEX) { + if (snapshotIndex != INVALID_LOG_INDEX && snapshotIndex + 1 >= firstAvailableLogIndex && + firstAvailableLogIndex > INVALID_LOG_INDEX) { // State Machine has already installed the snapshot. Return the // latest snapshot index to the Leader. From c3d615b4f28b92e1d271b84752f1c9b0fc23341c Mon Sep 17 00:00:00 2001 From: Potato Date: Sat, 23 Mar 2024 00:53:30 +0800 Subject: [PATCH 46/76] RATIS-2047. Avoid unnecessary warn log when creating raft group (#1054) --- .../ratis/server/storage/RaftStorageImpl.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageImpl.java index fbb7bf7d46..ce809cad8f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageImpl.java @@ -19,7 +19,6 @@ import java.io.InputStream; import java.io.OutputStream; -import java.nio.file.NoSuchFileException; import java.util.concurrent.atomic.AtomicReference; import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.server.RaftConfiguration; @@ -153,14 +152,16 @@ public void writeRaftConfiguration(LogEntryProto conf) { public RaftConfiguration readRaftConfiguration() { File confFile = storageDir.getMetaConfFile(); - try (InputStream fio = FileUtils.newInputStream(confFile)) { - LogEntryProto confProto = LogEntryProto.newBuilder().mergeFrom(fio).build(); - return LogProtoUtils.toRaftConfiguration(confProto); - } catch (FileNotFoundException | NoSuchFileException e) { - return null; - } catch (Exception e) { - LOG.error("Failed reading configuration from file:" + confFile, e); + if (!confFile.exists()) { return null; + } else { + try (InputStream fio = FileUtils.newInputStream(confFile)) { + LogEntryProto confProto = LogEntryProto.newBuilder().mergeFrom(fio).build(); + return LogProtoUtils.toRaftConfiguration(confProto); + } catch (Exception e) { + LOG.error("Failed reading configuration from file:" + confFile, e); + return null; + } } } From be497d5c472038242f8493ef5e56b56b7ce69ef1 Mon Sep 17 00:00:00 2001 From: Potato Date: Sat, 23 Mar 2024 00:58:31 +0800 Subject: [PATCH 47/76] RATIS-2048. Fix SimpleStateMachineStorage potential NPE exception (#1056) --- .../ratis/statemachine/impl/SimpleStateMachineStorage.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java index 37e94a8a0a..88cc57dab5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java @@ -116,7 +116,7 @@ public void cleanupOldSnapshots(SnapshotRetentionPolicy snapshotRetentionPolicy) final List allSnapshotFiles = getSingleFileSnapshotInfos(stateMachineDir.toPath()); - if (allSnapshotFiles.size() > snapshotRetentionPolicy.getNumSnapshotsRetained()) { + if (allSnapshotFiles.size() > numSnapshotsRetained) { allSnapshotFiles.sort(Comparator.comparing(SingleFileSnapshotInfo::getIndex).reversed()); allSnapshotFiles.subList(numSnapshotsRetained, allSnapshotFiles.size()) .stream() From 0f631fef459b1263f266fa443fdb26cfbc2f0a75 Mon Sep 17 00:00:00 2001 From: Potato Date: Sun, 24 Mar 2024 02:11:39 +0800 Subject: [PATCH 48/76] RATIS-2036. Avoid trigger snapshot when removing raftGroup (#1055) --- .../ratis/server/RaftServerConfigKeys.java | 12 ++++++++ .../ratis/server/impl/RaftServerImpl.java | 1 + .../server/impl/StateMachineUpdater.java | 28 ++++++++++++++++++- 3 files changed, 40 insertions(+), 1 deletion(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java index 565e881269..7419ca095f 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java @@ -647,6 +647,18 @@ static void setTriggerWhenStopEnabled(RaftProperties properties, boolean trigger setBoolean(properties::setBoolean, TRIGGER_WHEN_STOP_ENABLED_KEY, triggerWhenStopEnabled); } + /** whether trigger snapshot when remove raft server */ + String TRIGGER_WHEN_REMOVE_ENABLED_KEY = PREFIX + ".trigger-when-remove.enabled"; + /** by default let the state machine to trigger snapshot when remove */ + boolean TRIGGER_WHEN_REMOVE_ENABLED_DEFAULT = true; + static boolean triggerWhenRemoveEnabled(RaftProperties properties) { + return getBoolean(properties::getBoolean, + TRIGGER_WHEN_REMOVE_ENABLED_KEY, TRIGGER_WHEN_REMOVE_ENABLED_DEFAULT, getDefaultLog()); + } + static void setTriggerWhenRemoveEnabled(RaftProperties properties, boolean triggerWhenRemoveEnabled) { + setBoolean(properties::setBoolean, TRIGGER_WHEN_REMOVE_ENABLED_KEY, triggerWhenRemoveEnabled); + } + /** The log index gap between to two snapshot creations. */ String CREATION_GAP_KEY = PREFIX + ".creation.gap"; long CREATION_GAP_DEFAULT = 1024; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 23cab70d3b..0ea3746293 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -461,6 +461,7 @@ void groupRemove(boolean deleteDirectory, boolean renameDirectory) { final RaftStorageDirectory dir = state.getStorage().getStorageDir(); /* Shutdown is triggered here inorder to avoid any locked files. */ + state.getStateMachineUpdater().setRemoving(); close(); getStateMachine().event().notifyGroupRemove(); if (deleteDirectory) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index b01270dc0b..fbd7f013fa 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -74,6 +74,8 @@ enum State { private final boolean triggerSnapshotWhenStopEnabled; + private final boolean triggerSnapshotWhenRemoveEnabled; + private final Long autoSnapshotThreshold; private final boolean purgeUptoSnapshotIndex; @@ -91,6 +93,8 @@ enum State { private final Consumer appliedIndexConsumer; + private volatile boolean isRemoving; + StateMachineUpdater(StateMachine stateMachine, RaftServerImpl server, ServerState serverState, long lastAppliedIndex, RaftProperties properties, Consumer appliedIndexConsumer) { this.name = serverState.getMemberId() + "-" + JavaUtils.getClassSimpleName(getClass()); @@ -106,6 +110,7 @@ enum State { this.snapshotIndex = new RaftLogIndex("snapshotIndex", lastAppliedIndex); this.triggerSnapshotWhenStopEnabled = RaftServerConfigKeys.Snapshot.triggerWhenStopEnabled(properties); + this.triggerSnapshotWhenRemoveEnabled = RaftServerConfigKeys.Snapshot.triggerWhenRemoveEnabled(properties); final boolean autoSnapshot = RaftServerConfigKeys.Snapshot.autoTriggerEnabled(properties); this.autoSnapshotThreshold = autoSnapshot? RaftServerConfigKeys.Snapshot.autoTriggerThreshold(properties): null; final int numSnapshotFilesRetained = RaftServerConfigKeys.Snapshot.retentionFileNum(properties); @@ -327,12 +332,33 @@ private boolean shouldTakeSnapshot() { if (autoSnapshotThreshold == null) { return false; } else if (shouldStop()) { - return triggerSnapshotWhenStopEnabled && getLastAppliedIndex() - snapshotIndex.get() > 0; + return shouldTakeSnapshotAtStop() && getLastAppliedIndex() - snapshotIndex.get() > 0; } return state == State.RUNNING && getStateMachineLastAppliedIndex() - snapshotIndex.get() >= autoSnapshotThreshold; } + /** + * In view of the three variables triggerSnapshotWhenStopEnabled, triggerSnapshotWhenRemoveEnabled and isRemoving, + * we can draw the following 8 combination: + * true true true => true + * true true false => true + * true false true => false + * true false false => true + * false true true => true + * false true false => false + * false false true => false + * false false false => false + * @return result + */ + private boolean shouldTakeSnapshotAtStop() { + return isRemoving ? triggerSnapshotWhenRemoveEnabled : triggerSnapshotWhenStopEnabled; + } + + void setRemoving() { + this.isRemoving = true; + } + private long getLastAppliedIndex() { return appliedIndex.get(); } From 54c2b1f7f82c5cb66fa6b652a29ff9cef22bf4b1 Mon Sep 17 00:00:00 2001 From: Potato Date: Mon, 25 Mar 2024 23:12:57 +0800 Subject: [PATCH 49/76] RATIS-2049. Suppress Sonar warning for volatile objects (#1057) --- .../java/org/apache/ratis/client/impl/RaftClientImpl.java | 1 + .../java/org/apache/ratis/util/MemoizedCheckedSupplier.java | 2 ++ .../src/main/java/org/apache/ratis/util/MemoizedSupplier.java | 1 + .../java/org/apache/ratis/examples/debug/server/Server.java | 1 + .../java/org/apache/ratis/examples/filestore/cli/Client.java | 3 +++ .../org/apache/ratis/examples/membership/server/CServer.java | 4 +--- .../java/org/apache/ratis/grpc/server/GrpcLogAppender.java | 4 ++-- .../main/java/org/apache/ratis/server/leader/LogAppender.java | 3 ++- .../java/org/apache/ratis/server/storage/TestRaftStorage.java | 1 + 9 files changed, 14 insertions(+), 6 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java index 1b82709daf..db789aef2f 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java @@ -172,6 +172,7 @@ private synchronized Set getAndReset() { private final RaftGroupId groupId; private final RetryPolicy retryPolicy; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftPeerId leaderId; /** The callIds of the replied requests. */ private final RepliedCallIds repliedCallIds; diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java index cf2d060239..0e9ae44fa5 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java @@ -44,6 +44,8 @@ public static MemoizedCheckedSupplier initializer; + + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RETURN value = null; /** diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java index f179d2dcd0..5c2754a307 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java @@ -40,6 +40,7 @@ public static MemoizedSupplier valueOf(Supplier supplier) { } private final Supplier initializer; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile T value = null; /** diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/debug/server/Server.java b/ratis-examples/src/main/java/org/apache/ratis/examples/debug/server/Server.java index 4377a1420f..07f7b20f44 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/debug/server/Server.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/debug/server/Server.java @@ -34,6 +34,7 @@ public final class Server { private Server(){ } + @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning public static void main(String[] args) throws IOException { if (args.length < 1) { System.err.println("The arguments should be "); diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java index 1856fc9b5d..caf2aa59b2 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java @@ -159,6 +159,9 @@ protected void dropCache() { Process pro = Runtime.getRuntime().exec(cmds); pro.waitFor(); } catch (Throwable t) { + if (t instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } System.err.println("Failed to run command:" + Arrays.toString(cmds) + ":" + t.getMessage()); } } diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/membership/server/CServer.java b/ratis-examples/src/main/java/org/apache/ratis/examples/membership/server/CServer.java index a846cd1e77..2145412e04 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/membership/server/CServer.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/membership/server/CServer.java @@ -32,7 +32,6 @@ import org.apache.ratis.thirdparty.com.google.common.base.MoreObjects; import org.apache.ratis.util.FileUtils; -import java.io.Closeable; import java.io.File; import java.io.IOException; import java.util.Collections; @@ -40,7 +39,7 @@ /** * A simple raft server using {@link CounterStateMachine}. */ -public class CServer implements Closeable { +public class CServer { public static final RaftGroupId GROUP_ID = RaftGroupId.randomId(); public static final String LOCAL_ADDR = "0.0.0.0"; @@ -78,7 +77,6 @@ public RaftPeer getPeer() { return server.getPeer(); } - @Override public void close() throws IOException { server.close(); FileUtils.deleteFully(storageDir); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index ec44d8c485..5f9c94eb24 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -156,7 +156,6 @@ synchronized int process(Event event) { private final TimeDuration logMessageBatchDuration; private final int maxOutstandingInstallSnapshots; private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); - private volatile StreamObservers appendLogRequestObserver; private final boolean useSeparateHBChannel; @@ -863,6 +862,7 @@ private TermIndex shouldNotifyToInstallSnapshot() { static class AppendEntriesRequest { private final Timekeeper timer; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Timekeeper.Context timerContext; private final long callId; @@ -871,7 +871,7 @@ static class AppendEntriesRequest { private final TermIndex firstEntry; private final TermIndex lastEntry; - + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Timestamp sendTime; AppendEntriesRequest(AppendEntriesRequestProto proto, RaftPeerId followerId, GrpcServerMetrics grpcServerMetrics) { diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java index 020a352c05..36331e3abb 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java @@ -87,10 +87,11 @@ default void stop() { default CompletableFuture stopAsync() { stop(); return CompletableFuture.supplyAsync(() -> { - for (; isRunning(); ) { + while (isRunning()) { try { Thread.sleep(10); } catch (InterruptedException e) { + Thread.currentThread().interrupt(); throw new CompletionException("stopAsync interrupted", e); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java index eb65f342e3..12cd771315 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java @@ -87,6 +87,7 @@ static RaftStorageImpl formatRaftStorage(File dir) throws IOException { return impl; } + @SuppressWarnings({"squid:S5783"}) // Suppress same exception warning @Test public void testNotExistent() throws IOException { FileUtils.deleteFully(storageDir); From 14dbbf3d7cf8c78788fe7be93abf100c7dd52580 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Tue, 26 Mar 2024 22:52:06 +0800 Subject: [PATCH 50/76] RATIS-2050. Add creationGap param to snapshot management API (#1058) --- .../client/api/SnapshotManagementApi.java | 22 +++++++++++++++++-- .../ratis/client/impl/ClientProtoUtils.java | 5 +++-- .../client/impl/SnapshotManagementImpl.java | 5 +++-- .../protocol/SnapshotManagementRequest.java | 18 +++++++++++++-- ratis-proto/src/main/proto/Raft.proto | 2 +- .../ratis/server/impl/RaftServerImpl.java | 5 +++-- 6 files changed, 46 insertions(+), 11 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java b/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java index edd0475442..f83d976040 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java @@ -27,6 +27,24 @@ */ public interface SnapshotManagementApi { - /** trigger create snapshot file. */ - RaftClientReply create(long timeoutMs) throws IOException; + /** The same as create(0, timeoutMs). */ + default RaftClientReply create(long timeoutMs) throws IOException { + return create(0, timeoutMs); + } + + /** The same as create(force? 1 : 0, timeoutMs). */ + default RaftClientReply create(boolean force, long timeoutMs) throws IOException { + return create(force? 1 : 0, timeoutMs); + } + + /** + * Trigger to create a snapshot. + * + * @param creationGap When (creationGap > 0) and (astAppliedIndex - lastSnapshotIndex < creationGap), + * return lastSnapshotIndex; otherwise, take a new snapshot and then return its index. + * When creationGap == 0, use the server configured value as the creationGap. + * @return a reply. When {@link RaftClientReply#isSuccess()} is true, + * {@link RaftClientReply#getLogIndex()} is the snapshot index fulfilling the operation. + */ + RaftClientReply create(long creationGap, long timeoutMs) throws IOException; } diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java index 003f202bd9..cab9606a0e 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java @@ -659,7 +659,8 @@ static SnapshotManagementRequest toSnapshotManagementRequest(SnapshotManagementR switch(p.getOpCase()) { case CREATE: return SnapshotManagementRequest.newCreate(clientId, serverId, - ProtoUtils.toRaftGroupId(m.getRaftGroupId()), m.getCallId(), m.getTimeoutMs()); + ProtoUtils.toRaftGroupId(m.getRaftGroupId()), m.getCallId(), m.getTimeoutMs(), + p.getCreate().getCreationGap()); default: throw new IllegalArgumentException("Unexpected op " + p.getOpCase() + " in " + p); } @@ -671,7 +672,7 @@ static SnapshotManagementRequestProto toSnapshotManagementRequestProto( .setRpcRequest(toRaftRpcRequestProtoBuilder(request)); final SnapshotManagementRequest.Create create = request.getCreate(); if (create != null) { - b.setCreate(SnapshotCreateRequestProto.newBuilder().build()); + b.setCreate(SnapshotCreateRequestProto.newBuilder().setCreationGap(create.getCreationGap()).build()); } return b.build(); } diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/SnapshotManagementImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/SnapshotManagementImpl.java index 1762dc0e49..65c54d0f21 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/SnapshotManagementImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/SnapshotManagementImpl.java @@ -37,9 +37,10 @@ class SnapshotManagementImpl implements SnapshotManagementApi { } @Override - public RaftClientReply create(long timeoutMs) throws IOException { + public RaftClientReply create(long creationGap, long timeoutMs) throws IOException { final long callId = CallId.getAndIncrement(); return client.io().sendRequestWithRetry(() -> SnapshotManagementRequest.newCreate(client.getId(), - Optional.ofNullable(server).orElseGet(client::getLeaderId), client.getGroupId(), callId, timeoutMs)); + Optional.ofNullable(server).orElseGet(client::getLeaderId), + client.getGroupId(), callId, timeoutMs, creationGap)); } } diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/SnapshotManagementRequest.java b/ratis-common/src/main/java/org/apache/ratis/protocol/SnapshotManagementRequest.java index 2ea2059b51..269fdfc591 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/SnapshotManagementRequest.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/SnapshotManagementRequest.java @@ -24,7 +24,16 @@ public final class SnapshotManagementRequest extends RaftClientRequest { public abstract static class Op { } - public static class Create extends Op { + + public static final class Create extends Op { + private final long creationGap; + private Create(long creationGap) { + this.creationGap = creationGap; + } + + public long getCreationGap() { + return creationGap; + } @Override public String toString() { @@ -35,8 +44,13 @@ public String toString() { public static SnapshotManagementRequest newCreate(ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId, long timeoutMs) { + return newCreate(clientId, serverId, groupId, callId, timeoutMs, 0); + } + + public static SnapshotManagementRequest newCreate(ClientId clientId, + RaftPeerId serverId, RaftGroupId groupId, long callId, long timeoutMs, long creationGap) { return new SnapshotManagementRequest(clientId, - serverId, groupId, callId, timeoutMs,new SnapshotManagementRequest.Create()); + serverId, groupId, callId, timeoutMs, new SnapshotManagementRequest.Create(creationGap)); } private final Op op; diff --git a/ratis-proto/src/main/proto/Raft.proto b/ratis-proto/src/main/proto/Raft.proto index edc57ec65e..b2e96e283e 100644 --- a/ratis-proto/src/main/proto/Raft.proto +++ b/ratis-proto/src/main/proto/Raft.proto @@ -470,7 +470,7 @@ message SnapshotManagementRequestProto { } message SnapshotCreateRequestProto { - + uint64 creationGap = 1; } message StartLeaderElectionRequestProto { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 0ea3746293..2cec095785 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1223,9 +1223,10 @@ CompletableFuture takeSnapshotAsync(SnapshotManagementRequest r LOG.info("{}: takeSnapshotAsync {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); assertGroup(getMemberId(), request); + Preconditions.assertNotNull(request.getCreate(), "create"); - //TODO(liuyaolong): get the gap value from shell command - long minGapValue = RaftServerConfigKeys.Snapshot.creationGap(proxy.getProperties()); + final long creationGap = request.getCreate().getCreationGap(); + long minGapValue = creationGap > 0? creationGap : RaftServerConfigKeys.Snapshot.creationGap(proxy.getProperties()); final long lastSnapshotIndex = Optional.ofNullable(stateMachine.getLatestSnapshot()) .map(SnapshotInfo::getIndex) .orElse(0L); From 397c2efdd79ff41ebf57a38bcc3681f2126091ec Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Tue, 26 Mar 2024 20:18:16 -0700 Subject: [PATCH 51/76] RATIS-2026. LogAppender to consume log entries with reference count (#1049) --- .../ratis/grpc/server/GrpcLogAppender.java | 29 ++++++++---- .../ratis/server/leader/LogAppender.java | 2 + .../apache/ratis/server/raftlog/RaftLog.java | 23 ++++++++++ .../ratis/server/leader/LogAppenderBase.java | 46 ++++++++++++++++--- .../server/leader/LogAppenderDefault.java | 33 ++++++++----- .../ratis/server/raftlog/RaftLogBase.java | 13 ++++-- .../server/raftlog/memory/MemoryRaftLog.java | 16 +++---- .../raftlog/segmented/SegmentedRaftLog.java | 18 ++++---- 8 files changed, 130 insertions(+), 50 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 5f9c94eb24..e23f2826e3 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -378,30 +378,39 @@ public Comparator getCallIdComparator() { } private void appendLog(boolean heartbeat) throws IOException { - final AppendEntriesRequestProto pending; + ReferenceCountedObject pending = null; final AppendEntriesRequest request; try (AutoCloseableLock writeLock = lock.writeLock(caller, LOG::trace)) { // Prepare and send the append request. // Note changes on follower's nextIndex and ops on pendingRequests should always be done under the write-lock - pending = newAppendEntriesRequest(callId.getAndIncrement(), heartbeat); + pending = nextAppendEntriesRequest(callId.getAndIncrement(), heartbeat); if (pending == null) { return; } - request = new AppendEntriesRequest(pending, getFollowerId(), grpcServerMetrics); + request = new AppendEntriesRequest(pending.get(), getFollowerId(), grpcServerMetrics); pendingRequests.put(request); - increaseNextIndex(pending); + increaseNextIndex(pending.get()); if (appendLogRequestObserver == null) { appendLogRequestObserver = new StreamObservers( getClient(), new AppendLogResponseHandler(), useSeparateHBChannel, getWaitTimeMin()); } + } catch(Exception e) { + if (pending != null) { + pending.release(); + } + throw e; } - final TimeDuration remaining = getRemainingWaitTime(); - if (remaining.isPositive()) { - sleep(remaining, heartbeat); - } - if (isRunning()) { - sendRequest(request, pending); + try { + final TimeDuration remaining = getRemainingWaitTime(); + if (remaining.isPositive()) { + sleep(remaining, heartbeat); + } + if (isRunning()) { + sendRequest(request, pending.get()); + } + } finally { + pending.release(); } } diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java index 36331e3abb..78f61300b8 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java @@ -125,7 +125,9 @@ default RaftPeerId getFollowerId() { * @param heartbeat the returned request must be a heartbeat. * * @return a new {@link AppendEntriesRequestProto} object. + * @deprecated this is no longer a public API. */ + @Deprecated AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean heartbeat) throws RaftLogIOException; /** @return a new {@link InstallSnapshotRequestProto} object. */ diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java index e4fbd664ea..ca785a4a6d 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java @@ -79,9 +79,23 @@ default ReferenceCountedObject retainLog(long index) throws RaftL /** * @return null if the log entry is not found in this log; * otherwise, return the {@link EntryWithData} corresponding to the given index. + * @deprecated use {@link #retainEntryWithData(long)}. */ + @Deprecated EntryWithData getEntryWithData(long index) throws RaftLogIOException; + /** + * @return null if the log entry is not found in this log; + * otherwise, return a retained reference of the {@link EntryWithData} corresponding to the given index. + * Since the returned reference is retained, the caller must call {@link ReferenceCountedObject#release()}} + * after use. + */ + default ReferenceCountedObject retainEntryWithData(long index) throws RaftLogIOException { + final ReferenceCountedObject wrap = ReferenceCountedObject.wrap(getEntryWithData(index)); + wrap.retain(); + return wrap; +} + /** * @param startIndex the starting log index (inclusive) * @param endIndex the ending log index (exclusive) @@ -172,6 +186,15 @@ default long getNextIndex() { * containing both the log entry and the state machine data. */ interface EntryWithData { + /** @return the index of this entry. */ + default long getIndex() { + try { + return getEntry(TimeDuration.ONE_MINUTE).getIndex(); + } catch (Exception e) { + throw new IllegalStateException("Failed to getIndex", e); + } + } + /** @return the serialized size including both log entry and state machine data. */ int getSerializedSize(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java index 958cc6fa81..de221432bc 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java @@ -33,11 +33,14 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -218,16 +221,35 @@ protected LongUnaryOperator getNextIndexForError(long newNextIndex) { }; } - @Override - public AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean heartbeat) + public AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean heartbeat) { + throw new UnsupportedOperationException("Use nextAppendEntriesRequest(" + callId + ", " + heartbeat +") instead."); + } + +/** + * Create a {@link AppendEntriesRequestProto} object using the {@link FollowerInfo} of this {@link LogAppender}. + * The {@link AppendEntriesRequestProto} object may contain zero or more log entries. + * When there is zero log entries, the {@link AppendEntriesRequestProto} object is a heartbeat. + * + * @param callId The call id of the returned request. + * @param heartbeat the returned request must be a heartbeat. + * + * @return a retained reference of {@link AppendEntriesRequestProto} object. + * Since the returned reference is retained, the caller must call {@link ReferenceCountedObject#release()}} + * after use. + */ + protected ReferenceCountedObject nextAppendEntriesRequest(long callId, boolean heartbeat) throws RaftLogIOException { final long heartbeatWaitTimeMs = getHeartbeatWaitTimeMs(); final TermIndex previous = getPrevious(follower.getNextIndex()); if (heartbeatWaitTimeMs <= 0L || heartbeat) { // heartbeat - return leaderState.newAppendEntriesRequestProto(follower, Collections.emptyList(), - hasPendingDataRequests()? null : previous, callId); + AppendEntriesRequestProto heartbeatRequest = + leaderState.newAppendEntriesRequestProto(follower, Collections.emptyList(), + hasPendingDataRequests() ? null : previous, callId); + ReferenceCountedObject ref = ReferenceCountedObject.wrap(heartbeatRequest); + ref.retain(); + return ref; } Preconditions.assertTrue(buffer.isEmpty(), () -> "buffer has " + buffer.getNumElements() + " elements."); @@ -236,10 +258,14 @@ public AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean he final long leaderNext = getRaftLog().getNextIndex(); final long followerNext = follower.getNextIndex(); final long halfMs = heartbeatWaitTimeMs/2; - for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; ) { - if (!buffer.offer(getRaftLog().getEntryWithData(next++))) { + final Map> offered = new HashMap<>(); + for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; next++) { + final ReferenceCountedObject entryWithData = getRaftLog().retainEntryWithData(next); + if (!buffer.offer(entryWithData.get())) { + entryWithData.release(); break; } + offered.put(next, entryWithData); } if (buffer.isEmpty()) { return null; @@ -248,9 +274,15 @@ public AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean he final List protos = buffer.pollList(getHeartbeatWaitTimeMs(), EntryWithData::getEntry, (entry, time, exception) -> LOG.warn("Failed to get " + entry + " in " + time.toString(TimeUnit.MILLISECONDS, 3), exception)); + for (EntryWithData entry : buffer) { + // Release remaining entries. + offered.remove(entry.getIndex()).release(); + } buffer.clear(); assertProtos(protos, followerNext, previous, snapshotIndex); - return leaderState.newAppendEntriesRequestProto(follower, protos, previous, callId); + AppendEntriesRequestProto appendEntriesProto = + leaderState.newAppendEntriesRequestProto(follower, protos, previous, callId); + return ReferenceCountedObject.delegateFrom(offered.values(), appendEntriesProto); } private void assertProtos(List protos, long nextIndex, TermIndex previous, long snapshotIndex) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java index 21ef70d4df..432a41992d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java @@ -26,6 +26,7 @@ import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.statemachine.SnapshotInfo; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.Timestamp; import java.io.IOException; @@ -58,11 +59,15 @@ private AppendEntriesReplyProto sendAppendEntriesWithRetries() throws InterruptedException, InterruptedIOException, RaftLogIOException { int retry = 0; - AppendEntriesRequestProto request = newAppendEntriesRequest(CallId.getAndIncrement(), false); + ReferenceCountedObject request = nextAppendEntriesRequest( + CallId.getAndIncrement(), false); while (isRunning()) { // keep retrying for IOException try { - if (request == null || request.getEntriesCount() == 0) { - request = newAppendEntriesRequest(CallId.getAndIncrement(), false); + if (request == null || request.get().getEntriesCount() == 0) { + if (request != null) { + request.release(); + } + request = nextAppendEntriesRequest(CallId.getAndIncrement(), false); } if (request == null) { @@ -73,14 +78,8 @@ private AppendEntriesReplyProto sendAppendEntriesWithRetries() return null; } - resetHeartbeatTrigger(); - final Timestamp sendTime = Timestamp.currentTime(); - getFollower().updateLastRpcSendTime(request.getEntriesCount() == 0); - final AppendEntriesReplyProto r = getServerRpc().appendEntries(request); - getFollower().updateLastRpcResponseTime(); - getFollower().updateLastRespondedAppendEntriesSendTime(sendTime); - - getLeaderState().onFollowerCommitIndex(getFollower(), r.getFollowerCommit()); + AppendEntriesReplyProto r = sendAppendEntries(request.get()); + request.release(); return r; } catch (InterruptedIOException | RaftLogIOException e) { throw e; @@ -98,6 +97,18 @@ private AppendEntriesReplyProto sendAppendEntriesWithRetries() return null; } + private AppendEntriesReplyProto sendAppendEntries(AppendEntriesRequestProto request) throws IOException { + resetHeartbeatTrigger(); + final Timestamp sendTime = Timestamp.currentTime(); + getFollower().updateLastRpcSendTime(request.getEntriesCount() == 0); + final AppendEntriesReplyProto r = getServerRpc().appendEntries(request); + getFollower().updateLastRpcResponseTime(); + getFollower().updateLastRespondedAppendEntriesSendTime(sendTime); + + getLeaderState().onFollowerCommitIndex(getFollower(), r.getFollowerCommit()); + return r; + } + private InstallSnapshotReplyProto installSnapshot(SnapshotInfo snapshot) throws InterruptedIOException { String requestId = UUID.randomUUID().toString(); InstallSnapshotReplyProto reply = null; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 0a9a1c93ca..284776d100 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -428,11 +428,16 @@ class EntryWithDataImpl implements EntryWithData { private ByteString checkStateMachineData(ByteString data) { if (data == null) { - throw new IllegalStateException("State machine data is null for log entry " + logEntry); + throw new IllegalStateException("State machine data is null for log entry " + this); } return data; } + @Override + public long getIndex() { + return logEntry.getIndex(); + } + @Override public int getSerializedSize() { return LogProtoUtils.getSerializedSize(logEntry); @@ -440,11 +445,11 @@ public int getSerializedSize() { @Override public LogEntryProto getEntry(TimeDuration timeout) throws RaftLogIOException, TimeoutException { - LogEntryProto entryProto; if (future == null) { return logEntry; } + final LogEntryProto entryProto; try { entryProto = future.thenApply(data -> LogProtoUtils.addStateMachineData(data, logEntry)) .get(timeout.getDuration(), timeout.getUnit()); @@ -457,14 +462,14 @@ public LogEntryProto getEntry(TimeDuration timeout) throws RaftLogIOException, T if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } - final String err = getName() + ": Failed readStateMachineData for " + toLogEntryString(logEntry); + final String err = getName() + ": Failed readStateMachineData for " + this; LOG.error(err, e); throw new RaftLogIOException(err, JavaUtils.unwrapCompletionException(e)); } // by this time we have already read the state machine data, // so the log entry data should be set now if (LogProtoUtils.isStateMachineDataEmpty(entryProto)) { - final String err = getName() + ": State machine data not set for " + toLogEntryString(logEntry); + final String err = getName() + ": State machine data not set for " + this; LOG.error(err); throw new RaftLogIOException(err); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index feedaeee4a..55036fac56 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -135,14 +135,14 @@ public ReferenceCountedObject retainLog(long index) { } @Override - public EntryWithData getEntryWithData(long index) { - // TODO. The reference counted object should be passed to LogAppender RATIS-2026. - ReferenceCountedObject ref = retainLog(index); - try { - return newEntryWithData(ref.get(), null); - } finally { - ref.release(); - } + public EntryWithData getEntryWithData(long index) throws RaftLogIOException { + throw new UnsupportedOperationException("Use retainEntryWithData(" + index + ") instead."); + } + + @Override + public ReferenceCountedObject retainEntryWithData(long index) { + final ReferenceCountedObject ref = retainLog(index); + return ref.delegate(newEntryWithData(ref.get(), null)); } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index bb0793abe3..b7dd32689b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -319,21 +319,19 @@ record = segment.getLogRecord(index); @Override public EntryWithData getEntryWithData(long index) throws RaftLogIOException { + throw new UnsupportedOperationException("Use retainEntryWithData(" + index + ") instead."); + } + + @Override + public ReferenceCountedObject retainEntryWithData(long index) throws RaftLogIOException { final ReferenceCountedObject entryRef = retainLog(index); if (entryRef == null) { throw new RaftLogIOException("Log entry not found: index = " + index); } - try { - // TODO. The reference counted object should be passed to LogAppender RATIS-2026. - return getEntryWithData(entryRef.get()); - } finally { - entryRef.release(); - } - } - private EntryWithData getEntryWithData(LogEntryProto entry) throws RaftLogIOException { + final LogEntryProto entry = entryRef.get(); if (!LogProtoUtils.isStateMachineDataEmpty(entry)) { - return newEntryWithData(entry, null); + return entryRef.delegate(newEntryWithData(entry, null)); } try { @@ -344,7 +342,7 @@ private EntryWithData getEntryWithData(LogEntryProto entry) throws RaftLogIOExce throw new CompletionException("Failed to read state machine data for log entry " + entry, ex); }); } - return newEntryWithData(entry, future); + return entryRef.delegate(newEntryWithData(entry, future)); } catch (Exception e) { final String err = getName() + ": Failed readStateMachineData for " + LogProtoUtils.toLogEntryString(entry); From a99e709fee3b922338936c6173e307ef221ef165 Mon Sep 17 00:00:00 2001 From: Potato Date: Sat, 30 Mar 2024 02:16:24 +0800 Subject: [PATCH 52/76] RATIS-2052. Add docs for triggering snapshot when removing raftGroup parameter. (#1061) --- ratis-docs/src/site/markdown/configurations.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/ratis-docs/src/site/markdown/configurations.md b/ratis-docs/src/site/markdown/configurations.md index 7c8fb001fb..95505fd549 100644 --- a/ratis-docs/src/site/markdown/configurations.md +++ b/ratis-docs/src/site/markdown/configurations.md @@ -494,6 +494,11 @@ further wait for 5sec for max times ((5sec*980)/2 times ~= 40min) | **Type** | boolean | | **Default** | true | +| **Property** | `raft.server.snapshot.trigger-when-remove.enabled` | +|:----------------|:--------------------------------------------------------| +| **Description** | whether to trigger snapshot when raft server is removed | +| **Type** | boolean | +| **Default** | true | | **Property** | `raft.server.snapshot.creation.gap` | |:----------------|:-----------------------------------------------------| From c73a3eb8c026133e0e9b8ada1a2aeb467812cb37 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Tue, 2 Apr 2024 07:09:24 -0700 Subject: [PATCH 53/76] RATIS-1979. Allow StateMachine.read to return a ReferentCountedObject (#1062) --- .../ratis/examples/filestore/FileInfo.java | 3 +- .../ratis/statemachine/StateMachine.java | 22 +++++++ .../ratis/server/raftlog/LogProtoUtils.java | 10 ++- .../ratis/server/raftlog/RaftLogBase.java | 62 ++++++++++++++++--- .../server/raftlog/memory/MemoryRaftLog.java | 2 +- .../server/raftlog/segmented/LogSegment.java | 2 +- .../raftlog/segmented/SegmentedRaftLog.java | 10 +-- 7 files changed, 95 insertions(+), 16 deletions(-) diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java index c7d8cb7cd1..bba001002a 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java @@ -19,6 +19,7 @@ import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LogUtils; @@ -84,7 +85,7 @@ ByteString read(CheckedFunction resolver, long offset, final ByteBuffer buffer = ByteBuffer.allocateDirect(FileStoreCommon.getChunkSize(length)); in.position(offset).read(buffer); buffer.flip(); - return ByteString.copyFrom(buffer); + return UnsafeByteOperations.unsafeWrap(buffer); } } diff --git a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java index 915b70bb81..b68c724ab6 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java @@ -88,6 +88,28 @@ default CompletableFuture read(LogEntryProto entry, TransactionConte return read(entry); } + /** + * Read asynchronously the state machine data from this state machine. + * StateMachines implement this method when the read result contains retained resources that should be released + * after use. + * + * @return a future for the read task. The result of the future is a {@link ReferenceCountedObject} wrapping the + * read result. Client code of this method must call {@link ReferenceCountedObject#release()} after + * use. + */ + default CompletableFuture> retainRead(LogEntryProto entry, + TransactionContext context) { + return read(entry, context).thenApply(r -> { + if (r == null) { + return null; + } + ReferenceCountedObject ref = ReferenceCountedObject.wrap(r); + ref.retain(); + return ref; + + }); + } + /** * Write asynchronously the state machine data in the given log entry to this state machine. * diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java index b177f0e14c..59da4c3fd6 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java @@ -137,8 +137,9 @@ public static LogEntryProto removeStateMachineData(LogEntryProto entry) { } private static LogEntryProto replaceStateMachineDataWithSerializedSize(LogEntryProto entry) { - return replaceStateMachineEntry(entry, + LogEntryProto replaced = replaceStateMachineEntry(entry, StateMachineEntryProto.newBuilder().setLogEntryProtoSerializedSize(entry.getSerializedSize())); + return copy(replaced); } private static LogEntryProto replaceStateMachineEntry(LogEntryProto proto, StateMachineEntryProto.Builder newEntry) { @@ -160,6 +161,13 @@ static LogEntryProto addStateMachineData(ByteString stateMachineData, LogEntryPr return replaceStateMachineEntry(entry, StateMachineEntryProto.newBuilder().setStateMachineData(stateMachineData)); } + public static boolean hasStateMachineData(LogEntryProto entry) { + return getStateMachineEntry(entry) + .map(StateMachineEntryProto::getStateMachineData) + .map(data -> !data.isEmpty()) + .orElse(false); + } + public static boolean isStateMachineDataEmpty(LogEntryProto entry) { return getStateMachineEntry(entry) .map(StateMachineEntryProto::getStateMachineData) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 284776d100..9e079564f1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -17,6 +17,7 @@ */ package org.apache.ratis.server.raftlog; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.proto.RaftProtos.LogEntryProto; @@ -410,8 +411,43 @@ public String getName() { return name; } - protected EntryWithData newEntryWithData(LogEntryProto logEntry, CompletableFuture future) { - return new EntryWithDataImpl(logEntry, future); + protected ReferenceCountedObject newEntryWithData(ReferenceCountedObject retained) { + return retained.delegate(new EntryWithDataImpl(retained.get(), null)); + } + + protected ReferenceCountedObject newEntryWithData(ReferenceCountedObject retained, + CompletableFuture> stateMachineDataFuture) { + final EntryWithDataImpl impl = new EntryWithDataImpl(retained.get(), stateMachineDataFuture); + return new ReferenceCountedObject() { + private CompletableFuture> future + = Objects.requireNonNull(stateMachineDataFuture, "stateMachineDataFuture == null"); + + @Override + public EntryWithData get() { + return impl; + } + + synchronized void updateFuture(Consumer> action) { + future = future.whenComplete((ref, e) -> { + if (ref != null) { + action.accept(ref); + } + }); + } + + @Override + public EntryWithData retain() { + retained.retain(); + updateFuture(ReferenceCountedObject::retain); + return impl; + } + + @Override + public boolean release() { + updateFuture(ReferenceCountedObject::release); + return retained.release(); + } + }; } /** @@ -419,14 +455,14 @@ protected EntryWithData newEntryWithData(LogEntryProto logEntry, CompletableFutu */ class EntryWithDataImpl implements EntryWithData { private final LogEntryProto logEntry; - private final CompletableFuture future; + private final CompletableFuture> future; - EntryWithDataImpl(LogEntryProto logEntry, CompletableFuture future) { + EntryWithDataImpl(LogEntryProto logEntry, CompletableFuture> future) { this.logEntry = logEntry; this.future = future == null? null: future.thenApply(this::checkStateMachineData); } - private ByteString checkStateMachineData(ByteString data) { + private ReferenceCountedObject checkStateMachineData(ReferenceCountedObject data) { if (data == null) { throw new IllegalStateException("State machine data is null for log entry " + this); } @@ -450,18 +486,21 @@ public LogEntryProto getEntry(TimeDuration timeout) throws RaftLogIOException, T } final LogEntryProto entryProto; + ReferenceCountedObject data; try { - entryProto = future.thenApply(data -> LogProtoUtils.addStateMachineData(data, logEntry)) - .get(timeout.getDuration(), timeout.getUnit()); + data = future.get(timeout.getDuration(), timeout.getUnit()); + entryProto = LogProtoUtils.addStateMachineData(data.get(), logEntry); } catch (TimeoutException t) { if (timeout.compareTo(stateMachineDataReadTimeout) > 0) { getRaftLogMetrics().onStateMachineDataReadTimeout(); } + discardData(); throw t; } catch (Exception e) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } + discardData(); final String err = getName() + ": Failed readStateMachineData for " + this; LOG.error(err, e); throw new RaftLogIOException(err, JavaUtils.unwrapCompletionException(e)); @@ -471,11 +510,20 @@ public LogEntryProto getEntry(TimeDuration timeout) throws RaftLogIOException, T if (LogProtoUtils.isStateMachineDataEmpty(entryProto)) { final String err = getName() + ": State machine data not set for " + this; LOG.error(err); + data.release(); throw new RaftLogIOException(err); } return entryProto; } + private void discardData() { + future.whenComplete((r, ex) -> { + if (r != null) { + r.release(); + } + }); + } + @Override public String toString() { return toLogEntryString(logEntry); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index 55036fac56..2aac6c1b1f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -142,7 +142,7 @@ public EntryWithData getEntryWithData(long index) throws RaftLogIOException { @Override public ReferenceCountedObject retainEntryWithData(long index) { final ReferenceCountedObject ref = retainLog(index); - return ref.delegate(newEntryWithData(ref.get(), null)); + return newEntryWithData(ref); } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 2fcd7914e5..2542e99987 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -73,7 +73,7 @@ static long getEntrySize(LogEntryProto entry, Op op) { case CHECK_SEGMENT_FILE_FULL: case LOAD_SEGMENT_FILE: case WRITE_CACHE_WITH_STATE_MACHINE_CACHE: - Preconditions.assertTrue(entry == LogProtoUtils.removeStateMachineData(entry), + Preconditions.assertTrue(!LogProtoUtils.hasStateMachineData(entry), () -> "Unexpected LogEntryProto with StateMachine data: op=" + op + ", entry=" + entry); break; case WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE: diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index b7dd32689b..a0f7c1e723 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -331,18 +331,18 @@ public ReferenceCountedObject retainEntryWithData(long index) thr final LogEntryProto entry = entryRef.get(); if (!LogProtoUtils.isStateMachineDataEmpty(entry)) { - return entryRef.delegate(newEntryWithData(entry, null)); + return newEntryWithData(entryRef); } try { - CompletableFuture future = null; + CompletableFuture> future = null; if (stateMachine != null) { - future = stateMachine.data().read(entry, server.getTransactionContext(entry, false)).exceptionally(ex -> { + future = stateMachine.data().retainRead(entry, server.getTransactionContext(entry, false)).exceptionally(ex -> { stateMachine.event().notifyLogFailed(ex, entry); throw new CompletionException("Failed to read state machine data for log entry " + entry, ex); }); } - return entryRef.delegate(newEntryWithData(entry, future)); + return future != null? newEntryWithData(entryRef, future): newEntryWithData(entryRef); } catch (Exception e) { final String err = getName() + ": Failed readStateMachineData for " + LogProtoUtils.toLogEntryString(entry); @@ -459,7 +459,7 @@ protected CompletableFuture appendEntryImpl(ReferenceCountedObject Date: Thu, 4 Apr 2024 00:16:16 +0800 Subject: [PATCH 54/76] RATIS-2051. Fix Sonar bugs for Ratis. (#1059) --- .../src/main/java/org/apache/ratis/util/LeakDetector.java | 1 + .../src/main/java/org/apache/ratis/util/PeerProxyMap.java | 1 + .../src/main/java/org/apache/ratis/util/Preconditions.java | 5 +++++ .../java/org/apache/ratis/grpc/server/GrpcLogAppender.java | 1 + .../org/apache/ratis/server/impl/ConfigurationManager.java | 2 ++ .../java/org/apache/ratis/server/impl/FollowerState.java | 1 + .../main/java/org/apache/ratis/server/impl/LeaderLease.java | 2 +- .../java/org/apache/ratis/server/impl/LeaderStateImpl.java | 3 ++- .../org/apache/ratis/server/impl/RaftConfigurationImpl.java | 1 + .../main/java/org/apache/ratis/server/impl/ServerState.java | 2 ++ .../org/apache/ratis/server/impl/StateMachineUpdater.java | 1 + .../apache/ratis/server/metrics/LeaderElectionMetrics.java | 1 + .../apache/ratis/server/raftlog/segmented/LogSegment.java | 4 ++-- .../ratis/server/raftlog/segmented/SegmentedRaftLog.java | 1 + .../server/raftlog/segmented/SegmentedRaftLogCache.java | 1 + .../server/raftlog/segmented/SegmentedRaftLogWorker.java | 1 + .../ratis/server/storage/RaftStorageDirectoryImpl.java | 1 + .../org/apache/ratis/server/storage/SnapshotManager.java | 1 + .../org/apache/ratis/statemachine/impl/BaseStateMachine.java | 1 + .../ratis/statemachine/impl/TransactionContextImpl.java | 5 +++++ 20 files changed, 32 insertions(+), 4 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java index d801868163..82202f2884 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java @@ -84,6 +84,7 @@ private void run() { tracker.reportLeak(); } } catch (InterruptedException e) { + Thread.currentThread().interrupt(); LOG.warn("Thread interrupted, exiting.", e); break; } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java index 0ce0595fa9..eda41a0096 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java @@ -45,6 +45,7 @@ public class PeerProxyMap implements RaftPeer.Add, Clos /** Peer and its proxy. */ private class PeerAndProxy { private final RaftPeer peer; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile PROXY proxy = null; private final LifeCycle lifeCycle; diff --git a/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java b/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java index c757de2990..f37b250f53 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java @@ -78,6 +78,11 @@ static void assertTrue(boolean value, Supplier message) { } } + static void assertSame(int expected, int computed, String name) { + assertTrue(expected == computed, + () -> name + ": expected == " + expected + " but computed == " + computed); + } + static void assertSame(long expected, long computed, String name) { assertTrue(expected == computed, () -> name + ": expected == " + expected + " but computed == " + computed); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index e23f2826e3..351e7f1091 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -156,6 +156,7 @@ synchronized int process(Event event) { private final TimeDuration logMessageBatchDuration; private final int maxOutstandingInstallSnapshots; private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile StreamObservers appendLogRequestObserver; private final boolean useSeparateHBChannel; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java index 0e020b7e36..10c59c8b19 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java @@ -41,8 +41,10 @@ public class ConfigurationManager { * The current raft configuration. If configurations is not empty, should be * the last entry of the map. Otherwise is initialConf. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftConfigurationImpl currentConf; /** Cache the peer corresponding to {@link #id}. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftPeer currentPeer; ConfigurationManager(RaftPeerId id, RaftConfigurationImpl initialConf) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java index 3911e39a5c..e980daede5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java @@ -59,6 +59,7 @@ int update(AtomicInteger outstanding) { private final RaftServerImpl server; private final Timestamp creationTime = Timestamp.currentTime(); + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Timestamp lastRpcTime = creationTime; private volatile boolean isRunning = true; private final AtomicInteger outstandingOp = new AtomicInteger(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java index 315cc9f143..3b8f53da14 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java @@ -92,7 +92,7 @@ private Timestamp getMaxTimestampWithMajorityAck(List followers) { return Timestamp.currentTime(); } - final int mid = followers.size() / 2; + final long mid = followers.size() / 2; return followers.stream() .map(FollowerInfo::getLastRespondedAppendEntriesSendTime) .sorted() diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index e8a4adc9ca..28f2350c75 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -269,7 +269,7 @@ static boolean isSameConf(CurrentOldFollowerInfos cached, RaftConfigurationImpl static class FollowerInfoMap { private final Map map = new ConcurrentHashMap<>(); - + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile CurrentOldFollowerInfos followerInfos; void put(RaftPeerId id, FollowerInfo info) { @@ -333,6 +333,7 @@ boolean isApplied() { private final RaftServerImpl server; private final RaftLog raftLog; private final long currentTerm; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile ConfigurationStagingState stagingState; private final FollowerInfoMap followerInfoMap = new FollowerInfoMap(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java index 98f41020e0..15a114cdb4 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java @@ -148,6 +148,7 @@ boolean isStable() { return oldConf == null; } + @SuppressWarnings({"squid:S6466"}) // Suppress ArrayIndexOutOfBoundsException warning boolean containsInConf(RaftPeerId peerId, RaftPeerRole... roles) { if (roles == null || roles.length == 0) { return conf.contains(peerId); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index af241a6331..0f46c6b523 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -87,6 +87,7 @@ class ServerState { /** * Candidate that this peer granted vote for in current term (or null if none) */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftPeerId votedFor; /** @@ -171,6 +172,7 @@ private RaftLog initRaftLog(LongSupplier getSnapshotIndexFromStateMachine, RaftP } } + @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning private static RaftLog initRaftLog(RaftGroupMemberId memberId, RaftServerImpl server, RaftStorage storage, Consumer logConsumer, LongSupplier getSnapshotIndexFromStateMachine, RaftProperties prop) throws IOException { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index fbd7f013fa..f13ee0d6d2 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -199,6 +199,7 @@ public void run() { } } catch (Throwable t) { if (t instanceof InterruptedException && state == State.STOP) { + Thread.currentThread().interrupt(); LOG.info("{} was interrupted. Exiting ...", this); } else { state = State.EXCEPTION; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/metrics/LeaderElectionMetrics.java b/ratis-server/src/main/java/org/apache/ratis/server/metrics/LeaderElectionMetrics.java index 7447498d42..1d044bb6fe 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/metrics/LeaderElectionMetrics.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/metrics/LeaderElectionMetrics.java @@ -51,6 +51,7 @@ public final class LeaderElectionMetrics extends RatisMetrics { private final Timekeeper electionTime = getRegistry().timer(LEADER_ELECTION_TIME_TAKEN); + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Timestamp lastElectionTime; private LeaderElectionMetrics(RaftGroupMemberId serverId, LongSupplier getLastLeaderElapsedTimeMs) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 2542e99987..f96e34e4cb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -294,10 +294,10 @@ File getFile() { private volatile boolean isOpen; private long totalFileSize = SegmentedRaftLogFormat.getHeaderLength(); /** Segment start index, inclusive. */ - private long startIndex; + private final long startIndex; /** Segment end index, inclusive. */ private volatile long endIndex; - private RaftStorage storage; + private final RaftStorage storage; private final SizeInBytes maxOpSize; private final LogEntryLoader cacheLoader; /** later replace it with a metric */ diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index a0f7c1e723..8d972a1e2c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -211,6 +211,7 @@ public TransactionContext getTransactionContext(LogEntryProto entry, boolean cre private final boolean stateMachineCachingEnabled; private final SegmentedRaftLogMetrics metrics; + @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning private SegmentedRaftLog(Builder b) { super(b.memberId, b.snapshotIndexSupplier, b.properties); this.metrics = new SegmentedRaftLogMetrics(b.memberId); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index d222482012..ad16332326 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -396,6 +396,7 @@ public String toString() { } private final String name; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile LogSegment openSegment; private final LogSegmentList closedSegments; private final RaftStorage storage; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index 02506079f8..7dc87faf75 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -143,6 +143,7 @@ synchronized void updateIndex(long i) { private volatile boolean running = true; private final ExecutorService workerThreadExecutor; private final RaftStorage storage; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile SegmentedRaftLogOutputStream out; private final Runnable submitUpdateCommitEvent; private final StateMachine stateMachine; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectoryImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectoryImpl.java index e7f69d1e24..119f7922db 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectoryImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectoryImpl.java @@ -210,6 +210,7 @@ void lock() throws IOException { * null if storage is already locked. * @throws IOException if locking fails. */ + @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning private FileLock tryLock(File lockF) throws IOException { boolean deletionHookAdded = false; if (!lockF.exists()) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java index c49a86ec59..794604d66b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java @@ -77,6 +77,7 @@ public class SnapshotManager { new File(dir.get().getRoot(), c.getFilename()).toPath()).toString(); } + @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning private FileChannel open(FileChunkProto chunk, File tmpSnapshotFile) throws IOException { final FileChannel out; final boolean exists = tmpSnapshotFile.exists(); diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java index 98f270d2d9..97cc3e6387 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java @@ -50,6 +50,7 @@ public class BaseStateMachine implements StateMachine, StateMachine.DataApi, StateMachine.EventApi, StateMachine.LeaderEventApi, StateMachine.FollowerEventApi { private final CompletableFuture server = new CompletableFuture<>(); + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftGroupId groupId; private final LifeCycle lifeCycle = new LifeCycle(JavaUtils.getClassSimpleName(getClass())); diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java index 44bd32c674..e582fb9e12 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java @@ -49,6 +49,7 @@ public class TransactionContextImpl implements TransactionContext { private final RaftClientRequest clientRequest; /** Exception from the {@link StateMachine} or from the log */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Exception exception; /** Data from the {@link StateMachine} */ @@ -60,6 +61,7 @@ public class TransactionContextImpl implements TransactionContext { * {@link StateMachine#startTransaction(RaftClientRequest)} and * {@link StateMachine#applyTransaction(TransactionContext)}. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Object stateMachineContext; /** @@ -70,11 +72,14 @@ public class TransactionContextImpl implements TransactionContext { private boolean shouldCommit = true; /** Committed LogEntry. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile LogEntryProto logEntry; /** Committed LogEntry copy. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Supplier logEntryCopy; /** For wrapping {@link #logEntry} in order to release the underlying buffer. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile ReferenceCountedObject delegatedRef; private final CompletableFuture logIndexFuture = new CompletableFuture<>(); From bc6221b32fff9022cebb02bb243aea9fff35e290 Mon Sep 17 00:00:00 2001 From: hao guo Date: Tue, 9 Apr 2024 02:10:22 +0800 Subject: [PATCH 55/76] RATIS-1519. When DataStreamManagement#read an exception occurs, remove DataStream (#596) --- .../netty/server/DataStreamManagement.java | 42 +++++++++++-------- .../ratis/datastream/DataStreamTestUtils.java | 7 +++- .../TestNettyDataStreamWithMock.java | 10 ++++- 3 files changed, 38 insertions(+), 21 deletions(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java index a6e9b815ee..e265d8b924 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java @@ -140,19 +140,19 @@ static class StreamInfo { private final boolean primary; private final LocalStream local; private final Set remotes; - private final RaftServer server; + private final Division division; private final AtomicReference> previous = new AtomicReference<>(CompletableFuture.completedFuture(null)); - StreamInfo(RaftClientRequest request, boolean primary, CompletableFuture stream, RaftServer server, + StreamInfo(RaftClientRequest request, boolean primary, CompletableFuture stream, Division division, CheckedBiFunction, Set, IOException> getStreams, Function metricsConstructor) throws IOException { this.request = request; this.primary = primary; this.local = new LocalStream(stream, metricsConstructor.apply(RequestType.LOCAL_WRITE)); - this.server = server; - final Set successors = getSuccessors(server.getId()); + this.division = division; + final Set successors = getSuccessors(division.getId()); final Set outs = getStreams.apply(request, successors); this.remotes = outs.stream() .map(o -> new RemoteStream(o, metricsConstructor.apply(RequestType.REMOTE_WRITE))) @@ -167,16 +167,12 @@ RaftClientRequest getRequest() { return request; } - Division getDivision() throws IOException { - return server.getDivision(request.getRaftGroupId()); + Division getDivision() { + return division; } Collection getCommitInfos() { - try { - return getDivision().getCommitInfos(); - } catch (IOException e) { - throw new IllegalStateException(e); - } + return getDivision().getCommitInfos(); } boolean isPrimary() { @@ -196,7 +192,7 @@ public String toString() { return JavaUtils.getClassSimpleName(getClass()) + ":" + request; } - private Set getSuccessors(RaftPeerId peerId) throws IOException { + private Set getSuccessors(RaftPeerId peerId) { final RaftConfiguration conf = getDivision().getRaftConf(); final RoutingTable routingTable = request.getRoutingTable(); @@ -208,7 +204,7 @@ private Set getSuccessors(RaftPeerId peerId) throws IOException { // Default start topology // get the other peers from the current configuration return conf.getCurrentPeers().stream() - .filter(p -> !p.getId().equals(server.getId())) + .filter(p -> !p.getId().equals(division.getId())) .collect(Collectors.toSet()); } @@ -276,7 +272,8 @@ private StreamInfo newStreamInfo(ByteBuf buf, final RaftClientRequest request = ClientProtoUtils.toRaftClientRequest( RaftClientRequestProto.parseFrom(buf.nioBuffer())); final boolean isPrimary = server.getId().equals(request.getServerId()); - return new StreamInfo(request, isPrimary, computeDataStreamIfAbsent(request), server, getStreams, + final Division division = server.getDivision(request.getRaftGroupId()); + return new StreamInfo(request, isPrimary, computeDataStreamIfAbsent(request), division, getStreams, getMetrics()::newRequestMetrics); } catch (Throwable e) { throw new CompletionException(e); @@ -411,6 +408,18 @@ void read(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, readImpl(request, ctx, getStreams); } catch (Throwable t) { replyDataStreamException(t, request, ctx); + removeDataStream(ClientInvocationId.valueOf(request.getClientId(), request.getStreamId()), null); + } + } + + private void removeDataStream(ClientInvocationId invocationId, StreamInfo info) { + final StreamInfo removed = streams.remove(invocationId); + if (info == null) { + info = removed; + } + if (info != null) { + info.getDivision().getDataStreamMap().remove(invocationId); + info.getLocal().cleanUp(); } } @@ -429,8 +438,6 @@ private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ct () -> newStreamInfo(request.slice(), getStreams)); info = streams.computeIfAbsent(key, id -> supplier.get()); if (!supplier.isInitialized()) { - final StreamInfo removed = streams.remove(key); - removed.getLocal().cleanUp(); throw new IllegalStateException("Failed to create a new stream for " + request + " since a stream already exists Key: " + key + " StreamInfo:" + info); } @@ -468,9 +475,8 @@ private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ct }, requestExecutor)).whenComplete((v, exception) -> { try { if (exception != null) { - final StreamInfo removed = streams.remove(key); replyDataStreamException(server, exception, info.getRequest(), request, ctx); - removed.getLocal().cleanUp(); + removeDataStream(key, info); } } finally { request.release(); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index 47138919df..7735c3e309 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -151,6 +151,7 @@ class MultiDataStreamStateMachine extends BaseStateMachine { @Override public CompletableFuture stream(RaftClientRequest request) { final SingleDataStream s = new SingleDataStream(request); + LOG.info("XXX {} put {}, {}", this, ClientInvocationId.valueOf(request), s); streams.put(ClientInvocationId.valueOf(request), s); return CompletableFuture.completedFuture(s); } @@ -179,7 +180,9 @@ SingleDataStream getSingleDataStream(RaftClientRequest request) { } SingleDataStream getSingleDataStream(ClientInvocationId invocationId) { - return streams.get(invocationId); + final SingleDataStream s = streams.get(invocationId); + LOG.info("XXX {}: get {} return {}", this, invocationId, s); + return s; } Collection getStreams() { @@ -329,6 +332,8 @@ static CompletableFuture writeAndCloseAndAssertReplies( static void assertHeader(RaftServer server, RaftClientRequest header, int dataSize, boolean stepDownLeader) throws Exception { + LOG.info("XXX {}: dataSize={}, stepDownLeader={}, header={}", + server.getId(), dataSize, stepDownLeader, header); // check header Assertions.assertEquals(RaftClientRequest.dataStreamRequestType(), header.getType()); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java index 503f8cf66e..1d8c67a43d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java @@ -59,12 +59,18 @@ public void setup() { RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.NETTY); } - RaftServer.Division mockDivision(RaftServer server) { + + RaftServer.Division mockDivision(RaftServer server, RaftGroupId groupId) { final RaftServer.Division division = mock(RaftServer.Division.class); when(division.getRaftServer()).thenReturn(server); when(division.getRaftConf()).thenAnswer(i -> getRaftConf()); final MultiDataStreamStateMachine stateMachine = new MultiDataStreamStateMachine(); + try { + stateMachine.initialize(server, groupId, null); + } catch (IOException e) { + throw new IllegalStateException(e); + } when(division.getStateMachine()).thenReturn(stateMachine); final DataStreamMap streamMap = RaftServerTestUtil.newDataStreamMap(server.getId()); @@ -95,7 +101,7 @@ private void testMockCluster(int numServers, RaftException leaderException, when(raftServer.getId()).thenReturn(peerId); when(raftServer.getPeer()).thenReturn(RaftPeer.newBuilder().setId(peerId).build()); if (getStateMachineException == null) { - final RaftServer.Division myDivision = mockDivision(raftServer); + final RaftServer.Division myDivision = mockDivision(raftServer, groupId); when(raftServer.getDivision(Mockito.any(RaftGroupId.class))).thenReturn(myDivision); } else { when(raftServer.getDivision(Mockito.any(RaftGroupId.class))).thenThrow(getStateMachineException); From 8638a2ffa802de576aae546c22160da3d9339a8f Mon Sep 17 00:00:00 2001 From: DaveTeng0 <109315747+DaveTeng0@users.noreply.github.com> Date: Wed, 10 Apr 2024 16:38:57 -0700 Subject: [PATCH 56/76] Ratis-2040. Fix RaftPeerId generated by command of "raftMetaConf" to use real PeerId (#1060) --- ratis-docs/src/site/markdown/cli.md | 2 +- .../cli/sh/local/RaftMetaConfCommand.java | 51 ++++++- .../cli/sh/LocalCommandIntegrationTest.java | 142 ++++++++++++++++++ 3 files changed, 190 insertions(+), 5 deletions(-) create mode 100644 ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java diff --git a/ratis-docs/src/site/markdown/cli.md b/ratis-docs/src/site/markdown/cli.md index 60958fc7ed..ab9f89982f 100644 --- a/ratis-docs/src/site/markdown/cli.md +++ b/ratis-docs/src/site/markdown/cli.md @@ -182,5 +182,5 @@ It has the following subcommands: ### local raftMetaConf Generate a new raft-meta.conf file based on original raft-meta.conf and new peers, which is used to move a raft node to a new node. ``` -$ ratis sh local raftMetaConf -peers -path +$ ratis sh local raftMetaConf -peers <[P0_ID|]P0_HOST:P0_PORT,[P1_ID|]P1_HOST:P1_PORT,[P2_ID|]P2_HOST:P2_PORT> -path ``` diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java index 231c643ac3..9f0558c5ea 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java @@ -24,6 +24,7 @@ import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto; import org.apache.ratis.proto.RaftProtos.RaftPeerProto; import org.apache.ratis.proto.RaftProtos.RaftPeerRole; +import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.shell.cli.RaftUtils; import org.apache.ratis.shell.cli.sh.command.AbstractCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -32,11 +33,14 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.net.InetSocketAddress; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; /** * Command for generate a new raft-meta.conf file based on original raft-meta.conf and new peers, @@ -49,6 +53,7 @@ public class RaftMetaConfCommand extends AbstractCommand { private static final String RAFT_META_CONF = "raft-meta.conf"; private static final String NEW_RAFT_META_CONF = "new-raft-meta.conf"; + private static final String SEPARATOR = "\\|"; /** * @param context command context */ @@ -69,11 +74,49 @@ public int run(CommandLine cl) throws IOException { printf("peers or path can't be empty."); return -1; } + Set addresses = new HashSet<>(); + Set ids = new HashSet<>(); List raftPeerProtos = new ArrayList<>(); - for (String address : peersStr.split(",")) { - String peerId = RaftUtils.getPeerId(parseInetSocketAddress(address)).toString(); + for (String idWithAddress : peersStr.split(",")) { + String[] peerIdWithAddressArray = idWithAddress.split(SEPARATOR); + + if (peerIdWithAddressArray.length < 1 || peerIdWithAddressArray.length > 2) { + String message = + "Failed to parse peer's ID and address for: %s, " + + "from option: -peers %s. \n" + + "Please make sure to provide list of peers" + + " in format <[P0_ID|]P0_HOST:P0_PORT,[P1_ID|]P1_HOST:P1_PORT,[P2_ID|]P2_HOST:P2_PORT>"; + printf(message, idWithAddress, peersStr); + return -1; + } + InetSocketAddress inetSocketAddress = parseInetSocketAddress( + peerIdWithAddressArray[peerIdWithAddressArray.length - 1]); + String addressString = inetSocketAddress.toString(); + if (addresses.contains(addressString)) { + printf("Found duplicated address: %s. Please make sure the address of peer have no duplicated value.", + addressString); + return -1; + } + addresses.add(addressString); + + String peerId; + if (peerIdWithAddressArray.length == 2) { + // Peer ID is provided + peerId = RaftPeerId.getRaftPeerId(peerIdWithAddressArray[0]).toString(); + + if (ids.contains(peerId)) { + printf("Found duplicated ID: %s. Please make sure the ID of peer have no duplicated value.", peerId); + return -1; + } + ids.add(peerId); + } else { + // If peer ID is not provided, use host address as peerId value + peerId = RaftUtils.getPeerId(inetSocketAddress).toString(); + } + raftPeerProtos.add(RaftPeerProto.newBuilder() - .setId(ByteString.copyFrom(peerId.getBytes(StandardCharsets.UTF_8))).setAddress(address) + .setId(ByteString.copyFrom(peerId.getBytes(StandardCharsets.UTF_8))) + .setAddress(addressString) .setStartupRole(RaftPeerRole.FOLLOWER).build()); } try (InputStream in = Files.newInputStream(Paths.get(path, RAFT_META_CONF)); @@ -93,7 +136,7 @@ public int run(CommandLine cl) throws IOException { @Override public String getUsage() { return String.format("%s" - + " -%s " + + " -%s <[P0_ID|]P0_HOST:P0_PORT,[P1_ID|]P1_HOST:P1_PORT,[P2_ID|]P2_HOST:P2_PORT>" + " -%s ", getCommandName(), PEER_OPTION_NAME, PATH_OPTION_NAME); } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java new file mode 100644 index 0000000000..4a07e372ca --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java @@ -0,0 +1,142 @@ +/* + * 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.ratis.shell.cli.sh; + +import org.apache.ratis.proto.RaftProtos.RaftPeerRole; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto; +import org.apache.ratis.proto.RaftProtos.RaftPeerProto; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +public class LocalCommandIntegrationTest { + + private static final String RAFT_META_CONF = "raft-meta.conf"; + private static final String NEW_RAFT_META_CONF = "new-raft-meta.conf"; + private static Pattern p = Pattern.compile("(?:\\w+\\|\\w+:\\d+,?)+"); + + + @Test + public void testDuplicatedPeerAddresses() throws Exception { + String[] duplicatedAddressesList = {"peer1_ID1|host1:9872,peer2_ID|host2:9872,peer1_ID2|host1:9872", + "host1:9872,host2:9872,host1:9872"}; + + testDuplicatedPeers(duplicatedAddressesList, "address", "host1:9872"); + } + + @Test + public void testDuplicatedPeerIds() throws Exception { + String[] duplicatedIdsList = {"peer1_ID1|host1:9872,peer2_ID|host2:9872,peer1_ID1|host3:9872"}; + + testDuplicatedPeers(duplicatedIdsList, "ID", "peer1_ID1"); + } + + private void testDuplicatedPeers(String[] peersList, String expectedErrorMessagePart, String expectedDuplicatedValue) throws Exception { + for (String peersStr : peersList) { + StringPrintStream out = new StringPrintStream(); + RatisShell shell = new RatisShell(out.getPrintStream()); + int ret = shell.run("local", "raftMetaConf", "-peers", peersStr, "-path", "test"); + Assertions.assertEquals(-1, ret); + String message = out.toString().trim(); + Assertions.assertEquals(String.format("Found duplicated %s: %s. Please make sure the %s of peer have no duplicated value.", + expectedErrorMessagePart, expectedDuplicatedValue, expectedErrorMessagePart), message); + } + } + + @Test + public void testRunMethod(@TempDir Path tempDir) throws Exception { + int index = 1; + generateRaftConf(tempDir.resolve(RAFT_META_CONF), index); + + String[] testPeersListArray = {"peer1_ID|host1:9872,peer2_ID|host2:9872,peer3_ID|host3:9872", + "host1:9872,host2:9872,host3:9872"}; + + for (String peersListStr : testPeersListArray) { + generateRaftConf(tempDir, index); + StringPrintStream out = new StringPrintStream(); + RatisShell shell = new RatisShell(out.getPrintStream()); + int ret = shell.run("local", "raftMetaConf", "-peers", peersListStr, "-path", tempDir.toString()); + Assertions.assertEquals(0, ret); + + // read & verify the contents of the new-raft-meta.conf file + long indexFromNewConf; + List peers; + try (InputStream in = Files.newInputStream(tempDir.resolve(NEW_RAFT_META_CONF))) { + LogEntryProto logEntry = LogEntryProto.newBuilder().mergeFrom(in).build(); + indexFromNewConf = logEntry.getIndex(); + peers = logEntry.getConfigurationEntry().getPeersList(); + } + + Assertions.assertEquals(index + 1, indexFromNewConf); + + String peersListStrFromNewMetaConf; + if (containsPeerId(peersListStr)) { + peersListStrFromNewMetaConf = peers.stream() + .map(peer -> peer.getId().toStringUtf8() + "|" + peer.getAddress()) + .collect(Collectors.joining(",")); + } else { + peersListStrFromNewMetaConf = peers.stream().map(RaftPeerProto::getAddress) + .collect(Collectors.joining(",")); + } + + Assertions.assertEquals(peersListStr, peersListStrFromNewMetaConf); + } + } + + + private void generateRaftConf(Path path, int index) throws IOException { + Map map = new HashMap<>(); + map.put("peer1_ID", "host1:9872"); + map.put("peer2_ID", "host2:9872"); + map.put("peer3_ID", "host3:9872"); + map.put("peer4_ID", "host4:9872"); + List raftPeerProtos = new ArrayList<>(); + for (Map.Entry en : map.entrySet()) { + raftPeerProtos.add(RaftPeerProto.newBuilder() + .setId(ByteString.copyFrom(en.getKey().getBytes(StandardCharsets.UTF_8))).setAddress(en.getValue()) + .setStartupRole(RaftPeerRole.FOLLOWER).build()); + } + + LogEntryProto generateLogEntryProto = LogEntryProto.newBuilder() + .setConfigurationEntry(RaftConfigurationProto.newBuilder().addAllPeers(raftPeerProtos).build()) + .setIndex(index).build(); + try (OutputStream out = Files.newOutputStream(path)) { + generateLogEntryProto.writeTo(out); + } + } + + private boolean containsPeerId(String str) { + return p.matcher(str).find(); + } + +} From 49b4006015699625c445d70e2113fd5904b4590d Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 12 Apr 2024 00:58:23 -0700 Subject: [PATCH 57/76] RATIS-1504. Add timeout handling to DataStreamManagement#checkSuccessRemoteWrite. (#1064) * RATIS-1504. Add timeout handling to DataStreamManagement#checkSuccessRemoteWrite. --- .../netty/client/NettyClientReplies.java | 29 ++++++++++--------- .../netty/client/NettyClientStreamRpc.java | 9 ++---- .../netty/server/DataStreamManagement.java | 14 +++++++-- 3 files changed, 30 insertions(+), 22 deletions(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java index fc97b6fe34..4c49b1d160 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java @@ -30,11 +30,10 @@ import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; public class NettyClientReplies { public static final Logger LOG = LoggerFactory.getLogger(NettyClientReplies.class); @@ -56,8 +55,8 @@ class ReplyMap { ReplyEntry submitRequest(RequestEntry requestEntry, boolean isClose, CompletableFuture f) { LOG.debug("put {} to the map for {}", requestEntry, clientInvocationId); - final MemoizedSupplier replySupplier = MemoizedSupplier.valueOf(() -> new ReplyEntry(isClose, f)); - return map.computeIfAbsent(requestEntry, r -> replySupplier.get()); + // ConcurrentHashMap.computeIfAbsent javadoc: the function is applied at most once per key. + return map.computeIfAbsent(requestEntry, r -> new ReplyEntry(isClose, f)); } void receiveReply(DataStreamReply reply) { @@ -147,7 +146,7 @@ public String toString() { static class ReplyEntry { private final boolean isClosed; private final CompletableFuture replyFuture; - private final AtomicReference> timeoutFuture = new AtomicReference<>(); + private ScheduledFuture timeoutFuture; // for reply timeout ReplyEntry(boolean isClosed, CompletableFuture replyFuture) { this.isClosed = isClosed; @@ -158,22 +157,26 @@ boolean isClosed() { return isClosed; } - void complete(DataStreamReply reply) { - cancelTimeoutFuture(); + synchronized void complete(DataStreamReply reply) { + cancel(timeoutFuture); replyFuture.complete(reply); } - void completeExceptionally(Throwable t) { - cancelTimeoutFuture(); + synchronized void completeExceptionally(Throwable t) { + cancel(timeoutFuture); replyFuture.completeExceptionally(t); } - private void cancelTimeoutFuture() { - Optional.ofNullable(timeoutFuture.get()).ifPresent(f -> f.cancel(false)); + static void cancel(ScheduledFuture future) { + if (future != null) { + future.cancel(true); + } } - void setTimeoutFuture(ScheduledFuture timeoutFuture) { - this.timeoutFuture.compareAndSet(null, timeoutFuture); + synchronized void scheduleTimeout(Supplier> scheduleMethod) { + if (!replyFuture.isDone()) { + timeoutFuture = scheduleMethod.get(); + } } } } diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java index b2dc3812f2..534fcc5818 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java @@ -53,7 +53,6 @@ import org.apache.ratis.thirdparty.io.netty.handler.codec.ByteToMessageDecoder; import org.apache.ratis.thirdparty.io.netty.handler.codec.MessageToMessageEncoder; import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; -import org.apache.ratis.thirdparty.io.netty.util.concurrent.ScheduledFuture; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.NetUtils; @@ -466,15 +465,13 @@ public CompletableFuture streamAsync(DataStreamRequest request) LOG.debug("{}: write after {}", this, request); final TimeDuration timeout = isClose ? closeTimeout : requestTimeout; - // if reply success cancel this future - final ScheduledFuture timeoutFuture = channel.eventLoop().schedule(() -> { + replyEntry.scheduleTimeout(() -> channel.eventLoop().schedule(() -> { if (!f.isDone()) { f.completeExceptionally(new TimeoutIOException( - "Timeout " + timeout + ": Failed to send " + request + " channel: " + channel)); + "Timeout " + timeout + ": Failed to send " + request + " via channel " + channel)); replyMap.fail(requestEntry); } - }, timeout.toLong(timeout.getUnit()), timeout.getUnit()); - replyEntry.setTimeoutFuture(timeoutFuture); + }, timeout.getDuration(), timeout.getUnit())); } }); return f; diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java index e265d8b924..74d5cd7fde 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java @@ -18,6 +18,7 @@ package org.apache.ratis.netty.server; +import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.client.impl.DataStreamClientImpl.DataStreamOutputImpl; import org.apache.ratis.conf.RaftProperties; @@ -219,6 +220,7 @@ private Set getSuccessors(RaftPeerId peerId) { private final ChannelMap channels; private final ExecutorService requestExecutor; private final ExecutorService writeExecutor; + private final TimeDuration requestTimeout; private final NettyServerStreamRpcMetrics nettyServerStreamRpcMetrics; @@ -235,6 +237,7 @@ private Set getSuccessors(RaftPeerId peerId) { this.writeExecutor = ConcurrentUtils.newThreadPoolWithMax(useCachedThreadPool, RaftServerConfigKeys.DataStream.asyncWriteThreadPoolSize(properties), name + "-write-"); + this.requestTimeout = RaftClientConfigKeys.DataStream.requestTimeout(server.getProperties()); this.nettyServerStreamRpcMetrics = metrics; } @@ -339,7 +342,7 @@ static DataStreamReplyByteBuffer newDataStreamReplyByteBuffer(DataStreamRequestB .build(); } - static void sendReply(List> remoteWrites, + private void sendReply(List> remoteWrites, DataStreamRequestByteBuf request, long bytesWritten, Collection commitInfos, ChannelHandlerContext ctx) { final boolean success = checkSuccessRemoteWrite(remoteWrites, bytesWritten, request); @@ -493,10 +496,15 @@ static void assertReplyCorrespondingToRequest( Preconditions.assertTrue(request.getStreamOffset() == reply.getStreamOffset()); } - static boolean checkSuccessRemoteWrite(List> replyFutures, long bytesWritten, + private boolean checkSuccessRemoteWrite(List> replyFutures, long bytesWritten, final DataStreamRequestByteBuf request) { for (CompletableFuture replyFuture : replyFutures) { - final DataStreamReply reply = replyFuture.join(); + final DataStreamReply reply; + try { + reply = replyFuture.get(requestTimeout.getDuration(), requestTimeout.getUnit()); + } catch (Exception e) { + throw new CompletionException("Failed to get reply for bytesWritten=" + bytesWritten + ", " + request, e); + } assertReplyCorrespondingToRequest(request, reply); if (!reply.isSuccess()) { LOG.warn("reply is not success, request: {}", request); From 8913a999dd79cda58d18e17f4dd7971dda7070b7 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Sat, 13 Apr 2024 02:05:11 +0800 Subject: [PATCH 58/76] RATIS-2056. Update purge index when snapshot installed. (#1065) --- .../org/apache/ratis/server/raftlog/RaftLogBase.java | 12 +++++++++--- .../server/raftlog/segmented/SegmentedRaftLog.java | 4 ++-- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 9e079564f1..a41d90b1a3 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -142,6 +142,14 @@ public boolean updateCommitIndex(long majorityIndex, long currentTerm, boolean i return false; } + protected void updatePurgeIndex(Long purged) { + try (AutoCloseableLock writeLock = writeLock()) { + if (purged != null) { + purgeIndex.updateToMax(purged, infoIndexChange); + } + } + } + protected void updateSnapshotIndexFromStateMachine() { updateSnapshotIndex(getSnapshotIndexFromStateMachine.getAsLong()); } @@ -340,9 +348,7 @@ public final CompletableFuture purge(long suggestedIndex) { LOG.info("{}: purge {}", getName(), suggestedIndex); final long finalSuggestedIndex = suggestedIndex; return purgeImpl(suggestedIndex).whenComplete((purged, e) -> { - if (purged != null) { - purgeIndex.updateToMax(purged, infoIndexChange); - } + updatePurgeIndex(purged); if (e != null) { LOG.warn(getName() + ": Failed to purge " + finalSuggestedIndex, e); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 8d972a1e2c..7f9ef30981 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -272,7 +272,7 @@ private void loadLogSegments(long lastIndexInSnapshot, if (!cache.isEmpty() && cache.getEndIndex() < lastIndexInSnapshot) { LOG.warn("End log index {} is smaller than last index in snapshot {}", cache.getEndIndex(), lastIndexInSnapshot); - purgeImpl(lastIndexInSnapshot); + purgeImpl(lastIndexInSnapshot).whenComplete((purged, e) -> updatePurgeIndex(purged)); } } } @@ -557,7 +557,7 @@ public CompletableFuture onSnapshotInstalled(long lastSnapshotIndex) { cacheEviction.signal(); } } - return purgeImpl(lastSnapshotIndex); + return purgeImpl(lastSnapshotIndex).whenComplete((purged, e) -> updatePurgeIndex(purged)); } @Override From cf9d2292a9f53792a7765da41e3b4b903c0ec1d9 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 15 Apr 2024 23:28:53 -0700 Subject: [PATCH 59/76] RATIS-2057. Add back JavaUtils.attemptUntilTrue(..). (#1066) --- .../main/java/org/apache/ratis/util/JavaUtils.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java index f689006db2..7d1d75309a 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java @@ -41,6 +41,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; +import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -244,6 +245,18 @@ static void attempt( attemptRepeatedly(CheckedRunnable.asCheckedSupplier(runnable), numAttempts, sleepTime, name, log); } + /** Attempt to wait the given condition to return true multiple times. */ + static void attemptUntilTrue( + BooleanSupplier condition, int numAttempts, TimeDuration sleepTime, String name, Logger log) + throws InterruptedException { + Objects.requireNonNull(condition, "condition == null"); + attempt(() -> { + if (!condition.getAsBoolean()) { + throw new IllegalStateException("Condition " + name + " is false."); + } + }, numAttempts, sleepTime, name, log); + } + static Timer runRepeatedly(Runnable runnable, long delay, long period, TimeUnit unit) { final Timer timer = new Timer(true); timer.schedule(new TimerTask() { From ee09075c0e930756e5ad390fc9250bd5d1ffea52 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Wed, 17 Apr 2024 10:10:13 -0700 Subject: [PATCH 60/76] RATIS-2059. Missing reference count when putting log entries to cache on follower. (#1067) --- .../apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 7f9ef30981..a3c18f01cf 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -457,7 +457,7 @@ protected CompletableFuture appendEntryImpl(ReferenceCountedObject Date: Fri, 19 Apr 2024 23:20:16 +0800 Subject: [PATCH 61/76] RATIS-2055. Move notifyTermIndexUpdated after leader.checkReady (#1068) --- .../ratis/server/impl/RaftServerImpl.java | 26 +++++++++++++------ 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 2cec095785..34fede6001 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1805,17 +1805,18 @@ TransactionContext getTransactionContext(LogEntryProto entry, Boolean createNew) CompletableFuture applyLogToStateMachine(ReferenceCountedObject nextRef) throws RaftLogIOException { LogEntryProto next = nextRef.get(); - if (!next.hasStateMachineLogEntry()) { - stateMachine.event().notifyTermIndexUpdated(next.getTerm(), next.getIndex()); - } + CompletableFuture messageFuture = null; - if (next.hasConfigurationEntry()) { + switch (next.getLogEntryBodyCase()) { + case CONFIGURATIONENTRY: // the reply should have already been set. only need to record // the new conf in the metadata file and notify the StateMachine. state.writeRaftConfiguration(next); - stateMachine.event().notifyConfigurationChanged(next.getTerm(), next.getIndex(), next.getConfigurationEntry()); + stateMachine.event().notifyConfigurationChanged(next.getTerm(), next.getIndex(), + next.getConfigurationEntry()); role.getLeaderState().ifPresent(leader -> leader.checkReady(next)); - } else if (next.hasStateMachineLogEntry()) { + break; + case STATEMACHINELOGENTRY: TransactionContext trx = getTransactionContext(next, true); final ClientInvocationId invocationId = ClientInvocationId.valueOf(next.getStateMachineLogEntry()); writeIndexCache.add(invocationId.getClientId(), ((TransactionContextImpl) trx).getLogIndexFuture()); @@ -1825,12 +1826,21 @@ CompletableFuture applyLogToStateMachine(ReferenceCountedObject stateMachineFuture = stateMachine.applyTransaction(trx); - return replyPendingRequest(invocationId, TermIndex.valueOf(next), stateMachineFuture); + messageFuture = replyPendingRequest(invocationId, TermIndex.valueOf(next), stateMachineFuture); } catch (Exception e) { throw new RaftLogIOException(e); } + break; + case METADATAENTRY: + break; + default: + throw new IllegalStateException("Unexpected LogEntryBodyCase " + next.getLogEntryBodyCase() + ", next=" + next); } - return null; + + if (next.getLogEntryBodyCase() != LogEntryProto.LogEntryBodyCase.STATEMACHINELOGENTRY) { + stateMachine.event().notifyTermIndexUpdated(next.getTerm(), next.getIndex()); + } + return messageFuture; } /** From 7f1b1b6be29fd0b8675e30be511b03f6342ad7e6 Mon Sep 17 00:00:00 2001 From: Potato Date: Thu, 25 Apr 2024 19:44:55 +0800 Subject: [PATCH 62/76] RATIS-2060 Avoid unexpected shutdown of RaftServerProxy during membership change (#1069) --- .../main/java/org/apache/ratis/server/impl/LeaderElection.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index 53e83d866e..d738c87578 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -333,7 +333,7 @@ private boolean askForVotes(Phase phase, int round) throws InterruptedException, return true; case NOT_IN_CONF: case SHUTDOWN: - server.getRaftServer().close(); + server.close(); server.getStateMachine().event().notifyServerShutdown(server.getRoleInfoProto()); return false; case TIMEOUT: From b0ec1143c49e61525baf5da935a3509a59246883 Mon Sep 17 00:00:00 2001 From: Hongbing Wang <284734261@qq.com> Date: Fri, 26 Apr 2024 23:14:07 +0800 Subject: [PATCH 63/76] RATIS-2061. Fix setCloseThreshold parameter in RaftServerConfigKeys (#1070) --- .../apache/ratis/server/RaftServerConfigKeys.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java index 7419ca095f..f6dcbaa66b 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java @@ -77,9 +77,14 @@ static TimeDuration sleepDeviationThreshold(RaftProperties properties) { return getTimeDuration(properties.getTimeDuration(SLEEP_DEVIATION_THRESHOLD_DEFAULT.getUnit()), SLEEP_DEVIATION_THRESHOLD_KEY, SLEEP_DEVIATION_THRESHOLD_DEFAULT, getDefaultLog()); } + /** @deprecated use {@link #setSleepDeviationThreshold(RaftProperties, TimeDuration)}. */ + @Deprecated static void setSleepDeviationThreshold(RaftProperties properties, int thresholdMs) { setInt(properties::setInt, SLEEP_DEVIATION_THRESHOLD_KEY, thresholdMs); } + static void setSleepDeviationThreshold(RaftProperties properties, TimeDuration threshold) { + setTimeDuration(properties::setTimeDuration, SLEEP_DEVIATION_THRESHOLD_KEY, threshold); + } String CLOSE_THRESHOLD_KEY = PREFIX + ".close.threshold"; TimeDuration CLOSE_THRESHOLD_DEFAULT = TimeDuration.valueOf(60, TimeUnit.SECONDS); @@ -87,8 +92,13 @@ static TimeDuration closeThreshold(RaftProperties properties) { return getTimeDuration(properties.getTimeDuration(CLOSE_THRESHOLD_DEFAULT.getUnit()), CLOSE_THRESHOLD_KEY, CLOSE_THRESHOLD_DEFAULT, getDefaultLog()); } - static void setCloseThreshold(RaftProperties properties, int thresholdMs) { - setInt(properties::setInt, CLOSE_THRESHOLD_KEY, thresholdMs); + /** @deprecated use {@link #setCloseThreshold(RaftProperties, TimeDuration)}. */ + @Deprecated + static void setCloseThreshold(RaftProperties properties, int thresholdSec) { + setInt(properties::setInt, CLOSE_THRESHOLD_KEY, thresholdSec); + } + static void setCloseThreshold(RaftProperties properties, TimeDuration threshold) { + setTimeDuration(properties::setTimeDuration, CLOSE_THRESHOLD_KEY, threshold); } /** From 1fe59e07dff9b02e46d150de8b46ed2d4fc3abce Mon Sep 17 00:00:00 2001 From: Potato Date: Tue, 30 Apr 2024 05:24:26 +0800 Subject: [PATCH 64/76] RATIS-2065. Avoid the out-of-heap memory OOM phenomenon of frequent creation and deletion of Raft group scenarios. (#1071) --- .../ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index 7dc87faf75..cd15ea5efd 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -38,6 +38,7 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.StateMachine.DataStream; import org.apache.ratis.statemachine.TransactionContext; +import org.apache.ratis.thirdparty.io.netty.util.internal.PlatformDependent; import org.apache.ratis.util.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -242,6 +243,7 @@ void close() { Optional.ofNullable(flushExecutor).ifPresent(ExecutorService::shutdown); ConcurrentUtils.shutdownAndWait(TimeDuration.ONE_SECOND.multiply(3), workerThreadExecutor, timeout -> LOG.warn("{}: shutdown timeout in " + timeout, name)); + PlatformDependent.freeDirectBuffer(writeBuffer); IOUtils.cleanup(LOG, out); LOG.info("{} close()", name); } From 4e3eee7c9f603d89b701921c3ab877134b342f0e Mon Sep 17 00:00:00 2001 From: Sadanand Shenoy Date: Sat, 4 May 2024 00:10:53 +0530 Subject: [PATCH 65/76] RATIS-2069. RaftMetaConf command sets incorrect peerAddress pattern. (#1075) --- .../ratis/shell/cli/sh/local/RaftMetaConfCommand.java | 2 +- .../ratis/shell/cli/sh/LocalCommandIntegrationTest.java | 9 +++++++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java index 9f0558c5ea..e258d863b8 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java @@ -91,7 +91,7 @@ public int run(CommandLine cl) throws IOException { } InetSocketAddress inetSocketAddress = parseInetSocketAddress( peerIdWithAddressArray[peerIdWithAddressArray.length - 1]); - String addressString = inetSocketAddress.toString(); + String addressString = inetSocketAddress.getHostString() + ":" + inetSocketAddress.getPort(); if (addresses.contains(addressString)) { printf("Found duplicated address: %s. Please make sure the address of peer have no duplicated value.", addressString); diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java index 4a07e372ca..a85c247d66 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java @@ -78,11 +78,11 @@ public void testRunMethod(@TempDir Path tempDir) throws Exception { int index = 1; generateRaftConf(tempDir.resolve(RAFT_META_CONF), index); - String[] testPeersListArray = {"peer1_ID|host1:9872,peer2_ID|host2:9872,peer3_ID|host3:9872", + String[] testPeersListArray = {"peer1_ID|localhost:9872,peer2_ID|host2:9872,peer3_ID|host3:9872", "host1:9872,host2:9872,host3:9872"}; for (String peersListStr : testPeersListArray) { - generateRaftConf(tempDir, index); + generateRaftConf(tempDir.resolve(RAFT_META_CONF), index); StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); int ret = shell.run("local", "raftMetaConf", "-peers", peersListStr, "-path", tempDir.toString()); @@ -99,6 +99,11 @@ public void testRunMethod(@TempDir Path tempDir) throws Exception { Assertions.assertEquals(index + 1, indexFromNewConf); + String addressRegex = "^[a-zA-Z0-9.-]+:\\d+$"; + Pattern pattern = Pattern.compile(addressRegex); + peers.forEach(p -> Assertions.assertTrue( + pattern.matcher(p.getAddress()).matches())); + String peersListStrFromNewMetaConf; if (containsPeerId(peersListStr)) { peersListStrFromNewMetaConf = peers.stream() From 51244e42cabb98f7ac9ca144e311222d9c6e5a45 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Fri, 3 May 2024 11:48:52 -0700 Subject: [PATCH 66/76] RATIS-2068. Avoid logging raw StateMachine data body. (#1074) --- .../server/GrpcServerProtocolService.java | 2 +- .../ratis/server/impl/RaftServerImpl.java | 5 +++-- .../ratis/server/raftlog/LogProtoUtils.java | 19 ++++++++++++------- .../ratis/server/raftlog/RaftLogBase.java | 4 ++-- .../raftlog/segmented/SegmentedRaftLog.java | 8 ++++---- .../ratis/server/util/ServerStringUtils.java | 8 ++++++-- 6 files changed, 28 insertions(+), 18 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index 9c426f322a..00cb699cfc 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -316,7 +316,7 @@ boolean isHeartbeat(AppendEntriesRequestProto request) { @Override String requestToString(AppendEntriesRequestProto request) { - return ServerStringUtils.toAppendEntriesRequestString(request); + return ServerStringUtils.toAppendEntriesRequestString(request, null); } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 34fede6001..4512a2c223 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1471,7 +1471,8 @@ public CompletableFuture appendEntriesAsync( return appendEntriesAsync(leaderId, request.getCallId(), previous, requestRef); } catch(Exception t) { - LOG.error("{}: Failed appendEntries* {}", getMemberId(), toAppendEntriesRequestString(r), t); + LOG.error("{}: Failed appendEntries* {}", getMemberId(), + toAppendEntriesRequestString(r, stateMachine::toStateMachineLogEntryString), t); throw IOUtils.asIOException(t); } finally { requestRef.release(); @@ -1530,7 +1531,7 @@ private CompletableFuture appendEntriesAsync(RaftPeerId final List entries = proto.getEntriesList(); final boolean isHeartbeat = entries.isEmpty(); logAppendEntries(isHeartbeat, () -> getMemberId() + ": appendEntries* " - + toAppendEntriesRequestString(proto)); + + toAppendEntriesRequestString(proto, stateMachine::toStateMachineLogEntryString)); final long leaderTerm = proto.getLeaderTerm(); final long currentTerm; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java index 59da4c3fd6..e969eaa484 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java @@ -19,7 +19,6 @@ import org.apache.ratis.proto.RaftProtos.*; import org.apache.ratis.protocol.ClientId; -import org.apache.ratis.protocol.ClientInvocationId; import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.server.RaftConfiguration; @@ -46,9 +45,10 @@ public static String toLogEntryString(LogEntryProto entry, Function proto -> "" + ClientInvocationId.valueOf(proto)) - .apply(entry.getStateMachineLogEntry()); + if (function == null) { + function = LogProtoUtils::stateMachineLogEntryProtoToString; + } + s = ", " + function.apply(entry.getStateMachineLogEntry()); } else if (entry.hasMetadataEntry()) { final MetadataProto metadata = entry.getMetadataEntry(); s = "(c:" + metadata.getCommitIndex() + ")"; @@ -70,7 +70,11 @@ static String peersToString(List peers) { } static String stateMachineLogEntryProtoToString(StateMachineLogEntryProto p) { - return "logData:" + p.getLogData() + ", stateMachineEntry:" + p.getType() + ":" + p.getStateMachineEntry(); + final StateMachineEntryProto stateMachineEntry = p.getStateMachineEntry(); + return p.getType() + + ": logData.size=" + p.getLogData().size() + + ", stateMachineData.size=" + stateMachineEntry.getStateMachineData().size() + + ", logEntryProtoSerializedSize=" + stateMachineEntry.getLogEntryProtoSerializedSize(); } public static String toLogEntryString(LogEntryProto entry) { @@ -82,10 +86,11 @@ public static String toLogEntriesString(List entries) { : entries.stream().map(LogProtoUtils::toLogEntryString).collect(Collectors.toList()).toString(); } - public static String toLogEntriesShortString(List entries) { + public static String toLogEntriesShortString(List entries, + Function stateMachineToString) { return entries == null ? null : entries.isEmpty()? "" - : "size=" + entries.size() + ", first=" + LogProtoUtils.toLogEntryString(entries.get(0)); + : "size=" + entries.size() + ", first=" + toLogEntryString(entries.get(0), stateMachineToString); } public static LogEntryProto toLogEntryProto(RaftConfiguration conf, Long term, long index) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index a41d90b1a3..3b9724cc6f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -199,10 +199,10 @@ private long appendImpl(long term, TransactionContext operation) throws StateMac appendEntry(operation.wrap(e), operation).whenComplete((returned, t) -> { if (t != null) { - LOG.error(name + ": Failed to write log entry " + LogProtoUtils.toLogEntryString(e), t); + LOG.error(name + ": Failed to write log entry " + toLogEntryString(e), t); } else if (returned != nextIndex) { LOG.error("{}: Indices mismatched: returned index={} but nextIndex={} for log entry {}", - name, returned, nextIndex, LogProtoUtils.toLogEntryString(e)); + name, returned, nextIndex, toLogEntryString(e)); } else { return; // no error } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index a3c18f01cf..7c33adf00a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -345,8 +345,7 @@ public ReferenceCountedObject retainEntryWithData(long index) thr } return future != null? newEntryWithData(entryRef, future): newEntryWithData(entryRef); } catch (Exception e) { - final String err = getName() + ": Failed readStateMachineData for " + - LogProtoUtils.toLogEntryString(entry); + final String err = getName() + ": Failed readStateMachineData for " + toLogEntryString(entry); LOG.error(err, e); throw new RaftLogIOException(err, JavaUtils.unwrapCompletionException(e)); } @@ -467,7 +466,7 @@ protected CompletableFuture appendEntryImpl(ReferenceCountedObject appendEntryTimerContext.stop()); } catch (Exception e) { - LOG.error("{}: Failed to append {}", getName(), LogProtoUtils.toLogEntryString(entry), e); + LOG.error("{}: Failed to append {}", getName(), toLogEntryString(entry), e); throw e; } finally { entryRef.release(); @@ -578,7 +577,8 @@ SegmentedRaftLogCache getRaftLogCache() { @Override public String toLogEntryString(LogEntryProto logEntry) { - return LogProtoUtils.toLogEntryString(logEntry, stateMachine::toStateMachineLogEntryString); + return LogProtoUtils.toLogEntryString(logEntry, stateMachine != null ? + stateMachine::toStateMachineLogEntryString : null); } public static Builder newBuilder() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java index 284664d012..6e0fce4d66 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java @@ -23,11 +23,13 @@ import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; +import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.util.ProtoUtils; import java.util.List; +import java.util.function.Function; /** * This class provides convenient utilities for converting Protocol Buffers messages to strings. @@ -41,7 +43,8 @@ public final class ServerStringUtils { private ServerStringUtils() {} - public static String toAppendEntriesRequestString(AppendEntriesRequestProto request) { + public static String toAppendEntriesRequestString(AppendEntriesRequestProto request, + Function stateMachineToString) { if (request == null) { return null; } @@ -51,7 +54,8 @@ public static String toAppendEntriesRequestString(AppendEntriesRequestProto requ + ",previous=" + TermIndex.valueOf(request.getPreviousLog()) + ",leaderCommit=" + request.getLeaderCommit() + ",initializing? " + request.getInitializing() - + "," + (entries.isEmpty()? "HEARTBEAT" : "entries: " + LogProtoUtils.toLogEntriesShortString(entries)); + + "," + (entries.isEmpty()? "HEARTBEAT" : "entries: " + + LogProtoUtils.toLogEntriesShortString(entries, stateMachineToString)); } public static String toAppendEntriesReplyString(AppendEntriesReplyProto reply) { From cbd9793a0e315276d370dbfc5dd6a75ccf67f454 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sun, 5 May 2024 08:04:09 +0200 Subject: [PATCH 67/76] RATIS-2070. Skip push build for dependabot (#1077) --- .github/workflows/post-commit.yml | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml index 2d3258ccc5..22a6df756e 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yml @@ -14,8 +14,12 @@ # limitations under the License. name: build-branch on: - - push - - pull_request + push: + branches-ignore: + - 'dependabot/**' + tags: + - '**' + pull_request: env: WITH_COVERAGE: true jobs: From 6fc42220bc8e39cf8f242f3205abcd4dce8bbe44 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 5 May 2024 13:28:02 +0200 Subject: [PATCH 68/76] RATIS-2071. Bump Bouncy Castle to 1.78.1 (#1076) --- ratis-test/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 5990ae057f..01d54ba15b 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -24,7 +24,7 @@ Apache Ratis Test - 1.76 + 1.78.1 From 58671923e78a10d5c6a7d49a06e2d95690d5e9bb Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 6 May 2024 17:44:26 +0200 Subject: [PATCH 69/76] RATIS-2072. Limit timeout of CI workflow jobs (#1078) --- .github/workflows/post-commit.yml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml index 22a6df756e..13a9214445 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yml @@ -25,6 +25,7 @@ env: jobs: build: runs-on: ubuntu-20.04 + timeout-minutes: 30 steps: - name: Checkout project uses: actions/checkout@v4 @@ -62,6 +63,7 @@ jobs: needs: - build runs-on: ubuntu-20.04 + timeout-minutes: 30 strategy: matrix: java: [ 11 ] @@ -93,6 +95,7 @@ jobs: rat: name: rat runs-on: ubuntu-20.04 + timeout-minutes: 15 steps: - name: Checkout project uses: actions/checkout@v4 @@ -116,6 +119,7 @@ jobs: author: name: author runs-on: ubuntu-20.04 + timeout-minutes: 15 steps: - name: Checkout project uses: actions/checkout@v4 @@ -130,6 +134,7 @@ jobs: unit: name: unit runs-on: ubuntu-20.04 + timeout-minutes: 60 strategy: matrix: profile: @@ -173,6 +178,7 @@ jobs: checkstyle: name: checkstyle runs-on: ubuntu-20.04 + timeout-minutes: 15 steps: - name: Checkout project uses: actions/checkout@v4 @@ -196,6 +202,7 @@ jobs: findbugs: name: findbugs runs-on: ubuntu-20.04 + timeout-minutes: 30 steps: - name: Setup java uses: actions/setup-java@v4 @@ -226,6 +233,7 @@ jobs: - build - unit runs-on: ubuntu-20.04 + timeout-minutes: 30 if: (github.repository == 'apache/ratis' || github.repository == 'apache/incubator-ratis') && github.event_name != 'pull_request' steps: - name: Checkout project From 839f8fb265983b5ce14cf2cb3650a70d342ffc04 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 6 May 2024 20:48:45 +0200 Subject: [PATCH 70/76] RATIS-2073. Enable Surefire process checkers (#1079) --- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0b3eedee12..0130258b18 100644 --- a/pom.xml +++ b/pom.xml @@ -176,7 +176,7 @@ 3.3.0 4.0.6 1.6.1 - 3.0.0 + 3.0.0-M4 3.5.3 @@ -634,6 +634,7 @@ false false false + all 600 -Xmx2048m -XX:+HeapDumpOnOutOfMemoryError @{argLine} From 3888520717402da8638b85ae17f1ab494ecd2872 Mon Sep 17 00:00:00 2001 From: Potato Date: Wed, 8 May 2024 00:28:46 +0800 Subject: [PATCH 71/76] RATIS-2067. Enhance content for concept in ratis-docs. (#1073) --- ratis-docs/src/site/markdown/concept/index.md | 66 +++++++++++++++++-- ratis-docs/src/site/markdown/snapshot.md | 2 + 2 files changed, 61 insertions(+), 7 deletions(-) diff --git a/ratis-docs/src/site/markdown/concept/index.md b/ratis-docs/src/site/markdown/concept/index.md index 279338a5fd..6ff77b2203 100644 --- a/ratis-docs/src/site/markdown/concept/index.md +++ b/ratis-docs/src/site/markdown/concept/index.md @@ -15,24 +15,76 @@ limitations under the License. --> # Concepts -TODO: complete this section + ## RaftServer +The `RaftServer` is a core component of Apache Ratis, +responsible for handling all operations related to the RAFT protocol. +A `RaftServer` manages all the `RaftGroup`(s) within the current process. +For each group, a corresponding `RaftServer.Division` handles the core functions +such as replication of log entries, voting, and leader election within that group. +Each division can independently act as a Leader, Candidate, Follower or Listener, +with the specific role depending on the outcome of elections and the state of the protocol. + ## RaftPeer and RaftPeerId +`RaftPeer` represents a participant node in the cluster, +including the node's unique identifier, IP address, and port number. +The unique identifier is represented by the `RaftPeerId` class, +which is crucial for distinguishing different nodes within a cluster. + ## RaftGroup -## Transport (gRPC,Netty ...) +A `RaftGroup` represents a collection of `RaftPeer`(s) in a Raft protocol cluster. +Each group has a unique identifier represented by the `RaftGroupId` class. +Multiple groups can operate independently within a physical network, +while each group managing its own consistency and state replication. -## Raft Log +## Transport (gRPC, Netty, etc.) -## StateMachine -A StateMachine is the abstraction point for user-code +Ratis supports various network transport protocols for node communication, +including gRPC (default) and Netty. +These transport layers in Ratis are used for data serialization and deserialization, +as well as ensuring safe and efficient data transmission between different nodes. + +## RaftLog + +The `RaftLog` is a core component of the Raft algorithm, +used to record all state change transactions. +Once a log entry has been acknowledged by a majority of peers, +the entry becomes committed. +The Raft log is key to achieving distributed data consistency. ## Snapshot +A `Snapshot` is a point-in-time copy of the current state of the `StateMachine`. +It can be used for quick recovery of the state after system restarts, +and for transferring the state to newly joined nodes. +When a snapshot has been taken, +the log entries earlier than the snapshot can be purged +in order to free up the storage space. + ## TermIndex -## Transaction (as in applyTransaction or startTransaction in StateMachine) +`TermIndex` is an order pair of `long` integers (term, index) as defined in the Raft protocol. +Term is the logical clock in Raft. +A newly elected leader starts a new term and remains the leader for the rest of the term. +Index is the position of log entries in the Raft log. + +## StateMachine + +In Ratis, `StateMachine` is the abstraction point for user-defined code. +Developers implement specific business logic or data storage operations at this layer. +The transactions committed through the Raft protocol will be applied to it. + +### The `applyTransaction` method + +In Ratis, transaction processing is implemented by the `StateMachine` +through the `applyTransaction` method. +A transaction usually changes the state of the `StateMachine`. + +### StateMachineStorage -## StateMachineStorage +`StateMachineStorage` is a component for storing data related to the `StateMachine`. +It is for persisting the Raft log and the snapshots +such that the state can be fully recovered even after system failures. diff --git a/ratis-docs/src/site/markdown/snapshot.md b/ratis-docs/src/site/markdown/snapshot.md index f20dc19d71..a9f06511fc 100644 --- a/ratis-docs/src/site/markdown/snapshot.md +++ b/ratis-docs/src/site/markdown/snapshot.md @@ -75,6 +75,8 @@ Note that Ratis imposes a minimal creation gap between two subsequent snapshot c ```java // SnapshotManagementApi RaftClientReply create(long timeoutMs) throws IOException; + RaftClientReply create(boolean force, long timeoutMs) throws IOException; + RaftClientReply create(long creationGap, long timeoutMs) throws IOException; ``` ```java From ac05d64f3f846d56658442896a39f4f5fd7495a2 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 7 May 2024 11:27:36 -0700 Subject: [PATCH 72/76] RATIS-2074. Intermittent fork timeout in TestRaftWithNetty#testBasicAppendEntriesKillLeader. (#1080) --- .../org/apache/ratis/netty/NettyClient.java | 21 ++++++++++++------- .../org/apache/ratis/netty/NettyRpcProxy.java | 4 ++-- .../org/apache/ratis/netty/NettyUtils.java | 20 ++++++++++++++++++ .../java/org/apache/ratis/RaftBasicTests.java | 2 ++ 4 files changed, 38 insertions(+), 9 deletions(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java index 0cf4bd3833..efea5fd0f8 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java @@ -33,13 +33,17 @@ import java.net.InetSocketAddress; public class NettyClient implements Closeable { - private final LifeCycle lifeCycle = new LifeCycle(JavaUtils.getClassSimpleName(getClass())); - + private final LifeCycle lifeCycle; + private final String serverAddress; private Channel channel; + NettyClient(String serverAddress) { + this.lifeCycle = new LifeCycle(JavaUtils.getClassSimpleName(getClass()) + "-" + serverAddress); + this.serverAddress = serverAddress; + } + /** Connects to the given server address. */ - public void connect(String serverAddress, EventLoopGroup group, - ChannelInitializer initializer) + public void connect(EventLoopGroup group, ChannelInitializer initializer) throws InterruptedException { final InetSocketAddress address = NetUtils.createSocketAddr(serverAddress); @@ -57,13 +61,16 @@ public void connect(String serverAddress, EventLoopGroup group, @Override public void close() { - lifeCycle.checkStateAndClose(() -> { - channel.close().syncUninterruptibly(); - }); + lifeCycle.checkStateAndClose(() -> NettyUtils.closeChannel(channel, serverAddress)); } public ChannelFuture writeAndFlush(Object msg) { lifeCycle.assertCurrentState(LifeCycle.States.RUNNING); return channel.writeAndFlush(msg); } + + @Override + public String toString() { + return lifeCycle.toString(); + } } diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java index b7a04b050b..b9788a8bb4 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java @@ -99,7 +99,7 @@ public static long getCallId(RaftNettyServerReplyProto proto) { class Connection implements Closeable { - private final NettyClient client = new NettyClient(); + private final NettyClient client = new NettyClient(peer.getAddress()); private final Queue> replies = new LinkedList<>(); @@ -137,7 +137,7 @@ protected void initChannel(SocketChannel ch) throws Exception { } }; - client.connect(peer.getAddress(), group, initializer); + client.connect(group, initializer); } synchronized ChannelFuture offer(RaftNettyServerRequestProto request, diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyUtils.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyUtils.java index 8cce291af9..37666bf189 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyUtils.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyUtils.java @@ -22,6 +22,8 @@ import org.apache.ratis.security.TlsConf.KeyManagerConf; import org.apache.ratis.security.TlsConf.PrivateKeyConf; import org.apache.ratis.security.TlsConf.TrustManagerConf; +import org.apache.ratis.thirdparty.io.netty.channel.Channel; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelFuture; import org.apache.ratis.thirdparty.io.netty.channel.EventLoopGroup; import org.apache.ratis.thirdparty.io.netty.channel.ServerChannel; import org.apache.ratis.thirdparty.io.netty.channel.epoll.Epoll; @@ -35,16 +37,19 @@ import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; import org.apache.ratis.util.ConcurrentUtils; +import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.net.ssl.KeyManager; import javax.net.ssl.TrustManager; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; public interface NettyUtils { Logger LOG = LoggerFactory.getLogger(NettyUtils.class); + TimeDuration CLOSE_TIMEOUT = TimeDuration.valueOf(5, TimeUnit.SECONDS); class Print { private static final AtomicBoolean PRINTED_EPOLL_UNAVAILABILITY_CAUSE = new AtomicBoolean(); @@ -176,4 +181,19 @@ static Class getServerChannelClass(EventLoopGroup event return eventLoopGroup instanceof EpollEventLoopGroup ? EpollServerSocketChannel.class : NioServerSocketChannel.class; } + + static void closeChannel(Channel channel, String name) { + final ChannelFuture f = channel.close(); + final boolean completed; + try { + completed = f.await(CLOSE_TIMEOUT.getDuration(), CLOSE_TIMEOUT.getUnit()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.info("Interrupted closeChannel {} ", name, e); + return; + } + if (!completed) { + LOG.warn("closeChannel {} is not yet completed in {}", name, CLOSE_TIMEOUT); + } + } } \ No newline at end of file diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index 13ee08c83b..9f360cb51e 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -45,6 +45,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -338,6 +339,7 @@ public String toString() { } @Test + @Timeout(value = 300) public void testWithLoad() throws Exception { runWithNewCluster(NUM_SERVERS, cluster -> testWithLoad(10, 300, false, cluster, LOG)); } From e1765016323735124e3aaa5cee44688d49d48791 Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Thu, 9 May 2024 01:24:50 +0800 Subject: [PATCH 73/76] RATIS-2078. Correct timeout unit for tests (#1083) --- .../apache/ratis/conf/TestRaftProperties.java | 2 +- .../ratis/grpc/TestRetryCacheWithGrpc.java | 2 +- .../ratis/server/impl/TestRaftServerJmx.java | 4 ++-- .../ratis/util/TestDataBlockingQueue.java | 8 ++++---- .../org/apache/ratis/util/TestDataQueue.java | 8 ++++---- .../org/apache/ratis/util/TestExitUtils.java | 4 ++-- .../org/apache/ratis/util/TestLifeCycle.java | 2 +- .../java/org/apache/ratis/util/TestMinMax.java | 2 +- .../apache/ratis/util/TestPeerProxyMap.java | 4 ++-- .../apache/ratis/util/TestPreconditions.java | 4 ++-- .../ratis/util/TestReferenceCountedObject.java | 4 ++-- .../ratis/util/TestResourceSemaphore.java | 2 +- .../apache/ratis/util/TestTimeDuration.java | 18 +++++++++--------- .../ratis/util/TestTimeoutScheduler.java | 10 +++++----- .../util/TestTraditionalBinaryPrefix.java | 4 ++-- 15 files changed, 39 insertions(+), 39 deletions(-) diff --git a/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java b/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java index 29bfc321df..d17d6fb6f6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java +++ b/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java @@ -41,7 +41,7 @@ static void setNonUnderscoreValue(RaftProperties p, String valueWithoutUnderscor } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testUnderscore() { final RaftProperties p = new RaftProperties(); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java index 879b9eabdb..b244691437 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java @@ -172,7 +172,7 @@ void run() throws Exception { } @Test - @Timeout(value = 10000) + @Timeout(value = 10) public void testRetryOnResourceUnavailableException() throws InterruptedException, IOException { RaftProperties properties = new RaftProperties(); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java index 39a1f8855b..6302c7e8b1 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java @@ -40,7 +40,7 @@ public class TestRaftServerJmx extends BaseTest { @Test - @Timeout(value = 30000) + @Timeout(value = 30) public void testJmxBeans() throws Exception { final int NUM_SERVERS = 3; final MiniRaftClusterWithSimulatedRpc cluster @@ -60,7 +60,7 @@ public void testJmxBeans() throws Exception { } @Test - @Timeout(value = 30000) + @Timeout(value = 30) public void testRegister() throws JMException { { final JmxRegister jmx = new JmxRegister(); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java b/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java index e9dcf61d09..3ad40d4ca7 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java @@ -38,25 +38,25 @@ public class TestDataBlockingQueue { final TimeDuration fast = TimeDuration.valueOf(10, TimeUnit.MILLISECONDS); @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testElementLimit() { TestDataQueue.runTestElementLimit(q); } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testByteLimit() { TestDataQueue.runTestByteLimit(q); } @Test - @Timeout(value = 1000) + @Timeout(value = 10) public void testSlowOfferFastPoll() throws Exception { runTestBlockingCalls(slow, fast, q); } @Test - @Timeout(value = 1000) + @Timeout(value = 10) public void testFastOfferSlowPoll() throws Exception { runTestBlockingCalls(fast, slow, q); } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java b/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java index 2dafa7e91c..42a55d90ff 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java @@ -47,7 +47,7 @@ static void assertSizes(long expectedNumElements, long expectedNumBytes, DataQue final DataQueue q = new DataQueue<>(null, byteLimit, elementLimit, Long::longValue); @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testElementLimit() { runTestElementLimit(q); } @@ -82,7 +82,7 @@ static void runTestElementLimit(DataQueue q) { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testByteLimit() { runTestByteLimit(q); } @@ -140,7 +140,7 @@ static void runTestByteLimit(DataQueue q) { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testIteratorAndRemove() { runTestIteratorAndRemove(q); } @@ -185,7 +185,7 @@ static void runTestIteratorAndRemove(DataQueue q) { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testTimeout() { assertSizes(0, 0, q); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java b/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java index 1dd5b270bb..3c7cc025c9 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java @@ -26,7 +26,7 @@ public class TestExitUtils extends BaseTest { /** Test if {@link BaseTest} can handle uncaught exception. */ @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testUncaughtException() throws Exception { Assertions.assertFalse(ExitUtils.isTerminated()); Assertions.assertFalse(ExitUtils.clear()); @@ -43,7 +43,7 @@ public void testUncaughtException() throws Exception { /** Test if {@link BaseTest} can handle ExitUtils.terminate(..). */ @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testExitStatus() { Assertions.assertFalse(ExitUtils.isTerminated()); Assertions.assertFalse(ExitUtils.clear()); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java index f817449995..f08ceb3461 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java @@ -35,7 +35,7 @@ public class TestLifeCycle { * while this test uses successors. */ @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testIsValid() { final Map> successors = new EnumMap<>(LifeCycle.State.class); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java b/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java index 050d9973aa..07a9882ad2 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java @@ -29,7 +29,7 @@ public class TestMinMax { @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testMinMax() { runTestMinMax(LongStream.empty()); runTestMinMax(LongStream.iterate(0, n -> n).limit(10)); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java b/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java index 118830bda8..a2255041f8 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java @@ -53,7 +53,7 @@ public String toString() { } @Test - @Timeout(value = 10_000) + @Timeout(value = 10) public void testCloseDeadLock() throws Exception { final PeerProxyMap map = new PeerProxyMap<>("test", DummyProxy::new); final RaftPeerId id = RaftPeerId.valueOf("s0"); @@ -118,7 +118,7 @@ public String toString() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testStackTrace() { final RaftPeerId id = RaftPeerId.valueOf("s0"); final RaftPeer peer = RaftPeer.newBuilder().setId(id).build(); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java b/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java index e99c11fdde..73cf4c1676 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java @@ -30,7 +30,7 @@ public class TestPreconditions extends BaseTest { @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testAssertUnique() { final Set empty = Collections.emptySet(); Preconditions.assertUnique(empty); @@ -56,7 +56,7 @@ public void testAssertUnique() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testAssertNull() { final Map map = new HashMap<>(); final String key = "abc1234"; diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java b/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java index 502261ec23..2436310141 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java @@ -41,7 +41,7 @@ static void assertRelease(ReferenceCountedObject ref, } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testWrap() { final String value = "testWrap"; final AtomicInteger retained = new AtomicInteger(); @@ -116,7 +116,7 @@ public void testWrap() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testReleaseWithoutRetaining() { final ReferenceCountedObject ref = ReferenceCountedObject.wrap(""); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java b/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java index 0f3612dce4..72c811d877 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java @@ -29,7 +29,7 @@ public class TestResourceSemaphore extends BaseTest { @Test - @Timeout(value = 5000) + @Timeout(value = 5) public void testGroup() throws InterruptedException, TimeoutException { final int FAILED_IN_ELEMENT_LIMIT = 0; final int FAILED_IN_BYTE_SIZE_LIMIT = 1; diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java index f2d58f48db..9ba5e57235 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java @@ -40,7 +40,7 @@ public class TestTimeDuration { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testAbbreviation() { Arrays.asList(TimeUnit.values()) .forEach(a -> assertNotNull(Abbreviation.valueOf(a.name()))); @@ -57,7 +57,7 @@ public void testAbbreviation() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testParse() { assertEquals(1L, parse("1_000_000 ns", TimeUnit.MILLISECONDS)); assertEquals(10L, parse("10_000_000 nanos", TimeUnit.MILLISECONDS)); @@ -97,7 +97,7 @@ public void testParse() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testRoundUp() { final long nanosPerSecond = 1_000_000_000L; final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); @@ -113,7 +113,7 @@ public void testRoundUp() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testTo() { final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); assertTo(1000, "1000ms", oneSecond, TimeUnit.MILLISECONDS); @@ -137,7 +137,7 @@ static TimeDuration assertTo(long expected, String expectedString, TimeDuration } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testAddAndSubtract() { final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); final TimeDuration tenSecond = TimeDuration.valueOf(10, TimeUnit.SECONDS); @@ -191,7 +191,7 @@ public void testAddAndSubtract() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testNegate() { assertNegate(0); assertNegate(1); @@ -213,7 +213,7 @@ private static void assertNegate(long n) { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testMultiply() { assertMultiply(0, TimeDuration.ONE_SECOND, TimeDuration.valueOf(0, TimeUnit.SECONDS)); assertMultiply(0.001, TimeDuration.ONE_SECOND, TimeDuration.ONE_MILLISECOND); @@ -244,7 +244,7 @@ private static void assertMultiply(TimeDuration t, double multiplier, TimeDurati } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testHigherLower() { final TimeUnit[] units = {TimeUnit.NANOSECONDS, TimeUnit.MICROSECONDS, TimeUnit.MILLISECONDS, TimeUnit.SECONDS, TimeUnit.MINUTES, TimeUnit.HOURS, TimeUnit.DAYS}; @@ -257,7 +257,7 @@ public void testHigherLower() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testCompareTo() { assertTimeDurationCompareTo(TimeDuration.ONE_MINUTE, TimeDuration.ONE_SECOND); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java index 79c97a6ede..e8e9d6eb0c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java @@ -47,7 +47,7 @@ void assertNoError() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testSingleTask() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); @@ -84,7 +84,7 @@ public void testSingleTask() throws Exception { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testMultipleTasks() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); @@ -131,7 +131,7 @@ public void testMultipleTasks() throws Exception { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testExtendingGracePeriod() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); @@ -182,7 +182,7 @@ public void testExtendingGracePeriod() throws Exception { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testRestartingScheduler() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); @@ -216,7 +216,7 @@ public void testRestartingScheduler() throws Exception { } @Test - @Timeout(value = 10_000) + @Timeout(value = 10) public void testShutdown() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); Assertions.assertEquals(TimeoutScheduler.DEFAULT_GRACE_PERIOD, scheduler.getGracePeriod()); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java index 988cc5c710..a699b91c81 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java @@ -28,7 +28,7 @@ public class TestTraditionalBinaryPrefix { @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testTraditionalBinaryPrefix() { //test string2long(..) String[] symbol = {"k", "m", "g", "t", "p", "e"}; @@ -147,7 +147,7 @@ private static String byteDescription(long len) { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testUnderscore() { final SizeInBytes value = SizeInBytes.valueOf("1_000_000_000_000_000"); assertEquals(1_000_000_000_000_000L, value.getSize()); From 7a06fd81e56b76ecbfd1a327ea6fa8a2e798fade Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Thu, 9 May 2024 14:34:08 +0800 Subject: [PATCH 74/76] [RATIS-2082] RaftPeers equal should also check address --- .../src/main/java/org/apache/ratis/protocol/RaftPeer.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java index e7bfee3cd9..65a77855c4 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java @@ -254,7 +254,9 @@ public String getDetails() { @Override public boolean equals(Object o) { - return (o instanceof RaftPeer) && id.equals(((RaftPeer) o).getId()); + return (o instanceof RaftPeer) && id.equals(((RaftPeer) o).getId()) && + address != null && ((RaftPeer) o).getAddress() != null && + address.equals(((RaftPeer) o).getAddress()); } @Override From 85a162600eadb07a9465b8466e53cf2f85338ccf Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Thu, 9 May 2024 16:04:03 +0800 Subject: [PATCH 75/76] Update LeaderElectionTests.java --- .../java/org/apache/ratis/server/impl/LeaderElectionTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index fbb909ac93..b88c676ec3 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -214,7 +214,7 @@ public void testYieldLeaderToHigherPriority() throws Exception { Assertions.assertEquals(2, followers.size()); RaftServer.Division newLeader = followers.get(0); - List peers = cluster.getPeers(); + List peers = new ArrayList<>(cluster.group.getPeers()); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); Assertions.assertTrue(reply.isSuccess()); From e2c015b058e3073f01075efae58b6d1e1db07c49 Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Thu, 9 May 2024 16:27:45 +0800 Subject: [PATCH 76/76] Update LeaderElectionTests.java --- .../java/org/apache/ratis/server/impl/LeaderElectionTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index b88c676ec3..6e914d7f02 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -630,7 +630,7 @@ void runTestPauseResumeLeaderElection(CLUSTER cluster) throws IOException, Inter Assertions.assertTrue(pauseLeaderReply.isSuccess()); client.io().send(new RaftTestUtil.SimpleMessage("message")); RaftServer.Division newLeader = followers.get(0); - List peers = cluster.getPeers(); + List peers = new ArrayList<>(cluster.group.getPeers()); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); Assertions.assertTrue(reply.isSuccess());