This is an automated email from the ASF dual-hosted git repository.
sanpwc pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git
The following commit(s) were added to refs/heads/main by this push:
new 544786dd8f IGNITE-22971 Reduce log pollution by RaftGroupServiceImpl
(#4214)
544786dd8f is described below
commit 544786dd8f062d44055d4f8f03fd036477542f56
Author: Alexander Polovtcev <[email protected]>
AuthorDate: Mon Aug 12 13:39:02 2024 +0300
IGNITE-22971 Reduce log pollution by RaftGroupServiceImpl (#4214)
---
.../internal/raft/PeerUnavailableException.java | 29 ++++++++++++++++++++++
.../ignite/internal/raft/RaftGroupServiceImpl.java | 18 ++++++++------
.../raft/jraft/rpc/impl/AbstractClientService.java | 5 ++--
.../raft/jraft/rpc/impl/IgniteRpcClient.java | 5 ++--
4 files changed, 45 insertions(+), 12 deletions(-)
diff --git
a/modules/raft/src/main/java/org/apache/ignite/internal/raft/PeerUnavailableException.java
b/modules/raft/src/main/java/org/apache/ignite/internal/raft/PeerUnavailableException.java
new file mode 100644
index 0000000000..d52822d49c
--- /dev/null
+++
b/modules/raft/src/main/java/org/apache/ignite/internal/raft/PeerUnavailableException.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft;
+
+/**
+ * Special type of exception used when a target peer is not present in the
physical topology.
+ *
+ * <p>The stacktrace is omitted on purpose as to reduce log pollution (this
exception is thrown and logged nearly immediately).
+ */
+public class PeerUnavailableException extends RuntimeException {
+ public PeerUnavailableException(String consistentId) {
+ super("Peer " + consistentId + " is unavailable", null, true, false);
+ }
+}
diff --git
a/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
b/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
index 1f04cfdd75..13065a1c78 100644
---
a/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
+++
b/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
@@ -21,6 +21,7 @@ import static java.lang.System.currentTimeMillis;
import static java.util.concurrent.ThreadLocalRandom.current;
import static java.util.stream.Collectors.toList;
import static
org.apache.ignite.internal.tostring.IgniteToStringBuilder.includeSensitive;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.AddLearnersRequest;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.AddPeerRequest;
@@ -41,13 +42,11 @@ import static
org.apache.ignite.raft.jraft.rpc.CliRequests.SnapshotRequest;
import static
org.apache.ignite.raft.jraft.rpc.CliRequests.TransferLeaderRequest;
import java.io.IOException;
-import java.net.ConnectException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
@@ -598,6 +597,8 @@ public class RaftGroupServiceImpl implements
RaftGroupService {
long stopTime,
CompletableFuture<? extends NetworkMessage> fut
) {
+ err = unwrapCause(err);
+
if (recoverable(err)) {
Peer randomPeer = randomNode(peer);
@@ -707,17 +708,18 @@ public class RaftGroupServiceImpl implements
RaftGroupService {
}
/**
- * Checks if an error is recoverable, for example, {@link
java.net.ConnectException}.
+ * Checks if an error is recoverable.
+ *
+ * <p>An error is considered recoverable if it's an instance of {@link
TimeoutException}, {@link IOException}
+ * or {@link PeerUnavailableException}.
*
* @param t The throwable.
* @return {@code True} if this is a recoverable exception.
*/
private static boolean recoverable(Throwable t) {
- if (t instanceof ExecutionException || t instanceof
CompletionException) {
- t = t.getCause();
- }
+ t = unwrapCause(t);
- return t instanceof TimeoutException || t instanceof IOException;
+ return t instanceof TimeoutException || t instanceof IOException || t
instanceof PeerUnavailableException;
}
private Peer randomNode() {
@@ -811,7 +813,7 @@ public class RaftGroupServiceImpl implements
RaftGroupService {
ClusterNode node =
cluster.topologyService().getByConsistentId(peer.consistentId());
if (node == null) {
- return CompletableFuture.failedFuture(new ConnectException("Peer "
+ peer.consistentId() + " is unavailable"));
+ return CompletableFuture.failedFuture(new
PeerUnavailableException(peer.consistentId()));
}
return CompletableFuture.completedFuture(node);
diff --git
a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/AbstractClientService.java
b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/AbstractClientService.java
index 9c342dfc92..07f4d4e1ef 100644
---
a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/AbstractClientService.java
+++
b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/AbstractClientService.java
@@ -26,6 +26,7 @@ import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
import org.apache.ignite.internal.logger.IgniteLogger;
import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.PeerUnavailableException;
import org.apache.ignite.network.ClusterNode;
import org.apache.ignite.internal.network.TopologyEventHandler;
import org.apache.ignite.raft.jraft.Status;
@@ -59,7 +60,7 @@ public abstract class AbstractClientService implements
ClientService, TopologyEv
/**
* The set of pinged consistent IDs.
*/
- protected Set<String> readyConsistentIds = new ConcurrentHashSet<>();
+ private final Set<String> readyConsistentIds = new ConcurrentHashSet<>();
public RpcClient getRpcClient() {
return this.rpcClient;
@@ -224,7 +225,7 @@ public abstract class AbstractClientService implements
ClientService, TopologyEv
}
}
else {
- if (ThrowUtil.hasCause(err, null,
ConnectException.class))
+ if (ThrowUtil.hasCause(err, null,
PeerUnavailableException.class, ConnectException.class))
readyConsistentIds.remove(peerId.getConsistentId()); // Force logical reconnect.
if (done != null) {
diff --git
a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/IgniteRpcClient.java
b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/IgniteRpcClient.java
index d6a8da02d8..9ef1e6ba0d 100644
---
a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/IgniteRpcClient.java
+++
b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/IgniteRpcClient.java
@@ -27,6 +27,7 @@ import java.util.concurrent.TimeoutException;
import java.util.function.BiPredicate;
import org.apache.ignite.internal.logger.IgniteLogger;
import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.PeerUnavailableException;
import org.apache.ignite.internal.tostring.S;
import org.apache.ignite.network.ClusterNode;
import org.apache.ignite.internal.network.ClusterService;
@@ -135,8 +136,8 @@ public class IgniteRpcClient implements RpcClientEx {
ClusterNode targetNode =
service.topologyService().getByConsistentId(peerId.getConsistentId());
if (targetNode == null) {
- // ConnectException will force a retry by the enclosing components.
- fut.completeExceptionally(new ConnectException());
+ // PeerUnavailableException will force a retry by the enclosing
components.
+ fut.completeExceptionally(new
PeerUnavailableException(peerId.getConsistentId()));
return;
}