You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "denis-chudov (via GitHub)" <gi...@apache.org> on 2023/04/24 18:32:32 UTC

[GitHub] [ignite-3] denis-chudov commented on a diff in pull request #1959: IGNITE-18961 Implement lease prolongation on placement driver side

denis-chudov commented on code in PR #1959:
URL: https://github.com/apache/ignite-3/pull/1959#discussion_r1175354765


##########
modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/PlacementDriverManager.java:
##########
@@ -220,10 +222,15 @@ private void onLeaderChange(ClusterNode leader, Long term) {
         }
     }
 
+    /** The logger. */
+    private static IgniteLogger LOG = Loggers.forClass(PlacementDriverManager.class);
+
     /**
      * Takes over active actor of placement driver group.
      */
     private void takeOverActiveActor() {
+        LOG.info("Active actor is starting.");

Review Comment:
   ```suggestion
           LOG.info("Placement driver active actor is starting.");
   ```



##########
modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/LeaseUpdater.java:
##########
@@ -238,7 +238,10 @@ public void run() {
 
                     // The lease is expired or close to this.
                     if (lease.getExpirationTime().getPhysical() < outdatedLeaseThreshold) {
-                        ClusterNode candidate = nextLeaseHolder(entry.getValue(), null);
+                        ClusterNode candidate = nextLeaseHolder(
+                                entry.getValue(),
+                                lease.isAccepted() ? lease.getLeaseholder().name() : null
+                        );

Review Comment:
   Why do we still select next lease holder every time? I thought we are going to change this in this ticket, this shouldn't be done for prolongation.
   
   Also, the condition for meta storage invoke on prolongation is not sufficient. Invoke on prolongation should be successfull only when 
   keys are equal 
   AND 
   leaseholder is the same
   AND
   lease is accepted
   AND 
   lease start and expiration times are the same.
   
   Shouldn't it?
   
   Also, invoke condition on lease granting should be revisited accordingly.



##########
modules/placement-driver/src/integrationTest/java/org/apache/ignite/internal/placementdriver/MultiActorPlacementDriverTest.java:
##########
@@ -0,0 +1,518 @@
+/*
+ * 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.placementdriver;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_ZONE_ID;
+import static org.apache.ignite.internal.placementdriver.PlacementDriverManager.PLACEMENTDRIVER_PREFIX;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.apache.ignite.lang.ByteArray.fromString;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.ignite.internal.affinity.AffinityUtils;
+import org.apache.ignite.internal.affinity.Assignment;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.raft.MetastorageGroupId;
+import org.apache.ignite.internal.placementdriver.PlacementDriverManagerTest.LogicalTopologyServiceTestImpl;
+import org.apache.ignite.internal.placementdriver.leases.Lease;
+import org.apache.ignite.internal.placementdriver.message.LeaseGrantedMessage;
+import org.apache.ignite.internal.placementdriver.message.LeaseGrantedMessageResponse;
+import org.apache.ignite.internal.placementdriver.message.PlacementDriverMessageGroup;
+import org.apache.ignite.internal.placementdriver.message.PlacementDriverMessagesFactory;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.raft.client.TopologyAwareRaftGroupServiceFactory;
+import org.apache.ignite.internal.raft.configuration.RaftConfiguration;
+import org.apache.ignite.internal.schema.configuration.ExtendedTableChange;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.table.distributed.replicator.TablePartitionId;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.lang.IgniteTriFunction;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.network.StaticNodeFinder;
+import org.apache.ignite.raft.jraft.rpc.impl.RaftGroupEventsClientListener;
+import org.apache.ignite.utils.ClusterServiceTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * There are tests of muti-nodes for placement driver.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public class MultiActorPlacementDriverTest extends IgniteAbstractTest {

Review Comment:
   We definitely should have tests for two conflicting placement driver active actors. For example, there should be test for prolongation scenario. Or are we going to cover this under https://issues.apache.org/jira/browse/IGNITE-19325 ?
   If so, the scenarios should be mentioned there, but I would say that we could cover more scenarios under regular tickets to reduce the amount of work required for IGNITE-19325.



##########
modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/PlacementDriverManager.java:
##########
@@ -234,6 +241,8 @@ private void takeOverActiveActor() {
      * Steps down as active actor.
      */
     private void stepDownActiveActor() {
+        LOG.info("Active actor is stopping.");

Review Comment:
   ```suggestion
           LOG.info("Placement driver active actor is stopping.");
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org