Skip to content

Commit

Permalink
[fix][proxy] Fix Proxy 502 gateway error when it is configured with K…
Browse files Browse the repository at this point in the history
…eystore TLS and admin API is called (#21077)
  • Loading branch information
Apurva007 authored and Technoboy- committed Sep 11, 2023
1 parent 635b5e5 commit 242e29d
Show file tree
Hide file tree
Showing 2 changed files with 171 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,10 @@
import org.apache.pulsar.client.api.Authentication;
import org.apache.pulsar.client.api.AuthenticationDataProvider;
import org.apache.pulsar.client.api.AuthenticationFactory;
import org.apache.pulsar.client.api.KeyStoreParams;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.common.util.SecurityUtility;
import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext;
import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData;
import org.eclipse.jetty.client.HttpClient;
import org.eclipse.jetty.client.HttpRequest;
Expand Down Expand Up @@ -269,20 +271,35 @@ protected HttpClient newHttpClient() {

SSLContext sslCtx;
AuthenticationDataProvider authData = auth.getAuthData();
if (authData.hasDataForTls()) {
sslCtx = SecurityUtility.createSslContext(
if (config.isBrokerClientTlsEnabledWithKeyStore()) {
KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : null;
sslCtx = KeyStoreSSLContext.createClientSslContext(
config.getBrokerClientSslProvider(),
params != null ? params.getKeyStoreType() : null,
params != null ? params.getKeyStorePath() : null,
params != null ? params.getKeyStorePassword() : null,
config.isTlsAllowInsecureConnection(),
trustCertificates,
authData.getTlsCertificates(),
authData.getTlsPrivateKey(),
config.getBrokerClientSslProvider()
);
config.getBrokerClientTlsTrustStoreType(),
config.getBrokerClientTlsTrustStore(),
config.getBrokerClientTlsTrustStorePassword(),
config.getBrokerClientTlsCiphers(),
config.getBrokerClientTlsProtocols());
} else {
sslCtx = SecurityUtility.createSslContext(
config.isTlsAllowInsecureConnection(),
trustCertificates,
config.getBrokerClientSslProvider()
);
if (authData.hasDataForTls()) {
sslCtx = SecurityUtility.createSslContext(
config.isTlsAllowInsecureConnection(),
trustCertificates,
authData.getTlsCertificates(),
authData.getTlsPrivateKey(),
config.getBrokerClientSslProvider()
);
} else {
sslCtx = SecurityUtility.createSslContext(
config.isTlsAllowInsecureConnection(),
trustCertificates,
config.getBrokerClientSslProvider()
);
}
}

SslContextFactory contextFactory = new SslContextFactory.Client();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,142 @@
/*
* 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.proxy.server;

import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.broker.authentication.AuthenticationProviderTls;
import org.apache.pulsar.broker.authentication.AuthenticationService;
import org.apache.pulsar.broker.resources.PulsarResources;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls;
import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.metadata.impl.ZKMetadataStore;
import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport;
import org.apache.pulsar.policies.data.loadbalancer.LoadReport;
import org.eclipse.jetty.servlet.ServletHolder;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

import java.util.HashSet;
import java.util.Optional;
import java.util.Set;

import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.spy;

public class AdminProxyHandlerKeystoreTLSTest extends MockedPulsarServiceBaseTest {


private final ProxyConfiguration proxyConfig = new ProxyConfiguration();

private WebServer webServer;

private BrokerDiscoveryProvider discoveryProvider;

private PulsarResources resource;

@BeforeMethod
@Override
protected void setup() throws Exception {

conf.setAuthenticationEnabled(true);
conf.setAuthorizationEnabled(false);
conf.setWebServicePortTls(Optional.of(0));
conf.setBrokerServicePortTls(Optional.of(0));
conf.setTlsEnabledWithKeyStore(true);
conf.setTlsAllowInsecureConnection(false);
conf.setTlsKeyStoreType(KEYSTORE_TYPE);
conf.setTlsKeyStore(BROKER_KEYSTORE_FILE_PATH);
conf.setTlsKeyStorePassword(BROKER_KEYSTORE_PW);
conf.setTlsTrustStoreType(KEYSTORE_TYPE);
conf.setTlsTrustStore(CLIENT_TRUSTSTORE_FILE_PATH);
conf.setTlsTrustStorePassword(CLIENT_TRUSTSTORE_PW);

super.internalSetup();

proxyConfig.setWebServicePort(Optional.of(0));
proxyConfig.setBrokerProxyAllowedTargetPorts("*");
proxyConfig.setServicePortTls(Optional.of(0));
proxyConfig.setWebServicePortTls(Optional.of(0));
proxyConfig.setTlsEnabledWithBroker(true);
proxyConfig.setTlsEnabledWithKeyStore(true);

proxyConfig.setTlsKeyStoreType(KEYSTORE_TYPE);
proxyConfig.setTlsKeyStore(BROKER_KEYSTORE_FILE_PATH);
proxyConfig.setTlsKeyStorePassword(BROKER_KEYSTORE_PW);
proxyConfig.setTlsTrustStoreType(KEYSTORE_TYPE);
proxyConfig.setTlsTrustStore(CLIENT_TRUSTSTORE_FILE_PATH);
proxyConfig.setTlsTrustStorePassword(CLIENT_TRUSTSTORE_PW);

proxyConfig.setMetadataStoreUrl(DUMMY_VALUE);
proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE);
proxyConfig.setBrokerClientTlsEnabledWithKeyStore(true);
proxyConfig.setBrokerClientTlsKeyStoreType(KEYSTORE_TYPE);
proxyConfig.setBrokerClientTlsKeyStore(BROKER_KEYSTORE_FILE_PATH);
proxyConfig.setBrokerClientTlsKeyStorePassword(BROKER_KEYSTORE_PW);
proxyConfig.setBrokerClientTlsTrustStoreType(KEYSTORE_TYPE);
proxyConfig.setBrokerClientTlsTrustStore(BROKER_TRUSTSTORE_FILE_PATH);
proxyConfig.setBrokerClientTlsTrustStorePassword(BROKER_TRUSTSTORE_PW);
Set<String> providers = new HashSet<>();
providers.add(AuthenticationProviderTls.class.getName());
proxyConfig.setAuthenticationProviders(providers);
proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationKeyStoreTls.class.getName());
proxyConfig.setBrokerClientAuthenticationParameters(String.format("keyStoreType:%s,keyStorePath:%s,keyStorePassword:%s",
KEYSTORE_TYPE, BROKER_KEYSTORE_FILE_PATH, BROKER_KEYSTORE_PW));

resource = new PulsarResources(new ZKMetadataStore(mockZooKeeper),
new ZKMetadataStore(mockZooKeeperGlobal));
webServer = new WebServer(proxyConfig, new AuthenticationService(
PulsarConfigurationLoader.convertFrom(proxyConfig)));
discoveryProvider = spy(new BrokerDiscoveryProvider(proxyConfig, resource));
LoadManagerReport report = new LoadReport(brokerUrl.toString(), brokerUrlTls.toString(), null, null);
doReturn(report).when(discoveryProvider).nextBroker();
ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider));
webServer.addServlet("/admin", servletHolder);
webServer.addServlet("/lookup", servletHolder);
webServer.start();
}

@AfterMethod(alwaysRun = true)
@Override
protected void cleanup() throws Exception {
webServer.stop();
super.internalCleanup();
}

PulsarAdmin getAdminClient() throws Exception {
return PulsarAdmin.builder()
.serviceHttpUrl("https://localhost:" + webServer.getListenPortHTTPS().get())
.useKeyStoreTls(true)
.allowTlsInsecureConnection(false)
.tlsTrustStorePath(BROKER_TRUSTSTORE_FILE_PATH)
.tlsTrustStorePassword(BROKER_TRUSTSTORE_PW)
.authentication(AuthenticationKeyStoreTls.class.getName(),
String.format("keyStoreType:%s,keyStorePath:%s,keyStorePassword:%s",
KEYSTORE_TYPE, BROKER_KEYSTORE_FILE_PATH, BROKER_KEYSTORE_PW))
.build();
}

@Test
public void testAdmin() throws Exception {
getAdminClient().clusters().createCluster(configClusterName, ClusterData.builder().serviceUrl(brokerUrl.toString()).build());
}

}

0 comments on commit 242e29d

Please sign in to comment.