santosh-d3vpl3x commented on code in PR #18843:
URL: https://github.com/apache/druid/pull/18843#discussion_r2901257012


##########
extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulLeaderSelector.java:
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.druid.consul.discovery;
+
+import com.ecwid.consul.v1.ConsulClient;
+import com.ecwid.consul.v1.QueryParams;
+import com.ecwid.consul.v1.Response;
+import com.ecwid.consul.v1.kv.model.GetValue;
+import com.ecwid.consul.v1.kv.model.PutParams;
+import com.ecwid.consul.v1.session.model.NewSession;
+import com.ecwid.consul.v1.session.model.Session;
+import com.google.common.base.Preconditions;
+import com.google.inject.Inject;
+import org.apache.druid.concurrent.LifecycleLock;
+import org.apache.druid.discovery.DruidLeaderSelector;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.server.DruidNode;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Consul-based implementation of {@link DruidLeaderSelector} using Consul 
sessions and KV locks.
+ *
+ * <p>{@link #registerListener(Listener)} starts background executors. Leader 
election runs on one single-thread
+ * executor and invokes {@link Listener} callbacks; session renewal runs on 
another single-thread executor.
+ * {@link #unregisterListener()} may invoke {@link Listener#stopBeingLeader()} 
on the calling thread.
+ *
+ * <p>Consul RPCs and retry backoffs are performed on the background threads; 
avoid invoking lifecycle methods from
+ * time-sensitive threads.
+ */
+public class ConsulLeaderSelector implements DruidLeaderSelector
+{
+  private static final Logger LOGGER = new Logger(ConsulLeaderSelector.class);
+
+  private final LifecycleLock lifecycleLock = new LifecycleLock();
+  private final DruidNode self;
+  private final String lockKey;
+  private final ConsulDiscoveryConfig config;
+  private final ConsulClient consulClient;
+  @Inject(optional = true)
+  @Nullable
+  private ServiceEmitter emitter = null;
+
+  private volatile DruidLeaderSelector.Listener listener = null;
+  private final AtomicBoolean leader = new AtomicBoolean(false);
+  private final AtomicInteger term = new AtomicInteger(0);
+
+  private ScheduledExecutorService executorService;
+  private ScheduledExecutorService sessionKeeperService;
+  private volatile String sessionId;
+  private volatile boolean stopping = false;
+  private long errorRetryCount = 0;
+
+  public ConsulLeaderSelector(
+      DruidNode self,
+      String lockKey,
+      ConsulDiscoveryConfig config,
+      ConsulClient consulClient
+  )
+  {
+    this.self = Preconditions.checkNotNull(self, "self");
+    this.lockKey = Preconditions.checkNotNull(lockKey, "lockKey");
+    this.config = Preconditions.checkNotNull(config, "config");
+    this.consulClient = Preconditions.checkNotNull(consulClient, 
"consulClient");
+
+    if (config.getLeader().getLeaderSessionTtl().getStandardSeconds() > 120) {
+      LOGGER.warn("leaderSessionTtl is %s; leader failover may take up to %s",
+                  config.getLeader().getLeaderSessionTtl(),
+                  
Duration.standardSeconds(config.getLeader().getLeaderSessionTtl().getStandardSeconds()
 * 2));
+    }
+  }
+
+  @Nullable
+  @Override
+  public String getCurrentLeader()
+  {
+    try {
+      Response<GetValue> response = consulClient.getKVValue(
+          lockKey,
+          config.getAuth().getAclToken(),
+          buildQueryParams()
+      );
+      if (response != null && response.getValue() != null && 
response.getValue().getValue() != null) {
+        return new 
String(Base64.getDecoder().decode(response.getValue().getValue()), 
StandardCharsets.UTF_8);
+      }
+      return null;
+    }
+    catch (Exception e) {
+      LOGGER.error(e, "Failed to get current leader from Consul");
+      return null;
+    }
+  }
+
+  @Override
+  public boolean isLeader()
+  {
+    return leader.get();
+  }
+
+  @Override
+  public int localTerm()
+  {
+    return term.get();
+  }
+
+  @Override
+  public void registerListener(Listener listener)
+  {
+    Preconditions.checkArgument(listener != null, "listener is null");
+
+    if (!lifecycleLock.canStart()) {
+      throw new ISE("can't start");
+    }
+
+    try {
+      this.listener = listener;
+      this.executorService = 
Execs.scheduledSingleThreaded("ConsulLeaderSelector-%d");
+      this.sessionKeeperService = 
Execs.scheduledSingleThreaded("ConsulSessionKeeper-%d");
+
+      startLeaderElection();
+
+      lifecycleLock.started();
+    }
+    catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+    finally {
+      lifecycleLock.exitStart();
+    }
+  }
+
+  @Override
+  public void unregisterListener()
+  {
+    if (!lifecycleLock.canStop()) {
+      throw new ISE("can't stop");
+    }
+
+    LOGGER.info("Unregistering leader selector for [%s]", lockKey);
+    stopping = true;
+
+    try {
+      if (leader.get()) {
+        try {
+          listener.stopBeingLeader();
+        }
+        catch (Exception e) {
+          LOGGER.error(e, "Exception while stopping being leader");
+        }
+        leader.set(false);
+      }
+
+      // Destroying session releases the Consul lock, allowing another node to 
become leader
+      if (sessionId != null) {
+        try {
+          consulClient.sessionDestroy(sessionId, buildQueryParams(), 
config.getAuth().getAclToken());
+        }
+        catch (Exception e) {
+          LOGGER.error(e, "Failed to destroy Consul session");
+        }
+        sessionId = null;
+      }
+
+      if (executorService != null) {
+        executorService.shutdownNow();
+        try {
+          if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
+            LOGGER.warn("Leader selector executor did not terminate in time");
+          }
+        }
+        catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+        }
+      }
+
+      if (sessionKeeperService != null) {
+        sessionKeeperService.shutdownNow();
+        try {
+          if (!sessionKeeperService.awaitTermination(5, TimeUnit.SECONDS)) {
+            LOGGER.warn("Session keeper service did not terminate in time");
+          }
+        }
+        catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+        }
+      }
+    }
+    finally {
+      lifecycleLock.exitStop();
+    }
+  }
+
+  private void startLeaderElection()
+  {
+    executorService.submit(this::leaderElectionLoop);
+    sessionKeeperService.submit(this::sessionKeeperLoop);
+  }
+
+  private void leaderElectionLoop()
+  {
+    LOGGER.info("Starting leader election loop for [%s]", lockKey);
+    ConsulMetrics.emitCount(emitter, "consul/leader/loop", "lock", lockKey, 
"state", "start");
+
+    while (!stopping && !Thread.currentThread().isInterrupted()) {
+      try {
+        if (sessionId == null) {
+          sessionId = createSession();
+        }
+
+        if (sessionId != null && !leader.get()) {
+          if (!isSessionValid(sessionId)) {
+            LOGGER.info("Follower session [%s] expired or invalid, 
recreating", shortSessionId(sessionId));
+            sessionId = null;
+            continue;
+          }
+        }
+

Review Comment:
   Fixed by guarding the session-creation path before lock acquisition. If 
`createSession()` returns `null`, the leader loop now logs, backs off for the 
health-check interval, and retries instead of passing a null session into the 
lock flow.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to