Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

build: disable testNewEnsembleBookieWithOneEmptyRegion method in TestRegionAwareEnsemblePlacementPolicy beyond jdk11 #4323

Merged
merged 2 commits into from
May 5, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,10 @@
import static org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy.REPP_REGIONS_TO_WRITE;
import static org.apache.bookkeeper.client.RoundRobinDistributionSchedule.writeSetFromValues;
import static org.apache.bookkeeper.feature.SettableFeatureProvider.DISABLE_ALL;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
Expand All @@ -45,7 +49,6 @@
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import junit.framework.TestCase;
import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
import org.apache.bookkeeper.conf.ClientConfiguration;
import org.apache.bookkeeper.feature.FeatureProvider;
Expand All @@ -58,14 +61,18 @@
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.util.BookKeeperConstants;
import org.apache.bookkeeper.util.StaticDNSResolver;
import org.junit.Test;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.condition.EnabledForJreRange;
import org.junit.jupiter.api.condition.JRE;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Test a region-aware ensemble placement policy.
*/
public class TestRegionAwareEnsemblePlacementPolicy extends TestCase {
public class TestRegionAwareEnsemblePlacementPolicy {

static final Logger LOG = LoggerFactory.getLogger(TestRegionAwareEnsemblePlacementPolicy.class);

Expand All @@ -89,9 +96,8 @@ static void updateMyRack(String rack) throws Exception {
StaticDNSResolver.addNodeToRack("localhost", rack);
}

@Override
@BeforeEach
protected void setUp() throws Exception {
super.setUp();
StaticDNSResolver.reset();
updateMyRack(NetworkTopology.DEFAULT_REGION_AND_RACK);
LOG.info("Set up static DNS Resolver.");
Expand Down Expand Up @@ -123,10 +129,9 @@ protected void setUp() throws Exception {
NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER);
}

@Override
@AfterEach
protected void tearDown() throws Exception {
repp.uninitalize();
super.tearDown();
}

static BookiesHealthInfo getBookiesHealthInfo() {
Expand Down Expand Up @@ -190,7 +195,7 @@ public void testNodeInSameRegion() throws Exception {
LOG.info("reorder set : {}", reorderSet);
LOG.info("expected set : {}", expectedSet);
LOG.info("reorder equals {}", reorderSet.equals(writeSet));
assertFalse(reorderSet.equals(writeSet));
assertNotEquals(reorderSet, writeSet);
assertEquals(expectedSet, reorderSet);
}

Expand Down Expand Up @@ -234,7 +239,7 @@ public void testNodeDown() throws Exception {
ensemble, getBookiesHealthInfo(), writeSet);
DistributionSchedule.WriteSet expectedSet = writeSetFromValues(3, 1, 2, 0);
LOG.info("reorder set : {}", reorderSet);
assertFalse(reorderSet.equals(origWriteSet));
assertNotEquals(reorderSet, origWriteSet);
assertEquals(expectedSet, reorderSet);
}

Expand Down Expand Up @@ -264,7 +269,7 @@ public void testNodeReadOnly() throws Exception {
ensemble, getBookiesHealthInfo(), writeSet);
DistributionSchedule.WriteSet expectedSet = writeSetFromValues(3, 1, 2, 0);
LOG.info("reorder set : {}", reorderSet);
assertFalse(reorderSet.equals(origWriteSet));
assertNotEquals(reorderSet, origWriteSet);
assertEquals(expectedSet, reorderSet);
}

Expand Down Expand Up @@ -294,7 +299,7 @@ public void testNodeSlow() throws Exception {
ensemble, getBookiesHealthInfo(new HashMap<>(), bookiePendingMap), writeSet);
DistributionSchedule.WriteSet expectedSet = writeSetFromValues(3, 1, 2, 0);
LOG.info("reorder set : {}", reorderSet);
assertFalse(reorderSet.equals(origWriteSet));
assertNotEquals(reorderSet, origWriteSet);
assertEquals(expectedSet, reorderSet);
}

