Skip to content

Commit

Permalink
[Testing] Fix Maven Surefire default exclude (apache#9836)
Browse files Browse the repository at this point in the history
* [Testing] Fix Maven Surefire default exclude

- Problem introduced in PR apache#9823
- In the shell script "**/*$*" gets replaced with "**/*" when using double quotes.
- Fix: use single quotes for the exclude parameter

* Attempt to fix flaky test that failed the build

* Fix ReplicatorTest retries

- must reset configuration before retrying
  • Loading branch information
lhotari authored Mar 8, 2021
1 parent 4320e4a commit dde59b3
Show file tree
Hide file tree
Showing 4 changed files with 41 additions and 56 deletions.
24 changes: 12 additions & 12 deletions build/run_unit_group.sh
Original file line number Diff line number Diff line change
Expand Up @@ -32,10 +32,10 @@ function broker_group_1() {
-DtestReuseFork=true

$MVN_TEST_COMMAND -pl pulsar-broker -Dinclude="org/apache/pulsar/broker/**/*.java" \
-Dexclude="**/*$*,org/apache/pulsar/broker/zookeeper/**/*.java,
-Dexclude='**/*$*,org/apache/pulsar/broker/zookeeper/**/*.java,
org/apache/pulsar/broker/loadbalance/**/*.java,
org/apache/pulsar/broker/service/**/*.java,
**/AdminApiOffloadTest.java"
**/AdminApiOffloadTest.java'

}

Expand Down Expand Up @@ -63,11 +63,11 @@ function broker_group_2() {
$MVN_TEST_COMMAND -pl pulsar-broker -Dinclude="org/apache/pulsar/broker/zookeeper/**/*.java,
org/apache/pulsar/broker/loadbalance/**/*.java,
org/apache/pulsar/broker/service/**/*.java" \
-Dexclude="**/*$*,**/ReplicatorTest.java,
-Dexclude='**/*$*,**/ReplicatorTest.java,
**/MessagePublishBufferThrottleTest.java,
**/TopicOwnerTest.java,
**/*StreamingDispatcher*Test.java,
**/AntiAffinityNamespaceGroupTest.java"
**/AntiAffinityNamespaceGroupTest.java'
}

function broker_client_api() {
Expand All @@ -80,17 +80,17 @@ function broker_client_api() {
-DtestReuseFork=true

$MVN_TEST_COMMAND -pl pulsar-broker -Dinclude="org/apache/pulsar/client/api/**/*.java" \
-Dexclude="**/*$*,**/DispatcherBlockConsumerTest.java,
**/SimpleProducerConsumerTest.java"
-Dexclude='**/*$*,**/DispatcherBlockConsumerTest.java,
**/SimpleProducerConsumerTest.java'
}

function broker_client_impl() {
$MVN_TEST_COMMAND -pl pulsar-broker -Dinclude="org/apache/pulsar/client/impl/**/*.java"
}

function broker_client_other() {
$MVN_TEST_COMMAND -pl pulsar-broker -Dexclude="**/*$*,org/apache/pulsar/broker/**/*.java,
org/apache/pulsar/client/**/*.java"
$MVN_TEST_COMMAND -pl pulsar-broker -Dexclude='**/*$*,org/apache/pulsar/broker/**/*.java,
org/apache/pulsar/client/**/*.java'
}

function proxy() {
Expand Down Expand Up @@ -119,20 +119,20 @@ function proxy() {
-DtestReuseFork=true

$MVN_TEST_COMMAND -pl pulsar-proxy -DtestForkCount=1 \
-Dexclude="**/*$*,**/ProxyRolesEnforcementTest.java,
-Dexclude='**/*$*,**/ProxyRolesEnforcementTest.java,
**/ProxyAuthenticationTest.java,
**/ProxyTest.java,
**/MessagePublishBufferThrottleTest.java" \
**/MessagePublishBufferThrottleTest.java' \
-DtestReuseFork=true
}

function other() {
build/retry.sh mvn -B -ntp install -PbrokerSkipTest \
-Dexclude="**/*$*,org/apache/pulsar/proxy/**/*.java,
-Dexclude='**/*$*,org/apache/pulsar/proxy/**/*.java,
**/ManagedLedgerTest.java,
**/TestPulsarKeyValueSchemaHandler.java,
**/PrimitiveSchemaTest.java,
BlobStoreManagedLedgerOffloaderTest.java"
BlobStoreManagedLedgerOffloaderTest.java'

$MVN_TEST_COMMAND -pl managed-ledger -Dinclude="**/ManagedLedgerTest.java" \
-DtestForkCount=1 \
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,14 @@
*/
package org.apache.pulsar.broker.service;

import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
import com.google.common.collect.Sets;

import java.lang.reflect.Method;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;

import lombok.Cleanup;

import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.MessageRoutingMode;
Expand All @@ -43,10 +43,6 @@
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;

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

/**
* Starts 3 brokers that are in 3 different clusters
*/
Expand All @@ -69,9 +65,6 @@ public void setup() throws Exception {
@AfterClass(alwaysRun = true, timeOut = 300000)
public void cleanup() throws Exception {
super.cleanup();
resetConfig1();
resetConfig2();
resetConfig3();
}

enum DispatchRateType {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -286,6 +286,10 @@ protected void cleanup() throws Exception {
bkEnsemble2.stop();
bkEnsemble3.stop();
globalZkS.stop();

resetConfig1();
resetConfig2();
resetConfig3();
}

static class MessageProducer implements AutoCloseable {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,12 @@
*/
package org.apache.pulsar.client.cli;

import java.time.Duration;
import org.apache.pulsar.broker.service.BrokerTestBase;
import org.apache.pulsar.websocket.WebSocketService;
import org.apache.pulsar.websocket.service.ProxyServer;
import org.awaitility.Awaitility;
import org.awaitility.pollinterval.PollInterval;
import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
Expand Down Expand Up @@ -49,7 +52,7 @@ protected void cleanup() throws Exception {
super.internalCleanup();
}

@Test(timeOut = 20000)
@Test(timeOut = 30000)
public void testWebSocketNonDurableSubscriptionMode() throws Exception {
Properties properties = new Properties();
properties.setProperty("serviceUrl", brokerUrl.toString());
Expand All @@ -73,38 +76,28 @@ public void testWebSocketNonDurableSubscriptionMode() throws Exception {
});

// Make sure subscription has been created
while (true) {
try {
List<String> subscriptions = admin.topics().getSubscriptions(topicName);
if (subscriptions.size() == 1) {
break;
}
} catch (Exception ignored) {
}
Thread.sleep(200);
}
Awaitility.await()
.pollInterval(Duration.ofMillis(200))
.ignoreExceptions().untilAsserted(() -> {
Assert.assertEquals(admin.topics().getSubscriptions(topicName).size(), 1);
});

PulsarClientTool pulsarClientToolProducer = new PulsarClientTool(properties);

String[] args = {"produce", "--messages", "Have a nice day", "-n", Integer.toString(numberOfMessages), "-r",
"20", "-p", "key1=value1", "-p", "key2=value2", "-k", "partition_key", topicName};
Assert.assertEquals(pulsarClientToolProducer.run(args), 0);
Assert.assertFalse(future.isCompletedExceptionally());
future.get();
Assert.assertFalse(future.isCompletedExceptionally());
executor.shutdown();
while (true) {
try {
List<String> subscriptions = admin.topics().getSubscriptions(topicName);
if (subscriptions.size() == 0) {
break;
}
} catch (Exception ignored) {
}
Thread.sleep(200);
}

Awaitility.await()
.ignoreExceptions().untilAsserted(() -> {
Assert.assertEquals(admin.topics().getSubscriptions(topicName).size(), 0);
});
}

@Test(timeOut = 20000)
@Test(timeOut = 30000)
public void testWebSocketDurableSubscriptionMode() throws Exception {
Properties properties = new Properties();
properties.setProperty("serviceUrl", brokerUrl.toString());
Expand All @@ -128,24 +121,19 @@ public void testWebSocketDurableSubscriptionMode() throws Exception {
});

// Make sure subscription has been created
while (true) {
try {
List<String> subscriptions = admin.topics().getSubscriptions(topicName);
if (subscriptions.size() == 1) {
break;
}
} catch (Exception ignored) {
}
Thread.sleep(200);
}
Awaitility.await()
.pollInterval(Duration.ofMillis(200))
.ignoreExceptions().untilAsserted(() -> {
Assert.assertEquals(admin.topics().getSubscriptions(topicName).size(), 1);
});

PulsarClientTool pulsarClientToolProducer = new PulsarClientTool(properties);

String[] args = {"produce", "--messages", "Have a nice day", "-n", Integer.toString(numberOfMessages), "-r",
"20", "-p", "key1=value1", "-p", "key2=value2", "-k", "partition_key", topicName};
Assert.assertEquals(pulsarClientToolProducer.run(args), 0);
Assert.assertFalse(future.isCompletedExceptionally());
future.get();
Assert.assertFalse(future.isCompletedExceptionally());
executor.shutdown();

//wait for close
Expand Down

0 comments on commit dde59b3

Please sign in to comment.