From cc97bd7bcf9ea71b30d9305277cf6e7f04962dc3 Mon Sep 17 00:00:00 2001 From: chenhang Date: Sat, 11 Dec 2021 15:15:32 +0800 Subject: [PATCH 01/18] first commit for service url provider design --- .../pulsar/client/api/ServiceUrlProvider.java | 7 + .../client/impl/AutoClusterFailover.java | 225 ++++++++++++++++++ .../impl/ControlledClusterFailover.java | 113 +++++++++ .../pulsar/client/impl/PulsarClientImpl.java | 6 + 4 files changed, 351 insertions(+) create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java index d4305ac2c3dd3..12e363e7b8f57 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java @@ -51,4 +51,11 @@ public interface ServiceUrlProvider { */ String getServiceUrl(); + /** + * Close the resource that the provider allocated. + * + */ + default void close() { + // do nothing + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java new file mode 100644 index 0000000000000..f0716dd1fbea3 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -0,0 +1,225 @@ +/** + * 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.pulsar.client.impl; + +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.ServiceUrlProvider; + +@Slf4j +public class AutoClusterFailover implements ServiceUrlProvider { + private PulsarClient pulsarClient; + private volatile String currentPulsarServiceUrl; + private final String primary; + private final String secondary; + private final long failoverDelayMs; + private final long switchBackDelayMs; + private final Timer timer; + private volatile long primaryFailedTimestamp; + private long primaryRecoverTimestamp; + private long secondaryFailedTimestamp; + private final int timeout = 30_000; + + private AutoClusterFailover(String primary, String secondary, long failoverDelayMs, long switchBackDelayMs) { + this.primary = primary; + this.secondary = secondary; + this.failoverDelayMs = failoverDelayMs; + this.switchBackDelayMs = switchBackDelayMs; + this.primaryFailedTimestamp = -1; + this.primaryRecoverTimestamp = -1; + this.secondaryFailedTimestamp = -1; + this.timer = new Timer("pulsar-service-provider"); + } + + @Override + public void initialize(PulsarClient client) { + this.pulsarClient = client; + this.currentPulsarServiceUrl = primary; + + // start to probe primary cluster active or not + this.timer.scheduleAtFixedRate(new TimerTask() { + @Override + public void run() { + // current pulsar serviceUrl is primary + if (currentPulsarServiceUrl.equals(primary)) { + if (probeAvailable(primary, timeout)) { + primaryFailedTimestamp = -1; + return; + } + + if (primaryFailedTimestamp == -1) { + primaryFailedTimestamp = System.currentTimeMillis(); + } else if (System.currentTimeMillis() - primaryFailedTimestamp < failoverDelayMs) { + return; + } else if (probeAvailable(secondary, timeout)){ + log.info("Current Pulsar service is primary: {}, it has been down for {} ms, " + + "switch to the secondary service: {}. The first primary service down at: {}", + currentPulsarServiceUrl, System.currentTimeMillis() - primaryFailedTimestamp, + secondary, primaryFailedTimestamp); + try { + pulsarClient.updateServiceUrl(secondary); + currentPulsarServiceUrl = secondary; + } catch (PulsarClientException e) { + log.error("Failed to switch to secondary service URL ", e); + } + } else { + log.error("Current Pulsar service is primary: {}, it has been down for {} ms. " + + "Failed to switch to secondary service URL, " + + "because secondary service URL is not available", + currentPulsarServiceUrl, System.currentTimeMillis() - primaryFailedTimestamp); + } + } else { // current pulsar service URL is secondary, probe whether we need to switch back to primary. + if (!probeAvailable(currentPulsarServiceUrl, timeout)) { + if (secondaryFailedTimestamp == -1) { + secondaryFailedTimestamp = System.currentTimeMillis(); + } else if (System.currentTimeMillis() - secondaryFailedTimestamp >= failoverDelayMs + && probeAvailable(primary, timeout)) { + log.info("Current Pulsar service is secondary: {}, it has been down for {} ms, " + + "switch back to primary service: {}", currentPulsarServiceUrl, + System.currentTimeMillis() - secondaryFailedTimestamp, primary); + try { + pulsarClient.updateServiceUrl(primary); + currentPulsarServiceUrl = primary; + return; + } catch (PulsarClientException e) { + log.error("Current Pulsar service is secondary: {}, it has been down for {} ms. " + + "Failed to switch to secondary service URL ", + currentPulsarServiceUrl, + System.currentTimeMillis() - secondaryFailedTimestamp, e); + } + } + + return; + } + + secondaryFailedTimestamp = -1; + + if (!probeAvailable(primary, timeout)) { + primaryRecoverTimestamp = -1; + return; + } + if (primaryRecoverTimestamp == -1) { + primaryRecoverTimestamp = System.currentTimeMillis(); + } else if (System.currentTimeMillis() - primaryRecoverTimestamp >= switchBackDelayMs) { + log.info("Current Pulsar service is secondary: {}, " + + "the primary service: {} has been recover for {} ms, " + + "switch back to the primary service", + currentPulsarServiceUrl, primary, System.currentTimeMillis() - primaryRecoverTimestamp); + try { + pulsarClient.updateServiceUrl(primary); + currentPulsarServiceUrl = primary; + } catch (PulsarClientException e) { + log.error("Current Pulsar service is secondary: {}, " + + "failed to switch back to primary service URL ", currentPulsarServiceUrl, e); + } + } + } + } + }, 30_000, 30_000); + + } + + @Override + public String getServiceUrl() { + return this.currentPulsarServiceUrl; + } + + @Override + public void close() { + this.timer.cancel(); + } + + private boolean probeAvailable(String url, int timeout) { + String hostAndPort = parseHostAndPort(url); + try { + Socket socket = new Socket(); + socket.connect(new InetSocketAddress(parseHost(hostAndPort), parsePort(hostAndPort)), timeout); + socket.close(); + return true; + } catch (Exception e) { + return false; + } + } + + private static String parseHostAndPort(String candidateBroker) { + int uriSeparatorPos = candidateBroker.indexOf("://"); + if (uriSeparatorPos == -1) { + throw new IllegalArgumentException("'" + candidateBroker + "' isn't an URI."); + } + return candidateBroker.substring(uriSeparatorPos + 3); + } + + private static String parseHost(String hostAndPort) { + int portSeparatorPos = hostAndPort.indexOf(":"); + if (portSeparatorPos == -1) { + throw new IllegalArgumentException("'" + hostAndPort + "' isn't an URI."); + } + return hostAndPort.substring(0, portSeparatorPos); + } + + private static Integer parsePort(String hostAndPort) { + int portSeparatorPos = hostAndPort.indexOf(":"); + if (portSeparatorPos == -1) { + throw new IllegalArgumentException("'" + hostAndPort + "' isn't an URI."); + } + return Integer.valueOf(hostAndPort.substring(portSeparatorPos+1)); + } + + public static class Builder { + private String primary; + private String secondary; + private long failoverDelayMs; + private long switchBackDelayMs; + + + public Builder primary(String primary) { + this.primary = primary; + return this; + } + + public Builder secondary(String secondary) { + this.secondary = secondary; + return this; + } + + public Builder failoverDelay(int failoverDelay, TimeUnit failoverDelayTimeUnit) { + this.failoverDelayMs = failoverDelayTimeUnit.toMillis(failoverDelay); + return this; + } + + public Builder switchBackDelay(int switchBackDelay, TimeUnit switchBackDelayTimeUnit) { + this.switchBackDelayMs = switchBackDelayTimeUnit.toMillis(switchBackDelay); + return this; + } + + public AutoClusterFailover build() { + return new AutoClusterFailover(primary, secondary, failoverDelayMs, switchBackDelayMs); + } + } + + public static Builder builder() { + return new Builder(); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java new file mode 100644 index 0000000000000..86b4d96facb43 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java @@ -0,0 +1,113 @@ +/** + * 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.pulsar.client.impl; + +import java.io.IOException; +import java.io.InputStream; +import java.net.URL; +import java.net.URLConnection; +import java.nio.charset.StandardCharsets; +import java.util.Timer; +import java.util.TimerTask; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.io.IOUtils; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.ServiceUrlProvider; + +@Slf4j +public class ControlledClusterFailover implements ServiceUrlProvider { + private PulsarClient pulsarClient; + private volatile String currentPulsarServiceUrl; + private final String defaultServiceUrl; + private final URL pulsarUrlProvider; + private final Timer timer; + + private ControlledClusterFailover(String defaultServiceUrl, String urlProvider) throws IOException { + this.defaultServiceUrl = defaultServiceUrl; + this.pulsarUrlProvider = new URL(urlProvider); + this.timer = new Timer("pulsar-service-provider"); + } + + @Override + public void initialize(PulsarClient client) { + this.pulsarClient = client; + this.currentPulsarServiceUrl = defaultServiceUrl; + + // start to check service url every 30 seconds + this.timer.scheduleAtFixedRate(new TimerTask() { + @Override + public void run() { + String newPulsarUrl = null; + try { + newPulsarUrl = fetchServiceUrl(); + if (!currentPulsarServiceUrl.equals(newPulsarUrl)) { + pulsarClient.updateServiceUrl(newPulsarUrl); + currentPulsarServiceUrl = newPulsarUrl; + } + } catch (IOException e) { + log.error("Failed to switch new Pulsar URL, current: {}, new: {}", + currentPulsarServiceUrl, newPulsarUrl, e); + } + } + }, 30_000, 30_000); + } + + private String fetchServiceUrl() throws IOException { + // call the service to get service URL + InputStream inputStream = null; + try { + URLConnection conn = pulsarUrlProvider.openConnection(); + inputStream = conn.getInputStream(); + return new String(IOUtils.toByteArray(inputStream), StandardCharsets.UTF_8); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + @Override + public String getServiceUrl() { + return this.currentPulsarServiceUrl; + } + + @Override + public void close() { + this.timer.cancel(); + } + + public static class Builder{ + private String defaultServiceUrl; + private String urlProvider; + + public Builder defaultServiceUrl(String defaultServiceUrl) { + this.defaultServiceUrl = defaultServiceUrl; + return this; + } + + public Builder urlProvider(String urlProvider) { + this.urlProvider = urlProvider; + return this; + } + + public ControlledClusterFailover build() throws IOException { + return new ControlledClusterFailover(defaultServiceUrl, urlProvider); + } + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index cf0a7509cbc96..103f91cbbae0f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -751,6 +751,12 @@ public void shutdown() throws PulsarClientException { throwable = t; } } + + // close the service url provider allocated resource. + if (conf.getServiceUrlProvider() != null) { + conf.getServiceUrlProvider().close(); + } + try { // Shutting down eventLoopGroup separately because in some cases, cnxPool might be using different // eventLoopGroup. From a8f75d54750759db5a70acd91d7532effd610f47 Mon Sep 17 00:00:00 2001 From: chenhang Date: Sat, 11 Dec 2021 15:30:58 +0800 Subject: [PATCH 02/18] fix a bug --- .../client/impl/AutoClusterFailover.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index f0716dd1fbea3..a9c4ce172759c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl; +import com.google.common.base.Strings; import java.net.InetSocketAddress; import java.net.Socket; import java.util.Timer; @@ -152,8 +153,12 @@ public void close() { } private boolean probeAvailable(String url, int timeout) { - String hostAndPort = parseHostAndPort(url); try { + String hostAndPort = parseHostAndPort(url); + if (Strings.isNullOrEmpty(hostAndPort)) { + return false; + } + Socket socket = new Socket(); socket.connect(new InetSocketAddress(parseHost(hostAndPort), parsePort(hostAndPort)), timeout); socket.close(); @@ -163,12 +168,16 @@ private boolean probeAvailable(String url, int timeout) { } } - private static String parseHostAndPort(String candidateBroker) { - int uriSeparatorPos = candidateBroker.indexOf("://"); + private static String parseHostAndPort(String url) { + if (Strings.isNullOrEmpty(url) || !url.startsWith("pulsar")) { + throw new IllegalArgumentException("'" + url + "' isn't an Pulsar service URL"); + } + + int uriSeparatorPos = url.indexOf("://"); if (uriSeparatorPos == -1) { - throw new IllegalArgumentException("'" + candidateBroker + "' isn't an URI."); + throw new IllegalArgumentException("'" + url + "' isn't an URI."); } - return candidateBroker.substring(uriSeparatorPos + 3); + return url.substring(uriSeparatorPos + 3); } private static String parseHost(String hostAndPort) { From 37fd325dc8ac303393279aacdd2d781223db79d6 Mon Sep 17 00:00:00 2001 From: chenhang Date: Tue, 14 Dec 2021 22:51:57 +0800 Subject: [PATCH 03/18] fix a bug --- .../org/apache/pulsar/client/impl/AutoClusterFailover.java | 2 +- .../pulsar/client/impl/ControlledClusterFailover.java | 7 ++++++- .../org/apache/pulsar/client/impl/PulsarClientImpl.java | 2 +- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index a9c4ce172759c..1f1a82ec1efa3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -48,6 +48,7 @@ private AutoClusterFailover(String primary, String secondary, long failoverDelay this.secondary = secondary; this.failoverDelayMs = failoverDelayMs; this.switchBackDelayMs = switchBackDelayMs; + this.currentPulsarServiceUrl = primary; this.primaryFailedTimestamp = -1; this.primaryRecoverTimestamp = -1; this.secondaryFailedTimestamp = -1; @@ -57,7 +58,6 @@ private AutoClusterFailover(String primary, String secondary, long failoverDelay @Override public void initialize(PulsarClient client) { this.pulsarClient = client; - this.currentPulsarServiceUrl = primary; // start to probe primary cluster active or not this.timer.scheduleAtFixedRate(new TimerTask() { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java index 86b4d96facb43..eb931d70102de 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java @@ -40,6 +40,7 @@ public class ControlledClusterFailover implements ServiceUrlProvider { private ControlledClusterFailover(String defaultServiceUrl, String urlProvider) throws IOException { this.defaultServiceUrl = defaultServiceUrl; + this.currentPulsarServiceUrl = defaultServiceUrl; this.pulsarUrlProvider = new URL(urlProvider); this.timer = new Timer("pulsar-service-provider"); } @@ -47,7 +48,6 @@ private ControlledClusterFailover(String defaultServiceUrl, String urlProvider) @Override public void initialize(PulsarClient client) { this.pulsarClient = client; - this.currentPulsarServiceUrl = defaultServiceUrl; // start to check service url every 30 seconds this.timer.scheduleAtFixedRate(new TimerTask() { @@ -57,6 +57,7 @@ public void run() { try { newPulsarUrl = fetchServiceUrl(); if (!currentPulsarServiceUrl.equals(newPulsarUrl)) { + log.info("Switch Pulsar service url from {} to {}", currentPulsarServiceUrl, newPulsarUrl); pulsarClient.updateServiceUrl(newPulsarUrl); currentPulsarServiceUrl = newPulsarUrl; } @@ -110,4 +111,8 @@ public ControlledClusterFailover build() throws IOException { return new ControlledClusterFailover(defaultServiceUrl, urlProvider); } } + + public static Builder builder() { + return new Builder(); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 103f91cbbae0f..a4c44219a25a0 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -784,7 +784,7 @@ public void shutdown() throws PulsarClientException { } catch (PulsarClientException e) { throwable = e; } - if (conf != null && conf.getAuthentication() != null) { + if (conf.getAuthentication() != null) { try { conf.getAuthentication().close(); } catch (Throwable t) { From f362c00d627ebcf96c30e53f7635de0c375e9c50 Mon Sep 17 00:00:00 2001 From: chenhang Date: Thu, 16 Dec 2021 18:27:05 +0800 Subject: [PATCH 04/18] simplify code and add Builder interface in pulsar-client-api --- .../api/AutoClusterFailoverBuilder.java | 74 +++++++ .../api/ControlledClusterFailoverBuilder.java | 56 +++++ .../pulsar/client/api/ServiceUrlProvider.java | 3 +- .../client/impl/AutoClusterFailover.java | 209 +++++++++--------- .../impl/ControlledClusterFailover.java | 58 ++--- 5 files changed, 263 insertions(+), 137 deletions(-) create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java new file mode 100644 index 0000000000000..a7639d4001f4e --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java @@ -0,0 +1,74 @@ +/** + * 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.pulsar.client.api; + +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.common.classification.InterfaceAudience; +import org.apache.pulsar.common.classification.InterfaceStability; + +/** + * {@link AutoClusterFailoverBuilder} is used to configure and create instance of {@link ServiceUrlProvider} + * + * @since 2.10.0 + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface AutoClusterFailoverBuilder { + /** + * Set the primary service url. + * + * @param primary + * @return + */ + AutoClusterFailoverBuilder primary(String primary); + + /** + * Set the secondary service url. + * + * @param secondary + * @return + */ + AutoClusterFailoverBuilder secondary(String secondary); + + /** + * Set the switch failoverDelay. When one cluster failed longer than failoverDelay, it will trigger cluster switch. + * + * @param failoverDelay + * @param timeUnit + * @return + */ + AutoClusterFailoverBuilder failoverDelay(int failoverDelay, TimeUnit timeUnit); + + /** + * Set the switchBackDelay. When switched to the secondary cluster, and after the primary cluster comes back, + * it will wait for switchBackDelay to switch back to the primary cluster. + * + * @param switchBackDelay + * @param timeUnit + * @return + */ + AutoClusterFailoverBuilder switchBackDelay(int switchBackDelay, TimeUnit timeUnit); + + /** + * Build the ServiceUrlProvider instance. + * + * @return + */ + ServiceUrlProvider build(); +} diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java new file mode 100644 index 0000000000000..051b39d143bd0 --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java @@ -0,0 +1,56 @@ +/** + * 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.pulsar.client.api; + +import java.io.IOException; +import org.apache.pulsar.common.classification.InterfaceAudience; +import org.apache.pulsar.common.classification.InterfaceStability; + +/** + * {@link ControlledClusterFailoverBuilder} is used to configure and create instance of {@link ServiceUrlProvider} + * + * @since 2.10.0 + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface ControlledClusterFailoverBuilder { + /** + * Set default service url. + * + * @param serviceUrl + * @return + */ + ControlledClusterFailoverBuilder defaultServiceUrl(String serviceUrl); + + /** + * Set the service url provider. ServiceUrlProvider will fetch serviceUrl from urlProvider periodically. + * + * @param urlProvider + * @return + */ + ControlledClusterFailoverBuilder urlProvider(String urlProvider); + + /** + * Build the ServiceUrlProvider instance. + * + * @return + * @throws IOException + */ + ServiceUrlProvider build() throws IOException; +} diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java index 12e363e7b8f57..3d3d799d2dc5b 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java @@ -31,7 +31,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public interface ServiceUrlProvider { +public interface ServiceUrlProvider extends AutoCloseable { /** * Initialize the service url provider with Pulsar client instance. @@ -55,6 +55,7 @@ public interface ServiceUrlProvider { * Close the resource that the provider allocated. * */ + @Override default void close() { // do nothing } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index 1f1a82ec1efa3..3843b3b9384ce 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -18,13 +18,16 @@ */ package org.apache.pulsar.client.impl; +import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; import com.google.common.base.Strings; +import io.netty.util.concurrent.DefaultThreadFactory; import java.net.InetSocketAddress; import java.net.Socket; -import java.util.Timer; -import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.AutoClusterFailoverBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.ServiceUrlProvider; @@ -35,24 +38,23 @@ public class AutoClusterFailover implements ServiceUrlProvider { private volatile String currentPulsarServiceUrl; private final String primary; private final String secondary; - private final long failoverDelayMs; - private final long switchBackDelayMs; - private final Timer timer; - private volatile long primaryFailedTimestamp; - private long primaryRecoverTimestamp; - private long secondaryFailedTimestamp; - private final int timeout = 30_000; - - private AutoClusterFailover(String primary, String secondary, long failoverDelayMs, long switchBackDelayMs) { + private final long failoverDelayNs; + private final long switchBackDelayNs; + private final ScheduledExecutorService executor; + private long recoverTimestamp; + private long failedTimestamp; + private final int TIMEOUT = 30_000; + + private AutoClusterFailover(String primary, String secondary, long failoverDelayNs, long switchBackDelayNs) { this.primary = primary; this.secondary = secondary; - this.failoverDelayMs = failoverDelayMs; - this.switchBackDelayMs = switchBackDelayMs; + this.failoverDelayNs = failoverDelayNs; + this.switchBackDelayNs = switchBackDelayNs; this.currentPulsarServiceUrl = primary; - this.primaryFailedTimestamp = -1; - this.primaryRecoverTimestamp = -1; - this.secondaryFailedTimestamp = -1; - this.timer = new Timer("pulsar-service-provider"); + this.recoverTimestamp = -1; + this.failedTimestamp = -1; + this.executor = Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("pulsar-service-provider")); } @Override @@ -60,85 +62,17 @@ public void initialize(PulsarClient client) { this.pulsarClient = client; // start to probe primary cluster active or not - this.timer.scheduleAtFixedRate(new TimerTask() { - @Override - public void run() { - // current pulsar serviceUrl is primary - if (currentPulsarServiceUrl.equals(primary)) { - if (probeAvailable(primary, timeout)) { - primaryFailedTimestamp = -1; - return; - } - - if (primaryFailedTimestamp == -1) { - primaryFailedTimestamp = System.currentTimeMillis(); - } else if (System.currentTimeMillis() - primaryFailedTimestamp < failoverDelayMs) { - return; - } else if (probeAvailable(secondary, timeout)){ - log.info("Current Pulsar service is primary: {}, it has been down for {} ms, " - + "switch to the secondary service: {}. The first primary service down at: {}", - currentPulsarServiceUrl, System.currentTimeMillis() - primaryFailedTimestamp, - secondary, primaryFailedTimestamp); - try { - pulsarClient.updateServiceUrl(secondary); - currentPulsarServiceUrl = secondary; - } catch (PulsarClientException e) { - log.error("Failed to switch to secondary service URL ", e); - } - } else { - log.error("Current Pulsar service is primary: {}, it has been down for {} ms. " - + "Failed to switch to secondary service URL, " - + "because secondary service URL is not available", - currentPulsarServiceUrl, System.currentTimeMillis() - primaryFailedTimestamp); - } - } else { // current pulsar service URL is secondary, probe whether we need to switch back to primary. - if (!probeAvailable(currentPulsarServiceUrl, timeout)) { - if (secondaryFailedTimestamp == -1) { - secondaryFailedTimestamp = System.currentTimeMillis(); - } else if (System.currentTimeMillis() - secondaryFailedTimestamp >= failoverDelayMs - && probeAvailable(primary, timeout)) { - log.info("Current Pulsar service is secondary: {}, it has been down for {} ms, " - + "switch back to primary service: {}", currentPulsarServiceUrl, - System.currentTimeMillis() - secondaryFailedTimestamp, primary); - try { - pulsarClient.updateServiceUrl(primary); - currentPulsarServiceUrl = primary; - return; - } catch (PulsarClientException e) { - log.error("Current Pulsar service is secondary: {}, it has been down for {} ms. " - + "Failed to switch to secondary service URL ", - currentPulsarServiceUrl, - System.currentTimeMillis() - secondaryFailedTimestamp, e); - } - } - - return; - } - - secondaryFailedTimestamp = -1; - - if (!probeAvailable(primary, timeout)) { - primaryRecoverTimestamp = -1; - return; - } - if (primaryRecoverTimestamp == -1) { - primaryRecoverTimestamp = System.currentTimeMillis(); - } else if (System.currentTimeMillis() - primaryRecoverTimestamp >= switchBackDelayMs) { - log.info("Current Pulsar service is secondary: {}, " - + "the primary service: {} has been recover for {} ms, " - + "switch back to the primary service", - currentPulsarServiceUrl, primary, System.currentTimeMillis() - primaryRecoverTimestamp); - try { - pulsarClient.updateServiceUrl(primary); - currentPulsarServiceUrl = primary; - } catch (PulsarClientException e) { - log.error("Current Pulsar service is secondary: {}, " - + "failed to switch back to primary service URL ", currentPulsarServiceUrl, e); - } - } - } + this.executor.scheduleAtFixedRate(catchingAndLoggingThrowables(() -> { + if (currentPulsarServiceUrl.equals(primary)) { + // current service url is primary, probe whether it is down + probeAndUpdateServiceUrl(secondary); + } else { + // current service url is secondary, probe whether it is down + probeAndUpdateServiceUrl(primary); + // secondary cluster is up, check whether need to switch back to primary + probeAndCheckSwitchBack(primary); } - }, 30_000, 30_000); + }), 30_000, 30_000, TimeUnit.MILLISECONDS); } @@ -149,10 +83,10 @@ public String getServiceUrl() { @Override public void close() { - this.timer.cancel(); + this.executor.shutdown(); } - private boolean probeAvailable(String url, int timeout) { + private boolean probeAvailable(String url) { try { String hostAndPort = parseHostAndPort(url); if (Strings.isNullOrEmpty(hostAndPort)) { @@ -160,10 +94,11 @@ private boolean probeAvailable(String url, int timeout) { } Socket socket = new Socket(); - socket.connect(new InetSocketAddress(parseHost(hostAndPort), parsePort(hostAndPort)), timeout); + socket.connect(new InetSocketAddress(parseHost(hostAndPort), parsePort(hostAndPort)), TIMEOUT); socket.close(); return true; } catch (Exception e) { + log.error("Failed to probe available, url: {}", url, e); return false; } } @@ -196,30 +131,90 @@ private static Integer parsePort(String hostAndPort) { return Integer.valueOf(hostAndPort.substring(portSeparatorPos+1)); } - public static class Builder { + private static Long Ns2Ms(long timeStampNs) { + return timeStampNs / 1000000; + } + + private void updateServiceUrl(String target) { + try { + pulsarClient.updateServiceUrl(target); + currentPulsarServiceUrl = target; + } catch (PulsarClientException e) { + log.error("Current Pulsar service is {}, " + + "failed to switch back to {} ", currentPulsarServiceUrl, target, e); + } + } + + private void probeAndUpdateServiceUrl(String targetServiceUrl) { + if (probeAvailable(currentPulsarServiceUrl)) { + failedTimestamp = -1; + return; + } + + long currentTimestamp = System.nanoTime(); + if (failedTimestamp == -1) { + failedTimestamp = currentTimestamp; + } else if (currentTimestamp - failedTimestamp >= failoverDelayNs) { + if (probeAvailable(targetServiceUrl)) { + log.info("Current Pulsar service is {}, it has been down for {} ms, " + + "switch to the service: {}. The current service down at: {}", + currentPulsarServiceUrl, Ns2Ms(currentTimestamp - failedTimestamp), + targetServiceUrl, Ns2Ms(failedTimestamp)); + updateServiceUrl(targetServiceUrl); + failedTimestamp = -1; + } else { + log.error("Current Pulsar service is {}, it has been down for {} ms. " + + "Failed to switch to service {}, " + + "because it is not available", + currentPulsarServiceUrl, Ns2Ms(currentTimestamp - failedTimestamp), + targetServiceUrl); + } + } + } + + private void probeAndCheckSwitchBack(String target) { + long currentTimestamp = System.nanoTime(); + if (!probeAvailable(target)) { + recoverTimestamp = -1; + return; + } + + if (recoverTimestamp == -1) { + recoverTimestamp = currentTimestamp; + } else if (currentTimestamp - recoverTimestamp >= switchBackDelayNs) { + log.info("Current Pulsar service is secondary: {}, " + + "the primary service: {} has been recover for {} ms, " + + "switch back to the primary service", + currentPulsarServiceUrl, target, Ns2Ms(currentTimestamp - recoverTimestamp)); + updateServiceUrl(target); + recoverTimestamp = -1; + } + } + + public static class AutoClusterFailoverBuilderImpl implements AutoClusterFailoverBuilder { private String primary; private String secondary; private long failoverDelayMs; private long switchBackDelayMs; - public Builder primary(String primary) { + public AutoClusterFailoverBuilder primary(String primary) { this.primary = primary; return this; } - public Builder secondary(String secondary) { + public AutoClusterFailoverBuilder secondary(String secondary) { this.secondary = secondary; return this; } - public Builder failoverDelay(int failoverDelay, TimeUnit failoverDelayTimeUnit) { - this.failoverDelayMs = failoverDelayTimeUnit.toMillis(failoverDelay); + public AutoClusterFailoverBuilder failoverDelay(int failoverDelay, TimeUnit failoverDelayTimeUnit) { + this.failoverDelayMs = failoverDelayTimeUnit.toNanos(failoverDelay); return this; } - public Builder switchBackDelay(int switchBackDelay, TimeUnit switchBackDelayTimeUnit) { - this.switchBackDelayMs = switchBackDelayTimeUnit.toMillis(switchBackDelay); + public AutoClusterFailoverBuilder switchBackDelay(int switchBackDelay, TimeUnit switchBackDelayTimeUnit) { + this.switchBackDelayMs = switchBackDelayTimeUnit.toNanos(switchBackDelay); return this; } @@ -228,7 +223,7 @@ public AutoClusterFailover build() { } } - public static Builder builder() { - return new Builder(); + public static AutoClusterFailoverBuilder builder() { + return new AutoClusterFailoverBuilderImpl(); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java index eb931d70102de..41348f3cea707 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java @@ -18,15 +18,19 @@ */ package org.apache.pulsar.client.impl; +import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; +import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; import java.io.InputStream; import java.net.URL; import java.net.URLConnection; import java.nio.charset.StandardCharsets; -import java.util.Timer; -import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.IOUtils; +import org.apache.pulsar.client.api.ControlledClusterFailoverBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; @@ -34,15 +38,14 @@ public class ControlledClusterFailover implements ServiceUrlProvider { private PulsarClient pulsarClient; private volatile String currentPulsarServiceUrl; - private final String defaultServiceUrl; private final URL pulsarUrlProvider; - private final Timer timer; + private final ScheduledExecutorService executor; private ControlledClusterFailover(String defaultServiceUrl, String urlProvider) throws IOException { - this.defaultServiceUrl = defaultServiceUrl; this.currentPulsarServiceUrl = defaultServiceUrl; this.pulsarUrlProvider = new URL(urlProvider); - this.timer = new Timer("pulsar-service-provider"); + this.executor = Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("pulsar-service-provider")); } @Override @@ -50,23 +53,20 @@ public void initialize(PulsarClient client) { this.pulsarClient = client; // start to check service url every 30 seconds - this.timer.scheduleAtFixedRate(new TimerTask() { - @Override - public void run() { - String newPulsarUrl = null; - try { - newPulsarUrl = fetchServiceUrl(); - if (!currentPulsarServiceUrl.equals(newPulsarUrl)) { - log.info("Switch Pulsar service url from {} to {}", currentPulsarServiceUrl, newPulsarUrl); - pulsarClient.updateServiceUrl(newPulsarUrl); - currentPulsarServiceUrl = newPulsarUrl; - } - } catch (IOException e) { - log.error("Failed to switch new Pulsar URL, current: {}, new: {}", - currentPulsarServiceUrl, newPulsarUrl, e); + this.executor.scheduleAtFixedRate(catchingAndLoggingThrowables(() -> { + String newPulsarUrl = null; + try { + newPulsarUrl = fetchServiceUrl(); + if (!currentPulsarServiceUrl.equals(newPulsarUrl)) { + log.info("Switch Pulsar service url from {} to {}", currentPulsarServiceUrl, newPulsarUrl); + pulsarClient.updateServiceUrl(newPulsarUrl); + currentPulsarServiceUrl = newPulsarUrl; } + } catch (IOException e) { + log.error("Failed to switch new Pulsar URL, current: {}, new: {}", + currentPulsarServiceUrl, newPulsarUrl, e); } - }, 30_000, 30_000); + }), 30_000, 30_000, TimeUnit.MILLISECONDS); } private String fetchServiceUrl() throws IOException { @@ -90,29 +90,29 @@ public String getServiceUrl() { @Override public void close() { - this.timer.cancel(); + this.executor.shutdown(); } - public static class Builder{ + public static class ControlledClusterFailoverBuilderImpl implements ControlledClusterFailoverBuilder { private String defaultServiceUrl; private String urlProvider; - public Builder defaultServiceUrl(String defaultServiceUrl) { - this.defaultServiceUrl = defaultServiceUrl; + public ControlledClusterFailoverBuilder defaultServiceUrl(String serviceUrl) { + this.defaultServiceUrl = serviceUrl; return this; } - public Builder urlProvider(String urlProvider) { + public ControlledClusterFailoverBuilder urlProvider(String urlProvider) { this.urlProvider = urlProvider; return this; } - public ControlledClusterFailover build() throws IOException { + public ServiceUrlProvider build() throws IOException { return new ControlledClusterFailover(defaultServiceUrl, urlProvider); } } - public static Builder builder() { - return new Builder(); + public static ControlledClusterFailoverBuilder builder() { + return new ControlledClusterFailoverBuilderImpl(); } } From 5ef9265590982cbfbf19327aa709775f7e6d639f Mon Sep 17 00:00:00 2001 From: chenhang Date: Thu, 16 Dec 2021 19:11:48 +0800 Subject: [PATCH 05/18] add tests --- .../api/AutoClusterFailoverBuilder.java | 4 +- .../client/impl/AutoClusterFailover.java | 18 ++++--- .../impl/ControlledClusterFailover.java | 2 + .../client/impl/AutoClusterFailoverTest.java | 49 +++++++++++++++++++ .../impl/ControlledClusterFailoverTest.java | 45 +++++++++++++++++ 5 files changed, 108 insertions(+), 10 deletions(-) create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java index a7639d4001f4e..94567245651f6 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java @@ -53,7 +53,7 @@ public interface AutoClusterFailoverBuilder { * @param timeUnit * @return */ - AutoClusterFailoverBuilder failoverDelay(int failoverDelay, TimeUnit timeUnit); + AutoClusterFailoverBuilder failoverDelay(long failoverDelay, TimeUnit timeUnit); /** * Set the switchBackDelay. When switched to the secondary cluster, and after the primary cluster comes back, @@ -63,7 +63,7 @@ public interface AutoClusterFailoverBuilder { * @param timeUnit * @return */ - AutoClusterFailoverBuilder switchBackDelay(int switchBackDelay, TimeUnit timeUnit); + AutoClusterFailoverBuilder switchBackDelay(long switchBackDelay, TimeUnit timeUnit); /** * Build the ServiceUrlProvider instance. diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index 3843b3b9384ce..1c7363b8b288d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -26,6 +26,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import lombok.Data; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.AutoClusterFailoverBuilder; import org.apache.pulsar.client.api.PulsarClient; @@ -33,6 +34,7 @@ import org.apache.pulsar.client.api.ServiceUrlProvider; @Slf4j +@Data public class AutoClusterFailover implements ServiceUrlProvider { private PulsarClient pulsarClient; private volatile String currentPulsarServiceUrl; @@ -194,8 +196,8 @@ private void probeAndCheckSwitchBack(String target) { public static class AutoClusterFailoverBuilderImpl implements AutoClusterFailoverBuilder { private String primary; private String secondary; - private long failoverDelayMs; - private long switchBackDelayMs; + private long failoverDelayNs; + private long switchBackDelayNs; public AutoClusterFailoverBuilder primary(String primary) { @@ -208,18 +210,18 @@ public AutoClusterFailoverBuilder secondary(String secondary) { return this; } - public AutoClusterFailoverBuilder failoverDelay(int failoverDelay, TimeUnit failoverDelayTimeUnit) { - this.failoverDelayMs = failoverDelayTimeUnit.toNanos(failoverDelay); + public AutoClusterFailoverBuilder failoverDelay(long failoverDelay, TimeUnit timeUnit) { + this.failoverDelayNs = timeUnit.toNanos(failoverDelay); return this; } - public AutoClusterFailoverBuilder switchBackDelay(int switchBackDelay, TimeUnit switchBackDelayTimeUnit) { - this.switchBackDelayMs = switchBackDelayTimeUnit.toNanos(switchBackDelay); + public AutoClusterFailoverBuilder switchBackDelay(long switchBackDelay, TimeUnit timeUnit) { + this.switchBackDelayNs = timeUnit.toNanos(switchBackDelay); return this; } - public AutoClusterFailover build() { - return new AutoClusterFailover(primary, secondary, failoverDelayMs, switchBackDelayMs); + public ServiceUrlProvider build() { + return new AutoClusterFailover(primary, secondary, failoverDelayNs, switchBackDelayNs); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java index 41348f3cea707..dad83c7e1992d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java @@ -28,6 +28,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import lombok.Data; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.IOUtils; import org.apache.pulsar.client.api.ControlledClusterFailoverBuilder; @@ -35,6 +36,7 @@ import org.apache.pulsar.client.api.ServiceUrlProvider; @Slf4j +@Data public class ControlledClusterFailover implements ServiceUrlProvider { private PulsarClient pulsarClient; private volatile String currentPulsarServiceUrl; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java new file mode 100644 index 0000000000000..4a9e94e833865 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java @@ -0,0 +1,49 @@ +/** + * 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.pulsar.client.impl; + +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.junit.Assert; +import org.junit.Test; + +public class AutoClusterFailoverTest { + @Test + public void testBuildAutoClusterFailoverInstance() { + String primary = "pulsar://localhost:6650"; + String secondary = "pulsar://localhost:6651"; + long failoverDelay = 30; + long switchBackDelay = 60; + ServiceUrlProvider provider = AutoClusterFailover.builder() + .primary(primary) + .secondary(secondary) + .failoverDelay(failoverDelay, TimeUnit.SECONDS) + .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) + .build(); + + AutoClusterFailover autoClusterFailover = (AutoClusterFailover) provider; + Assert.assertTrue(provider instanceof AutoClusterFailover); + Assert.assertEquals(primary, provider.getServiceUrl()); + Assert.assertEquals(primary, autoClusterFailover.getPrimary()); + Assert.assertEquals(secondary, autoClusterFailover.getSecondary()); + Assert.assertEquals(TimeUnit.SECONDS.toNanos(failoverDelay), autoClusterFailover.getFailoverDelayNs()); + Assert.assertEquals(TimeUnit.SECONDS.toNanos(switchBackDelay), autoClusterFailover.getSwitchBackDelayNs()); + + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java new file mode 100644 index 0000000000000..4d06bd4823a41 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java @@ -0,0 +1,45 @@ +/** + * 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.pulsar.client.impl; + +import java.io.IOException; +import java.net.URL; +import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.junit.Assert; +import org.junit.Test; + +public class ControlledClusterFailoverTest { + @Test + public void testBuildControlledClusterFailoverInstance() throws IOException { + String defaultServiceUrl = "pulsar://localhost:6650"; + String urlProvider = "http://localhost:8080"; + + ServiceUrlProvider provider = ControlledClusterFailover.builder() + .defaultServiceUrl(defaultServiceUrl) + .urlProvider(urlProvider) + .build(); + + ControlledClusterFailover controlledClusterFailover = (ControlledClusterFailover) provider; + + Assert.assertTrue(provider instanceof ControlledClusterFailover); + Assert.assertEquals(defaultServiceUrl, provider.getServiceUrl()); + Assert.assertEquals(defaultServiceUrl, controlledClusterFailover.getCurrentPulsarServiceUrl()); + Assert.assertTrue(new URL(urlProvider).equals(controlledClusterFailover.getPulsarUrlProvider())); + } +} From f1f569e58a143c9c04fee347301cc32bb0dfcace Mon Sep 17 00:00:00 2001 From: chenhang Date: Thu, 16 Dec 2021 22:26:38 +0800 Subject: [PATCH 06/18] add tests to cover service url switch --- .../client/impl/AutoClusterFailover.java | 10 +++--- .../impl/ControlledClusterFailover.java | 9 +++-- .../client/impl/AutoClusterFailoverTest.java | 32 ++++++++++++++++++ .../impl/ControlledClusterFailoverTest.java | 33 +++++++++++++++++++ 4 files changed, 77 insertions(+), 7 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index 1c7363b8b288d..5486111df2a73 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import io.netty.util.concurrent.DefaultThreadFactory; import java.net.InetSocketAddress; @@ -45,6 +46,7 @@ public class AutoClusterFailover implements ServiceUrlProvider { private final ScheduledExecutorService executor; private long recoverTimestamp; private long failedTimestamp; + private final int interval = 30_000; private final int TIMEOUT = 30_000; private AutoClusterFailover(String primary, String secondary, long failoverDelayNs, long switchBackDelayNs) { @@ -74,7 +76,7 @@ public void initialize(PulsarClient client) { // secondary cluster is up, check whether need to switch back to primary probeAndCheckSwitchBack(primary); } - }), 30_000, 30_000, TimeUnit.MILLISECONDS); + }), getInterval(), getInterval(), TimeUnit.MILLISECONDS); } @@ -88,7 +90,7 @@ public void close() { this.executor.shutdown(); } - private boolean probeAvailable(String url) { + boolean probeAvailable(String url) { try { String hostAndPort = parseHostAndPort(url); if (Strings.isNullOrEmpty(hostAndPort)) { @@ -159,9 +161,9 @@ private void probeAndUpdateServiceUrl(String targetServiceUrl) { } else if (currentTimestamp - failedTimestamp >= failoverDelayNs) { if (probeAvailable(targetServiceUrl)) { log.info("Current Pulsar service is {}, it has been down for {} ms, " - + "switch to the service: {}. The current service down at: {}", + + "switch to the service {}. The current service down at {}", currentPulsarServiceUrl, Ns2Ms(currentTimestamp - failedTimestamp), - targetServiceUrl, Ns2Ms(failedTimestamp)); + targetServiceUrl, failedTimestamp); updateServiceUrl(targetServiceUrl); failedTimestamp = -1; } else { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java index dad83c7e1992d..f3c6df9975464 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; +import com.google.common.base.Strings; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; import java.io.InputStream; @@ -42,6 +43,7 @@ public class ControlledClusterFailover implements ServiceUrlProvider { private volatile String currentPulsarServiceUrl; private final URL pulsarUrlProvider; private final ScheduledExecutorService executor; + private final int interval = 30_000; private ControlledClusterFailover(String defaultServiceUrl, String urlProvider) throws IOException { this.currentPulsarServiceUrl = defaultServiceUrl; @@ -59,7 +61,8 @@ public void initialize(PulsarClient client) { String newPulsarUrl = null; try { newPulsarUrl = fetchServiceUrl(); - if (!currentPulsarServiceUrl.equals(newPulsarUrl)) { + if (!Strings.isNullOrEmpty(newPulsarUrl) && + !currentPulsarServiceUrl.equals(newPulsarUrl)) { log.info("Switch Pulsar service url from {} to {}", currentPulsarServiceUrl, newPulsarUrl); pulsarClient.updateServiceUrl(newPulsarUrl); currentPulsarServiceUrl = newPulsarUrl; @@ -68,10 +71,10 @@ public void initialize(PulsarClient client) { log.error("Failed to switch new Pulsar URL, current: {}, new: {}", currentPulsarServiceUrl, newPulsarUrl, e); } - }), 30_000, 30_000, TimeUnit.MILLISECONDS); + }), getInterval(), getInterval(), TimeUnit.MILLISECONDS); } - private String fetchServiceUrl() throws IOException { + String fetchServiceUrl() throws IOException { // call the service to get service URL InputStream inputStream = null; try { diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java index 4a9e94e833865..b5cd83e64fcbe 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java @@ -19,9 +19,13 @@ package org.apache.pulsar.client.impl; import java.util.concurrent.TimeUnit; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.awaitility.Awaitility; import org.junit.Assert; import org.junit.Test; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; public class AutoClusterFailoverTest { @Test @@ -44,6 +48,34 @@ public void testBuildAutoClusterFailoverInstance() { Assert.assertEquals(secondary, autoClusterFailover.getSecondary()); Assert.assertEquals(TimeUnit.SECONDS.toNanos(failoverDelay), autoClusterFailover.getFailoverDelayNs()); Assert.assertEquals(TimeUnit.SECONDS.toNanos(switchBackDelay), autoClusterFailover.getSwitchBackDelayNs()); + } + + @Test + public void testAutoClusterFailoverSwitch() { + String primary = "pulsar://localhost:6650"; + String secondary = "pulsar://localhost:6651"; + long failoverDelay = 0; + long switchBackDelay = 0; + ServiceUrlProvider provider = AutoClusterFailover.builder() + .primary(primary) + .secondary(secondary) + .failoverDelay(failoverDelay, TimeUnit.SECONDS) + .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) + .build(); + + AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); + PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); + Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); + Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); + Mockito.doReturn(1_000).when(autoClusterFailover).getInterval(); + + autoClusterFailover.initialize(pulsarClient); + + Awaitility.await().untilAsserted(() -> + Assert.assertEquals(secondary, autoClusterFailover.getServiceUrl())); + Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(primary); + Awaitility.await().untilAsserted(() -> + Assert.assertEquals(primary, autoClusterFailover.getServiceUrl())); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java index 4d06bd4823a41..86c7dd6456be2 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java @@ -20,9 +20,13 @@ import java.io.IOException; import java.net.URL; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.awaitility.Awaitility; import org.junit.Assert; import org.junit.Test; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; public class ControlledClusterFailoverTest { @Test @@ -42,4 +46,33 @@ public void testBuildControlledClusterFailoverInstance() throws IOException { Assert.assertEquals(defaultServiceUrl, controlledClusterFailover.getCurrentPulsarServiceUrl()); Assert.assertTrue(new URL(urlProvider).equals(controlledClusterFailover.getPulsarUrlProvider())); } + + @Test + public void testControlledClusterFailoverSwitch() throws IOException { + String defaultServiceUrl = "pulsar://localhost:6650"; + String backupServiceUrl = "pulsar://localhost:6651"; + String urlProvider = "http://localhost:8080"; + + ServiceUrlProvider provider = ControlledClusterFailover.builder() + .defaultServiceUrl(defaultServiceUrl) + .urlProvider(urlProvider) + .build(); + + ControlledClusterFailover controlledClusterFailover = Mockito.spy((ControlledClusterFailover) provider); + PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); + Mockito.doReturn(1_000).when(controlledClusterFailover).getInterval(); + controlledClusterFailover.initialize(pulsarClient); + + Awaitility.await().untilAsserted(() -> + Assert.assertEquals(defaultServiceUrl, controlledClusterFailover.getServiceUrl())); + + Mockito.doReturn(backupServiceUrl).when(controlledClusterFailover).fetchServiceUrl(); + Awaitility.await().untilAsserted(() -> + Assert.assertEquals(backupServiceUrl, controlledClusterFailover.getServiceUrl())); + + Mockito.doReturn(defaultServiceUrl).when(controlledClusterFailover).fetchServiceUrl(); + Awaitility.await().untilAsserted(() -> + Assert.assertEquals(defaultServiceUrl, controlledClusterFailover.getServiceUrl())); + + } } From 04abd2bb6f224f62aada6435c50d7317fed3ec78 Mon Sep 17 00:00:00 2001 From: chenhang Date: Thu, 16 Dec 2021 22:31:38 +0800 Subject: [PATCH 07/18] tune code --- .../pulsar/client/impl/AutoClusterFailover.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index 5486111df2a73..50ab6174a80af 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -19,7 +19,6 @@ package org.apache.pulsar.client.impl; import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import io.netty.util.concurrent.DefaultThreadFactory; import java.net.InetSocketAddress; @@ -135,8 +134,8 @@ private static Integer parsePort(String hostAndPort) { return Integer.valueOf(hostAndPort.substring(portSeparatorPos+1)); } - private static Long Ns2Ms(long timeStampNs) { - return timeStampNs / 1000000; + private static long nanosToMillis(long nanos) { + return Math.max(0L, Math.round(nanos / 1_000_000.0d)); } private void updateServiceUrl(String target) { @@ -162,7 +161,7 @@ private void probeAndUpdateServiceUrl(String targetServiceUrl) { if (probeAvailable(targetServiceUrl)) { log.info("Current Pulsar service is {}, it has been down for {} ms, " + "switch to the service {}. The current service down at {}", - currentPulsarServiceUrl, Ns2Ms(currentTimestamp - failedTimestamp), + currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), targetServiceUrl, failedTimestamp); updateServiceUrl(targetServiceUrl); failedTimestamp = -1; @@ -170,7 +169,7 @@ currentPulsarServiceUrl, Ns2Ms(currentTimestamp - failedTimestamp), log.error("Current Pulsar service is {}, it has been down for {} ms. " + "Failed to switch to service {}, " + "because it is not available", - currentPulsarServiceUrl, Ns2Ms(currentTimestamp - failedTimestamp), + currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), targetServiceUrl); } } @@ -189,7 +188,7 @@ private void probeAndCheckSwitchBack(String target) { log.info("Current Pulsar service is secondary: {}, " + "the primary service: {} has been recover for {} ms, " + "switch back to the primary service", - currentPulsarServiceUrl, target, Ns2Ms(currentTimestamp - recoverTimestamp)); + currentPulsarServiceUrl, target, nanosToMillis(currentTimestamp - recoverTimestamp)); updateServiceUrl(target); recoverTimestamp = -1; } From 179372fc4062ba58ae54ed56b3d268229fd75d2d Mon Sep 17 00:00:00 2001 From: chenhang Date: Sat, 25 Dec 2021 12:52:42 +0800 Subject: [PATCH 08/18] add authentication support --- .../api/AutoClusterFailoverBuilder.java | 16 ++++++ .../pulsar/client/api/PulsarClient.java | 16 ++++++ .../client/impl/AutoClusterFailover.java | 54 +++++++++++++++---- .../pulsar/client/impl/PulsarClientImpl.java | 12 +++++ 4 files changed, 87 insertions(+), 11 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java index 94567245651f6..f965a912df202 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java @@ -46,6 +46,22 @@ public interface AutoClusterFailoverBuilder { */ AutoClusterFailoverBuilder secondary(String secondary); + /** + * set primary authentication. + * + * @param authentication + * @return + */ + AutoClusterFailoverBuilder primaryAuthentication(Authentication authentication); + + /** + * set secondary authentication. + * + * @param authentication + * @return + */ + AutoClusterFailoverBuilder secondaryAuthentication(Authentication authentication); + /** * Set the switch failoverDelay. When one cluster failed longer than failoverDelay, it will trigger cluster switch. * diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java index 83097f6872136..18d9448a1d63c 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java @@ -256,6 +256,22 @@ static ClientBuilder builder() { */ void updateServiceUrl(String serviceUrl) throws PulsarClientException; + /** + * Update the service URL and authentication this client is using. + * + *

This will force the client close all existing connections and to restart service discovery to the new service + * endpoint. + * + * @param serviceUrl + * the new service URL this client should connect to + * @param authentication + * the authentication provider this client should connect to + * @throws PulsarClientException + * in case the serviceUrl is not valid + */ + void updateServiceUrlAndAuthentication(String serviceUrl, Authentication authentication) + throws PulsarClientException; + /** * Get the list of partitions for a given topic. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index 50ab6174a80af..c01734e4beab3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -23,15 +23,18 @@ import io.netty.util.concurrent.DefaultThreadFactory; import java.net.InetSocketAddress; import java.net.Socket; +import java.util.Objects; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import lombok.Data; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AutoClusterFailoverBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.checkerframework.checker.nullness.qual.Nullable; @Slf4j @Data @@ -40,6 +43,8 @@ public class AutoClusterFailover implements ServiceUrlProvider { private volatile String currentPulsarServiceUrl; private final String primary; private final String secondary; + private final Authentication primaryAuthentication; + private final Authentication secondaryAuthentication; private final long failoverDelayNs; private final long switchBackDelayNs; private final ScheduledExecutorService executor; @@ -48,9 +53,12 @@ public class AutoClusterFailover implements ServiceUrlProvider { private final int interval = 30_000; private final int TIMEOUT = 30_000; - private AutoClusterFailover(String primary, String secondary, long failoverDelayNs, long switchBackDelayNs) { + private AutoClusterFailover(String primary, String secondary, long failoverDelayNs, long switchBackDelayNs, + Authentication primaryAuthentication, Authentication secondaryAuthentication) { this.primary = primary; this.secondary = secondary; + this.primaryAuthentication = primaryAuthentication; + this.secondaryAuthentication = secondaryAuthentication; this.failoverDelayNs = failoverDelayNs; this.switchBackDelayNs = switchBackDelayNs; this.currentPulsarServiceUrl = primary; @@ -68,12 +76,12 @@ public void initialize(PulsarClient client) { this.executor.scheduleAtFixedRate(catchingAndLoggingThrowables(() -> { if (currentPulsarServiceUrl.equals(primary)) { // current service url is primary, probe whether it is down - probeAndUpdateServiceUrl(secondary); + probeAndUpdateServiceUrl(secondary, secondaryAuthentication); } else { // current service url is secondary, probe whether it is down - probeAndUpdateServiceUrl(primary); + probeAndUpdateServiceUrl(primary, primaryAuthentication); // secondary cluster is up, check whether need to switch back to primary - probeAndCheckSwitchBack(primary); + probeAndCheckSwitchBack(primary, primaryAuthentication); } }), getInterval(), getInterval(), TimeUnit.MILLISECONDS); @@ -138,9 +146,9 @@ private static long nanosToMillis(long nanos) { return Math.max(0L, Math.round(nanos / 1_000_000.0d)); } - private void updateServiceUrl(String target) { + private void updateServiceUrl(String target, Authentication authentication) { try { - pulsarClient.updateServiceUrl(target); + pulsarClient.updateServiceUrlAndAuthentication(target, authentication); currentPulsarServiceUrl = target; } catch (PulsarClientException e) { log.error("Current Pulsar service is {}, " @@ -148,7 +156,7 @@ private void updateServiceUrl(String target) { } } - private void probeAndUpdateServiceUrl(String targetServiceUrl) { + private void probeAndUpdateServiceUrl(String targetServiceUrl, Authentication authentication) { if (probeAvailable(currentPulsarServiceUrl)) { failedTimestamp = -1; return; @@ -163,7 +171,7 @@ private void probeAndUpdateServiceUrl(String targetServiceUrl) { + "switch to the service {}. The current service down at {}", currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), targetServiceUrl, failedTimestamp); - updateServiceUrl(targetServiceUrl); + updateServiceUrl(targetServiceUrl, authentication); failedTimestamp = -1; } else { log.error("Current Pulsar service is {}, it has been down for {} ms. " @@ -175,7 +183,7 @@ currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), } } - private void probeAndCheckSwitchBack(String target) { + private void probeAndCheckSwitchBack(String target, Authentication authentication) { long currentTimestamp = System.nanoTime(); if (!probeAvailable(target)) { recoverTimestamp = -1; @@ -189,7 +197,7 @@ private void probeAndCheckSwitchBack(String target) { + "the primary service: {} has been recover for {} ms, " + "switch back to the primary service", currentPulsarServiceUrl, target, nanosToMillis(currentTimestamp - recoverTimestamp)); - updateServiceUrl(target); + updateServiceUrl(target, authentication); recoverTimestamp = -1; } } @@ -197,6 +205,8 @@ private void probeAndCheckSwitchBack(String target) { public static class AutoClusterFailoverBuilderImpl implements AutoClusterFailoverBuilder { private String primary; private String secondary; + private Authentication primaryAuthentication = null; + private Authentication secondaryAuthentication = null; private long failoverDelayNs; private long switchBackDelayNs; @@ -211,6 +221,16 @@ public AutoClusterFailoverBuilder secondary(String secondary) { return this; } + public AutoClusterFailoverBuilder primaryAuthentication(Authentication authentication) { + this.primaryAuthentication = authentication; + return this; + } + + public AutoClusterFailoverBuilder secondaryAuthentication(Authentication authentication) { + this.secondaryAuthentication = authentication; + return this; + } + public AutoClusterFailoverBuilder failoverDelay(long failoverDelay, TimeUnit timeUnit) { this.failoverDelayNs = timeUnit.toNanos(failoverDelay); return this; @@ -222,7 +242,19 @@ public AutoClusterFailoverBuilder switchBackDelay(long switchBackDelay, TimeUnit } public ServiceUrlProvider build() { - return new AutoClusterFailover(primary, secondary, failoverDelayNs, switchBackDelayNs); + Objects.requireNonNull(primary, "primary service url shouldn't be null"); + Objects.requireNonNull(secondary, "secondary service url shouldn't be null"); + checkArgument(failoverDelayNs >= 0, "failoverDelayMs should >= 0"); + checkArgument(switchBackDelayNs >= 0, "switchBackDelayMs should >= 0"); + + return new AutoClusterFailover(primary, secondary, failoverDelayNs, switchBackDelayNs, + primaryAuthentication, secondaryAuthentication); + } + + public static void checkArgument(boolean expression, @Nullable Object errorMessage) { + if (!expression) { + throw new IllegalArgumentException(String.valueOf(errorMessage)); + } } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index a4c44219a25a0..63f1477c5f2e5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -48,6 +48,7 @@ import lombok.Builder; import lombok.Getter; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -863,6 +864,17 @@ public synchronized void updateServiceUrl(String serviceUrl) throws PulsarClient cnxPool.closeAllConnections(); } + @Override + public synchronized void updateServiceUrlAndAuthentication(String serviceUrl, Authentication authentication) + throws PulsarClientException { + log.info("Updating service URL to {} and authentication to {}", serviceUrl, authentication); + + conf.setServiceUrl(serviceUrl); + conf.setAuthentication(authentication); + lookup.updateServiceUrl(serviceUrl); + cnxPool.closeAllConnections(); + } + public CompletableFuture getConnection(final String topic) { TopicName topicName = TopicName.get(topic); return lookup.getBroker(topicName) From fa176bb8acd5cda7678d1049635664bf9aadbcac Mon Sep 17 00:00:00 2001 From: chenhang Date: Tue, 4 Jan 2022 12:26:48 +0800 Subject: [PATCH 09/18] add authentication support --- .../api/AutoClusterFailoverBuilder.java | 51 +++++- .../pulsar/client/api/PulsarClient.java | 32 ++-- .../client/impl/AutoClusterFailover.java | 112 ++++++++++-- .../impl/ControlledClusterFailover.java | 159 ++++++++++++++++-- .../pulsar/client/impl/PulsarClientImpl.java | 25 ++- .../client/impl/AutoClusterFailoverTest.java | 136 ++++++++++++++- .../impl/ControlledClusterFailoverTest.java | 47 +++++- 7 files changed, 508 insertions(+), 54 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java index f965a912df202..02c2f1c1b6fe4 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java @@ -47,7 +47,7 @@ public interface AutoClusterFailoverBuilder { AutoClusterFailoverBuilder secondary(String secondary); /** - * set primary authentication. + * Set primary authentication. * * @param authentication * @return @@ -55,13 +55,60 @@ public interface AutoClusterFailoverBuilder { AutoClusterFailoverBuilder primaryAuthentication(Authentication authentication); /** - * set secondary authentication. + * Set secondary authentication. * * @param authentication * @return */ AutoClusterFailoverBuilder secondaryAuthentication(Authentication authentication); + /** + * Set primary tlsTrustCertsFilePath. + * + * @param tlsTrustCertsFilePath + * @return + */ + AutoClusterFailoverBuilder primaryTlsTrustCertsFilePath(String tlsTrustCertsFilePath); + + /** + * Set secondary tlsTrustCertsFilePath. + * + * @param tlsTrustCertsFilePath + * @return + */ + AutoClusterFailoverBuilder secondaryTlsTrustCertsFilePath(String tlsTrustCertsFilePath); + + /** + * Set primary tlsTrustStorePath. + * + * @param tlsTrustStorePath + * @return + */ + AutoClusterFailoverBuilder primaryTlsTrustStorePath(String tlsTrustStorePath); + + /** + * Set secondary tlsTrustStorePath. + * + * @param tlsTrustStorePath + * @return + */ + AutoClusterFailoverBuilder secondaryTlsTrustStorePath(String tlsTrustStorePath); + + /** + * Set primary tlsTrustStorePassword. + * + * @param tlsTrustStorePassword + * @return + */ + AutoClusterFailoverBuilder primaryTlsTrustStorePassword(String tlsTrustStorePassword); + + /** + * Set secondary tlsTrustStorePassword. + * + * @param tlsTrustStorePassword + * @return + */ + AutoClusterFailoverBuilder secondaryTlsTrustStorePassword(String tlsTrustStorePassword); /** * Set the switch failoverDelay. When one cluster failed longer than failoverDelay, it will trigger cluster switch. * diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java index 18d9448a1d63c..f737d6bacdecb 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.api; import java.io.Closeable; +import java.io.IOException; import java.util.List; import java.util.concurrent.CompletableFuture; import org.apache.pulsar.client.api.transaction.TransactionBuilder; @@ -257,20 +258,29 @@ static ClientBuilder builder() { void updateServiceUrl(String serviceUrl) throws PulsarClientException; /** - * Update the service URL and authentication this client is using. + * Update the authentication this client is using. * - *

This will force the client close all existing connections and to restart service discovery to the new service - * endpoint. - * - * @param serviceUrl - * the new service URL this client should connect to * @param authentication - * the authentication provider this client should connect to - * @throws PulsarClientException - * in case the serviceUrl is not valid + * + * @throws IOException + */ + void updateAuthentication(Authentication authentication) + throws IOException; + + /** + * Update the tlsTrustCertsFilePath this client is using. + * + * @param tlsTrustCertsFilePath + */ + void updateTlsTrustCertsFilePath(String tlsTrustCertsFilePath); + + /** + * Update the tlsTrustStorePath and tlsTrustStorePassword this client is using. + * + * @param tlsTrustStorePath + * @param tlsTrustStorePassword */ - void updateServiceUrlAndAuthentication(String serviceUrl, Authentication authentication) - throws PulsarClientException; + void updateTlsTrustStorePathAndPassword(String tlsTrustStorePath, String tlsTrustStorePassword); /** * Get the list of partitions for a given topic. diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index c01734e4beab3..eeaed2c833007 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -21,6 +21,7 @@ import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; import com.google.common.base.Strings; import io.netty.util.concurrent.DefaultThreadFactory; +import java.io.IOException; import java.net.InetSocketAddress; import java.net.Socket; import java.util.Objects; @@ -32,7 +33,6 @@ import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AutoClusterFailoverBuilder; import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.checkerframework.checker.nullness.qual.Nullable; @@ -45,20 +45,35 @@ public class AutoClusterFailover implements ServiceUrlProvider { private final String secondary; private final Authentication primaryAuthentication; private final Authentication secondaryAuthentication; + private final String primaryTlsTrustCertsFilePath; + private final String secondaryTlsTrustCertsFilePath; + private String primaryTlsTrustStorePath; + private String secondaryTlsTrustStorePath; + private String primaryTlsTrustStorePassword; + private String secondaryTlsTrustStorePassword; private final long failoverDelayNs; private final long switchBackDelayNs; private final ScheduledExecutorService executor; private long recoverTimestamp; private long failedTimestamp; private final int interval = 30_000; - private final int TIMEOUT = 30_000; + private static final int TIMEOUT = 30_000; private AutoClusterFailover(String primary, String secondary, long failoverDelayNs, long switchBackDelayNs, - Authentication primaryAuthentication, Authentication secondaryAuthentication) { + Authentication primaryAuthentication, Authentication secondaryAuthentication, + String primaryTlsTrustCertsFilePath, String secondaryTlsTrustCertsFilePath, + String primaryTlsTrustStorePath, String secondaryTlsTrustStorePath, + String primaryTlsTrustStorePassword, String secondaryTlsTrustStorePassword) { this.primary = primary; this.secondary = secondary; this.primaryAuthentication = primaryAuthentication; this.secondaryAuthentication = secondaryAuthentication; + this.primaryTlsTrustCertsFilePath = primaryTlsTrustCertsFilePath; + this.secondaryTlsTrustCertsFilePath = secondaryTlsTrustCertsFilePath; + this.primaryTlsTrustStorePath = primaryTlsTrustStorePath; + this.secondaryTlsTrustStorePath = secondaryTlsTrustStorePath; + this.primaryTlsTrustStorePassword = primaryTlsTrustStorePassword; + this.secondaryTlsTrustStorePassword = secondaryTlsTrustStorePassword; this.failoverDelayNs = failoverDelayNs; this.switchBackDelayNs = switchBackDelayNs; this.currentPulsarServiceUrl = primary; @@ -76,12 +91,15 @@ public void initialize(PulsarClient client) { this.executor.scheduleAtFixedRate(catchingAndLoggingThrowables(() -> { if (currentPulsarServiceUrl.equals(primary)) { // current service url is primary, probe whether it is down - probeAndUpdateServiceUrl(secondary, secondaryAuthentication); + probeAndUpdateServiceUrl(secondary, secondaryAuthentication, secondaryTlsTrustCertsFilePath, + secondaryTlsTrustStorePath, secondaryTlsTrustStorePassword); } else { // current service url is secondary, probe whether it is down - probeAndUpdateServiceUrl(primary, primaryAuthentication); + probeAndUpdateServiceUrl(primary, primaryAuthentication, primaryTlsTrustCertsFilePath, + primaryTlsTrustStorePath, primaryTlsTrustStorePassword); // secondary cluster is up, check whether need to switch back to primary - probeAndCheckSwitchBack(primary, primaryAuthentication); + probeAndCheckSwitchBack(primary, primaryAuthentication, primaryTlsTrustCertsFilePath, + primaryTlsTrustStorePath, primaryTlsTrustStorePassword); } }), getInterval(), getInterval(), TimeUnit.MILLISECONDS); @@ -109,7 +127,7 @@ boolean probeAvailable(String url) { socket.close(); return true; } catch (Exception e) { - log.error("Failed to probe available, url: {}", url, e); + log.warn("Failed to probe available, url: {}", url, e); return false; } } @@ -146,17 +164,37 @@ private static long nanosToMillis(long nanos) { return Math.max(0L, Math.round(nanos / 1_000_000.0d)); } - private void updateServiceUrl(String target, Authentication authentication) { + private void updateServiceUrl(String target, + Authentication authentication, + String tlsTrustCertsFilePath, + String tlsTrustStorePath, + String tlsTrustStorePassword) { try { - pulsarClient.updateServiceUrlAndAuthentication(target, authentication); + if (!Strings.isNullOrEmpty(tlsTrustCertsFilePath)) { + pulsarClient.updateTlsTrustCertsFilePath(tlsTrustCertsFilePath); + } + + if (authentication != null) { + pulsarClient.updateAuthentication(authentication); + } + + if (!Strings.isNullOrEmpty(tlsTrustStorePath)) { + pulsarClient.updateTlsTrustStorePathAndPassword(tlsTrustStorePath, tlsTrustStorePassword); + } + + pulsarClient.updateServiceUrl(target); currentPulsarServiceUrl = target; - } catch (PulsarClientException e) { + } catch (IOException e) { log.error("Current Pulsar service is {}, " + "failed to switch back to {} ", currentPulsarServiceUrl, target, e); } } - private void probeAndUpdateServiceUrl(String targetServiceUrl, Authentication authentication) { + private void probeAndUpdateServiceUrl(String targetServiceUrl, + Authentication authentication, + String tlsTrustCertsFilePath, + String tlsTrustStorePath, + String tlsTrustStorePassword) { if (probeAvailable(currentPulsarServiceUrl)) { failedTimestamp = -1; return; @@ -171,7 +209,8 @@ private void probeAndUpdateServiceUrl(String targetServiceUrl, Authentication au + "switch to the service {}. The current service down at {}", currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), targetServiceUrl, failedTimestamp); - updateServiceUrl(targetServiceUrl, authentication); + updateServiceUrl(targetServiceUrl, authentication, tlsTrustCertsFilePath, + tlsTrustStorePath, tlsTrustStorePassword); failedTimestamp = -1; } else { log.error("Current Pulsar service is {}, it has been down for {} ms. " @@ -183,7 +222,11 @@ currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), } } - private void probeAndCheckSwitchBack(String target, Authentication authentication) { + private void probeAndCheckSwitchBack(String target, + Authentication authentication, + String tlsTrustCertsFilePath, + String tlsTrustStorePath, + String tlsTrustStorePassword) { long currentTimestamp = System.nanoTime(); if (!probeAvailable(target)) { recoverTimestamp = -1; @@ -197,7 +240,7 @@ private void probeAndCheckSwitchBack(String target, Authentication authenticatio + "the primary service: {} has been recover for {} ms, " + "switch back to the primary service", currentPulsarServiceUrl, target, nanosToMillis(currentTimestamp - recoverTimestamp)); - updateServiceUrl(target, authentication); + updateServiceUrl(target, authentication, tlsTrustCertsFilePath, tlsTrustStorePath, tlsTrustStorePassword); recoverTimestamp = -1; } } @@ -207,6 +250,12 @@ public static class AutoClusterFailoverBuilderImpl implements AutoClusterFailove private String secondary; private Authentication primaryAuthentication = null; private Authentication secondaryAuthentication = null; + private String primaryTlsTrustCertsFilePath = null; + private String secondaryTlsTrustCertsFilePath = null; + private String primaryTlsTrustStorePath = null; + private String secondaryTlsTrustStorePath = null; + private String primaryTlsTrustStorePassword = null; + private String secondaryTlsTrustStorePassword = null; private long failoverDelayNs; private long switchBackDelayNs; @@ -231,6 +280,36 @@ public AutoClusterFailoverBuilder secondaryAuthentication(Authentication authent return this; } + public AutoClusterFailoverBuilder primaryTlsTrustCertsFilePath(String tlsTrustCertsFilePath) { + this.primaryTlsTrustCertsFilePath = tlsTrustCertsFilePath; + return this; + } + + public AutoClusterFailoverBuilder secondaryTlsTrustCertsFilePath(String tlsTrustCertsFilePath) { + this.secondaryTlsTrustCertsFilePath = tlsTrustCertsFilePath; + return this; + } + + public AutoClusterFailoverBuilder primaryTlsTrustStorePath(String tlsTrustStorePath) { + this.primaryTlsTrustStorePath = tlsTrustStorePath; + return this; + } + + public AutoClusterFailoverBuilder secondaryTlsTrustStorePath(String tlsTrustStorePath) { + this.secondaryTlsTrustStorePath = tlsTrustStorePath; + return this; + } + + public AutoClusterFailoverBuilder primaryTlsTrustStorePassword(String tlsTrustStorePassword) { + this.primaryTlsTrustStorePassword = tlsTrustStorePassword; + return this; + } + + public AutoClusterFailoverBuilder secondaryTlsTrustStorePassword(String tlsTrustStorePassword) { + this.secondaryTlsTrustStorePassword = tlsTrustStorePassword; + return this; + } + public AutoClusterFailoverBuilder failoverDelay(long failoverDelay, TimeUnit timeUnit) { this.failoverDelayNs = timeUnit.toNanos(failoverDelay); return this; @@ -248,7 +327,10 @@ public ServiceUrlProvider build() { checkArgument(switchBackDelayNs >= 0, "switchBackDelayMs should >= 0"); return new AutoClusterFailover(primary, secondary, failoverDelayNs, switchBackDelayNs, - primaryAuthentication, secondaryAuthentication); + primaryAuthentication, secondaryAuthentication, + primaryTlsTrustCertsFilePath, secondaryTlsTrustCertsFilePath, + primaryTlsTrustStorePath, secondaryTlsTrustStorePath, + primaryTlsTrustStorePassword, secondaryTlsTrustStorePassword); } public static void checkArgument(boolean expression, @Nullable Object errorMessage) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java index f3c6df9975464..e7ac460aa2781 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java @@ -19,6 +19,8 @@ package org.apache.pulsar.client.impl; import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; @@ -26,24 +28,37 @@ import java.net.URL; import java.net.URLConnection; import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.Objects; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import lombok.Data; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.IOUtils; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ControlledClusterFailoverBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.apache.pulsar.common.util.ObjectMapperFactory; @Slf4j -@Data public class ControlledClusterFailover implements ServiceUrlProvider { private PulsarClient pulsarClient; private volatile String currentPulsarServiceUrl; + private volatile ControlledConfiguration currentControlledConfiguration; private final URL pulsarUrlProvider; private final ScheduledExecutorService executor; private final int interval = 30_000; + private ObjectMapper objectMapper = null; + + private static final String AUTH_SASL = "org.apache.pulsar.client.impl.auth.AuthenticationSasl"; + private static final String AUTH_ATHENZ = "org.apache.pulsar.client.impl.auth.AuthenticationAthenz"; + private static final String AUTH_OAUTH2 = "org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2"; + private static final String AUTH_TLS = "org.apache.pulsar.client.impl.auth.AuthenticationTls"; + private static final String AUTH_TOKEN = "org.apache.pulsar.client.impl.auth.AuthenticationToken"; + private static final String AUTH_KEY_STORE_TLS = "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls"; private ControlledClusterFailover(String defaultServiceUrl, String urlProvider) throws IOException { this.currentPulsarServiceUrl = defaultServiceUrl; @@ -58,29 +73,95 @@ public void initialize(PulsarClient client) { // start to check service url every 30 seconds this.executor.scheduleAtFixedRate(catchingAndLoggingThrowables(() -> { - String newPulsarUrl = null; + ControlledConfiguration controlledConfiguration = null; try { - newPulsarUrl = fetchServiceUrl(); - if (!Strings.isNullOrEmpty(newPulsarUrl) && - !currentPulsarServiceUrl.equals(newPulsarUrl)) { - log.info("Switch Pulsar service url from {} to {}", currentPulsarServiceUrl, newPulsarUrl); - pulsarClient.updateServiceUrl(newPulsarUrl); - currentPulsarServiceUrl = newPulsarUrl; + controlledConfiguration = fetchControlledConfiguration(); + if (controlledConfiguration != null + && !Strings.isNullOrEmpty(controlledConfiguration.getServiceUrl()) + && !controlledConfiguration.equals(currentControlledConfiguration)) { + log.info("Switch Pulsar service url from {} to {}", + currentControlledConfiguration, controlledConfiguration.toString()); + + Authentication authentication = null; + if (!Strings.isNullOrEmpty(controlledConfiguration.authPluginClassName)) { + String authPluginClassName = controlledConfiguration.authPluginClassName; + Map authParams = controlledConfiguration.getAuthParams(); + String authParamsString = controlledConfiguration.getAuthParamsString(); + String token = controlledConfiguration.getToken(); + + switch (authPluginClassName) { + case AUTH_SASL: + case AUTH_ATHENZ: + if (authParams != null && !authParams.isEmpty()) { + authentication = + AuthenticationFactory.create(authPluginClassName, authParams); + } + break; + + case AUTH_TOKEN: + if (!Strings.isNullOrEmpty(token)) { + authentication = AuthenticationFactory.token(token); + } + break; + + case AUTH_OAUTH2: + case AUTH_TLS: + case AUTH_KEY_STORE_TLS: + default: + if (!Strings.isNullOrEmpty(authParamsString)) { + authentication = + AuthenticationFactory.create(authPluginClassName, authParamsString); + } + break; + } + } + + String tlsTrustCertsFilePath = controlledConfiguration.getTlsTrustCertsFilePath(); + String serviceUrl = controlledConfiguration.getServiceUrl(); + + if (authentication != null) { + pulsarClient.updateAuthentication(authentication); + } + + if (!Strings.isNullOrEmpty(tlsTrustCertsFilePath)) { + pulsarClient.updateTlsTrustCertsFilePath(tlsTrustCertsFilePath); + } + + pulsarClient.updateServiceUrl(serviceUrl); + currentPulsarServiceUrl = serviceUrl; + currentControlledConfiguration = controlledConfiguration; } } catch (IOException e) { - log.error("Failed to switch new Pulsar URL, current: {}, new: {}", - currentPulsarServiceUrl, newPulsarUrl, e); + log.error("Failed to switch new Pulsar url, current: {}, new: {}", + currentControlledConfiguration, controlledConfiguration, e); } }), getInterval(), getInterval(), TimeUnit.MILLISECONDS); } - String fetchServiceUrl() throws IOException { + public int getInterval() { + return this.interval; + } + + public String getCurrentPulsarServiceUrl() { + return this.currentPulsarServiceUrl; + } + + public URL getPulsarUrlProvider() { + return this.pulsarUrlProvider; + } + + protected ControlledConfiguration fetchControlledConfiguration() throws IOException { // call the service to get service URL InputStream inputStream = null; try { URLConnection conn = pulsarUrlProvider.openConnection(); inputStream = conn.getInputStream(); - return new String(IOUtils.toByteArray(inputStream), StandardCharsets.UTF_8); + String jsonStr = new String(IOUtils.toByteArray(inputStream), StandardCharsets.UTF_8); + ObjectMapper objectMapper = getObjectMapper(); + return objectMapper.readValue(jsonStr, ControlledConfiguration.class); + } catch (IOException e) { + log.warn("Failed to fetch controlled configuration. ", e); + return null; } finally { if (inputStream != null) { inputStream.close(); @@ -88,6 +169,60 @@ String fetchServiceUrl() throws IOException { } } + private ObjectMapper getObjectMapper() { + if (objectMapper == null) { + objectMapper = new ObjectMapper(); + } + return objectMapper; + } + + @Data + protected static class ControlledConfiguration { + private String serviceUrl; + private String tlsTrustCertsFilePath; + + private String authPluginClassName; + + private String authParamsString; + private String token; + private Map authParams; + + public String toJson() { + ObjectMapper objectMapper = ObjectMapperFactory.getThreadLocal(); + try { + return objectMapper.writeValueAsString(this); + } catch (JsonProcessingException e) { + log.warn("Failed to write as json. ", e); + return null; + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof ControlledConfiguration) { + ControlledConfiguration other = (ControlledConfiguration) obj; + return Objects.equals(serviceUrl, other.serviceUrl) + && Objects.equals(tlsTrustCertsFilePath, other.tlsTrustCertsFilePath) + && Objects.equals(authPluginClassName, other.authPluginClassName) + && Objects.equals(authParamsString, other.authParamsString) + && Objects.equals(token, other.token) + && Objects.equals(authParams, other.authParams); + } + + return false; + } + + @Override + public int hashCode() { + return Objects.hash(serviceUrl, + tlsTrustCertsFilePath, + authPluginClassName, + authParamsString, + token, + authParams); + } + } + @Override public String getServiceUrl() { return this.currentPulsarServiceUrl; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 63f1477c5f2e5..18c135988759d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -27,6 +27,7 @@ import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; import io.netty.util.concurrent.DefaultThreadFactory; +import java.io.IOException; import java.time.Clock; import java.util.ArrayList; import java.util.Collections; @@ -865,14 +866,24 @@ public synchronized void updateServiceUrl(String serviceUrl) throws PulsarClient } @Override - public synchronized void updateServiceUrlAndAuthentication(String serviceUrl, Authentication authentication) - throws PulsarClientException { - log.info("Updating service URL to {} and authentication to {}", serviceUrl, authentication); - - conf.setServiceUrl(serviceUrl); + public void updateAuthentication(Authentication authentication) throws IOException { + log.info("Updating authentication to {}", authentication); + conf.getAuthentication().close(); conf.setAuthentication(authentication); - lookup.updateServiceUrl(serviceUrl); - cnxPool.closeAllConnections(); + conf.getAuthentication().start(); + } + + @Override + public void updateTlsTrustCertsFilePath(String tlsTrustCertsFilePath) { + log.info("Updating tlsTrustCertsFilePath to {}", tlsTrustCertsFilePath); + conf.setTlsTrustCertsFilePath(tlsTrustCertsFilePath); + } + + @Override + public void updateTlsTrustStorePathAndPassword(String tlsTrustStorePath, String tlsTrustStorePassword) { + log.info("Updating tlsTrustStorePath to {}, tlsTrustStorePassword to *****", tlsTrustStorePath); + conf.setTlsTrustStorePath(tlsTrustStorePath); + conf.setTlsTrustStorePassword(tlsTrustStorePassword); } public CompletableFuture getConnection(final String topic) { diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java index b5cd83e64fcbe..47482bad81018 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java @@ -18,8 +18,12 @@ */ package org.apache.pulsar.client.impl; +import java.io.IOException; import java.util.concurrent.TimeUnit; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.awaitility.Awaitility; import org.junit.Assert; @@ -29,7 +33,7 @@ public class AutoClusterFailoverTest { @Test - public void testBuildAutoClusterFailoverInstance() { + public void testBuildAutoClusterFailoverInstance() throws PulsarClientException { String primary = "pulsar://localhost:6650"; String secondary = "pulsar://localhost:6651"; long failoverDelay = 30; @@ -48,10 +52,42 @@ public void testBuildAutoClusterFailoverInstance() { Assert.assertEquals(secondary, autoClusterFailover.getSecondary()); Assert.assertEquals(TimeUnit.SECONDS.toNanos(failoverDelay), autoClusterFailover.getFailoverDelayNs()); Assert.assertEquals(TimeUnit.SECONDS.toNanos(switchBackDelay), autoClusterFailover.getSwitchBackDelayNs()); + Assert.assertNull(autoClusterFailover.getPrimaryTlsTrustCertsFilePath()); + Assert.assertNull(autoClusterFailover.getPrimaryAuthentication()); + Assert.assertNull(autoClusterFailover.getSecondaryAuthentication()); + Assert.assertNull(autoClusterFailover.getSecondaryTlsTrustCertsFilePath()); + + String primaryTlsTrustCertsFilePath = "primary/path"; + String secondaryTlsTrustCertsFilePath = "primary/path"; + Authentication primaryAuthentication = AuthenticationFactory.create( + "org.apache.pulsar.client.impl.auth.AuthenticationTls", + "tlsCertFile:/path/to/primary-my-role.cert.pem," + + "tlsKeyFile:/path/to/primary-my-role.key-pk8.pem"); + + Authentication secondaryAuthentication = AuthenticationFactory.create( + "org.apache.pulsar.client.impl.auth.AuthenticationTls", + "tlsCertFile:/path/to/secondary-my-role.cert.pem," + + "tlsKeyFile:/path/to/secondary-role.key-pk8.pem"); + ServiceUrlProvider provider1 = AutoClusterFailover.builder() + .primary(primary) + .secondary(secondary) + .failoverDelay(failoverDelay, TimeUnit.SECONDS) + .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) + .primaryTlsTrustCertsFilePath(primaryTlsTrustCertsFilePath) + .secondaryTlsTrustCertsFilePath(secondaryTlsTrustCertsFilePath) + .primaryAuthentication(primaryAuthentication) + .secondaryAuthentication(secondaryAuthentication) + .build(); + + AutoClusterFailover autoClusterFailover1 = (AutoClusterFailover) provider1; + Assert.assertEquals(primaryTlsTrustCertsFilePath, autoClusterFailover1.getPrimaryTlsTrustCertsFilePath()); + Assert.assertEquals(primaryAuthentication, autoClusterFailover1.getPrimaryAuthentication()); + Assert.assertEquals(secondaryTlsTrustCertsFilePath, autoClusterFailover1.getSecondaryTlsTrustCertsFilePath()); + Assert.assertEquals(secondaryAuthentication, autoClusterFailover1.getSecondaryAuthentication()); } @Test - public void testAutoClusterFailoverSwitch() { + public void testAutoClusterFailoverSwitchWithoutAuthentication() { String primary = "pulsar://localhost:6650"; String secondary = "pulsar://localhost:6651"; long failoverDelay = 0; @@ -74,8 +110,104 @@ public void testAutoClusterFailoverSwitch() { Awaitility.await().untilAsserted(() -> Assert.assertEquals(secondary, autoClusterFailover.getServiceUrl())); + // primary cluster came back Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(primary); Awaitility.await().untilAsserted(() -> Assert.assertEquals(primary, autoClusterFailover.getServiceUrl())); } + + @Test + public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException { + String primary = "pulsar+ssl://localhost:6651"; + String secondary = "pulsar+ssl://localhost:6661"; + long failoverDelay = 0; + long switchBackDelay = 0; + String primaryTlsTrustCertsFilePath = "primary/path"; + String secondaryTlsTrustCertsFilePath = "primary/path"; + Authentication primaryAuthentication = AuthenticationFactory.create( + "org.apache.pulsar.client.impl.auth.AuthenticationTls", + "tlsCertFile:/path/to/primary-my-role.cert.pem," + + "tlsKeyFile:/path/to/primary-my-role.key-pk8.pem"); + + Authentication secondaryAuthentication = AuthenticationFactory.create( + "org.apache.pulsar.client.impl.auth.AuthenticationTls", + "tlsCertFile:/path/to/secondary-my-role.cert.pem," + + "tlsKeyFile:/path/to/secondary-role.key-pk8.pem"); + + ServiceUrlProvider provider = AutoClusterFailover.builder() + .primary(primary) + .secondary(secondary) + .failoverDelay(failoverDelay, TimeUnit.SECONDS) + .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) + .primaryTlsTrustCertsFilePath(primaryTlsTrustCertsFilePath) + .secondaryTlsTrustCertsFilePath(secondaryTlsTrustCertsFilePath) + .primaryAuthentication(primaryAuthentication) + .secondaryAuthentication(secondaryAuthentication) + .build(); + + AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); + PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); + Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); + Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); + Mockito.doReturn(1_000).when(autoClusterFailover).getInterval(); + + autoClusterFailover.initialize(pulsarClient); + + Awaitility.await().untilAsserted(() -> + Assert.assertEquals(secondary, autoClusterFailover.getServiceUrl())); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()).updateTlsTrustCertsFilePath(secondaryTlsTrustCertsFilePath); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()).updateAuthentication(secondaryAuthentication); + + // primary cluster came back + Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(primary); + Awaitility.await().untilAsserted(() -> + Assert.assertEquals(primary, autoClusterFailover.getServiceUrl())); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()).updateTlsTrustCertsFilePath(primaryTlsTrustCertsFilePath); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()).updateAuthentication(primaryAuthentication); + + } + + @Test + public void testAutoClusterFailoverSwitchTlsTrustStore() throws IOException { + String primary = "pulsar+ssl://localhost:6651"; + String secondary = "pulsar+ssl://localhost:6661"; + long failoverDelay = 0; + long switchBackDelay = 0; + String primaryTlsTrustStorePath = "primary/path"; + String secondaryTlsTrustStorePath = "secondary/path"; + String primaryTlsTrustStorePassword = "primaryPassword"; + String secondaryTlsTrustStorePassword = "secondaryPassword"; + + ServiceUrlProvider provider = AutoClusterFailover.builder() + .primary(primary) + .secondary(secondary) + .failoverDelay(failoverDelay, TimeUnit.SECONDS) + .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) + .primaryTlsTrustStorePath(primaryTlsTrustStorePath) + .primaryTlsTrustStorePassword(primaryTlsTrustStorePassword) + .secondaryTlsTrustStorePath(secondaryTlsTrustStorePath) + .secondaryTlsTrustStorePassword(secondaryTlsTrustStorePassword) + .build(); + + AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); + PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); + Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); + Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); + Mockito.doReturn(1_000).when(autoClusterFailover).getInterval(); + + autoClusterFailover.initialize(pulsarClient); + + Awaitility.await().untilAsserted(() -> + Assert.assertEquals(secondary, autoClusterFailover.getServiceUrl())); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()) + .updateTlsTrustStorePathAndPassword(secondaryTlsTrustStorePath, secondaryTlsTrustStorePassword); + + // primary cluster came back + Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(primary); + Awaitility.await().untilAsserted(() -> + Assert.assertEquals(primary, autoClusterFailover.getServiceUrl())); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()) + .updateTlsTrustStorePathAndPassword(primaryTlsTrustStorePath, primaryTlsTrustStorePassword); + + } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java index 86c7dd6456be2..41f7d373068f1 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.net.URL; +import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.awaitility.Awaitility; @@ -49,9 +50,19 @@ public void testBuildControlledClusterFailoverInstance() throws IOException { @Test public void testControlledClusterFailoverSwitch() throws IOException { - String defaultServiceUrl = "pulsar://localhost:6650"; - String backupServiceUrl = "pulsar://localhost:6651"; + String defaultServiceUrl = "pulsar+ssl://localhost:6651"; + String backupServiceUrl = "pulsar+ssl://localhost:6661"; String urlProvider = "http://localhost:8080"; + String tlsTrustCertsFilePath = "backup/path"; + String authPluginClassName = "org.apache.pulsar.client.impl.auth.AuthenticationToken"; + String token = "xxxaaabbee"; + + ControlledClusterFailover.ControlledConfiguration controlledConfiguration = + new ControlledClusterFailover.ControlledConfiguration(); + controlledConfiguration.setServiceUrl(backupServiceUrl); + controlledConfiguration.setTlsTrustCertsFilePath(tlsTrustCertsFilePath); + controlledConfiguration.setAuthPluginClassName(authPluginClassName); + controlledConfiguration.setToken(token); ServiceUrlProvider provider = ControlledClusterFailover.builder() .defaultServiceUrl(defaultServiceUrl) @@ -61,18 +72,44 @@ public void testControlledClusterFailoverSwitch() throws IOException { ControlledClusterFailover controlledClusterFailover = Mockito.spy((ControlledClusterFailover) provider); PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); Mockito.doReturn(1_000).when(controlledClusterFailover).getInterval(); + controlledClusterFailover.initialize(pulsarClient); Awaitility.await().untilAsserted(() -> Assert.assertEquals(defaultServiceUrl, controlledClusterFailover.getServiceUrl())); - Mockito.doReturn(backupServiceUrl).when(controlledClusterFailover).fetchServiceUrl(); + Mockito.doReturn(controlledConfiguration).when(controlledClusterFailover) + .fetchControlledConfiguration(); Awaitility.await().untilAsserted(() -> Assert.assertEquals(backupServiceUrl, controlledClusterFailover.getServiceUrl())); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()) + .updateServiceUrl(backupServiceUrl); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()) + .updateTlsTrustCertsFilePath(tlsTrustCertsFilePath); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()) + .updateAuthentication(Mockito.any(Authentication.class)); + + // update controlled configuration + String backupServiceUrlV1 = "pulsar+ssl://localhost:6662"; + String tlsTrustCertsFilePathV1 = "backup/pathV1"; + String authPluginClassNameV1 = "org.apache.pulsar.client.impl.auth.AuthenticationToken"; + String tokenV1 = "xxxaaabbeev1"; + ControlledClusterFailover.ControlledConfiguration controlledConfiguration1 = + new ControlledClusterFailover.ControlledConfiguration(); + controlledConfiguration1.setServiceUrl(backupServiceUrlV1); + controlledConfiguration1.setTlsTrustCertsFilePath(tlsTrustCertsFilePathV1); + controlledConfiguration1.setAuthPluginClassName(authPluginClassNameV1); + controlledConfiguration1.setToken(tokenV1); + Mockito.doReturn(controlledConfiguration1).when(controlledClusterFailover) + .fetchControlledConfiguration(); - Mockito.doReturn(defaultServiceUrl).when(controlledClusterFailover).fetchServiceUrl(); Awaitility.await().untilAsserted(() -> - Assert.assertEquals(defaultServiceUrl, controlledClusterFailover.getServiceUrl())); + Assert.assertEquals(backupServiceUrlV1, controlledClusterFailover.getServiceUrl())); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()).updateServiceUrl(backupServiceUrlV1); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()) + .updateTlsTrustCertsFilePath(tlsTrustCertsFilePathV1); + Mockito.verify(pulsarClient, Mockito.atLeastOnce()) + .updateAuthentication(Mockito.any(Authentication.class)); } } From ee812ff4a60d839a3a29dd7673ce9f7dae9b2d62 Mon Sep 17 00:00:00 2001 From: chenhang Date: Tue, 4 Jan 2022 15:56:21 +0800 Subject: [PATCH 10/18] support multi secondary cluster --- .../api/AutoClusterFailoverBuilder.java | 20 ++- .../client/impl/AutoClusterFailover.java | 141 ++++++++++++++---- .../client/impl/AutoClusterFailoverTest.java | 47 +++--- 3 files changed, 151 insertions(+), 57 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java index 02c2f1c1b6fe4..ccbf878e8bc41 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.pulsar.common.classification.InterfaceAudience; import org.apache.pulsar.common.classification.InterfaceStability; @@ -44,7 +45,7 @@ public interface AutoClusterFailoverBuilder { * @param secondary * @return */ - AutoClusterFailoverBuilder secondary(String secondary); + AutoClusterFailoverBuilder secondary(List secondary); /** * Set primary authentication. @@ -60,7 +61,7 @@ public interface AutoClusterFailoverBuilder { * @param authentication * @return */ - AutoClusterFailoverBuilder secondaryAuthentication(Authentication authentication); + AutoClusterFailoverBuilder secondaryAuthentication(List authentication); /** * Set primary tlsTrustCertsFilePath. @@ -76,7 +77,7 @@ public interface AutoClusterFailoverBuilder { * @param tlsTrustCertsFilePath * @return */ - AutoClusterFailoverBuilder secondaryTlsTrustCertsFilePath(String tlsTrustCertsFilePath); + AutoClusterFailoverBuilder secondaryTlsTrustCertsFilePath(List tlsTrustCertsFilePath); /** * Set primary tlsTrustStorePath. @@ -92,7 +93,7 @@ public interface AutoClusterFailoverBuilder { * @param tlsTrustStorePath * @return */ - AutoClusterFailoverBuilder secondaryTlsTrustStorePath(String tlsTrustStorePath); + AutoClusterFailoverBuilder secondaryTlsTrustStorePath(List tlsTrustStorePath); /** * Set primary tlsTrustStorePassword. @@ -108,7 +109,7 @@ public interface AutoClusterFailoverBuilder { * @param tlsTrustStorePassword * @return */ - AutoClusterFailoverBuilder secondaryTlsTrustStorePassword(String tlsTrustStorePassword); + AutoClusterFailoverBuilder secondaryTlsTrustStorePassword(List tlsTrustStorePassword); /** * Set the switch failoverDelay. When one cluster failed longer than failoverDelay, it will trigger cluster switch. * @@ -128,6 +129,15 @@ public interface AutoClusterFailoverBuilder { */ AutoClusterFailoverBuilder switchBackDelay(long switchBackDelay, TimeUnit timeUnit); + /** + * Set the checkInterval for probe. + * + * @param interval + * @param timeUnit + * @return + */ + AutoClusterFailoverBuilder checkInterval(long interval, TimeUnit timeUnit); + /** * Build the ServiceUrlProvider instance. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index eeaed2c833007..11cb3bc89b12f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.Socket; +import java.util.List; import java.util.Objects; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -42,43 +43,45 @@ public class AutoClusterFailover implements ServiceUrlProvider { private PulsarClient pulsarClient; private volatile String currentPulsarServiceUrl; private final String primary; - private final String secondary; + private final List secondary; private final Authentication primaryAuthentication; - private final Authentication secondaryAuthentication; + private final List secondaryAuthentications; private final String primaryTlsTrustCertsFilePath; - private final String secondaryTlsTrustCertsFilePath; + private final List secondaryTlsTrustCertsFilePaths; private String primaryTlsTrustStorePath; - private String secondaryTlsTrustStorePath; + private List secondaryTlsTrustStorePaths; private String primaryTlsTrustStorePassword; - private String secondaryTlsTrustStorePassword; + private List secondaryTlsTrustStorePasswords; private final long failoverDelayNs; private final long switchBackDelayNs; private final ScheduledExecutorService executor; private long recoverTimestamp; private long failedTimestamp; - private final int interval = 30_000; + private final long intervalMs; private static final int TIMEOUT = 30_000; - private AutoClusterFailover(String primary, String secondary, long failoverDelayNs, long switchBackDelayNs, - Authentication primaryAuthentication, Authentication secondaryAuthentication, - String primaryTlsTrustCertsFilePath, String secondaryTlsTrustCertsFilePath, - String primaryTlsTrustStorePath, String secondaryTlsTrustStorePath, - String primaryTlsTrustStorePassword, String secondaryTlsTrustStorePassword) { + private AutoClusterFailover(String primary, List secondary, long failoverDelayNs, long switchBackDelayNs, + long intervalMs, Authentication primaryAuthentication, + List secondaryAuthentications, String primaryTlsTrustCertsFilePath, + List secondaryTlsTrustCertsFilePaths, String primaryTlsTrustStorePath, + List secondaryTlsTrustStorePaths, String primaryTlsTrustStorePassword, + List secondaryTlsTrustStorePasswords) { this.primary = primary; this.secondary = secondary; this.primaryAuthentication = primaryAuthentication; - this.secondaryAuthentication = secondaryAuthentication; + this.secondaryAuthentications = secondaryAuthentications; this.primaryTlsTrustCertsFilePath = primaryTlsTrustCertsFilePath; - this.secondaryTlsTrustCertsFilePath = secondaryTlsTrustCertsFilePath; + this.secondaryTlsTrustCertsFilePaths = secondaryTlsTrustCertsFilePaths; this.primaryTlsTrustStorePath = primaryTlsTrustStorePath; - this.secondaryTlsTrustStorePath = secondaryTlsTrustStorePath; + this.secondaryTlsTrustStorePaths = secondaryTlsTrustStorePaths; this.primaryTlsTrustStorePassword = primaryTlsTrustStorePassword; - this.secondaryTlsTrustStorePassword = secondaryTlsTrustStorePassword; + this.secondaryTlsTrustStorePasswords = secondaryTlsTrustStorePasswords; this.failoverDelayNs = failoverDelayNs; this.switchBackDelayNs = switchBackDelayNs; this.currentPulsarServiceUrl = primary; this.recoverTimestamp = -1; this.failedTimestamp = -1; + this.intervalMs = intervalMs; this.executor = Executors.newSingleThreadScheduledExecutor( new DefaultThreadFactory("pulsar-service-provider")); } @@ -91,8 +94,8 @@ public void initialize(PulsarClient client) { this.executor.scheduleAtFixedRate(catchingAndLoggingThrowables(() -> { if (currentPulsarServiceUrl.equals(primary)) { // current service url is primary, probe whether it is down - probeAndUpdateServiceUrl(secondary, secondaryAuthentication, secondaryTlsTrustCertsFilePath, - secondaryTlsTrustStorePath, secondaryTlsTrustStorePassword); + probeAndUpdateServiceUrl(secondary, secondaryAuthentications, secondaryTlsTrustCertsFilePaths, + secondaryTlsTrustStorePaths, secondaryTlsTrustStorePasswords); } else { // current service url is secondary, probe whether it is down probeAndUpdateServiceUrl(primary, primaryAuthentication, primaryTlsTrustCertsFilePath, @@ -101,7 +104,7 @@ public void initialize(PulsarClient client) { probeAndCheckSwitchBack(primary, primaryAuthentication, primaryTlsTrustCertsFilePath, primaryTlsTrustStorePath, primaryTlsTrustStorePassword); } - }), getInterval(), getInterval(), TimeUnit.MILLISECONDS); + }), intervalMs, intervalMs, TimeUnit.MILLISECONDS); } @@ -190,6 +193,44 @@ private void updateServiceUrl(String target, } } + private void probeAndUpdateServiceUrl(List targetServiceUrls, + List authentications, + List tlsTrustCertsFilePaths, + List tlsTrustStorePaths, + List tlsTrustStorePasswords) { + if (probeAvailable(currentPulsarServiceUrl)) { + failedTimestamp = -1; + return; + } + + long currentTimestamp = System.nanoTime(); + if (failedTimestamp == -1) { + failedTimestamp = currentTimestamp; + } else if (currentTimestamp - failedTimestamp >= failoverDelayNs) { + for (int i = 0; i < targetServiceUrls.size(); ++i) { + if (probeAvailable(targetServiceUrls.get(i))) { + log.info("Current Pulsar service is {}, it has been down for {} ms, " + + "switch to the service {}. The current service down at {}", + currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), + targetServiceUrls.get(i), failedTimestamp); + updateServiceUrl(targetServiceUrls.get(i), + authentications != null ? authentications.get(i) : null, + tlsTrustCertsFilePaths != null ? tlsTrustCertsFilePaths.get(i) : null, + tlsTrustStorePaths != null ? tlsTrustStorePaths.get(i) : null, + tlsTrustStorePasswords != null ? tlsTrustStorePasswords.get(i) : null); + failedTimestamp = -1; + break; + } else { + log.warn("Current Pulsar service is {}, it has been down for {} ms. " + + "Failed to switch to service {}, " + + "because it is not available, continue to probe next pulsar service.", + currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), + targetServiceUrls.get(i)); + } + } + } + } + private void probeAndUpdateServiceUrl(String targetServiceUrl, Authentication authentication, String tlsTrustCertsFilePath, @@ -247,86 +288,120 @@ private void probeAndCheckSwitchBack(String target, public static class AutoClusterFailoverBuilderImpl implements AutoClusterFailoverBuilder { private String primary; - private String secondary; + private List secondary; private Authentication primaryAuthentication = null; - private Authentication secondaryAuthentication = null; + private List secondaryAuthentication = null; private String primaryTlsTrustCertsFilePath = null; - private String secondaryTlsTrustCertsFilePath = null; + private List secondaryTlsTrustCertsFilePath = null; private String primaryTlsTrustStorePath = null; - private String secondaryTlsTrustStorePath = null; + private List secondaryTlsTrustStorePath = null; private String primaryTlsTrustStorePassword = null; - private String secondaryTlsTrustStorePassword = null; + private List secondaryTlsTrustStorePassword = null; private long failoverDelayNs; private long switchBackDelayNs; + private long checkIntervalMs = 30_000; - + @Override public AutoClusterFailoverBuilder primary(String primary) { this.primary = primary; return this; } - public AutoClusterFailoverBuilder secondary(String secondary) { + @Override + public AutoClusterFailoverBuilder secondary(List secondary) { this.secondary = secondary; return this; } + @Override public AutoClusterFailoverBuilder primaryAuthentication(Authentication authentication) { this.primaryAuthentication = authentication; return this; } - public AutoClusterFailoverBuilder secondaryAuthentication(Authentication authentication) { + @Override + public AutoClusterFailoverBuilder secondaryAuthentication(List authentication) { this.secondaryAuthentication = authentication; return this; } + @Override public AutoClusterFailoverBuilder primaryTlsTrustCertsFilePath(String tlsTrustCertsFilePath) { this.primaryTlsTrustCertsFilePath = tlsTrustCertsFilePath; return this; } - public AutoClusterFailoverBuilder secondaryTlsTrustCertsFilePath(String tlsTrustCertsFilePath) { + @Override + public AutoClusterFailoverBuilder secondaryTlsTrustCertsFilePath(List tlsTrustCertsFilePath) { this.secondaryTlsTrustCertsFilePath = tlsTrustCertsFilePath; return this; } + @Override public AutoClusterFailoverBuilder primaryTlsTrustStorePath(String tlsTrustStorePath) { this.primaryTlsTrustStorePath = tlsTrustStorePath; return this; } - public AutoClusterFailoverBuilder secondaryTlsTrustStorePath(String tlsTrustStorePath) { + @Override + public AutoClusterFailoverBuilder secondaryTlsTrustStorePath(List tlsTrustStorePath) { this.secondaryTlsTrustStorePath = tlsTrustStorePath; return this; } + @Override public AutoClusterFailoverBuilder primaryTlsTrustStorePassword(String tlsTrustStorePassword) { this.primaryTlsTrustStorePassword = tlsTrustStorePassword; return this; } - public AutoClusterFailoverBuilder secondaryTlsTrustStorePassword(String tlsTrustStorePassword) { + @Override + public AutoClusterFailoverBuilder secondaryTlsTrustStorePassword(List tlsTrustStorePassword) { this.secondaryTlsTrustStorePassword = tlsTrustStorePassword; return this; } + @Override public AutoClusterFailoverBuilder failoverDelay(long failoverDelay, TimeUnit timeUnit) { this.failoverDelayNs = timeUnit.toNanos(failoverDelay); return this; } + @Override public AutoClusterFailoverBuilder switchBackDelay(long switchBackDelay, TimeUnit timeUnit) { this.switchBackDelayNs = timeUnit.toNanos(switchBackDelay); return this; } + @Override + public AutoClusterFailoverBuilder checkInterval(long interval, TimeUnit timeUnit) { + this.checkIntervalMs = timeUnit.toMillis(interval); + return this; + } + + @Override public ServiceUrlProvider build() { Objects.requireNonNull(primary, "primary service url shouldn't be null"); Objects.requireNonNull(secondary, "secondary service url shouldn't be null"); - checkArgument(failoverDelayNs >= 0, "failoverDelayMs should >= 0"); - checkArgument(switchBackDelayNs >= 0, "switchBackDelayMs should >= 0"); - - return new AutoClusterFailover(primary, secondary, failoverDelayNs, switchBackDelayNs, + checkArgument(failoverDelayNs >= 0, "failoverDelay should >= 0"); + checkArgument(switchBackDelayNs >= 0, "switchBackDelay should >= 0"); + checkArgument(checkIntervalMs >= 0, "checkInterval should >= 0"); + int secondarySize = secondary.size(); + + checkArgument(secondaryAuthentication == null + || secondaryAuthentication.size() == secondarySize, + "secondaryAuthentication should be null or size equal with secondary url size"); + checkArgument(secondaryTlsTrustCertsFilePath == null + || secondaryTlsTrustCertsFilePath.size() == secondarySize, + "secondaryTlsTrustCertsFilePath should be null or size equal with secondary url size"); + checkArgument(secondaryTlsTrustStorePath == null + || secondaryTlsTrustStorePath.size() == secondarySize, + "secondaryTlsTrustStorePath should be null or size equal with secondary url size"); + checkArgument(secondaryTlsTrustStorePassword == null + || secondaryTlsTrustStorePassword.size() == secondarySize, + "secondaryTlsTrustStorePassword should be null or size equal with secondary url size"); + + return new AutoClusterFailover(primary, secondary, failoverDelayNs, switchBackDelayNs, checkIntervalMs, primaryAuthentication, secondaryAuthentication, primaryTlsTrustCertsFilePath, secondaryTlsTrustCertsFilePath, primaryTlsTrustStorePath, secondaryTlsTrustStorePath, diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java index 47482bad81018..2ef862a61887b 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import java.io.IOException; +import java.util.Collections; import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; @@ -38,24 +39,27 @@ public void testBuildAutoClusterFailoverInstance() throws PulsarClientException String secondary = "pulsar://localhost:6651"; long failoverDelay = 30; long switchBackDelay = 60; + long checkInterval = 1_000; ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) - .secondary(secondary) + .secondary(Collections.singletonList(secondary)) .failoverDelay(failoverDelay, TimeUnit.SECONDS) .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) + .checkInterval(checkInterval, TimeUnit.MILLISECONDS) .build(); AutoClusterFailover autoClusterFailover = (AutoClusterFailover) provider; Assert.assertTrue(provider instanceof AutoClusterFailover); Assert.assertEquals(primary, provider.getServiceUrl()); Assert.assertEquals(primary, autoClusterFailover.getPrimary()); - Assert.assertEquals(secondary, autoClusterFailover.getSecondary()); + Assert.assertEquals(secondary, autoClusterFailover.getSecondary().get(0)); Assert.assertEquals(TimeUnit.SECONDS.toNanos(failoverDelay), autoClusterFailover.getFailoverDelayNs()); Assert.assertEquals(TimeUnit.SECONDS.toNanos(switchBackDelay), autoClusterFailover.getSwitchBackDelayNs()); + Assert.assertEquals(checkInterval, autoClusterFailover.getIntervalMs()); Assert.assertNull(autoClusterFailover.getPrimaryTlsTrustCertsFilePath()); Assert.assertNull(autoClusterFailover.getPrimaryAuthentication()); - Assert.assertNull(autoClusterFailover.getSecondaryAuthentication()); - Assert.assertNull(autoClusterFailover.getSecondaryTlsTrustCertsFilePath()); + Assert.assertNull(autoClusterFailover.getSecondaryAuthentications()); + Assert.assertNull(autoClusterFailover.getSecondaryTlsTrustCertsFilePaths()); String primaryTlsTrustCertsFilePath = "primary/path"; String secondaryTlsTrustCertsFilePath = "primary/path"; @@ -70,20 +74,22 @@ public void testBuildAutoClusterFailoverInstance() throws PulsarClientException + "tlsKeyFile:/path/to/secondary-role.key-pk8.pem"); ServiceUrlProvider provider1 = AutoClusterFailover.builder() .primary(primary) - .secondary(secondary) + .secondary(Collections.singletonList(secondary)) .failoverDelay(failoverDelay, TimeUnit.SECONDS) .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) + .checkInterval(checkInterval, TimeUnit.MILLISECONDS) .primaryTlsTrustCertsFilePath(primaryTlsTrustCertsFilePath) - .secondaryTlsTrustCertsFilePath(secondaryTlsTrustCertsFilePath) + .secondaryTlsTrustCertsFilePath(Collections.singletonList(secondaryTlsTrustCertsFilePath)) .primaryAuthentication(primaryAuthentication) - .secondaryAuthentication(secondaryAuthentication) + .secondaryAuthentication(Collections.singletonList(secondaryAuthentication)) .build(); AutoClusterFailover autoClusterFailover1 = (AutoClusterFailover) provider1; Assert.assertEquals(primaryTlsTrustCertsFilePath, autoClusterFailover1.getPrimaryTlsTrustCertsFilePath()); Assert.assertEquals(primaryAuthentication, autoClusterFailover1.getPrimaryAuthentication()); - Assert.assertEquals(secondaryTlsTrustCertsFilePath, autoClusterFailover1.getSecondaryTlsTrustCertsFilePath()); - Assert.assertEquals(secondaryAuthentication, autoClusterFailover1.getSecondaryAuthentication()); + Assert.assertEquals(secondaryTlsTrustCertsFilePath, + autoClusterFailover1.getSecondaryTlsTrustCertsFilePaths().get(0)); + Assert.assertEquals(secondaryAuthentication, autoClusterFailover1.getSecondaryAuthentications().get(0)); } @Test @@ -92,18 +98,19 @@ public void testAutoClusterFailoverSwitchWithoutAuthentication() { String secondary = "pulsar://localhost:6651"; long failoverDelay = 0; long switchBackDelay = 0; + long checkInterval = 1_000; ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) - .secondary(secondary) + .secondary(Collections.singletonList(secondary)) .failoverDelay(failoverDelay, TimeUnit.SECONDS) .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) + .checkInterval(checkInterval, TimeUnit.MILLISECONDS) .build(); AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); - Mockito.doReturn(1_000).when(autoClusterFailover).getInterval(); autoClusterFailover.initialize(pulsarClient); @@ -122,6 +129,7 @@ public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException String secondary = "pulsar+ssl://localhost:6661"; long failoverDelay = 0; long switchBackDelay = 0; + long checkInterval = 1_000; String primaryTlsTrustCertsFilePath = "primary/path"; String secondaryTlsTrustCertsFilePath = "primary/path"; Authentication primaryAuthentication = AuthenticationFactory.create( @@ -136,20 +144,20 @@ public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) - .secondary(secondary) + .secondary(Collections.singletonList(secondary)) + .checkInterval(checkInterval, TimeUnit.MILLISECONDS) .failoverDelay(failoverDelay, TimeUnit.SECONDS) .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) .primaryTlsTrustCertsFilePath(primaryTlsTrustCertsFilePath) - .secondaryTlsTrustCertsFilePath(secondaryTlsTrustCertsFilePath) + .secondaryTlsTrustCertsFilePath(Collections.singletonList(secondaryTlsTrustCertsFilePath)) .primaryAuthentication(primaryAuthentication) - .secondaryAuthentication(secondaryAuthentication) + .secondaryAuthentication(Collections.singletonList(secondaryAuthentication)) .build(); AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); - Mockito.doReturn(1_000).when(autoClusterFailover).getInterval(); autoClusterFailover.initialize(pulsarClient); @@ -173,6 +181,7 @@ public void testAutoClusterFailoverSwitchTlsTrustStore() throws IOException { String secondary = "pulsar+ssl://localhost:6661"; long failoverDelay = 0; long switchBackDelay = 0; + long checkInterval = 1_000; String primaryTlsTrustStorePath = "primary/path"; String secondaryTlsTrustStorePath = "secondary/path"; String primaryTlsTrustStorePassword = "primaryPassword"; @@ -180,20 +189,20 @@ public void testAutoClusterFailoverSwitchTlsTrustStore() throws IOException { ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) - .secondary(secondary) + .secondary(Collections.singletonList(secondary)) .failoverDelay(failoverDelay, TimeUnit.SECONDS) .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) + .checkInterval(checkInterval, TimeUnit.MILLISECONDS) .primaryTlsTrustStorePath(primaryTlsTrustStorePath) .primaryTlsTrustStorePassword(primaryTlsTrustStorePassword) - .secondaryTlsTrustStorePath(secondaryTlsTrustStorePath) - .secondaryTlsTrustStorePassword(secondaryTlsTrustStorePassword) + .secondaryTlsTrustStorePath(Collections.singletonList(secondaryTlsTrustStorePath)) + .secondaryTlsTrustStorePassword(Collections.singletonList(secondaryTlsTrustStorePassword)) .build(); AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); - Mockito.doReturn(1_000).when(autoClusterFailover).getInterval(); autoClusterFailover.initialize(pulsarClient); From abfe14f71e9186e8fb8486d0ed613583c4bf83e4 Mon Sep 17 00:00:00 2001 From: chenhang Date: Tue, 4 Jan 2022 17:08:42 +0800 Subject: [PATCH 11/18] format code --- .../api/ControlledClusterFailoverBuilder.java | 9 +++++ .../client/impl/AutoClusterFailover.java | 8 ++-- .../impl/ControlledClusterFailover.java | 39 ++++++++++++++----- .../impl/ControlledClusterFailoverTest.java | 4 +- 4 files changed, 46 insertions(+), 14 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java index 051b39d143bd0..2f1e6f9208c98 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.api; import java.io.IOException; +import java.util.concurrent.TimeUnit; import org.apache.pulsar.common.classification.InterfaceAudience; import org.apache.pulsar.common.classification.InterfaceStability; @@ -46,6 +47,14 @@ public interface ControlledClusterFailoverBuilder { */ ControlledClusterFailoverBuilder urlProvider(String urlProvider); + /** + * Set the probe check interval. + * @param interval + * @param timeUnit + * @return + */ + ControlledClusterFailoverBuilder checkInterval(long interval, TimeUnit timeUnit); + /** * Build the ServiceUrlProvider instance. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index 11cb3bc89b12f..f4486207fc171 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -30,6 +30,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import lombok.Data; +import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AutoClusterFailoverBuilder; @@ -302,13 +303,13 @@ public static class AutoClusterFailoverBuilderImpl implements AutoClusterFailove private long checkIntervalMs = 30_000; @Override - public AutoClusterFailoverBuilder primary(String primary) { + public AutoClusterFailoverBuilder primary(@NonNull String primary) { this.primary = primary; return this; } @Override - public AutoClusterFailoverBuilder secondary(List secondary) { + public AutoClusterFailoverBuilder secondary(@NonNull List secondary) { this.secondary = secondary; return this; } @@ -382,7 +383,8 @@ public AutoClusterFailoverBuilder checkInterval(long interval, TimeUnit timeUnit @Override public ServiceUrlProvider build() { Objects.requireNonNull(primary, "primary service url shouldn't be null"); - Objects.requireNonNull(secondary, "secondary service url shouldn't be null"); + checkArgument(secondary != null && secondary.size() > 0, + "secondary cluster service url shouldn't be null and should set at least one"); checkArgument(failoverDelayNs >= 0, "failoverDelay should >= 0"); checkArgument(switchBackDelayNs >= 0, "switchBackDelay should >= 0"); checkArgument(checkIntervalMs >= 0, "checkInterval should >= 0"); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java index e7ac460aa2781..0a0dea0ab7124 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java @@ -34,6 +34,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import lombok.Data; +import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.IOUtils; import org.apache.pulsar.client.api.Authentication; @@ -42,6 +43,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.checkerframework.checker.nullness.qual.Nullable; @Slf4j public class ControlledClusterFailover implements ServiceUrlProvider { @@ -50,7 +52,7 @@ public class ControlledClusterFailover implements ServiceUrlProvider { private volatile ControlledConfiguration currentControlledConfiguration; private final URL pulsarUrlProvider; private final ScheduledExecutorService executor; - private final int interval = 30_000; + private long interval; private ObjectMapper objectMapper = null; private static final String AUTH_SASL = "org.apache.pulsar.client.impl.auth.AuthenticationSasl"; @@ -60,9 +62,10 @@ public class ControlledClusterFailover implements ServiceUrlProvider { private static final String AUTH_TOKEN = "org.apache.pulsar.client.impl.auth.AuthenticationToken"; private static final String AUTH_KEY_STORE_TLS = "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls"; - private ControlledClusterFailover(String defaultServiceUrl, String urlProvider) throws IOException { + private ControlledClusterFailover(String defaultServiceUrl, String urlProvider, long interval) throws IOException { this.currentPulsarServiceUrl = defaultServiceUrl; this.pulsarUrlProvider = new URL(urlProvider); + this.interval = interval; this.executor = Executors.newSingleThreadScheduledExecutor( new DefaultThreadFactory("pulsar-service-provider")); } @@ -135,11 +138,7 @@ public void initialize(PulsarClient client) { log.error("Failed to switch new Pulsar url, current: {}, new: {}", currentControlledConfiguration, controlledConfiguration, e); } - }), getInterval(), getInterval(), TimeUnit.MILLISECONDS); - } - - public int getInterval() { - return this.interval; + }), interval, interval, TimeUnit.MILLISECONDS); } public String getCurrentPulsarServiceUrl() { @@ -236,19 +235,39 @@ public void close() { public static class ControlledClusterFailoverBuilderImpl implements ControlledClusterFailoverBuilder { private String defaultServiceUrl; private String urlProvider; + private long interval = 30_000; - public ControlledClusterFailoverBuilder defaultServiceUrl(String serviceUrl) { + @Override + public ControlledClusterFailoverBuilder defaultServiceUrl(@NonNull String serviceUrl) { this.defaultServiceUrl = serviceUrl; return this; } - public ControlledClusterFailoverBuilder urlProvider(String urlProvider) { + @Override + public ControlledClusterFailoverBuilder urlProvider(@NonNull String urlProvider) { this.urlProvider = urlProvider; return this; } + @Override + public ControlledClusterFailoverBuilder checkInterval(long interval, @NonNull TimeUnit timeUnit) { + this.interval = timeUnit.toMillis(interval); + return this; + } + + @Override public ServiceUrlProvider build() throws IOException { - return new ControlledClusterFailover(defaultServiceUrl, urlProvider); + Objects.requireNonNull(defaultServiceUrl, "default service url shouldn't be null"); + Objects.requireNonNull(urlProvider, "urlProvider shouldn't be null"); + checkArgument(interval >= 0, "checkInterval should >= 0"); + + return new ControlledClusterFailover(defaultServiceUrl, urlProvider, interval); + } + + public static void checkArgument(boolean expression, @Nullable Object errorMessage) { + if (!expression) { + throw new IllegalArgumentException(String.valueOf(errorMessage)); + } } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java index 41f7d373068f1..cfbf50ab0a05f 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.net.URL; +import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; @@ -56,6 +57,7 @@ public void testControlledClusterFailoverSwitch() throws IOException { String tlsTrustCertsFilePath = "backup/path"; String authPluginClassName = "org.apache.pulsar.client.impl.auth.AuthenticationToken"; String token = "xxxaaabbee"; + long interval = 1_000; ControlledClusterFailover.ControlledConfiguration controlledConfiguration = new ControlledClusterFailover.ControlledConfiguration(); @@ -67,11 +69,11 @@ public void testControlledClusterFailoverSwitch() throws IOException { ServiceUrlProvider provider = ControlledClusterFailover.builder() .defaultServiceUrl(defaultServiceUrl) .urlProvider(urlProvider) + .checkInterval(interval, TimeUnit.MILLISECONDS) .build(); ControlledClusterFailover controlledClusterFailover = Mockito.spy((ControlledClusterFailover) provider); PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); - Mockito.doReturn(1_000).when(controlledClusterFailover).getInterval(); controlledClusterFailover.initialize(pulsarClient); From fecb2dfaa287b7cf8531f2b79e2a86b01ab9a82f Mon Sep 17 00:00:00 2001 From: chenhang Date: Wed, 5 Jan 2022 15:19:55 +0800 Subject: [PATCH 12/18] address comments --- .../api/AutoClusterFailoverBuilder.java | 9 +- .../client/impl/AutoClusterFailover.java | 155 +++++++----------- .../impl/ControlledClusterFailover.java | 54 +----- .../client/impl/AutoClusterFailoverTest.java | 35 +++- .../impl/ControlledClusterFailoverTest.java | 8 +- 5 files changed, 100 insertions(+), 161 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java index ccbf878e8bc41..255a71adb05d5 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.api; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.pulsar.common.classification.InterfaceAudience; import org.apache.pulsar.common.classification.InterfaceStability; @@ -61,7 +62,7 @@ public interface AutoClusterFailoverBuilder { * @param authentication * @return */ - AutoClusterFailoverBuilder secondaryAuthentication(List authentication); + AutoClusterFailoverBuilder secondaryAuthentication(Map authentication); /** * Set primary tlsTrustCertsFilePath. @@ -77,7 +78,7 @@ public interface AutoClusterFailoverBuilder { * @param tlsTrustCertsFilePath * @return */ - AutoClusterFailoverBuilder secondaryTlsTrustCertsFilePath(List tlsTrustCertsFilePath); + AutoClusterFailoverBuilder secondaryTlsTrustCertsFilePath(Map tlsTrustCertsFilePath); /** * Set primary tlsTrustStorePath. @@ -93,7 +94,7 @@ public interface AutoClusterFailoverBuilder { * @param tlsTrustStorePath * @return */ - AutoClusterFailoverBuilder secondaryTlsTrustStorePath(List tlsTrustStorePath); + AutoClusterFailoverBuilder secondaryTlsTrustStorePath(Map tlsTrustStorePath); /** * Set primary tlsTrustStorePassword. @@ -109,7 +110,7 @@ public interface AutoClusterFailoverBuilder { * @param tlsTrustStorePassword * @return */ - AutoClusterFailoverBuilder secondaryTlsTrustStorePassword(List tlsTrustStorePassword); + AutoClusterFailoverBuilder secondaryTlsTrustStorePassword(Map tlsTrustStorePassword); /** * Set the switch failoverDelay. When one cluster failed longer than failoverDelay, it will trigger cluster switch. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index f4486207fc171..f78d2fbe902a7 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -24,7 +24,9 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.Socket; +import java.net.URI; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -36,6 +38,7 @@ import org.apache.pulsar.client.api.AutoClusterFailoverBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.apache.pulsar.common.net.ServiceURI; import org.checkerframework.checker.nullness.qual.Nullable; @Slf4j @@ -46,13 +49,13 @@ public class AutoClusterFailover implements ServiceUrlProvider { private final String primary; private final List secondary; private final Authentication primaryAuthentication; - private final List secondaryAuthentications; + private final Map secondaryAuthentications; private final String primaryTlsTrustCertsFilePath; - private final List secondaryTlsTrustCertsFilePaths; + private final Map secondaryTlsTrustCertsFilePaths; private String primaryTlsTrustStorePath; - private List secondaryTlsTrustStorePaths; + private Map secondaryTlsTrustStorePaths; private String primaryTlsTrustStorePassword; - private List secondaryTlsTrustStorePasswords; + private Map secondaryTlsTrustStorePasswords; private final long failoverDelayNs; private final long switchBackDelayNs; private final ScheduledExecutorService executor; @@ -61,28 +64,23 @@ public class AutoClusterFailover implements ServiceUrlProvider { private final long intervalMs; private static final int TIMEOUT = 30_000; - private AutoClusterFailover(String primary, List secondary, long failoverDelayNs, long switchBackDelayNs, - long intervalMs, Authentication primaryAuthentication, - List secondaryAuthentications, String primaryTlsTrustCertsFilePath, - List secondaryTlsTrustCertsFilePaths, String primaryTlsTrustStorePath, - List secondaryTlsTrustStorePaths, String primaryTlsTrustStorePassword, - List secondaryTlsTrustStorePasswords) { - this.primary = primary; - this.secondary = secondary; - this.primaryAuthentication = primaryAuthentication; - this.secondaryAuthentications = secondaryAuthentications; - this.primaryTlsTrustCertsFilePath = primaryTlsTrustCertsFilePath; - this.secondaryTlsTrustCertsFilePaths = secondaryTlsTrustCertsFilePaths; - this.primaryTlsTrustStorePath = primaryTlsTrustStorePath; - this.secondaryTlsTrustStorePaths = secondaryTlsTrustStorePaths; - this.primaryTlsTrustStorePassword = primaryTlsTrustStorePassword; - this.secondaryTlsTrustStorePasswords = secondaryTlsTrustStorePasswords; - this.failoverDelayNs = failoverDelayNs; - this.switchBackDelayNs = switchBackDelayNs; - this.currentPulsarServiceUrl = primary; + private AutoClusterFailover(AutoClusterFailoverBuilderImpl builder) { + this.primary = builder.primary; + this.secondary = builder.secondary; + this.primaryAuthentication = builder.primaryAuthentication; + this.secondaryAuthentications = builder.secondaryAuthentications; + this.primaryTlsTrustCertsFilePath = builder.primaryTlsTrustCertsFilePath; + this.secondaryTlsTrustCertsFilePaths = builder.secondaryTlsTrustCertsFilePaths; + this.primaryTlsTrustStorePath = builder.primaryTlsTrustStorePath; + this.secondaryTlsTrustStorePaths = builder.secondaryTlsTrustStorePaths; + this.primaryTlsTrustStorePassword = builder.primaryTlsTrustStorePassword; + this.secondaryTlsTrustStorePasswords = builder.secondaryTlsTrustStorePasswords; + this.failoverDelayNs = builder.failoverDelayNs; + this.switchBackDelayNs = builder.switchBackDelayNs; + this.currentPulsarServiceUrl = builder.primary; this.recoverTimestamp = -1; this.failedTimestamp = -1; - this.intervalMs = intervalMs; + this.intervalMs = builder.checkIntervalMs; this.executor = Executors.newSingleThreadScheduledExecutor( new DefaultThreadFactory("pulsar-service-provider")); } @@ -121,13 +119,9 @@ public void close() { boolean probeAvailable(String url) { try { - String hostAndPort = parseHostAndPort(url); - if (Strings.isNullOrEmpty(hostAndPort)) { - return false; - } - + URI uri = ServiceURI.create(url).getUri(); Socket socket = new Socket(); - socket.connect(new InetSocketAddress(parseHost(hostAndPort), parsePort(hostAndPort)), TIMEOUT); + socket.connect(new InetSocketAddress(uri.getHost(), uri.getPort()), TIMEOUT); socket.close(); return true; } catch (Exception e) { @@ -136,34 +130,6 @@ boolean probeAvailable(String url) { } } - private static String parseHostAndPort(String url) { - if (Strings.isNullOrEmpty(url) || !url.startsWith("pulsar")) { - throw new IllegalArgumentException("'" + url + "' isn't an Pulsar service URL"); - } - - int uriSeparatorPos = url.indexOf("://"); - if (uriSeparatorPos == -1) { - throw new IllegalArgumentException("'" + url + "' isn't an URI."); - } - return url.substring(uriSeparatorPos + 3); - } - - private static String parseHost(String hostAndPort) { - int portSeparatorPos = hostAndPort.indexOf(":"); - if (portSeparatorPos == -1) { - throw new IllegalArgumentException("'" + hostAndPort + "' isn't an URI."); - } - return hostAndPort.substring(0, portSeparatorPos); - } - - private static Integer parsePort(String hostAndPort) { - int portSeparatorPos = hostAndPort.indexOf(":"); - if (portSeparatorPos == -1) { - throw new IllegalArgumentException("'" + hostAndPort + "' isn't an URI."); - } - return Integer.valueOf(hostAndPort.substring(portSeparatorPos+1)); - } - private static long nanosToMillis(long nanos) { return Math.max(0L, Math.round(nanos / 1_000_000.0d)); } @@ -195,10 +161,10 @@ private void updateServiceUrl(String target, } private void probeAndUpdateServiceUrl(List targetServiceUrls, - List authentications, - List tlsTrustCertsFilePaths, - List tlsTrustStorePaths, - List tlsTrustStorePasswords) { + Map authentications, + Map tlsTrustCertsFilePaths, + Map tlsTrustStorePaths, + Map tlsTrustStorePasswords) { if (probeAvailable(currentPulsarServiceUrl)) { failedTimestamp = -1; return; @@ -208,25 +174,24 @@ private void probeAndUpdateServiceUrl(List targetServiceUrls, if (failedTimestamp == -1) { failedTimestamp = currentTimestamp; } else if (currentTimestamp - failedTimestamp >= failoverDelayNs) { - for (int i = 0; i < targetServiceUrls.size(); ++i) { - if (probeAvailable(targetServiceUrls.get(i))) { + for (String targetServiceUrl : targetServiceUrls) { + if (probeAvailable(targetServiceUrl)) { log.info("Current Pulsar service is {}, it has been down for {} ms, " + "switch to the service {}. The current service down at {}", currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), - targetServiceUrls.get(i), failedTimestamp); - updateServiceUrl(targetServiceUrls.get(i), - authentications != null ? authentications.get(i) : null, - tlsTrustCertsFilePaths != null ? tlsTrustCertsFilePaths.get(i) : null, - tlsTrustStorePaths != null ? tlsTrustStorePaths.get(i) : null, - tlsTrustStorePasswords != null ? tlsTrustStorePasswords.get(i) : null); + targetServiceUrl, failedTimestamp); + updateServiceUrl(targetServiceUrl, + authentications != null ? authentications.get(targetServiceUrl) : null, + tlsTrustCertsFilePaths != null ? tlsTrustCertsFilePaths.get(targetServiceUrl) : null, + tlsTrustStorePaths != null ? tlsTrustStorePaths.get(targetServiceUrl) : null, + tlsTrustStorePasswords != null ? tlsTrustStorePasswords.get(targetServiceUrl) : null); failedTimestamp = -1; break; } else { log.warn("Current Pulsar service is {}, it has been down for {} ms. " + "Failed to switch to service {}, " + "because it is not available, continue to probe next pulsar service.", - currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), - targetServiceUrls.get(i)); + currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), targetServiceUrl); } } } @@ -291,13 +256,13 @@ public static class AutoClusterFailoverBuilderImpl implements AutoClusterFailove private String primary; private List secondary; private Authentication primaryAuthentication = null; - private List secondaryAuthentication = null; + private Map secondaryAuthentications = null; private String primaryTlsTrustCertsFilePath = null; - private List secondaryTlsTrustCertsFilePath = null; + private Map secondaryTlsTrustCertsFilePaths = null; private String primaryTlsTrustStorePath = null; - private List secondaryTlsTrustStorePath = null; + private Map secondaryTlsTrustStorePaths = null; private String primaryTlsTrustStorePassword = null; - private List secondaryTlsTrustStorePassword = null; + private Map secondaryTlsTrustStorePasswords = null; private long failoverDelayNs; private long switchBackDelayNs; private long checkIntervalMs = 30_000; @@ -321,8 +286,8 @@ public AutoClusterFailoverBuilder primaryAuthentication(Authentication authentic } @Override - public AutoClusterFailoverBuilder secondaryAuthentication(List authentication) { - this.secondaryAuthentication = authentication; + public AutoClusterFailoverBuilder secondaryAuthentication(Map authentication) { + this.secondaryAuthentications = authentication; return this; } @@ -333,8 +298,8 @@ public AutoClusterFailoverBuilder primaryTlsTrustCertsFilePath(String tlsTrustCe } @Override - public AutoClusterFailoverBuilder secondaryTlsTrustCertsFilePath(List tlsTrustCertsFilePath) { - this.secondaryTlsTrustCertsFilePath = tlsTrustCertsFilePath; + public AutoClusterFailoverBuilder secondaryTlsTrustCertsFilePath(Map tlsTrustCertsFilePath) { + this.secondaryTlsTrustCertsFilePaths = tlsTrustCertsFilePath; return this; } @@ -345,8 +310,8 @@ public AutoClusterFailoverBuilder primaryTlsTrustStorePath(String tlsTrustStoreP } @Override - public AutoClusterFailoverBuilder secondaryTlsTrustStorePath(List tlsTrustStorePath) { - this.secondaryTlsTrustStorePath = tlsTrustStorePath; + public AutoClusterFailoverBuilder secondaryTlsTrustStorePath(Map tlsTrustStorePath) { + this.secondaryTlsTrustStorePaths = tlsTrustStorePath; return this; } @@ -357,8 +322,8 @@ public AutoClusterFailoverBuilder primaryTlsTrustStorePassword(String tlsTrustSt } @Override - public AutoClusterFailoverBuilder secondaryTlsTrustStorePassword(List tlsTrustStorePassword) { - this.secondaryTlsTrustStorePassword = tlsTrustStorePassword; + public AutoClusterFailoverBuilder secondaryTlsTrustStorePassword(Map tlsTrustStorePassword) { + this.secondaryTlsTrustStorePasswords = tlsTrustStorePassword; return this; } @@ -390,24 +355,20 @@ public ServiceUrlProvider build() { checkArgument(checkIntervalMs >= 0, "checkInterval should >= 0"); int secondarySize = secondary.size(); - checkArgument(secondaryAuthentication == null - || secondaryAuthentication.size() == secondarySize, + checkArgument(secondaryAuthentications == null + || secondaryAuthentications.size() == secondarySize, "secondaryAuthentication should be null or size equal with secondary url size"); - checkArgument(secondaryTlsTrustCertsFilePath == null - || secondaryTlsTrustCertsFilePath.size() == secondarySize, + checkArgument(secondaryTlsTrustCertsFilePaths == null + || secondaryTlsTrustCertsFilePaths.size() == secondarySize, "secondaryTlsTrustCertsFilePath should be null or size equal with secondary url size"); - checkArgument(secondaryTlsTrustStorePath == null - || secondaryTlsTrustStorePath.size() == secondarySize, + checkArgument(secondaryTlsTrustStorePaths == null + || secondaryTlsTrustStorePaths.size() == secondarySize, "secondaryTlsTrustStorePath should be null or size equal with secondary url size"); - checkArgument(secondaryTlsTrustStorePassword == null - || secondaryTlsTrustStorePassword.size() == secondarySize, + checkArgument(secondaryTlsTrustStorePasswords == null + || secondaryTlsTrustStorePasswords.size() == secondarySize, "secondaryTlsTrustStorePassword should be null or size equal with secondary url size"); - return new AutoClusterFailover(primary, secondary, failoverDelayNs, switchBackDelayNs, checkIntervalMs, - primaryAuthentication, secondaryAuthentication, - primaryTlsTrustCertsFilePath, secondaryTlsTrustCertsFilePath, - primaryTlsTrustStorePath, secondaryTlsTrustStorePath, - primaryTlsTrustStorePassword, secondaryTlsTrustStorePassword); + return new AutoClusterFailover(this); } public static void checkArgument(boolean expression, @Nullable Object errorMessage) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java index 0a0dea0ab7124..5bcccaefd12ba 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java @@ -28,7 +28,6 @@ import java.net.URL; import java.net.URLConnection; import java.nio.charset.StandardCharsets; -import java.util.Map; import java.util.Objects; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -55,13 +54,6 @@ public class ControlledClusterFailover implements ServiceUrlProvider { private long interval; private ObjectMapper objectMapper = null; - private static final String AUTH_SASL = "org.apache.pulsar.client.impl.auth.AuthenticationSasl"; - private static final String AUTH_ATHENZ = "org.apache.pulsar.client.impl.auth.AuthenticationAthenz"; - private static final String AUTH_OAUTH2 = "org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2"; - private static final String AUTH_TLS = "org.apache.pulsar.client.impl.auth.AuthenticationTls"; - private static final String AUTH_TOKEN = "org.apache.pulsar.client.impl.auth.AuthenticationToken"; - private static final String AUTH_KEY_STORE_TLS = "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls"; - private ControlledClusterFailover(String defaultServiceUrl, String urlProvider, long interval) throws IOException { this.currentPulsarServiceUrl = defaultServiceUrl; this.pulsarUrlProvider = new URL(urlProvider); @@ -86,37 +78,10 @@ public void initialize(PulsarClient client) { currentControlledConfiguration, controlledConfiguration.toString()); Authentication authentication = null; - if (!Strings.isNullOrEmpty(controlledConfiguration.authPluginClassName)) { - String authPluginClassName = controlledConfiguration.authPluginClassName; - Map authParams = controlledConfiguration.getAuthParams(); - String authParamsString = controlledConfiguration.getAuthParamsString(); - String token = controlledConfiguration.getToken(); - - switch (authPluginClassName) { - case AUTH_SASL: - case AUTH_ATHENZ: - if (authParams != null && !authParams.isEmpty()) { - authentication = - AuthenticationFactory.create(authPluginClassName, authParams); - } - break; - - case AUTH_TOKEN: - if (!Strings.isNullOrEmpty(token)) { - authentication = AuthenticationFactory.token(token); - } - break; - - case AUTH_OAUTH2: - case AUTH_TLS: - case AUTH_KEY_STORE_TLS: - default: - if (!Strings.isNullOrEmpty(authParamsString)) { - authentication = - AuthenticationFactory.create(authPluginClassName, authParamsString); - } - break; - } + if (!Strings.isNullOrEmpty(controlledConfiguration.authPluginClassName) + && !Strings.isNullOrEmpty(controlledConfiguration.getAuthParamsString())) { + authentication = AuthenticationFactory.create(controlledConfiguration.getAuthPluginClassName(), + controlledConfiguration.getAuthParamsString()); } String tlsTrustCertsFilePath = controlledConfiguration.getTlsTrustCertsFilePath(); @@ -181,10 +146,7 @@ protected static class ControlledConfiguration { private String tlsTrustCertsFilePath; private String authPluginClassName; - private String authParamsString; - private String token; - private Map authParams; public String toJson() { ObjectMapper objectMapper = ObjectMapperFactory.getThreadLocal(); @@ -203,9 +165,7 @@ public boolean equals(Object obj) { return Objects.equals(serviceUrl, other.serviceUrl) && Objects.equals(tlsTrustCertsFilePath, other.tlsTrustCertsFilePath) && Objects.equals(authPluginClassName, other.authPluginClassName) - && Objects.equals(authParamsString, other.authParamsString) - && Objects.equals(token, other.token) - && Objects.equals(authParams, other.authParams); + && Objects.equals(authParamsString, other.authParamsString); } return false; @@ -216,9 +176,7 @@ public int hashCode() { return Objects.hash(serviceUrl, tlsTrustCertsFilePath, authPluginClassName, - authParamsString, - token, - authParams); + authParamsString); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java index 2ef862a61887b..af24a4ff206c4 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; @@ -72,6 +74,12 @@ public void testBuildAutoClusterFailoverInstance() throws PulsarClientException "org.apache.pulsar.client.impl.auth.AuthenticationTls", "tlsCertFile:/path/to/secondary-my-role.cert.pem," + "tlsKeyFile:/path/to/secondary-role.key-pk8.pem"); + Map secondaryTlsTrustCertsFilePaths = new HashMap<>(); + secondaryTlsTrustCertsFilePaths.put(secondary, secondaryTlsTrustCertsFilePath); + + Map secondaryAuthentications = new HashMap<>(); + secondaryAuthentications.put(secondary, secondaryAuthentication); + ServiceUrlProvider provider1 = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) @@ -79,17 +87,17 @@ public void testBuildAutoClusterFailoverInstance() throws PulsarClientException .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) .checkInterval(checkInterval, TimeUnit.MILLISECONDS) .primaryTlsTrustCertsFilePath(primaryTlsTrustCertsFilePath) - .secondaryTlsTrustCertsFilePath(Collections.singletonList(secondaryTlsTrustCertsFilePath)) + .secondaryTlsTrustCertsFilePath(secondaryTlsTrustCertsFilePaths) .primaryAuthentication(primaryAuthentication) - .secondaryAuthentication(Collections.singletonList(secondaryAuthentication)) + .secondaryAuthentication(secondaryAuthentications) .build(); AutoClusterFailover autoClusterFailover1 = (AutoClusterFailover) provider1; Assert.assertEquals(primaryTlsTrustCertsFilePath, autoClusterFailover1.getPrimaryTlsTrustCertsFilePath()); Assert.assertEquals(primaryAuthentication, autoClusterFailover1.getPrimaryAuthentication()); Assert.assertEquals(secondaryTlsTrustCertsFilePath, - autoClusterFailover1.getSecondaryTlsTrustCertsFilePaths().get(0)); - Assert.assertEquals(secondaryAuthentication, autoClusterFailover1.getSecondaryAuthentications().get(0)); + autoClusterFailover1.getSecondaryTlsTrustCertsFilePaths().get(secondary)); + Assert.assertEquals(secondaryAuthentication, autoClusterFailover1.getSecondaryAuthentications().get(secondary)); } @Test @@ -142,6 +150,12 @@ public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException "tlsCertFile:/path/to/secondary-my-role.cert.pem," + "tlsKeyFile:/path/to/secondary-role.key-pk8.pem"); + Map secondaryTlsTrustCertsFilePaths = new HashMap<>(); + secondaryTlsTrustCertsFilePaths.put(secondary, secondaryTlsTrustCertsFilePath); + + Map secondaryAuthentications = new HashMap<>(); + secondaryAuthentications.put(secondary, secondaryAuthentication); + ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) @@ -149,9 +163,9 @@ public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException .failoverDelay(failoverDelay, TimeUnit.SECONDS) .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) .primaryTlsTrustCertsFilePath(primaryTlsTrustCertsFilePath) - .secondaryTlsTrustCertsFilePath(Collections.singletonList(secondaryTlsTrustCertsFilePath)) + .secondaryTlsTrustCertsFilePath(secondaryTlsTrustCertsFilePaths) .primaryAuthentication(primaryAuthentication) - .secondaryAuthentication(Collections.singletonList(secondaryAuthentication)) + .secondaryAuthentication(secondaryAuthentications) .build(); AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); @@ -187,6 +201,11 @@ public void testAutoClusterFailoverSwitchTlsTrustStore() throws IOException { String primaryTlsTrustStorePassword = "primaryPassword"; String secondaryTlsTrustStorePassword = "secondaryPassword"; + Map secondaryTlsTrustStorePaths = new HashMap<>(); + secondaryTlsTrustStorePaths.put(secondary, secondaryTlsTrustStorePath); + Map secondaryTlsTrustStorePasswords = new HashMap<>(); + secondaryTlsTrustStorePasswords.put(secondary, secondaryTlsTrustStorePassword); + ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) @@ -195,8 +214,8 @@ public void testAutoClusterFailoverSwitchTlsTrustStore() throws IOException { .checkInterval(checkInterval, TimeUnit.MILLISECONDS) .primaryTlsTrustStorePath(primaryTlsTrustStorePath) .primaryTlsTrustStorePassword(primaryTlsTrustStorePassword) - .secondaryTlsTrustStorePath(Collections.singletonList(secondaryTlsTrustStorePath)) - .secondaryTlsTrustStorePassword(Collections.singletonList(secondaryTlsTrustStorePassword)) + .secondaryTlsTrustStorePath(secondaryTlsTrustStorePaths) + .secondaryTlsTrustStorePassword(secondaryTlsTrustStorePasswords) .build(); AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java index cfbf50ab0a05f..27e983ce1ea61 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java @@ -56,7 +56,7 @@ public void testControlledClusterFailoverSwitch() throws IOException { String urlProvider = "http://localhost:8080"; String tlsTrustCertsFilePath = "backup/path"; String authPluginClassName = "org.apache.pulsar.client.impl.auth.AuthenticationToken"; - String token = "xxxaaabbee"; + String authParamsString = "token:xxxaaabbee"; long interval = 1_000; ControlledClusterFailover.ControlledConfiguration controlledConfiguration = @@ -64,7 +64,7 @@ public void testControlledClusterFailoverSwitch() throws IOException { controlledConfiguration.setServiceUrl(backupServiceUrl); controlledConfiguration.setTlsTrustCertsFilePath(tlsTrustCertsFilePath); controlledConfiguration.setAuthPluginClassName(authPluginClassName); - controlledConfiguration.setToken(token); + controlledConfiguration.setAuthParamsString(authParamsString); ServiceUrlProvider provider = ControlledClusterFailover.builder() .defaultServiceUrl(defaultServiceUrl) @@ -95,13 +95,13 @@ public void testControlledClusterFailoverSwitch() throws IOException { String backupServiceUrlV1 = "pulsar+ssl://localhost:6662"; String tlsTrustCertsFilePathV1 = "backup/pathV1"; String authPluginClassNameV1 = "org.apache.pulsar.client.impl.auth.AuthenticationToken"; - String tokenV1 = "xxxaaabbeev1"; + String authParamsStringV1 = "token:xxxaaabbeev1"; ControlledClusterFailover.ControlledConfiguration controlledConfiguration1 = new ControlledClusterFailover.ControlledConfiguration(); controlledConfiguration1.setServiceUrl(backupServiceUrlV1); controlledConfiguration1.setTlsTrustCertsFilePath(tlsTrustCertsFilePathV1); controlledConfiguration1.setAuthPluginClassName(authPluginClassNameV1); - controlledConfiguration1.setToken(tokenV1); + controlledConfiguration1.setAuthParamsString(authParamsStringV1); Mockito.doReturn(controlledConfiguration1).when(controlledClusterFailover) .fetchControlledConfiguration(); From 8017d6787a75d220a366fdebdb34ecccf6af168a Mon Sep 17 00:00:00 2001 From: chenhang Date: Wed, 5 Jan 2022 15:27:27 +0800 Subject: [PATCH 13/18] address comments --- .../org/apache/pulsar/client/impl/PulsarClientImpl.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 18c135988759d..e9fcaee241d7e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -755,7 +755,7 @@ public void shutdown() throws PulsarClientException { } // close the service url provider allocated resource. - if (conf.getServiceUrlProvider() != null) { + if (conf != null && conf.getServiceUrlProvider() != null) { conf.getServiceUrlProvider().close(); } @@ -786,7 +786,7 @@ public void shutdown() throws PulsarClientException { } catch (PulsarClientException e) { throwable = e; } - if (conf.getAuthentication() != null) { + if (conf != null && conf.getAuthentication() != null) { try { conf.getAuthentication().close(); } catch (Throwable t) { @@ -868,7 +868,9 @@ public synchronized void updateServiceUrl(String serviceUrl) throws PulsarClient @Override public void updateAuthentication(Authentication authentication) throws IOException { log.info("Updating authentication to {}", authentication); - conf.getAuthentication().close(); + if (conf.getAuthentication() != null) { + conf.getAuthentication().close(); + } conf.setAuthentication(authentication); conf.getAuthentication().start(); } From 1c8730f60c2457e5fe2979cd1407eca2359bc325 Mon Sep 17 00:00:00 2001 From: chenhang Date: Thu, 6 Jan 2022 22:46:19 +0800 Subject: [PATCH 14/18] add secondaryChoosePolicy support --- .../client/api/AutoClusterFailoverBuilder.java | 12 ++++++++++++ .../pulsar/client/impl/AutoClusterFailover.java | 9 +++++++++ 2 files changed, 21 insertions(+) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java index 255a71adb05d5..129e031eb7a8c 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java @@ -32,6 +32,11 @@ @InterfaceAudience.Public @InterfaceStability.Evolving public interface AutoClusterFailoverBuilder { + + @SuppressWarnings("checkstyle:javadoctype") + enum SecondaryChoosePolicy { + ORDER + } /** * Set the primary service url. * @@ -48,6 +53,13 @@ public interface AutoClusterFailoverBuilder { */ AutoClusterFailoverBuilder secondary(List secondary); + /** + * Set secondary choose policy + * @param policy + * @return + */ + AutoClusterFailoverBuilder secondaryChoosePolicy(SecondaryChoosePolicy policy); + /** * Set primary authentication. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index f78d2fbe902a7..b21aff0240808 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -48,6 +48,7 @@ public class AutoClusterFailover implements ServiceUrlProvider { private volatile String currentPulsarServiceUrl; private final String primary; private final List secondary; + private final AutoClusterFailoverBuilder.SecondaryChoosePolicy secondaryChoosePolicy; private final Authentication primaryAuthentication; private final Map secondaryAuthentications; private final String primaryTlsTrustCertsFilePath; @@ -67,6 +68,7 @@ public class AutoClusterFailover implements ServiceUrlProvider { private AutoClusterFailover(AutoClusterFailoverBuilderImpl builder) { this.primary = builder.primary; this.secondary = builder.secondary; + this.secondaryChoosePolicy = builder.secondaryChoosePolicy; this.primaryAuthentication = builder.primaryAuthentication; this.secondaryAuthentications = builder.secondaryAuthentications; this.primaryTlsTrustCertsFilePath = builder.primaryTlsTrustCertsFilePath; @@ -263,6 +265,7 @@ public static class AutoClusterFailoverBuilderImpl implements AutoClusterFailove private Map secondaryTlsTrustStorePaths = null; private String primaryTlsTrustStorePassword = null; private Map secondaryTlsTrustStorePasswords = null; + private SecondaryChoosePolicy secondaryChoosePolicy = SecondaryChoosePolicy.ORDER; private long failoverDelayNs; private long switchBackDelayNs; private long checkIntervalMs = 30_000; @@ -279,6 +282,12 @@ public AutoClusterFailoverBuilder secondary(@NonNull List secondary) { return this; } + @Override + public AutoClusterFailoverBuilder secondaryChoosePolicy(@NonNull SecondaryChoosePolicy policy) { + this.secondaryChoosePolicy = policy; + return this; + } + @Override public AutoClusterFailoverBuilder primaryAuthentication(Authentication authentication) { this.primaryAuthentication = authentication; From 1f7f9a8e61849c9fff0dcd6818e76497ce9250c7 Mon Sep 17 00:00:00 2001 From: chenhang Date: Thu, 6 Jan 2022 22:50:52 +0800 Subject: [PATCH 15/18] add default value doc --- .../apache/pulsar/client/api/AutoClusterFailoverBuilder.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java index 129e031eb7a8c..385e0abb67b9d 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java @@ -54,7 +54,7 @@ enum SecondaryChoosePolicy { AutoClusterFailoverBuilder secondary(List secondary); /** - * Set secondary choose policy + * Set secondary choose policy. The default secondary choose policy is `ORDER`. * @param policy * @return */ From 5f50a288d96e05815f63e762b4c0aabe07910108 Mon Sep 17 00:00:00 2001 From: chenhang Date: Wed, 26 Jan 2022 22:45:03 +0800 Subject: [PATCH 16/18] support http header --- .../api/AutoClusterFailoverBuilder.java | 38 +----- .../api/ControlledClusterFailoverBuilder.java | 10 +- .../pulsar/client/api/PulsarClient.java | 26 ----- pulsar-client/pom.xml | 5 + .../client/impl/AutoClusterFailover.java | 71 ++++------- .../impl/ControlledClusterFailover.java | 110 ++++++++++-------- .../pulsar/client/impl/PulsarClientImpl.java | 3 - .../client/impl/AutoClusterFailoverTest.java | 46 +++++--- .../impl/ControlledClusterFailoverTest.java | 39 ++++--- 9 files changed, 153 insertions(+), 195 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java index 385e0abb67b9d..f3c1de2227e36 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java @@ -25,7 +25,7 @@ import org.apache.pulsar.common.classification.InterfaceStability; /** - * {@link AutoClusterFailoverBuilder} is used to configure and create instance of {@link ServiceUrlProvider} + * {@link AutoClusterFailoverBuilder} is used to configure and create instance of {@link ServiceUrlProvider}. * * @since 2.10.0 */ @@ -34,7 +34,7 @@ public interface AutoClusterFailoverBuilder { @SuppressWarnings("checkstyle:javadoctype") - enum SecondaryChoosePolicy { + enum FailoverPolicy { ORDER } /** @@ -58,15 +58,7 @@ enum SecondaryChoosePolicy { * @param policy * @return */ - AutoClusterFailoverBuilder secondaryChoosePolicy(SecondaryChoosePolicy policy); - - /** - * Set primary authentication. - * - * @param authentication - * @return - */ - AutoClusterFailoverBuilder primaryAuthentication(Authentication authentication); + AutoClusterFailoverBuilder failoverPolicy(FailoverPolicy policy); /** * Set secondary authentication. @@ -76,14 +68,6 @@ enum SecondaryChoosePolicy { */ AutoClusterFailoverBuilder secondaryAuthentication(Map authentication); - /** - * Set primary tlsTrustCertsFilePath. - * - * @param tlsTrustCertsFilePath - * @return - */ - AutoClusterFailoverBuilder primaryTlsTrustCertsFilePath(String tlsTrustCertsFilePath); - /** * Set secondary tlsTrustCertsFilePath. * @@ -92,14 +76,6 @@ enum SecondaryChoosePolicy { */ AutoClusterFailoverBuilder secondaryTlsTrustCertsFilePath(Map tlsTrustCertsFilePath); - /** - * Set primary tlsTrustStorePath. - * - * @param tlsTrustStorePath - * @return - */ - AutoClusterFailoverBuilder primaryTlsTrustStorePath(String tlsTrustStorePath); - /** * Set secondary tlsTrustStorePath. * @@ -108,14 +84,6 @@ enum SecondaryChoosePolicy { */ AutoClusterFailoverBuilder secondaryTlsTrustStorePath(Map tlsTrustStorePath); - /** - * Set primary tlsTrustStorePassword. - * - * @param tlsTrustStorePassword - * @return - */ - AutoClusterFailoverBuilder primaryTlsTrustStorePassword(String tlsTrustStorePassword); - /** * Set secondary tlsTrustStorePassword. * diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java index 2f1e6f9208c98..21f745bb8cdb8 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ControlledClusterFailoverBuilder.java @@ -19,12 +19,13 @@ package org.apache.pulsar.client.api; import java.io.IOException; +import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.pulsar.common.classification.InterfaceAudience; import org.apache.pulsar.common.classification.InterfaceStability; /** - * {@link ControlledClusterFailoverBuilder} is used to configure and create instance of {@link ServiceUrlProvider} + * {@link ControlledClusterFailoverBuilder} is used to configure and create instance of {@link ServiceUrlProvider}. * * @since 2.10.0 */ @@ -47,6 +48,13 @@ public interface ControlledClusterFailoverBuilder { */ ControlledClusterFailoverBuilder urlProvider(String urlProvider); + /** + * Set the service url provider header to authenticate provider service. + * @param header + * @return + */ + ControlledClusterFailoverBuilder urlProviderHeader(Map header); + /** * Set the probe check interval. * @param interval diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java index f737d6bacdecb..83097f6872136 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java @@ -19,7 +19,6 @@ package org.apache.pulsar.client.api; import java.io.Closeable; -import java.io.IOException; import java.util.List; import java.util.concurrent.CompletableFuture; import org.apache.pulsar.client.api.transaction.TransactionBuilder; @@ -257,31 +256,6 @@ static ClientBuilder builder() { */ void updateServiceUrl(String serviceUrl) throws PulsarClientException; - /** - * Update the authentication this client is using. - * - * @param authentication - * - * @throws IOException - */ - void updateAuthentication(Authentication authentication) - throws IOException; - - /** - * Update the tlsTrustCertsFilePath this client is using. - * - * @param tlsTrustCertsFilePath - */ - void updateTlsTrustCertsFilePath(String tlsTrustCertsFilePath); - - /** - * Update the tlsTrustStorePath and tlsTrustStorePassword this client is using. - * - * @param tlsTrustStorePath - * @param tlsTrustStorePassword - */ - void updateTlsTrustStorePathAndPassword(String tlsTrustStorePath, String tlsTrustStorePassword); - /** * Get the list of partitions for a given topic. * diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 5338d23241c0a..625e61cc726d9 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -114,6 +114,11 @@ gson + + org.apache.httpcomponents + httpclient + + diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index b21aff0240808..726f9b1780768 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -38,20 +38,20 @@ import org.apache.pulsar.client.api.AutoClusterFailoverBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.common.net.ServiceURI; -import org.checkerframework.checker.nullness.qual.Nullable; @Slf4j @Data public class AutoClusterFailover implements ServiceUrlProvider { - private PulsarClient pulsarClient; + private PulsarClientImpl pulsarClient; private volatile String currentPulsarServiceUrl; private final String primary; private final List secondary; - private final AutoClusterFailoverBuilder.SecondaryChoosePolicy secondaryChoosePolicy; - private final Authentication primaryAuthentication; + private final AutoClusterFailoverBuilder.FailoverPolicy failoverPolicy; + private Authentication primaryAuthentication; private final Map secondaryAuthentications; - private final String primaryTlsTrustCertsFilePath; + private String primaryTlsTrustCertsFilePath; private final Map secondaryTlsTrustCertsFilePaths; private String primaryTlsTrustStorePath; private Map secondaryTlsTrustStorePaths; @@ -68,14 +68,10 @@ public class AutoClusterFailover implements ServiceUrlProvider { private AutoClusterFailover(AutoClusterFailoverBuilderImpl builder) { this.primary = builder.primary; this.secondary = builder.secondary; - this.secondaryChoosePolicy = builder.secondaryChoosePolicy; - this.primaryAuthentication = builder.primaryAuthentication; + this.failoverPolicy = builder.failoverPolicy; this.secondaryAuthentications = builder.secondaryAuthentications; - this.primaryTlsTrustCertsFilePath = builder.primaryTlsTrustCertsFilePath; this.secondaryTlsTrustCertsFilePaths = builder.secondaryTlsTrustCertsFilePaths; - this.primaryTlsTrustStorePath = builder.primaryTlsTrustStorePath; this.secondaryTlsTrustStorePaths = builder.secondaryTlsTrustStorePaths; - this.primaryTlsTrustStorePassword = builder.primaryTlsTrustStorePassword; this.secondaryTlsTrustStorePasswords = builder.secondaryTlsTrustStorePasswords; this.failoverDelayNs = builder.failoverDelayNs; this.switchBackDelayNs = builder.switchBackDelayNs; @@ -89,7 +85,14 @@ private AutoClusterFailover(AutoClusterFailoverBuilderImpl builder) { @Override public void initialize(PulsarClient client) { - this.pulsarClient = client; + this.pulsarClient = (PulsarClientImpl) client; + ClientConfigurationData config = pulsarClient.getConfiguration(); + if (config != null) { + this.primaryAuthentication = config.getAuthentication(); + this.primaryTlsTrustCertsFilePath = config.getTlsTrustCertsFilePath(); + this.primaryTlsTrustStorePath = config.getTlsTrustStorePath(); + this.primaryTlsTrustStorePassword = config.getTlsTrustStorePassword(); + } // start to probe primary cluster active or not this.executor.scheduleAtFixedRate(catchingAndLoggingThrowables(() -> { @@ -101,7 +104,7 @@ public void initialize(PulsarClient client) { // current service url is secondary, probe whether it is down probeAndUpdateServiceUrl(primary, primaryAuthentication, primaryTlsTrustCertsFilePath, primaryTlsTrustStorePath, primaryTlsTrustStorePassword); - // secondary cluster is up, check whether need to switch back to primary + // secondary cluster is up, check whether need to switch back to primary or not probeAndCheckSwitchBack(primary, primaryAuthentication, primaryTlsTrustCertsFilePath, primaryTlsTrustStorePath, primaryTlsTrustStorePassword); } @@ -193,7 +196,7 @@ currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), log.warn("Current Pulsar service is {}, it has been down for {} ms. " + "Failed to switch to service {}, " + "because it is not available, continue to probe next pulsar service.", - currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), targetServiceUrl); + currentPulsarServiceUrl, nanosToMillis(currentTimestamp - failedTimestamp), targetServiceUrl); } } } @@ -257,15 +260,11 @@ private void probeAndCheckSwitchBack(String target, public static class AutoClusterFailoverBuilderImpl implements AutoClusterFailoverBuilder { private String primary; private List secondary; - private Authentication primaryAuthentication = null; private Map secondaryAuthentications = null; - private String primaryTlsTrustCertsFilePath = null; private Map secondaryTlsTrustCertsFilePaths = null; - private String primaryTlsTrustStorePath = null; private Map secondaryTlsTrustStorePaths = null; - private String primaryTlsTrustStorePassword = null; private Map secondaryTlsTrustStorePasswords = null; - private SecondaryChoosePolicy secondaryChoosePolicy = SecondaryChoosePolicy.ORDER; + private FailoverPolicy failoverPolicy = FailoverPolicy.ORDER; private long failoverDelayNs; private long switchBackDelayNs; private long checkIntervalMs = 30_000; @@ -283,14 +282,8 @@ public AutoClusterFailoverBuilder secondary(@NonNull List secondary) { } @Override - public AutoClusterFailoverBuilder secondaryChoosePolicy(@NonNull SecondaryChoosePolicy policy) { - this.secondaryChoosePolicy = policy; - return this; - } - - @Override - public AutoClusterFailoverBuilder primaryAuthentication(Authentication authentication) { - this.primaryAuthentication = authentication; + public AutoClusterFailoverBuilder failoverPolicy(@NonNull FailoverPolicy policy) { + this.failoverPolicy = policy; return this; } @@ -300,36 +293,18 @@ public AutoClusterFailoverBuilder secondaryAuthentication(Map tlsTrustCertsFilePath) { this.secondaryTlsTrustCertsFilePaths = tlsTrustCertsFilePath; return this; } - @Override - public AutoClusterFailoverBuilder primaryTlsTrustStorePath(String tlsTrustStorePath) { - this.primaryTlsTrustStorePath = tlsTrustStorePath; - return this; - } - @Override public AutoClusterFailoverBuilder secondaryTlsTrustStorePath(Map tlsTrustStorePath) { this.secondaryTlsTrustStorePaths = tlsTrustStorePath; return this; } - @Override - public AutoClusterFailoverBuilder primaryTlsTrustStorePassword(String tlsTrustStorePassword) { - this.primaryTlsTrustStorePassword = tlsTrustStorePassword; - return this; - } - @Override public AutoClusterFailoverBuilder secondaryTlsTrustStorePassword(Map tlsTrustStorePassword) { this.secondaryTlsTrustStorePasswords = tlsTrustStorePassword; @@ -359,9 +334,9 @@ public ServiceUrlProvider build() { Objects.requireNonNull(primary, "primary service url shouldn't be null"); checkArgument(secondary != null && secondary.size() > 0, "secondary cluster service url shouldn't be null and should set at least one"); - checkArgument(failoverDelayNs >= 0, "failoverDelay should >= 0"); - checkArgument(switchBackDelayNs >= 0, "switchBackDelay should >= 0"); - checkArgument(checkIntervalMs >= 0, "checkInterval should >= 0"); + checkArgument(failoverDelayNs > 0, "failoverDelay should > 0"); + checkArgument(switchBackDelayNs > 0, "switchBackDelay should > 0"); + checkArgument(checkIntervalMs > 0, "checkInterval should > 0"); int secondarySize = secondary.size(); checkArgument(secondaryAuthentications == null @@ -380,7 +355,7 @@ public ServiceUrlProvider build() { return new AutoClusterFailover(this); } - public static void checkArgument(boolean expression, @Nullable Object errorMessage) { + public static void checkArgument(boolean expression, @NonNull Object errorMessage) { if (!expression) { throw new IllegalArgumentException(String.valueOf(errorMessage)); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java index 5bcccaefd12ba..323e04f7c79c4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java @@ -24,10 +24,7 @@ import com.google.common.base.Strings; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; -import java.io.InputStream; -import java.net.URL; -import java.net.URLConnection; -import java.nio.charset.StandardCharsets; +import java.util.Map; import java.util.Objects; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -35,7 +32,14 @@ import lombok.Data; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.io.IOUtils; +import org.apache.http.HttpEntity; +import org.apache.http.HttpHeaders; +import org.apache.http.client.ResponseHandler; +import org.apache.http.client.methods.HttpUriRequest; +import org.apache.http.client.methods.RequestBuilder; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ControlledClusterFailoverBuilder; @@ -46,25 +50,50 @@ @Slf4j public class ControlledClusterFailover implements ServiceUrlProvider { - private PulsarClient pulsarClient; + private PulsarClientImpl pulsarClient; private volatile String currentPulsarServiceUrl; private volatile ControlledConfiguration currentControlledConfiguration; - private final URL pulsarUrlProvider; private final ScheduledExecutorService executor; - private long interval; + private final long interval; private ObjectMapper objectMapper = null; + private final CloseableHttpClient httpClient; + private final HttpUriRequest request; + private final ResponseHandler responseHandler; - private ControlledClusterFailover(String defaultServiceUrl, String urlProvider, long interval) throws IOException { - this.currentPulsarServiceUrl = defaultServiceUrl; - this.pulsarUrlProvider = new URL(urlProvider); - this.interval = interval; + private ControlledClusterFailover(ControlledClusterFailoverBuilderImpl builder) throws IOException { + this.currentPulsarServiceUrl = builder.defaultServiceUrl; + this.interval = builder.interval; this.executor = Executors.newSingleThreadScheduledExecutor( new DefaultThreadFactory("pulsar-service-provider")); + this.httpClient = HttpClients.custom().build(); + + RequestBuilder requestBuilder = RequestBuilder.get() + .setUri(builder.urlProvider) + .setHeader(HttpHeaders.CONTENT_TYPE, "application/json"); + + if (builder.header != null && !builder.header.isEmpty()) { + builder.header.forEach(requestBuilder::setHeader); + } + this.request = requestBuilder.build(); + responseHandler = httpResponse -> { + int status = httpResponse.getStatusLine().getStatusCode(); + if (status >= 200 && status < 300) { + HttpEntity entity = httpResponse.getEntity(); + return entity != null ? EntityUtils.toString(entity) : null; + } else { + log.warn("Unexpected response status: {}", status); + return null; + } + }; + } + + public HttpUriRequest getRequest() { + return this.request; } @Override public void initialize(PulsarClient client) { - this.pulsarClient = client; + this.pulsarClient = (PulsarClientImpl) client; // start to check service url every 30 seconds this.executor.scheduleAtFixedRate(catchingAndLoggingThrowables(() -> { @@ -110,26 +139,14 @@ public String getCurrentPulsarServiceUrl() { return this.currentPulsarServiceUrl; } - public URL getPulsarUrlProvider() { - return this.pulsarUrlProvider; - } - protected ControlledConfiguration fetchControlledConfiguration() throws IOException { // call the service to get service URL - InputStream inputStream = null; try { - URLConnection conn = pulsarUrlProvider.openConnection(); - inputStream = conn.getInputStream(); - String jsonStr = new String(IOUtils.toByteArray(inputStream), StandardCharsets.UTF_8); - ObjectMapper objectMapper = getObjectMapper(); - return objectMapper.readValue(jsonStr, ControlledConfiguration.class); + String jsonStr = httpClient.execute(request, responseHandler); + return getObjectMapper().readValue(jsonStr, ControlledConfiguration.class); } catch (IOException e) { log.warn("Failed to fetch controlled configuration. ", e); return null; - } finally { - if (inputStream != null) { - inputStream.close(); - } } } @@ -157,27 +174,6 @@ public String toJson() { return null; } } - - @Override - public boolean equals(Object obj) { - if (obj instanceof ControlledConfiguration) { - ControlledConfiguration other = (ControlledConfiguration) obj; - return Objects.equals(serviceUrl, other.serviceUrl) - && Objects.equals(tlsTrustCertsFilePath, other.tlsTrustCertsFilePath) - && Objects.equals(authPluginClassName, other.authPluginClassName) - && Objects.equals(authParamsString, other.authParamsString); - } - - return false; - } - - @Override - public int hashCode() { - return Objects.hash(serviceUrl, - tlsTrustCertsFilePath, - authPluginClassName, - authParamsString); - } } @Override @@ -188,11 +184,19 @@ public String getServiceUrl() { @Override public void close() { this.executor.shutdown(); + if (httpClient != null) { + try { + httpClient.close(); + } catch (IOException e) { + log.error("Failed to close http client."); + } + } } public static class ControlledClusterFailoverBuilderImpl implements ControlledClusterFailoverBuilder { private String defaultServiceUrl; private String urlProvider; + private Map header = null; private long interval = 30_000; @Override @@ -207,6 +211,12 @@ public ControlledClusterFailoverBuilder urlProvider(@NonNull String urlProvider) return this; } + @Override + public ControlledClusterFailoverBuilder urlProviderHeader(Map header) { + this.header = header; + return this; + } + @Override public ControlledClusterFailoverBuilder checkInterval(long interval, @NonNull TimeUnit timeUnit) { this.interval = timeUnit.toMillis(interval); @@ -217,9 +227,9 @@ public ControlledClusterFailoverBuilder checkInterval(long interval, @NonNull Ti public ServiceUrlProvider build() throws IOException { Objects.requireNonNull(defaultServiceUrl, "default service url shouldn't be null"); Objects.requireNonNull(urlProvider, "urlProvider shouldn't be null"); - checkArgument(interval >= 0, "checkInterval should >= 0"); + checkArgument(interval > 0, "checkInterval should > 0"); - return new ControlledClusterFailover(defaultServiceUrl, urlProvider, interval); + return new ControlledClusterFailover(this); } public static void checkArgument(boolean expression, @Nullable Object errorMessage) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index e9fcaee241d7e..99278dd1614a4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -865,7 +865,6 @@ public synchronized void updateServiceUrl(String serviceUrl) throws PulsarClient cnxPool.closeAllConnections(); } - @Override public void updateAuthentication(Authentication authentication) throws IOException { log.info("Updating authentication to {}", authentication); if (conf.getAuthentication() != null) { @@ -875,13 +874,11 @@ public void updateAuthentication(Authentication authentication) throws IOExcepti conf.getAuthentication().start(); } - @Override public void updateTlsTrustCertsFilePath(String tlsTrustCertsFilePath) { log.info("Updating tlsTrustCertsFilePath to {}", tlsTrustCertsFilePath); conf.setTlsTrustCertsFilePath(tlsTrustCertsFilePath); } - @Override public void updateTlsTrustStorePathAndPassword(String tlsTrustStorePath, String tlsTrustStorePassword) { log.info("Updating tlsTrustStorePath to {}, tlsTrustStorePassword to *****", tlsTrustStorePath); conf.setTlsTrustStorePath(tlsTrustStorePath); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java index af24a4ff206c4..1233555e8d3a7 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java @@ -28,12 +28,14 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.Test; import org.mockito.Mockito; import org.powermock.api.mockito.PowerMockito; +import org.testng.Assert; +import org.testng.annotations.Test; +@Test(groups = "broker-impl") public class AutoClusterFailoverTest { @Test public void testBuildAutoClusterFailoverInstance() throws PulsarClientException { @@ -86,15 +88,11 @@ public void testBuildAutoClusterFailoverInstance() throws PulsarClientException .failoverDelay(failoverDelay, TimeUnit.SECONDS) .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) .checkInterval(checkInterval, TimeUnit.MILLISECONDS) - .primaryTlsTrustCertsFilePath(primaryTlsTrustCertsFilePath) .secondaryTlsTrustCertsFilePath(secondaryTlsTrustCertsFilePaths) - .primaryAuthentication(primaryAuthentication) .secondaryAuthentication(secondaryAuthentications) .build(); AutoClusterFailover autoClusterFailover1 = (AutoClusterFailover) provider1; - Assert.assertEquals(primaryTlsTrustCertsFilePath, autoClusterFailover1.getPrimaryTlsTrustCertsFilePath()); - Assert.assertEquals(primaryAuthentication, autoClusterFailover1.getPrimaryAuthentication()); Assert.assertEquals(secondaryTlsTrustCertsFilePath, autoClusterFailover1.getSecondaryTlsTrustCertsFilePaths().get(secondary)); Assert.assertEquals(secondaryAuthentication, autoClusterFailover1.getSecondaryAuthentications().get(secondary)); @@ -104,9 +102,12 @@ public void testBuildAutoClusterFailoverInstance() throws PulsarClientException public void testAutoClusterFailoverSwitchWithoutAuthentication() { String primary = "pulsar://localhost:6650"; String secondary = "pulsar://localhost:6651"; - long failoverDelay = 0; - long switchBackDelay = 0; + long failoverDelay = 1; + long switchBackDelay = 1; long checkInterval = 1_000; + + ClientConfigurationData configurationData = new ClientConfigurationData(); + ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) @@ -116,9 +117,10 @@ public void testAutoClusterFailoverSwitchWithoutAuthentication() { .build(); AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); - PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); + PulsarClientImpl pulsarClient = PowerMockito.mock(PulsarClientImpl.class); Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); + Mockito.doReturn(configurationData).when(pulsarClient).getConfiguration(); autoClusterFailover.initialize(pulsarClient); @@ -135,8 +137,8 @@ public void testAutoClusterFailoverSwitchWithoutAuthentication() { public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException { String primary = "pulsar+ssl://localhost:6651"; String secondary = "pulsar+ssl://localhost:6661"; - long failoverDelay = 0; - long switchBackDelay = 0; + long failoverDelay = 1; + long switchBackDelay = 1; long checkInterval = 1_000; String primaryTlsTrustCertsFilePath = "primary/path"; String secondaryTlsTrustCertsFilePath = "primary/path"; @@ -156,22 +158,25 @@ public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException Map secondaryAuthentications = new HashMap<>(); secondaryAuthentications.put(secondary, secondaryAuthentication); + ClientConfigurationData configurationData = new ClientConfigurationData(); + configurationData.setTlsTrustCertsFilePath(primaryTlsTrustCertsFilePath); + configurationData.setAuthentication(primaryAuthentication); + ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) .checkInterval(checkInterval, TimeUnit.MILLISECONDS) .failoverDelay(failoverDelay, TimeUnit.SECONDS) .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) - .primaryTlsTrustCertsFilePath(primaryTlsTrustCertsFilePath) .secondaryTlsTrustCertsFilePath(secondaryTlsTrustCertsFilePaths) - .primaryAuthentication(primaryAuthentication) .secondaryAuthentication(secondaryAuthentications) .build(); AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); - PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); + PulsarClientImpl pulsarClient = PowerMockito.mock(PulsarClientImpl.class); Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); + Mockito.doReturn(configurationData).when(pulsarClient).getConfiguration(); autoClusterFailover.initialize(pulsarClient); @@ -193,8 +198,8 @@ public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException public void testAutoClusterFailoverSwitchTlsTrustStore() throws IOException { String primary = "pulsar+ssl://localhost:6651"; String secondary = "pulsar+ssl://localhost:6661"; - long failoverDelay = 0; - long switchBackDelay = 0; + long failoverDelay = 1; + long switchBackDelay = 1; long checkInterval = 1_000; String primaryTlsTrustStorePath = "primary/path"; String secondaryTlsTrustStorePath = "secondary/path"; @@ -206,22 +211,25 @@ public void testAutoClusterFailoverSwitchTlsTrustStore() throws IOException { Map secondaryTlsTrustStorePasswords = new HashMap<>(); secondaryTlsTrustStorePasswords.put(secondary, secondaryTlsTrustStorePassword); + ClientConfigurationData configurationData = new ClientConfigurationData(); + configurationData.setTlsTrustStorePath(primaryTlsTrustStorePath); + configurationData.setTlsTrustStorePassword(primaryTlsTrustStorePassword); + ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) .failoverDelay(failoverDelay, TimeUnit.SECONDS) .switchBackDelay(switchBackDelay, TimeUnit.SECONDS) .checkInterval(checkInterval, TimeUnit.MILLISECONDS) - .primaryTlsTrustStorePath(primaryTlsTrustStorePath) - .primaryTlsTrustStorePassword(primaryTlsTrustStorePassword) .secondaryTlsTrustStorePath(secondaryTlsTrustStorePaths) .secondaryTlsTrustStorePassword(secondaryTlsTrustStorePasswords) .build(); AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); - PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); + PulsarClientImpl pulsarClient = PowerMockito.mock(PulsarClientImpl.class); Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); + Mockito.doReturn(configurationData).when(pulsarClient).getConfiguration(); autoClusterFailover.initialize(pulsarClient); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java index 27e983ce1ea61..b9bda83d758d8 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java @@ -19,34 +19,47 @@ package org.apache.pulsar.client.impl; import java.io.IOException; -import java.net.URL; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.TimeUnit; +import org.apache.http.client.methods.HttpUriRequest; import org.apache.pulsar.client.api.Authentication; -import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.Test; import org.mockito.Mockito; import org.powermock.api.mockito.PowerMockito; +import org.testng.Assert; +import org.testng.annotations.Test; +@Test(groups = "broker-impl") public class ControlledClusterFailoverTest { @Test public void testBuildControlledClusterFailoverInstance() throws IOException { String defaultServiceUrl = "pulsar://localhost:6650"; String urlProvider = "http://localhost:8080"; + String keyA = "key-a"; + String valueA = "value-a"; + String keyB = "key-b"; + String valueB = "value-b"; + Map header = new HashMap<>(); + header.put(keyA, valueA); + header.put(keyB, valueB); ServiceUrlProvider provider = ControlledClusterFailover.builder() - .defaultServiceUrl(defaultServiceUrl) - .urlProvider(urlProvider) - .build(); + .defaultServiceUrl(defaultServiceUrl) + .urlProvider(urlProvider) + .urlProviderHeader(header) + .build(); ControlledClusterFailover controlledClusterFailover = (ControlledClusterFailover) provider; + HttpUriRequest request = controlledClusterFailover.getRequest(); Assert.assertTrue(provider instanceof ControlledClusterFailover); Assert.assertEquals(defaultServiceUrl, provider.getServiceUrl()); Assert.assertEquals(defaultServiceUrl, controlledClusterFailover.getCurrentPulsarServiceUrl()); - Assert.assertTrue(new URL(urlProvider).equals(controlledClusterFailover.getPulsarUrlProvider())); + Assert.assertEquals(urlProvider, request.getURI().toString()); + Assert.assertEquals(request.getFirstHeader(keyA).getValue(), valueA); + Assert.assertEquals(request.getFirstHeader(keyB).getValue(), valueB); } @Test @@ -67,13 +80,13 @@ public void testControlledClusterFailoverSwitch() throws IOException { controlledConfiguration.setAuthParamsString(authParamsString); ServiceUrlProvider provider = ControlledClusterFailover.builder() - .defaultServiceUrl(defaultServiceUrl) - .urlProvider(urlProvider) - .checkInterval(interval, TimeUnit.MILLISECONDS) - .build(); + .defaultServiceUrl(defaultServiceUrl) + .urlProvider(urlProvider) + .checkInterval(interval, TimeUnit.MILLISECONDS) + .build(); ControlledClusterFailover controlledClusterFailover = Mockito.spy((ControlledClusterFailover) provider); - PulsarClient pulsarClient = PowerMockito.mock(PulsarClientImpl.class); + PulsarClientImpl pulsarClient = PowerMockito.mock(PulsarClientImpl.class); controlledClusterFailover.initialize(pulsarClient); From f5f162b7102835a4101edd9e75caf3669004a883 Mon Sep 17 00:00:00 2001 From: chenhang Date: Wed, 26 Jan 2022 22:54:33 +0800 Subject: [PATCH 17/18] format code --- .../org/apache/pulsar/client/impl/AutoClusterFailoverTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java index 1233555e8d3a7..573ddd8086da7 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java @@ -25,7 +25,6 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; -import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; From 5d9b48b9a893d8fb8fd1daae0379edccdf888c43 Mon Sep 17 00:00:00 2001 From: chenhang Date: Sat, 29 Jan 2022 11:41:16 +0800 Subject: [PATCH 18/18] use AsyncHttpClient to perform http request, avoiding introduce other dependency --- pulsar-client/pom.xml | 5 - .../impl/ControlledClusterFailover.java | 93 ++++++++++++------- .../impl/ControlledClusterFailoverTest.java | 12 +-- 3 files changed, 65 insertions(+), 45 deletions(-) diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 625e61cc726d9..5338d23241c0a 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -114,11 +114,6 @@ gson - - org.apache.httpcomponents - httpclient - - diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java index 323e04f7c79c4..50060a25217de 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java @@ -21,74 +21,88 @@ import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpResponse; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.charset.StandardCharsets; import java.util.Map; import java.util.Objects; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import lombok.Data; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; -import org.apache.http.HttpEntity; -import org.apache.http.HttpHeaders; -import org.apache.http.client.ResponseHandler; -import org.apache.http.client.methods.HttpUriRequest; -import org.apache.http.client.methods.RequestBuilder; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.HttpClients; -import org.apache.http.util.EntityUtils; +import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ControlledClusterFailoverBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.AsyncHttpClientConfig; +import org.asynchttpclient.BoundRequestBuilder; +import org.asynchttpclient.DefaultAsyncHttpClient; +import org.asynchttpclient.DefaultAsyncHttpClientConfig; +import org.asynchttpclient.Request; +import org.asynchttpclient.Response; +import org.asynchttpclient.channel.DefaultKeepAliveStrategy; import org.checkerframework.checker.nullness.qual.Nullable; @Slf4j public class ControlledClusterFailover implements ServiceUrlProvider { + private static final int DEFAULT_CONNECT_TIMEOUT_IN_SECONDS = 10; + private static final int DEFAULT_READ_TIMEOUT_IN_SECONDS = 30; + private static final int DEFAULT_MAX_REDIRECTS = 20; + private PulsarClientImpl pulsarClient; private volatile String currentPulsarServiceUrl; private volatile ControlledConfiguration currentControlledConfiguration; private final ScheduledExecutorService executor; private final long interval; private ObjectMapper objectMapper = null; - private final CloseableHttpClient httpClient; - private final HttpUriRequest request; - private final ResponseHandler responseHandler; + private final AsyncHttpClient httpClient; + private final BoundRequestBuilder requestBuilder; private ControlledClusterFailover(ControlledClusterFailoverBuilderImpl builder) throws IOException { this.currentPulsarServiceUrl = builder.defaultServiceUrl; this.interval = builder.interval; this.executor = Executors.newSingleThreadScheduledExecutor( new DefaultThreadFactory("pulsar-service-provider")); - this.httpClient = HttpClients.custom().build(); - RequestBuilder requestBuilder = RequestBuilder.get() - .setUri(builder.urlProvider) - .setHeader(HttpHeaders.CONTENT_TYPE, "application/json"); + this.httpClient = buildHttpClient(); + this.requestBuilder = httpClient.prepareGet(builder.urlProvider) + .addHeader("Accept", "application/json"); if (builder.header != null && !builder.header.isEmpty()) { - builder.header.forEach(requestBuilder::setHeader); + builder.header.forEach(requestBuilder::addHeader); } - this.request = requestBuilder.build(); - responseHandler = httpResponse -> { - int status = httpResponse.getStatusLine().getStatusCode(); - if (status >= 200 && status < 300) { - HttpEntity entity = httpResponse.getEntity(); - return entity != null ? EntityUtils.toString(entity) : null; - } else { - log.warn("Unexpected response status: {}", status); - return null; - } - }; } - public HttpUriRequest getRequest() { - return this.request; + private AsyncHttpClient buildHttpClient() { + DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder(); + confBuilder.setFollowRedirect(true); + confBuilder.setMaxRedirects(DEFAULT_MAX_REDIRECTS); + confBuilder.setConnectTimeout(DEFAULT_CONNECT_TIMEOUT_IN_SECONDS * 1000); + confBuilder.setReadTimeout(DEFAULT_READ_TIMEOUT_IN_SECONDS * 1000); + confBuilder.setUserAgent(String.format("Pulsar-Java-v%s", PulsarVersion.getVersion())); + confBuilder.setKeepAliveStrategy(new DefaultKeepAliveStrategy() { + @Override + public boolean keepAlive(InetSocketAddress remoteAddress, Request ahcRequest, + HttpRequest request, HttpResponse response) { + // Close connection upon a server error or per HTTP spec + return (response.status().code() / 100 != 5) + && super.keepAlive(remoteAddress, ahcRequest, request, response); + } + }); + AsyncHttpClientConfig config = confBuilder.build(); + return new DefaultAsyncHttpClient(config); } @Override @@ -139,15 +153,26 @@ public String getCurrentPulsarServiceUrl() { return this.currentPulsarServiceUrl; } + @VisibleForTesting + protected BoundRequestBuilder getRequestBuilder() { + return this.requestBuilder; + } + protected ControlledConfiguration fetchControlledConfiguration() throws IOException { // call the service to get service URL try { - String jsonStr = httpClient.execute(request, responseHandler); - return getObjectMapper().readValue(jsonStr, ControlledConfiguration.class); - } catch (IOException e) { - log.warn("Failed to fetch controlled configuration. ", e); - return null; + Response response = requestBuilder.execute().get(); + int statusCode = response.getStatusCode(); + if (statusCode == 200) { + String content = response.getResponseBody(StandardCharsets.UTF_8); + return getObjectMapper().readValue(content, ControlledConfiguration.class); + } + log.warn("Failed to fetch controlled configuration, status code: {}", statusCode); + } catch (InterruptedException | ExecutionException e) { + log.error("Failed to fetch controlled configuration ", e); } + + return null; } private ObjectMapper getObjectMapper() { diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java index b9bda83d758d8..1fd4a017ec8d1 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java @@ -22,9 +22,9 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.http.client.methods.HttpUriRequest; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.asynchttpclient.Request; import org.awaitility.Awaitility; import org.mockito.Mockito; import org.powermock.api.mockito.PowerMockito; @@ -36,7 +36,7 @@ public class ControlledClusterFailoverTest { @Test public void testBuildControlledClusterFailoverInstance() throws IOException { String defaultServiceUrl = "pulsar://localhost:6650"; - String urlProvider = "http://localhost:8080"; + String urlProvider = "http://localhost:8080/test"; String keyA = "key-a"; String valueA = "value-a"; String keyB = "key-b"; @@ -52,14 +52,14 @@ public void testBuildControlledClusterFailoverInstance() throws IOException { .build(); ControlledClusterFailover controlledClusterFailover = (ControlledClusterFailover) provider; - HttpUriRequest request = controlledClusterFailover.getRequest(); + Request request = controlledClusterFailover.getRequestBuilder().build(); Assert.assertTrue(provider instanceof ControlledClusterFailover); Assert.assertEquals(defaultServiceUrl, provider.getServiceUrl()); Assert.assertEquals(defaultServiceUrl, controlledClusterFailover.getCurrentPulsarServiceUrl()); - Assert.assertEquals(urlProvider, request.getURI().toString()); - Assert.assertEquals(request.getFirstHeader(keyA).getValue(), valueA); - Assert.assertEquals(request.getFirstHeader(keyB).getValue(), valueB); + Assert.assertEquals(urlProvider, request.getUri().toUrl()); + Assert.assertEquals(request.getHeaders().get(keyA), valueA); + Assert.assertEquals(request.getHeaders().get(keyB), valueB); } @Test