Expand Down Expand Up @@ -326,7 +331,7 @@ public void testTwoNodesSlow() throws Exception {
ensemble, getBookiesHealthInfo(new HashMap<>(), bookiePendingMap), writeSet);
DistributionSchedule.WriteSet expectedSet = writeSetFromValues(3, 2, 0, 1);
LOG.info("reorder set : {}", reorderSet);
assertFalse(reorderSet.equals(origWriteSet));
assertNotEquals(reorderSet, origWriteSet);
assertEquals(expectedSet, reorderSet);
}

Expand Down Expand Up @@ -355,7 +360,7 @@ public void testTwoNodesDown() throws Exception {
ensemble, getBookiesHealthInfo(), writeSet);
DistributionSchedule.WriteSet expectedSet = writeSetFromValues(3, 2, 0, 1);
LOG.info("reorder set : {}", reorderSet);
assertFalse(reorderSet.equals(origWriteSet));
assertNotEquals(reorderSet, origWriteSet);
assertEquals(expectedSet, reorderSet);
}

Expand Down Expand Up @@ -386,7 +391,7 @@ public void testNodeDownAndNodeSlow() throws Exception {
ensemble, getBookiesHealthInfo(new HashMap<>(), bookiePendingMap), writeSet);
DistributionSchedule.WriteSet expectedSet = writeSetFromValues(3, 2, 0, 1);
LOG.info("reorder set : {}", reorderSet);
assertFalse(reorderSet.equals(origWriteSet));
assertNotEquals(reorderSet, origWriteSet);
assertEquals(expectedSet, reorderSet);
}

Expand Down Expand Up @@ -420,7 +425,7 @@ public void testNodeDownAndReadOnlyAndNodeSlow() throws Exception {
ensemble, getBookiesHealthInfo(new HashMap<>(), bookiePendingMap), writeSet);
DistributionSchedule.WriteSet expectedSet = writeSetFromValues(3, 1, 2, 0);
LOG.info("reorder set : {}", reorderSet);
assertFalse(reorderSet.equals(origWriteSet));
assertNotEquals(reorderSet, origWriteSet);
assertEquals(expectedSet, reorderSet);
}

Expand Down Expand Up @@ -472,7 +477,7 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRegion() throws Excepti
BookieId replacedBookie = repp.replaceBookie(1, 1, 1, null,
new ArrayList<BookieId>(), addr2.toBookieId(), excludedAddrs).getResult();

assertFalse(addr1.toBookieId().equals(replacedBookie));
assertNotEquals(addr1.toBookieId(), replacedBookie);
assertTrue(addr3.toBookieId().equals(replacedBookie)
|| addr4.toBookieId().equals(replacedBookie));
}
Expand Down Expand Up @@ -506,6 +511,7 @@ public void testNewEnsembleBookieWithNotEnoughBookies() throws Exception {
}

