${project.groupId}
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java
index 28c880f85e0cd..2b60d486fdbd1 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java
@@ -52,7 +52,6 @@
import lombok.Getter;
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.commons.lang3.tuple.Pair;
-import org.apache.http.conn.ssl.DefaultHostnameVerifier;
import org.apache.pulsar.PulsarVersion;
import org.apache.pulsar.client.api.Authentication;
import org.apache.pulsar.client.api.AuthenticationDataProvider;
@@ -60,6 +59,7 @@
import org.apache.pulsar.client.api.PulsarClientException.TimeoutException;
import org.apache.pulsar.client.impl.BinaryProtoLookupService.LookupDataResult;
import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
+import org.apache.pulsar.client.impl.tls.TlsHostnameVerifier;
import org.apache.pulsar.common.api.AuthData;
import org.apache.pulsar.common.api.proto.PulsarApi;
import org.apache.pulsar.common.protocol.Commands;
@@ -142,7 +142,7 @@ public class ClientCnx extends PulsarHandler {
protected String remoteHostName = null;
private boolean isTlsHostnameVerificationEnable;
- private static final DefaultHostnameVerifier HOSTNAME_VERIFIER = new DefaultHostnameVerifier();
+ private static final TlsHostnameVerifier HOSTNAME_VERIFIER = new TlsHostnameVerifier();
private ScheduledFuture> timeoutTask;
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/DomainType.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/DomainType.java
new file mode 100644
index 0000000000000..5ed4f67cb9193
--- /dev/null
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/DomainType.java
@@ -0,0 +1,31 @@
+/**
+ * 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.tls;
+
+/**
+ * Domain types differentiated by Mozilla Public Suffix List.
+ *
+ * @since 4.5
+ */
+public enum DomainType {
+
+ UNKNOWN, ICANN, PRIVATE
+
+}
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/InetAddressUtils.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/InetAddressUtils.java
new file mode 100644
index 0000000000000..1a9bf61f7c13d
--- /dev/null
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/InetAddressUtils.java
@@ -0,0 +1,117 @@
+/**
+ * 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.
+ */
+
+/**
+ * From Apache HTTP client
+ */
+
+package org.apache.pulsar.client.impl.tls;
+
+import java.util.regex.Pattern;
+
+/**
+ * A collection of utilities relating to InetAddresses.
+ *
+ * @since 4.0
+ */
+public class InetAddressUtils {
+
+ private InetAddressUtils() {
+ }
+
+ private static final String IPV4_BASIC_PATTERN_STRING =
+ "(([1-9]|[1-9][0-9]|1[0-9]{2}|2[0-4][0-9]|25[0-5])\\.){1}" + // initial first field, 1-255
+ "(([0-9]|[1-9][0-9]|1[0-9]{2}|2[0-4][0-9]|25[0-5])\\.){2}" + // following 2 fields, 0-255 followed by .
+ "([0-9]|[1-9][0-9]|1[0-9]{2}|2[0-4][0-9]|25[0-5])"; // final field, 0-255
+
+ private static final Pattern IPV4_PATTERN =
+ Pattern.compile("^" + IPV4_BASIC_PATTERN_STRING + "$");
+
+ private static final Pattern IPV4_MAPPED_IPV6_PATTERN = // TODO does not allow for redundant leading zeros
+ Pattern.compile("^::[fF]{4}:" + IPV4_BASIC_PATTERN_STRING + "$");
+
+ private static final Pattern IPV6_STD_PATTERN =
+ Pattern.compile(
+ "^[0-9a-fA-F]{1,4}(:[0-9a-fA-F]{1,4}){7}$");
+
+ private static final Pattern IPV6_HEX_COMPRESSED_PATTERN =
+ Pattern.compile(
+ "^(([0-9A-Fa-f]{1,4}(:[0-9A-Fa-f]{1,4}){0,5})?)" + // 0-6 hex fields
+ "::" +
+ "(([0-9A-Fa-f]{1,4}(:[0-9A-Fa-f]{1,4}){0,5})?)$"); // 0-6 hex fields
+
+ /*
+ * The above pattern is not totally rigorous as it allows for more than 7 hex fields in total
+ */
+ private static final char COLON_CHAR = ':';
+
+ // Must not have more than 7 colons (i.e. 8 fields)
+ private static final int MAX_COLON_COUNT = 7;
+
+ /**
+ * Checks whether the parameter is a valid IPv4 address
+ *
+ * @param input the address string to check for validity
+ * @return true if the input parameter is a valid IPv4 address
+ */
+ public static boolean isIPv4Address(final String input) {
+ return IPV4_PATTERN.matcher(input).matches();
+ }
+
+ public static boolean isIPv4MappedIPv64Address(final String input) {
+ return IPV4_MAPPED_IPV6_PATTERN.matcher(input).matches();
+ }
+
+ /**
+ * Checks whether the parameter is a valid standard (non-compressed) IPv6 address
+ *
+ * @param input the address string to check for validity
+ * @return true if the input parameter is a valid standard (non-compressed) IPv6 address
+ */
+ public static boolean isIPv6StdAddress(final String input) {
+ return IPV6_STD_PATTERN.matcher(input).matches();
+ }
+
+ /**
+ * Checks whether the parameter is a valid compressed IPv6 address
+ *
+ * @param input the address string to check for validity
+ * @return true if the input parameter is a valid compressed IPv6 address
+ */
+ public static boolean isIPv6HexCompressedAddress(final String input) {
+ int colonCount = 0;
+ for(int i = 0; i < input.length(); i++) {
+ if (input.charAt(i) == COLON_CHAR) {
+ colonCount++;
+ }
+ }
+ return colonCount <= MAX_COLON_COUNT && IPV6_HEX_COMPRESSED_PATTERN.matcher(input).matches();
+ }
+
+ /**
+ * Checks whether the parameter is a valid IPv6 address (including compressed).
+ *
+ * @param input the address string to check for validity
+ * @return true if the input parameter is a valid standard or compressed IPv6 address
+ */
+ public static boolean isIPv6Address(final String input) {
+ return isIPv6StdAddress(input) || isIPv6HexCompressedAddress(input);
+ }
+
+}
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/NoopHostnameVerifier.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/NoopHostnameVerifier.java
new file mode 100644
index 0000000000000..7962e3e2cb57c
--- /dev/null
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/NoopHostnameVerifier.java
@@ -0,0 +1,43 @@
+/**
+ * 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.
+ */
+
+/**
+ * From Apache HTTP client
+ */
+
+package org.apache.pulsar.client.impl.tls;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.SSLSession;
+
+public class NoopHostnameVerifier implements HostnameVerifier {
+
+ public static final NoopHostnameVerifier INSTANCE = new NoopHostnameVerifier();
+
+ @Override
+ public boolean verify(final String s, final SSLSession sslSession) {
+ return true;
+ }
+
+ @Override
+ public final String toString() {
+ return "NO_OP";
+ }
+
+}
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/PublicSuffixList.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/PublicSuffixList.java
new file mode 100644
index 0000000000000..eb1671ace54bc
--- /dev/null
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/PublicSuffixList.java
@@ -0,0 +1,59 @@
+/**
+ * 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.
+ */
+
+/**
+ * From Apache HTTP client
+ */
+
+package org.apache.pulsar.client.impl.tls;
+
+import java.util.Collections;
+import java.util.List;
+
+import lombok.Data;
+
+/**
+ * Public suffix is a set of DNS names or wildcards concatenated with dots. It represents the part of a domain name
+ * which is not under the control of the individual registrant
+ *
+ * An up-to-date list of suffixes can be obtained from publicsuffix.org
+ *
+ * @since 4.4
+ */
+@Data
+public final class PublicSuffixList {
+
+ private final DomainType type;
+ private final List rules;
+ private final List exceptions;
+
+ /**
+ * @since 4.5
+ */
+ public PublicSuffixList(final DomainType type, final List rules, final List exceptions) {
+ this.type = type;
+ this.rules = Collections.unmodifiableList(rules);
+ this.exceptions = Collections
+ .unmodifiableList(exceptions != null ? exceptions : Collections. emptyList());
+ }
+
+ public PublicSuffixList(final List rules, final List exceptions) {
+ this(DomainType.UNKNOWN, rules, exceptions);
+ }
+}
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/PublicSuffixMatcher.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/PublicSuffixMatcher.java
new file mode 100644
index 0000000000000..05d2ecaec8715
--- /dev/null
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/PublicSuffixMatcher.java
@@ -0,0 +1,192 @@
+/**
+ * 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.
+ */
+
+/**
+ * From Apache HTTP client
+ */
+
+package org.apache.pulsar.client.impl.tls;
+
+import java.net.IDN;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Utility class that can test if DNS names match the content of the Public Suffix List.
+ *
+ * An up-to-date list of suffixes can be obtained from
+ * publicsuffix.org
+ *
+ * @see org.apache.pulsar.client.impl.tls.PublicSuffixList
+ *
+ * @since 4.4
+ */
+public final class PublicSuffixMatcher {
+
+ private final Map rules;
+ private final Map exceptions;
+
+ public PublicSuffixMatcher(final Collection rules, final Collection exceptions) {
+ this(DomainType.UNKNOWN, rules, exceptions);
+ }
+
+ /**
+ * @since 4.5
+ */
+ public PublicSuffixMatcher(
+ final DomainType domainType, final Collection rules, final Collection exceptions) {
+ this.rules = new ConcurrentHashMap(rules.size());
+ for (final String rule: rules) {
+ this.rules.put(rule, domainType);
+ }
+ this.exceptions = new ConcurrentHashMap();
+ if (exceptions != null) {
+ for (final String exception: exceptions) {
+ this.exceptions.put(exception, domainType);
+ }
+ }
+ }
+
+ /**
+ * @since 4.5
+ */
+ public PublicSuffixMatcher(final Collection lists) {
+ this.rules = new ConcurrentHashMap();
+ this.exceptions = new ConcurrentHashMap();
+ for (final PublicSuffixList list: lists) {
+ final DomainType domainType = list.getType();
+ final List rules = list.getRules();
+ for (final String rule: rules) {
+ this.rules.put(rule, domainType);
+ }
+ final List exceptions = list.getExceptions();
+ if (exceptions != null) {
+ for (final String exception: exceptions) {
+ this.exceptions.put(exception, domainType);
+ }
+ }
+ }
+ }
+
+ private static boolean hasEntry(final Map map, final String rule, final DomainType expectedType) {
+ if (map == null) {
+ return false;
+ }
+ final DomainType domainType = map.get(rule);
+ if (domainType == null) {
+ return false;
+ } else {
+ return expectedType == null || domainType.equals(expectedType);
+ }
+ }
+
+ private boolean hasRule(final String rule, final DomainType expectedType) {
+ return hasEntry(this.rules, rule, expectedType);
+ }
+
+ private boolean hasException(final String exception, final DomainType expectedType) {
+ return hasEntry(this.exceptions, exception, expectedType);
+ }
+
+ /**
+ * Returns registrable part of the domain for the given domain name or {@code null}
+ * if given domain represents a public suffix.
+ *
+ * @param domain
+ * @return domain root
+ */
+ public String getDomainRoot(final String domain) {
+ return getDomainRoot(domain, null);
+ }
+
+ /**
+ * Returns registrable part of the domain for the given domain name or {@code null}
+ * if given domain represents a public suffix.
+ *
+ * @param domain
+ * @param expectedType expected domain type or {@code null} if any.
+ * @return domain root
+ *
+ * @since 4.5
+ */
+ public String getDomainRoot(final String domain, final DomainType expectedType) {
+ if (domain == null) {
+ return null;
+ }
+ if (domain.startsWith(".")) {
+ return null;
+ }
+ String domainName = null;
+ String segment = domain.toLowerCase(Locale.ROOT);
+ while (segment != null) {
+
+ // An exception rule takes priority over any other matching rule.
+ if (hasException(IDN.toUnicode(segment), expectedType)) {
+ return segment;
+ }
+
+ if (hasRule(IDN.toUnicode(segment), expectedType)) {
+ break;
+ }
+
+ final int nextdot = segment.indexOf('.');
+ final String nextSegment = nextdot != -1 ? segment.substring(nextdot + 1) : null;
+
+ if (nextSegment != null) {
+ if (hasRule("*." + IDN.toUnicode(nextSegment), expectedType)) {
+ break;
+ }
+ }
+ if (nextdot != -1) {
+ domainName = segment;
+ }
+ segment = nextSegment;
+ }
+ return domainName;
+ }
+
+ /**
+ * Tests whether the given domain matches any of entry from the public suffix list.
+ */
+ public boolean matches(final String domain) {
+ return matches(domain, null);
+ }
+
+ /**
+ * Tests whether the given domain matches any of entry from the public suffix list.
+ *
+ * @param domain
+ * @param expectedType expected domain type or {@code null} if any.
+ * @return {@code true} if the given domain matches any of the public suffixes.
+ *
+ * @since 4.5
+ */
+ public boolean matches(final String domain, final DomainType expectedType) {
+ if (domain == null) {
+ return false;
+ }
+ final String domainRoot = getDomainRoot(
+ domain.startsWith(".") ? domain.substring(1) : domain, expectedType);
+ return domainRoot == null;
+ }
+
+}
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/SubjectName.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/SubjectName.java
new file mode 100644
index 0000000000000..2885b884b9f7b
--- /dev/null
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/SubjectName.java
@@ -0,0 +1,44 @@
+/**
+ * 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.
+ */
+
+/**
+ * From Apache HTTP client
+ */
+
+package org.apache.pulsar.client.impl.tls;
+
+import lombok.Data;
+
+@Data
+final class SubjectName {
+
+ static final int DNS = 2;
+ static final int IP = 7;
+
+ private final String value;
+ private final int type;
+
+ static SubjectName IP(final String value) {
+ return new SubjectName(value, IP);
+ }
+
+ static SubjectName DNS(final String value) {
+ return new SubjectName(value, DNS);
+ }
+}
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/TlsHostnameVerifier.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/TlsHostnameVerifier.java
new file mode 100644
index 0000000000000..535765659602f
--- /dev/null
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/tls/TlsHostnameVerifier.java
@@ -0,0 +1,319 @@
+/**
+ * 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.
+ */
+
+/**
+ * From Apache HTTP client
+ */
+
+package org.apache.pulsar.client.impl.tls;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateParsingException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.NoSuchElementException;
+
+import javax.naming.InvalidNameException;
+import javax.naming.NamingException;
+import javax.naming.directory.Attribute;
+import javax.naming.directory.Attributes;
+import javax.naming.ldap.LdapName;
+import javax.naming.ldap.Rdn;
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.SSLPeerUnverifiedException;
+import javax.net.ssl.SSLSession;
+import javax.security.auth.x500.X500Principal;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class TlsHostnameVerifier implements HostnameVerifier {
+
+ enum HostNameType {
+
+ IPv4(7), IPv6(7), DNS(2);
+
+ final int subjectType;
+
+ HostNameType(final int subjectType) {
+ this.subjectType = subjectType;
+ }
+ }
+
+ private final PublicSuffixMatcher publicSuffixMatcher;
+
+ public TlsHostnameVerifier(final PublicSuffixMatcher publicSuffixMatcher) {
+ this.publicSuffixMatcher = publicSuffixMatcher;
+ }
+
+ public TlsHostnameVerifier() {
+ this(null);
+ }
+
+ @Override
+ public boolean verify(final String host, final SSLSession session) {
+ try {
+ final Certificate[] certs = session.getPeerCertificates();
+ final X509Certificate x509 = (X509Certificate) certs[0];
+ verify(host, x509);
+ return true;
+ } catch (final SSLException ex) {
+ if (log.isDebugEnabled()) {
+ log.debug(ex.getMessage(), ex);
+ }
+ return false;
+ }
+ }
+
+ public void verify(
+ final String host, final X509Certificate cert) throws SSLException {
+ final HostNameType hostType = determineHostFormat(host);
+ final List subjectAlts = getSubjectAltNames(cert);
+ if (subjectAlts != null && !subjectAlts.isEmpty()) {
+ switch (hostType) {
+ case IPv4:
+ matchIPAddress(host, subjectAlts);
+ break;
+ case IPv6:
+ matchIPv6Address(host, subjectAlts);
+ break;
+ default:
+ matchDNSName(host, subjectAlts, this.publicSuffixMatcher);
+ }
+ } else {
+ // CN matching has been deprecated by rfc2818 and can be used
+ // as fallback only when no subjectAlts are available
+ final X500Principal subjectPrincipal = cert.getSubjectX500Principal();
+ final String cn = extractCN(subjectPrincipal.getName(X500Principal.RFC2253));
+ if (cn == null) {
+ throw new SSLException("Certificate subject for <" + host + "> doesn't contain " +
+ "a common name and does not have alternative names");
+ }
+ matchCN(host, cn, this.publicSuffixMatcher);
+ }
+ }
+
+ static void matchIPAddress(final String host, final List subjectAlts) throws SSLException {
+ for (int i = 0; i < subjectAlts.size(); i++) {
+ final SubjectName subjectAlt = subjectAlts.get(i);
+ if (subjectAlt.getType() == SubjectName.IP) {
+ if (host.equals(subjectAlt.getValue())) {
+ return;
+ }
+ }
+ }
+ throw new SSLPeerUnverifiedException("Certificate for <" + host + "> doesn't match any " +
+ "of the subject alternative names: " + subjectAlts);
+ }
+
+ static void matchIPv6Address(final String host, final List subjectAlts) throws SSLException {
+ final String normalisedHost = normaliseAddress(host);
+ for (int i = 0; i < subjectAlts.size(); i++) {
+ final SubjectName subjectAlt = subjectAlts.get(i);
+ if (subjectAlt.getType() == SubjectName.IP) {
+ final String normalizedSubjectAlt = normaliseAddress(subjectAlt.getValue());
+ if (normalisedHost.equals(normalizedSubjectAlt)) {
+ return;
+ }
+ }
+ }
+ throw new SSLPeerUnverifiedException("Certificate for <" + host + "> doesn't match any " +
+ "of the subject alternative names: " + subjectAlts);
+ }
+
+ static void matchDNSName(final String host, final List subjectAlts,
+ final PublicSuffixMatcher publicSuffixMatcher) throws SSLException {
+ final String normalizedHost = host.toLowerCase(Locale.ROOT);
+ for (int i = 0; i < subjectAlts.size(); i++) {
+ final SubjectName subjectAlt = subjectAlts.get(i);
+ if (subjectAlt.getType() == SubjectName.DNS) {
+ final String normalizedSubjectAlt = subjectAlt.getValue().toLowerCase(Locale.ROOT);
+ if (matchIdentityStrict(normalizedHost, normalizedSubjectAlt, publicSuffixMatcher)) {
+ return;
+ }
+ }
+ }
+ throw new SSLPeerUnverifiedException("Certificate for <" + host + "> doesn't match any " +
+ "of the subject alternative names: " + subjectAlts);
+ }
+
+ static void matchCN(final String host, final String cn,
+ final PublicSuffixMatcher publicSuffixMatcher) throws SSLException {
+ final String normalizedHost = host.toLowerCase(Locale.ROOT);
+ final String normalizedCn = cn.toLowerCase(Locale.ROOT);
+ if (!matchIdentityStrict(normalizedHost, normalizedCn, publicSuffixMatcher)) {
+ throw new SSLPeerUnverifiedException("Certificate for <" + host + "> doesn't match " +
+ "common name of the certificate subject: " + cn);
+ }
+ }
+
+ static boolean matchDomainRoot(final String host, final String domainRoot) {
+ if (domainRoot == null) {
+ return false;
+ }
+ return host.endsWith(domainRoot) && (host.length() == domainRoot.length()
+ || host.charAt(host.length() - domainRoot.length() - 1) == '.');
+ }
+
+ private static boolean matchIdentity(final String host, final String identity,
+ final PublicSuffixMatcher publicSuffixMatcher,
+ final boolean strict) {
+ if (publicSuffixMatcher != null && host.contains(".")) {
+ if (!matchDomainRoot(host, publicSuffixMatcher.getDomainRoot(identity, DomainType.ICANN))) {
+ return false;
+ }
+ }
+
+ // RFC 2818, 3.1. Server Identity
+ // "...Names may contain the wildcard
+ // character * which is considered to match any single domain name
+ // component or component fragment..."
+ // Based on this statement presuming only singular wildcard is legal
+ final int asteriskIdx = identity.indexOf('*');
+ if (asteriskIdx != -1) {
+ final String prefix = identity.substring(0, asteriskIdx);
+ final String suffix = identity.substring(asteriskIdx + 1);
+ if (!prefix.isEmpty() && !host.startsWith(prefix)) {
+ return false;
+ }
+ if (!suffix.isEmpty() && !host.endsWith(suffix)) {
+ return false;
+ }
+ // Additional sanity checks on content selected by wildcard can be done here
+ if (strict) {
+ final String remainder = host.substring(
+ prefix.length(), host.length() - suffix.length());
+ if (remainder.contains(".")) {
+ return false;
+ }
+ }
+ return true;
+ }
+ return host.equalsIgnoreCase(identity);
+ }
+
+ static boolean matchIdentity(final String host, final String identity,
+ final PublicSuffixMatcher publicSuffixMatcher) {
+ return matchIdentity(host, identity, publicSuffixMatcher, false);
+ }
+
+ static boolean matchIdentity(final String host, final String identity) {
+ return matchIdentity(host, identity, null, false);
+ }
+
+ static boolean matchIdentityStrict(final String host, final String identity,
+ final PublicSuffixMatcher publicSuffixMatcher) {
+ return matchIdentity(host, identity, publicSuffixMatcher, true);
+ }
+
+ static boolean matchIdentityStrict(final String host, final String identity) {
+ return matchIdentity(host, identity, null, true);
+ }
+
+ static String extractCN(final String subjectPrincipal) throws SSLException {
+ if (subjectPrincipal == null) {
+ return null;
+ }
+ try {
+ final LdapName subjectDN = new LdapName(subjectPrincipal);
+ final List rdns = subjectDN.getRdns();
+ for (int i = rdns.size() - 1; i >= 0; i--) {
+ final Rdn rds = rdns.get(i);
+ final Attributes attributes = rds.toAttributes();
+ final Attribute cn = attributes.get("cn");
+ if (cn != null) {
+ try {
+ final Object value = cn.get();
+ if (value != null) {
+ return value.toString();
+ }
+ } catch (final NoSuchElementException ignore) {
+ // ignore exception
+ } catch (final NamingException ignore) {
+ // ignore exception
+ }
+ }
+ }
+ return null;
+ } catch (final InvalidNameException e) {
+ throw new SSLException(subjectPrincipal + " is not a valid X500 distinguished name");
+ }
+ }
+
+ static HostNameType determineHostFormat(final String host) {
+ if (InetAddressUtils.isIPv4Address(host)) {
+ return HostNameType.IPv4;
+ }
+ String s = host;
+ if (s.startsWith("[") && s.endsWith("]")) {
+ s = host.substring(1, host.length() - 1);
+ }
+ if (InetAddressUtils.isIPv6Address(s)) {
+ return HostNameType.IPv6;
+ }
+ return HostNameType.DNS;
+ }
+
+ static List getSubjectAltNames(final X509Certificate cert) {
+ try {
+ final Collection> entries = cert.getSubjectAlternativeNames();
+ if (entries == null) {
+ return Collections.emptyList();
+ }
+ final List result = new ArrayList();
+ for (final List> entry : entries) {
+ final Integer type = entry.size() >= 2 ? (Integer) entry.get(0) : null;
+ if (type != null) {
+ final Object o = entry.get(1);
+ if (o instanceof String) {
+ result.add(new SubjectName((String) o, type.intValue()));
+ } else if (o instanceof byte[]) {
+ // TODO ASN.1 DER encoded form
+ }
+ }
+ }
+ return result;
+ } catch (final CertificateParsingException ignore) {
+ return Collections.emptyList();
+ }
+ }
+
+ /*
+ * Normalize IPv6 or DNS name.
+ */
+ static String normaliseAddress(final String hostname) {
+ if (hostname == null) {
+ return hostname;
+ }
+ try {
+ final InetAddress inetAddress = InetAddress.getByName(hostname);
+ return inetAddress.getHostAddress();
+ } catch (final UnknownHostException unexpected) { // Should not happen, because we check for IPv6 address above
+ return hostname;
+ }
+ }
+
+}
diff --git a/pulsar-functions/localrun-shaded/pom.xml b/pulsar-functions/localrun-shaded/pom.xml
index ff1579047a169..680c6e40923a3 100644
--- a/pulsar-functions/localrun-shaded/pom.xml
+++ b/pulsar-functions/localrun-shaded/pom.xml
@@ -80,10 +80,6 @@
com.google
org.apache.pulsar.functions.runtime.shaded.com.google
-
- org.apache.http
- org.apache.pulsar.functions.runtime.shaded.org.apache.http
-
org.apache.jute
org.apache.pulsar.functions.runtime.shaded.org.apache.jute
@@ -254,10 +250,6 @@
jline
org.apache.pulsar.functions.runtime.shaded.jline
-
- commons-logging
- org.apache.pulsar.functions.runtime.shaded.commons-logging
-
org.bouncycastle
org.apache.pulsar.functions.runtime.shaded.org.bouncycastle
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
index e786ba8b69208..23d3ecb2aedca 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
@@ -36,27 +36,28 @@
import io.netty.handler.ssl.SslHandler;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.FutureListener;
-import java.util.function.Supplier;
-import lombok.Getter;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
import javax.net.ssl.SSLSession;
-import org.apache.http.conn.ssl.DefaultHostnameVerifier;
+import lombok.Getter;
+
import org.apache.pulsar.PulsarVersion;
import org.apache.pulsar.client.api.Authentication;
import org.apache.pulsar.client.api.AuthenticationDataProvider;
+import org.apache.pulsar.client.impl.tls.TlsHostnameVerifier;
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
import org.apache.pulsar.common.api.AuthData;
+import org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge;
+import org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.PulsarDecoder;
import org.apache.pulsar.common.stats.Rate;
-import org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge;
-import org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -342,7 +343,7 @@ private boolean verifyTlsHostName(String hostname, ChannelHandlerContext ctx) {
SSLSession sslSession = null;
if (sslHandler != null) {
sslSession = ((SslHandler) sslHandler).engine().getSession();
- return (new DefaultHostnameVerifier()).verify(hostname, sslSession);
+ return (new TlsHostnameVerifier()).verify(hostname, sslSession);
}
return false;
}
diff --git a/pulsar-sql/presto-pulsar/pom.xml b/pulsar-sql/presto-pulsar/pom.xml
index 73dd85686fc7b..dd92cc8e7a6f2 100644
--- a/pulsar-sql/presto-pulsar/pom.xml
+++ b/pulsar-sql/presto-pulsar/pom.xml
@@ -124,7 +124,6 @@
javax.annotation:*
org.glassfish.hk2*:*
- org.apache.httpcomponents:*
org.eclipse.jetty:*
@@ -158,10 +157,6 @@
org.eclipse.jetty
org.apache.pulsar.shade.org.eclipse.jetty
-
- org.apache.http
- org.apache.pulsar.shade.org.apache.http
-