Skip to content

Commit

Permalink
Fix init WebSocketService with ClusterData (#11234)
Browse files Browse the repository at this point in the history
1. Fix init WebSocketService with ClusterData.
2. Refactor ProxyServiceStarter to be easier for test.
3. Add test case for ProxyServiceStarter.
  • Loading branch information
Technoboy- committed Jul 6, 2021
1 parent ec8afd6 commit 93e145b
Show file tree
Hide file tree
Showing 3 changed files with 470 additions and 43 deletions.
Expand Up @@ -21,14 +21,17 @@
import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.commons.lang3.StringUtils.isBlank;
import static org.apache.commons.lang3.StringUtils.isEmpty;
import static org.apache.commons.lang3.StringUtils.isNotBlank;
import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed;
import static org.slf4j.bridge.SLF4JBridgeHandler.install;
import static org.slf4j.bridge.SLF4JBridgeHandler.removeHandlersForRootLogger;

import com.google.common.annotations.VisibleForTesting;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.authentication.AuthenticationService;
import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServletWithClassLoader;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.websocket.WebSocketConsumerServlet;
import org.apache.pulsar.websocket.WebSocketPingPongServlet;
import org.apache.pulsar.websocket.WebSocketProducerServlet;
Expand Down Expand Up @@ -82,6 +85,8 @@ public class ProxyServiceStarter {
@Parameter(names = { "-h", "--help" }, description = "Show this help message")
private boolean help = false;

private ProxyConfiguration config;

public ProxyServiceStarter(String[] args) throws Exception {
try {

Expand All @@ -108,7 +113,7 @@ public ProxyServiceStarter(String[] args) throws Exception {
}

// load config file
final ProxyConfiguration config = PulsarConfigurationLoader.create(configFile, ProxyConfiguration.class);
config = PulsarConfigurationLoader.create(configFile, ProxyConfiguration.class);

if (!isBlank(zookeeperServers)) {
// Use zookeeperServers from command line
Expand Down Expand Up @@ -136,55 +141,58 @@ public ProxyServiceStarter(String[] args) throws Exception {
checkArgument(!isEmpty(config.getZookeeperServers()), "zookeeperServers must be provided");
}

AuthenticationService authenticationService = new AuthenticationService(
PulsarConfigurationLoader.convertFrom(config));
// create proxy service
ProxyService proxyService = new ProxyService(config, authenticationService);
// create a web-service
final WebServer server = new WebServer(config, authenticationService);

Runtime.getRuntime().addShutdownHook(new Thread(() -> {
try {
proxyService.close();
server.stop();
} catch (Exception e) {
log.warn("server couldn't stop gracefully {}", e.getMessage(), e);
}
}));
} catch (Exception e) {
log.error("Failed to start pulsar proxy service. error msg " + e.getMessage(), e);
throw new PulsarServerException(e);
}
}

proxyService.start();
public static void main(String[] args) throws Exception {
ProxyServiceStarter serviceStarter = new ProxyServiceStarter(args);
serviceStarter.start();
}

// Setup metrics
DefaultExports.initialize();
public void start() throws Exception {
AuthenticationService authenticationService = new AuthenticationService(
PulsarConfigurationLoader.convertFrom(config));
// create proxy service
ProxyService proxyService = new ProxyService(config, authenticationService);
// create a web-service
final WebServer server = new WebServer(config, authenticationService);

// Report direct memory from Netty counters
Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() {
@Override
public double get() {
return getJvmDirectMemoryUsed();
}
}).register(CollectorRegistry.defaultRegistry);
Runtime.getRuntime().addShutdownHook(new Thread(() -> {
try {
proxyService.close();
server.stop();
} catch (Exception e) {
log.warn("server couldn't stop gracefully {}", e.getMessage(), e);
}
}));

Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() {
@Override
public double get() {
return PlatformDependent.maxDirectMemory();
}
}).register(CollectorRegistry.defaultRegistry);
proxyService.start();

addWebServerHandlers(server, config, proxyService, proxyService.getDiscoveryProvider());
// Setup metrics
DefaultExports.initialize();

// start web-service
server.start();
// Report direct memory from Netty counters
Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() {
@Override
public double get() {
return getJvmDirectMemoryUsed();
}
}).register(CollectorRegistry.defaultRegistry);

} catch (Exception e) {
log.error("Failed to start pulsar proxy service. error msg " + e.getMessage(), e);
throw new PulsarServerException(e);
}
}
Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() {
@Override
public double get() {
return PlatformDependent.maxDirectMemory();
}
}).register(CollectorRegistry.defaultRegistry);

public static void main(String[] args) throws Exception {
new ProxyServiceStarter(args);
addWebServerHandlers(server, config, proxyService, proxyService.getDiscoveryProvider());

// start web-service
server.start();
}

