forked from apache/pulsar
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[fix][broker] catch exception for brokerInterceptor (apache#19147)
- Loading branch information
1 parent
0f025f3
commit 5d1fc6d
Showing
6 changed files
with
267 additions
and
6 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
102 changes: 102 additions & 0 deletions
102
...-broker/src/test/java/org/apache/pulsar/broker/intercept/ExceptionsBrokerInterceptor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,102 @@ | ||
/* | ||
* 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.broker.intercept; | ||
|
||
import java.io.IOException; | ||
import java.util.Map; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
import javax.servlet.ServletException; | ||
import javax.servlet.ServletRequest; | ||
import javax.servlet.ServletResponse; | ||
import org.apache.pulsar.broker.PulsarService; | ||
import org.apache.pulsar.broker.service.Consumer; | ||
import org.apache.pulsar.broker.service.Producer; | ||
import org.apache.pulsar.broker.service.ServerCnx; | ||
import org.apache.pulsar.common.api.proto.BaseCommand; | ||
import org.apache.pulsar.common.api.proto.CommandAck; | ||
import org.apache.pulsar.common.intercept.InterceptException; | ||
|
||
public class ExceptionsBrokerInterceptor implements BrokerInterceptor { | ||
|
||
|
||
private AtomicInteger producerCount = new AtomicInteger(); | ||
private AtomicInteger consumerCount = new AtomicInteger(); | ||
private AtomicInteger messageAckCount = new AtomicInteger(); | ||
|
||
public AtomicInteger getProducerCount() { | ||
return producerCount; | ||
} | ||
|
||
public AtomicInteger getConsumerCount() { | ||
return consumerCount; | ||
} | ||
|
||
public AtomicInteger getMessageAckCount() { | ||
return messageAckCount; | ||
} | ||
|
||
@Override | ||
public void producerCreated(ServerCnx cnx, Producer producer, Map<String, String> metadata) { | ||
producerCount.incrementAndGet(); | ||
throw new RuntimeException("exception when intercept producer created"); | ||
} | ||
|
||
@Override | ||
public void consumerCreated(ServerCnx cnx, Consumer consumer, Map<String, String> metadata) { | ||
consumerCount.incrementAndGet(); | ||
throw new RuntimeException("exception when intercept consumer created"); | ||
} | ||
|
||
@Override | ||
public void messageAcked(ServerCnx cnx, Consumer consumer, CommandAck ackCmd) { | ||
messageAckCount.incrementAndGet(); | ||
throw new RuntimeException("exception when intercept consumer ack message"); | ||
} | ||
|
||
@Override | ||
public void onPulsarCommand(BaseCommand command, ServerCnx cnx) throws InterceptException { | ||
|
||
} | ||
|
||
@Override | ||
public void onConnectionClosed(ServerCnx cnx) { | ||
|
||
} | ||
|
||
@Override | ||
public void onWebserviceRequest(ServletRequest request) throws IOException, ServletException, InterceptException { | ||
|
||
} | ||
|
||
@Override | ||
public void onWebserviceResponse(ServletRequest request, ServletResponse response) | ||
throws IOException, ServletException { | ||
|
||
} | ||
|
||
@Override | ||
public void initialize(PulsarService pulsarService) throws Exception { | ||
|
||
} | ||
|
||
@Override | ||
public void close() { | ||
|
||
} | ||
} |
117 changes: 117 additions & 0 deletions
117
...ker/src/test/java/org/apache/pulsar/broker/intercept/ExceptionsBrokerInterceptorTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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. | ||
*/ | ||
package org.apache.pulsar.broker.intercept; | ||
|
||
import static org.mockito.Mockito.mock; | ||
import static org.testng.Assert.assertEquals; | ||
import static org.testng.Assert.assertNotNull; | ||
import static org.testng.Assert.assertTrue; | ||
|
||
import java.nio.charset.StandardCharsets; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.concurrent.TimeUnit; | ||
import org.apache.pulsar.broker.testcontext.PulsarTestContext; | ||
import org.apache.pulsar.client.api.Message; | ||
import org.apache.pulsar.client.api.Producer; | ||
import org.apache.pulsar.client.api.ProducerConsumerBase; | ||
import org.apache.pulsar.client.impl.ClientCnx; | ||
import org.apache.pulsar.client.impl.ConsumerImpl; | ||
import org.apache.pulsar.common.nar.NarClassLoader; | ||
import org.awaitility.Awaitility; | ||
import org.testng.annotations.BeforeMethod; | ||
import org.testng.annotations.Test; | ||
|
||
public class ExceptionsBrokerInterceptorTest extends ProducerConsumerBase { | ||
|
||
private String interceptorName = "exception_interceptor"; | ||
|
||
@BeforeMethod | ||
public void setup() throws Exception { | ||
conf.setSystemTopicEnabled(false); | ||
conf.setTopicLevelPoliciesEnabled(false); | ||
this.conf.setDisableBrokerInterceptors(false); | ||
|
||
|
||
this.enableBrokerInterceptor = true; | ||
super.internalSetup(); | ||
super.producerBaseSetup(); | ||
} | ||
|
||
@Override | ||
protected void cleanup() throws Exception { | ||
super.internalCleanup(); | ||
} | ||
|
||
@Override | ||
protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { | ||
Map<String, BrokerInterceptorWithClassLoader> listenerMap = new HashMap<>(); | ||
BrokerInterceptor interceptor = new ExceptionsBrokerInterceptor(); | ||
NarClassLoader narClassLoader = mock(NarClassLoader.class); | ||
listenerMap.put(interceptorName, new BrokerInterceptorWithClassLoader(interceptor, narClassLoader)); | ||
pulsarTestContextBuilder.brokerInterceptor(new BrokerInterceptors(listenerMap)); | ||
} | ||
|
||
@Test | ||
public void testMessageAckedExceptions() throws Exception { | ||
String topic = "persistent://public/default/test"; | ||
String subName = "test-sub"; | ||
int messageNumber = 10; | ||
admin.topics().createNonPartitionedTopic(topic); | ||
|
||
BrokerInterceptors listener = (BrokerInterceptors) pulsar.getBrokerInterceptor(); | ||
assertNotNull(listener); | ||
BrokerInterceptorWithClassLoader brokerInterceptor = listener.getInterceptors().get(interceptorName); | ||
assertNotNull(brokerInterceptor); | ||
BrokerInterceptor interceptor = brokerInterceptor.getInterceptor(); | ||
assertTrue(interceptor instanceof ExceptionsBrokerInterceptor); | ||
|
||
Producer<byte[]> producer = pulsarClient.newProducer().topic(topic).create(); | ||
|
||
ConsumerImpl consumer = (ConsumerImpl) pulsarClient | ||
.newConsumer() | ||
.topic(topic) | ||
.subscriptionName(subName) | ||
.acknowledgmentGroupTime(0, TimeUnit.MILLISECONDS) | ||
.isAckReceiptEnabled(true) | ||
.subscribe(); | ||
|
||
Awaitility.await().until(() -> ((ExceptionsBrokerInterceptor) interceptor).getProducerCount().get() == 1); | ||
Awaitility.await().until(() -> ((ExceptionsBrokerInterceptor) interceptor).getConsumerCount().get() == 1); | ||
|
||
for (int i = 0; i < messageNumber; i ++) { | ||
producer.send("test".getBytes(StandardCharsets.UTF_8)); | ||
} | ||
|
||
int receiveCounter = 0; | ||
Message message; | ||
while((message = consumer.receive(3, TimeUnit.SECONDS)) != null) { | ||
receiveCounter ++; | ||
consumer.acknowledge(message); | ||
} | ||
assertEquals(receiveCounter, 10); | ||
Awaitility.await().until(() | ||
-> ((ExceptionsBrokerInterceptor) interceptor).getMessageAckCount().get() == messageNumber); | ||
|
||
ClientCnx clientCnx = consumer.getClientCnx(); | ||
// no duplicated responses received from broker | ||
assertEquals(clientCnx.getDuplicatedResponseCount(), 0); | ||
} | ||
|
||
} |
Oops, something went wrong.