getNodes()
+ {
+ return nodes;
+ }
+
+ public long getConsulIndex()
+ {
+ return consulIndex;
+ }
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulClients.java b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulClients.java
new file mode 100644
index 000000000000..eb32637b7168
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulClients.java
@@ -0,0 +1,202 @@
+/*
+ * 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.ConsulRawClient;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.security.TLSUtils;
+import org.apache.http.HttpRequestInterceptor;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.config.Registry;
+import org.apache.http.config.RegistryBuilder;
+import org.apache.http.conn.socket.ConnectionSocketFactory;
+import org.apache.http.conn.ssl.NoopHostnameVerifier;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.SSLContext;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Helper for constructing a ConsulClient with TLS support.
+ *
+ * This class properly configures HTTPS transport for the Ecwid Consul client
+ * using Druid's standard TLS infrastructure.
+ */
+final class ConsulClients
+{
+ private static final Logger LOGGER = new Logger(ConsulClients.class);
+
+ private ConsulClients()
+ {
+ }
+
+ static ConsulClient create(ConsulDiscoveryConfig config)
+ {
+ ConsulDiscoveryConfig.ConnectionConfig connection = config.getConnection();
+ ConsulDiscoveryConfig.AuthConfig auth = config.getAuth();
+
+ ConsulSSLConfig sslConfig = connection.getSslClientConfig();
+ String basicUser = auth.getBasicAuthUser();
+ String basicPass = auth.getBasicAuthPassword();
+ boolean tlsConfigured = sslConfig != null && sslConfig.getTrustStorePath() != null;
+
+ // Validate basic auth over HTTP security requirements
+ if (basicUser != null && basicPass != null && !tlsConfigured) {
+ if (!auth.getAllowBasicAuthOverHttp()) {
+ throw new IllegalStateException(
+ "Basic authentication credentials are configured but TLS is not enabled. " +
+ "This would transmit credentials in cleartext over the network. " +
+ "Either configure TLS (connection.sslClientConfig.trustStorePath) or explicitly allow " +
+ "insecure transmission by setting auth.allowBasicAuthOverHttp=true " +
+ "(only use this for sidecar TLS termination scenarios)."
+ );
+ }
+ LOGGER.warn(
+ "Using Basic Auth to Consul over plain HTTP (host: %s, port: %d) with allowBasicAuthOverHttp=true. " +
+ "Credentials will be transmitted in cleartext. " +
+ "Only use this configuration with sidecar TLS termination or in secure network environments.",
+ connection.getHost(),
+ connection.getPort()
+ );
+ }
+
+ if (tlsConfigured) {
+ try {
+ SSLContext sslContext = buildSslContext(sslConfig);
+ HttpClient httpClient = createHttpClientWithOptionalBasicAuth(sslContext, basicUser, basicPass, connection, sslConfig);
+
+ String httpsHost = "https://" + connection.getHost();
+
+ ConsulRawClient rawClient = new ConsulRawClient(httpsHost, connection.getPort(), httpClient);
+ LOGGER.info("Created Consul client with HTTPS to %s:%d", connection.getHost(), connection.getPort());
+ return new ConsulClient(rawClient);
+ }
+ catch (Exception e) {
+ // TLS was explicitly configured; fail fast rather than silently downgrade to HTTP
+ LOGGER.error(e, "Failed to configure TLS for Consul client (host: %s, port: %d)", connection.getHost(), connection.getPort());
+ throw new IllegalStateException("Consul TLS configuration failed; refusing to fall back to HTTP", e);
+ }
+ }
+
+ // No TLS configured - use plain HTTP
+ HttpClient httpClient = createHttpClientWithOptionalBasicAuth(null, basicUser, basicPass, connection, null);
+ String httpHost = "http://" + connection.getHost();
+ ConsulRawClient rawClient = new ConsulRawClient(httpHost, connection.getPort(), httpClient);
+ LOGGER.info("Created Consul client with HTTP to %s:%d", connection.getHost(), connection.getPort());
+ return new ConsulClient(rawClient);
+ }
+
+ /**
+ * Build SSLContext from ConsulSSLConfig using Druid's standard TLS infrastructure.
+ */
+ private static SSLContext buildSslContext(ConsulSSLConfig config)
+ {
+ try {
+ return new TLSUtils.ClientSSLContextBuilder()
+ .setProtocol(config.getProtocol())
+ .setTrustStoreType(config.getTrustStoreType())
+ .setTrustStorePath(config.getTrustStorePath())
+ .setTrustStoreAlgorithm(config.getTrustStoreAlgorithm())
+ .setTrustStorePasswordProvider(config.getTrustStorePasswordProvider())
+ .setKeyStoreType(config.getKeyStoreType())
+ .setKeyStorePath(config.getKeyStorePath())
+ .setKeyStoreAlgorithm(config.getKeyManagerFactoryAlgorithm())
+ .setCertAlias(config.getCertAlias())
+ .setKeyStorePasswordProvider(config.getKeyStorePasswordProvider())
+ .setKeyManagerFactoryPasswordProvider(config.getKeyManagerPasswordProvider())
+ .setValidateHostnames(config.getValidateHostnames())
+ .build();
+ }
+ catch (Exception e) {
+ LOGGER.error(e, "Failed to build SSLContext from ConsulSSLConfig");
+ throw new IllegalStateException("Failed to build SSLContext", e);
+ }
+ }
+
+ /**
+ * Create an HttpClient with the given SSLContext.
+ */
+ private static HttpClient createHttpClientWithOptionalBasicAuth(
+ SSLContext sslContext,
+ String basicUser,
+ String basicPass,
+ ConsulDiscoveryConfig.ConnectionConfig connection,
+ ConsulSSLConfig sslConfig
+ )
+ {
+ HttpClientBuilder httpBuilder = HttpClients.custom();
+
+ // Always use a PoolingHttpClientConnectionManager with proper pool sizing
+ // This prevents ConnectionPoolTimeoutException when multiple threads use the client concurrently
+ PoolingHttpClientConnectionManager connectionManager;
+
+ if (sslContext != null) {
+ // Configure hostname verification based on sslConfig.validateHostnames
+ HostnameVerifier hostnameVerifier = (sslConfig != null && Boolean.FALSE.equals(sslConfig.getValidateHostnames()))
+ ? NoopHostnameVerifier.INSTANCE
+ : SSLConnectionSocketFactory.getDefaultHostnameVerifier();
+
+ SSLConnectionSocketFactory sslSocketFactory = new SSLConnectionSocketFactory(sslContext, hostnameVerifier);
+ Registry registry = RegistryBuilder.create()
+ .register("https", sslSocketFactory)
+ .build();
+ connectionManager = new PoolingHttpClientConnectionManager(registry);
+ httpBuilder.setSSLContext(sslContext);
+ } else {
+ connectionManager = new PoolingHttpClientConnectionManager();
+ }
+
+ // Configure connection pool sizes (configurable for large clusters)
+ connectionManager.setMaxTotal(connection.getMaxTotalConnections());
+ connectionManager.setDefaultMaxPerRoute(connection.getMaxConnectionsPerRoute());
+
+ httpBuilder.setConnectionManager(connectionManager);
+
+ if (basicUser != null && basicPass != null) {
+ final String token = Base64.getEncoder().encodeToString((basicUser + ":" + basicPass).getBytes(StandardCharsets.UTF_8));
+ HttpRequestInterceptor authInjector = (request, context) -> {
+ if (!request.containsHeader("Authorization")) {
+ request.addHeader("Authorization", "Basic " + token);
+ }
+ };
+ httpBuilder.addInterceptorFirst(authInjector);
+ }
+
+ RequestConfig requestConfig = RequestConfig.custom()
+ .setConnectTimeout((int) connection.getConnectTimeout().getMillis())
+ .setSocketTimeout((int) connection.getSocketTimeout().getMillis())
+ .setConnectionRequestTimeout((int) TimeUnit.SECONDS.toMillis(10))
+ .build();
+ httpBuilder.setDefaultRequestConfig(requestConfig);
+
+ LOGGER.info("ConsulClient configured with connectTimeout=%s, socketTimeout=%s",
+ connection.getConnectTimeout(), connection.getSocketTimeout());
+
+ return httpBuilder.build();
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDiscoveryConfig.java b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDiscoveryConfig.java
new file mode 100644
index 000000000000..087045c320a4
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDiscoveryConfig.java
@@ -0,0 +1,770 @@
+/*
+ * 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.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Configuration for Consul-based service discovery.
+ */
+public class ConsulDiscoveryConfig
+{
+ private static final Logger LOGGER = new Logger(ConsulDiscoveryConfig.class);
+ private static final long MIN_LEADER_SESSION_TTL_SECONDS = 10;
+
+ @JsonProperty("connection")
+ private final ConnectionConfig connection;
+
+ @JsonProperty("auth")
+ private final AuthConfig auth;
+
+ @JsonProperty("service")
+ private final ServiceConfig service;
+
+ @JsonProperty("leader")
+ private final LeaderElectionConfig leader;
+
+ @JsonProperty("watch")
+ private final WatchConfig watch;
+
+ @JsonCreator
+ public static ConsulDiscoveryConfig create(
+ @JsonProperty("connection") @Nullable ConnectionConfig connection,
+ @JsonProperty("auth") @Nullable AuthConfig auth,
+ @JsonProperty("service") ServiceConfig service,
+ @JsonProperty("leader") @Nullable LeaderElectionConfig leader,
+ @JsonProperty("watch") @Nullable WatchConfig watch
+ )
+ {
+ if (service == null) {
+ throw new IAE("service cannot be null");
+ }
+
+ LeaderElectionConfig finalLeader = computeLeaderElectionConfig(leader, service.getHealthCheckInterval());
+ return new ConsulDiscoveryConfig(connection, auth, service, finalLeader, watch);
+ }
+
+ private static LeaderElectionConfig computeLeaderElectionConfig(
+ @Nullable LeaderElectionConfig leader,
+ Duration healthCheckInterval
+ )
+ {
+ if (leader != null) {
+ // Compute default TTL based on health check interval when not explicitly set
+ if (leader.getLeaderSessionTtl() == null) {
+ return new LeaderElectionConfig(
+ leader.getCoordinatorLeaderLockPath(),
+ leader.getOverlordLeaderLockPath(),
+ null,
+ leader.getLeaderMaxErrorRetries(),
+ leader.getLeaderRetryBackoffMax(),
+ healthCheckInterval
+ );
+ } else {
+ return leader;
+ }
+ } else {
+ return new LeaderElectionConfig(null, null, null, null, null, healthCheckInterval);
+ }
+ }
+
+ private ConsulDiscoveryConfig(
+ ConnectionConfig connection,
+ AuthConfig auth,
+ ServiceConfig service,
+ LeaderElectionConfig leader,
+ WatchConfig watch
+ )
+ {
+ this.connection = connection == null ? new ConnectionConfig(null, null, null, null, null, null, null) : connection;
+ this.auth = auth == null ? new AuthConfig(null, null, null, null) : auth;
+ this.service = service;
+ this.leader = leader;
+ this.watch = watch == null ? new WatchConfig(null, null, null, null) : watch;
+
+ validateCrossFieldConstraints();
+ }
+
+ private void validateCrossFieldConstraints()
+ {
+ // Socket timeout must exceed watch timeout to avoid premature disconnects
+ if (connection.getSocketTimeout().compareTo(watch.getWatchSeconds()) <= 0) {
+ throw new IAE(
+ StringUtils.format(
+ "socketTimeout [%s] must be greater than watchSeconds [%s]",
+ connection.getSocketTimeout(),
+ watch.getWatchSeconds()
+ )
+ );
+ }
+
+ long serviceTtlSeconds = Math.max(30, service.getHealthCheckInterval().getStandardSeconds() * 3);
+ if (service.getDeregisterAfter().getStandardSeconds() < serviceTtlSeconds) {
+ throw new IAE(
+ StringUtils.format(
+ "deregisterAfter (%ds) must be >= service TTL (%ds = 3 × healthCheckInterval)",
+ service.getDeregisterAfter().getStandardSeconds(),
+ serviceTtlSeconds
+ )
+ );
+ }
+
+ // Large watchSeconds relative to session TTL can delay failure detection
+ if (watch.getWatchSeconds().getStandardSeconds() > leader.getLeaderSessionTtl().getStandardSeconds() * 2) {
+ LOGGER.warn(
+ "watchSeconds (%ds) is much larger than leaderSessionTtl (%ds): delayed failure detection possible",
+ watch.getWatchSeconds().getStandardSeconds(),
+ leader.getLeaderSessionTtl().getStandardSeconds()
+ );
+ }
+ }
+
+ public ConnectionConfig getConnection()
+ {
+ return connection;
+ }
+
+ public AuthConfig getAuth()
+ {
+ return auth;
+ }
+
+ public ServiceConfig getService()
+ {
+ return service;
+ }
+
+ public LeaderElectionConfig getLeader()
+ {
+ return leader;
+ }
+
+ public WatchConfig getWatch()
+ {
+ return watch;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ ConsulDiscoveryConfig that = (ConsulDiscoveryConfig) o;
+ return Objects.equals(connection, that.connection) &&
+ Objects.equals(auth, that.auth) &&
+ Objects.equals(service, that.service) &&
+ Objects.equals(leader, that.leader) &&
+ Objects.equals(watch, that.watch);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(connection, auth, service, leader, watch);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "ConsulDiscoveryConfig{" +
+ "connection=" + connection +
+ ", auth=" + auth +
+ ", service=" + service +
+ ", leader=" + leader +
+ ", watch=" + watch +
+ '}';
+ }
+
+ public static class ConnectionConfig
+ {
+ private static final long DEFAULT_CONNECT_TIMEOUT_MS = 10_000;
+ private static final long DEFAULT_SOCKET_TIMEOUT_MS = 75_000;
+ private static final int DEFAULT_MAX_TOTAL_CONNECTIONS = 50;
+ private static final int DEFAULT_MAX_CONNECTIONS_PER_ROUTE = 20;
+
+ private final String host;
+ private final int port;
+ private final Duration connectTimeout;
+ private final Duration socketTimeout;
+ @Nullable
+ private final ConsulSSLConfig sslClientConfig;
+ private final int maxTotalConnections;
+ private final int maxConnectionsPerRoute;
+
+ @JsonCreator
+ public ConnectionConfig(
+ @JsonProperty("host") @Nullable String host,
+ @JsonProperty("port") @Nullable Integer port,
+ @JsonProperty("connectTimeout") @Nullable Duration connectTimeout,
+ @JsonProperty("socketTimeout") @Nullable Duration socketTimeout,
+ @JsonProperty("sslClientConfig") @Nullable ConsulSSLConfig sslClientConfig,
+ @JsonProperty("maxTotalConnections") @Nullable Integer maxTotalConnections,
+ @JsonProperty("maxConnectionsPerRoute") @Nullable Integer maxConnectionsPerRoute
+ )
+ {
+ this.host = host == null ? "localhost" : host;
+ this.port = validatePort(port);
+ this.connectTimeout = validatePositive(connectTimeout, DEFAULT_CONNECT_TIMEOUT_MS, "connectTimeout");
+ this.socketTimeout = validatePositive(socketTimeout, DEFAULT_SOCKET_TIMEOUT_MS, "socketTimeout");
+ this.sslClientConfig = sslClientConfig;
+ this.maxTotalConnections = validateConnectionPoolSize(
+ maxTotalConnections,
+ DEFAULT_MAX_TOTAL_CONNECTIONS,
+ "maxTotalConnections"
+ );
+ this.maxConnectionsPerRoute = validateConnectionPoolSize(
+ maxConnectionsPerRoute,
+ DEFAULT_MAX_CONNECTIONS_PER_ROUTE,
+ "maxConnectionsPerRoute"
+ );
+ }
+
+ private static int validatePort(Integer port)
+ {
+ int portValue = port == null ? 8500 : port;
+ if (portValue < 1 || portValue > 65535) {
+ throw new IllegalArgumentException("port must be between 1 and 65535");
+ }
+ return portValue;
+ }
+
+ private static int validateConnectionPoolSize(Integer value, int defaultValue, String name)
+ {
+ int result = value == null ? defaultValue : value;
+ if (result <= 0) {
+ throw new IAE(name + " must be positive");
+ }
+ return result;
+ }
+
+ @JsonProperty
+ public String getHost()
+ {
+ return host;
+ }
+
+ @JsonProperty
+ public int getPort()
+ {
+ return port;
+ }
+
+ @JsonProperty
+ public Duration getConnectTimeout()
+ {
+ return connectTimeout;
+ }
+
+ @JsonProperty
+ public Duration getSocketTimeout()
+ {
+ return socketTimeout;
+ }
+
+ @JsonProperty
+ @Nullable
+ public ConsulSSLConfig getSslClientConfig()
+ {
+ return sslClientConfig;
+ }
+
+ @JsonProperty
+ public int getMaxTotalConnections()
+ {
+ return maxTotalConnections;
+ }
+
+ @JsonProperty
+ public int getMaxConnectionsPerRoute()
+ {
+ return maxConnectionsPerRoute;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ ConnectionConfig that = (ConnectionConfig) o;
+ return port == that.getPort() &&
+ maxTotalConnections == that.getMaxTotalConnections() &&
+ maxConnectionsPerRoute == that.getMaxConnectionsPerRoute() &&
+ Objects.equals(host, that.getHost()) &&
+ Objects.equals(connectTimeout, that.getConnectTimeout()) &&
+ Objects.equals(socketTimeout, that.getSocketTimeout()) &&
+ Objects.equals(sslClientConfig, that.getSslClientConfig());
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(host, port, connectTimeout, socketTimeout, sslClientConfig, maxTotalConnections, maxConnectionsPerRoute);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "ConnectionConfig{host='" + host + "', port=" + port +
+ ", connectTimeout=" + connectTimeout + ", socketTimeout=" + socketTimeout +
+ ", maxTotalConnections=" + maxTotalConnections + ", maxConnectionsPerRoute=" + maxConnectionsPerRoute + '}';
+ }
+ }
+
+ public static class AuthConfig
+ {
+ @Nullable
+ private final String aclToken;
+ @Nullable
+ private final String basicAuthUser;
+ @Nullable
+ private final String basicAuthPassword;
+ private final boolean allowBasicAuthOverHttp;
+
+ @JsonCreator
+ public AuthConfig(
+ @JsonProperty("aclToken") @Nullable String aclToken,
+ @JsonProperty("basicAuthUser") @Nullable String basicAuthUser,
+ @JsonProperty("basicAuthPassword") @Nullable String basicAuthPassword,
+ @JsonProperty("allowBasicAuthOverHttp") @Nullable Boolean allowBasicAuthOverHttp
+ )
+ {
+ this.aclToken = aclToken;
+ this.basicAuthUser = basicAuthUser;
+ this.basicAuthPassword = basicAuthPassword;
+ this.allowBasicAuthOverHttp = allowBasicAuthOverHttp != null ? allowBasicAuthOverHttp : false;
+ }
+
+ @JsonProperty
+ @Nullable
+ public String getAclToken()
+ {
+ return aclToken;
+ }
+
+ @JsonProperty
+ @Nullable
+ public String getBasicAuthUser()
+ {
+ return basicAuthUser;
+ }
+
+ @JsonProperty
+ @Nullable
+ public String getBasicAuthPassword()
+ {
+ return basicAuthPassword;
+ }
+
+ @JsonProperty
+ public boolean getAllowBasicAuthOverHttp()
+ {
+ return allowBasicAuthOverHttp;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ AuthConfig that = (AuthConfig) o;
+ return allowBasicAuthOverHttp == that.getAllowBasicAuthOverHttp() &&
+ Objects.equals(aclToken, that.getAclToken()) &&
+ Objects.equals(basicAuthUser, that.getBasicAuthUser()) &&
+ Objects.equals(basicAuthPassword, that.getBasicAuthPassword());
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(aclToken, basicAuthUser, basicAuthPassword, allowBasicAuthOverHttp);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "AuthConfig{aclToken=" + mask(aclToken) +
+ ", basicAuthUser=" + mask(basicAuthUser) +
+ ", basicAuthPassword=" + mask(basicAuthPassword) +
+ ", allowBasicAuthOverHttp=" + allowBasicAuthOverHttp + '}';
+ }
+
+ private static String mask(String value)
+ {
+ if (value == null) {
+ return String.valueOf(value);
+ }
+ return "*****";
+ }
+ }
+
+ public static class ServiceConfig
+ {
+ private static final long DEFAULT_HEALTH_CHECK_INTERVAL_MS = 10_000;
+ private static final long DEFAULT_DEREGISTER_AFTER_MS = 90_000;
+
+ private final String servicePrefix;
+ @Nullable
+ private final String datacenter;
+ @Nullable
+ private final Map serviceTags;
+ private final Duration healthCheckInterval;
+ private final Duration deregisterAfter;
+
+ @JsonCreator
+ public ServiceConfig(
+ @JsonProperty("servicePrefix") String servicePrefix,
+ @JsonProperty("datacenter") @Nullable String datacenter,
+ @JsonProperty("serviceTags") @Nullable Map serviceTags,
+ @JsonProperty("healthCheckInterval") @Nullable Duration healthCheckInterval,
+ @JsonProperty("deregisterAfter") @Nullable Duration deregisterAfter
+ )
+ {
+ if (servicePrefix == null || servicePrefix.isEmpty()) {
+ throw new IAE("servicePrefix cannot be null or empty");
+ }
+ this.servicePrefix = servicePrefix;
+ this.datacenter = datacenter;
+ this.serviceTags = serviceTags == null
+ ? null
+ : Collections.unmodifiableMap(new LinkedHashMap<>(serviceTags));
+ this.healthCheckInterval = validatePositive(healthCheckInterval, DEFAULT_HEALTH_CHECK_INTERVAL_MS, "healthCheckInterval");
+ this.deregisterAfter = validateNonNegative(deregisterAfter, DEFAULT_DEREGISTER_AFTER_MS, "deregisterAfter");
+ }
+
+ @JsonProperty
+ public String getServicePrefix()
+ {
+ return servicePrefix;
+ }
+
+ @JsonProperty
+ @Nullable
+ public String getDatacenter()
+ {
+ return datacenter;
+ }
+
+ @JsonProperty
+ @Nullable
+ public Map getServiceTags()
+ {
+ return serviceTags == null ? null : Collections.unmodifiableMap(serviceTags);
+ }
+
+ @JsonProperty
+ public Duration getHealthCheckInterval()
+ {
+ return healthCheckInterval;
+ }
+
+ @JsonProperty
+ public Duration getDeregisterAfter()
+ {
+ return deregisterAfter;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ ServiceConfig that = (ServiceConfig) o;
+ return Objects.equals(servicePrefix, that.getServicePrefix()) &&
+ Objects.equals(datacenter, that.getDatacenter()) &&
+ Objects.equals(serviceTags, that.getServiceTags()) &&
+ Objects.equals(healthCheckInterval, that.getHealthCheckInterval()) &&
+ Objects.equals(deregisterAfter, that.getDeregisterAfter());
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(servicePrefix, datacenter, serviceTags, healthCheckInterval, deregisterAfter);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "ServiceConfig{servicePrefix='" + servicePrefix + "', datacenter='" + datacenter +
+ "', healthCheckInterval=" + healthCheckInterval + ", deregisterAfter=" + deregisterAfter + '}';
+ }
+ }
+
+ public static class LeaderElectionConfig
+ {
+ private static final long DEFAULT_LEADER_RETRY_BACKOFF_MAX_MS = 300_000;
+ private static final long DEFAULT_LEADER_MAX_ERROR_RETRIES = 20;
+
+ private final String coordinatorLeaderLockPath;
+ private final String overlordLeaderLockPath;
+ private final Duration leaderSessionTtl;
+ private final long leaderMaxErrorRetries;
+ private final Duration leaderRetryBackoffMax;
+
+ @JsonCreator
+ public LeaderElectionConfig(
+ @JsonProperty("coordinatorLeaderLockPath") @Nullable String coordinatorLeaderLockPath,
+ @JsonProperty("overlordLeaderLockPath") @Nullable String overlordLeaderLockPath,
+ @JsonProperty("leaderSessionTtl") @Nullable Duration leaderSessionTtl,
+ @JsonProperty("leaderMaxErrorRetries") @Nullable Long leaderMaxErrorRetries,
+ @JsonProperty("leaderRetryBackoffMax") @Nullable Duration leaderRetryBackoffMax,
+ @JsonProperty("healthCheckInterval") @Nullable Duration healthCheckInterval
+ )
+ {
+ this.coordinatorLeaderLockPath = coordinatorLeaderLockPath != null
+ ? coordinatorLeaderLockPath
+ : "druid/leader/coordinator";
+ this.overlordLeaderLockPath = overlordLeaderLockPath != null
+ ? overlordLeaderLockPath
+ : "druid/leader/overlord";
+ this.leaderSessionTtl = computeLeaderSessionTtl(leaderSessionTtl, healthCheckInterval);
+ this.leaderMaxErrorRetries = (leaderMaxErrorRetries == null || leaderMaxErrorRetries <= 0)
+ ? DEFAULT_LEADER_MAX_ERROR_RETRIES
+ : leaderMaxErrorRetries;
+ this.leaderRetryBackoffMax = validatePositive(
+ leaderRetryBackoffMax,
+ DEFAULT_LEADER_RETRY_BACKOFF_MAX_MS,
+ "leaderRetryBackoffMax"
+ );
+ }
+
+ private static Duration computeLeaderSessionTtl(Duration leaderSessionTtl, Duration healthCheckInterval)
+ {
+ Duration ttl = leaderSessionTtl;
+ if (ttl == null) {
+ long defaultTtlSeconds = 45; // Default TTL when healthCheckInterval is null
+ if (healthCheckInterval != null) {
+ defaultTtlSeconds = Math.max(45, healthCheckInterval.getStandardSeconds() * 3);
+ }
+ ttl = Duration.standardSeconds(defaultTtlSeconds);
+ }
+ if (ttl.getStandardSeconds() < MIN_LEADER_SESSION_TTL_SECONDS) {
+ throw new IAE(
+ StringUtils.format(
+ "leaderSessionTtl [%s] must be at least %d seconds",
+ ttl,
+ MIN_LEADER_SESSION_TTL_SECONDS
+ )
+ );
+ }
+ return ttl;
+ }
+
+ @JsonProperty
+ public String getCoordinatorLeaderLockPath()
+ {
+ return coordinatorLeaderLockPath;
+ }
+
+ @JsonProperty
+ public String getOverlordLeaderLockPath()
+ {
+ return overlordLeaderLockPath;
+ }
+
+ @JsonProperty
+ public Duration getLeaderSessionTtl()
+ {
+ return leaderSessionTtl;
+ }
+
+ @JsonProperty
+ public long getLeaderMaxErrorRetries()
+ {
+ return leaderMaxErrorRetries;
+ }
+
+ @JsonProperty
+ public Duration getLeaderRetryBackoffMax()
+ {
+ return leaderRetryBackoffMax;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ LeaderElectionConfig that = (LeaderElectionConfig) o;
+ return leaderMaxErrorRetries == that.getLeaderMaxErrorRetries() &&
+ Objects.equals(coordinatorLeaderLockPath, that.getCoordinatorLeaderLockPath()) &&
+ Objects.equals(overlordLeaderLockPath, that.getOverlordLeaderLockPath()) &&
+ Objects.equals(leaderSessionTtl, that.getLeaderSessionTtl()) &&
+ Objects.equals(leaderRetryBackoffMax, that.getLeaderRetryBackoffMax());
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(
+ coordinatorLeaderLockPath,
+ overlordLeaderLockPath,
+ leaderSessionTtl,
+ leaderMaxErrorRetries,
+ leaderRetryBackoffMax
+ );
+ }
+
+ @Override
+ public String toString()
+ {
+ return "LeaderElectionConfig{coordinatorPath='" + coordinatorLeaderLockPath +
+ "', overlordPath='" + overlordLeaderLockPath +
+ "', sessionTtl=" + leaderSessionTtl +
+ ", maxRetries=" + (leaderMaxErrorRetries == Long.MAX_VALUE ? "unlimited" : leaderMaxErrorRetries) + '}';
+ }
+ }
+
+ public static class WatchConfig
+ {
+ private static final long DEFAULT_WATCH_TIMEOUT_MS = 60_000;
+ private static final long DEFAULT_WATCH_RETRY_DELAY_MS = 10_000;
+ private static final long DEFAULT_CIRCUIT_BREAKER_SLEEP_MS = 120_000; // 2 minutes
+
+ private final Duration watchSeconds;
+ private final long maxWatchRetries;
+ private final Duration watchRetryDelay;
+ private final Duration circuitBreakerSleep;
+
+ @JsonCreator
+ public WatchConfig(
+ @JsonProperty("watchSeconds") @Nullable Duration watchSeconds,
+ @JsonProperty("maxWatchRetries") @Nullable Long maxWatchRetries,
+ @JsonProperty("watchRetryDelay") @Nullable Duration watchRetryDelay,
+ @JsonProperty("circuitBreakerSleep") @Nullable Duration circuitBreakerSleep
+ )
+ {
+ this.watchSeconds = validatePositive(watchSeconds, DEFAULT_WATCH_TIMEOUT_MS, "watchSeconds");
+ this.maxWatchRetries = (maxWatchRetries == null || maxWatchRetries <= 0) ? Long.MAX_VALUE : maxWatchRetries;
+ this.watchRetryDelay = validateNonNegative(watchRetryDelay, DEFAULT_WATCH_RETRY_DELAY_MS, "watchRetryDelay");
+ this.circuitBreakerSleep = validatePositive(circuitBreakerSleep, DEFAULT_CIRCUIT_BREAKER_SLEEP_MS, "circuitBreakerSleep");
+ }
+
+ @JsonProperty
+ public Duration getWatchSeconds()
+ {
+ return watchSeconds;
+ }
+
+ @JsonProperty
+ public long getMaxWatchRetries()
+ {
+ return maxWatchRetries;
+ }
+
+ @JsonProperty
+ public Duration getWatchRetryDelay()
+ {
+ return watchRetryDelay;
+ }
+
+ @JsonProperty
+ public Duration getCircuitBreakerSleep()
+ {
+ return circuitBreakerSleep;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ WatchConfig that = (WatchConfig) o;
+ return maxWatchRetries == that.getMaxWatchRetries() &&
+ Objects.equals(watchSeconds, that.getWatchSeconds()) &&
+ Objects.equals(watchRetryDelay, that.getWatchRetryDelay()) &&
+ Objects.equals(circuitBreakerSleep, that.getCircuitBreakerSleep());
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(watchSeconds, maxWatchRetries, watchRetryDelay, circuitBreakerSleep);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "WatchConfig{watchSeconds=" + watchSeconds +
+ ", maxRetries=" + (maxWatchRetries == Long.MAX_VALUE ? "unlimited" : maxWatchRetries) +
+ ", retryDelay=" + watchRetryDelay +
+ ", circuitBreakerSleep=" + circuitBreakerSleep + '}';
+ }
+ }
+
+ private static Duration validatePositive(Duration value, long defaultMs, String name)
+ {
+ Duration result = value;
+ if (result == null) {
+ result = Duration.millis(defaultMs);
+ }
+ if (result.getMillis() <= 0) {
+ throw new IAE(name + " must be positive");
+ }
+ return result;
+ }
+
+ private static Duration validateNonNegative(Duration value, long defaultMs, String name)
+ {
+ Duration result = value;
+ if (result == null) {
+ result = Duration.millis(defaultMs);
+ }
+ if (result.getMillis() < 0) {
+ throw new IAE(name + " cannot be negative");
+ }
+ return result;
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDiscoveryModule.java b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDiscoveryModule.java
new file mode 100644
index 000000000000..ffd00ab09e7f
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDiscoveryModule.java
@@ -0,0 +1,198 @@
+/*
+ * 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.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.inject.Binder;
+import com.google.inject.Inject;
+import com.google.inject.Key;
+import com.google.inject.Provider;
+import org.apache.druid.client.coordinator.Coordinator;
+import org.apache.druid.client.indexing.IndexingService;
+import org.apache.druid.discovery.DruidLeaderSelector;
+import org.apache.druid.discovery.DruidNodeAnnouncer;
+import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
+import org.apache.druid.guice.JsonConfigProvider;
+import org.apache.druid.guice.LazySingleton;
+import org.apache.druid.guice.PolyBind;
+import org.apache.druid.guice.annotations.Json;
+import org.apache.druid.guice.annotations.Self;
+import org.apache.druid.initialization.DruidModule;
+import org.apache.druid.server.DruidNode;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Guice module for Consul-based service discovery and leader election.
+ *
+ * To enable Consul discovery, set {@code druid.discovery.type=consul} and configure
+ * the extension under {@code druid.discovery.consul.*}. See the extension documentation
+ * for the full list of configuration options.
+ */
+public class ConsulDiscoveryModule implements DruidModule
+{
+ private static final String CONSUL_KEY = "consul";
+
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ return Collections.emptyList();
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ JsonConfigProvider.bind(binder, "druid.discovery.consul", ConsulDiscoveryConfig.class);
+
+ // Bind ConsulClient first since ConsulApiClient depends on it
+ binder.bind(ConsulClient.class)
+ .toProvider(ConsulClientProvider.class)
+ .in(LazySingleton.class);
+
+ binder.bind(ConsulApiClient.class)
+ .toProvider(ConsulApiClientProvider.class)
+ .in(LazySingleton.class);
+
+ PolyBind.optionBinder(binder, Key.get(DruidNodeDiscoveryProvider.class))
+ .addBinding(CONSUL_KEY)
+ .to(ConsulDruidNodeDiscoveryProvider.class)
+ .in(LazySingleton.class);
+
+ PolyBind.optionBinder(binder, Key.get(DruidNodeAnnouncer.class))
+ .addBinding(CONSUL_KEY)
+ .to(ConsulDruidNodeAnnouncer.class)
+ .in(LazySingleton.class);
+
+ PolyBind.optionBinder(binder, Key.get(DruidLeaderSelector.class, Coordinator.class))
+ .addBinding(CONSUL_KEY)
+ .toProvider(CoordinatorLeaderSelectorProvider.class)
+ .in(LazySingleton.class);
+
+ PolyBind.optionBinder(binder, Key.get(DruidLeaderSelector.class, IndexingService.class))
+ .addBinding(CONSUL_KEY)
+ .toProvider(OverlordLeaderSelectorProvider.class)
+ .in(LazySingleton.class);
+ }
+
+ private static class ConsulApiClientProvider implements Provider
+ {
+ private final ConsulClient consulClient;
+ private final ConsulDiscoveryConfig config;
+ private final ObjectMapper jsonMapper;
+
+ @Inject
+ ConsulApiClientProvider(
+ ConsulClient consulClient,
+ ConsulDiscoveryConfig config,
+ @Json ObjectMapper jsonMapper
+ )
+ {
+ this.consulClient = consulClient;
+ this.config = config;
+ this.jsonMapper = jsonMapper;
+ }
+
+ @Override
+ public ConsulApiClient get()
+ {
+ return new DefaultConsulApiClient(consulClient, config, jsonMapper);
+ }
+ }
+
+ private static class ConsulClientProvider implements Provider
+ {
+ private final ConsulDiscoveryConfig config;
+
+ @Inject
+ ConsulClientProvider(ConsulDiscoveryConfig config)
+ {
+ this.config = config;
+ }
+
+ @Override
+ public ConsulClient get()
+ {
+ return ConsulClients.create(config);
+ }
+ }
+
+ private static class CoordinatorLeaderSelectorProvider implements Provider
+ {
+ private final DruidNode self;
+ private final ConsulDiscoveryConfig config;
+ private final ConsulClient consulClient;
+
+ @Inject
+ CoordinatorLeaderSelectorProvider(
+ @Self DruidNode self,
+ ConsulDiscoveryConfig config,
+ ConsulClient consulClient
+ )
+ {
+ this.self = self;
+ this.config = config;
+ this.consulClient = consulClient;
+ }
+
+ @Override
+ public DruidLeaderSelector get()
+ {
+ return new ConsulLeaderSelector(
+ self,
+ config.getLeader().getCoordinatorLeaderLockPath(),
+ config,
+ consulClient
+ );
+ }
+ }
+
+ private static class OverlordLeaderSelectorProvider implements Provider
+ {
+ private final DruidNode self;
+ private final ConsulDiscoveryConfig config;
+ private final ConsulClient consulClient;
+
+ @Inject
+ OverlordLeaderSelectorProvider(
+ @Self DruidNode self,
+ ConsulDiscoveryConfig config,
+ ConsulClient consulClient
+ )
+ {
+ this.self = self;
+ this.config = config;
+ this.consulClient = consulClient;
+ }
+
+ @Override
+ public DruidLeaderSelector get()
+ {
+ return new ConsulLeaderSelector(
+ self,
+ config.getLeader().getOverlordLeaderLockPath(),
+ config,
+ consulClient
+ );
+ }
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDruidNodeAnnouncer.java b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDruidNodeAnnouncer.java
new file mode 100644
index 000000000000..63aba8ee25ae
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDruidNodeAnnouncer.java
@@ -0,0 +1,298 @@
+/*
+ * 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.DiscoveryDruidNode;
+import org.apache.druid.discovery.DruidNodeAnnouncer;
+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 javax.annotation.Nullable;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Announces Druid nodes to Consul and maintains their health status via TTL checks.
+ */
+@ManageLifecycle
+public class ConsulDruidNodeAnnouncer implements DruidNodeAnnouncer
+{
+ private static final Logger LOGGER = new Logger(ConsulDruidNodeAnnouncer.class);
+
+ private final ConsulApiClient consulApiClient;
+ private final ConsulDiscoveryConfig config;
+ private final ConcurrentMap announcedNodes = new ConcurrentHashMap<>();
+ private final Set registeringNodes = ConcurrentHashMap.newKeySet();
+ private final ScheduledExecutorService healthCheckExecutor;
+
+ private static final int MAX_FAILURES_BEFORE_REREGISTER = 3;
+ private static final long EXECUTOR_TERMINATION_TIMEOUT_SECONDS = 10;
+ private final ConcurrentMap consecutiveFailures = new ConcurrentHashMap<>();
+
+ @Inject(optional = true)
+ @Nullable
+ private ServiceEmitter emitter;
+ private final LifecycleLock lifecycleLock = new LifecycleLock();
+
+ @Inject
+ public ConsulDruidNodeAnnouncer(
+ ConsulApiClient consulApiClient,
+ ConsulDiscoveryConfig config
+ )
+ {
+ this.consulApiClient = Preconditions.checkNotNull(consulApiClient, "consulApiClient");
+ this.config = Preconditions.checkNotNull(config, "config");
+ this.healthCheckExecutor = Execs.scheduledSingleThreaded("ConsulHealthCheck-%d");
+ }
+
+ @LifecycleStart
+ public void start()
+ {
+ if (!lifecycleLock.canStart()) {
+ throw new ISE("can't start");
+ }
+
+ try {
+ LOGGER.info("Starting ConsulDruidNodeAnnouncer");
+
+ long intervalMs = config.getService().getHealthCheckInterval().getMillis();
+ healthCheckExecutor.scheduleAtFixedRate(
+ this::updateHealthChecks,
+ 0L,
+ intervalMs,
+ TimeUnit.MILLISECONDS
+ );
+ lifecycleLock.started();
+ }
+ finally {
+ lifecycleLock.exitStart();
+ }
+ }
+
+ @LifecycleStop
+ public void stop()
+ {
+ if (!lifecycleLock.canStop()) {
+ throw new ISE("can't stop");
+ }
+
+ LOGGER.info("Stopping ConsulDruidNodeAnnouncer");
+
+ healthCheckExecutor.shutdownNow();
+
+ // Wait for health check to finish so we don't deregister while health check is in progress
+ try {
+ if (!healthCheckExecutor.awaitTermination(EXECUTOR_TERMINATION_TIMEOUT_SECONDS, TimeUnit.SECONDS)) {
+ LOGGER.warn("Health check executor did not terminate in time");
+ }
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ LOGGER.warn("Interrupted while waiting for health check termination");
+ }
+
+ for (String serviceId : announcedNodes.keySet()) {
+ try {
+ consulApiClient.deregisterService(serviceId);
+ }
+ catch (Exception e) {
+ LOGGER.error(e, "Failed to deregister service [%s] during shutdown", serviceId);
+ }
+ }
+
+ announcedNodes.clear();
+ lifecycleLock.exitStop();
+ }
+
+ @Override
+ public void announce(DiscoveryDruidNode discoveryDruidNode)
+ {
+ if (!lifecycleLock.awaitStarted(1, TimeUnit.SECONDS)) {
+ throw new ISE("Announcer not started");
+ }
+
+ String serviceId = ConsulServiceIds.serviceId(config, discoveryDruidNode);
+
+ // Prevent concurrent duplicate registrations for the same serviceId
+ if (!registeringNodes.add(serviceId)) {
+ LOGGER.warn("Registration already in progress for serviceId [%s]", serviceId);
+ return;
+ }
+
+ try {
+ // If already announced, skip duplicate registration
+ if (announcedNodes.containsKey(serviceId)) {
+ LOGGER.warn("ServiceId [%s] already announced, skipping", serviceId);
+ return;
+ }
+
+ long registerStart = System.nanoTime();
+
+ // Register in Consul, then track locally atomically in this block
+ consulApiClient.registerService(discoveryDruidNode);
+ announcedNodes.put(serviceId, discoveryDruidNode);
+
+ long registerLatency = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - registerStart);
+ ConsulMetrics.emitTimer(emitter, "consul/register/latency", registerLatency,
+ "role", discoveryDruidNode.getNodeRole().getJsonName());
+
+ LOGGER.info("Successfully announced serviceId [%s]", serviceId);
+ ConsulMetrics.emitCount(
+ emitter,
+ "consul/announce/success",
+ "role",
+ discoveryDruidNode.getNodeRole().getJsonName()
+ );
+ }
+ catch (Exception e) {
+ // Cleanup partial registration if Consul was updated before failure
+ try {
+ consulApiClient.deregisterService(serviceId);
+ }
+ catch (Exception cleanup) {
+ LOGGER.debug(cleanup, "Cleanup deregister failed for serviceId [%s] after announce error", serviceId);
+ }
+
+ LOGGER.error(e, "Exception during announce for DiscoveryDruidNode[%s]", discoveryDruidNode);
+ ConsulMetrics.emitCount(
+ emitter,
+ "consul/announce/failure",
+ "role",
+ discoveryDruidNode.getNodeRole().getJsonName()
+ );
+ throw new RuntimeException("Failed to announce serviceId [" + serviceId + "]", e);
+ }
+ finally {
+ registeringNodes.remove(serviceId);
+ }
+ }
+
+ @Override
+ public void unannounce(DiscoveryDruidNode discoveryDruidNode)
+ {
+ if (!lifecycleLock.awaitStarted(1, TimeUnit.SECONDS)) {
+ throw new ISE("Announcer not started");
+ }
+
+ LOGGER.info("Unannouncing DiscoveryDruidNode[%s]", discoveryDruidNode);
+
+ try {
+ String serviceId = ConsulServiceIds.serviceId(config, discoveryDruidNode);
+ consulApiClient.deregisterService(serviceId);
+ announcedNodes.remove(serviceId);
+
+ LOGGER.info("Successfully unannounced DiscoveryDruidNode[%s]", discoveryDruidNode);
+ ConsulMetrics.emitCount(emitter, "consul/unannounce/success",
+ "role", discoveryDruidNode.getNodeRole().getJsonName());
+ }
+ catch (Exception e) {
+ // Unannouncement happens during shutdown, don't throw
+ if (e instanceof InterruptedException) {
+ Thread.currentThread().interrupt();
+ }
+
+ LOGGER.error(e, "Failed to unannounce DiscoveryDruidNode[%s]", discoveryDruidNode);
+ ConsulMetrics.emitCount(emitter, "consul/unannounce/failure",
+ "role", discoveryDruidNode.getNodeRole().getJsonName());
+ }
+ }
+
+ private void updateHealthChecks()
+ {
+ int nodeCount = announcedNodes.size();
+ if (nodeCount == 0) {
+ return; // Silent when nothing to do
+ }
+
+ LOGGER.debug("Updating health checks for %d nodes", nodeCount);
+
+ int successCount = 0;
+ int failureCount = 0;
+
+ for (Map.Entry entry : announcedNodes.entrySet()) {
+ String serviceId = entry.getKey();
+ try {
+ long healthCheckStart = System.nanoTime();
+
+ consulApiClient.passTtlCheck(serviceId, "Druid node is healthy");
+
+ long healthCheckLatency = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - healthCheckStart);
+ ConsulMetrics.emitTimer(emitter, "consul/healthcheck/latency", healthCheckLatency,
+ "serviceId", serviceId);
+
+ successCount++;
+
+ consecutiveFailures.remove(serviceId);
+ }
+ catch (Exception e) {
+ failureCount++;
+
+ int failures = consecutiveFailures
+ .computeIfAbsent(serviceId, k -> new AtomicInteger(0))
+ .incrementAndGet();
+
+ // Keep WARN for failures - these matter
+ LOGGER.warn(e, "Health check failed [%d/%d] for [%s]",
+ failures, MAX_FAILURES_BEFORE_REREGISTER, serviceId);
+ ConsulMetrics.emitCount(emitter, "consul/healthcheck/failure",
+ "serviceId", serviceId, "consecutiveFailures", String.valueOf(failures));
+
+ if (failures >= MAX_FAILURES_BEFORE_REREGISTER) {
+ // Keep WARN for recovery actions - these are important state changes
+ LOGGER.warn("Re-registering [%s] after %d failures", serviceId, failures);
+ try {
+ DiscoveryDruidNode node = announcedNodes.get(serviceId);
+ consulApiClient.registerService(node);
+ consulApiClient.passTtlCheck(serviceId, "Re-registered");
+ consecutiveFailures.remove(serviceId);
+ ConsulMetrics.emitCount(emitter, "consul/healthcheck/reregister",
+ "serviceId", serviceId, "totalFailures", String.valueOf(failures));
+ LOGGER.info("Successfully re-registered [%s]", serviceId);
+ }
+ catch (Exception reregEx) {
+ LOGGER.error(reregEx, "Re-registration failed for [%s]", serviceId);
+ ConsulMetrics.emitCount(emitter, "consul/healthcheck/reregister/failure",
+ "serviceId", serviceId, "totalFailures", String.valueOf(failures));
+ }
+ }
+ }
+ }
+
+ if (failureCount > 0) {
+ LOGGER.info("Health checks: %d/%d failed", failureCount, nodeCount);
+ } else {
+ LOGGER.debug("Health checks: %d/%d succeeded", successCount, nodeCount);
+ }
+ }
+
+}
diff --git a/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDruidNodeDiscoveryProvider.java b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulDruidNodeDiscoveryProvider.java
new file mode 100644
index 000000000000..188b7d7538bc
--- /dev/null
+++ b/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}.
+ *
+ * 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.
+ *
+ *
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 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 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 nodes = consulApiClient.getHealthyServices(nodeRole);
+ Map 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
+ );
+
+ long watchLatency = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - watchStart);
+ ConsulMetrics.emitTimer(emitter, "consul/watch/latency", watchLatency, "role", nodeRole.getJsonName());
+
+ long newIndex = watchResult.getConsulIndex();
+ if (newIndex != consulIndex) {
+ consulIndex = newIndex;
+
+ List newNodes = watchResult.getNodes();
+ Map newNodeMap = new HashMap<>();
+ for (DiscoveryDruidNode node : newNodes) {
+ newNodeMap.put(node.getDruidNode().getHostAndPortToUse(), node);
+ }
+
+ Collection currentNodes = baseNodeRoleWatcher.getAllNodes();
+ Map currentNodeMap = new HashMap<>();
+ for (DiscoveryDruidNode node : currentNodes) {
+ currentNodeMap.put(node.getDruidNode().getHostAndPortToUse(), node);
+ }
+
+ for (Map.Entry entry : newNodeMap.entrySet()) {
+ if (!currentNodeMap.containsKey(entry.getKey())) {
+ try {
+ baseNodeRoleWatcher.childAdded(entry.getValue());
+ ConsulMetrics.emitCount(emitter, "consul/watch/added", "role", nodeRole.getJsonName());
+ }
+ catch (RejectedExecutionException e) {
+ LOGGER.debug("Ignoring node add during shutdown for role[%s]", nodeRole);
+ }
+ }
+ }
+
+ for (Map.Entry entry : currentNodeMap.entrySet()) {
+ if (!newNodeMap.containsKey(entry.getKey())) {
+ try {
+ baseNodeRoleWatcher.childRemoved(entry.getValue());
+ ConsulMetrics.emitCount(emitter, "consul/watch/removed", "role", nodeRole.getJsonName());
+ }
+ catch (RejectedExecutionException e) {
+ // Expected during shutdown - executor is terminated
+ LOGGER.debug("Ignoring node removal during shutdown for role[%s]", nodeRole);
+ }
+ }
+ }
+ }
+
+ retryCount.set(0);
+ }
+ catch (Exception ex) {
+ if (Thread.currentThread().isInterrupted()) {
+ LOGGER.info("Watch interrupted during shutdown for role[%s]", nodeRole);
+ break;
+ }
+
+ if (isSocketTimeout(ex)) {
+ LOGGER.debug("Watch timeout for role[%s], re-issuing blocking query.", nodeRole);
+ continue;
+ }
+
+ LOGGER.warn(ex, "Exception while watching for role[%s], will retry.", nodeRole);
+
+ ConsulMetrics.emitCount(emitter, "consul/watch/error", "role", nodeRole.getJsonName());
+
+ long count = retryCount.incrementAndGet();
+ if (config.getWatch().getMaxWatchRetries() != Long.MAX_VALUE && count > config.getWatch().getMaxWatchRetries()) {
+ long circuitBreakerSleepMs = config.getWatch().getCircuitBreakerSleep().getMillis();
+ LOGGER.error(
+ "Max watch retries [%d] exceeded for role[%s]; circuit breaker OPEN, sleeping %s then retry",
+ config.getWatch().getMaxWatchRetries(),
+ nodeRole,
+ config.getWatch().getCircuitBreakerSleep()
+ );
+ ConsulMetrics.emitCount(emitter, "consul/watch/circuit_breaker",
+ "state", "open", "role", nodeRole.getJsonName());
+
+ sleep(circuitBreakerSleepMs);
+
+ retryCount.set(0);
+ cacheInitialized = false;
+ ConsulMetrics.emitCount(emitter, "consul/watch/circuit_breaker",
+ "state", "reset", "role", nodeRole.getJsonName());
+ continue;
+ }
+
+ long base = Math.max(1L, config.getWatch().getWatchRetryDelay().getMillis());
+ int exp = (int) Math.min(6, count);
+ long backoff = Math.min(300_000L, base * (1L << exp));
+ long jitter = (long) (backoff * (0.5 + ThreadLocalRandom.current().nextDouble()));
+ sleep(jitter);
+ }
+ }
+
+ LOGGER.info("Exited Watch for role[%s].", nodeRole);
+ }
+
+ private void sleep(long ms)
+ {
+ try {
+ Thread.sleep(ms);
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+
+ /**
+ * Checks if the exception is or is caused by a SocketTimeoutException.
+ * This is expected behavior for Consul blocking queries that time out.
+ */
+ private boolean isSocketTimeout(Throwable ex)
+ {
+ Throwable current = ex;
+ while (current != null) {
+ if (current instanceof SocketTimeoutException) {
+ return true;
+ }
+ current = current.getCause();
+ }
+ return false;
+ }
+
+ public void start()
+ {
+ if (!lifecycleLock.canStart()) {
+ throw new ISE("can't start.");
+ }
+
+ try {
+ LOGGER.info("Starting NodeRoleWatcher for role[%s]...", nodeRole);
+ this.watchExecutor = Execs.singleThreaded(this.getClass().getName() + nodeRole.getJsonName());
+ watchExecutor.submit(this::watch);
+ lifecycleLock.started();
+ ConsulMetrics.emitCount(
+ emitter,
+ "consul/watch/lifecycle",
+ "role",
+ nodeRole.getJsonName(),
+ "state",
+ "start"
+ );
+ LOGGER.info("Started NodeRoleWatcher for role[%s].", nodeRole);
+ }
+ finally {
+ lifecycleLock.exitStart();
+ }
+ }
+
+ public void stop()
+ {
+ if (!lifecycleLock.canStop()) {
+ throw new ISE("can't stop.");
+ }
+
+ try {
+ LOGGER.info("Stopping NodeRoleWatcher for role[%s]...", nodeRole);
+ watchExecutor.shutdownNow();
+
+ if (!watchExecutor.awaitTermination(15, TimeUnit.SECONDS)) {
+ LOGGER.warn("Failed to stop watchExecutor for role[%s]", nodeRole);
+ }
+ ConsulMetrics.emitCount(
+ emitter,
+ "consul/watch/lifecycle",
+ "role",
+ nodeRole.getJsonName(),
+ "state",
+ "stop"
+ );
+ LOGGER.info("Stopped NodeRoleWatcher for role[%s].", nodeRole);
+ }
+ catch (Exception ex) {
+ LOGGER.error(ex, "Failed to stop NodeRoleWatcher for role[%s].", nodeRole);
+ }
+ finally {
+ // Allow restart and leave lock in a clean state
+ lifecycleLock.exitStopAndReset();
+ }
+ }
+
+ @Override
+ public Collection getAllNodes()
+ {
+ return baseNodeRoleWatcher.getAllNodes();
+ }
+
+ @Override
+ public void registerListener(Listener listener)
+ {
+ baseNodeRoleWatcher.registerListener(listener);
+ }
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulLeaderSelector.java b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulLeaderSelector.java
new file mode 100644
index 000000000000..1627e45cc89f
--- /dev/null
+++ b/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.
+ *
+ * {@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.
+ *
+ *
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 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;
+ }
+ }
+
+ boolean acquired = tryAcquireLock(sessionId);
+
+ if (acquired && !leader.get()) {
+ boolean interrupted = Thread.currentThread().isInterrupted();
+ if (stopping || interrupted) {
+ LOGGER.info(
+ "Skipping leadership for [%s] because selector is stopping (interrupted=%s)",
+ lockKey,
+ interrupted
+ );
+ } else if (sessionId == null) {
+ LOGGER.warn("Skipping leadership for [%s] because session is null", lockKey);
+ } else if (validateLockOwnership(sessionId)) {
+ long electionStart = System.nanoTime();
+ becomeLeader();
+ long electionLatency = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - electionStart);
+ ConsulMetrics.emitTimer(emitter, "consul/leader/election_latency", electionLatency,
+ "lock", lockKey);
+ } else {
+ LOGGER.warn("Lock ownership validation failed for [%s]; will retry", lockKey);
+ emitOwnershipMismatchMetric();
+ }
+ } else if (!acquired && leader.get()) {
+ loseLeadership();
+ }
+
+ if (leader.get()) {
+ // Session renewal handled by sessionKeeperLoop; here we just verify lock ownership
+ Thread.sleep(config.getService().getHealthCheckInterval().getMillis());
+ if (sessionId != null && !validateLockOwnership(sessionId)) {
+ LOGGER.warn("Main Loop: Lost lock ownership check for [%s], stepping down", lockKey);
+ loseLeadership();
+ }
+ } else {
+ Thread.sleep(config.getService().getHealthCheckInterval().getMillis());
+ }
+ errorRetryCount = 0;
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ break;
+ }
+ catch (Exception e) {
+ LOGGER.error(e, "Error in leader election loop");
+
+ if (leader.get()) {
+ loseLeadership();
+ }
+ sessionId = null;
+
+ errorRetryCount++;
+ long maxLeaderRetries = config.getLeader().getLeaderMaxErrorRetries();
+ if (errorRetryCount > maxLeaderRetries) {
+ LOGGER.error(
+ "Leader selector for [%s] exceeded max error retries [%d], giving up.",
+ lockKey,
+ maxLeaderRetries
+ );
+ ConsulMetrics.emitCount(emitter, "consul/leader/giveup", "lock", lockKey);
+ break;
+ }
+ long base = Math.max(1L, config.getWatch().getWatchRetryDelay().getMillis());
+ int exp = (int) Math.min(6, errorRetryCount);
+ long backoffCap = config.getLeader().getLeaderRetryBackoffMax().getMillis();
+ long backoff = Math.min(backoffCap, base * (1L << exp));
+ long sleepMs = (long) (backoff * (0.5 + ThreadLocalRandom.current().nextDouble()));
+ try {
+ Thread.sleep(sleepMs);
+ }
+ catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ break;
+ }
+ }
+ }
+
+ LOGGER.info("Exiting leader election loop for [%s]", lockKey);
+ ConsulMetrics.emitCount(emitter, "consul/leader/loop", "lock", lockKey, "state", "stop");
+ }
+
+ private void sessionKeeperLoop()
+ {
+ if (stopping) {
+ return;
+ }
+
+ String currentSessionId = this.sessionId;
+ if (currentSessionId != null) {
+ final String truncatedId = shortSessionId(currentSessionId);
+ try {
+ Response response = consulClient.renewSession(
+ currentSessionId,
+ buildQueryParams(),
+ config.getAuth().getAclToken()
+ );
+
+ if (response == null || response.getValue() == null) {
+ LOGGER.warn("Session Keeper: Failed to renew session [%s], it may have expired", truncatedId);
+ // Don't null it out here, main loop handles recreating if it fails to use it
+ // But we can signal leadership loss if we thought we were leader
+ if (leader.get()) {
+ LOGGER.error("Session Keeper: Leader lost session [%s], triggering step down", truncatedId);
+ // Trigger immediate check in main loop or let main loop fail on next check
+ // Ideally, we could interrupt main loop, but for safety we let main loop handle state
+ }
+ ConsulMetrics.emitCount(emitter, "consul/leader/renew/fail", "lock", lockKey);
+ } else {
+ LOGGER.debug("Session Keeper: Successfully renewed session [%s]", truncatedId);
+ }
+ }
+ catch (Exception e) {
+ LOGGER.error(e, "Session Keeper: Exception renewing session [%s]", truncatedId);
+ ConsulMetrics.emitCount(emitter, "consul/leader/renew/fail", "lock", lockKey);
+ }
+ }
+
+ if (!stopping) {
+ sessionKeeperService.schedule(
+ this::sessionKeeperLoop,
+ config.getService().getHealthCheckInterval().getMillis() / 3, // Renew more frequently than TTL
+ TimeUnit.MILLISECONDS
+ );
+ }
+ }
+
+ private boolean isSessionValid(String sid)
+ {
+ try {
+ Response resp = consulClient.getSessionInfo(sid, buildQueryParams());
+ return resp != null && resp.getValue() != null;
+ }
+ catch (Exception e) {
+ LOGGER.debug(e, "Failed to validate session [%s]", sid);
+ return false;
+ }
+ }
+
+ private String createSession()
+ {
+ NewSession newSession = new NewSession();
+ newSession.setName(StringUtils.format("druid-leader-%s", self.getHostAndPortToUse()));
+
+ long ttlSeconds = config.getLeader().getLeaderSessionTtl().getStandardSeconds();
+ newSession.setTtl(StringUtils.format("%ds", ttlSeconds));
+
+ newSession.setBehavior(Session.Behavior.DELETE);
+
+ // Lock delay - prevents rapid re-acquisition after session invalidation (in seconds)
+ newSession.setLockDelay(5L);
+
+ Response response = consulClient.sessionCreate(
+ newSession,
+ buildQueryParams(),
+ config.getAuth().getAclToken()
+ );
+ String sessionId = response.getValue();
+
+ LOGGER.info("Created session [%s...] for leader election, TTL=%ds",
+ shortSessionIdWithoutEllipsis(sessionId),
+ ttlSeconds);
+
+ return sessionId;
+ }
+
+ private boolean tryAcquireLock(String sessionId)
+ {
+ try {
+ String leaderValue = self.getServiceScheme() + "://" + self.getHostAndPortToUse();
+
+ PutParams putParams = new PutParams();
+ putParams.setAcquireSession(sessionId);
+ Response response = consulClient.setKVValue(
+ lockKey,
+ leaderValue,
+ config.getAuth().getAclToken(),
+ putParams,
+ buildQueryParams()
+ );
+
+ return response != null && Boolean.TRUE.equals(response.getValue());
+ }
+ catch (Exception e) {
+ LOGGER.error(e, "Failed to acquire lock on key [%s]", lockKey);
+ return false;
+ }
+ }
+
+ private void becomeLeader()
+ {
+ String currentSession = this.sessionId;
+
+ if (currentSession == null || stopping || Thread.currentThread().isInterrupted()) {
+ LOGGER.warn("Aborting promotion: session=%s, stopping=%s",
+ currentSession != null ? currentSession.substring(0, Math.min(8, currentSession.length())) + "..." : "null",
+ stopping);
+ return;
+ }
+
+ if (!validateLockOwnership(currentSession)) {
+ LOGGER.warn("Ownership check failed for [%s]", lockKey);
+ emitOwnershipMismatchMetric();
+ return;
+ }
+
+ if (!leader.compareAndSet(false, true)) {
+ LOGGER.info("Already leader for [%s]", lockKey);
+ return;
+ }
+
+ // Re-validate after CAS to handle race conditions
+ if (!validateLockOwnership(currentSession)) {
+ leader.set(false);
+ LOGGER.error("Lost ownership during promotion for [%s]", lockKey);
+ emitOwnershipMismatchMetric();
+ return;
+ }
+
+ int newTerm = term.incrementAndGet();
+
+ try {
+ listener.becomeLeader();
+ LOGGER.info("Became leader for [%s], term=%d", lockKey, newTerm);
+ ConsulMetrics.emitCount(emitter, "consul/leader/become", "lock", lockKey);
+ }
+ catch (Exception ex) {
+ LOGGER.error(ex, "Listener failed during promotion, destroyed session");
+ leader.set(false);
+ destroySession(currentSession);
+ throw ex;
+ }
+ }
+
+ private void destroySession(String sessionId)
+ {
+ if (sessionId != null) {
+ try {
+ consulClient.sessionDestroy(sessionId, buildQueryParams(), config.getAuth().getAclToken());
+ }
+ catch (Exception e) {
+ LOGGER.error(e, "Failed to destroy session [%s]", sessionId);
+ }
+ }
+ }
+
+ private boolean validateLockOwnership(String expectedSessionId)
+ {
+ try {
+ Response response = consulClient.getKVValue(
+ lockKey,
+ config.getAuth().getAclToken(),
+ buildQueryParams()
+ );
+ if (response == null || response.getValue() == null) {
+ LOGGER.warn("Lock key [%s] missing when validating ownership", lockKey);
+ return false;
+ }
+ String actualSessionId = response.getValue().getSession();
+ if (actualSessionId == null) {
+ LOGGER.warn("Lock key [%s] has no session owner", lockKey);
+ return false;
+ }
+ boolean matches = expectedSessionId.equals(actualSessionId);
+ if (!matches) {
+ LOGGER.warn(
+ "Lock key [%s] owned by session [%s], expected [%s]",
+ lockKey,
+ actualSessionId,
+ expectedSessionId
+ );
+ }
+ return matches;
+ }
+ catch (Exception e) {
+ LOGGER.error(e, "Failed to validate lock ownership for [%s]", lockKey);
+ return false;
+ }
+ }
+
+ private void emitOwnershipMismatchMetric()
+ {
+ ConsulMetrics.emitCount(
+ emitter,
+ "consul/leader/ownership_mismatch",
+ "lock",
+ lockKey
+ );
+ }
+
+ private void loseLeadership()
+ {
+ LOGGER.info("Losing leadership for [%s]", lockKey);
+
+ leader.set(false);
+
+ try {
+ listener.stopBeingLeader();
+ LOGGER.info("Successfully stepped down as leader for [%s]", lockKey);
+ ConsulMetrics.emitCount(emitter, "consul/leader/stop", "lock", lockKey);
+ }
+ catch (Exception ex) {
+ LOGGER.error(ex, "listener.stopBeingLeader() failed");
+ }
+ }
+
+ private QueryParams buildQueryParams()
+ {
+ if (config.getService().getDatacenter() != null) {
+ return new QueryParams(config.getService().getDatacenter());
+ }
+ return QueryParams.DEFAULT;
+ }
+
+ private static String shortSessionId(@Nullable String sessionId)
+ {
+ if (sessionId == null) {
+ return "null";
+ }
+ if (sessionId.length() <= 8) {
+ return sessionId;
+ }
+ return sessionId.substring(0, 8) + "...";
+ }
+
+ private static String shortSessionIdWithoutEllipsis(@Nullable String sessionId)
+ {
+ if (sessionId == null) {
+ return "null";
+ }
+ if (sessionId.length() <= 8) {
+ return sessionId;
+ }
+ return sessionId.substring(0, 8);
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulMetrics.java b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulMetrics.java
new file mode 100644
index 000000000000..f5cc4f73ebba
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulMetrics.java
@@ -0,0 +1,86 @@
+/*
+ * 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 org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+
+/**
+ * Minimal helper to emit metrics if a ServiceEmitter is available.
+ */
+final class ConsulMetrics
+{
+ private ConsulMetrics()
+ {
+ }
+
+ static void emitCount(ServiceEmitter emitter, String metric, String... dims)
+ {
+ if (emitter == null) {
+ return;
+ }
+ ServiceMetricEvent.Builder b = ServiceMetricEvent.builder();
+ if (dims != null && dims.length % 2 == 1) {
+ // ignore last odd key with no value to avoid exceptions
+ }
+ if (dims != null) {
+ for (int i = 0; i + 1 < dims.length; i += 2) {
+ b.setDimension(dims[i], dims[i + 1]);
+ }
+ }
+ emitter.emit(b.setMetric(metric, 1));
+ }
+
+ static void emitTimer(ServiceEmitter emitter, String metric, long millis, String... dims)
+ {
+ if (emitter == null) {
+ return;
+ }
+ ServiceMetricEvent.Builder b = ServiceMetricEvent.builder();
+ if (dims != null && dims.length % 2 == 1) {
+ // ignore last odd key with no value to avoid exceptions
+ }
+ if (dims != null) {
+ for (int i = 0; i + 1 < dims.length; i += 2) {
+ b.setDimension(dims[i], dims[i + 1]);
+ }
+ }
+ emitter.emit(b.setMetric(metric, millis));
+ }
+
+ @SuppressWarnings("unused")
+ static void emitGauge(ServiceEmitter emitter, String metric, Number value, String... dims)
+ {
+ if (emitter == null) {
+ return;
+ }
+ ServiceMetricEvent.Builder b = ServiceMetricEvent.builder();
+ if (dims != null && dims.length % 2 == 1) {
+ // ignore last odd key with no value to avoid exceptions
+ }
+ if (dims != null) {
+ for (int i = 0; i + 1 < dims.length; i += 2) {
+ b.setDimension(dims[i], dims[i + 1]);
+ }
+ }
+ emitter.emit(b.setMetric(metric, value));
+ }
+}
+
diff --git a/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulSSLConfig.java b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulSSLConfig.java
new file mode 100644
index 000000000000..274d710ef833
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulSSLConfig.java
@@ -0,0 +1,169 @@
+/*
+ * 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.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.metadata.PasswordProvider;
+
+import javax.annotation.Nullable;
+
+/**
+ * SSL/TLS configuration specific to Consul client connections.
+ * This is isolated from global Druid SSL configuration to avoid side effects.
+ */
+@SuppressWarnings({"unused", "deprecation"})
+public class ConsulSSLConfig
+{
+ @JsonProperty
+ @Nullable
+ private String protocol;
+
+ @JsonProperty
+ @Nullable
+ private String trustStoreType;
+
+ @JsonProperty
+ @Nullable
+ private String trustStorePath;
+
+ @JsonProperty
+ @Nullable
+ private String trustStoreAlgorithm;
+
+ @JsonProperty("trustStorePassword")
+ @Nullable
+ private PasswordProvider trustStorePasswordProvider;
+
+ @JsonProperty
+ @Nullable
+ private String keyStorePath;
+
+ @JsonProperty
+ @Nullable
+ private String keyStoreType;
+
+ @JsonProperty
+ @Nullable
+ private String certAlias;
+
+ @JsonProperty("keyStorePassword")
+ @Nullable
+ private PasswordProvider keyStorePasswordProvider;
+
+ @JsonProperty("keyManagerPassword")
+ @Nullable
+ private PasswordProvider keyManagerPasswordProvider;
+
+ @JsonProperty
+ @Nullable
+ private String keyManagerFactoryAlgorithm;
+
+ @JsonProperty
+ @Nullable
+ private Boolean validateHostnames;
+
+ @Nullable
+ public String getProtocol()
+ {
+ return protocol;
+ }
+
+ @Nullable
+ public String getTrustStoreType()
+ {
+ return trustStoreType;
+ }
+
+ @Nullable
+ public String getTrustStorePath()
+ {
+ return trustStorePath;
+ }
+
+ @Nullable
+ public String getTrustStoreAlgorithm()
+ {
+ return trustStoreAlgorithm;
+ }
+
+ @Nullable
+ public PasswordProvider getTrustStorePasswordProvider()
+ {
+ return trustStorePasswordProvider;
+ }
+
+ @Nullable
+ public String getKeyStorePath()
+ {
+ return keyStorePath;
+ }
+
+ @Nullable
+ public String getKeyStoreType()
+ {
+ return keyStoreType;
+ }
+
+ @Nullable
+ public PasswordProvider getKeyStorePasswordProvider()
+ {
+ return keyStorePasswordProvider;
+ }
+
+ @Nullable
+ public String getCertAlias()
+ {
+ return certAlias;
+ }
+
+ @Nullable
+ public PasswordProvider getKeyManagerPasswordProvider()
+ {
+ return keyManagerPasswordProvider;
+ }
+
+ @Nullable
+ public String getKeyManagerFactoryAlgorithm()
+ {
+ return keyManagerFactoryAlgorithm;
+ }
+
+ @Nullable
+ public Boolean getValidateHostnames()
+ {
+ return validateHostnames;
+ }
+
+ @Override
+ public String toString()
+ {
+ return "ConsulSSLConfig{" +
+ "protocol='" + protocol + '\'' +
+ ", trustStoreType='" + trustStoreType + '\'' +
+ ", trustStorePath='" + trustStorePath + '\'' +
+ ", trustStoreAlgorithm='" + trustStoreAlgorithm + '\'' +
+ ", keyStorePath='" + keyStorePath + '\'' +
+ ", keyStoreType='" + keyStoreType + '\'' +
+ ", certAlias='" + certAlias + '\'' +
+ ", keyManagerFactoryAlgorithm='" + keyManagerFactoryAlgorithm + '\'' +
+ ", validateHostnames=" + validateHostnames +
+ '}';
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulServiceIds.java b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulServiceIds.java
new file mode 100644
index 000000000000..a3910f3c5731
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/ConsulServiceIds.java
@@ -0,0 +1,48 @@
+/*
+ * 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 org.apache.druid.discovery.DiscoveryDruidNode;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.java.util.common.StringUtils;
+
+final class ConsulServiceIds
+{
+ private ConsulServiceIds()
+ {
+ }
+
+ static String serviceName(ConsulDiscoveryConfig config, NodeRole nodeRole)
+ {
+ return config.getService().getServicePrefix() + "-" + nodeRole.getJsonName();
+ }
+
+ static String serviceId(ConsulDiscoveryConfig config, DiscoveryDruidNode node)
+ {
+ return serviceName(config, node.getNodeRole()) + "-" +
+ node.getDruidNode().getHost() + "-" +
+ node.getDruidNode().getPortToUse();
+ }
+
+ static String nodeKvKey(ConsulDiscoveryConfig config, String serviceId)
+ {
+ return StringUtils.format("%s/nodes/%s", config.getService().getServicePrefix(), serviceId);
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/DefaultConsulApiClient.java b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/DefaultConsulApiClient.java
new file mode 100644
index 000000000000..752cd48a7eb1
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/main/java/org/apache/druid/consul/discovery/DefaultConsulApiClient.java
@@ -0,0 +1,283 @@
+/*
+ * 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.agent.model.NewService;
+import com.ecwid.consul.v1.health.HealthServicesRequest;
+import com.ecwid.consul.v1.health.model.HealthService;
+import com.ecwid.consul.v1.kv.model.GetValue;
+import com.ecwid.consul.v1.kv.model.PutParams;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import org.apache.druid.discovery.DiscoveryDruidNode;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Default implementation of {@link ConsulApiClient} using the Ecwid Consul client library.
+ */
+public class DefaultConsulApiClient implements ConsulApiClient
+{
+ private static final Logger LOGGER = new Logger(DefaultConsulApiClient.class);
+
+ // Consul service metadata has a limit of 512 characters per value
+ // Use a safe limit to avoid edge cases (450 chars leaves room for key name overhead)
+ private static final int MAX_METADATA_VALUE_SIZE = 450;
+ private static final long MIN_SESSION_TTL_SECONDS = 30;
+ private static final long MIN_HEALTH_CHECK_INTERVAL_SECONDS = 1;
+
+ private final ConsulClient consulClient;
+ private final ConsulDiscoveryConfig config;
+ private final ObjectMapper jsonMapper;
+
+ public DefaultConsulApiClient(
+ ConsulClient consulClient,
+ ConsulDiscoveryConfig config,
+ ObjectMapper jsonMapper
+ )
+ {
+ this.consulClient = Preconditions.checkNotNull(consulClient, "consulClient");
+ this.config = Preconditions.checkNotNull(config, "config");
+ this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper");
+
+ LOGGER.info(
+ "Created DefaultConsulApiClient for [%s:%d] with service prefix [%s]",
+ config.getConnection().getHost(),
+ config.getConnection().getPort(),
+ config.getService().getServicePrefix()
+ );
+ }
+
+ @Override
+ public void registerService(DiscoveryDruidNode node) throws Exception
+ {
+ String serviceId = ConsulServiceIds.serviceId(config, node);
+ String serviceName = ConsulServiceIds.serviceName(config, node.getNodeRole());
+
+ NewService service = new NewService();
+ service.setId(serviceId);
+ service.setName(serviceName);
+ service.setAddress(node.getDruidNode().getHost());
+ service.setPort(node.getDruidNode().getPortToUse());
+
+ List tags = new ArrayList<>();
+ tags.add("druid");
+ tags.add("role:" + node.getNodeRole().getJsonName());
+ if (config.getService().getServiceTags() != null) {
+ for (Map.Entry e : config.getService().getServiceTags().entrySet()) {
+ if (e.getKey() != null && e.getValue() != null) {
+ tags.add(e.getKey() + ":" + e.getValue());
+ }
+ }
+ }
+ service.setTags(tags);
+
+ // Serialize the full DiscoveryDruidNode as metadata
+ String nodeJson = jsonMapper.writeValueAsString(node);
+
+ // Consul service metadata has a 512 character limit per value
+ // If the JSON is too large, store it in Consul KV and reference it from metadata
+ Map meta = new HashMap<>();
+ if (nodeJson.length() <= MAX_METADATA_VALUE_SIZE) {
+ // Small enough - store directly in metadata
+ meta.put("druid_node", nodeJson);
+ } else {
+ // Too large - store in KV and reference it
+ String kvKey = ConsulServiceIds.nodeKvKey(config, serviceId);
+ PutParams putParams = new PutParams(); // No session locking for simple storage
+ consulClient.setKVValue(kvKey, nodeJson, config.getAuth().getAclToken(), putParams, buildQueryParams());
+ meta.put("druid_node_kv", kvKey);
+ LOGGER.debug(
+ "Node metadata for [%s] is too large (%d chars), stored in KV at [%s]",
+ serviceId,
+ nodeJson.length(),
+ kvKey
+ );
+ }
+ service.setMeta(meta);
+
+ NewService.Check check = new NewService.Check();
+ long intervalSeconds = Math.max(MIN_HEALTH_CHECK_INTERVAL_SECONDS, config.getService().getHealthCheckInterval().getStandardSeconds());
+ long ttlSeconds = Math.max(MIN_SESSION_TTL_SECONDS, intervalSeconds * 3);
+ check.setTtl(StringUtils.format("%ds", ttlSeconds));
+ check.setDeregisterCriticalServiceAfter(
+ StringUtils.format("%ds", config.getService().getDeregisterAfter().getStandardSeconds())
+ );
+ service.setCheck(check);
+
+ consulClient.agentServiceRegister(service, config.getAuth().getAclToken());
+ LOGGER.info("Registered service [%s] with Consul", serviceId);
+
+ try {
+ consulClient.agentCheckPass("service:" + serviceId, "Druid node is healthy", config.getAuth().getAclToken());
+ }
+ catch (Exception e) {
+ // Log but don't fail - the periodic health check will eventually mark it as passing
+ LOGGER.warn(e, "Failed to immediately mark service [%s] as healthy, will retry via periodic health check", serviceId);
+ }
+ }
+
+ @Override
+ @SuppressWarnings("RedundantThrows")
+ public void deregisterService(String serviceId) throws Exception
+ {
+ try {
+ String kvKey = ConsulServiceIds.nodeKvKey(config, serviceId);
+ consulClient.deleteKVValue(kvKey, config.getAuth().getAclToken());
+ }
+ catch (Exception e) {
+ LOGGER.debug(e, "Failed to delete KV entry for service [%s] during deregistration", serviceId);
+ }
+
+ consulClient.agentServiceDeregister(serviceId, config.getAuth().getAclToken());
+ LOGGER.info("Deregistered service [%s] from Consul", serviceId);
+ }
+
+ @Override
+ @SuppressWarnings("RedundantThrows")
+ public void passTtlCheck(String serviceId, String note) throws Exception
+ {
+ consulClient.agentCheckPass("service:" + serviceId, note, config.getAuth().getAclToken());
+ }
+
+ @Override
+ @SuppressWarnings("RedundantThrows")
+ public List getHealthyServices(NodeRole nodeRole) throws Exception
+ {
+ String serviceName = makeServiceName(nodeRole);
+ HealthServicesRequest request = HealthServicesRequest.newBuilder()
+ .setPassing(true)
+ .setQueryParams(buildQueryParams())
+ .setToken(config.getAuth().getAclToken())
+ .build();
+ Response> response = consulClient.getHealthServices(serviceName, request);
+
+ return parseHealthServices(response.getValue());
+ }
+
+ @Override
+ @SuppressWarnings("RedundantThrows")
+ public ConsulWatchResult watchServices(NodeRole nodeRole, long lastIndex, long waitSeconds) throws Exception
+ {
+ String serviceName = makeServiceName(nodeRole);
+
+ HealthServicesRequest request = HealthServicesRequest.newBuilder()
+ .setPassing(true)
+ .setQueryParams(buildQueryParams(waitSeconds, lastIndex))
+ .setToken(config.getAuth().getAclToken())
+ .build();
+ Response> response = consulClient.getHealthServices(serviceName, request);
+
+ List nodes = parseHealthServices(response.getValue());
+ long newIndex = response.getConsulIndex() != null ? response.getConsulIndex() : lastIndex;
+
+ return new ConsulWatchResult(nodes, newIndex);
+ }
+
+ private QueryParams buildQueryParams()
+ {
+ if (config.getService().getDatacenter() != null) {
+ return new QueryParams(config.getService().getDatacenter());
+ }
+ return QueryParams.DEFAULT;
+ }
+
+ private QueryParams buildQueryParams(long waitSeconds, long index)
+ {
+ if (config.getService().getDatacenter() != null) {
+ return new QueryParams(config.getService().getDatacenter(), waitSeconds, index);
+ }
+ return new QueryParams(waitSeconds, index);
+ }
+
+ @Override
+ public void close()
+ {
+ // Consul client doesn't need explicit cleanup
+ LOGGER.info("Closed Consul client");
+ }
+
+ private String makeServiceName(NodeRole nodeRole)
+ {
+ return ConsulServiceIds.serviceName(config, nodeRole);
+ }
+
+ private List parseHealthServices(List healthServices)
+ {
+ if (healthServices == null || healthServices.isEmpty()) {
+ return Collections.emptyList();
+ }
+
+ List nodes = new ArrayList<>();
+ for (HealthService healthService : healthServices) {
+ try {
+ if (healthService.getService() == null ||
+ healthService.getService().getMeta() == null) {
+ continue;
+ }
+
+ Map meta = healthService.getService().getMeta();
+ final String nodeJson;
+
+ if (meta.containsKey("druid_node")) {
+ nodeJson = meta.get("druid_node");
+ } else if (meta.containsKey("druid_node_kv")) {
+ String kvKey = meta.get("druid_node_kv");
+ Response kvResponse = consulClient.getKVValue(kvKey, config.getAuth().getAclToken(), buildQueryParams());
+ if (kvResponse != null && kvResponse.getValue() != null && kvResponse.getValue().getValue() != null) {
+ nodeJson = new String(
+ Base64.getDecoder().decode(kvResponse.getValue().getValue()),
+ StandardCharsets.UTF_8
+ );
+ } else {
+ LOGGER.warn("KV entry [%s] not found for service metadata", kvKey);
+ continue;
+ }
+ } else {
+ continue;
+ }
+
+ DiscoveryDruidNode node = jsonMapper.readValue(nodeJson, DiscoveryDruidNode.class);
+ nodes.add(node);
+ }
+ catch (IOException e) {
+ LOGGER.error(e, "Failed to parse DiscoveryDruidNode from Consul service metadata");
+ }
+ catch (Exception e) {
+ LOGGER.error(e, "Failed to retrieve or parse DiscoveryDruidNode from Consul");
+ }
+ }
+
+ return nodes;
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/consul-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
new file mode 100644
index 000000000000..234de62020da
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@@ -0,0 +1,17 @@
+# 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.
+
+org.apache.druid.consul.discovery.ConsulDiscoveryModule
diff --git a/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulClientsSecurityTest.java b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulClientsSecurityTest.java
new file mode 100644
index 000000000000..d8e79ecc3e68
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulClientsSecurityTest.java
@@ -0,0 +1,151 @@
+/*
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for Consul client security validation, especially around basic auth over HTTP.
+ */
+public class ConsulClientsSecurityTest
+{
+ @Test
+ public void testBasicAuthOverHttpFailsFastByDefault()
+ {
+ ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .basicAuthUser("admin")
+ .basicAuthPassword("secret")
+ // No TLS configured
+ .build();
+
+ IllegalStateException exception = Assert.assertThrows(
+ IllegalStateException.class,
+ () -> ConsulClients.create(config)
+ );
+
+ Assert.assertTrue(
+ "Exception should mention TLS not enabled",
+ exception.getMessage().contains("TLS is not enabled")
+ );
+ Assert.assertTrue(
+ "Exception should mention cleartext transmission",
+ exception.getMessage().contains("cleartext")
+ );
+ Assert.assertTrue(
+ "Exception should mention allowBasicAuthOverHttp flag",
+ exception.getMessage().contains("allowBasicAuthOverHttp")
+ );
+ }
+
+ @Test
+ public void testBasicAuthOverHttpSucceedsWithExplicitFlag()
+ {
+ ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .basicAuthUser("admin")
+ .basicAuthPassword("secret")
+ .allowBasicAuthOverHttp(true)
+ // No TLS configured
+ .build();
+
+ // Should not throw with the flag enabled
+ ConsulClient client = ConsulClients.create(config);
+ Assert.assertNotNull(client);
+ }
+
+ @Test
+ public void testBasicAuthOverHttpFailsWhenExplicitlyDisabled()
+ {
+ ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .basicAuthUser("admin")
+ .basicAuthPassword("secret")
+ .allowBasicAuthOverHttp(false)
+ // No TLS configured
+ .build();
+
+ IllegalStateException exception = Assert.assertThrows(
+ IllegalStateException.class,
+ () -> ConsulClients.create(config)
+ );
+
+ Assert.assertTrue(
+ "Exception should mention TLS not enabled",
+ exception.getMessage().contains("TLS is not enabled")
+ );
+ }
+
+ @Test
+ public void testNoBasicAuthOverHttpSucceeds()
+ {
+ ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ // No basic auth configured
+ .build();
+
+ // Should succeed without basic auth even without TLS
+ ConsulClient client = ConsulClients.create(config);
+ Assert.assertNotNull(client);
+ }
+
+ @Test
+ public void testBasicAuthUserWithoutPasswordDoesNotTriggerValidation()
+ {
+ ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .basicAuthUser("admin")
+ // No password configured
+ .build();
+
+ // Should succeed - validation only applies when both user and password are set
+ ConsulClient client = ConsulClients.create(config);
+ Assert.assertNotNull(client);
+ }
+
+ @Test
+ public void testBasicAuthPasswordWithoutUserDoesNotTriggerValidation()
+ {
+ ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .basicAuthPassword("secret")
+ // No user configured
+ .build();
+
+ // Should succeed - validation only applies when both user and password are set
+ ConsulClient client = ConsulClients.create(config);
+ Assert.assertNotNull(client);
+ }
+
+ @Test
+ public void testAllowBasicAuthOverHttpDefaultsToFalse()
+ {
+ ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .build();
+
+ Assert.assertFalse(
+ "allowBasicAuthOverHttp should default to false",
+ config.getAuth().getAllowBasicAuthOverHttp()
+ );
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulDiscoveryConfigTest.java b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulDiscoveryConfigTest.java
new file mode 100644
index 000000000000..c4c5df5ffb47
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulDiscoveryConfigTest.java
@@ -0,0 +1,292 @@
+/*
+ * 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.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.joda.time.Duration;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ConsulDiscoveryConfigTest
+{
+ private final ObjectMapper jsonMapper = new DefaultObjectMapper();
+
+ @Test
+ public void testDefaultValuesSerde() throws Exception
+ {
+ testSerde("{\"service\": {\"servicePrefix\": \"druid\"}}\n");
+ }
+
+ @Test
+ public void testCustomizedValuesSerde() throws Exception
+ {
+ testSerde(
+ "{\n"
+ + " \"connection\": { \"host\": \"consul.example.com\", \"port\": 8600 },\n"
+ + " \"auth\": { \"aclToken\": \"secret-token\" },\n"
+ + " \"service\": { \"servicePrefix\": \"test-druid\", \"datacenter\": \"dc1\",\n"
+ + " \"healthCheckInterval\": \"PT5S\", \"deregisterAfter\": \"PT30S\" },\n"
+ + " \"watch\": { \"watchSeconds\": \"PT30S\", \"maxWatchRetries\": 100, \"watchRetryDelay\": \"PT5S\" }\n"
+ + "}\n"
+ );
+ }
+
+ @Test
+ public void testBasicAuthConfigurationSerde() throws Exception
+ {
+ testSerde(
+ "{\n"
+ + " \"auth\": { \"basicAuthUser\": \"admin\", \"basicAuthPassword\": \"secret\" },\n"
+ + " \"service\": { \"servicePrefix\": \"druid\" }\n"
+ + "}\n"
+ );
+ }
+
+ @Test
+ public void testBasicAuthWithAllowOverHttpSerde() throws Exception
+ {
+ testSerde(
+ "{\n"
+ + " \"auth\": { \"basicAuthUser\": \"admin\", \"basicAuthPassword\": \"secret\", \"allowBasicAuthOverHttp\": true },\n"
+ + " \"service\": { \"servicePrefix\": \"druid\" }\n"
+ + "}\n"
+ );
+ }
+
+ @Test
+ public void testAllowBasicAuthOverHttpDefaultsToFalse() throws Exception
+ {
+ ConsulDiscoveryConfig config = testSerdeAndReturn(
+ "{\n"
+ + " \"auth\": { \"basicAuthUser\": \"admin\", \"basicAuthPassword\": \"secret\" },\n"
+ + " \"service\": { \"servicePrefix\": \"druid\" }\n"
+ + "}\n"
+ );
+ Assert.assertFalse(config.getAuth().getAllowBasicAuthOverHttp());
+ }
+
+ @Test
+ public void testAllowBasicAuthOverHttpExplicitlySet() throws Exception
+ {
+ ConsulDiscoveryConfig config = testSerdeAndReturn(
+ "{\n"
+ + " \"auth\": { \"basicAuthUser\": \"admin\", \"basicAuthPassword\": \"secret\", \"allowBasicAuthOverHttp\": true },\n"
+ + " \"service\": { \"servicePrefix\": \"druid\" }\n"
+ + "}\n"
+ );
+ Assert.assertTrue(config.getAuth().getAllowBasicAuthOverHttp());
+ }
+
+ @Test
+ public void testNegativeMaxWatchRetriesMeansUnlimited() throws Exception
+ {
+ ConsulDiscoveryConfig config = testSerdeAndReturn(
+ "{\n"
+ + " \"service\": { \"servicePrefix\": \"druid\" },\n"
+ + " \"watch\": { \"maxWatchRetries\": -1 }\n"
+ + "}\n"
+ );
+ Assert.assertEquals(Long.MAX_VALUE, config.getWatch().getMaxWatchRetries());
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testNullServicePrefixThrows()
+ {
+ TestUtils.builder().servicePrefix(null).build();
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testEmptyServicePrefixThrows()
+ {
+ TestUtils.builder().servicePrefix("").build();
+ }
+
+ @Test
+ public void testLeaderRetryOverrides() throws Exception
+ {
+ ConsulDiscoveryConfig config = testSerdeAndReturn(
+ "{\n"
+ + " \"service\": { \"servicePrefix\": \"druid\" },\n"
+ + " \"leader\": { \"leaderMaxErrorRetries\": 5, \"leaderRetryBackoffMax\": \"PT30S\" }\n"
+ + "}\n"
+ );
+ Assert.assertEquals(5L, config.getLeader().getLeaderMaxErrorRetries());
+ Assert.assertEquals(Duration.millis(30000), config.getLeader().getLeaderRetryBackoffMax());
+ }
+
+ @Test
+ public void testSocketTimeoutMustExceedWatchSeconds()
+ {
+ try {
+ TestUtils.builder()
+ .servicePrefix("druid")
+ .socketTimeout(Duration.millis(5000))
+ .watchSeconds(Duration.millis(60000))
+ .build();
+ Assert.fail("Expected IllegalArgumentException for socketTimeout <= watchSeconds");
+ }
+ catch (IllegalArgumentException expected) {
+ // expected
+ }
+ }
+
+ @Test
+ public void testDefaultSocketTimeoutGreaterThanWatchSeconds()
+ {
+ ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .watchSeconds(Duration.millis(60000))
+ .build();
+
+ Assert.assertTrue(config.getConnection().getSocketTimeout().isLongerThan(config.getWatch().getWatchSeconds()));
+ }
+
+ @Test
+ public void testToStringMasksSensitiveData()
+ {
+ ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .datacenter("dc1")
+ .aclToken("secret-acl-token")
+ .basicAuthUser("admin")
+ .basicAuthPassword("password")
+ .healthCheckInterval(Duration.standardSeconds(10))
+ .deregisterAfter(Duration.standardSeconds(35))
+ .watchSeconds(Duration.millis(1000))
+ .build();
+
+ String toString = config.toString();
+
+ Assert.assertFalse(toString.contains("secret-acl-token"));
+ Assert.assertFalse(toString.contains("password"));
+ Assert.assertFalse(toString.contains("admin"));
+ Assert.assertTrue(toString.contains("*****"));
+ Assert.assertTrue(toString.contains("localhost"));
+ Assert.assertTrue(toString.contains("druid"));
+ }
+
+ @Test
+ public void testLeaderSessionTtlDefault() throws Exception
+ {
+ ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .healthCheckInterval(Duration.standardSeconds(10))
+ .build();
+
+ // Default should be max(45s, 3 * healthCheckInterval)
+ Assert.assertEquals(Duration.standardSeconds(45), config.getLeader().getLeaderSessionTtl());
+ }
+
+ @Test
+ public void testLeaderSessionTtlCustom() throws Exception
+ {
+ ConsulDiscoveryConfig config = testSerdeAndReturn(
+ "{\n"
+ + " \"service\": { \"servicePrefix\": \"druid\" },\n"
+ + " \"leader\": { \"leaderSessionTtl\": \"PT60S\" }\n"
+ + "}\n"
+ );
+
+ Assert.assertEquals(Duration.standardSeconds(60), config.getLeader().getLeaderSessionTtl());
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testLeaderSessionTtlTooLow()
+ {
+ TestUtils.builder()
+ .servicePrefix("druid")
+ .leaderSessionTtl(Duration.standardSeconds(5))
+ .build();
+ }
+
+ @Test
+ public void testLeaderSessionTtlSerde() throws Exception
+ {
+ testSerde(
+ "{\n"
+ + " \"service\": { \"servicePrefix\": \"druid\" },\n"
+ + " \"leader\": { \"leaderSessionTtl\": \"PT90S\" }\n"
+ + "}\n"
+ );
+ }
+
+ @Test
+ public void testLeaderSessionTtlDependsOnHealthCheckInterval() throws Exception
+ {
+ // With healthCheckInterval = 20s, default leaderSessionTtl should be 60s (3 * 20s)
+ ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .healthCheckInterval(Duration.standardSeconds(20))
+ .build();
+
+ // Should be 3 * healthCheckInterval = 60s (greater than minimum 45s)
+ Assert.assertEquals(Duration.standardSeconds(60), config.getLeader().getLeaderSessionTtl());
+ }
+
+ @Test
+ public void testServiceTagsDefensiveCopyAndUnmodifiable()
+ {
+ java.util.Map originalTags = new java.util.LinkedHashMap<>();
+ originalTags.put("key1", "value1");
+
+ ConsulDiscoveryConfig.ServiceConfig serviceConfig = new ConsulDiscoveryConfig.ServiceConfig(
+ "druid",
+ "dc1",
+ originalTags,
+ Duration.standardSeconds(10),
+ Duration.standardSeconds(90)
+ );
+
+ // Original map modifications should not affect stored map
+ originalTags.put("key1", "mutated");
+
+ Assert.assertEquals("value1", serviceConfig.getServiceTags().get("key1"));
+
+ try {
+ serviceConfig.getServiceTags().put("key2", "value2");
+ Assert.fail("Expected UnsupportedOperationException when mutating serviceTags");
+ }
+ catch (UnsupportedOperationException expected) {
+ // expected
+ }
+ }
+
+ private void testSerde(String jsonStr) throws Exception
+ {
+ ConsulDiscoveryConfig config = jsonMapper.readValue(jsonStr, ConsulDiscoveryConfig.class);
+ ConsulDiscoveryConfig roundTrip = jsonMapper.readValue(
+ jsonMapper.writeValueAsString(config),
+ ConsulDiscoveryConfig.class
+ );
+ Assert.assertEquals(config, roundTrip);
+ }
+
+ private ConsulDiscoveryConfig testSerdeAndReturn(String jsonStr) throws Exception
+ {
+ ConsulDiscoveryConfig config = jsonMapper.readValue(jsonStr, ConsulDiscoveryConfig.class);
+ ConsulDiscoveryConfig roundTrip = jsonMapper.readValue(
+ jsonMapper.writeValueAsString(config),
+ ConsulDiscoveryConfig.class
+ );
+ Assert.assertEquals(config, roundTrip);
+ return config;
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulDruidNodeAnnouncerTest.java b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulDruidNodeAnnouncerTest.java
new file mode 100644
index 000000000000..11dfe1f86064
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulDruidNodeAnnouncerTest.java
@@ -0,0 +1,267 @@
+/*
+ * 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 org.apache.druid.discovery.DiscoveryDruidNode;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.server.DruidNode;
+import org.easymock.Capture;
+import org.easymock.EasyMock;
+import org.joda.time.Duration;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+@RunWith(JUnit4.class)
+public class ConsulDruidNodeAnnouncerTest
+{
+ private final DiscoveryDruidNode testNode = new DiscoveryDruidNode(
+ new DruidNode("druid/broker", "test-host", true, 8082, null, true, false),
+ NodeRole.BROKER,
+ null
+ );
+
+ private final ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .healthCheckInterval(Duration.millis(10000))
+ .deregisterAfter(Duration.millis(90000))
+ .watchSeconds(Duration.millis(60000))
+ .watchRetryDelay(Duration.millis(10000))
+ .build();
+
+ private ConsulApiClient mockConsulApiClient;
+ private ConsulDruidNodeAnnouncer announcer;
+
+ @Before
+ public void setUp()
+ {
+ mockConsulApiClient = EasyMock.createMock(ConsulApiClient.class);
+ }
+
+ @After
+ public void tearDown()
+ {
+ // Don't call stop() here - each test will handle its own lifecycle
+ }
+
+ @Test
+ public void testAnnounce() throws Exception
+ {
+ Capture nodeCapture = Capture.newInstance();
+ mockConsulApiClient.registerService(EasyMock.capture(nodeCapture));
+ EasyMock.expectLastCall().once();
+
+ // Expect health check TTL updates (may happen during test)
+ mockConsulApiClient.passTtlCheck(EasyMock.anyString(), EasyMock.anyString());
+ EasyMock.expectLastCall().anyTimes();
+
+ // Expect deregisterService to be called during stop()
+ mockConsulApiClient.deregisterService(EasyMock.anyString());
+ EasyMock.expectLastCall().once();
+
+ EasyMock.replay(mockConsulApiClient);
+
+ announcer = new ConsulDruidNodeAnnouncer(mockConsulApiClient, config);
+ announcer.start();
+ announcer.announce(testNode);
+ Assert.assertEquals(testNode, nodeCapture.getValue());
+
+ // Explicitly stop to trigger cleanup
+ announcer.stop();
+
+ EasyMock.verify(mockConsulApiClient);
+ }
+
+ @Test
+ public void testUnannounce() throws Exception
+ {
+ Capture registerCapture = Capture.newInstance();
+ mockConsulApiClient.registerService(EasyMock.capture(registerCapture));
+ EasyMock.expectLastCall().once();
+
+ // Expect deregisterService to be called once from unannounce()
+ Capture deregisterCapture = Capture.newInstance();
+ mockConsulApiClient.deregisterService(EasyMock.capture(deregisterCapture));
+ EasyMock.expectLastCall().once();
+
+ // Expect health check TTL updates (may happen during test)
+ mockConsulApiClient.passTtlCheck(EasyMock.anyString(), EasyMock.anyString());
+ EasyMock.expectLastCall().anyTimes();
+
+ EasyMock.replay(mockConsulApiClient);
+
+ announcer = new ConsulDruidNodeAnnouncer(mockConsulApiClient, config);
+ announcer.start();
+ announcer.announce(testNode);
+ announcer.unannounce(testNode);
+
+ EasyMock.verify(mockConsulApiClient);
+
+ // Verify service ID format
+ String expectedServiceId = "druid-broker-test-host-8082";
+ Assert.assertEquals(expectedServiceId, deregisterCapture.getValue());
+ }
+
+ @Test
+ public void testAnnounceFails() throws Exception
+ {
+ mockConsulApiClient.registerService(EasyMock.anyObject(DiscoveryDruidNode.class));
+ EasyMock.expectLastCall().andThrow(new RuntimeException("Consul unavailable"));
+
+ // Expect health check TTL updates (may happen during test)
+ mockConsulApiClient.passTtlCheck(EasyMock.anyString(), EasyMock.anyString());
+ EasyMock.expectLastCall().anyTimes();
+
+ // deregisterService expected as cleanup after failed announce
+ mockConsulApiClient.deregisterService(EasyMock.anyString());
+ EasyMock.expectLastCall().once();
+
+ EasyMock.replay(mockConsulApiClient);
+
+ announcer = new ConsulDruidNodeAnnouncer(mockConsulApiClient, config);
+ announcer.start();
+
+ try {
+ announcer.announce(testNode);
+ Assert.fail("Expected RuntimeException");
+ }
+ catch (RuntimeException e) {
+ Assert.assertTrue(e.getMessage().contains("Failed to announce"));
+ }
+
+ // Don't need to stop since no nodes were announced
+
+ EasyMock.verify(mockConsulApiClient);
+ }
+
+ @Test
+ public void testDuplicateAnnounceIsSkipped() throws Exception
+ {
+ // registerService should be called only once even if announce is invoked twice
+ mockConsulApiClient.registerService(EasyMock.eq(testNode));
+ EasyMock.expectLastCall().once();
+
+ mockConsulApiClient.passTtlCheck(EasyMock.anyString(), EasyMock.anyString());
+ EasyMock.expectLastCall().anyTimes();
+
+ // stop() will deregister exactly once for the single announced node
+ mockConsulApiClient.deregisterService(EasyMock.anyString());
+ EasyMock.expectLastCall().once();
+
+ EasyMock.replay(mockConsulApiClient);
+
+ announcer = new ConsulDruidNodeAnnouncer(mockConsulApiClient, config);
+ announcer.start();
+
+ announcer.announce(testNode);
+ announcer.announce(testNode); // should be a no-op
+
+ announcer.stop();
+
+ EasyMock.verify(mockConsulApiClient);
+ }
+
+ @Test
+ public void testAnnounceFailureTriggersCleanup() throws Exception
+ {
+ // First attempt fails, should trigger deregister cleanup; second attempt succeeds
+ mockConsulApiClient.registerService(EasyMock.eq(testNode));
+ EasyMock.expectLastCall().andThrow(new RuntimeException("boom"));
+
+ mockConsulApiClient.deregisterService(EasyMock.anyString());
+ EasyMock.expectLastCall().once();
+
+ mockConsulApiClient.registerService(EasyMock.eq(testNode));
+ EasyMock.expectLastCall().once();
+
+ mockConsulApiClient.passTtlCheck(EasyMock.anyString(), EasyMock.anyString());
+ EasyMock.expectLastCall().anyTimes();
+
+ // One successful announce means one deregister on stop
+ mockConsulApiClient.deregisterService(EasyMock.anyString());
+ EasyMock.expectLastCall().once();
+
+ EasyMock.replay(mockConsulApiClient);
+
+ announcer = new ConsulDruidNodeAnnouncer(mockConsulApiClient, config);
+ announcer.start();
+
+ try {
+ announcer.announce(testNode);
+ Assert.fail("Expected failure on first announce");
+ }
+ catch (RuntimeException expected) {
+ Assert.assertTrue(expected.getMessage().contains("Failed to announce"));
+ }
+
+ announcer.announce(testNode); // succeeds
+
+ announcer.stop();
+
+ EasyMock.verify(mockConsulApiClient);
+ }
+
+ @Test
+ public void testConcurrentAnnounceOnlyRegistersOnce() throws Exception
+ {
+ // registerService should be called once despite concurrent announces
+ mockConsulApiClient.registerService(EasyMock.eq(testNode));
+ EasyMock.expectLastCall().once();
+
+ mockConsulApiClient.passTtlCheck(EasyMock.anyString(), EasyMock.anyString());
+ EasyMock.expectLastCall().anyTimes();
+
+ mockConsulApiClient.deregisterService(EasyMock.anyString());
+ EasyMock.expectLastCall().once();
+
+ EasyMock.replay(mockConsulApiClient);
+
+ announcer = new ConsulDruidNodeAnnouncer(mockConsulApiClient, config);
+ announcer.start();
+
+ CountDownLatch latch = new CountDownLatch(2);
+
+ Runnable announceTask = () -> {
+ try {
+ announcer.announce(testNode);
+ }
+ finally {
+ latch.countDown();
+ }
+ };
+
+ Thread t1 = new Thread(announceTask);
+ Thread t2 = new Thread(announceTask);
+ t1.start();
+ t2.start();
+
+ Assert.assertTrue("Announce tasks did not finish", latch.await(5, TimeUnit.SECONDS));
+
+ announcer.stop();
+
+ EasyMock.verify(mockConsulApiClient);
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulDruidNodeDiscoveryProviderTest.java b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulDruidNodeDiscoveryProviderTest.java
new file mode 100644
index 000000000000..9520d29fdedb
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulDruidNodeDiscoveryProviderTest.java
@@ -0,0 +1,216 @@
+/*
+ * 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.collect.ImmutableList;
+import org.apache.druid.discovery.DiscoveryDruidNode;
+import org.apache.druid.discovery.DruidNodeDiscovery;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.server.DruidNode;
+import org.easymock.EasyMock;
+import org.joda.time.Duration;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+public class ConsulDruidNodeDiscoveryProviderTest
+{
+ private final DiscoveryDruidNode node1 = new DiscoveryDruidNode(
+ new DruidNode("druid/broker", "host1", true, 8082, null, true, false),
+ NodeRole.BROKER,
+ null
+ );
+
+ private final DiscoveryDruidNode node2 = new DiscoveryDruidNode(
+ new DruidNode("druid/broker", "host2", true, 8082, null, true, false),
+ NodeRole.BROKER,
+ null
+ );
+
+ private final ConsulDiscoveryConfig config = TestUtils.builder()
+ .servicePrefix("druid")
+ .healthCheckInterval(Duration.millis(10000))
+ .deregisterAfter(Duration.millis(90000))
+ .watchSeconds(Duration.millis(60000))
+ .watchRetryDelay(Duration.millis(10000))
+ .build();
+
+ private ConsulApiClient mockConsulApiClient;
+ private ConsulDruidNodeDiscoveryProvider provider;
+
+ @Before
+ public void setUp()
+ {
+ mockConsulApiClient = EasyMock.createMock(ConsulApiClient.class);
+ provider = new ConsulDruidNodeDiscoveryProvider(mockConsulApiClient, config);
+ }
+
+ @After
+ public void tearDown()
+ {
+ if (provider != null) {
+ provider.stop();
+ }
+ }
+
+ @Test
+ public void testGetForNode() throws Exception
+ {
+ List nodes = ImmutableList.of(node1, node2);
+
+ EasyMock.expect(mockConsulApiClient.getHealthyServices(NodeRole.BROKER))
+ .andReturn(nodes)
+ .once();
+
+ EasyMock.replay(mockConsulApiClient);
+
+ provider.start();
+
+ boolean found = provider.getForNode(node1.getDruidNode(), NodeRole.BROKER).getAsBoolean();
+ Assert.assertTrue(found);
+
+ EasyMock.verify(mockConsulApiClient);
+ }
+
+ @Test
+ public void testGetForNodeRole() throws Exception
+ {
+ List initialNodes = ImmutableList.of(node1);
+
+ EasyMock.expect(mockConsulApiClient.getHealthyServices(NodeRole.BROKER))
+ .andReturn(initialNodes)
+ .once();
+
+ // First watch call with index 0
+ EasyMock.expect(mockConsulApiClient.watchServices(
+ EasyMock.eq(NodeRole.BROKER),
+ EasyMock.eq(0L),
+ EasyMock.anyLong()
+ ))
+ .andReturn(new ConsulApiClient.ConsulWatchResult(initialNodes, 1L))
+ .once();
+
+ // Subsequent watch calls with index 1 (the response index from first call)
+ EasyMock.expect(mockConsulApiClient.watchServices(
+ EasyMock.eq(NodeRole.BROKER),
+ EasyMock.eq(1L),
+ EasyMock.anyLong()
+ ))
+ .andReturn(new ConsulApiClient.ConsulWatchResult(initialNodes, 1L))
+ .anyTimes();
+
+ EasyMock.replay(mockConsulApiClient);
+
+ provider.start();
+
+ DruidNodeDiscovery discovery = provider.getForNodeRole(NodeRole.BROKER);
+ Assert.assertNotNull(discovery);
+
+ // Wait a bit for cache to initialize
+ Thread.sleep(500);
+
+ Collection nodes = discovery.getAllNodes();
+ Assert.assertEquals(1, nodes.size());
+ Assert.assertTrue(nodes.contains(node1));
+
+ EasyMock.verify(mockConsulApiClient);
+ }
+
+ @Test
+ public void testListenerNotifications() throws Exception
+ {
+ List initialNodes = ImmutableList.of(node1);
+ List updatedNodes = ImmutableList.of(node1, node2);
+
+ CountDownLatch initLatch = new CountDownLatch(1);
+ CountDownLatch addedLatch = new CountDownLatch(1);
+
+ EasyMock.expect(mockConsulApiClient.getHealthyServices(NodeRole.BROKER))
+ .andReturn(initialNodes)
+ .once();
+
+ // First watch returns no changes
+ EasyMock.expect(mockConsulApiClient.watchServices(
+ EasyMock.eq(NodeRole.BROKER),
+ EasyMock.eq(0L),
+ EasyMock.anyLong()
+ ))
+ .andReturn(new ConsulApiClient.ConsulWatchResult(initialNodes, 0L))
+ .once();
+
+ // Second watch returns updated nodes
+ EasyMock.expect(mockConsulApiClient.watchServices(
+ EasyMock.eq(NodeRole.BROKER),
+ EasyMock.eq(0L),
+ EasyMock.anyLong()
+ ))
+ .andReturn(new ConsulApiClient.ConsulWatchResult(updatedNodes, 1L))
+ .once();
+
+ // Continue returning updated nodes
+ EasyMock.expect(mockConsulApiClient.watchServices(
+ EasyMock.eq(NodeRole.BROKER),
+ EasyMock.eq(1L),
+ EasyMock.anyLong()
+ ))
+ .andReturn(new ConsulApiClient.ConsulWatchResult(updatedNodes, 1L))
+ .anyTimes();
+
+ EasyMock.replay(mockConsulApiClient);
+
+ provider.start();
+
+ DruidNodeDiscovery discovery = provider.getForNodeRole(NodeRole.BROKER);
+
+ discovery.registerListener(new DruidNodeDiscovery.Listener()
+ {
+ @Override
+ public void nodesAdded(Collection nodes)
+ {
+ if (nodes.contains(node2)) {
+ addedLatch.countDown();
+ }
+ }
+
+ @Override
+ public void nodesRemoved(Collection nodes)
+ {
+ // Not expected
+ }
+
+ @Override
+ public void nodeViewInitialized()
+ {
+ initLatch.countDown();
+ }
+ });
+
+ Assert.assertTrue("Initialization timed out", initLatch.await(5, TimeUnit.SECONDS));
+ Assert.assertTrue("Node addition not detected", addedLatch.await(5, TimeUnit.SECONDS));
+
+ EasyMock.verify(mockConsulApiClient);
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulLeaderSelectorTest.java b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulLeaderSelectorTest.java
new file mode 100644
index 000000000000..abc2cb73159f
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulLeaderSelectorTest.java
@@ -0,0 +1,326 @@
+/*
+ * 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 org.apache.druid.discovery.DruidLeaderSelector;
+import org.apache.druid.server.DruidNode;
+import org.easymock.Capture;
+import org.easymock.EasyMock;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+public class ConsulLeaderSelectorTest
+{
+ private static final String LOCK_KEY = "druid/leader/coordinator";
+ private static final String SESSION_ID = "test-session-id";
+ private static final String SESSION_ID_TWO = "test-session-id-2";
+
+ private ConsulClient mockConsulClient;
+ private ConsulLeaderSelector leaderSelector;
+ private ConsulDiscoveryConfig testConfig;
+ private DruidNode selfNode;
+
+ @Before
+ public void setUp()
+ {
+ selfNode = new DruidNode(
+ "druid/coordinator",
+ "test-host",
+ true,
+ 8081,
+ null,
+ true,
+ false
+ );
+
+ testConfig = TestUtils.minimalConfig();
+
+ mockConsulClient = EasyMock.createMock(ConsulClient.class);
+ leaderSelector = new ConsulLeaderSelector(selfNode, LOCK_KEY, testConfig, mockConsulClient);
+ }
+
+ @After
+ public void tearDown()
+ {
+ // Only unregister if we actually registered a listener
+ // Some tests just create the selector without registering
+ }
+
+ @Test
+ public void testGetCurrentLeader()
+ {
+ String leaderValue = "http://host:port";
+ GetValue getValue = new GetValue();
+ getValue.setValue(Base64.getEncoder().encodeToString(leaderValue.getBytes(StandardCharsets.UTF_8)));
+ Response response = new Response<>(getValue, 0L, true, 0L);
+
+ EasyMock.expect(mockConsulClient.getKVValue(
+ EasyMock.eq(LOCK_KEY),
+ EasyMock.isNull(),
+ EasyMock.eq(QueryParams.DEFAULT)
+ ))
+ .andReturn(response)
+ .once();
+
+ EasyMock.replay(mockConsulClient);
+
+ String currentLeader = leaderSelector.getCurrentLeader();
+ Assert.assertEquals(leaderValue, currentLeader);
+
+ EasyMock.verify(mockConsulClient);
+ }
+
+ @Test
+ public void testGetCurrentLeaderNoValue()
+ {
+ Response response = new Response<>(null, 0L, true, 0L);
+
+ EasyMock.expect(mockConsulClient.getKVValue(
+ EasyMock.eq(LOCK_KEY),
+ EasyMock.isNull(),
+ EasyMock.eq(QueryParams.DEFAULT)
+ ))
+ .andReturn(response)
+ .once();
+
+ EasyMock.replay(mockConsulClient);
+
+ String currentLeader = leaderSelector.getCurrentLeader();
+ Assert.assertNull(currentLeader);
+
+ EasyMock.verify(mockConsulClient);
+ }
+
+ @Test
+ public void testIsLeaderInitiallyFalse()
+ {
+ Assert.assertFalse(leaderSelector.isLeader());
+ }
+
+ @Test
+ public void testLocalTermInitiallyZero()
+ {
+ Assert.assertEquals(0, leaderSelector.localTerm());
+ }
+
+ @Test
+ public void testBecomeLeader() throws Exception
+ {
+ CountDownLatch becameLeaderLatch = new CountDownLatch(1);
+ CountDownLatch sessionCreatedLatch = new CountDownLatch(1);
+
+ // Mock session creation
+ Capture sessionCapture = Capture.newInstance();
+ EasyMock.expect(mockConsulClient.sessionCreate(
+ EasyMock.capture(sessionCapture),
+ EasyMock.eq(QueryParams.DEFAULT),
+ EasyMock.isNull()
+ ))
+ .andAnswer(() -> {
+ sessionCreatedLatch.countDown();
+ return new Response<>(SESSION_ID, 0L, true, 0L);
+ })
+ .once();
+
+ expectSessionInfoCheck(SESSION_ID);
+
+ // Mock successful lock acquisition
+ Capture putParamsCapture = Capture.newInstance();
+ EasyMock.expect(mockConsulClient.setKVValue(
+ EasyMock.eq(LOCK_KEY),
+ EasyMock.anyString(),
+ EasyMock.isNull(),
+ EasyMock.capture(putParamsCapture),
+ EasyMock.eq(QueryParams.DEFAULT)
+ ))
+ .andReturn(new Response<>(true, 0L, true, 0L))
+ .once();
+
+ // Mock session renewal
+ Session session = new Session();
+ session.setId(SESSION_ID);
+ EasyMock.expect(mockConsulClient.renewSession(
+ EasyMock.eq(SESSION_ID),
+ EasyMock.eq(QueryParams.DEFAULT),
+ EasyMock.isNull()
+ ))
+ .andReturn(new Response<>(session, 0L, true, 0L))
+ .anyTimes();
+
+ expectLockOwnershipCheck(SESSION_ID);
+
+ EasyMock.replay(mockConsulClient);
+
+ DruidLeaderSelector.Listener listener = new DruidLeaderSelector.Listener()
+ {
+ @Override
+ public void becomeLeader()
+ {
+ becameLeaderLatch.countDown();
+ }
+
+ @Override
+ public void stopBeingLeader()
+ {
+ // Not expected in this test
+ }
+ };
+
+ leaderSelector.registerListener(listener);
+
+ // Wait for session creation and leader election
+ Assert.assertTrue("Session not created", sessionCreatedLatch.await(5, TimeUnit.SECONDS));
+ Assert.assertTrue("Did not become leader", becameLeaderLatch.await(5, TimeUnit.SECONDS));
+
+ // Verify we became leader
+ Assert.assertTrue(leaderSelector.isLeader());
+ Assert.assertEquals(1, leaderSelector.localTerm());
+
+ // Verify session was created correctly
+ NewSession createdSession = sessionCapture.getValue();
+ Assert.assertNotNull(createdSession);
+ Assert.assertEquals(Session.Behavior.DELETE, createdSession.getBehavior());
+ Assert.assertEquals(5L, createdSession.getLockDelay());
+
+ // Verify lock acquisition used the session
+ PutParams putParams = putParamsCapture.getValue();
+ Assert.assertEquals(SESSION_ID, putParams.getAcquireSession());
+
+ EasyMock.verify(mockConsulClient);
+ }
+
+ @Test
+ public void testUnregisterDestroysSession() throws Exception
+ {
+ // Mock session creation
+ EasyMock.expect(mockConsulClient.sessionCreate(
+ EasyMock.anyObject(NewSession.class),
+ EasyMock.eq(QueryParams.DEFAULT),
+ EasyMock.isNull()
+ ))
+ .andReturn(new Response<>(SESSION_ID, 0L, true, 0L))
+ .once();
+
+ expectSessionInfoCheck(SESSION_ID);
+
+ // Mock session renewal
+ Session session = new Session();
+ session.setId(SESSION_ID);
+ EasyMock.expect(mockConsulClient.renewSession(
+ EasyMock.eq(SESSION_ID),
+ EasyMock.eq(QueryParams.DEFAULT),
+ EasyMock.isNull()
+ ))
+ .andReturn(new Response<>(session, 0L, true, 0L))
+ .anyTimes();
+
+ // Mock lock acquisition attempts
+ EasyMock.expect(mockConsulClient.setKVValue(
+ EasyMock.eq(LOCK_KEY),
+ EasyMock.anyString(),
+ EasyMock.isNull(),
+ EasyMock.anyObject(PutParams.class),
+ EasyMock.eq(QueryParams.DEFAULT)
+ ))
+ .andReturn(new Response<>(false, 0L, true, 0L))
+ .anyTimes();
+
+ // Mock session destruction
+ Capture sessionIdCapture = Capture.newInstance();
+ EasyMock.expect(mockConsulClient.sessionDestroy(
+ EasyMock.capture(sessionIdCapture),
+ EasyMock.eq(QueryParams.DEFAULT),
+ EasyMock.isNull()
+ ))
+ .andReturn(new Response<>(null, 0L, true, 0L))
+ .once();
+
+ EasyMock.replay(mockConsulClient);
+
+ DruidLeaderSelector.Listener listener = new DruidLeaderSelector.Listener()
+ {
+ @Override
+ public void becomeLeader()
+ {
+ // Not expected
+ }
+
+ @Override
+ public void stopBeingLeader()
+ {
+ // Not expected
+ }
+
+ };
+
+ leaderSelector.registerListener(listener);
+
+ // Wait a bit for session creation
+ Thread.sleep(500);
+
+ // Unregister should destroy the session
+ leaderSelector.unregisterListener();
+
+ // Verify session was destroyed
+ Assert.assertEquals(SESSION_ID, sessionIdCapture.getValue());
+
+ EasyMock.verify(mockConsulClient);
+ }
+
+ private void expectLockOwnershipCheck(String sessionId)
+ {
+ GetValue value = new GetValue();
+ value.setSession(sessionId);
+ Response response = new Response<>(value, 0L, true, 0L);
+ EasyMock.expect(mockConsulClient.getKVValue(
+ EasyMock.eq(LOCK_KEY),
+ EasyMock.isNull(),
+ EasyMock.eq(QueryParams.DEFAULT)
+ ))
+ .andReturn(response)
+ .anyTimes();
+ }
+
+ private void expectSessionInfoCheck(String sessionId)
+ {
+ Session session = new Session();
+ session.setId(sessionId);
+ EasyMock.expect(mockConsulClient.getSessionInfo(
+ EasyMock.eq(sessionId),
+ EasyMock.eq(QueryParams.DEFAULT)
+ ))
+ .andReturn(new Response<>(session, 0L, true, 0L))
+ .anyTimes();
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulServiceIdsTest.java b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulServiceIdsTest.java
new file mode 100644
index 000000000000..b11c5c2e15ca
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/ConsulServiceIdsTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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 org.apache.druid.discovery.DiscoveryDruidNode;
+import org.apache.druid.discovery.NodeRole;
+import org.apache.druid.server.DruidNode;
+import org.joda.time.Duration;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ConsulServiceIdsTest
+{
+ @Test
+ public void testServiceNameAndIdAndKvKey()
+ {
+ ConsulDiscoveryConfig.ServiceConfig serviceConfig = new ConsulDiscoveryConfig.ServiceConfig(
+ "druid",
+ "dc1",
+ null,
+ Duration.standardSeconds(10),
+ Duration.standardSeconds(90)
+ );
+
+ ConsulDiscoveryConfig config = ConsulDiscoveryConfig.create(
+ new ConsulDiscoveryConfig.ConnectionConfig(null, null, null, null, null, null, null),
+ null,
+ serviceConfig,
+ null,
+ null
+ );
+
+ NodeRole role = NodeRole.PEON;
+ String serviceName = ConsulServiceIds.serviceName(config, role);
+ Assert.assertEquals("druid-peon", serviceName);
+
+ DruidNode druidNode = new DruidNode("service", "host", false, 8080, null, true, false);
+ DiscoveryDruidNode discoveryNode = new DiscoveryDruidNode(druidNode, role, null);
+
+ String serviceId = ConsulServiceIds.serviceId(config, discoveryNode);
+ Assert.assertEquals("druid-peon-host-8080", serviceId);
+
+ String kvKey = ConsulServiceIds.nodeKvKey(config, serviceId);
+ Assert.assertEquals("druid/nodes/druid-peon-host-8080", kvKey);
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/TestUtils.java b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/TestUtils.java
new file mode 100644
index 000000000000..1380373e24e8
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/test/java/org/apache/druid/consul/discovery/TestUtils.java
@@ -0,0 +1,224 @@
+/*
+ * 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 org.joda.time.Duration;
+
+/**
+ * Test utilities for Consul discovery extension tests.
+ */
+public class TestUtils
+{
+
+ /**
+ * Creates a minimal ConsulDiscoveryConfig for testing.
+ */
+ public static ConsulDiscoveryConfig minimalConfig()
+ {
+ return builder().build();
+ }
+
+ /**
+ * Builder for ConsulDiscoveryConfig to simplify test configuration.
+ */
+ public static class Builder
+ {
+ private String host = "localhost";
+ private Integer port = 8500;
+ private Duration connectTimeout;
+ private Duration socketTimeout;
+ private ConsulSSLConfig sslClientConfig;
+ private String aclToken;
+ private String basicAuthUser;
+ private String basicAuthPassword;
+ private Boolean allowBasicAuthOverHttp;
+ private String servicePrefix = "test-service";
+ private String datacenter;
+ private java.util.Map serviceTags;
+ private Duration healthCheckInterval;
+ private Duration deregisterAfter;
+ private String coordinatorLeaderLockPath = "test/coordinator/leader";
+ private String overlordLeaderLockPath = "test/overlord/leader";
+ private Duration leaderSessionTtl;
+ private Long leaderMaxErrorRetries;
+ private Duration leaderRetryBackoffMax;
+ private Duration watchSeconds;
+ private Long maxWatchRetries;
+ private Duration watchRetryDelay;
+
+ public Builder host(String host)
+ {
+ this.host = host;
+ return this;
+ }
+
+ public Builder port(int port)
+ {
+ this.port = port;
+ return this;
+ }
+
+ public Builder connectTimeout(Duration connectTimeout)
+ {
+ this.connectTimeout = connectTimeout;
+ return this;
+ }
+
+ public Builder socketTimeout(Duration socketTimeout)
+ {
+ this.socketTimeout = socketTimeout;
+ return this;
+ }
+
+ public Builder sslClientConfig(ConsulSSLConfig sslClientConfig)
+ {
+ this.sslClientConfig = sslClientConfig;
+ return this;
+ }
+
+ public Builder aclToken(String aclToken)
+ {
+ this.aclToken = aclToken;
+ return this;
+ }
+
+ public Builder basicAuthUser(String basicAuthUser)
+ {
+ this.basicAuthUser = basicAuthUser;
+ return this;
+ }
+
+ public Builder basicAuthPassword(String basicAuthPassword)
+ {
+ this.basicAuthPassword = basicAuthPassword;
+ return this;
+ }
+
+ public Builder allowBasicAuthOverHttp(Boolean allowBasicAuthOverHttp)
+ {
+ this.allowBasicAuthOverHttp = allowBasicAuthOverHttp;
+ return this;
+ }
+
+ public Builder servicePrefix(String servicePrefix)
+ {
+ this.servicePrefix = servicePrefix;
+ return this;
+ }
+
+ public Builder datacenter(String datacenter)
+ {
+ this.datacenter = datacenter;
+ return this;
+ }
+
+ public Builder serviceTags(java.util.Map serviceTags)
+ {
+ this.serviceTags = serviceTags;
+ return this;
+ }
+
+ public Builder healthCheckInterval(Duration healthCheckInterval)
+ {
+ this.healthCheckInterval = healthCheckInterval;
+ return this;
+ }
+
+ public Builder deregisterAfter(Duration deregisterAfter)
+ {
+ this.deregisterAfter = deregisterAfter;
+ return this;
+ }
+
+ public Builder coordinatorLeaderLockPath(String coordinatorLeaderLockPath)
+ {
+ this.coordinatorLeaderLockPath = coordinatorLeaderLockPath;
+ return this;
+ }
+
+ public Builder overlordLeaderLockPath(String overlordLeaderLockPath)
+ {
+ this.overlordLeaderLockPath = overlordLeaderLockPath;
+ return this;
+ }
+
+ public Builder leaderSessionTtl(Duration leaderSessionTtl)
+ {
+ this.leaderSessionTtl = leaderSessionTtl;
+ return this;
+ }
+
+ public Builder leaderMaxErrorRetries(Long leaderMaxErrorRetries)
+ {
+ this.leaderMaxErrorRetries = leaderMaxErrorRetries;
+ return this;
+ }
+
+ public Builder leaderRetryBackoffMax(Duration leaderRetryBackoffMax)
+ {
+ this.leaderRetryBackoffMax = leaderRetryBackoffMax;
+ return this;
+ }
+
+ public Builder watchSeconds(Duration watchSeconds)
+ {
+ this.watchSeconds = watchSeconds;
+ return this;
+ }
+
+ public Builder maxWatchRetries(Long maxWatchRetries)
+ {
+ this.maxWatchRetries = maxWatchRetries;
+ return this;
+ }
+
+ public Builder watchRetryDelay(Duration watchRetryDelay)
+ {
+ this.watchRetryDelay = watchRetryDelay;
+ return this;
+ }
+
+ public ConsulDiscoveryConfig build()
+ {
+ return ConsulDiscoveryConfig.create(
+ new ConsulDiscoveryConfig.ConnectionConfig(host, port, connectTimeout, socketTimeout, sslClientConfig, null, null),
+ new ConsulDiscoveryConfig.AuthConfig(aclToken, basicAuthUser, basicAuthPassword, allowBasicAuthOverHttp),
+ new ConsulDiscoveryConfig.ServiceConfig(servicePrefix, datacenter, serviceTags, healthCheckInterval, deregisterAfter),
+ new ConsulDiscoveryConfig.LeaderElectionConfig(
+ coordinatorLeaderLockPath,
+ overlordLeaderLockPath,
+ leaderSessionTtl,
+ leaderMaxErrorRetries,
+ leaderRetryBackoffMax,
+ healthCheckInterval
+ ),
+ new ConsulDiscoveryConfig.WatchConfig(watchSeconds, maxWatchRetries, watchRetryDelay, null)
+ );
+ }
+ }
+
+ /**
+ * Creates a Builder for ConsulDiscoveryConfig.
+ */
+ public static Builder builder()
+ {
+ return new Builder();
+ }
+}
diff --git a/extensions-contrib/consul-extensions/src/test/resources/tls/.gitignore b/extensions-contrib/consul-extensions/src/test/resources/tls/.gitignore
new file mode 100644
index 000000000000..84e54361ffd8
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/test/resources/tls/.gitignore
@@ -0,0 +1,11 @@
+# Generated test certificates - do not commit
+*.pem
+*.srl
+*.p12
+*.jks
+*.crt
+*.key
+*.csr
+*.req
+*.cnf
+consul-config.json
diff --git a/extensions-contrib/consul-extensions/src/test/resources/tls/README.md b/extensions-contrib/consul-extensions/src/test/resources/tls/README.md
new file mode 100644
index 000000000000..4e71e39d1bb8
--- /dev/null
+++ b/extensions-contrib/consul-extensions/src/test/resources/tls/README.md
@@ -0,0 +1,52 @@
+
+
+# Consul TLS/mTLS Testing
+
+**Integration tests for Consul with TLS/mTLS have been moved to the `embedded-tests` module.**
+
+## Running the Tests
+
+```bash
+# Run all Consul discovery tests (Plain HTTP, TLS, and mTLS)
+mvn verify -P docker-tests -pl embedded-tests -Ddruid.testing.consul.enabled=true
+```
+
+## Test Classes
+
+The following test classes in `embedded-tests` provide comprehensive coverage:
+
+- **`ConsulDiscoveryPlainDockerTest`** - Plain HTTP (no encryption)
+- **`ConsulDiscoveryTLSDockerTest`** - Server-side TLS only
+- **`ConsulDiscoveryMTLSDockerTest`** - Mutual TLS (client certificates required)
+
+Each test class:
+- ✅ Automatically generates TLS certificates at runtime using Bouncy Castle
+- ✅ Starts Consul in a Testcontainers Docker container with appropriate security config
+- ✅ Tests service discovery, leader election, and ingestion workflows
+- ✅ Cleans up certificates and containers after tests complete
+
+## Implementation Details
+
+See:
+- `embedded-tests/src/test/java/org/apache/druid/testing/embedded/consul/ConsulClusterResource.java` - Consul container management with TLS support
+- `embedded-tests/src/test/java/org/apache/druid/testing/embedded/docker/BaseConsulDiscoveryDockerTest.java` - Shared test logic
+- `services/src/test/java/org/apache/druid/testing/utils/TLSCertificateGenerator.java` - Certificate generation utility
diff --git a/licenses.yaml b/licenses.yaml
index 63b1922127e4..a89f3df985f3 100644
--- a/licenses.yaml
+++ b/licenses.yaml
@@ -904,7 +904,7 @@ libraries:
name: kubernetes official java client
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 19.0.0
libraries:
@@ -929,7 +929,7 @@ libraries:
name: org.apache.commons commons-collections4
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 4.4
libraries:
@@ -939,7 +939,7 @@ libraries:
name: io.sundr builder-annotations
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 0.22.0
libraries:
@@ -949,7 +949,7 @@ libraries:
name: com.squareup.okio okio
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 1.17.2
libraries:
@@ -959,7 +959,7 @@ libraries:
name: io.gsonfire gson-fire
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 1.8.5
libraries:
@@ -969,7 +969,7 @@ libraries:
name: io.swagger swagger-annotations
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 1.6.12
libraries:
@@ -979,7 +979,7 @@ libraries:
name: io.swagger swagger-annotations
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 1.6.11
libraries:
@@ -989,7 +989,7 @@ libraries:
name: io.swagger swagger-annotations
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 2.8.6
libraries:
@@ -1000,7 +1000,7 @@ libraries:
name: org.bitbucket.b_c jose4j
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 0.9.6
libraries:
@@ -1010,7 +1010,7 @@ libraries:
name: org.joda joda-convert
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 2.2.1
libraries:
@@ -1030,7 +1030,7 @@ libraries:
name: com.squareup.okhttp3 okhttp
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 4.12.0
libraries:
@@ -1041,7 +1041,7 @@ libraries:
name: com.squareup.okhttp3 okhttp logging-interceptor
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 4.11.0
libraries:
@@ -1051,7 +1051,7 @@ libraries:
name: com.squareup.okio okio
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 3.6.0
libraries:
@@ -1062,7 +1062,7 @@ libraries:
name: io.prometheus simpleclient
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 0.16.0
libraries:
@@ -1097,7 +1097,7 @@ libraries:
name: org.yaml snakeyaml
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 2.2
libraries:
@@ -1107,7 +1107,7 @@ libraries:
name: com.flipkart.zjsonpatch zjsonpatch
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 0.4.14
libraries:
@@ -1117,7 +1117,7 @@ libraries:
name: org.bouncycastle bcprov-jdk18on
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: MIT License
version: "1.79"
libraries:
@@ -1130,7 +1130,7 @@ libraries:
name: com.github.vladimir-bukhtoyarov bucket4j-core
license_category: binary
-module: extensions/druid-kubernetes-extensions
+module: extensions-core/kubernetes-extensions
license_name: Apache License version 2.0
version: 7.6.0
libraries:
@@ -6799,3 +6799,14 @@ copyright: Paul Henschel
version: 4.5.5
license_file_path: licenses/bin/zustand.MIT
# Web console modules end
+
+---
+
+name: Consul API
+license_category: binary
+module: extensions-contrib/consul-extensions
+license_name: Apache License version 2.0
+version: 1.4.5
+copyright: Ecwid
+libraries:
+ - com.ecwid.consul: consul-api
diff --git a/pom.xml b/pom.xml
index 7f1328cd41b1..d5fb90ef8b4a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -239,6 +239,7 @@
extensions-contrib/cassandra-storage
extensions-contrib/dropwizard-emitter
extensions-contrib/cloudfiles-extensions
+ extensions-contrib/consul-extensions
extensions-contrib/graphite-emitter
extensions-contrib/distinctcount
extensions-contrib/druid-exact-count-bitmap
@@ -427,6 +428,11 @@
+
+ org.bouncycastle
+ bcprov-jdk18on
+ 1.79
+
org.bouncycastle
bcpkix-jdk18on
@@ -1156,7 +1162,7 @@
org.easymock
easymock
- 5.2.0
+ 5.4.0
test
diff --git a/services/pom.xml b/services/pom.xml
index 1876f6191b45..1ae9176fafa1 100644
--- a/services/pom.xml
+++ b/services/pom.xml
@@ -305,6 +305,16 @@
junit-jupiter-migrationsupport
test
+
+ org.bouncycastle
+ bcprov-jdk18on
+ test
+
+
+ org.bouncycastle
+ bcpkix-jdk18on
+ test
+
org.junit.jupiter
junit-jupiter-params
diff --git a/services/src/test/java/org/apache/druid/testing/utils/TLSCertificateBundle.java b/services/src/test/java/org/apache/druid/testing/utils/TLSCertificateBundle.java
new file mode 100644
index 000000000000..c5587cc2db50
--- /dev/null
+++ b/services/src/test/java/org/apache/druid/testing/utils/TLSCertificateBundle.java
@@ -0,0 +1,121 @@
+/*
+ * 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.testing.utils;
+
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.logger.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Comparator;
+
+/**
+ * Bundle containing paths to generated TLS certificates and keystores.
+ * Used for testing TLS/mTLS connectivity with services like Consul.
+ */
+public class TLSCertificateBundle
+{
+ private static final Logger log = new Logger(TLSCertificateBundle.class);
+
+ private final Path certificateDirectory;
+ private final Path trustStorePath;
+ private final Path keyStorePath;
+
+ public TLSCertificateBundle(Path certDir, Path trustStore, Path keyStore)
+ {
+ this.certificateDirectory = certDir;
+ this.trustStorePath = trustStore;
+ this.keyStorePath = keyStore;
+ }
+
+ /**
+ * Returns the absolute path to the directory containing all certificate files.
+ * This directory can be mounted to Docker containers.
+ */
+ public String getCertificateDirectory()
+ {
+ return certificateDirectory.toAbsolutePath().toString();
+ }
+
+ /**
+ * Returns the absolute path to the PKCS12 truststore containing the CA certificate.
+ * Use this to configure Java clients to trust the server certificate.
+ */
+ public String getTrustStorePath()
+ {
+ return trustStorePath.toAbsolutePath().toString();
+ }
+
+ /**
+ * Returns the absolute path to the PKCS12 keystore containing the client certificate.
+ * Use this to configure Java clients for mutual TLS (mTLS).
+ */
+ public String getKeyStorePath()
+ {
+ return keyStorePath.toAbsolutePath().toString();
+ }
+
+ /**
+ * Returns the password for the truststore and keystore.
+ * Always returns "changeit" for test certificates.
+ */
+ public String getStorePassword()
+ {
+ return "changeit";
+ }
+
+ /**
+ * Cleans up the temporary directory containing all generated certificates.
+ * Should be called after tests complete.
+ */
+ public void cleanup()
+ {
+ try {
+ Files.walk(certificateDirectory)
+ .sorted(Comparator.reverseOrder())
+ .map(Path::toFile)
+ .forEach(File::delete);
+ log.debug("Cleaned up certificate directory: %s", certificateDirectory);
+ }
+ catch (IOException e) {
+ log.warn(e, "Failed to clean up certificate directory: %s", certificateDirectory);
+ }
+ }
+
+ /**
+ * Verifies that all required files exist in the bundle.
+ *
+ * @throws ISE if any required files are missing
+ */
+ public void validate()
+ {
+ if (!Files.exists(certificateDirectory)) {
+ throw new ISE("Certificate directory does not exist: %s", certificateDirectory);
+ }
+ if (!Files.exists(trustStorePath)) {
+ throw new ISE("Truststore does not exist: %s", trustStorePath);
+ }
+ if (!Files.exists(keyStorePath)) {
+ throw new ISE("Keystore does not exist: %s", keyStorePath);
+ }
+ }
+}
diff --git a/services/src/test/java/org/apache/druid/testing/utils/TLSCertificateGenerator.java b/services/src/test/java/org/apache/druid/testing/utils/TLSCertificateGenerator.java
new file mode 100644
index 000000000000..f09461e45b2d
--- /dev/null
+++ b/services/src/test/java/org/apache/druid/testing/utils/TLSCertificateGenerator.java
@@ -0,0 +1,318 @@
+/*
+ * 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.testing.utils;
+
+import org.apache.druid.java.util.common.FileUtils;
+import org.bouncycastle.asn1.x500.X500Name;
+import org.bouncycastle.asn1.x500.X500NameBuilder;
+import org.bouncycastle.asn1.x509.BasicConstraints;
+import org.bouncycastle.asn1.x509.Extension;
+import org.bouncycastle.asn1.x509.GeneralName;
+import org.bouncycastle.asn1.x509.GeneralNames;
+import org.bouncycastle.asn1.x509.KeyUsage;
+import org.bouncycastle.cert.X509v3CertificateBuilder;
+import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter;
+import org.bouncycastle.cert.jcajce.JcaX509v3CertificateBuilder;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.operator.ContentSigner;
+import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
+import org.bouncycastle.util.io.pem.PemObject;
+import org.bouncycastle.util.io.pem.PemWriter;
+
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.KeyStore;
+import java.security.PrivateKey;
+import java.security.SecureRandom;
+import java.security.Security;
+import java.security.cert.Certificate;
+import java.security.cert.X509Certificate;
+import java.util.Date;
+
+/**
+ * Utility class for generating TLS certificates and keystores for testing.
+ * Generates self-signed CA certificates, server certificates with SANs,
+ * and client certificates for mutual TLS.
+ */
+public class TLSCertificateGenerator
+{
+ private static final String SIGNATURE_ALGORITHM = "SHA256withRSA";
+ private static final int KEY_SIZE = 2048;
+ private static final long VALIDITY_DAYS = 365;
+ private static final String STORE_PASSWORD = "changeit";
+
+ static {
+ // Register Bouncy Castle as a security provider
+ if (Security.getProvider(BouncyCastleProvider.PROVIDER_NAME) == null) {
+ Security.addProvider(new BouncyCastleProvider());
+ }
+ }
+
+ private static X500Name createX500Name(String commonName)
+ {
+ X500NameBuilder builder = new X500NameBuilder();
+ builder.addRDN(org.bouncycastle.asn1.x500.style.RFC4519Style.c, "US");
+ builder.addRDN(org.bouncycastle.asn1.x500.style.RFC4519Style.o, "Apache Druid Test");
+ builder.addRDN(org.bouncycastle.asn1.x500.style.RFC4519Style.cn, commonName);
+ return builder.build();
+ }
+
+ /**
+ * Generates a complete set of TLS certificates to a temporary directory.
+ * The directory will contain:
+ * - CA certificate and key (PEM format)
+ * - Server certificate and key with SAN for localhost (PEM format)
+ * - Client certificate and key for mTLS (PEM format)
+ * - Java truststore (PKCS12) with CA certificate
+ * - Java keystore (PKCS12) with client certificate
+ *
+ * Caller is responsible for cleanup via {@link TLSCertificateBundle#cleanup()}.
+ *
+ * @return bundle containing paths to all generated files
+ * @throws Exception if certificate generation fails
+ */
+ public static TLSCertificateBundle generateToTempDirectory() throws Exception
+ {
+ Path tempDir = FileUtils.createTempDir().toPath();
+ // Set readable and executable permissions for all users (required for containers)
+ tempDir.toFile().setReadable(true, false);
+ tempDir.toFile().setExecutable(true, false);
+ return generateToDirectory(tempDir);
+ }
+
+ /**
+ * Generates a complete set of TLS certificates to the specified directory.
+ *
+ * @param directory target directory for certificate files
+ * @return bundle containing paths to all generated files
+ * @throws Exception if certificate generation fails
+ */
+ public static TLSCertificateBundle generateToDirectory(Path directory) throws Exception
+ {
+ // Generate CA certificate and key
+ KeyPair caKeyPair = generateKeyPair();
+ X509Certificate caCert = generateCACertificate(caKeyPair);
+
+ // Generate server certificate with SAN for localhost and additional hostnames
+ KeyPair serverKeyPair = generateKeyPair();
+ X509Certificate serverCert = generateServerCertificate(
+ serverKeyPair,
+ caKeyPair.getPrivate(),
+ new String[]{"localhost", "127.0.0.1", "server.dc1", "consul"}
+ );
+
+ // Generate client certificate for mTLS
+ KeyPair clientKeyPair = generateKeyPair();
+ X509Certificate clientCert = generateClientCertificate(
+ clientKeyPair,
+ caKeyPair.getPrivate(),
+ "druid-client"
+ );
+
+ // Write PEM files (for Consul and other services)
+ writePEM(directory.resolve("ca-cert.pem"), "CERTIFICATE", caCert.getEncoded());
+ writePEM(directory.resolve("ca-key.pem"), "PRIVATE KEY", caKeyPair.getPrivate().getEncoded());
+ writePEM(directory.resolve("consul-server-cert.pem"), "CERTIFICATE", serverCert.getEncoded());
+ writePEM(directory.resolve("consul-server-key.pem"), "PRIVATE KEY", serverKeyPair.getPrivate().getEncoded());
+ writePEM(directory.resolve("client-cert.pem"), "CERTIFICATE", clientCert.getEncoded());
+ writePEM(directory.resolve("client-key.pem"), "PRIVATE KEY", clientKeyPair.getPrivate().getEncoded());
+
+ // Create Java keystores (for Druid clients)
+ Path trustStore = createTrustStore(directory, caCert);
+ Path keyStore = createKeyStore(directory, clientCert, clientKeyPair.getPrivate());
+
+ return new TLSCertificateBundle(directory, trustStore, keyStore);
+ }
+
+ private static KeyPair generateKeyPair() throws Exception
+ {
+ KeyPairGenerator keyGen = KeyPairGenerator.getInstance("RSA");
+ keyGen.initialize(KEY_SIZE, new SecureRandom());
+ return keyGen.generateKeyPair();
+ }
+
+ private static X509v3CertificateBuilder createCertificateBuilder(X500Name subject, KeyPair keyPair)
+ {
+ X500Name issuer = createX500Name("Test CA");
+ BigInteger serial = BigInteger.valueOf(System.currentTimeMillis());
+ Date notBefore = new Date();
+ Date notAfter = new Date(notBefore.getTime() + VALIDITY_DAYS * 24 * 60 * 60 * 1000L);
+
+ return new JcaX509v3CertificateBuilder(
+ issuer,
+ serial,
+ notBefore,
+ notAfter,
+ subject,
+ keyPair.getPublic()
+ );
+ }
+
+ private static X509Certificate buildCertificate(X509v3CertificateBuilder certBuilder, PrivateKey signingKey) throws Exception
+ {
+ ContentSigner signer = new JcaContentSignerBuilder(SIGNATURE_ALGORITHM)
+ .setProvider(BouncyCastleProvider.PROVIDER_NAME)
+ .build(signingKey);
+
+ return new JcaX509CertificateConverter()
+ .setProvider(BouncyCastleProvider.PROVIDER_NAME)
+ .getCertificate(certBuilder.build(signer));
+ }
+
+ private static X509Certificate generateCACertificate(KeyPair keyPair) throws Exception
+ {
+ X500Name subject = createX500Name("Test CA");
+ X509v3CertificateBuilder certBuilder = createCertificateBuilder(subject, keyPair);
+
+ certBuilder.addExtension(Extension.basicConstraints, true, new BasicConstraints(true));
+ certBuilder.addExtension(Extension.keyUsage, true, new KeyUsage(KeyUsage.keyCertSign | KeyUsage.cRLSign));
+
+ return buildCertificate(certBuilder, keyPair.getPrivate());
+ }
+
+ private static boolean isIpAddress(String hostname)
+ {
+ if (hostname == null || hostname.isEmpty()) {
+ return false;
+ }
+ String[] parts = hostname.split("\\.");
+ if (parts.length != 4) {
+ return false;
+ }
+ try {
+ for (String part : parts) {
+ int value = Integer.parseInt(part);
+ if (value < 0 || value > 255) {
+ return false;
+ }
+ }
+ return true;
+ }
+ catch (NumberFormatException e) {
+ return false;
+ }
+ }
+
+ private static X509Certificate generateServerCertificate(
+ KeyPair keyPair,
+ PrivateKey caKey,
+ String[] hostnames
+ ) throws Exception
+ {
+ X500Name subject = createX500Name(hostnames[0]);
+ X509v3CertificateBuilder certBuilder = createCertificateBuilder(subject, keyPair);
+
+ GeneralName[] altNames = new GeneralName[hostnames.length];
+ for (int i = 0; i < hostnames.length; i++) {
+ String hostname = hostnames[i];
+ if (isIpAddress(hostname)) {
+ altNames[i] = new GeneralName(GeneralName.iPAddress, hostname);
+ } else {
+ altNames[i] = new GeneralName(GeneralName.dNSName, hostname);
+ }
+ }
+ certBuilder.addExtension(Extension.subjectAlternativeName, false, new GeneralNames(altNames));
+ certBuilder.addExtension(Extension.keyUsage, true, new KeyUsage(KeyUsage.digitalSignature | KeyUsage.keyEncipherment));
+
+ return buildCertificate(certBuilder, caKey);
+ }
+
+ private static X509Certificate generateClientCertificate(
+ KeyPair keyPair,
+ PrivateKey caKey,
+ String commonName
+ ) throws Exception
+ {
+ X500Name subject = createX500Name(commonName);
+ X509v3CertificateBuilder certBuilder = createCertificateBuilder(subject, keyPair);
+
+ certBuilder.addExtension(Extension.keyUsage, true, new KeyUsage(KeyUsage.digitalSignature | KeyUsage.keyEncipherment));
+
+ return buildCertificate(certBuilder, caKey);
+ }
+
+ private static void writePEM(Path path, String type, byte[] content) throws Exception
+ {
+ try (Writer fileWriter = new OutputStreamWriter(Files.newOutputStream(path), StandardCharsets.UTF_8);
+ PemWriter pemWriter = new PemWriter(fileWriter)) {
+ pemWriter.writeObject(new PemObject(type, content));
+ }
+ setWorldReadable(path);
+ }
+
+ private static Path createTrustStore(Path directory, X509Certificate caCert) throws Exception
+ {
+ Path trustStorePath = directory.resolve("truststore.p12");
+ KeyStore trustStore = KeyStore.getInstance("PKCS12");
+ trustStore.load(null, null);
+ trustStore.setCertificateEntry("ca", caCert);
+
+ try (FileOutputStream fos = new FileOutputStream(trustStorePath.toFile())) {
+ trustStore.store(fos, STORE_PASSWORD.toCharArray());
+ }
+
+ setWorldReadable(trustStorePath);
+
+ return trustStorePath;
+ }
+
+ private static Path createKeyStore(
+ Path directory,
+ X509Certificate cert,
+ PrivateKey key
+ ) throws Exception
+ {
+ Path keyStorePath = directory.resolve("client.p12");
+ KeyStore keyStore = KeyStore.getInstance("PKCS12");
+ keyStore.load(null, null);
+ keyStore.setKeyEntry(
+ "client",
+ key,
+ STORE_PASSWORD.toCharArray(),
+ new Certificate[]{cert}
+ );
+
+ try (FileOutputStream fos = new FileOutputStream(keyStorePath.toFile())) {
+ keyStore.store(fos, STORE_PASSWORD.toCharArray());
+ }
+
+ setWorldReadable(keyStorePath);
+
+ return keyStorePath;
+ }
+
+ private static void setWorldReadable(Path path)
+ {
+ try {
+ Files.setPosixFilePermissions(path, PosixFilePermissions.fromString("rw-r--r--"));
+ }
+ catch (Exception e) {
+ path.toFile().setReadable(true, false);
+ }
+ }
+}
diff --git a/website/.spelling b/website/.spelling
index a6e7cf573889..2f7b2fe82dfc 100644
--- a/website/.spelling
+++ b/website/.spelling
@@ -2442,6 +2442,151 @@ DDSketch
druid-ddsketch
numBins
ZGC
+mTLS
+BouncyCastle
+PKCS12
+datacenter
+deregister
+deregisters
+deregistration
+HashiCorp
+Consul
+Consul's
+sessionId
+watchSeconds
+healthCheckInterval
+servicePrefix
+aclToken
+coordinatorLeaderLockPath
+overlordLeaderLockPath
+deregisterAfter
+tlsCertificatePath
+tlsKeyPath
+tlsCaCertPath
+tlsVerifyHostname
+basicAuthUser
+basicAuthPassword
+enableTls
+druid-consul-extensions
+KV
+TTL
+metadata
+healthcare
+Health
+HealthCheck
+production
+format
+ACL
+RBAC
+NTP
+PEM
+PEM-encoded
+latency
+ms
+classpath
+10s
+limits
+50ms
+512KB
+Liveness
+PKCS#1
+PKCS#8
+RSA
+CAs
+PEM
+unencrypted
+Keystores
+keystores
+TrustManagerFactory
+KeyManagerFactory
+AZ
+AZs
+DCs
+3x
+30s
+druid.discovery.consul.basicAuthPassword
+druid.discovery.consul.aclToken
+trustStoreType
+sslClientConfig
+druid.discovery.consul.port
+druid.discovery.consul.servicePrefix
+druid.discovery.consul.basicAuthUser
+druid.discovery.consul.sslClientConfig.protocol
+TLSv1.3
+druid.discovery.consul.sslClientConfig.trustStoreType
+PKCS12
+druid.discovery.consul.sslClientConfig.trustStorePath
+druid.discovery.consul.sslClientConfig.trustStorePassword
+druid.discovery.consul.sslClientConfig.validateHostnames
+druid.discovery.consul.sslClientConfig.keyStoreType
+druid.discovery.consul.sslClientConfig.keyStorePath
+druid.discovery.consul.sslClientConfig.keyStorePassword
+druid.discovery.consul.sslClientConfig.certAlias
+openssl
+pkcs12
+inkey
+passout
+storetype
+storepass
+noprompt
+client-cert.pem
+client-key.pem
+client-keystore.p12
+consul-ca.pem
+truststore.p12
+ca.pem
+druid.discovery.consul.enableTls
+druid.discovery.consul.tlsCaCertPath
+druid.discovery.consul.tlsCaCertPath
+druid.discovery.consul.enableTls
+druid.discovery.consul.tlsCertificatePath
+druid.discovery.consul.tlsKeyPath
+pem
+cert.pem
+key.pem
+hcl
+service_prefix
+key_prefix
+session_prefix
+druid.discovery.consul.healthCheckInterval
+PT30S
+druid.discovery.consul.deregisterAfter
+PT180S
+druid.discovery.consul.watchSeconds
+discovery.consul.watchSeconds
+PT120S
+DiscoveryDruidNode
+druid.discovery.type
+druid.discovery.consul.host
+druid.discovery.consul.port
+druid.discovery.consul.servicePrefix
+druid.coordinator.selector.type
+druid.indexer.selector.type
+druid.zk.service.host
+druid.zk.paths.base
+druid-consul-extensions
+druid.extensions
+druid-consul-extensions
+fro
+p12
+ui
+druid_consul_watch_error
+promql
+healthcheck
+5m
+druid_consul_leader_renew_fail
+druid.discovery.consul.serviceTags.az
+us-east-1a
+druid.discovery.consul.serviceTags.tier
+druid.discovery.consul.serviceTags.version
+180s
+120s
+TTLs
+backoff
+SRE
+reachability
+ownership_mismatch
+giveup
- ../docs/development/extensions-contrib/spectator-histogram.md
SpectatorHistogram
@@ -2471,3 +2616,10 @@ TableSpec
- ../docs/development/extensions-contrib/prometheus.md
TTL
+- ../docs/development/extensions-contrib/consul.md
+minimum
+than
+75s
+maxWatchRetries
+cleartext
+nginx