public static void addWebServerHandlers(WebServer server,
Expand Down Expand Up @@ -225,7 +233,7 @@ public static void addWebServerHandlers(WebServer server,
if (config.isWebSocketServiceEnabled()) {
// add WebSocket servlet
// Use local broker address to avoid different IP address when using a VIP for service discovery
WebSocketService webSocketService = new WebSocketService(null, PulsarConfigurationLoader.convertFrom(config));
WebSocketService webSocketService = new WebSocketService(createClusterData(config), PulsarConfigurationLoader.convertFrom(config));
webSocketService.start();
final WebSocketServlet producerWebSocketServlet = new WebSocketProducerServlet(webSocketService);
server.addServlet(WebSocketProducerServlet.SERVLET_PATH,
Expand Down Expand Up @@ -253,6 +261,29 @@ public static void addWebServerHandlers(WebServer server,
}
}

private static ClusterData createClusterData(ProxyConfiguration config) {
if (isNotBlank(config.getBrokerServiceURL()) || isNotBlank(config.getBrokerServiceURLTLS())) {
return ClusterData.builder()
.serviceUrl(config.getBrokerWebServiceURL())
.serviceUrlTls(config.getBrokerWebServiceURLTLS())
.brokerServiceUrl(config.getBrokerServiceURL())
.brokerServiceUrlTls(config.getBrokerServiceURLTLS())
.build();
} else if (isNotBlank(config.getBrokerWebServiceURL()) || isNotBlank(config.getBrokerWebServiceURLTLS())) {
return ClusterData.builder()
.serviceUrl(config.getBrokerWebServiceURL())
.serviceUrlTls(config.getBrokerWebServiceURLTLS())
.build();
} else {
return null;
}
}

@VisibleForTesting
public ProxyConfiguration getConfig() {
return config;
}

private static final Logger log = LoggerFactory.getLogger(ProxyServiceStarter.class);

}
@@ -0,0 +1,160 @@
/**
* 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 lombok.Cleanup;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;

import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.common.util.ObjectMapperFactory;
import org.apache.pulsar.websocket.data.ProducerMessage;
import org.eclipse.jetty.client.HttpClient;
import org.eclipse.jetty.util.BufferUtil;
import org.eclipse.jetty.websocket.api.Session;
import org.eclipse.jetty.websocket.api.WebSocketAdapter;
import org.eclipse.jetty.websocket.api.WebSocketPingPongListener;
import org.eclipse.jetty.websocket.api.annotations.WebSocket;
import org.eclipse.jetty.websocket.client.WebSocketClient;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

import java.net.URI;
import java.nio.ByteBuffer;
import java.util.Base64;
import java.util.Optional;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.Future;

import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;

public class ProxyServiceStarterTest extends MockedPulsarServiceBaseTest {

@Override
@BeforeClass
protected void setup() throws Exception {
internalSetup();
String[] args = new String[]{"-c", "./src/test/resources/proxy.conf"};
ProxyServiceStarter serviceStarter = new ProxyServiceStarter(args);
serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl());
serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress());
serviceStarter.getConfig().setServicePort(Optional.of(11000));
serviceStarter.getConfig().setWebSocketServiceEnabled(true);
serviceStarter.start();
}

@Override
@AfterClass(alwaysRun = true)
protected void cleanup() throws Exception {
internalCleanup();
}

@Test
public void testEnableWebSocketServer() throws Exception {
HttpClient httpClient = new HttpClient();
WebSocketClient webSocketClient = new WebSocketClient(httpClient);
webSocketClient.start();
MyWebSocket myWebSocket = new MyWebSocket();
String webSocketUri = "ws://localhost:8080/ws/pingpong";
Future<Session> sessionFuture = webSocketClient.connect(myWebSocket, URI.create(webSocketUri));
sessionFuture.get().getRemote().sendPing(ByteBuffer.wrap("ping".getBytes()));
assertTrue(myWebSocket.getResponse().contains("ping"));
}

@Test
public void testProducer() throws Exception {
@Cleanup
PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://localhost:11000")
.build();

@Cleanup
Producer<byte[]> producer = client.newProducer()
.topic("persistent://sample/test/local/websocket-topic")
.create();

for (int i = 0; i < 10; i++) {
producer.send("test".getBytes());
}
}

@Test
public void testProduceAndConsumeMessageWithWebsocket() throws Exception {
HttpClient producerClient = new HttpClient();
WebSocketClient producerWebSocketClient = new WebSocketClient(producerClient);
producerWebSocketClient.start();
MyWebSocket producerSocket = new MyWebSocket();
String produceUri = "ws://localhost:8080/ws/producer/persistent/sample/test/local/websocket-topic";
Future<Session> producerSession = producerWebSocketClient.connect(producerSocket, URI.create(produceUri));

ProducerMessage produceRequest = new ProducerMessage();
produceRequest.setContext("context");
produceRequest.setPayload(Base64.getEncoder().encodeToString("my payload".getBytes()));

HttpClient consumerClient = new HttpClient();
WebSocketClient consumerWebSocketClient = new WebSocketClient(consumerClient);
consumerWebSocketClient.start();
MyWebSocket consumerSocket = new MyWebSocket();
String consumeUri = "ws://localhost:8080/ws/consumer/persistent/sample/test/local/websocket-topic/my-sub";
Future<Session> consumerSession = consumerWebSocketClient.connect(consumerSocket, URI.create(consumeUri));
consumerSession.get().getRemote().sendPing(ByteBuffer.wrap("ping".getBytes()));
producerSession.get().getRemote().sendString(ObjectMapperFactory.getThreadLocal().writeValueAsString(produceRequest));
assertTrue(consumerSocket.getResponse().contains("ping"));
ProducerMessage message = ObjectMapperFactory.getThreadLocal().readValue(consumerSocket.getResponse(), ProducerMessage.class);
assertEquals(new String(Base64.getDecoder().decode(message.getPayload())), "my payload");
}

@WebSocket
public static class MyWebSocket extends WebSocketAdapter implements WebSocketPingPongListener {

ArrayBlockingQueue<String> incomingMessages = new ArrayBlockingQueue<>(10);

@Override
public void onWebSocketText(String message) {
incomingMessages.add(message);
}

@Override
public void onWebSocketClose(int i, String s) {
}

@Override
public void onWebSocketConnect(Session session) {
}

@Override
public void onWebSocketError(Throwable throwable) {
}

@Override
public void onWebSocketPing(ByteBuffer payload) {
}

@Override
public void onWebSocketPong(ByteBuffer payload) {
incomingMessages.add(BufferUtil.toDetailString(payload));
}

public String getResponse() throws InterruptedException {
return incomingMessages.take();
}
}

}

0 comments on commit 93e145b

Please sign in to comment.