@Test
@EnabledForJreRange(max = JRE.JAVA_11)
public void testNewEnsembleBookieWithOneEmptyRegion() throws Exception {
BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181);
BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181);
Expand Down Expand Up @@ -1422,17 +1428,17 @@ private void basicReorderReadSequenceWithLocalRegionTest(String myRegion, boolea
.resolve(address).getHostName()));
}
BookieId remoteAddress = ensemble.get(readSet.get(k));
assertFalse(myRegion.equals(StaticDNSResolver.getRegion(repp.bookieAddressResolver
.resolve(remoteAddress).getHostName())));
assertNotEquals(myRegion, StaticDNSResolver.getRegion(repp.bookieAddressResolver
.resolve(remoteAddress).getHostName()));
k++;
BookieId localAddress = ensemble.get(readSet.get(k));
assertEquals(myRegion, StaticDNSResolver.getRegion(repp.bookieAddressResolver
.resolve(localAddress).getHostName()));
k++;
for (; k < ensembleSize; k++) {
BookieId address = ensemble.get(readSet.get(k));
assertFalse(myRegion.equals(StaticDNSResolver.getRegion(repp.bookieAddressResolver
.resolve(address).getHostName())));
assertNotEquals(myRegion, StaticDNSResolver.getRegion(repp.bookieAddressResolver
.resolve(address).getHostName()));
}
}
}
Expand Down Expand Up @@ -1734,12 +1740,12 @@ public void testNewEnsembleSetWithFiveRegions() throws Exception {
List<BookieId> ensemble2 = repp.newEnsemble(3, 3, 2,
null, new HashSet<>()).getResult();
ensemble1.retainAll(ensemble2);
assert(ensemble1.size() >= 1);
assert(!ensemble1.isEmpty());

List<BookieId> ensemble3 = repp.newEnsemble(3, 3, 2,
null, new HashSet<>()).getResult();
ensemble2.removeAll(ensemble3);
assert(ensemble2.size() >= 1);
assert(!ensemble2.isEmpty());
} catch (BKNotEnoughBookiesException bnebe) {
fail("Should not get not enough bookies exception even there is only one rack.");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,9 @@
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.TestInfo;
import org.junit.rules.TestName;
import org.junit.rules.Timeout;
import org.slf4j.Logger;
Expand All @@ -101,6 +104,8 @@ public abstract class BookKeeperClusterTestCase {
@Rule
public final Timeout globalTimeout;

protected String testName;

// Metadata service related variables
protected final ZooKeeperCluster zkUtil;
protected ZooKeeper zkc;
Expand Down Expand Up @@ -158,10 +163,21 @@ public BookKeeperClusterTestCase(int numBookies, int numOfZKNodes, int testTimeo
}

@Before
@BeforeEach
public void setUp() throws Exception {
setUp("/ledgers");
}

@Before
public void setTestNameJunit4() {
testName = runtime.getMethodName();
}

@BeforeEach
void setTestNameJunit5(TestInfo testInfo) {
testName = testInfo.getDisplayName();
}

protected void setUp(String ledgersRootPath) throws Exception {
LOG.info("Setting up test {}", getClass());
InMemoryMetaStore.reset();
Expand All @@ -176,7 +192,7 @@ protected void setUp(String ledgersRootPath) throws Exception {
this.metadataServiceUri = getMetadataServiceUri(ledgersRootPath);
startBKCluster(metadataServiceUri);
LOG.info("Setup testcase {} @ metadata service {} in {} ms.",
runtime.getMethodName(), metadataServiceUri, sw.elapsed(TimeUnit.MILLISECONDS));
testName, metadataServiceUri, sw.elapsed(TimeUnit.MILLISECONDS));
} catch (Exception e) {
LOG.error("Error setting up", e);
throw e;
Expand All @@ -188,6 +204,7 @@ protected String getMetadataServiceUri(String ledgersRootPath) {
}

@After
@AfterEach
public void tearDown() throws Exception {
boolean failed = false;
for (Throwable e : asyncExceptions) {
Expand Down Expand Up @@ -219,7 +236,7 @@ public void tearDown() throws Exception {
LOG.error("Got Exception while trying to cleanupTempDirs", e);
tearDownException = e;
}
LOG.info("Tearing down test {} in {} ms.", runtime.getMethodName(), sw.elapsed(TimeUnit.MILLISECONDS));
LOG.info("Tearing down test {} in {} ms.", testName, sw.elapsed(TimeUnit.MILLISECONDS));
if (tearDownException != null) {
throw tearDownException;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
import org.apache.bookkeeper.proto.BookieServer;
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.util.PortManager;
import org.junit.Test;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.condition.EnabledForJreRange;
import org.junit.jupiter.api.condition.JRE;

Expand Down Expand Up @@ -71,7 +71,7 @@ public void testBookieServerZKRequestTimeoutBehaviour() throws Exception {
Thread[] threads = new Thread[threadCount * 2];
threadCount = Thread.enumerate(threads);
for (int i = 0; i < threadCount; i++) {
if (threads[i].getName().indexOf("SendThread") != -1) {
if (threads[i].getName().contains("SendThread")) {
threadset.add(threads[i]);
}
}
Expand All @@ -95,7 +95,7 @@ conf, new TestBookieImpl(conf),
threads = new Thread[threadCount * 2];
threadCount = Thread.enumerate(threads);
for (int i = 0; i < threadCount; i++) {
if (threads[i].getName().indexOf("SendThread") != -1
if (threads[i].getName().contains("SendThread")
&& !threadset.contains(threads[i])) {
sendthread = threads[i];
break;
Expand Down
Loading