santosh-d3vpl3x commented on code in PR #18843: URL: https://github.com/apache/druid/pull/18843#discussion_r2901257856
########## extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDruidNodeDiscoveryProvider.java: ########## @@ -0,0 +1,444 @@ +/* + * 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.google.common.base.Preconditions; +import com.google.inject.Inject; +import org.apache.druid.concurrent.LifecycleLock; +import org.apache.druid.discovery.BaseNodeRoleWatcher; +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscovery; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +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 javax.annotation.Nullable; +import java.net.SocketTimeoutException; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BooleanSupplier; + +/** + * Consul-based implementation of {@link DruidNodeDiscoveryProvider}. + * + * <p>Each {@link NodeRoleWatcher} performs synchronous (potentially blocking) Consul queries on its own single-thread + * executor. Listener callbacks are dispatched via a shared single-thread executor to preserve callback ordering. + * + * <p>Consul queries can block up to {@code watchSeconds}; avoid invoking lifecycle methods from time-critical threads. + */ +@ManageLifecycle +public class ConsulDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider +{ + private static final Logger LOGGER = new Logger(ConsulDruidNodeDiscoveryProvider.class); + + private final ConsulApiClient consulApiClient; + private final ConsulDiscoveryConfig config; + + @Inject(optional = true) + @Nullable + private ServiceEmitter emitter; + + private ScheduledExecutorService listenerExecutor; + + private final ConcurrentHashMap<NodeRole, NodeRoleWatcher> nodeRoleWatchers = new ConcurrentHashMap<>(); + + private final LifecycleLock lifecycleLock = new LifecycleLock(); + + @Inject + public ConsulDruidNodeDiscoveryProvider( + ConsulApiClient consulApiClient, + ConsulDiscoveryConfig config + ) + { + this.consulApiClient = Preconditions.checkNotNull(consulApiClient, "consulApiClient"); + this.config = Preconditions.checkNotNull(config, "config"); + } + + @Override + public BooleanSupplier getForNode(DruidNode node, NodeRole nodeRole) + { + return () -> { + try { + List<DiscoveryDruidNode> nodes = consulApiClient.getHealthyServices(nodeRole); + return nodes.stream() + .anyMatch(n -> n.getDruidNode().getHostAndPortToUse().equals(node.getHostAndPortToUse())); + } + catch (Exception e) { + LOGGER.error(e, "Error checking for node [%s] with role [%s]", node, nodeRole); + return false; + } + }; + } + + @Override + public DruidNodeDiscovery getForNodeRole(NodeRole nodeRole) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + + return nodeRoleWatchers.computeIfAbsent( + nodeRole, + role -> { + LOGGER.info("Creating NodeRoleWatcher for role[%s].", role); + NodeRoleWatcher watcher = new NodeRoleWatcher( + listenerExecutor, + role, + consulApiClient, + config, + emitter + ); + watcher.start(); + LOGGER.info("Created NodeRoleWatcher for role[%s].", role); + return watcher; + } + ); + } + + @LifecycleStart + public void start() + { + if (!lifecycleLock.canStart()) { + throw new ISE("can't start."); + } + + try { + LOGGER.info("Starting ConsulDruidNodeDiscoveryProvider"); + + // Single-threaded executor ensures listener callbacks execute in-order, preventing race conditions + listenerExecutor = Execs.scheduledSingleThreaded("ConsulDruidNodeDiscoveryProvider-ListenerExecutor"); + + LOGGER.info("Started ConsulDruidNodeDiscoveryProvider"); + + lifecycleLock.started(); + } + finally { + lifecycleLock.exitStart(); + } + } + + @LifecycleStop + public void stop() + { + if (!lifecycleLock.canStop()) { + throw new ISE("can't stop."); + } + + LOGGER.info("Stopping ConsulDruidNodeDiscoveryProvider"); + + for (NodeRoleWatcher watcher : nodeRoleWatchers.values()) { + watcher.stop(); + } + nodeRoleWatchers.clear(); + + // Watcher threads must finish before shutting down listener executor to avoid RejectedExecutionException + try { + listenerExecutor.shutdown(); + if (!listenerExecutor.awaitTermination(10, TimeUnit.SECONDS)) { + LOGGER.warn("Listener executor did not terminate in time"); + listenerExecutor.shutdownNow(); + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.warn("Interrupted while waiting for listener executor termination"); + listenerExecutor.shutdownNow(); + } + + LOGGER.info("Stopped ConsulDruidNodeDiscoveryProvider"); + lifecycleLock.exitStopAndReset(); + } + + static class NodeRoleWatcher implements DruidNodeDiscovery + { + private static final Logger LOGGER = new Logger(NodeRoleWatcher.class); + + private final ConsulApiClient consulApiClient; + private final ConsulDiscoveryConfig config; + @Nullable + private final ServiceEmitter emitter; + + private ExecutorService watchExecutor; + + private final LifecycleLock lifecycleLock = new LifecycleLock(); + + private final NodeRole nodeRole; + private final BaseNodeRoleWatcher baseNodeRoleWatcher; + + private final AtomicLong retryCount = new AtomicLong(0); + + /** + * Creates a watcher for a single {@link NodeRole}. Consul calls are performed on the watch executor. Listener + * callbacks are dispatched via {@code listenerExecutor}. + */ + NodeRoleWatcher( + ScheduledExecutorService listenerExecutor, + NodeRole nodeRole, + ConsulApiClient consulApiClient, + ConsulDiscoveryConfig config, + @Nullable ServiceEmitter emitter + ) + { + this.nodeRole = nodeRole; + this.consulApiClient = consulApiClient; + this.config = config; + this.emitter = emitter; + this.baseNodeRoleWatcher = BaseNodeRoleWatcher.create(listenerExecutor, nodeRole); + } + + private void watch() + { + boolean cacheInitialized = false; + long consulIndex = 0; + + if (!lifecycleLock.awaitStarted()) { + LOGGER.error("Lifecycle not started, Exited Watch for role[%s].", nodeRole); + return; + } + + while (lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) { + try { + if (!cacheInitialized) { + List<DiscoveryDruidNode> nodes = consulApiClient.getHealthyServices(nodeRole); + Map<String, DiscoveryDruidNode> nodeMap = new HashMap<>(); + for (DiscoveryDruidNode node : nodes) { + nodeMap.put(node.getDruidNode().getHostAndPortToUse(), node); + } + baseNodeRoleWatcher.resetNodes(nodeMap); + baseNodeRoleWatcher.cacheInitialized(); + cacheInitialized = true; + + LOGGER.info("Cache initialized for role[%s] with [%d] nodes", nodeRole, nodes.size()); + } + + long watchStart = System.nanoTime(); + + long watchSeconds = config.getWatch().getWatchSeconds().getStandardSeconds(); + ConsulApiClient.ConsulWatchResult watchResult = consulApiClient.watchServices( + nodeRole, + consulIndex, + watchSeconds + ); Review Comment: Handled on the config side by rejecting `watchSeconds` values below one second. That prevents `getStandardSeconds()` from truncating sub-second durations to `0` and avoids accidental tight watch loops. -- 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]
