This is an automated email from the ASF dual-hosted git repository.

sk0x50 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 0ce80f0cec2 IGNITE-27725 Add additional logs to Placement Driver 
(#7533)
0ce80f0cec2 is described below

commit 0ce80f0cec226d8f31f32a316e8e2edd7f7255cd
Author: Mirza Aliev <[email protected]>
AuthorDate: Fri Feb 6 22:20:42 2026 +0400

    IGNITE-27725 Add additional logs to Placement Driver (#7533)
---
 .../internal/placementdriver/LeaseUpdater.java     |  4 ++--
 .../placementdriver/PlacementDriverManager.java    | 11 +++++++++
 .../placementdriver/leases/LeaseTracker.java       |  6 ++++-
 .../raft/client/TopologyAwareRaftGroupService.java | 26 ++++++++++++++++++----
 4 files changed, 40 insertions(+), 7 deletions(-)

diff --git 
a/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/LeaseUpdater.java
 
b/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/LeaseUpdater.java
index 6f0b9cb517e..a06a09e7810 100644
--- 
a/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/LeaseUpdater.java
+++ 
b/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/LeaseUpdater.java
@@ -198,7 +198,7 @@ public class LeaseUpdater {
                 return;
             }
 
-            LOG.info("Placement driver active actor is starting.");
+            LOG.info("Placement driver active actor is starting 
[nodeName={}].", nodeName);
 
             leaseNegotiator = new LeaseNegotiator(clusterService, 
throttledLogExecutor);
 
@@ -223,7 +223,7 @@ public class LeaseUpdater {
                 return;
             }
 
-            LOG.info("Placement driver active actor is stopping.");
+            LOG.info("Placement driver active actor is stopping 
[nodeName={}].", nodeName);
 
             leaseNegotiator = null;
 
diff --git 
a/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/PlacementDriverManager.java
 
b/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/PlacementDriverManager.java
index b290ca7c1f4..8d68a31baf4 100644
--- 
a/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/PlacementDriverManager.java
+++ 
b/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/PlacementDriverManager.java
@@ -39,6 +39,8 @@ import org.apache.ignite.internal.hlc.ClockService;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
 import org.apache.ignite.internal.lang.ByteArray;
 import org.apache.ignite.internal.lang.NodeStoppingException;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.manager.ComponentContext;
 import org.apache.ignite.internal.manager.IgniteComponent;
 import org.apache.ignite.internal.metastorage.MetaStorageManager;
@@ -67,6 +69,9 @@ import org.jetbrains.annotations.TestOnly;
  * The another role of the manager is providing a node, which is leaseholder 
at the moment, for a particular replication group.
  */
 public class PlacementDriverManager implements IgniteComponent {
+    /** Ignite logger. */
+    private static final IgniteLogger LOG = 
Loggers.forClass(PlacementDriverManager.class);
+
     private static final String PLACEMENTDRIVER_LEASES_KEY_STRING = 
"placementdriver.leases";
 
     public static final ByteArray PLACEMENTDRIVER_LEASES_KEY = 
ByteArray.fromString(PLACEMENTDRIVER_LEASES_KEY_STRING);
@@ -209,6 +214,8 @@ public class PlacementDriverManager implements 
IgniteComponent {
                                     true
                             );
 
+                            LOG.info("Placement driver raft client has started 
[group={}, node={}].", replicationGroupId, thisNodeName);
+
                             return 
raftClient.subscribeLeader(this::onLeaderChange).thenApply(v -> raftClient);
                         } catch (NodeStoppingException e) {
                             return failedFuture(e);
@@ -277,6 +284,10 @@ public class PlacementDriverManager implements 
IgniteComponent {
 
     private void onLeaderChange(InternalClusterNode leader, long term) {
         inBusyLock(busyLock, () -> {
+            String thisNodeName = 
clusterService.topologyService().localMember().name();
+
+            LOG.info("Placement driver received leader changed event 
[leader={}, term={}, nodeName={}]", leader, term, thisNodeName);
+
             if (leader.equals(clusterService.topologyService().localMember())) 
{
                 takeOverActiveActorBusy();
             } else {
diff --git 
a/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/leases/LeaseTracker.java
 
b/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/leases/LeaseTracker.java
index 4cd05b51f12..e112fa6b896 100644
--- 
a/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/leases/LeaseTracker.java
+++ 
b/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/leases/LeaseTracker.java
@@ -135,6 +135,8 @@ public class LeaseTracker extends 
AbstractEventProducer<PrimaryReplicaEvent, Pri
      */
     public void startTrack(long recoveryRevision) {
         inBusyLock(busyLock, () -> {
+            LOG.info("Starting lease tracker recovery [revision={}].", 
recoveryRevision);
+
             msManager.registerExactWatch(PLACEMENTDRIVER_LEASES_KEY, 
updateListener);
 
             loadLeasesBusyAsync(recoveryRevision);
@@ -149,6 +151,8 @@ public class LeaseTracker extends 
AbstractEventProducer<PrimaryReplicaEvent, Pri
 
         busyLock.block();
 
+        LOG.info("Stopping lease tracker.");
+
         
primaryReplicaWaiters.values().forEach(PendingComparableValuesTracker::close);
         primaryReplicaWaiters.clear();
 
@@ -456,7 +460,7 @@ public class LeaseTracker extends 
AbstractEventProducer<PrimaryReplicaEvent, Pri
             leases = new Leases(leasesMap, leasesBytes);
         }
 
-        LOG.info("Leases cache recovered [leases={}]", leases);
+        LOG.info("Leases cache recovered [revision={}, leases={}]", 
recoveryRevision, leases);
     }
 
     /**
diff --git 
a/modules/replicator/src/main/java/org/apache/ignite/internal/raft/client/TopologyAwareRaftGroupService.java
 
b/modules/replicator/src/main/java/org/apache/ignite/internal/raft/client/TopologyAwareRaftGroupService.java
index 7351333554b..6d291d438a4 100644
--- 
a/modules/replicator/src/main/java/org/apache/ignite/internal/raft/client/TopologyAwareRaftGroupService.java
+++ 
b/modules/replicator/src/main/java/org/apache/ignite/internal/raft/client/TopologyAwareRaftGroupService.java
@@ -148,6 +148,12 @@ public class TopologyAwareRaftGroupService implements 
RaftGroupService {
         this.eventsClientListener.addLeaderElectionListener(groupId(), 
serverEventHandler);
 
         topologyEventsListener = new LogicalTopologyEventListener() {
+            @Override
+            public void onTopologyLeap(LogicalTopologySnapshot newTopology) {
+                LOG.info("Topology leap event received [grpId={}, 
topology={}].", groupId(), newTopology);
+                // TODO: IGNITE-27770 - Need to re-establish subscriptions to 
all peers after topology leap
+            }
+
             @Override
             public void onNodeJoined(LogicalNode appearedNode, 
LogicalTopologySnapshot newTopology) {
                 Peer peer = new Peer(appearedNode.name(), 0);
@@ -162,6 +168,12 @@ public class TopologyAwareRaftGroupService implements 
RaftGroupService {
                     subscribeToNode(appearedNode, peer, leftWhileSubscribing)
                             .thenComposeAsync(subscribed -> {
                                 if (subscribed) {
+                                    LOG.info("Successfully subscribed to new 
peer for leader election notifications "
+                                            + "[grpId={}, consistentId={}]",
+                                            groupId(),
+                                            peer.consistentId()
+                                    );
+
                                     return refreshAndGetLeaderWithTerm()
                                             .thenAcceptAsync(leaderWithTerm -> 
{
                                                 if (!leaderWithTerm.isEmpty()
@@ -314,14 +326,17 @@ public class TopologyAwareRaftGroupService implements 
RaftGroupService {
                 sendWithRetry(node, msg, msgSendFut, leftWhileCalling);
             } else if (invokeCause instanceof RecipientLeftException) {
                 LOG.info(
-                        "Could not subscribe to leader update from a specific 
node, because the node had left the cluster: [node={}]",
-                        node
+                        "Could not subscribe to leader update from a specific 
node, "
+                                + "because the node had left the cluster: 
[node={}, grpId={}].",
+                        node,
+                        groupId()
                 );
 
                 msgSendFut.complete(false);
             } else {
                 if (!(invokeCause instanceof NodeStoppingException)) {
-                    LOG.error("Could not send the subscribe message to the 
node: [node={}, msg={}]", invokeThrowable, node, msg);
+                    LOG.error("Could not send the subscribe message to the 
node: [node={}, grpId={}, msg={}]",
+                            invokeThrowable, node, groupId(), msg);
                 }
 
                 msgSendFut.completeExceptionally(invokeThrowable);
@@ -394,7 +409,10 @@ public class TopologyAwareRaftGroupService implements 
RaftGroupService {
                                     leaderWithTerm.term()
                             );
                         } else {
-                            LOG.warn("Leader host occurred to leave the 
topology [nodeId = {}].", leaderWithTerm.leader().consistentId());
+                            LOG.warn("Leader host occurred to leave the 
topology [nodeId={}, grpId={}].",
+                                    leaderWithTerm.leader().consistentId(),
+                                    groupId()
+                            );
                         }
                     }
                 }, executor);

Reply via email to