From ee22969a61e650e8ff24ef4e36ffd14b2a80f469 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 12:09:03 +0200 Subject: [PATCH 01/56] [fix][metadata] Fix ephemeral status of ZK nodes --- .../auth/MockedPulsarServiceBaseTest.java | 20 ++++- .../broker/testcontext/PulsarTestContext.java | 77 +++++++++++++++++-- .../client/api/BrokerServiceLookupTest.java | 17 ++++ .../pulsar/metadata/impl/ZKMetadataStore.java | 5 +- .../metadata/MetadataStoreExtendedTest.java | 5 +- .../org/apache/zookeeper/MockZooKeeper.java | 42 ++++++---- .../zookeeper/MockZooKeeperSession.java | 26 +++++-- 7 files changed, 159 insertions(+), 33 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index 42e2c00f73acf..81d1a105c175f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -150,6 +150,9 @@ public static String getTlsFileForClient(String name) { private final List closeables = new ArrayList<>(); + // Set to true in test's constructor to use a real Zookeeper (TestZKServer) + protected boolean useTestZookeeper; + public MockedPulsarServiceBaseTest() { resetConfig(); } @@ -461,7 +464,6 @@ protected PulsarTestContext.Builder createPulsarTestContextBuilder(ServiceConfig PulsarTestContext.Builder builder = PulsarTestContext.builder() .spyByDefault() .config(conf) - .withMockZookeeper(true) .pulsarServiceCustomizer(pulsarService -> { try { beforePulsarStart(pulsarService); @@ -470,9 +472,25 @@ protected PulsarTestContext.Builder createPulsarTestContextBuilder(ServiceConfig } }) .brokerServiceCustomizer(this::customizeNewBrokerService); + configureMetadataStores(builder); return builder; } + /** + * Configures the metadata stores for the PulsarTestContext.Builder instance. + * Set useTestZookeeper to true in the test's constructor to use TestZKServer which is a real ZooKeeper + * implementation. + * + * @param builder the PulsarTestContext.Builder instance to configure + */ + protected void configureMetadataStores(PulsarTestContext.Builder builder) { + if (useTestZookeeper) { + builder.withTestZookeeper(); + } else { + builder.withMockZookeeper(); + } + } + protected PulsarTestContext createAdditionalPulsarTestContext(ServiceConfiguration conf) throws Exception { return createAdditionalPulsarTestContext(conf, null); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java index 6403c3bcec4c3..82a1b574ac921 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java @@ -27,7 +27,6 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.time.Duration; -import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Optional; @@ -65,6 +64,7 @@ import org.apache.pulsar.compaction.CompactionServiceFactory; import org.apache.pulsar.compaction.Compactor; import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; +import org.apache.pulsar.metadata.TestZKServer; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -72,9 +72,11 @@ import org.apache.pulsar.metadata.impl.MetadataStoreFactoryImpl; import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.MockZooKeeper; import org.apache.zookeeper.MockZooKeeperSession; -import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; import org.jetbrains.annotations.NotNull; import org.mockito.Mockito; import org.mockito.internal.util.MockUtil; @@ -477,16 +479,77 @@ public Builder withMockZookeeper(boolean useSeparateGlobalZk) { private MockZooKeeper createMockZooKeeper() throws Exception { MockZooKeeper zk = MockZooKeeper.newInstance(MoreExecutors.newDirectExecutorService()); - List dummyAclList = new ArrayList<>(0); + initializeZookeeper(zk); + registerCloseable(zk::shutdown); + return zk; + } + private static void initializeZookeeper(ZooKeeper zk) throws KeeperException, InterruptedException { ZkUtils.createFullPathOptimistic(zk, "/ledgers/available/192.168.1.1:" + 5000, - "".getBytes(StandardCharsets.UTF_8), dummyAclList, CreateMode.PERSISTENT); + "".getBytes(StandardCharsets.UTF_8), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zk.create("/ledgers/LAYOUT", "1\nflat:1".getBytes(StandardCharsets.UTF_8), dummyAclList, + zk.create("/ledgers/LAYOUT", "1\nflat:1".getBytes(StandardCharsets.UTF_8), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } - registerCloseable(zk::shutdown); - return zk; + /** + * Configure this PulsarTestContext to use a test ZooKeeper instance which is + * shared for both the local and configuration metadata stores. + * + * @return the builder + */ + public Builder withTestZookeeper() { + return withTestZookeeper(false); + } + + /** + * Configure this PulsarTestContext to use a test ZooKeeper instance. + * + * @param useSeparateGlobalZk if true, the global (configuration) zookeeper will be a separate instance + * @return the builder + */ + public Builder withTestZookeeper(boolean useSeparateGlobalZk) { + try { + TestZKServer localZk = createTestZookeeper(); + MetadataStoreExtended localStore = + createTestZookeeperMetadataStore(localZk, MetadataStoreConfig.METADATA_STORE); + localMetadataStore(localStore); + MetadataStoreExtended configStore; + if (useSeparateGlobalZk) { + TestZKServer globalZk = createTestZookeeper(); + configStore = createTestZookeeperMetadataStore(globalZk, + MetadataStoreConfig.CONFIGURATION_METADATA_STORE); + } else { + configStore = + createTestZookeeperMetadataStore(localZk, MetadataStoreConfig.CONFIGURATION_METADATA_STORE); + } + configurationMetadataStore(configStore); + } catch (Exception e) { + throw new RuntimeException(e); + } + return this; + } + + private TestZKServer createTestZookeeper() throws Exception { + TestZKServer testZKServer = new TestZKServer(); + try (ZooKeeper zkc = new ZooKeeper(testZKServer.getConnectionString(), 5000, event -> { + })) { + initializeZookeeper(zkc); + } + registerCloseable(testZKServer); + return testZKServer; + } + + private MetadataStoreExtended createTestZookeeperMetadataStore(TestZKServer zkServer, + String metadataStoreName) { + try { + MetadataStoreExtended store = MetadataStoreExtended.create("zk:" + zkServer.getConnectionString(), + MetadataStoreConfig.builder().metadataStoreName(metadataStoreName).build()); + registerCloseable(store); + return store; + } catch (MetadataStoreException e) { + throw new RuntimeException(e); + } } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index 07deb9007c487..34db053271e1a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -112,14 +112,28 @@ import org.awaitility.reflect.WhiteboxImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testng.SkipException; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "broker-api") public class BrokerServiceLookupTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(BrokerServiceLookupTest.class); + @DataProvider + private static Object[] booleanValues() { + return new Object[]{ true, false }; + } + + @Factory(dataProvider = "booleanValues") + public BrokerServiceLookupTest(boolean useTestZookeeper) { + // when set to true, TestZKServer is used which is a real ZooKeeper implementation + this.useTestZookeeper = useTestZookeeper; + } + @BeforeMethod @Override protected void setup() throws Exception { @@ -1197,6 +1211,9 @@ public String authenticate(AuthenticationDataSource authData) throws Authenticat @Test public void testLookupConnectionNotCloseIfGetUnloadingExOrMetadataEx() throws Exception { + if (useTestZookeeper) { + throw new SkipException("This test case depends on MockZooKeeper"); + } String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); admin.topics().createNonPartitionedTopic(tpName); PulsarClientImpl pulsarClientImpl = (PulsarClientImpl) pulsarClient; diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index 8fd8252152898..f67f5bff73296 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -71,9 +71,10 @@ @Slf4j public class ZKMetadataStore extends AbstractBatchedMetadataStore implements MetadataStoreExtended, MetadataStoreLifecycle { - public static final String ZK_SCHEME = "zk"; public static final String ZK_SCHEME_IDENTIFIER = "zk:"; + // ephemeralOwner value for persistent nodes + private static final long NOT_EPHEMERAL = 0L; private final String zkConnectString; private final String rootPath; @@ -478,7 +479,7 @@ public void close() throws Exception { private Stat getStat(String path, org.apache.zookeeper.data.Stat zkStat) { return new Stat(path, zkStat.getVersion(), zkStat.getCtime(), zkStat.getMtime(), - zkStat.getEphemeralOwner() != -1, + zkStat.getEphemeralOwner() != NOT_EPHEMERAL, zkStat.getEphemeralOwner() == zkc.getSessionId()); } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java index a4c937611fd3f..b71511aabceae 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.metadata; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @@ -74,14 +75,14 @@ public void testPersistentOrEphemeralPut(String provider, Supplier urlSu store.put(key1, "value-1".getBytes(), Optional.empty(), EnumSet.noneOf(CreateOption.class)).join(); var value = store.get(key1).join().get(); assertEquals(value.getValue(), "value-1".getBytes()); - // assertFalse(value.getStat().isEphemeral()); // Todo : fix zkStat.getEphemeralOwner() != 0 from test zk + assertFalse(value.getStat().isEphemeral()); assertTrue(value.getStat().isFirstVersion()); var version = value.getStat().getVersion(); store.put(key1, "value-2".getBytes(), Optional.empty(), EnumSet.noneOf(CreateOption.class)).join(); value = store.get(key1).join().get(); assertEquals(value.getValue(), "value-2".getBytes()); - //assertFalse(value.getStat().isEphemeral()); // Todo : fix zkStat.getEphemeralOwner() != 0 from test zk + assertFalse(value.getStat().isEphemeral()); assertEquals(value.getStat().getVersion(), version + 1); final String key2 = newKey(); diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index f32036e53f001..7b23d0814aab9 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -64,6 +64,9 @@ import org.slf4j.LoggerFactory; public class MockZooKeeper extends ZooKeeper { + // ephemeralOwner value for persistent nodes + private static final long NOT_EPHEMERAL = 0L; + @Data @AllArgsConstructor private static class MockZNode { @@ -84,13 +87,13 @@ static MockZNode of(byte[] content, int version, long ephemeralOwner) { private ExecutorService executor; private Watcher sessionWatcher; - private long sessionId = 0L; + private long sessionId = 1L; private int readOpDelayMs; private ReentrantLock mutex; private AtomicLong sequentialIdGenerator; - private ThreadLocal epheralOwnerThreadLocal; + private ThreadLocal ephemeralOwnerThreadLocal; //see details of Objenesis caching - http://objenesis.org/details.html //see supported jvms - https://github.com/easymock/objenesis/blob/master/SupportedJVMs.md @@ -156,7 +159,7 @@ private static MockZooKeeper createMockZooKeeperInstance(ExecutorService executo ObjectInstantiator mockZooKeeperInstantiator = objenesis.getInstantiatorOf(MockZooKeeper.class); MockZooKeeper zk = mockZooKeeperInstantiator.newInstance(); - zk.epheralOwnerThreadLocal = new ThreadLocal<>(); + zk.ephemeralOwnerThreadLocal = new ThreadLocal<>(); zk.init(executor); zk.readOpDelayMs = readOpDelayMs; zk.mutex = new ReentrantLock(); @@ -278,7 +281,7 @@ public String create(String path, byte[] data, List acl, CreateMode createM MockZNode.of(parentNode.getContent(), parentVersion + 1, parentNode.getEphemeralOwner())); } - tree.put(path, MockZNode.of(data, 0, createMode.isEphemeral() ? getEphemeralOwner() : -1L)); + tree.put(path, MockZNode.of(data, 0, createMode.isEphemeral() ? getEphemeralOwner() : NOT_EPHEMERAL)); toNotifyCreate.addAll(watchers.get(path)); @@ -310,19 +313,19 @@ public String create(String path, byte[] data, List acl, CreateMode createM } protected long getEphemeralOwner() { - Long epheralOwner = epheralOwnerThreadLocal.get(); - if (epheralOwner != null) { - return epheralOwner; + Long ephemeralOwner = ephemeralOwnerThreadLocal.get(); + if (ephemeralOwner != null) { + return ephemeralOwner; } return getSessionId(); } - public void overrideEpheralOwner(long epheralOwner) { - epheralOwnerThreadLocal.set(epheralOwner); + public void overrideEphemeralOwner(long ephemeralOwner) { + ephemeralOwnerThreadLocal.set(ephemeralOwner); } - public void removeEpheralOwnerOverride() { - epheralOwnerThreadLocal.remove(); + public void removeEphemeralOwnerOverride() { + ephemeralOwnerThreadLocal.remove(); } @Override @@ -373,7 +376,7 @@ public void create(final String path, final byte[] data, final List acl, Cr cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); } else { tree.put(name, MockZNode.of(data, 0, - createMode != null && createMode.isEphemeral() ? getEphemeralOwner() : -1L)); + createMode != null && createMode.isEphemeral() ? getEphemeralOwner() : NOT_EPHEMERAL)); watchers.removeAll(name); unlockIfLocked(); cb.processResult(0, path, ctx, name); @@ -678,9 +681,7 @@ private static Stat createStatForZNode(MockZNode zNode) { private static Stat applyToStat(MockZNode zNode, Stat stat) { stat.setVersion(zNode.getVersion()); - if (zNode.getEphemeralOwner() != -1L) { - stat.setEphemeralOwner(zNode.getEphemeralOwner()); - } + stat.setEphemeralOwner(zNode.getEphemeralOwner()); return stat; } @@ -1199,5 +1200,16 @@ private void triggerPersistentWatches(String path, String parent, EventType even }); } + public void deleteEphemeralNodes(long sessionId) { + if (sessionId != NOT_EPHEMERAL) { + lock(); + try { + tree.values().removeIf(zNode -> zNode.getEphemeralOwner() == sessionId); + } finally { + unlockIfLocked(); + } + } + } + private static final Logger log = LoggerFactory.getLogger(MockZooKeeper.class); } diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java index a286a75aa9103..a75402018ac04 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java @@ -88,10 +88,10 @@ public void register(Watcher watcher) { public String create(String path, byte[] data, List acl, CreateMode createMode) throws KeeperException, InterruptedException { try { - mockZooKeeper.overrideEpheralOwner(getSessionId()); + mockZooKeeper.overrideEphemeralOwner(getSessionId()); return mockZooKeeper.create(path, data, acl, createMode); } finally { - mockZooKeeper.removeEpheralOwnerOverride(); + mockZooKeeper.removeEphemeralOwnerOverride(); } } @@ -99,10 +99,10 @@ public String create(String path, byte[] data, List acl, CreateMode createM public void create(final String path, final byte[] data, final List acl, CreateMode createMode, final AsyncCallback.StringCallback cb, final Object ctx) { try { - mockZooKeeper.overrideEpheralOwner(getSessionId()); + mockZooKeeper.overrideEphemeralOwner(getSessionId()); mockZooKeeper.create(path, data, acl, createMode, cb, ctx); } finally { - mockZooKeeper.removeEpheralOwnerOverride(); + mockZooKeeper.removeEphemeralOwnerOverride(); } } @@ -188,12 +188,22 @@ public void delete(final String path, int version, final VoidCallback cb, final @Override public void multi(Iterable ops, AsyncCallback.MultiCallback cb, Object ctx) { - mockZooKeeper.multi(ops, cb, ctx); + try { + mockZooKeeper.overrideEphemeralOwner(getSessionId()); + mockZooKeeper.multi(ops, cb, ctx); + } finally { + mockZooKeeper.removeEphemeralOwnerOverride(); + } } @Override public List multi(Iterable ops) throws InterruptedException, KeeperException { - return mockZooKeeper.multi(ops); + try { + mockZooKeeper.overrideEphemeralOwner(getSessionId()); + return mockZooKeeper.multi(ops); + } finally { + mockZooKeeper.removeEphemeralOwnerOverride(); + } } @Override @@ -221,12 +231,16 @@ public void addWatch(String basePath, AddWatchMode mode, VoidCallback cb, Object public void close() throws InterruptedException { if (closeMockZooKeeperOnClose) { mockZooKeeper.close(); + } else { + mockZooKeeper.deleteEphemeralNodes(getSessionId()); } } public void shutdown() throws InterruptedException { if (closeMockZooKeeperOnClose) { mockZooKeeper.shutdown(); + } else { + mockZooKeeper.deleteEphemeralNodes(getSessionId()); } } From c45c6d6a62750b9de8e0cbcae4bb547a639a35a0 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 12:47:55 +0200 Subject: [PATCH 02/56] Run pulsar-metadata tests with MockZooKeeper --- .../pulsar/metadata/impl/ZKMetadataStore.java | 9 +++- .../metadata/BaseMetadataStoreTest.java | 31 +++++++++++- .../MockZooKeeperMetadataStoreProvider.java | 49 +++++++++++++++++++ .../impl/MetadataStoreFactoryImplTest.java | 21 ++++---- .../org/apache/zookeeper/MockZooKeeper.java | 26 +++++++++- .../zookeeper/MockZooKeeperSession.java | 5 +- 6 files changed, 126 insertions(+), 15 deletions(-) create mode 100644 pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MockZooKeeperMetadataStoreProvider.java diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index f67f5bff73296..dab3553f8ce1b 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -130,12 +130,17 @@ public ZKMetadataStore(ZooKeeper zkc) { @VisibleForTesting @SneakyThrows public ZKMetadataStore(ZooKeeper zkc, MetadataStoreConfig config) { - super(config); + this(zkc, config, false); + } + @VisibleForTesting + @SneakyThrows + public ZKMetadataStore(ZooKeeper zkc, MetadataStoreConfig config, boolean isZkManaged) { + super(config); this.zkConnectString = null; this.rootPath = null; this.metadataStoreConfig = null; - this.isZkManaged = false; + this.isZkManaged = isZkManaged; this.zkc = zkc; this.sessionWatcher = new ZKSessionWatcher(zkc, this::receivedSessionEvent); zkc.addWatch("/", this::handleWatchEvent, AddWatchMode.PERSISTENT_RECURSIVE); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index d0265e3ca44ee..c77fba9ceeafb 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -30,6 +30,10 @@ import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.MetadataStoreFactory; +import org.apache.pulsar.metadata.impl.MetadataStoreFactoryImpl; import org.apache.pulsar.tests.TestRetrySupport; import org.assertj.core.util.Files; import org.testng.annotations.AfterClass; @@ -37,16 +41,27 @@ import org.testng.annotations.DataProvider; public abstract class BaseMetadataStoreTest extends TestRetrySupport { + private static String originalMetadatastoreProvidersPropertyValue; protected TestZKServer zks; protected EtcdCluster etcdCluster; - protected OxiaContainer oxiaServer; + private String mockZkUrl; + // reference to keep the MockZooKeeper instance alive in MockZookeeperMetadataStoreProvider + private MetadataStore mockZkStoreRef; @BeforeClass(alwaysRun = true) @Override public void setup() throws Exception { incrementSetupNumber(); zks = new TestZKServer(); + originalMetadatastoreProvidersPropertyValue = + System.getProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY); + // register MockZooKeeperMetadataStoreProvider + System.setProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY, + MockZooKeeperMetadataStoreProvider.class.getName()); + mockZkUrl = "mock-zk:" + UUID.randomUUID(); + // create a reference in MockZooKeeperMetadataStoreProvider to keep the MockZooKeeper instance alive + mockZkStoreRef = MetadataStoreFactory.create(mockZkUrl, MetadataStoreConfig.builder().build()); } @AfterClass(alwaysRun = true) @@ -67,6 +82,19 @@ public void cleanup() throws Exception { oxiaServer.close(); oxiaServer = null; } + + if (mockZkStoreRef != null) { + mockZkStoreRef.close(); + mockZkStoreRef = null; + mockZkUrl = null; + } + + if (originalMetadatastoreProvidersPropertyValue != null) { + System.setProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY, + originalMetadatastoreProvidersPropertyValue); + } else { + System.clearProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY); + } } private static String createTempFolder() { @@ -88,6 +116,7 @@ public Object[][] implementations() { {"RocksDB", stringSupplier(() -> "rocksdb:" + createTempFolder())}, {"Etcd", stringSupplier(() -> "etcd:" + getEtcdClusterConnectString())}, {"Oxia", stringSupplier(() -> "oxia://" + getOxiaServerConnectString())}, + {"MockZooKeeper", stringSupplier(() -> mockZkUrl)}, }; } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MockZooKeeperMetadataStoreProvider.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MockZooKeeperMetadataStoreProvider.java new file mode 100644 index 0000000000000..994a97c2b1053 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MockZooKeeperMetadataStoreProvider.java @@ -0,0 +1,49 @@ +/* + * 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.metadata; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.MetadataStoreProvider; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.apache.zookeeper.MockZooKeeper; +import org.apache.zookeeper.MockZooKeeperSession; + +public class MockZooKeeperMetadataStoreProvider implements MetadataStoreProvider { + private static final String MOCK_ZK_SCHEME = "mock-zk"; + private static final ConcurrentMap mockZooKeepers = new ConcurrentHashMap<>(); + + @Override + public String urlScheme() { + return MOCK_ZK_SCHEME; + } + + @Override + public MetadataStore create(String metadataURL, MetadataStoreConfig metadataStoreConfig, + boolean enableSessionWatcher) throws MetadataStoreException { + MockZooKeeper mockZooKeeper = mockZooKeepers.computeIfAbsent(metadataURL, + k -> MockZooKeeper.newInstance().registerCloseable(() -> mockZooKeepers.remove(k))); + MockZooKeeperSession mockZooKeeperSession = MockZooKeeperSession.newInstance(mockZooKeeper, true); + ZKMetadataStore zkMetadataStore = new ZKMetadataStore(mockZooKeeperSession, metadataStoreConfig, true); + return zkMetadataStore; + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java index 6ede02b67136e..a0f726ed3dc9d 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java @@ -21,6 +21,10 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import io.opentelemetry.api.OpenTelemetry; +import java.util.EnumSet; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; import lombok.Cleanup; import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataStore; @@ -32,26 +36,25 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.util.EnumSet; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; public class MetadataStoreFactoryImplTest { - - private static Object originalProperty; + private static String originalMetadatastoreProvidersPropertyValue; @BeforeClass public void setMetadataStoreProperty() { - originalProperty = System.getProperties().get(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY); + originalMetadatastoreProvidersPropertyValue = + System.getProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY); System.setProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY, MyMetadataStoreProvider.class.getName()); } @AfterClass public void resetMetadataStoreProperty() { - if (originalProperty != null) { - System.getProperties().put(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY, originalProperty); + if (originalMetadatastoreProvidersPropertyValue != null) { + System.setProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY, + originalMetadatastoreProvidersPropertyValue); + } else { + System.clearProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY); } } diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 7b23d0814aab9..ebcb63204229a 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -94,6 +94,8 @@ static MockZNode of(byte[] content, int version, long ephemeralOwner) { private AtomicLong sequentialIdGenerator; private ThreadLocal ephemeralOwnerThreadLocal; + private int referenceCount; + private List closeables; //see details of Objenesis caching - http://objenesis.org/details.html //see supported jvms - https://github.com/easymock/objenesis/blob/master/SupportedJVMs.md @@ -165,6 +167,7 @@ private static MockZooKeeper createMockZooKeeperInstance(ExecutorService executo zk.mutex = new ReentrantLock(); zk.lockInstance = ThreadLocal.withInitial(zk::createLock); zk.sequentialIdGenerator = new AtomicLong(); + zk.closeables = new ArrayList<>(); return zk; } @@ -1085,9 +1088,28 @@ public void addWatch(String basePath, Watcher watcher, AddWatchMode mode, VoidCa }); } + public synchronized void increaseRefCount() { + referenceCount++; + } + + public synchronized MockZooKeeper registerCloseable(AutoCloseable closeable) { + closeables.add(closeable); + return this; + } + @Override - public void close() throws InterruptedException { - shutdown(); + public synchronized void close() throws InterruptedException { + if (--referenceCount <= 0) { + shutdown(); + closeables.forEach(c -> { + try { + c.close(); + } catch (Exception e) { + log.error("Error closing closeable", e); + } + }); + closeables.clear(); + } } public void shutdown() throws InterruptedException { diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java index a75402018ac04..3479aac9717bc 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java @@ -40,7 +40,7 @@ public class MockZooKeeperSession extends ZooKeeper { private MockZooKeeper mockZooKeeper; - private long sessionId = 0L; + private long sessionId = 1L; private static final Objenesis objenesis = new ObjenesisStd(); @@ -59,6 +59,9 @@ public static MockZooKeeperSession newInstance(MockZooKeeper mockZooKeeper, bool mockZooKeeperSession.mockZooKeeper = mockZooKeeper; mockZooKeeperSession.sessionId = sessionIdGenerator.getAndIncrement(); mockZooKeeperSession.closeMockZooKeeperOnClose = closeMockZooKeeperOnClose; + if (closeMockZooKeeperOnClose) { + mockZooKeeper.increaseRefCount(); + } return mockZooKeeperSession; } From cd5610ab89e13a89f61d5effa1af2cd26aad1130 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 13:44:21 +0200 Subject: [PATCH 03/56] Improve test names --- .../metadata/BaseMetadataStoreTest.java | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index c77fba9ceeafb..938615052a7c0 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -147,7 +147,26 @@ private synchronized String getEtcdClusterConnectString() { } public static Supplier stringSupplier(Supplier supplier) { - return supplier; + return new StringSupplier(supplier); + } + + // Implements toString() so that the test name is more descriptive + private static class StringSupplier implements Supplier { + private final Supplier supplier; + + public StringSupplier(Supplier supplier) { + this.supplier = supplier; + } + + @Override + public String get() { + return supplier.get(); + } + + @Override + public String toString() { + return get(); + } } protected String newKey() { From 244a65da5c0133945e6037785b904c8dd75dcd71 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 14:39:44 +0200 Subject: [PATCH 04/56] Fix Stat handling in MockZooKeeper --- .../org/apache/zookeeper/MockZooKeeper.java | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index ebcb63204229a..1df3bde4679e2 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -444,9 +444,11 @@ public void getData(final String path, boolean watch, final DataCallback cb, fin } MockZNode value; + Stat stat; lock(); try { value = tree.get(path); + stat = createStatForZNode(value); } finally { unlockIfLocked(); } @@ -454,7 +456,7 @@ public void getData(final String path, boolean watch, final DataCallback cb, fin if (value == null) { cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null, null); } else { - cb.processResult(0, path, ctx, value.getContent(), createStatForZNode(value)); + cb.processResult(0, path, ctx, value.getContent(), stat); } } catch (Throwable ex) { log.error("get data : {} error", path, ex); @@ -488,7 +490,6 @@ public void getData(final String path, final Watcher watcher, final DataCallback if (watcher != null) { watchers.put(path, watcher); } - Stat stat = createStatForZNode(value); unlockIfLocked(); cb.processResult(0, path, ctx, value.getContent(), stat); @@ -622,6 +623,8 @@ public void getChildren(final String path, boolean watcher, final Children2Callb Set children = new TreeSet<>(); try { lock(); + MockZNode mockZNode = tree.get(path); + Stat stat = mockZNode != null ? createStatForZNode(mockZNode) : null; Optional failure = programmedFailure(Op.GET_CHILDREN, path); if (failure.isPresent()) { unlockIfLocked(); @@ -631,7 +634,7 @@ public void getChildren(final String path, boolean watcher, final Children2Callb unlockIfLocked(); cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null, null); return; - } else if (!tree.containsKey(path)) { + } else if (mockZNode == null) { unlockIfLocked(); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null, null); return; @@ -642,17 +645,15 @@ public void getChildren(final String path, boolean watcher, final Children2Callb tree.subMap(firstKey, false, lastKey, false).forEach((key, value) -> { String relativePath = key.replace(firstKey, ""); - // Only return first-level children String child = relativePath.split("/", 2)[0]; children.add(child); }); - cb.processResult(0, path, ctx, new ArrayList<>(children), new Stat()); + unlockIfLocked(); + cb.processResult(0, path, ctx, new ArrayList<>(children), stat); } catch (Throwable ex) { log.error("get children : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); - } finally { - unlockIfLocked(); } }); @@ -737,9 +738,11 @@ public void exists(String path, Watcher watcher, StatCallback cb, Object ctx) { watchers.put(path, watcher); } - if (tree.containsKey(path)) { + MockZNode mockZNode = tree.get(path); + if (mockZNode != null) { + Stat stat = createStatForZNode(mockZNode); unlockIfLocked(); - cb.processResult(0, path, ctx, new Stat()); + cb.processResult(0, path, ctx, stat); } else { unlockIfLocked(); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); @@ -825,8 +828,8 @@ public void setData(final String path, final byte[] data, int version, final Sta executor.execute(() -> { try { final Set toNotify = Sets.newHashSet(); - Stat stat; lock(); + Stat stat; try { Optional failure = programmedFailure(Op.SET, path); if (failure.isPresent()) { @@ -851,8 +854,9 @@ public void setData(final String path, final byte[] data, int version, final Sta // Check version if (version != -1 && version != currentVersion) { log.debug("[{}] Current version: {} -- Expected: {}", path, currentVersion, version); + Stat currentStat = createStatForZNode(mockZNode); unlockIfLocked(); - cb.processResult(KeeperException.Code.BADVERSION.intValue(), path, ctx, null); + cb.processResult(KeeperException.Code.BADVERSION.intValue(), path, ctx, currentStat); return; } From d9ec2d16cc930a8b2e3eceb3f8635a2a02a4cb89 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 15:27:37 +0200 Subject: [PATCH 05/56] Cleanup watchers tied to a particular ZK session in MockZooKeeper --- .../org/apache/zookeeper/MockZooKeeper.java | 143 +++++++------ .../zookeeper/MockZooKeeperSession.java | 188 ++++++++++++++---- 2 files changed, 233 insertions(+), 98 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 1df3bde4679e2..34625692c7226 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -27,7 +27,10 @@ import com.google.common.collect.Sets; import io.netty.util.concurrent.DefaultThreadFactory; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.TreeMap; @@ -80,7 +83,7 @@ static MockZNode of(byte[] content, int version, long ephemeralOwner) { } private TreeMap tree; - private SetMultimap watchers; + private SetMultimap watchers; private volatile boolean stopped; private AtomicReference alwaysFail; private CopyOnWriteArrayList failures; @@ -93,7 +96,7 @@ static MockZNode of(byte[] content, int version, long ephemeralOwner) { private ReentrantLock mutex; private AtomicLong sequentialIdGenerator; - private ThreadLocal ephemeralOwnerThreadLocal; + private ThreadLocal overriddenSessionIdThreadLocal; private int referenceCount; private List closeables; @@ -115,12 +118,10 @@ private static class Failure { } } - @Data - @AllArgsConstructor - private static class PersistentWatcher { - final String path; - final Watcher watcher; - final AddWatchMode mode; + private record PersistentWatcher(String path, Watcher watcher, AddWatchMode mode, long sessionId) { + } + + private record NodeWatcher(Watcher watcher, long sessionId) { } private List persistentWatchers; @@ -161,7 +162,7 @@ private static MockZooKeeper createMockZooKeeperInstance(ExecutorService executo ObjectInstantiator mockZooKeeperInstantiator = objenesis.getInstantiatorOf(MockZooKeeper.class); MockZooKeeper zk = mockZooKeeperInstantiator.newInstance(); - zk.ephemeralOwnerThreadLocal = new ThreadLocal<>(); + zk.overriddenSessionIdThreadLocal = new ThreadLocal<>(); zk.init(executor); zk.readOpDelayMs = readOpDelayMs; zk.mutex = new ReentrantLock(); @@ -178,7 +179,7 @@ private void init(ExecutorService executor) { } else { this.executor = Executors.newFixedThreadPool(1, new DefaultThreadFactory("mock-zookeeper")); } - SetMultimap w = HashMultimap.create(); + SetMultimap w = HashMultimap.create(); watchers = Multimaps.synchronizedSetMultimap(w); stopped = false; alwaysFail = new AtomicReference<>(KeeperException.Code.OK); @@ -258,8 +259,6 @@ public String create(String path, byte[] data, List acl, CreateMode createM lock(); try { - - maybeThrowProgrammedFailure(Op.CREATE, path); if (stopped) { @@ -284,12 +283,11 @@ public String create(String path, byte[] data, List acl, CreateMode createM MockZNode.of(parentNode.getContent(), parentVersion + 1, parentNode.getEphemeralOwner())); } - tree.put(path, MockZNode.of(data, 0, createMode.isEphemeral() ? getEphemeralOwner() : NOT_EPHEMERAL)); - - toNotifyCreate.addAll(watchers.get(path)); + tree.put(path, MockZNode.of(data, 0, createMode.isEphemeral() ? getSessionId() : NOT_EPHEMERAL)); + toNotifyCreate.addAll(getWatchers(path)); if (!parent.isEmpty()) { - toNotifyParent.addAll(watchers.get(parent)); + toNotifyParent.addAll(getWatchers(parent)); } watchers.removeAll(path); } finally { @@ -315,20 +313,30 @@ public String create(String path, byte[] data, List acl, CreateMode createM return path; } - protected long getEphemeralOwner() { - Long ephemeralOwner = ephemeralOwnerThreadLocal.get(); - if (ephemeralOwner != null) { - return ephemeralOwner; + private Collection getWatchers(String path) { + Set nodeWatchers = watchers.get(path); + if (nodeWatchers != null) { + return nodeWatchers.stream().map(NodeWatcher::watcher).toList(); + } else { + return Collections.emptyList(); + } + } + + @Override + public long getSessionId() { + Long overriddenSessionId = overriddenSessionIdThreadLocal.get(); + if (overriddenSessionId != null) { + return overriddenSessionId; } - return getSessionId(); + return sessionId; } - public void overrideEphemeralOwner(long ephemeralOwner) { - ephemeralOwnerThreadLocal.set(ephemeralOwner); + public void overrideSessionId(long sessionId) { + overriddenSessionIdThreadLocal.set(sessionId); } - public void removeEphemeralOwnerOverride() { - ephemeralOwnerThreadLocal.remove(); + public void removeSessionIdOverride() { + overriddenSessionIdThreadLocal.remove(); } @Override @@ -346,12 +354,12 @@ public void create(final String path, final byte[] data, final List acl, Cr } final Set toNotifyCreate = Sets.newHashSet(); - toNotifyCreate.addAll(watchers.get(path)); + toNotifyCreate.addAll(getWatchers(path)); final Set toNotifyParent = Sets.newHashSet(); final String parent = path.substring(0, path.lastIndexOf("/")); if (!parent.isEmpty()) { - toNotifyParent.addAll(watchers.get(parent)); + toNotifyParent.addAll(getWatchers(parent)); } final String name; @@ -379,7 +387,7 @@ public void create(final String path, final byte[] data, final List acl, Cr cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); } else { tree.put(name, MockZNode.of(data, 0, - createMode != null && createMode.isEphemeral() ? getEphemeralOwner() : NOT_EPHEMERAL)); + createMode != null && createMode.isEphemeral() ? getSessionId() : NOT_EPHEMERAL)); watchers.removeAll(name); unlockIfLocked(); cb.processResult(0, path, ctx, name); @@ -417,7 +425,7 @@ public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperExce throw new KeeperException.NoNodeException(path); } else { if (watcher != null) { - watchers.put(path, watcher); + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } if (stat != null) { applyToStat(value, stat); @@ -488,7 +496,7 @@ public void getData(final String path, final Watcher watcher, final DataCallback cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null, null); } else { if (watcher != null) { - watchers.put(path, watcher); + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } Stat stat = createStatForZNode(value); unlockIfLocked(); @@ -542,7 +550,7 @@ public void getChildren(final String path, final Watcher watcher, final Children } if (watcher != null) { - watchers.put(path, watcher); + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } cb.processResult(0, path, ctx, children); } catch (Throwable ex) { @@ -578,7 +586,7 @@ public List getChildren(String path, Watcher watcher) throws KeeperExcep }); if (watcher != null) { - watchers.put(path, watcher); + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } return new ArrayList<>(children); @@ -700,7 +708,7 @@ public Stat exists(String path, Watcher watcher) throws KeeperException, Interru } if (watcher != null) { - watchers.put(path, watcher); + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } if (tree.containsKey(path)) { @@ -735,7 +743,7 @@ public void exists(String path, Watcher watcher, StatCallback cb, Object ctx) { } if (watcher != null) { - watchers.put(path, watcher); + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } MockZNode mockZNode = tree.get(path); @@ -802,7 +810,7 @@ public Stat setData(final String path, byte[] data, int version) throws KeeperEx newZNode = MockZNode.of(data, currentVersion + 1, mockZNode.getEphemeralOwner()); tree.put(path, newZNode); - toNotify.addAll(watchers.get(path)); + toNotify.addAll(getWatchers(path)); watchers.removeAll(path); } finally { unlockIfLocked(); @@ -869,7 +877,7 @@ public void setData(final String path, final byte[] data, int version, final Sta } cb.processResult(0, path, ctx, stat); - toNotify.addAll(watchers.get(path)); + toNotify.addAll(getWatchers(path)); watchers.removeAll(path); for (Watcher watcher : toNotify) { @@ -912,12 +920,12 @@ public void delete(final String path, int version) throws InterruptedException, tree.remove(path); toNotifyDelete = Sets.newHashSet(); - toNotifyDelete.addAll(watchers.get(path)); + toNotifyDelete.addAll(getWatchers(path)); toNotifyParent = Sets.newHashSet(); parent = path.substring(0, path.lastIndexOf("/")); if (!parent.isEmpty()) { - toNotifyParent.addAll(watchers.get(parent)); + toNotifyParent.addAll(getWatchers(parent)); } watchers.removeAll(path); @@ -947,12 +955,12 @@ public void delete(final String path, int version, final VoidCallback cb, final try { lock(); final Set toNotifyDelete = Sets.newHashSet(); - toNotifyDelete.addAll(watchers.get(path)); + toNotifyDelete.addAll(getWatchers(path)); final Set toNotifyParent = Sets.newHashSet(); - final String parent = path.substring(0, path.lastIndexOf("/")); + final String parent = path.substring(0, path.lastIndexOf('/')); if (!parent.isEmpty()) { - toNotifyParent.addAll(watchers.get(parent)); + toNotifyParent.addAll(getWatchers(parent)); } watchers.removeAll(path); @@ -1026,15 +1034,15 @@ public List multi(Iterable ops) throws Interr case ZooDefs.OpCode.create -> { org.apache.zookeeper.Op.Create opc = ((org.apache.zookeeper.Op.Create) op); CreateMode cm = CreateMode.fromFlag(opc.flags); - String path = this.create(op.getPath(), opc.data, null, cm); + String path = create(op.getPath(), opc.data, null, cm); res.add(new OpResult.CreateResult(path)); } case ZooDefs.OpCode.delete -> { - this.delete(op.getPath(), (int) FieldUtils.readField(op, "version", true)); + delete(op.getPath(), (int) FieldUtils.readField(op, "version", true)); res.add(new OpResult.DeleteResult()); } case ZooDefs.OpCode.setData -> { - Stat stat = this.setData( + Stat stat = setData( op.getPath(), (byte[]) FieldUtils.readField(op, "data", true), (int) FieldUtils.readField(op, "version", true)); @@ -1042,7 +1050,7 @@ public List multi(Iterable ops) throws Interr } case ZooDefs.OpCode.getChildren -> { try { - List children = this.getChildren(op.getPath(), null); + List children = getChildren(op.getPath(), null); res.add(new OpResult.GetChildrenResult(children)); } catch (KeeperException e) { res.add(new OpResult.ErrorResult(e.code().intValue())); @@ -1051,7 +1059,7 @@ public List multi(Iterable ops) throws Interr case ZooDefs.OpCode.getData -> { Stat stat = new Stat(); try { - byte[] payload = this.getData(op.getPath(), null, stat); + byte[] payload = getData(op.getPath(), null, stat); res.add(new OpResult.GetDataResult(payload, stat)); } catch (KeeperException e) { res.add(new OpResult.ErrorResult(e.code().intValue())); @@ -1073,7 +1081,7 @@ public List multi(Iterable ops) throws Interr @Override public synchronized void addWatch(String basePath, Watcher watcher, AddWatchMode mode) { - persistentWatchers.add(new PersistentWatcher(basePath, watcher, mode)); + persistentWatchers.add(new PersistentWatcher(basePath, watcher, mode, getSessionId())); } @Override @@ -1082,12 +1090,11 @@ public void addWatch(String basePath, Watcher watcher, AddWatchMode mode, VoidCa cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), basePath, ctx); return; } - + long currentSessionId = getSessionId(); executor.execute(() -> { synchronized (MockZooKeeper.this) { - persistentWatchers.add(new PersistentWatcher(basePath, watcher, mode)); + persistentWatchers.add(new PersistentWatcher(basePath, watcher, mode, currentSessionId)); } - cb.processResult(KeeperException.Code.OK.intValue(), basePath, ctx); }); } @@ -1134,7 +1141,7 @@ public void shutdown() throws InterruptedException { } Optional programmedFailure(Op op, String path) { - KeeperException.Code error = this.alwaysFail.get(); + KeeperException.Code error = alwaysFail.get(); if (error != KeeperException.Code.OK) { return Optional.of(error); } @@ -1171,22 +1178,17 @@ public void delay(long millis, BiPredicate predicate) { } public void setAlwaysFail(KeeperException.Code rc) { - this.alwaysFail.set(rc); + alwaysFail.set(rc); } public void unsetAlwaysFail() { - this.alwaysFail.set(KeeperException.Code.OK); + alwaysFail.set(KeeperException.Code.OK); } public void setSessionId(long id) { sessionId = id; } - @Override - public long getSessionId() { - return sessionId; - } - private boolean hasChildren(String path) { return !tree.subMap(path + '/', path + '0').isEmpty(); } @@ -1207,13 +1209,13 @@ private void checkReadOpDelay() { } private void triggerPersistentWatches(String path, String parent, EventType eventType) { - persistentWatchers.forEach(w -> { + getPersistentWatchersCopy().forEach(w -> { if (w.mode == AddWatchMode.PERSISTENT_RECURSIVE) { - if (path.startsWith(w.getPath())) { + if (path.startsWith(w.path())) { w.watcher.process(new WatchedEvent(eventType, KeeperState.SyncConnected, path)); } } else if (w.mode == AddWatchMode.PERSISTENT) { - if (w.getPath().equals(path)) { + if (w.path().equals(path)) { w.watcher.process(new WatchedEvent(eventType, KeeperState.SyncConnected, path)); } @@ -1226,6 +1228,10 @@ private void triggerPersistentWatches(String path, String parent, EventType even }); } + private synchronized List getPersistentWatchersCopy() { + return new ArrayList<>(persistentWatchers); + } + public void deleteEphemeralNodes(long sessionId) { if (sessionId != NOT_EPHEMERAL) { lock(); @@ -1237,5 +1243,20 @@ public void deleteEphemeralNodes(long sessionId) { } } + public synchronized void deleteWatchers(long sessionId) { + lock(); + try { + // remove all persistent watchers for the session + persistentWatchers.removeIf(w -> w.sessionId == sessionId); + // remove all watchers for the session + List> watchersForSession = + watchers.entries().stream().filter(e -> e.getValue().sessionId == sessionId).toList(); + watchersForSession + .forEach(e -> watchers.remove(e.getKey(), e.getValue())); + } finally { + unlockIfLocked(); + } + } + private static final Logger log = LoggerFactory.getLogger(MockZooKeeper.class); } diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java index 3479aac9717bc..e2f267ff566b4 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java @@ -84,17 +84,22 @@ public States getState() { @Override public void register(Watcher watcher) { - mockZooKeeper.register(watcher); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.register(watcher); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public String create(String path, byte[] data, List acl, CreateMode createMode) throws KeeperException, InterruptedException { try { - mockZooKeeper.overrideEphemeralOwner(getSessionId()); + mockZooKeeper.overrideSessionId(getSessionId()); return mockZooKeeper.create(path, data, acl, createMode); } finally { - mockZooKeeper.removeEphemeralOwnerOverride(); + mockZooKeeper.removeSessionIdOverride(); } } @@ -102,148 +107,257 @@ public String create(String path, byte[] data, List acl, CreateMode createM public void create(final String path, final byte[] data, final List acl, CreateMode createMode, final AsyncCallback.StringCallback cb, final Object ctx) { try { - mockZooKeeper.overrideEphemeralOwner(getSessionId()); + mockZooKeeper.overrideSessionId(getSessionId()); mockZooKeeper.create(path, data, acl, createMode, cb, ctx); } finally { - mockZooKeeper.removeEphemeralOwnerOverride(); + mockZooKeeper.removeSessionIdOverride(); } } @Override public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException { - return mockZooKeeper.getData(path, watcher, stat); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.getData(path, watcher, stat); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void getData(final String path, boolean watch, final DataCallback cb, final Object ctx) { - mockZooKeeper.getData(path, watch, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.getData(path, watch, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void getData(final String path, final Watcher watcher, final DataCallback cb, final Object ctx) { - mockZooKeeper.getData(path, watcher, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.getData(path, watcher, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void getChildren(final String path, final Watcher watcher, final ChildrenCallback cb, final Object ctx) { - mockZooKeeper.getChildren(path, watcher, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.getChildren(path, watcher, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public List getChildren(String path, Watcher watcher) throws KeeperException { - return mockZooKeeper.getChildren(path, watcher); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.getChildren(path, watcher); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public List getChildren(String path, boolean watch) throws KeeperException, InterruptedException { - return mockZooKeeper.getChildren(path, watch); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.getChildren(path, watch); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void getChildren(final String path, boolean watcher, final Children2Callback cb, final Object ctx) { - mockZooKeeper.getChildren(path, watcher, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.getChildren(path, watcher, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException { - return mockZooKeeper.exists(path, watch); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.exists(path, watch); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException { - return mockZooKeeper.exists(path, watcher); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.exists(path, watcher); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void exists(String path, boolean watch, StatCallback cb, Object ctx) { - mockZooKeeper.exists(path, watch, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.exists(path, watch, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void exists(String path, Watcher watcher, StatCallback cb, Object ctx) { - mockZooKeeper.exists(path, watcher, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.exists(path, watcher, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void sync(String path, VoidCallback cb, Object ctx) { - mockZooKeeper.sync(path, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.sync(path, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public Stat setData(final String path, byte[] data, int version) throws KeeperException, InterruptedException { - return mockZooKeeper.setData(path, data, version); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + return mockZooKeeper.setData(path, data, version); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void setData(final String path, final byte[] data, int version, final StatCallback cb, final Object ctx) { - mockZooKeeper.setData(path, data, version, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.setData(path, data, version, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void delete(final String path, int version) throws InterruptedException, KeeperException { - mockZooKeeper.delete(path, version); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.delete(path, version); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void delete(final String path, int version, final VoidCallback cb, final Object ctx) { - mockZooKeeper.delete(path, version, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.delete(path, version, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void multi(Iterable ops, AsyncCallback.MultiCallback cb, Object ctx) { try { - mockZooKeeper.overrideEphemeralOwner(getSessionId()); + mockZooKeeper.overrideSessionId(getSessionId()); mockZooKeeper.multi(ops, cb, ctx); } finally { - mockZooKeeper.removeEphemeralOwnerOverride(); + mockZooKeeper.removeSessionIdOverride(); } } @Override public List multi(Iterable ops) throws InterruptedException, KeeperException { try { - mockZooKeeper.overrideEphemeralOwner(getSessionId()); + mockZooKeeper.overrideSessionId(getSessionId()); return mockZooKeeper.multi(ops); } finally { - mockZooKeeper.removeEphemeralOwnerOverride(); + mockZooKeeper.removeSessionIdOverride(); } } @Override public void addWatch(String basePath, Watcher watcher, AddWatchMode mode, VoidCallback cb, Object ctx) { - mockZooKeeper.addWatch(basePath, watcher, mode, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.addWatch(basePath, watcher, mode, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void addWatch(String basePath, Watcher watcher, AddWatchMode mode) throws KeeperException, InterruptedException { - mockZooKeeper.addWatch(basePath, watcher, mode); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.addWatch(basePath, watcher, mode); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void addWatch(String basePath, AddWatchMode mode) throws KeeperException, InterruptedException { - mockZooKeeper.addWatch(basePath, mode); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.addWatch(basePath, mode); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void addWatch(String basePath, AddWatchMode mode, VoidCallback cb, Object ctx) { - mockZooKeeper.addWatch(basePath, mode, cb, ctx); + try { + mockZooKeeper.overrideSessionId(getSessionId()); + mockZooKeeper.addWatch(basePath, mode, cb, ctx); + } finally { + mockZooKeeper.removeSessionIdOverride(); + } } @Override public void close() throws InterruptedException { - if (closeMockZooKeeperOnClose) { - mockZooKeeper.close(); - } else { - mockZooKeeper.deleteEphemeralNodes(getSessionId()); - } + internalClose(false); } public void shutdown() throws InterruptedException { - if (closeMockZooKeeperOnClose) { - mockZooKeeper.shutdown(); - } else { + internalClose(true); + } + + private void internalClose(boolean shutdown) throws InterruptedException { + try { + mockZooKeeper.overrideSessionId(getSessionId()); mockZooKeeper.deleteEphemeralNodes(getSessionId()); + mockZooKeeper.deleteWatchers(getSessionId()); + if (closeMockZooKeeperOnClose) { + if (shutdown) { + mockZooKeeper.shutdown(); + } else { + mockZooKeeper.close(); + } + } + } finally { + mockZooKeeper.removeSessionIdOverride(); } } From 153289ebe15cf73019731af71b86f5d644d36deb Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 15:42:44 +0200 Subject: [PATCH 06/56] Improve MockZNode handling --- .../org/apache/zookeeper/MockZooKeeper.java | 97 ++++++++++++------- 1 file changed, 60 insertions(+), 37 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 34625692c7226..b3616399da705 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -47,7 +47,6 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiPredicate; import lombok.AllArgsConstructor; -import lombok.Data; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.zookeeper.AsyncCallback.Children2Callback; @@ -70,15 +69,51 @@ public class MockZooKeeper extends ZooKeeper { // ephemeralOwner value for persistent nodes private static final long NOT_EPHEMERAL = 0L; - @Data @AllArgsConstructor private static class MockZNode { byte[] content; int version; long ephemeralOwner; + long creationTimestamp; + long modificationTimestamp; static MockZNode of(byte[] content, int version, long ephemeralOwner) { - return new MockZNode(content, version, ephemeralOwner); + return new MockZNode(content, version, ephemeralOwner, System.currentTimeMillis(), + System.currentTimeMillis()); + } + + public synchronized void updateVersion() { + version++; + modificationTimestamp = System.currentTimeMillis(); + } + + public synchronized void updateData(byte[] data) { + content = data; + updateVersion(); + } + + public Stat getStat() { + return applyToStat(new Stat()); + } + + public synchronized Stat applyToStat(Stat stat) { + stat.setCtime(creationTimestamp); + stat.setMtime(modificationTimestamp); + stat.setVersion(version); + stat.setEphemeralOwner(ephemeralOwner); + return stat; + } + + public synchronized int getVersion() { + return version; + } + + public synchronized byte[] getContent() { + return content; + } + + public long getEphemeralOwner() { + return ephemeralOwner; } } @@ -277,13 +312,10 @@ public String create(String path, byte[] data, List acl, CreateMode createM MockZNode parentNode = tree.get(parent); int parentVersion = tree.get(parent).getVersion(); path = path + parentVersion; - - // Update parent version - tree.put(parent, - MockZNode.of(parentNode.getContent(), parentVersion + 1, parentNode.getEphemeralOwner())); + parentNode.updateVersion(); } - tree.put(path, MockZNode.of(data, 0, createMode.isEphemeral() ? getSessionId() : NOT_EPHEMERAL)); + tree.put(path, createMockZNode(data, createMode)); toNotifyCreate.addAll(getWatchers(path)); if (!parent.isEmpty()) { @@ -386,8 +418,7 @@ public void create(final String path, final byte[] data, final List acl, Cr parent))); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); } else { - tree.put(name, MockZNode.of(data, 0, - createMode != null && createMode.isEphemeral() ? getSessionId() : NOT_EPHEMERAL)); + tree.put(name, createMockZNode(data, createMode)); watchers.removeAll(name); unlockIfLocked(); cb.processResult(0, path, ctx, name); @@ -415,6 +446,11 @@ public void create(final String path, final byte[] data, final List acl, Cr } + private MockZNode createMockZNode(byte[] data, CreateMode createMode) { + return MockZNode.of(data, 0, + createMode != null && createMode.isEphemeral() ? getSessionId() : NOT_EPHEMERAL); + } + @Override public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException { lock(); @@ -428,7 +464,7 @@ public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperExce watchers.put(path, new NodeWatcher(watcher, getSessionId())); } if (stat != null) { - applyToStat(value, stat); + value.applyToStat(stat); } return value.getContent(); } @@ -456,7 +492,7 @@ public void getData(final String path, boolean watch, final DataCallback cb, fin lock(); try { value = tree.get(path); - stat = createStatForZNode(value); + stat = value.getStat(); } finally { unlockIfLocked(); } @@ -498,7 +534,7 @@ public void getData(final String path, final Watcher watcher, final DataCallback if (watcher != null) { watchers.put(path, new NodeWatcher(watcher, getSessionId())); } - Stat stat = createStatForZNode(value); + Stat stat = value.getStat(); unlockIfLocked(); cb.processResult(0, path, ctx, value.getContent(), stat); } @@ -632,7 +668,7 @@ public void getChildren(final String path, boolean watcher, final Children2Callb try { lock(); MockZNode mockZNode = tree.get(path); - Stat stat = mockZNode != null ? createStatForZNode(mockZNode) : null; + Stat stat = mockZNode != null ? mockZNode.getStat() : null; Optional failure = programmedFailure(Op.GET_CHILDREN, path); if (failure.isPresent()) { unlockIfLocked(); @@ -678,7 +714,7 @@ public Stat exists(String path, boolean watch) throws KeeperException, Interrupt } if (tree.containsKey(path)) { - return createStatForZNode(tree.get(path)); + return tree.get(path).getStat(); } else { return null; } @@ -687,16 +723,6 @@ public Stat exists(String path, boolean watch) throws KeeperException, Interrupt } } - private static Stat createStatForZNode(MockZNode zNode) { - return applyToStat(zNode, new Stat()); - } - - private static Stat applyToStat(MockZNode zNode, Stat stat) { - stat.setVersion(zNode.getVersion()); - stat.setEphemeralOwner(zNode.getEphemeralOwner()); - return stat; - } - @Override public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException { lock(); @@ -712,7 +738,7 @@ public Stat exists(String path, Watcher watcher) throws KeeperException, Interru } if (tree.containsKey(path)) { - return createStatForZNode(tree.get(path)); + return tree.get(path).getStat(); } else { return null; } @@ -748,7 +774,7 @@ public void exists(String path, Watcher watcher, StatCallback cb, Object ctx) { MockZNode mockZNode = tree.get(path); if (mockZNode != null) { - Stat stat = createStatForZNode(mockZNode); + Stat stat = mockZNode.getStat(); unlockIfLocked(); cb.processResult(0, path, ctx, stat); } else { @@ -784,9 +810,8 @@ public void sync(String path, VoidCallback cb, Object ctx) { @Override public Stat setData(final String path, byte[] data, int version) throws KeeperException, InterruptedException { final Set toNotify = Sets.newHashSet(); - MockZNode newZNode; - lock(); + Stat stat; try { maybeThrowProgrammedFailure(Op.SET, path); @@ -807,9 +832,8 @@ public Stat setData(final String path, byte[] data, int version) throws KeeperEx } log.debug("[{}] Updating -- current version: {}", path, currentVersion); - newZNode = MockZNode.of(data, currentVersion + 1, mockZNode.getEphemeralOwner()); - tree.put(path, newZNode); - + mockZNode.updateData(data); + stat = mockZNode.getStat(); toNotify.addAll(getWatchers(path)); watchers.removeAll(path); } finally { @@ -823,7 +847,7 @@ public Stat setData(final String path, byte[] data, int version) throws KeeperEx .process(new WatchedEvent(EventType.NodeDataChanged, KeeperState.SyncConnected, path))); }); - return createStatForZNode(newZNode); + return stat; } @Override @@ -862,16 +886,15 @@ public void setData(final String path, final byte[] data, int version, final Sta // Check version if (version != -1 && version != currentVersion) { log.debug("[{}] Current version: {} -- Expected: {}", path, currentVersion, version); - Stat currentStat = createStatForZNode(mockZNode); + Stat currentStat = mockZNode.getStat(); unlockIfLocked(); cb.processResult(KeeperException.Code.BADVERSION.intValue(), path, ctx, currentStat); return; } log.debug("[{}] Updating -- current version: {}", path, currentVersion); - MockZNode newZNode = MockZNode.of(data, currentVersion + 1, mockZNode.getEphemeralOwner()); - tree.put(path, newZNode); - stat = createStatForZNode(newZNode); + mockZNode.updateData(data); + stat = mockZNode.getStat(); } finally { unlockIfLocked(); } From 69ca2df5f29c351e177e46627619f401d80698a5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 15:58:52 +0200 Subject: [PATCH 07/56] Pass the overridden session id to the executor thread --- .../org/apache/zookeeper/MockZooKeeper.java | 53 +++++++++++++++---- 1 file changed, 42 insertions(+), 11 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index b3616399da705..bd9735e59d16d 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -375,8 +375,9 @@ public void removeSessionIdOverride() { public void create(final String path, final byte[] data, final List acl, CreateMode createMode, final StringCallback cb, final Object ctx) { - + long currentSessionId = getSessionId(); executor.execute(() -> { + overrideSessionId(currentSessionId); try { lock(); @@ -441,6 +442,7 @@ public void create(final String path, final byte[] data, final List acl, Cr cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); } finally { unlockIfLocked(); + removeSessionIdOverride(); } }); @@ -475,7 +477,9 @@ public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperExce @Override public void getData(final String path, boolean watch, final DataCallback cb, final Object ctx) { + long currentSessionId = getSessionId(); executor.execute(() -> { + overrideSessionId(currentSessionId); try { checkReadOpDelay(); Optional failure = programmedFailure(Op.GET, path); @@ -505,14 +509,18 @@ public void getData(final String path, boolean watch, final DataCallback cb, fin } catch (Throwable ex) { log.error("get data : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); + } finally { + removeSessionIdOverride(); } }); } @Override public void getData(final String path, final Watcher watcher, final DataCallback cb, final Object ctx) { + long currentSessionId = getSessionId(); executor.execute(() -> { checkReadOpDelay(); + overrideSessionId(currentSessionId); try { lock(); Optional failure = programmedFailure(Op.GET, path); @@ -543,13 +551,16 @@ public void getData(final String path, final Watcher watcher, final DataCallback cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); } finally { unlockIfLocked(); + removeSessionIdOverride(); } }); } @Override public void getChildren(final String path, final Watcher watcher, final ChildrenCallback cb, final Object ctx) { + long currentSessionId = getSessionId(); executor.execute(() -> { + overrideSessionId(currentSessionId); List children = Lists.newArrayList(); try { lock(); @@ -594,6 +605,7 @@ public void getChildren(final String path, final Watcher watcher, final Children cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); } finally { unlockIfLocked(); + removeSessionIdOverride(); } }); @@ -663,7 +675,9 @@ public List getChildren(String path, boolean watch) throws KeeperExcepti @Override public void getChildren(final String path, boolean watcher, final Children2Callback cb, final Object ctx) { + long currentSessionId = getSessionId(); executor.execute(() -> { + overrideSessionId(currentSessionId); Set children = new TreeSet<>(); try { lock(); @@ -698,6 +712,8 @@ public void getChildren(final String path, boolean watcher, final Children2Callb } catch (Throwable ex) { log.error("get children : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); + } finally { + removeSessionIdOverride(); } }); @@ -754,7 +770,9 @@ public void exists(String path, boolean watch, StatCallback cb, Object ctx) { @Override public void exists(String path, Watcher watcher, StatCallback cb, Object ctx) { + long currentSessionId = getSessionId(); executor.execute(() -> { + overrideSessionId(currentSessionId); try { lock(); Optional failure = programmedFailure(Op.EXISTS, path); @@ -786,23 +804,30 @@ public void exists(String path, Watcher watcher, StatCallback cb, Object ctx) { cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); } finally { unlockIfLocked(); + removeSessionIdOverride(); } }); } @Override public void sync(String path, VoidCallback cb, Object ctx) { + long currentSessionId = getSessionId(); executor.execute(() -> { - Optional failure = programmedFailure(Op.SYNC, path); - if (failure.isPresent()) { - cb.processResult(failure.get().intValue(), path, ctx); - return; - } else if (stopped) { - cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx); - return; - } + overrideSessionId(currentSessionId); + try { + Optional failure = programmedFailure(Op.SYNC, path); + if (failure.isPresent()) { + cb.processResult(failure.get().intValue(), path, ctx); + return; + } else if (stopped) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx); + return; + } - cb.processResult(0, path, ctx); + cb.processResult(0, path, ctx); + } finally { + removeSessionIdOverride(); + } }); } @@ -856,8 +881,9 @@ public void setData(final String path, final byte[] data, int version, final Sta cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); return; } - + long currentSessionId = getSessionId(); executor.execute(() -> { + overrideSessionId(currentSessionId); try { final Set toNotify = Sets.newHashSet(); lock(); @@ -911,6 +937,8 @@ public void setData(final String path, final byte[] data, int version, final Sta } catch (Throwable ex) { log.error("Update data : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); + } finally { + removeSessionIdOverride(); } }); } @@ -974,7 +1002,9 @@ public void delete(final String path, int version) throws InterruptedException, @Override public void delete(final String path, int version, final VoidCallback cb, final Object ctx) { + long currentSessionId = getSessionId(); Runnable r = () -> { + overrideSessionId(currentSessionId); try { lock(); final Set toNotifyDelete = Sets.newHashSet(); @@ -1027,6 +1057,7 @@ public void delete(final String path, int version, final VoidCallback cb, final cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx); } finally { unlockIfLocked(); + removeSessionIdOverride(); } }; From d60b9bced19a7307b86eba92f5bd0270707f3428 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 16:20:49 +0200 Subject: [PATCH 08/56] Don't use direct executor with MockZooKeeper since it could cause ordering problems --- .../apache/pulsar/broker/testcontext/PulsarTestContext.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java index 82a1b574ac921..de8b24edd7888 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.testcontext; -import com.google.common.util.concurrent.MoreExecutors; import io.netty.channel.EventLoopGroup; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; @@ -478,7 +477,7 @@ public Builder withMockZookeeper(boolean useSeparateGlobalZk) { } private MockZooKeeper createMockZooKeeper() throws Exception { - MockZooKeeper zk = MockZooKeeper.newInstance(MoreExecutors.newDirectExecutorService()); + MockZooKeeper zk = MockZooKeeper.newInstance(); initializeZookeeper(zk); registerCloseable(zk::shutdown); return zk; From 2298de6370c3e37abf001e118f2e4e6cd823304d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 16:28:51 +0200 Subject: [PATCH 09/56] Ensure that connection strings don't change during a single execution --- .../metadata/BaseMetadataStoreTest.java | 36 +++++++++++++------ 1 file changed, 26 insertions(+), 10 deletions(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index 938615052a7c0..66b9480fee254 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -48,12 +48,22 @@ public abstract class BaseMetadataStoreTest extends TestRetrySupport { private String mockZkUrl; // reference to keep the MockZooKeeper instance alive in MockZookeeperMetadataStoreProvider private MetadataStore mockZkStoreRef; + private String zksConnectionString; + private String memoryConnectionString; + private String rocksdbConnectionString; + private File rocksDbDirectory; + private boolean running; @BeforeClass(alwaysRun = true) @Override public void setup() throws Exception { + running = true; incrementSetupNumber(); zks = new TestZKServer(); + zksConnectionString = zks.getConnectionString(); + memoryConnectionString = "memory:" + UUID.randomUUID(); + rocksDbDirectory = Files.newTemporaryFolder().getAbsoluteFile(); + rocksdbConnectionString = "rocksdb:" + rocksDbDirectory; originalMetadatastoreProvidersPropertyValue = System.getProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY); // register MockZooKeeperMetadataStoreProvider @@ -67,6 +77,7 @@ public void setup() throws Exception { @AfterClass(alwaysRun = true) @Override public void cleanup() throws Exception { + running = false; markCurrentSetupNumberCleaned(); if (zks != null) { zks.close(); @@ -89,6 +100,11 @@ public void cleanup() throws Exception { mockZkUrl = null; } + if (rocksDbDirectory != null) { + Files.delete(rocksDbDirectory); + rocksDbDirectory = null; + } + if (originalMetadatastoreProvidersPropertyValue != null) { System.setProperty(MetadataStoreFactoryImpl.METADATASTORE_PROVIDERS_PROPERTY, originalMetadatastoreProvidersPropertyValue); @@ -97,12 +113,6 @@ public void cleanup() throws Exception { } } - private static String createTempFolder() { - File temp = Files.newTemporaryFolder(); - temp.deleteOnExit(); - return temp.getAbsolutePath(); - } - @DataProvider(name = "impl") public Object[][] implementations() { // A Supplier must be used for the Zookeeper connection string parameter. The retried test run will @@ -111,9 +121,9 @@ public Object[][] implementations() { // The new connection string won't be available to the test method unless a // Supplier lambda is used for providing the value. return new Object[][]{ - {"ZooKeeper", stringSupplier(() -> zks.getConnectionString())}, - {"Memory", stringSupplier(() -> "memory:" + UUID.randomUUID())}, - {"RocksDB", stringSupplier(() -> "rocksdb:" + createTempFolder())}, + {"ZooKeeper", stringSupplier(() -> zksConnectionString)}, + {"Memory", stringSupplier(() -> memoryConnectionString)}, + {"RocksDB", stringSupplier(() -> rocksdbConnectionString)}, {"Etcd", stringSupplier(() -> "etcd:" + getEtcdClusterConnectString())}, {"Oxia", stringSupplier(() -> "oxia://" + getOxiaServerConnectString())}, {"MockZooKeeper", stringSupplier(() -> mockZkUrl)}, @@ -123,13 +133,16 @@ public Object[][] implementations() { @DataProvider(name = "distributedImpl") public Object[][] distributedImplementations() { return new Object[][]{ - {"ZooKeeper", stringSupplier(() -> zks.getConnectionString())}, + {"ZooKeeper", stringSupplier(() -> zksConnectionString)}, {"Etcd", stringSupplier(() -> "etcd:" + getEtcdClusterConnectString())}, {"Oxia", stringSupplier(() -> "oxia://" + getOxiaServerConnectString())}, }; } protected synchronized String getOxiaServerConnectString() { + if (!running) { + return null; + } if (oxiaServer == null) { oxiaServer = new OxiaContainer(OxiaContainer.DEFAULT_IMAGE_NAME); oxiaServer.start(); @@ -138,6 +151,9 @@ protected synchronized String getOxiaServerConnectString() { } private synchronized String getEtcdClusterConnectString() { + if (!running) { + return null; + } if (etcdCluster == null) { etcdCluster = EtcdClusterExtension.builder().withClusterName("test").withNodes(1).withSsl(false).build() .cluster(); From 0f5c46f58effb439f60856341c5e8f88fac894c0 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 17:32:36 +0200 Subject: [PATCH 10/56] Improve checkContainers so that it's only called for ZK --- .../apache/pulsar/metadata/BaseMetadataStoreTest.java | 11 +++++++++++ .../java/org/apache/pulsar/metadata/CounterTest.java | 2 +- .../org/apache/pulsar/metadata/MetadataStoreTest.java | 8 ++++---- .../bookkeeper/PulsarLedgerIdGeneratorTest.java | 2 +- 4 files changed, 17 insertions(+), 6 deletions(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index 66b9480fee254..4c3e897cd51ea 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -228,4 +228,15 @@ public static boolean retryStrategically(Predicate predicate, int retryCou } return false; } + + /** + * Delete all the empty container nodes + * @param provider the metadata store provider + * @throws Exception + */ + protected void maybeTriggerDeletingEmptyContainers(String provider) throws Exception { + if ("ZooKeeper".equals(provider) && zks != null) { + zks.checkContainers(); + } + } } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java index c5b4012f0c8f9..f6d1d7553099b 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java @@ -85,7 +85,7 @@ public void testCounterDoesNotAutoReset(String provider, Supplier urlSup store1.close(); // Delete all the empty container nodes - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); @Cleanup MetadataStoreExtended store2 = MetadataStoreExtended.create(metadataUrl, MetadataStoreConfig.builder().build()); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java index 2c589dfd48222..27d40dd9ba8c2 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java @@ -432,18 +432,18 @@ public void testDeleteUnusedDirectories(String provider, Supplier urlSup store.delete(prefix + "/a1/b1/c1", Optional.empty()).join(); store.delete(prefix + "/a1/b1/c2", Optional.empty()).join(); - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); assertFalse(store.exists(prefix + "/a1/b1").join()); store.delete(prefix + "/a1/b2/c1", Optional.empty()).join(); - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); assertFalse(store.exists(prefix + "/a1/b2").join()); - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); assertFalse(store.exists(prefix + "/a1").join()); - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); assertFalse(store.exists(prefix).join()); } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerIdGeneratorTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerIdGeneratorTest.java index 73d5f451c1ff1..da3fd7f7bd443 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerIdGeneratorTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerIdGeneratorTest.java @@ -242,7 +242,7 @@ public void testEnsureCounterIsNotResetWithContainerNodes(String provider, Suppl l1.await(); log.info("res1 : {}", res1); - zks.checkContainers(); + maybeTriggerDeletingEmptyContainers(provider); CountDownLatch l2 = new CountDownLatch(1); AtomicLong res2 = new AtomicLong(); From 701dfad797d9188aeb0e614182dc2be0c3c3004e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 17:35:34 +0200 Subject: [PATCH 11/56] Skip testDeleteUnusedDirectories for MockZooKeeper --- .../java/org/apache/pulsar/metadata/MetadataStoreTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java index 27d40dd9ba8c2..ddc71a1200aca 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java @@ -68,6 +68,7 @@ import org.assertj.core.util.Lists; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; +import org.testng.SkipException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -415,8 +416,8 @@ public void testDeleteRecursive(String provider, Supplier urlSupplier) t @Test(dataProvider = "impl") public void testDeleteUnusedDirectories(String provider, Supplier urlSupplier) throws Exception { - if (provider.equals("Oxia")) { - return; + if (provider.equals("Oxia") || provider.equals("MockZooKeeper")) { + throw new SkipException("Oxia and MockZooKeeper do not support deleteUnusedDirectories"); } @Cleanup From 13ac174ea0006586537cbbe20015e40f72d674d2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 18:08:36 +0200 Subject: [PATCH 12/56] Improve assertions in getChildren test --- .../apache/pulsar/metadata/MetadataStoreTest.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java index ddc71a1200aca..5dbb585dae199 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.metadata; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -667,21 +668,25 @@ public void testGetChildren(String provider, Supplier urlSupplier) throw store.put("/b/c/b/1", "value1".getBytes(StandardCharsets.UTF_8), Optional.empty()).join(); List subPaths = store.getChildren("/").get(); - Set expectedSet = "ZooKeeper".equals(provider) ? Set.of("a", "b", "zookeeper") : Set.of("a", "b"); + Set ignoredRootPaths = Set.of("zookeeper"); + Set expectedSet = Set.of("a", "b"); for (String subPath : subPaths) { - assertTrue(expectedSet.contains(subPath)); + if (ignoredRootPaths.contains(subPath)) { + continue; + } + assertThat(expectedSet).contains(subPath); } List subPaths2 = store.getChildren("/a").get(); Set expectedSet2 = Set.of("a-1", "a-2"); for (String subPath : subPaths2) { - assertTrue(expectedSet2.contains(subPath)); + assertThat(expectedSet2).contains(subPath); } List subPaths3 = store.getChildren("/b").get(); Set expectedSet3 = Set.of("c"); for (String subPath : subPaths3) { - assertTrue(expectedSet3.contains(subPath)); + assertThat(expectedSet3).contains(subPath); } } From 368993bcbf7a0bbd8a8ffad5ab84c0093d56264a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 18:09:13 +0200 Subject: [PATCH 13/56] Fix MockZooKeeper's getChildren implementation --- .../org/apache/zookeeper/MockZooKeeper.java | 89 +++++-------------- 1 file changed, 20 insertions(+), 69 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index bd9735e59d16d..7f7d065376cac 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -20,7 +20,6 @@ import com.google.common.collect.HashMultimap; import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Multimaps; import com.google.common.collect.SetMultimap; @@ -34,7 +33,6 @@ import java.util.Optional; import java.util.Set; import java.util.TreeMap; -import java.util.TreeSet; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -561,7 +559,6 @@ public void getChildren(final String path, final Watcher watcher, final Children long currentSessionId = getSessionId(); executor.execute(() -> { overrideSessionId(currentSessionId); - List children = Lists.newArrayList(); try { lock(); Optional failure = programmedFailure(Op.GET_CHILDREN, path); @@ -581,21 +578,8 @@ public void getChildren(final String path, final Watcher watcher, final Children return; } - for (String item : tree.tailMap(path).keySet()) { - if (!item.startsWith(path)) { - break; - } else { - if (path.length() >= item.length()) { - continue; - } - - String child = item.substring(path.length() + 1); - if (item.charAt(path.length()) == '/' && !child.contains("/")) { - children.add(child); - } - } - } - + List children = findFirstLevelChildren(path); + unlockIfLocked(); if (watcher != null) { watchers.put(path, new NodeWatcher(watcher, getSessionId())); } @@ -604,7 +588,6 @@ public void getChildren(final String path, final Watcher watcher, final Children log.error("get children : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); } finally { - unlockIfLocked(); removeSessionIdOverride(); } @@ -621,23 +604,11 @@ public List getChildren(String path, Watcher watcher) throws KeeperExcep throw new KeeperException.NoNodeException(); } - String firstKey = path.equals("/") ? path : path + "/"; - String lastKey = path.equals("/") ? "0" : path + "0"; // '0' is lexicographically just after '/' - - Set children = new TreeSet<>(); - tree.subMap(firstKey, false, lastKey, false).forEach((key, value) -> { - String relativePath = key.replace(firstKey, ""); - - // Only return first-level children - String child = relativePath.split("/", 2)[0]; - children.add(child); - }); - if (watcher != null) { watchers.put(path, new NodeWatcher(watcher, getSessionId())); } - return new ArrayList<>(children); + return findFirstLevelChildren(path); } finally { unlockIfLocked(); } @@ -645,32 +616,7 @@ public List getChildren(String path, Watcher watcher) throws KeeperExcep @Override public List getChildren(String path, boolean watch) throws KeeperException, InterruptedException { - lock(); - try { - maybeThrowProgrammedFailure(Op.GET_CHILDREN, path); - - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } else if (!tree.containsKey(path)) { - throw new KeeperException.NoNodeException(); - } - - String firstKey = path.equals("/") ? path : path + "/"; - String lastKey = path.equals("/") ? "0" : path + "0"; // '0' is lexicographically just after '/' - - Set children = new TreeSet<>(); - tree.subMap(firstKey, false, lastKey, false).forEach((key, value) -> { - String relativePath = key.replace(firstKey, ""); - - // Only return first-level children - String child = relativePath.split("/", 2)[0]; - children.add(child); - }); - - return new ArrayList<>(children); - } finally { - unlockIfLocked(); - } + return getChildren(path, null); } @Override @@ -678,7 +624,6 @@ public void getChildren(final String path, boolean watcher, final Children2Callb long currentSessionId = getSessionId(); executor.execute(() -> { overrideSessionId(currentSessionId); - Set children = new TreeSet<>(); try { lock(); MockZNode mockZNode = tree.get(path); @@ -698,17 +643,9 @@ public void getChildren(final String path, boolean watcher, final Children2Callb return; } - String firstKey = path.equals("/") ? path : path + "/"; - String lastKey = path.equals("/") ? "0" : path + "0"; // '0' is lexicographically just after '/' - - tree.subMap(firstKey, false, lastKey, false).forEach((key, value) -> { - String relativePath = key.replace(firstKey, ""); - // Only return first-level children - String child = relativePath.split("/", 2)[0]; - children.add(child); - }); + List children = findFirstLevelChildren(path); unlockIfLocked(); - cb.processResult(0, path, ctx, new ArrayList<>(children), stat); + cb.processResult(0, path, ctx, children, stat); } catch (Throwable ex) { log.error("get children : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); @@ -719,6 +656,20 @@ public void getChildren(final String path, boolean watcher, final Children2Callb } + private List findFirstLevelChildren(String path) { + List children = new ArrayList<>(); + String requiredPrefix = path.equals("/") ? "/" : path + "/"; + for (String key : tree.tailMap(path).keySet()) { + if (key.startsWith(requiredPrefix)) { + String relativePath = key.substring(requiredPrefix.length()); + if (relativePath.indexOf('/') == -1) { + children.add(relativePath); + } + } + } + return children; + } + @Override public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException { lock(); From ea760b4c98fc3c215fc10b2116e20d6c3cde29a4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 18:33:26 +0200 Subject: [PATCH 14/56] Consistently unlock before calling callbacks --- .../main/java/org/apache/zookeeper/MockZooKeeper.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 7f7d065376cac..bd3e2a5c9e791 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -380,6 +380,7 @@ public void create(final String path, final byte[] data, final List acl, Cr lock(); if (stopped) { + unlockIfLocked(); cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); return; } @@ -437,9 +438,9 @@ public void create(final String path, final byte[] data, final List acl, Cr } } catch (Throwable ex) { log.error("create path : {} error", path, ex); + unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); } finally { - unlockIfLocked(); removeSessionIdOverride(); } }); @@ -546,9 +547,9 @@ public void getData(final String path, final Watcher watcher, final DataCallback } } catch (Throwable ex) { log.error("get data : {} error", path, ex); + unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); } finally { - unlockIfLocked(); removeSessionIdOverride(); } }); @@ -586,6 +587,7 @@ public void getChildren(final String path, final Watcher watcher, final Children cb.processResult(0, path, ctx, children); } catch (Throwable ex) { log.error("get children : {} error", path, ex); + unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); } finally { removeSessionIdOverride(); @@ -648,6 +650,7 @@ public void getChildren(final String path, boolean watcher, final Children2Callb cb.processResult(0, path, ctx, children, stat); } catch (Throwable ex) { log.error("get children : {} error", path, ex); + unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); } finally { removeSessionIdOverride(); @@ -752,9 +755,9 @@ public void exists(String path, Watcher watcher, StatCallback cb, Object ctx) { } } catch (Throwable ex) { log.error("exist : {} error", path, ex); + unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); } finally { - unlockIfLocked(); removeSessionIdOverride(); } }); @@ -1005,9 +1008,9 @@ public void delete(final String path, int version, final VoidCallback cb, final } } catch (Throwable ex) { log.error("delete path : {} error", path, ex); + unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx); } finally { - unlockIfLocked(); removeSessionIdOverride(); } }; From 4989e21fb796e4a5dc9c65b7e5eaa3ac6de023f1 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 20:58:51 +0200 Subject: [PATCH 15/56] Restore separate mockZooKeeperGlobal instance --- .../apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index 81d1a105c175f..11ab325324f5b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -487,7 +487,7 @@ protected void configureMetadataStores(PulsarTestContext.Builder builder) { if (useTestZookeeper) { builder.withTestZookeeper(); } else { - builder.withMockZookeeper(); + builder.withMockZookeeper(true); } } From 9cbd80ffec7b1486339e0ea5dcb272cc62cfb8d4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Feb 2025 21:18:07 +0200 Subject: [PATCH 16/56] Show test name for BrokerServiceLookupTest in IntelliJ --- .../client/api/BrokerServiceLookupTest.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index 34db053271e1a..c3bedb324c615 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -112,6 +112,7 @@ import org.awaitility.reflect.WhiteboxImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testng.ITest; import org.testng.SkipException; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -120,8 +121,9 @@ import org.testng.annotations.Test; @Test(groups = "broker-api") -public class BrokerServiceLookupTest extends ProducerConsumerBase { +public class BrokerServiceLookupTest extends ProducerConsumerBase implements ITest { private static final Logger log = LoggerFactory.getLogger(BrokerServiceLookupTest.class); + private String testName; @DataProvider private static Object[] booleanValues() { @@ -134,6 +136,16 @@ public BrokerServiceLookupTest(boolean useTestZookeeper) { this.useTestZookeeper = useTestZookeeper; } + @Override + public String getTestName() { + return testName; + } + + @BeforeMethod + public void applyTestName(Method method) { + testName = method.getName() + " with " + (useTestZookeeper ? "TestZKServer" : "MockZooKeeper"); + } + @BeforeMethod @Override protected void setup() throws Exception { @@ -147,6 +159,7 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { internalCleanup(); + testName = null; } @Override From ee670b32dd1607e3e931003e4acf62416b5ccbfa Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 08:41:15 +0200 Subject: [PATCH 17/56] Optimize MockZooKeeper getChildren --- .../src/main/java/org/apache/zookeeper/MockZooKeeper.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index bd3e2a5c9e791..a0a0fc0fbd1a4 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -662,7 +662,8 @@ public void getChildren(final String path, boolean watcher, final Children2Callb private List findFirstLevelChildren(String path) { List children = new ArrayList<>(); String requiredPrefix = path.equals("/") ? "/" : path + "/"; - for (String key : tree.tailMap(path).keySet()) { + String lastKey = path.equals("/") ? "0" : path + "0"; // '0' is lexicographically just after '/' + for (String key : tree.subMap(requiredPrefix, false, lastKey, false).keySet()) { if (key.startsWith(requiredPrefix)) { String relativePath = key.substring(requiredPrefix.length()); if (relativePath.indexOf('/') == -1) { From 53e587e5d2717bca56b284a16091f1d2bdb5af68 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 09:05:26 +0200 Subject: [PATCH 18/56] Improve BaseMetadataStoreTest: reduce duplication --- .../metadata/BaseMetadataStoreTest.java | 20 ++++++++++++++----- .../pulsar/metadata/MetadataCacheTest.java | 12 ++--------- .../LedgerUnderreplicationManagerTest.java | 8 ++++---- 3 files changed, 21 insertions(+), 19 deletions(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index 4c3e897cd51ea..039bb1f2239ea 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -25,6 +25,8 @@ import io.streamnative.oxia.testcontainers.OxiaContainer; import java.io.File; import java.net.URI; +import java.util.Arrays; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletionException; import java.util.function.Predicate; @@ -132,11 +134,19 @@ public Object[][] implementations() { @DataProvider(name = "distributedImpl") public Object[][] distributedImplementations() { - return new Object[][]{ - {"ZooKeeper", stringSupplier(() -> zksConnectionString)}, - {"Etcd", stringSupplier(() -> "etcd:" + getEtcdClusterConnectString())}, - {"Oxia", stringSupplier(() -> "oxia://" + getOxiaServerConnectString())}, - }; + return filterImplementations("ZooKeeper", "Etcd", "Oxia"); + } + + @DataProvider(name = "zkImpl") + public Object[][] zkImplementation() { + return filterImplementations("ZooKeeper"); + } + + protected Object[][] filterImplementations(String... providers) { + Set providersSet = Set.of(providers); + return Arrays.stream(implementations()) + .filter(impl -> providersSet.contains(impl[0])) + .toArray(Object[][]::new); } protected synchronized String getOxiaServerConnectString() { diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java index ddd975e422ab8..fca1d44b77f48 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java @@ -71,7 +71,6 @@ import org.apache.pulsar.metadata.cache.impl.MetadataCacheImpl; import org.awaitility.Awaitility; import org.mockito.stubbing.Answer; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -113,14 +112,7 @@ public void emptyCacheTest(String provider, Supplier urlSupplier) throws } } - @DataProvider(name = "zk") - public Object[][] zkimplementations() { - return new Object[][] { - { "ZooKeeper", stringSupplier(() -> zks.getConnectionString()) }, - }; - } - - @Test(dataProvider = "zk") + @Test(dataProvider = "zkImpl") public void crossStoreAddDelete(String provider, Supplier urlSupplier) throws Exception { @Cleanup MetadataStore store1 = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); @@ -185,7 +177,7 @@ private void multiStoreAddDelete(List> caches, int addOn, }); } - @Test(dataProvider = "zk") + @Test(dataProvider = "zkImpl") public void crossStoreUpdates(String provider, Supplier urlSupplier) throws Exception { String testName = "cross store updates"; @Cleanup diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java index 0e9c781fb9143..94cc3a23c01a5 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java @@ -300,10 +300,10 @@ public void testMarkingAsReplicated(String provider, Supplier urlSupplie assertEquals(l, lB.get(), "Should be the ledger I marked"); } - - @Test(timeOut = 10000) - public void testZkMetasStoreMarkReplicatedDeleteEmptyParentNodes() throws Exception { - methodSetup(stringSupplier(() -> zks.getConnectionString())); + @Test(dataProvider = "zkImpl", timeOut = 10000) + public void testZkMetasStoreMarkReplicatedDeleteEmptyParentNodes(String provider, Supplier urlSupplier) + throws Exception { + methodSetup(urlSupplier); String missingReplica = "localhost:3181"; From aee6696828bd819d77780e356a3a68b7470febfd Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 09:21:43 +0200 Subject: [PATCH 19/56] Use zkImpl so that provider name shows up in IntelliJ test runner --- .../pulsar/metadata/MetadataCacheTest.java | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java index fca1d44b77f48..b5e23ab1bca82 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java @@ -487,11 +487,10 @@ public void readModifyUpdate(String provider, Supplier urlSupplier) thro * * @throws Exception */ - @Test - public void readModifyUpdateBadVersionRetry() throws Exception { - String url = zks.getConnectionString(); + @Test(dataProvider = "zkImpl") + public void readModifyUpdateBadVersionRetry(String provider, Supplier urlSupplier) throws Exception { @Cleanup - MetadataStore store = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build()); + MetadataStore store = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); MetadataCache cache = store.getMetadataCache(MyClass.class); @@ -505,7 +504,8 @@ public void readModifyUpdateBadVersionRetry() throws Exception { final var sourceStores = new ArrayList(); for (int i = 0; i < 20; i++) { - final var sourceStore = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build()); + final var sourceStore = + MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); sourceStores.add(sourceStore); final var objCache = sourceStore.getMetadataCache(MyClass.class); futures.add(objCache.readModifyUpdate(key1, v -> new MyClass(v.a, v.b + 1))); @@ -516,11 +516,10 @@ public void readModifyUpdateBadVersionRetry() throws Exception { } } - @Test - public void readModifyUpdateOrCreateRetryTimeout() throws Exception { - String url = zks.getConnectionString(); + @Test(dataProvider = "zkImpl") + public void readModifyUpdateOrCreateRetryTimeout(String provider, Supplier urlSupplier) throws Exception { @Cleanup - MetadataStore store = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build()); + MetadataStore store = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); MetadataCache cache = store.getMetadataCache(MyClass.class, MetadataCacheConfig.builder() .retryBackoff(new BackoffBuilder() From 38f4b77dd2721ca500b8fd2603d78603ca8f662a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 09:28:43 +0200 Subject: [PATCH 20/56] Also test MockZooKeeper with zk-only tests --- .../org/apache/pulsar/metadata/BaseMetadataStoreTest.java | 6 +++--- .../org/apache/pulsar/metadata/MetadataCacheTest.java | 8 ++++---- .../bookkeeper/LedgerUnderreplicationManagerTest.java | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index 039bb1f2239ea..bbeac6d3c9121 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -137,9 +137,9 @@ public Object[][] distributedImplementations() { return filterImplementations("ZooKeeper", "Etcd", "Oxia"); } - @DataProvider(name = "zkImpl") - public Object[][] zkImplementation() { - return filterImplementations("ZooKeeper"); + @DataProvider(name = "zkImpls") + public Object[][] zkImplementations() { + return filterImplementations("ZooKeeper", "MockZooKeeper"); } protected Object[][] filterImplementations(String... providers) { diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java index b5e23ab1bca82..d3f2995a84505 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java @@ -112,7 +112,7 @@ public void emptyCacheTest(String provider, Supplier urlSupplier) throws } } - @Test(dataProvider = "zkImpl") + @Test(dataProvider = "zkImpls") public void crossStoreAddDelete(String provider, Supplier urlSupplier) throws Exception { @Cleanup MetadataStore store1 = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); @@ -177,7 +177,7 @@ private void multiStoreAddDelete(List> caches, int addOn, }); } - @Test(dataProvider = "zkImpl") + @Test(dataProvider = "zkImpls") public void crossStoreUpdates(String provider, Supplier urlSupplier) throws Exception { String testName = "cross store updates"; @Cleanup @@ -487,7 +487,7 @@ public void readModifyUpdate(String provider, Supplier urlSupplier) thro * * @throws Exception */ - @Test(dataProvider = "zkImpl") + @Test(dataProvider = "zkImpls") public void readModifyUpdateBadVersionRetry(String provider, Supplier urlSupplier) throws Exception { @Cleanup MetadataStore store = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); @@ -516,7 +516,7 @@ public void readModifyUpdateBadVersionRetry(String provider, Supplier ur } } - @Test(dataProvider = "zkImpl") + @Test(dataProvider = "zkImpls") public void readModifyUpdateOrCreateRetryTimeout(String provider, Supplier urlSupplier) throws Exception { @Cleanup MetadataStore store = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java index 94cc3a23c01a5..ac73491a81c65 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerUnderreplicationManagerTest.java @@ -300,7 +300,7 @@ public void testMarkingAsReplicated(String provider, Supplier urlSupplie assertEquals(l, lB.get(), "Should be the ledger I marked"); } - @Test(dataProvider = "zkImpl", timeOut = 10000) + @Test(dataProvider = "zkImpls", timeOut = 10000) public void testZkMetasStoreMarkReplicatedDeleteEmptyParentNodes(String provider, Supplier urlSupplier) throws Exception { methodSetup(urlSupplier); From 89283e0b4f7f6c487a05e24c67e0eab0f6b37584 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 09:48:33 +0200 Subject: [PATCH 21/56] Allow filtering providers with TEST_METADATA_PROVIDERS env variable --- .../apache/pulsar/metadata/BaseMetadataStoreTest.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index bbeac6d3c9121..28f160b3bbc7a 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -32,6 +32,7 @@ import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.MetadataStoreFactory; @@ -117,6 +118,14 @@ public void cleanup() throws Exception { @DataProvider(name = "impl") public Object[][] implementations() { + // If the environment variable TEST_METADATA_PROVIDERS is set, only run the specified implementations + if (StringUtils.isNotBlank(System.getenv("TEST_METADATA_PROVIDERS"))) { + return filterImplementations(System.getenv("TEST_METADATA_PROVIDERS").split(",")); + } + return allImplementations(); + } + + private Object[][] allImplementations() { // A Supplier must be used for the Zookeeper connection string parameter. The retried test run will // use the same arguments as the failed attempt. // The Zookeeper test server gets restarted by TestRetrySupport before the retry. @@ -144,7 +153,7 @@ public Object[][] zkImplementations() { protected Object[][] filterImplementations(String... providers) { Set providersSet = Set.of(providers); - return Arrays.stream(implementations()) + return Arrays.stream(allImplementations()) .filter(impl -> providersSet.contains(impl[0])) .toArray(Object[][]::new); } From 4d833fe89c4d64f275daae2ee5e832537368f795 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 10:06:40 +0200 Subject: [PATCH 22/56] Get rid of reflection in MockZooKeeper --- .../java/org/apache/zookeeper/MockZooKeeper.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index a0a0fc0fbd1a4..529d64780d94e 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -46,7 +46,6 @@ import java.util.function.BiPredicate; import lombok.AllArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.zookeeper.AsyncCallback.Children2Callback; import org.apache.zookeeper.AsyncCallback.ChildrenCallback; import org.apache.zookeeper.AsyncCallback.DataCallback; @@ -57,6 +56,8 @@ import org.apache.zookeeper.Watcher.Event.KeeperState; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.proto.DeleteRequest; +import org.apache.zookeeper.proto.SetDataRequest; import org.objenesis.Objenesis; import org.objenesis.ObjenesisStd; import org.objenesis.instantiator.ObjectInstantiator; @@ -1047,14 +1048,13 @@ public List multi(Iterable ops) throws Interr res.add(new OpResult.CreateResult(path)); } case ZooDefs.OpCode.delete -> { - delete(op.getPath(), (int) FieldUtils.readField(op, "version", true)); + DeleteRequest deleteRequest = (DeleteRequest) op.toRequestRecord(); + delete(op.getPath(), deleteRequest.getVersion()); res.add(new OpResult.DeleteResult()); } case ZooDefs.OpCode.setData -> { - Stat stat = setData( - op.getPath(), - (byte[]) FieldUtils.readField(op, "data", true), - (int) FieldUtils.readField(op, "version", true)); + SetDataRequest setDataRequest = (SetDataRequest) op.toRequestRecord(); + Stat stat = setData(op.getPath(), setDataRequest.getData(), setDataRequest.getVersion()); res.add(new OpResult.SetDataResult(stat)); } case ZooDefs.OpCode.getChildren -> { @@ -1082,8 +1082,6 @@ public List multi(Iterable ops) throws Interr for (int i = res.size(); i < total; i++) { res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); } - } catch (IllegalAccessException e) { - throw new IllegalStateException(e); } return res; } From cb32580e568a89bc57c397974af9cbff0e0ea5a9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 10:07:08 +0200 Subject: [PATCH 23/56] Handle any exception in multi ops --- .../org/apache/zookeeper/MockZooKeeper.java | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 529d64780d94e..062060ee3632b 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -19,7 +19,6 @@ package org.apache.zookeeper; import com.google.common.collect.HashMultimap; -import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Multimaps; import com.google.common.collect.SetMultimap; @@ -44,6 +43,8 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiPredicate; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import lombok.AllArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.zookeeper.AsyncCallback.Children2Callback; @@ -1036,8 +1037,10 @@ public void multi(Iterable ops, AsyncCallback.MultiCall } @Override - public List multi(Iterable ops) throws InterruptedException, KeeperException { + public List multi(Iterable opsParam) throws InterruptedException, KeeperException { List res = new ArrayList<>(); + List ops = + StreamSupport.stream(opsParam.spliterator(), false).collect(Collectors.toList()); try { for (org.apache.zookeeper.Op op : ops) { switch (op.getType()) { @@ -1076,12 +1079,18 @@ public List multi(Iterable ops) throws Interr } } } - } catch (KeeperException e) { - res.add(new OpResult.ErrorResult(e.code().intValue())); - int total = Iterables.size(ops); - for (int i = res.size(); i < total; i++) { + } catch (Exception e) { + if (e instanceof KeeperException keeperException) { + res.add(new OpResult.ErrorResult(keeperException.code().intValue())); + } + for (int i = res.size(); i < ops.size(); i++) { res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); } + if (e instanceof InterruptedException || e instanceof KeeperException) { + throw e; + } else { + log.error("Error in multi", e); + } } return res; } From 9d1c9942171abc86a7847fc06094746feaed9675 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 10:13:11 +0200 Subject: [PATCH 24/56] Fix checkstyle --- .../src/main/java/org/apache/zookeeper/MockZooKeeper.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 062060ee3632b..ef2609f006dc8 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -1037,7 +1037,8 @@ public void multi(Iterable ops, AsyncCallback.MultiCall } @Override - public List multi(Iterable opsParam) throws InterruptedException, KeeperException { + public List multi(Iterable opsParam) + throws InterruptedException, KeeperException { List res = new ArrayList<>(); List ops = StreamSupport.stream(opsParam.spliterator(), false).collect(Collectors.toList()); From 85e217c605fd87903e97e4590a6638b376de63d4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 10:18:47 +0200 Subject: [PATCH 25/56] Log errors in MockZooKeeper multi --- .../src/main/java/org/apache/zookeeper/MockZooKeeper.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index ef2609f006dc8..b6f49e9fc70c4 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -1081,6 +1081,7 @@ public List multi(Iterable opsParam) } } } catch (Exception e) { + log.error("Error in multi", e); if (e instanceof KeeperException keeperException) { res.add(new OpResult.ErrorResult(keeperException.code().intValue())); } @@ -1090,7 +1091,7 @@ public List multi(Iterable opsParam) if (e instanceof InterruptedException || e instanceof KeeperException) { throw e; } else { - log.error("Error in multi", e); + throw new KeeperException.SystemErrorException(); } } return res; From 6e2175b02b15aabab13626f9b1709d434d6993b1 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 10:26:45 +0200 Subject: [PATCH 26/56] Fix error handling in MockZooKeeper multi --- .../org/apache/zookeeper/MockZooKeeper.java | 85 ++++++++++--------- 1 file changed, 45 insertions(+), 40 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index b6f49e9fc70c4..1b76cf670435f 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -43,8 +43,6 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiPredicate; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; import lombok.AllArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.zookeeper.AsyncCallback.Children2Callback; @@ -1037,61 +1035,68 @@ public void multi(Iterable ops, AsyncCallback.MultiCall } @Override - public List multi(Iterable opsParam) - throws InterruptedException, KeeperException { + public List multi(Iterable ops) throws InterruptedException, KeeperException { List res = new ArrayList<>(); - List ops = - StreamSupport.stream(opsParam.spliterator(), false).collect(Collectors.toList()); - try { - for (org.apache.zookeeper.Op op : ops) { - switch (op.getType()) { - case ZooDefs.OpCode.create -> { + for (org.apache.zookeeper.Op op : ops) { + switch (op.getType()) { + case ZooDefs.OpCode.create -> { + try { org.apache.zookeeper.Op.Create opc = ((org.apache.zookeeper.Op.Create) op); CreateMode cm = CreateMode.fromFlag(opc.flags); String path = create(op.getPath(), opc.data, null, cm); res.add(new OpResult.CreateResult(path)); + } catch (KeeperException e) { + res.add(new OpResult.ErrorResult(e.code().intValue())); + } catch (Exception e) { + res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); } - case ZooDefs.OpCode.delete -> { + } + case ZooDefs.OpCode.delete -> { + try { DeleteRequest deleteRequest = (DeleteRequest) op.toRequestRecord(); delete(op.getPath(), deleteRequest.getVersion()); res.add(new OpResult.DeleteResult()); + } catch (KeeperException e) { + res.add(new OpResult.ErrorResult(e.code().intValue())); + } catch (Exception e) { + res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); } - case ZooDefs.OpCode.setData -> { + } + case ZooDefs.OpCode.setData -> { + try { SetDataRequest setDataRequest = (SetDataRequest) op.toRequestRecord(); Stat stat = setData(op.getPath(), setDataRequest.getData(), setDataRequest.getVersion()); res.add(new OpResult.SetDataResult(stat)); + } catch (KeeperException e) { + res.add(new OpResult.ErrorResult(e.code().intValue())); + } catch (Exception e) { + res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); } - case ZooDefs.OpCode.getChildren -> { - try { - List children = getChildren(op.getPath(), null); - res.add(new OpResult.GetChildrenResult(children)); - } catch (KeeperException e) { - res.add(new OpResult.ErrorResult(e.code().intValue())); - } + } + case ZooDefs.OpCode.getChildren -> { + try { + List children = getChildren(op.getPath(), null); + res.add(new OpResult.GetChildrenResult(children)); + } catch (KeeperException e) { + res.add(new OpResult.ErrorResult(e.code().intValue())); + } catch (Exception e) { + res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); } - case ZooDefs.OpCode.getData -> { - Stat stat = new Stat(); - try { - byte[] payload = getData(op.getPath(), null, stat); - res.add(new OpResult.GetDataResult(payload, stat)); - } catch (KeeperException e) { - res.add(new OpResult.ErrorResult(e.code().intValue())); - } + } + case ZooDefs.OpCode.getData -> { + Stat stat = new Stat(); + try { + byte[] payload = getData(op.getPath(), null, stat); + res.add(new OpResult.GetDataResult(payload, stat)); + } catch (KeeperException e) { + res.add(new OpResult.ErrorResult(e.code().intValue())); + } catch (Exception e) { + res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); } } - } - } catch (Exception e) { - log.error("Error in multi", e); - if (e instanceof KeeperException keeperException) { - res.add(new OpResult.ErrorResult(keeperException.code().intValue())); - } - for (int i = res.size(); i < ops.size(); i++) { - res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); - } - if (e instanceof InterruptedException || e instanceof KeeperException) { - throw e; - } else { - throw new KeeperException.SystemErrorException(); + default -> { + res.add(new OpResult.ErrorResult(KeeperException.Code.APIERROR.intValue())); + } } } return res; From 4a5a49225ebfd56b9ebff2ecaed2c96548c3c740 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 10:36:04 +0200 Subject: [PATCH 27/56] Consistently trigger watches in executor --- .../org/apache/zookeeper/MockZooKeeper.java | 70 +++++++++++++------ 1 file changed, 48 insertions(+), 22 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 1b76cf670435f..caa167b9dd4c4 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -326,6 +326,10 @@ public String create(String path, byte[] data, List acl, CreateMode createM final String finalPath = path; executor.execute(() -> { + if (stopped) { + return; + } + triggerPersistentWatches(finalPath, parent, EventType.NodeCreated); toNotifyCreate.forEach( @@ -423,18 +427,24 @@ public void create(final String path, final byte[] data, final List acl, Cr unlockIfLocked(); cb.processResult(0, path, ctx, name); - triggerPersistentWatches(path, parent, EventType.NodeCreated); - - toNotifyCreate.forEach( - watcher -> watcher.process( - new WatchedEvent(EventType.NodeCreated, - KeeperState.SyncConnected, - name))); - toNotifyParent.forEach( - watcher -> watcher.process( - new WatchedEvent(EventType.NodeChildrenChanged, - KeeperState.SyncConnected, - parent))); + executor.execute(() -> { + if (stopped) { + return; + } + + triggerPersistentWatches(path, parent, EventType.NodeCreated); + + toNotifyCreate.forEach( + watcher -> watcher.process( + new WatchedEvent(EventType.NodeCreated, + KeeperState.SyncConnected, + name))); + toNotifyParent.forEach( + watcher -> watcher.process( + new WatchedEvent(EventType.NodeChildrenChanged, + KeeperState.SyncConnected, + parent))); + }); } } catch (Throwable ex) { log.error("create path : {} error", path, ex); @@ -821,6 +831,10 @@ public Stat setData(final String path, byte[] data, int version) throws KeeperEx } executor.execute(() -> { + if (stopped) { + return; + } + triggerPersistentWatches(path, null, EventType.NodeDataChanged); toNotify.forEach(watcher -> watcher @@ -884,11 +898,17 @@ public void setData(final String path, final byte[] data, int version, final Sta toNotify.addAll(getWatchers(path)); watchers.removeAll(path); - for (Watcher watcher : toNotify) { - watcher.process(new WatchedEvent(EventType.NodeDataChanged, KeeperState.SyncConnected, path)); - } + executor.execute(() -> { + if (stopped) { + return; + } - triggerPersistentWatches(path, null, EventType.NodeDataChanged); + triggerPersistentWatches(path, null, EventType.NodeDataChanged); + + for (Watcher watcher : toNotify) { + watcher.process(new WatchedEvent(EventType.NodeDataChanged, KeeperState.SyncConnected, path)); + } + }); } catch (Throwable ex) { log.error("Update data : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); @@ -1000,12 +1020,18 @@ public void delete(final String path, int version, final VoidCallback cb, final unlockIfLocked(); cb.processResult(0, path, ctx); - toNotifyDelete.forEach(watcher -> watcher - .process(new WatchedEvent(EventType.NodeDeleted, KeeperState.SyncConnected, path))); - toNotifyParent.forEach(watcher -> watcher - .process(new WatchedEvent(EventType.NodeChildrenChanged, KeeperState.SyncConnected, - parent))); - triggerPersistentWatches(path, parent, EventType.NodeDeleted); + executor.execute(() -> { + if (stopped) { + return; + } + + triggerPersistentWatches(path, parent, EventType.NodeDeleted); + toNotifyDelete.forEach(watcher -> watcher + .process(new WatchedEvent(EventType.NodeDeleted, KeeperState.SyncConnected, path))); + toNotifyParent.forEach(watcher -> watcher + .process(new WatchedEvent(EventType.NodeChildrenChanged, KeeperState.SyncConnected, + parent))); + }); } } catch (Throwable ex) { log.error("delete path : {} error", path, ex); From 5bd681ab0003ff8181b6c7eecbf8370bac9a0557 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 10:42:26 +0200 Subject: [PATCH 28/56] Improve error logging in MockZooKeeper multi --- .../org/apache/zookeeper/MockZooKeeper.java | 61 ++++++++++--------- 1 file changed, 31 insertions(+), 30 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index caa167b9dd4c4..e2627d73b532d 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -1066,61 +1066,43 @@ public List multi(Iterable ops) throws Interr for (org.apache.zookeeper.Op op : ops) { switch (op.getType()) { case ZooDefs.OpCode.create -> { - try { + handleOperation("create", op, () -> { org.apache.zookeeper.Op.Create opc = ((org.apache.zookeeper.Op.Create) op); CreateMode cm = CreateMode.fromFlag(opc.flags); String path = create(op.getPath(), opc.data, null, cm); res.add(new OpResult.CreateResult(path)); - } catch (KeeperException e) { - res.add(new OpResult.ErrorResult(e.code().intValue())); - } catch (Exception e) { - res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); - } + }, res); } case ZooDefs.OpCode.delete -> { - try { + handleOperation("delete", op, () -> { DeleteRequest deleteRequest = (DeleteRequest) op.toRequestRecord(); delete(op.getPath(), deleteRequest.getVersion()); res.add(new OpResult.DeleteResult()); - } catch (KeeperException e) { - res.add(new OpResult.ErrorResult(e.code().intValue())); - } catch (Exception e) { - res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); - } + }, res); } case ZooDefs.OpCode.setData -> { - try { + handleOperation("setData", op, () -> { SetDataRequest setDataRequest = (SetDataRequest) op.toRequestRecord(); Stat stat = setData(op.getPath(), setDataRequest.getData(), setDataRequest.getVersion()); res.add(new OpResult.SetDataResult(stat)); - } catch (KeeperException e) { - res.add(new OpResult.ErrorResult(e.code().intValue())); - } catch (Exception e) { - res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); - } + }, res); } case ZooDefs.OpCode.getChildren -> { - try { + handleOperation("getChildren", op, () -> { List children = getChildren(op.getPath(), null); res.add(new OpResult.GetChildrenResult(children)); - } catch (KeeperException e) { - res.add(new OpResult.ErrorResult(e.code().intValue())); - } catch (Exception e) { - res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); - } + }, res); } case ZooDefs.OpCode.getData -> { Stat stat = new Stat(); - try { + handleOperation("getData", op, () -> { byte[] payload = getData(op.getPath(), null, stat); res.add(new OpResult.GetDataResult(payload, stat)); - } catch (KeeperException e) { - res.add(new OpResult.ErrorResult(e.code().intValue())); - } catch (Exception e) { - res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); - } + }, res); } default -> { + log.error("Unsupported operation for path {} type {} kind {} request {}", op.getPath(), + op.getType(), op.getKind(), op.toRequestRecord()); res.add(new OpResult.ErrorResult(KeeperException.Code.APIERROR.intValue())); } } @@ -1128,6 +1110,25 @@ public List multi(Iterable ops) throws Interr return res; } + interface ZkOpHandler { + void handle() throws KeeperException, InterruptedException; + } + + private void handleOperation(String opName, org.apache.zookeeper.Op op, ZkOpHandler handler, List res) { + try { + handler.handle(); + } catch (Exception e) { + log.error("Error handling {} operation for path {} type {} kind {} request {} {}", opName, op.getPath(), + op.getType(), op.getKind(), op.toRequestRecord(), + e instanceof KeeperException ? e.getMessage() : e); + if (e instanceof KeeperException keeperException) { + res.add(new OpResult.ErrorResult(keeperException.code().intValue())); + } else { + res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); + } + } + } + @Override public synchronized void addWatch(String basePath, Watcher watcher, AddWatchMode mode) { persistentWatchers.add(new PersistentWatcher(basePath, watcher, mode, getSessionId())); From 4c27282e9d9e9ed37879b1aa777a46c0f0c5ace8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 11:05:25 +0200 Subject: [PATCH 29/56] Always use a single threaded executor --- .../org/apache/zookeeper/MockZooKeeper.java | 36 ++++--------------- 1 file changed, 7 insertions(+), 29 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index e2627d73b532d..74efdcd0426dd 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -160,30 +160,12 @@ private record NodeWatcher(Watcher watcher, long sessionId) { private List persistentWatchers; public static MockZooKeeper newInstance() { - return newInstance(null); + return newInstance(-1); } - public static MockZooKeeper newInstance(ExecutorService executor) { - return newInstance(executor, -1); - } - - public static MockZooKeeper newInstanceForGlobalZK(ExecutorService executor) { - return newInstanceForGlobalZK(executor, -1); - } - - public static MockZooKeeper newInstanceForGlobalZK(ExecutorService executor, int readOpDelayMs) { - try { - return createMockZooKeeperInstance(executor, readOpDelayMs); - } catch (RuntimeException e) { - throw e; - } catch (Exception e) { - throw new IllegalStateException("Cannot create object", e); - } - } - - public static MockZooKeeper newInstance(ExecutorService executor, int readOpDelayMs) { + public static MockZooKeeper newInstance(int readOpDelayMs) { try { - return createMockZooKeeperInstance(executor, readOpDelayMs); + return createMockZooKeeperInstance(readOpDelayMs); } catch (RuntimeException e) { throw e; } catch (Exception e) { @@ -191,12 +173,12 @@ public static MockZooKeeper newInstance(ExecutorService executor, int readOpDela } } - private static MockZooKeeper createMockZooKeeperInstance(ExecutorService executor, int readOpDelayMs) { + private static MockZooKeeper createMockZooKeeperInstance(int readOpDelayMs) { ObjectInstantiator mockZooKeeperInstantiator = objenesis.getInstantiatorOf(MockZooKeeper.class); MockZooKeeper zk = mockZooKeeperInstantiator.newInstance(); zk.overriddenSessionIdThreadLocal = new ThreadLocal<>(); - zk.init(executor); + zk.init(); zk.readOpDelayMs = readOpDelayMs; zk.mutex = new ReentrantLock(); zk.lockInstance = ThreadLocal.withInitial(zk::createLock); @@ -205,13 +187,9 @@ private static MockZooKeeper createMockZooKeeperInstance(ExecutorService executo return zk; } - private void init(ExecutorService executor) { + private void init() { tree = Maps.newTreeMap(); - if (executor != null) { - this.executor = executor; - } else { - this.executor = Executors.newFixedThreadPool(1, new DefaultThreadFactory("mock-zookeeper")); - } + this.executor = Executors.newSingleThreadExecutor(new DefaultThreadFactory("mock-zookeeper")); SetMultimap w = HashMultimap.create(); watchers = Multimaps.synchronizedSetMultimap(w); stopped = false; From be6a424bf716c1655855968d6448cba3bf5786c2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 12:09:42 +0200 Subject: [PATCH 30/56] Replace locks in MockZooKeeper with single threaded execution --- .../org/apache/zookeeper/MockZooKeeper.java | 780 ++++++++---------- .../zookeeper/MockZooKeeperSession.java | 4 +- 2 files changed, 347 insertions(+), 437 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 74efdcd0426dd..461ccc1850cfa 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -32,19 +32,17 @@ import java.util.Optional; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.Callable; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiPredicate; import lombok.AllArgsConstructor; -import lombok.extern.slf4j.Slf4j; import org.apache.zookeeper.AsyncCallback.Children2Callback; import org.apache.zookeeper.AsyncCallback.ChildrenCallback; import org.apache.zookeeper.AsyncCallback.DataCallback; @@ -117,19 +115,18 @@ public long getEphemeralOwner() { private TreeMap tree; private SetMultimap watchers; - private volatile boolean stopped; + private AtomicBoolean stopped; private AtomicReference alwaysFail; private CopyOnWriteArrayList failures; private ExecutorService executor; - private Watcher sessionWatcher; + private volatile Watcher sessionWatcher; private long sessionId = 1L; private int readOpDelayMs; - private ReentrantLock mutex; - private AtomicLong sequentialIdGenerator; private ThreadLocal overriddenSessionIdThreadLocal; + private ThreadLocal inExecutorThreadLocal; private int referenceCount; private List closeables; @@ -178,10 +175,9 @@ private static MockZooKeeper createMockZooKeeperInstance(int readOpDelayMs) { objenesis.getInstantiatorOf(MockZooKeeper.class); MockZooKeeper zk = mockZooKeeperInstantiator.newInstance(); zk.overriddenSessionIdThreadLocal = new ThreadLocal<>(); + zk.inExecutorThreadLocal = ThreadLocal.withInitial(() -> false); zk.init(); zk.readOpDelayMs = readOpDelayMs; - zk.mutex = new ReentrantLock(); - zk.lockInstance = ThreadLocal.withInitial(zk::createLock); zk.sequentialIdGenerator = new AtomicLong(); zk.closeables = new ArrayList<>(); return zk; @@ -192,7 +188,7 @@ private void init() { this.executor = Executors.newSingleThreadExecutor(new DefaultThreadFactory("mock-zookeeper")); SetMultimap w = HashMultimap.create(); watchers = Multimaps.synchronizedSetMultimap(w); - stopped = false; + stopped = new AtomicBoolean(false); alwaysFail = new AtomicReference<>(KeeperException.Code.OK); failures = new CopyOnWriteArrayList<>(); persistentWatchers = new ArrayList<>(); @@ -215,96 +211,140 @@ public States getState() { return States.CONNECTED; } + @Override + public void register(Watcher watcher) { + sessionWatcher = watcher; + } - @Slf4j - private static class SingleAcquireAndReleaseLock { - private final AtomicBoolean acquired = new AtomicBoolean(false); - private final Lock lock; + @Override + public String create(String path, byte[] data, List acl, CreateMode createMode) + throws KeeperException, InterruptedException { + return runInExecutorReturningValue(() -> internalCreate(path, data, createMode)); + } - SingleAcquireAndReleaseLock(Lock lock) { - this.lock = lock; - } + private T runInExecutorReturningValue(Callable task) throws InterruptedException, KeeperException { + return runInExecutorReturningValue(task, true); + } - public void lock() { - if (acquired.compareAndSet(false, true)) { - lock.lock(); - } else { - throw new IllegalStateException("Lock was already acquired!"); + private T runInExecutorReturningValue(Callable task, boolean allowRunningInCurrentThread) + throws InterruptedException, KeeperException { + if (allowRunningInCurrentThread && inExecutorThreadLocal.get()) { + try { + return task.call(); + } catch (Exception e) { + if (e instanceof KeeperException ke) { + throw ke; + } + if (e instanceof InterruptedException ie) { + throw ie; + } + log.error("Unexpected exception", e); + throw new KeeperException.SystemErrorException(); } } - - public void unlockIfNeeded() { - if (acquired.compareAndSet(true, false)) { - lock.unlock(); + try { + long currentSessionId = getSessionId(); + return executor.submit(() -> { + inExecutorThreadLocal.set(true); + overrideSessionId(currentSessionId); + try { + return task.call(); + } finally { + removeSessionIdOverride(); + inExecutorThreadLocal.set(false); + } + }).get(); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof KeeperException ke) { + throw ke; } + if (cause instanceof InterruptedException ie) { + throw ie; + } + log.error("Unexpected exception", e); + throw new KeeperException.SystemErrorException(); } } - private ThreadLocal lockInstance; - - private SingleAcquireAndReleaseLock createLock() { - return new SingleAcquireAndReleaseLock(mutex); + private void runInExecutorAsync(Runnable runnable) { + runInExecutorAsync(runnable, true); } - private void lock() { - lockInstance.get().lock(); - } - - private void unlockIfLocked() { - lockInstance.get().unlockIfNeeded(); + private void runInExecutorAsync(Runnable runnable, boolean allowRunningInCurrentThread) { + if (allowRunningInCurrentThread && inExecutorThreadLocal.get()) { + try { + runnable.run(); + } catch (Throwable t) { + log.error("Unexpected exception", t); + } + return; + } + long currentSessionId = getSessionId(); + executor.submit(() -> { + try { + inExecutorThreadLocal.set(true); + overrideSessionId(currentSessionId); + try { + runnable.run(); + } finally { + removeSessionIdOverride(); + inExecutorThreadLocal.set(false); + } + } catch (Throwable t) { + log.error("Unexpected exception", t); + } + }); } - @Override - public void register(Watcher watcher) { - lock(); - sessionWatcher = watcher; - unlockIfLocked(); + private void runInExecutorSync(Runnable runnable) { + try { + runInExecutorReturningValue(() -> { + runnable.run(); + return null; + }); + } catch (Exception e) { + log.error("Unexpected error", e); + } } - @Override - public String create(String path, byte[] data, List acl, CreateMode createMode) - throws KeeperException, InterruptedException { + private String internalCreate(String path, byte[] data, CreateMode createMode) throws KeeperException { final Set toNotifyCreate = Sets.newHashSet(); final Set toNotifyParent = Sets.newHashSet(); final String parent = path.substring(0, path.lastIndexOf("/")); - lock(); - try { - maybeThrowProgrammedFailure(Op.CREATE, path); + maybeThrowProgrammedFailure(Op.CREATE, path); - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } + if (isStopped()) { + throw new KeeperException.ConnectionLossException(); + } - if (tree.containsKey(path)) { - throw new KeeperException.NodeExistsException(path); - } + if (tree.containsKey(path)) { + throw new KeeperException.NodeExistsException(path); + } - if (!parent.isEmpty() && !tree.containsKey(parent)) { - throw new KeeperException.NoNodeException(); - } + if (!parent.isEmpty() && !tree.containsKey(parent)) { + throw new KeeperException.NoNodeException(); + } - if (createMode.isSequential()) { - MockZNode parentNode = tree.get(parent); - int parentVersion = tree.get(parent).getVersion(); - path = path + parentVersion; - parentNode.updateVersion(); - } + if (createMode.isSequential()) { + MockZNode parentNode = tree.get(parent); + int parentVersion = tree.get(parent).getVersion(); + path = path + parentVersion; + parentNode.updateVersion(); + } - tree.put(path, createMockZNode(data, createMode)); + tree.put(path, createMockZNode(data, createMode)); - toNotifyCreate.addAll(getWatchers(path)); - if (!parent.isEmpty()) { - toNotifyParent.addAll(getWatchers(parent)); - } - watchers.removeAll(path); - } finally { - unlockIfLocked(); + toNotifyCreate.addAll(getWatchers(path)); + if (!parent.isEmpty()) { + toNotifyParent.addAll(getWatchers(parent)); } + watchers.removeAll(path); final String finalPath = path; executor.execute(() -> { - if (stopped) { + if (isStopped()) { return; } @@ -354,15 +394,13 @@ public void removeSessionIdOverride() { @Override public void create(final String path, final byte[] data, final List acl, CreateMode createMode, final StringCallback cb, final Object ctx) { - - long currentSessionId = getSessionId(); - executor.execute(() -> { - overrideSessionId(currentSessionId); + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); + return; + } + runInExecutorAsync(() -> { try { - lock(); - - if (stopped) { - unlockIfLocked(); + if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); return; } @@ -385,31 +423,23 @@ public void create(final String path, final byte[] data, final List acl, Cr Optional failure = programmedFailure(Op.CREATE, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx, null); - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); } else if (tree.containsKey(path)) { - unlockIfLocked(); cb.processResult(KeeperException.Code.NODEEXISTS.intValue(), path, ctx, null); } else if (!parent.isEmpty() && !tree.containsKey(parent)) { - unlockIfLocked(); - toNotifyParent.forEach(watcher -> watcher - .process(new WatchedEvent(EventType.NodeChildrenChanged, KeeperState.SyncConnected, - parent))); + runNotifications(() -> { + toNotifyParent.forEach(watcher -> watcher + .process(new WatchedEvent(EventType.NodeChildrenChanged, KeeperState.SyncConnected, + parent))); + }); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); } else { tree.put(name, createMockZNode(data, createMode)); watchers.removeAll(name); - unlockIfLocked(); cb.processResult(0, path, ctx, name); - - executor.execute(() -> { - if (stopped) { - return; - } - + runNotifications(() -> { triggerPersistentWatches(path, parent, EventType.NodeCreated); toNotifyCreate.forEach( @@ -426,13 +456,22 @@ public void create(final String path, final byte[] data, final List acl, Cr } } catch (Throwable ex) { log.error("create path : {} error", path, ex); - unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); - } finally { - removeSessionIdOverride(); } }); + } + public void runNotifications(Runnable runnable) { + executor.execute(() -> { + if (isStopped()) { + return; + } + runnable.run(); + }); + } + + private boolean isStopped() { + return stopped.get(); } private MockZNode createMockZNode(byte[] data, CreateMode createMode) { @@ -441,167 +480,117 @@ private MockZNode createMockZNode(byte[] data, CreateMode createMode) { } @Override - public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException { - lock(); - try { - maybeThrowProgrammedFailure(Op.GET, path); - MockZNode value = tree.get(path); - if (value == null) { - throw new KeeperException.NoNodeException(path); - } else { - if (watcher != null) { - watchers.put(path, new NodeWatcher(watcher, getSessionId())); - } - if (stat != null) { - value.applyToStat(stat); - } - return value.getContent(); + public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException, InterruptedException { + return runInExecutorReturningValue(() -> internalGetData(path, watcher, stat)); + } + + private byte[] internalGetData(String path, Watcher watcher, Stat stat) throws KeeperException { + maybeThrowProgrammedFailure(Op.GET, path); + MockZNode value = tree.get(path); + if (value == null) { + throw new KeeperException.NoNodeException(path); + } else { + if (watcher != null) { + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } - } finally { - unlockIfLocked(); + if (stat != null) { + value.applyToStat(stat); + } + return value.getContent(); } } @Override public void getData(final String path, boolean watch, final DataCallback cb, final Object ctx) { - long currentSessionId = getSessionId(); - executor.execute(() -> { - overrideSessionId(currentSessionId); - try { - checkReadOpDelay(); - Optional failure = programmedFailure(Op.GET, path); - if (failure.isPresent()) { - cb.processResult(failure.get().intValue(), path, ctx, null, null); - return; - } else if (stopped) { - cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null, null); - return; - } - - MockZNode value; - Stat stat; - lock(); - try { - value = tree.get(path); - stat = value.getStat(); - } finally { - unlockIfLocked(); - } - - if (value == null) { - cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null, null); - } else { - cb.processResult(0, path, ctx, value.getContent(), stat); - } - } catch (Throwable ex) { - log.error("get data : {} error", path, ex); - cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); - } finally { - removeSessionIdOverride(); - } - }); + getData(path, null, cb, ctx); } @Override public void getData(final String path, final Watcher watcher, final DataCallback cb, final Object ctx) { - long currentSessionId = getSessionId(); - executor.execute(() -> { + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null, null); + return; + } + runInExecutorAsync(() -> { checkReadOpDelay(); - overrideSessionId(currentSessionId); try { - lock(); Optional failure = programmedFailure(Op.GET, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx, null, null); return; - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null, null); return; } MockZNode value = tree.get(path); if (value == null) { - unlockIfLocked(); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null, null); } else { if (watcher != null) { watchers.put(path, new NodeWatcher(watcher, getSessionId())); } Stat stat = value.getStat(); - unlockIfLocked(); cb.processResult(0, path, ctx, value.getContent(), stat); } } catch (Throwable ex) { log.error("get data : {} error", path, ex); - unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); - } finally { - removeSessionIdOverride(); } }); } @Override public void getChildren(final String path, final Watcher watcher, final ChildrenCallback cb, final Object ctx) { - long currentSessionId = getSessionId(); - executor.execute(() -> { - overrideSessionId(currentSessionId); + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); + return; + } + runInExecutorAsync(() -> { try { - lock(); Optional failure = programmedFailure(Op.GET_CHILDREN, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx, null); return; - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); return; } if (!tree.containsKey(path)) { - unlockIfLocked(); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); return; } List children = findFirstLevelChildren(path); - unlockIfLocked(); if (watcher != null) { watchers.put(path, new NodeWatcher(watcher, getSessionId())); } cb.processResult(0, path, ctx, children); } catch (Throwable ex) { log.error("get children : {} error", path, ex); - unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); - } finally { - removeSessionIdOverride(); } - }); } @Override - public List getChildren(String path, Watcher watcher) throws KeeperException { - lock(); - try { - maybeThrowProgrammedFailure(Op.GET_CHILDREN, path); + public List getChildren(String path, Watcher watcher) throws KeeperException, InterruptedException { + return runInExecutorReturningValue(() -> internalGetChildren(path, watcher)); + } - if (!tree.containsKey(path)) { - throw new KeeperException.NoNodeException(); - } + private List internalGetChildren(String path, Watcher watcher) throws KeeperException { + maybeThrowProgrammedFailure(Op.GET_CHILDREN, path); - if (watcher != null) { - watchers.put(path, new NodeWatcher(watcher, getSessionId())); - } + if (!tree.containsKey(path)) { + throw new KeeperException.NoNodeException(); + } - return findFirstLevelChildren(path); - } finally { - unlockIfLocked(); + if (watcher != null) { + watchers.put(path, new NodeWatcher(watcher, getSessionId())); } + + return findFirstLevelChildren(path); } @Override @@ -611,40 +600,33 @@ public List getChildren(String path, boolean watch) throws KeeperExcepti @Override public void getChildren(final String path, boolean watcher, final Children2Callback cb, final Object ctx) { - long currentSessionId = getSessionId(); - executor.execute(() -> { - overrideSessionId(currentSessionId); + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null, null); + return; + } + runInExecutorAsync(() -> { try { - lock(); MockZNode mockZNode = tree.get(path); Stat stat = mockZNode != null ? mockZNode.getStat() : null; Optional failure = programmedFailure(Op.GET_CHILDREN, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx, null, null); return; - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null, null); return; } else if (mockZNode == null) { - unlockIfLocked(); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null, null); return; } List children = findFirstLevelChildren(path); - unlockIfLocked(); cb.processResult(0, path, ctx, children, stat); } catch (Throwable ex) { log.error("get children : {} error", path, ex); - unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null, null); - } finally { - removeSessionIdOverride(); } }); - } private List findFirstLevelChildren(String path) { @@ -664,46 +646,30 @@ private List findFirstLevelChildren(String path) { @Override public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException { - lock(); - try { - maybeThrowProgrammedFailure(Op.EXISTS, path); + return runInExecutorReturningValue(() -> internalGetStat(path, null)); + } - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } + private Stat internalGetStat(String path, Watcher watcher) throws KeeperException { + maybeThrowProgrammedFailure(Op.EXISTS, path); - if (tree.containsKey(path)) { - return tree.get(path).getStat(); - } else { - return null; - } - } finally { - unlockIfLocked(); + if (isStopped()) { + throw new KeeperException.ConnectionLossException(); + } + + if (watcher != null) { + watchers.put(path, new NodeWatcher(watcher, getSessionId())); + } + + if (tree.containsKey(path)) { + return tree.get(path).getStat(); + } else { + return null; } } @Override public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException { - lock(); - try { - maybeThrowProgrammedFailure(Op.EXISTS, path); - - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } - - if (watcher != null) { - watchers.put(path, new NodeWatcher(watcher, getSessionId())); - } - - if (tree.containsKey(path)) { - return tree.get(path).getStat(); - } else { - return null; - } - } finally { - unlockIfLocked(); - } + return runInExecutorReturningValue(() -> internalGetStat(path, watcher)); } @Override @@ -713,18 +679,17 @@ public void exists(String path, boolean watch, StatCallback cb, Object ctx) { @Override public void exists(String path, Watcher watcher, StatCallback cb, Object ctx) { - long currentSessionId = getSessionId(); - executor.execute(() -> { - overrideSessionId(currentSessionId); + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); + return; + } + runInExecutorAsync(() -> { try { - lock(); Optional failure = programmedFailure(Op.EXISTS, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx, null); return; - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); return; } @@ -736,83 +701,68 @@ public void exists(String path, Watcher watcher, StatCallback cb, Object ctx) { MockZNode mockZNode = tree.get(path); if (mockZNode != null) { Stat stat = mockZNode.getStat(); - unlockIfLocked(); cb.processResult(0, path, ctx, stat); } else { - unlockIfLocked(); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); } } catch (Throwable ex) { log.error("exist : {} error", path, ex); - unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); - } finally { - removeSessionIdOverride(); } }); } @Override public void sync(String path, VoidCallback cb, Object ctx) { - long currentSessionId = getSessionId(); - executor.execute(() -> { - overrideSessionId(currentSessionId); - try { - Optional failure = programmedFailure(Op.SYNC, path); - if (failure.isPresent()) { - cb.processResult(failure.get().intValue(), path, ctx); - return; - } else if (stopped) { - cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx); - return; - } - - cb.processResult(0, path, ctx); - } finally { - removeSessionIdOverride(); + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx); + return; + } + runInExecutorAsync(() -> { + Optional failure = programmedFailure(Op.SYNC, path); + if (failure.isPresent()) { + cb.processResult(failure.get().intValue(), path, ctx); + return; + } else if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx); + return; } + cb.processResult(0, path, ctx); }); - } @Override public Stat setData(final String path, byte[] data, int version) throws KeeperException, InterruptedException { - final Set toNotify = Sets.newHashSet(); - lock(); - Stat stat; - try { - maybeThrowProgrammedFailure(Op.SET, path); + return runInExecutorReturningValue(() -> internalSetData(path, data, version)); + } - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } + private Stat internalSetData(String path, byte[] data, int version) throws KeeperException { + final Set toNotify = Sets.newHashSet(); + maybeThrowProgrammedFailure(Op.SET, path); - if (!tree.containsKey(path)) { - throw new KeeperException.NoNodeException(); - } + if (isStopped()) { + throw new KeeperException.ConnectionLossException(); + } - MockZNode mockZNode = tree.get(path); - int currentVersion = mockZNode.getVersion(); + if (!tree.containsKey(path)) { + throw new KeeperException.NoNodeException(); + } - // Check version - if (version != -1 && version != currentVersion) { - throw new KeeperException.BadVersionException(path); - } + MockZNode mockZNode = tree.get(path); + int currentVersion = mockZNode.getVersion(); - log.debug("[{}] Updating -- current version: {}", path, currentVersion); - mockZNode.updateData(data); - stat = mockZNode.getStat(); - toNotify.addAll(getWatchers(path)); - watchers.removeAll(path); - } finally { - unlockIfLocked(); + // Check version + if (version != -1 && version != currentVersion) { + throw new KeeperException.BadVersionException(path); } - executor.execute(() -> { - if (stopped) { - return; - } + log.debug("[{}] Updating -- current version: {}", path, currentVersion); + mockZNode.updateData(data); + Stat stat = mockZNode.getStat(); + toNotify.addAll(getWatchers(path)); + watchers.removeAll(path); + runNotifications(() -> { triggerPersistentWatches(path, null, EventType.NodeDataChanged); toNotify.forEach(watcher -> watcher @@ -824,63 +774,48 @@ public Stat setData(final String path, byte[] data, int version) throws KeeperEx @Override public void setData(final String path, final byte[] data, int version, final StatCallback cb, final Object ctx) { - if (stopped) { + if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); return; } - long currentSessionId = getSessionId(); - executor.execute(() -> { - overrideSessionId(currentSessionId); + runInExecutorAsync(() -> { try { final Set toNotify = Sets.newHashSet(); - lock(); Stat stat; - try { - Optional failure = programmedFailure(Op.SET, path); - if (failure.isPresent()) { - unlockIfLocked(); - cb.processResult(failure.get().intValue(), path, ctx, null); - return; - } else if (stopped) { - unlockIfLocked(); - cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); - return; - } - - if (!tree.containsKey(path)) { - unlockIfLocked(); - cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); - return; - } + Optional failure = programmedFailure(Op.SET, path); + if (failure.isPresent()) { + cb.processResult(failure.get().intValue(), path, ctx, null); + return; + } else if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); + return; + } - MockZNode mockZNode = tree.get(path); - int currentVersion = mockZNode.getVersion(); + if (!tree.containsKey(path)) { + cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); + return; + } - // Check version - if (version != -1 && version != currentVersion) { - log.debug("[{}] Current version: {} -- Expected: {}", path, currentVersion, version); - Stat currentStat = mockZNode.getStat(); - unlockIfLocked(); - cb.processResult(KeeperException.Code.BADVERSION.intValue(), path, ctx, currentStat); - return; - } + MockZNode mockZNode = tree.get(path); + int currentVersion = mockZNode.getVersion(); - log.debug("[{}] Updating -- current version: {}", path, currentVersion); - mockZNode.updateData(data); - stat = mockZNode.getStat(); - } finally { - unlockIfLocked(); + // Check version + if (version != -1 && version != currentVersion) { + log.debug("[{}] Current version: {} -- Expected: {}", path, currentVersion, version); + Stat currentStat = mockZNode.getStat(); + cb.processResult(KeeperException.Code.BADVERSION.intValue(), path, ctx, currentStat); + return; } + + log.debug("[{}] Updating -- current version: {}", path, currentVersion); + mockZNode.updateData(data); + stat = mockZNode.getStat(); cb.processResult(0, path, ctx, stat); toNotify.addAll(getWatchers(path)); watchers.removeAll(path); - executor.execute(() -> { - if (stopped) { - return; - } - + runNotifications(() -> { triggerPersistentWatches(path, null, EventType.NodeDataChanged); for (Watcher watcher : toNotify) { @@ -890,58 +825,54 @@ public void setData(final String path, final byte[] data, int version, final Sta } catch (Throwable ex) { log.error("Update data : {} error", path, ex); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx, null); - } finally { - removeSessionIdOverride(); } }); } @Override public void delete(final String path, int version) throws InterruptedException, KeeperException { + runInExecutorReturningValue(() -> { + internalDelete(path, version); + return null; + }); + } + + private void internalDelete(String path, int version) throws KeeperException { maybeThrowProgrammedFailure(Op.DELETE, path); final Set toNotifyDelete; final Set toNotifyParent; final String parent; - lock(); - try { - if (stopped) { - throw new KeeperException.ConnectionLossException(); - } else if (!tree.containsKey(path)) { - throw new KeeperException.NoNodeException(path); - } else if (hasChildren(path)) { - throw new KeeperException.NotEmptyException(path); - } + if (isStopped()) { + throw new KeeperException.ConnectionLossException(); + } else if (!tree.containsKey(path)) { + throw new KeeperException.NoNodeException(path); + } else if (hasChildren(path)) { + throw new KeeperException.NotEmptyException(path); + } - if (version != -1) { - int currentVersion = tree.get(path).getVersion(); - if (version != currentVersion) { - throw new KeeperException.BadVersionException(path); - } + if (version != -1) { + int currentVersion = tree.get(path).getVersion(); + if (version != currentVersion) { + throw new KeeperException.BadVersionException(path); } + } - tree.remove(path); - - toNotifyDelete = Sets.newHashSet(); - toNotifyDelete.addAll(getWatchers(path)); + tree.remove(path); - toNotifyParent = Sets.newHashSet(); - parent = path.substring(0, path.lastIndexOf("/")); - if (!parent.isEmpty()) { - toNotifyParent.addAll(getWatchers(parent)); - } + toNotifyDelete = Sets.newHashSet(); + toNotifyDelete.addAll(getWatchers(path)); - watchers.removeAll(path); - } finally { - unlockIfLocked(); + toNotifyParent = Sets.newHashSet(); + parent = path.substring(0, path.lastIndexOf("/")); + if (!parent.isEmpty()) { + toNotifyParent.addAll(getWatchers(parent)); } - executor.execute(() -> { - if (stopped) { - return; - } + watchers.removeAll(path); + runNotifications(() -> { for (Watcher watcher1 : toNotifyDelete) { watcher1.process(new WatchedEvent(EventType.NodeDeleted, KeeperState.SyncConnected, path)); } @@ -955,11 +886,12 @@ public void delete(final String path, int version) throws InterruptedException, @Override public void delete(final String path, int version, final VoidCallback cb, final Object ctx) { - long currentSessionId = getSessionId(); - Runnable r = () -> { - overrideSessionId(currentSessionId); + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx); + return; + } + runInExecutorAsync(() -> { try { - lock(); final Set toNotifyDelete = Sets.newHashSet(); toNotifyDelete.addAll(getWatchers(path)); @@ -972,37 +904,26 @@ public void delete(final String path, int version, final VoidCallback cb, final Optional failure = programmedFailure(Op.DELETE, path); if (failure.isPresent()) { - unlockIfLocked(); cb.processResult(failure.get().intValue(), path, ctx); - } else if (stopped) { - unlockIfLocked(); + } else if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx); } else if (!tree.containsKey(path)) { - unlockIfLocked(); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx); } else if (hasChildren(path)) { - unlockIfLocked(); cb.processResult(KeeperException.Code.NOTEMPTY.intValue(), path, ctx); } else { if (version != -1) { int currentVersion = tree.get(path).getVersion(); if (version != currentVersion) { - unlockIfLocked(); cb.processResult(KeeperException.Code.BADVERSION.intValue(), path, ctx); return; } } tree.remove(path); - - unlockIfLocked(); cb.processResult(0, path, ctx); - executor.execute(() -> { - if (stopped) { - return; - } - + runNotifications(() -> { triggerPersistentWatches(path, parent, EventType.NodeDeleted); toNotifyDelete.forEach(watcher -> watcher .process(new WatchedEvent(EventType.NodeDeleted, KeeperState.SyncConnected, path))); @@ -1013,33 +934,33 @@ public void delete(final String path, int version, final VoidCallback cb, final } } catch (Throwable ex) { log.error("delete path : {} error", path, ex); - unlockIfLocked(); cb.processResult(KeeperException.Code.SYSTEMERROR.intValue(), path, ctx); - } finally { - removeSessionIdOverride(); } - }; - - try { - executor.execute(r); - } catch (RejectedExecutionException ree) { - cb.processResult(KeeperException.Code.SESSIONEXPIRED.intValue(), path, ctx); - } - + }); } @Override public void multi(Iterable ops, AsyncCallback.MultiCallback cb, Object ctx) { - try { - List res = multi(ops); - cb.processResult(KeeperException.Code.OK.intValue(), null, ctx, res); - } catch (Exception e) { - cb.processResult(KeeperException.Code.APIERROR.intValue(), null, ctx, null); + if (isStopped()) { + cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), null, ctx, null); + return; } + runInExecutorAsync(() -> { + try { + List res = multi(ops); + cb.processResult(KeeperException.Code.OK.intValue(), null, ctx, res); + } catch (Exception e) { + cb.processResult(KeeperException.Code.APIERROR.intValue(), null, ctx, null); + } + }); } @Override public List multi(Iterable ops) throws InterruptedException, KeeperException { + return runInExecutorReturningValue(() -> internalMulti(ops)); + } + + private List internalMulti(Iterable ops) { List res = new ArrayList<>(); for (org.apache.zookeeper.Op op : ops) { switch (op.getType()) { @@ -1108,21 +1029,20 @@ private void handleOperation(String opName, org.apache.zookeeper.Op op, ZkOpHand } @Override - public synchronized void addWatch(String basePath, Watcher watcher, AddWatchMode mode) { - persistentWatchers.add(new PersistentWatcher(basePath, watcher, mode, getSessionId())); + public void addWatch(String basePath, Watcher watcher, AddWatchMode mode) { + runInExecutorSync(() -> { + persistentWatchers.add(new PersistentWatcher(basePath, watcher, mode, getSessionId())); + }); } @Override public void addWatch(String basePath, Watcher watcher, AddWatchMode mode, VoidCallback cb, Object ctx) { - if (stopped) { + if (isStopped()) { cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), basePath, ctx); return; } - long currentSessionId = getSessionId(); - executor.execute(() -> { - synchronized (MockZooKeeper.this) { - persistentWatchers.add(new PersistentWatcher(basePath, watcher, mode, currentSessionId)); - } + runInExecutorAsync(() -> { + addWatch(basePath, watcher, mode); cb.processResult(KeeperException.Code.OK.intValue(), basePath, ctx); }); } @@ -1152,19 +1072,18 @@ public synchronized void close() throws InterruptedException { } public void shutdown() throws InterruptedException { - lock(); - try { - stopped = true; - tree.clear(); - watchers.clear(); - try { - executor.shutdownNow(); - executor.awaitTermination(5, TimeUnit.SECONDS); - } catch (InterruptedException ex) { - log.error("MockZooKeeper shutdown had error", ex); - } - } finally { - unlockIfLocked(); + if (stopped.compareAndSet(false, true)) { + runInExecutorSync(() -> { + tree.clear(); + watchers.clear(); + persistentWatchers.clear(); + try { + executor.shutdownNow(); + executor.awaitTermination(5, TimeUnit.SECONDS); + } catch (InterruptedException ex) { + log.error("MockZooKeeper shutdown had error", ex); + } + }); } } @@ -1237,7 +1156,7 @@ private void checkReadOpDelay() { } private void triggerPersistentWatches(String path, String parent, EventType eventType) { - getPersistentWatchersCopy().forEach(w -> { + persistentWatchers.forEach(w -> { if (w.mode == AddWatchMode.PERSISTENT_RECURSIVE) { if (path.startsWith(w.path())) { w.watcher.process(new WatchedEvent(eventType, KeeperState.SyncConnected, path)); @@ -1256,24 +1175,17 @@ private void triggerPersistentWatches(String path, String parent, EventType even }); } - private synchronized List getPersistentWatchersCopy() { - return new ArrayList<>(persistentWatchers); - } - public void deleteEphemeralNodes(long sessionId) { if (sessionId != NOT_EPHEMERAL) { - lock(); - try { + runInExecutorSync(() -> { tree.values().removeIf(zNode -> zNode.getEphemeralOwner() == sessionId); - } finally { - unlockIfLocked(); - } + }); } } + public synchronized void deleteWatchers(long sessionId) { - lock(); - try { + runInExecutorSync(() -> { // remove all persistent watchers for the session persistentWatchers.removeIf(w -> w.sessionId == sessionId); // remove all watchers for the session @@ -1281,9 +1193,7 @@ public synchronized void deleteWatchers(long sessionId) { watchers.entries().stream().filter(e -> e.getValue().sessionId == sessionId).toList(); watchersForSession .forEach(e -> watchers.remove(e.getKey(), e.getValue())); - } finally { - unlockIfLocked(); - } + }); } private static final Logger log = LoggerFactory.getLogger(MockZooKeeper.class); diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java index e2f267ff566b4..c812423b7280d 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeperSession.java @@ -115,7 +115,7 @@ public void create(final String path, final byte[] data, final List acl, Cr } @Override - public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException { + public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException, InterruptedException { try { mockZooKeeper.overrideSessionId(getSessionId()); return mockZooKeeper.getData(path, watcher, stat); @@ -155,7 +155,7 @@ public void getChildren(final String path, final Watcher watcher, final Children } @Override - public List getChildren(String path, Watcher watcher) throws KeeperException { + public List getChildren(String path, Watcher watcher) throws KeeperException, InterruptedException { try { mockZooKeeper.overrideSessionId(getSessionId()); return mockZooKeeper.getChildren(path, watcher); From ccf1371b705fdd75019c4bc50a543e85dd581eab Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 12:30:47 +0200 Subject: [PATCH 31/56] Improve solution to run pulsar-metadata tests since specific providers --- .../org/apache/pulsar/metadata/BaseMetadataStoreTest.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index 28f160b3bbc7a..03a6b91b62d4b 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -44,6 +44,10 @@ import org.testng.annotations.DataProvider; public abstract class BaseMetadataStoreTest extends TestRetrySupport { + // to debug specific implementations, set the TEST_METADATA_PROVIDERS environment variable + // or temporarily hard code this value in the test class before running tests in the IDE + // supported values are ZooKeeper,Memory,RocksDB,Etcd,Oxia,MockZooKeeper + private static final String TEST_METADATA_PROVIDERS = System.getenv("TEST_METADATA_PROVIDERS"); private static String originalMetadatastoreProvidersPropertyValue; protected TestZKServer zks; protected EtcdCluster etcdCluster; @@ -119,8 +123,8 @@ public void cleanup() throws Exception { @DataProvider(name = "impl") public Object[][] implementations() { // If the environment variable TEST_METADATA_PROVIDERS is set, only run the specified implementations - if (StringUtils.isNotBlank(System.getenv("TEST_METADATA_PROVIDERS"))) { - return filterImplementations(System.getenv("TEST_METADATA_PROVIDERS").split(",")); + if (StringUtils.isNotBlank(TEST_METADATA_PROVIDERS)) { + return filterImplementations(TEST_METADATA_PROVIDERS.split(",")); } return allImplementations(); } From 3994410a59759eaf9373e49992364d90a461fcb6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 12:37:33 +0200 Subject: [PATCH 32/56] Shutdown outside of executor thread --- .../java/org/apache/zookeeper/MockZooKeeper.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 461ccc1850cfa..2eda0986f337e 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -1077,13 +1077,13 @@ public void shutdown() throws InterruptedException { tree.clear(); watchers.clear(); persistentWatchers.clear(); - try { - executor.shutdownNow(); - executor.awaitTermination(5, TimeUnit.SECONDS); - } catch (InterruptedException ex) { - log.error("MockZooKeeper shutdown had error", ex); - } }); + try { + executor.shutdownNow(); + executor.awaitTermination(5, TimeUnit.SECONDS); + } catch (InterruptedException ex) { + log.error("MockZooKeeper shutdown had error", ex); + } } } From becc2214ef9e34427839860150e9dee07c3edd89 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 13:20:11 +0200 Subject: [PATCH 33/56] Fix hasChildren in MockZooKeeper --- .../src/main/java/org/apache/zookeeper/MockZooKeeper.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 2eda0986f337e..092345212dd69 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -644,6 +644,10 @@ private List findFirstLevelChildren(String path) { return children; } + private boolean hasChildren(String path) { + return !findFirstLevelChildren(path).isEmpty(); + } + @Override public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException { return runInExecutorReturningValue(() -> internalGetStat(path, null)); @@ -1136,10 +1140,6 @@ public void setSessionId(long id) { sessionId = id; } - private boolean hasChildren(String path) { - return !tree.subMap(path + '/', path + '0').isEmpty(); - } - @Override public String toString() { return "MockZookeeper"; From cfc246758b4cc76786d997494ab1fa91defcae08 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 14:47:44 +0200 Subject: [PATCH 34/56] Pass path in NoNodeException --- .../src/main/java/org/apache/zookeeper/MockZooKeeper.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 092345212dd69..9d1390b938633 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -324,7 +324,7 @@ private String internalCreate(String path, byte[] data, CreateMode createMode) t } if (!parent.isEmpty() && !tree.containsKey(parent)) { - throw new KeeperException.NoNodeException(); + throw new KeeperException.NoNodeException(parent); } if (createMode.isSequential()) { @@ -583,7 +583,7 @@ private List internalGetChildren(String path, Watcher watcher) throws Ke maybeThrowProgrammedFailure(Op.GET_CHILDREN, path); if (!tree.containsKey(path)) { - throw new KeeperException.NoNodeException(); + throw new KeeperException.NoNodeException(path); } if (watcher != null) { @@ -749,7 +749,7 @@ private Stat internalSetData(String path, byte[] data, int version) throws Keepe } if (!tree.containsKey(path)) { - throw new KeeperException.NoNodeException(); + throw new KeeperException.NoNodeException(path); } MockZNode mockZNode = tree.get(path); From 18ab5f32999dd9ce802be5c1c9b915f4c292df6d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 14:59:46 +0200 Subject: [PATCH 35/56] Add logging to see what happens --- .../bookkeeper/LedgerManagerIteratorTest.java | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java index f8a51602686ed..b955ccf0504ac 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java @@ -407,14 +407,16 @@ public void checkConcurrentModifications(String provider, Supplier urlSu ExecutorService executor = Executors.newCachedThreadPool(); final ConcurrentSkipListSet createdLedgers = new ConcurrentSkipListSet<>(); for (int i = 0; i < numWriters; ++i) { + int writerIndex = i; Future f = executor.submit(() -> { @Cleanup LedgerManager writerLM = new PulsarLedgerManager(store, ledgersRoot); Random writerRNG = new Random(rng.nextLong()); - + log.info("Writer {} waiting", writerIndex); latch.await(); - + log.info("Writer {} started", writerIndex); while (MathUtils.elapsedNanos(start) < runtime) { + log.info("Writer {} writing", writerIndex); long candidate = 0; do { candidate = Math.abs(writerRNG.nextLong()); @@ -426,18 +428,22 @@ public void checkConcurrentModifications(String provider, Supplier urlSu createLedger(writerLM, candidate); removeLedger(writerLM, candidate); } + log.info("Writer {} finished", writerIndex); return null; }); futures.add(f); } for (int i = 0; i < numCheckers; ++i) { + int checkerIndex = i; Future f = executor.submit(() -> { @Cleanup LedgerManager checkerLM = new PulsarLedgerManager(store, ledgersRoot); + log.info("Checker {} waiting", checkerIndex); latch.await(); - + log.info("Checker {} started", checkerIndex); while (MathUtils.elapsedNanos(start) < runtime) { + log.info("Checker {} checking", checkerIndex); LedgerRangeIterator lri = checkerLM.getLedgerRanges(0); Set returnedIds = ledgerRangeToSet(lri); for (long id : mustExist) { @@ -449,15 +455,19 @@ public void checkConcurrentModifications(String provider, Supplier urlSu assertTrue(ledgersReadAsync.contains(id)); } } + log.info("Checker {} finished", checkerIndex); return null; }); futures.add(f); } latch.countDown(); + log.info("Waiting for futures"); for (Future f : futures) { + log.info("Waiting for future"); f.get(); } + log.info("Completed"); executor.shutdownNow(); } From 4033fc3be1e3d838267bfe349a28745dd36a6ca0 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 15:12:39 +0200 Subject: [PATCH 36/56] Increase timeout to make the test pass on MockZooKeeper --- .../pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java index b955ccf0504ac..f4bac7bb622aa 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java @@ -373,7 +373,7 @@ public void testWithSeveralIncompletePaths(String provider, Supplier url assertEquals(ledgersReadAsync, ids, "Comparing LedgersIds read asynchronously"); } - @Test(timeOut = 30000, dataProvider = "impl") + @Test(timeOut = 60000, dataProvider = "impl") public void checkConcurrentModifications(String provider, Supplier urlSupplier) throws Throwable { @Cleanup MetadataStoreExtended store = From 02d01e32973a0f583e6fb48195673c1dd7ec9d3c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 15:04:46 +0200 Subject: [PATCH 37/56] Reduce logging in MockZooKeeper for multiops --- .../src/main/java/org/apache/zookeeper/MockZooKeeper.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 9d1390b938633..bd5a7404577b0 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -1021,12 +1021,11 @@ private void handleOperation(String opName, org.apache.zookeeper.Op op, ZkOpHand try { handler.handle(); } catch (Exception e) { - log.error("Error handling {} operation for path {} type {} kind {} request {} {}", opName, op.getPath(), - op.getType(), op.getKind(), op.toRequestRecord(), - e instanceof KeeperException ? e.getMessage() : e); if (e instanceof KeeperException keeperException) { res.add(new OpResult.ErrorResult(keeperException.code().intValue())); } else { + log.error("Error handling {} operation for path {} type {} kind {} request {}", opName, op.getPath(), + op.getType(), op.getKind(), op.toRequestRecord(), e); res.add(new OpResult.ErrorResult(KeeperException.Code.RUNTIMEINCONSISTENCY.intValue())); } } From 385236047196a20fc1502ce560f384166a0446d1 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 15:10:59 +0200 Subject: [PATCH 38/56] Disable invalid test BrokerServiceLookupTest.testLookupConnectionNotCloseIfFailedToAcquireOwnershipOfBundle --- .../org/apache/pulsar/client/api/BrokerServiceLookupTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index c3bedb324c615..45c30a0670eae 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -1342,7 +1342,8 @@ private void makeAcquireBundleLockSuccess() throws Exception { } } - @Test(timeOut = 30000) + // TODO: This test is disabled since it's invalid. The test fails for both TestZKServer and MockZooKeeper. + @Test(timeOut = 30000, enabled = false) public void testLookupConnectionNotCloseIfFailedToAcquireOwnershipOfBundle() throws Exception { String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); admin.topics().createNonPartitionedTopic(tpName); From 28bb9285e43ae04f0a2ee9c2629e4a344581db52 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 15 Feb 2025 15:26:31 +0200 Subject: [PATCH 39/56] Optimize MockZooKeeper getChildren --- .../org/apache/zookeeper/MockZooKeeper.java | 61 ++++++++++++------- 1 file changed, 38 insertions(+), 23 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index bd5a7404577b0..bff694da1eeb0 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -72,18 +72,19 @@ private static class MockZNode { long ephemeralOwner; long creationTimestamp; long modificationTimestamp; + List children; static MockZNode of(byte[] content, int version, long ephemeralOwner) { return new MockZNode(content, version, ephemeralOwner, System.currentTimeMillis(), - System.currentTimeMillis()); + System.currentTimeMillis(), new ArrayList<>()); } - public synchronized void updateVersion() { + public void updateVersion() { version++; modificationTimestamp = System.currentTimeMillis(); } - public synchronized void updateData(byte[] data) { + public void updateData(byte[] data) { content = data; updateVersion(); } @@ -92,7 +93,7 @@ public Stat getStat() { return applyToStat(new Stat()); } - public synchronized Stat applyToStat(Stat stat) { + public Stat applyToStat(Stat stat) { stat.setCtime(creationTimestamp); stat.setMtime(modificationTimestamp); stat.setVersion(version); @@ -100,17 +101,21 @@ public synchronized Stat applyToStat(Stat stat) { return stat; } - public synchronized int getVersion() { + public int getVersion() { return version; } - public synchronized byte[] getContent() { + public byte[] getContent() { return content; } public long getEphemeralOwner() { return ephemeralOwner; } + + public List getChildren() { + return children; + } } private TreeMap tree; @@ -311,7 +316,7 @@ private void runInExecutorSync(Runnable runnable) { private String internalCreate(String path, byte[] data, CreateMode createMode) throws KeeperException { final Set toNotifyCreate = Sets.newHashSet(); final Set toNotifyParent = Sets.newHashSet(); - final String parent = path.substring(0, path.lastIndexOf("/")); + final String parent = getParentName(path); maybeThrowProgrammedFailure(Op.CREATE, path); @@ -327,13 +332,17 @@ private String internalCreate(String path, byte[] data, CreateMode createMode) t throw new KeeperException.NoNodeException(parent); } + MockZNode parentNode = tree.get(parent); + if (createMode.isSequential()) { - MockZNode parentNode = tree.get(parent); int parentVersion = tree.get(parent).getVersion(); path = path + parentVersion; parentNode.updateVersion(); } + if (parentNode != null) { + parentNode.getChildren().add(getNodeName(path)); + } tree.put(path, createMockZNode(data, createMode)); toNotifyCreate.addAll(getWatchers(path)); @@ -365,6 +374,14 @@ private String internalCreate(String path, byte[] data, CreateMode createMode) t return path; } + private static String getParentName(String path) { + return path.substring(0, path.lastIndexOf("/")); + } + + private static String getNodeName(String path) { + return path.substring(path.lastIndexOf('/') + 1); + } + private Collection getWatchers(String path) { Set nodeWatchers = watchers.get(path); if (nodeWatchers != null) { @@ -409,7 +426,7 @@ public void create(final String path, final byte[] data, final List acl, Cr toNotifyCreate.addAll(getWatchers(path)); final Set toNotifyParent = Sets.newHashSet(); - final String parent = path.substring(0, path.lastIndexOf("/")); + final String parent = getParentName(path); if (!parent.isEmpty()) { toNotifyParent.addAll(getWatchers(parent)); } @@ -436,6 +453,9 @@ public void create(final String path, final byte[] data, final List acl, Cr }); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); } else { + if (!parent.isEmpty()) { + tree.get(parent).getChildren().add(getNodeName(name)); + } tree.put(name, createMockZNode(data, createMode)); watchers.removeAll(name); cb.processResult(0, path, ctx, name); @@ -630,22 +650,11 @@ public void getChildren(final String path, boolean watcher, final Children2Callb } private List findFirstLevelChildren(String path) { - List children = new ArrayList<>(); - String requiredPrefix = path.equals("/") ? "/" : path + "/"; - String lastKey = path.equals("/") ? "0" : path + "0"; // '0' is lexicographically just after '/' - for (String key : tree.subMap(requiredPrefix, false, lastKey, false).keySet()) { - if (key.startsWith(requiredPrefix)) { - String relativePath = key.substring(requiredPrefix.length()); - if (relativePath.indexOf('/') == -1) { - children.add(relativePath); - } - } - } - return children; + return new ArrayList<>(tree.get(path).getChildren()); } private boolean hasChildren(String path) { - return !findFirstLevelChildren(path).isEmpty(); + return !tree.get(path).getChildren().isEmpty(); } @Override @@ -863,13 +872,16 @@ private void internalDelete(String path, int version) throws KeeperException { } } + parent = getParentName(path); tree.remove(path); + if (!parent.isEmpty()) { + tree.get(parent).getChildren().remove(getNodeName(path)); + } toNotifyDelete = Sets.newHashSet(); toNotifyDelete.addAll(getWatchers(path)); toNotifyParent = Sets.newHashSet(); - parent = path.substring(0, path.lastIndexOf("/")); if (!parent.isEmpty()) { toNotifyParent.addAll(getWatchers(parent)); } @@ -925,6 +937,9 @@ public void delete(final String path, int version, final VoidCallback cb, final } tree.remove(path); + if (!parent.isEmpty()) { + tree.get(parent).getChildren().remove(getNodeName(path)); + } cb.processResult(0, path, ctx); runNotifications(() -> { From 7d833f82a847c7e8cd8ea1f8537f619ca3db2083 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 08:17:01 +0200 Subject: [PATCH 40/56] Fix some NPEs at startup when LeaderElectionService isn't yet available --- .../pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java | 2 +- .../java/org/apache/pulsar/broker/web/PulsarWebResource.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index 48a6121b9dd13..f86b608d93722 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -583,7 +583,7 @@ private void updateBundleData() { for (String bundle : bundleData.keySet()) { if (!activeBundles.contains(bundle)){ bundleData.remove(bundle); - if (pulsar.getLeaderElectionService().isLeader()){ + if (pulsar.getLeaderElectionService() != null && pulsar.getLeaderElectionService().isLeader()){ deleteBundleDataFromMetadataStore(bundle); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java index 2e198eb99752e..f191fc3a38423 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java @@ -1012,7 +1012,7 @@ protected static boolean isLeaderBroker(PulsarService pulsar) { if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return true; } - return pulsar.getLeaderElectionService().isLeader(); + return pulsar.getLeaderElectionService() != null && pulsar.getLeaderElectionService().isLeader(); } public void validateTenantOperation(String tenant, TenantOperation operation) { From 87fad55686a47b171e6e0e292d9b7c1153874fde Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 08:24:41 +0200 Subject: [PATCH 41/56] Add better exception message --- .../pulsar/broker/loadbalance/impl/LoadManagerShared.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java index 7ca2b926db7db..591b061253d3b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java @@ -285,7 +285,7 @@ public static CompletableFuture> applyNamespacePoliciesAsync( public static String getBundleRangeFromBundleName(String bundleName) { // the bundle format is property/cluster/namespace/0x00000000_0xFFFFFFFF int pos = bundleName.lastIndexOf("/"); - checkArgument(pos != -1); + checkArgument(pos != -1, "Invalid bundle name format: %s", bundleName); return bundleName.substring(pos + 1); } @@ -293,7 +293,7 @@ public static String getBundleRangeFromBundleName(String bundleName) { public static String getNamespaceNameFromBundleName(String bundleName) { // the bundle format is property/cluster/namespace/0x00000000_0xFFFFFFFF int pos = bundleName.lastIndexOf('/'); - checkArgument(pos != -1); + checkArgument(pos != -1, "Invalid bundle name format: %s", bundleName); return bundleName.substring(0, pos); } From 6db5a4db109523101501069e1bdb09bb13cfbf7d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 08:34:06 +0200 Subject: [PATCH 42/56] Fix exceptions caused by invalid bundle keys --- .../org/apache/pulsar/client/api/BrokerServiceLookupTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index 45c30a0670eae..c25908ab56cbf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -92,6 +92,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.PoliciesUtil; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.common.util.SecurityUtility; @@ -322,8 +323,9 @@ public void testMultipleBrokerLookup() throws Exception { @Test public void testConcurrentWriteBrokerData() throws Exception { Map map = new ConcurrentHashMap<>(); + List boundaries = PoliciesUtil.getBundles(100).getBoundaries(); for (int i = 0; i < 100; i++) { - map.put("key"+ i, new NamespaceBundleStats()); + map.put("my-property/my-ns/" + boundaries.get(i), new NamespaceBundleStats()); } BrokerService brokerService = mock(BrokerService.class); doReturn(brokerService).when(pulsar).getBrokerService(); From 5212fa2bbf8c8445709f50624d1e1aac168855f7 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 08:58:35 +0200 Subject: [PATCH 43/56] Allow proper shutdown of BrokerService in BrokerServiceLookupTest.testConcurrentWriteBrokerData --- .../org/apache/pulsar/client/api/BrokerServiceLookupTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index c25908ab56cbf..2d3f235ee141f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -327,6 +327,7 @@ public void testConcurrentWriteBrokerData() throws Exception { for (int i = 0; i < 100; i++) { map.put("my-property/my-ns/" + boundaries.get(i), new NamespaceBundleStats()); } + BrokerService originalBrokerService = pulsar.getBrokerService(); BrokerService brokerService = mock(BrokerService.class); doReturn(brokerService).when(pulsar).getBrokerService(); doReturn(map).when(brokerService).getBundleStats(); @@ -357,6 +358,8 @@ public void testConcurrentWriteBrokerData() throws Exception { for (Future future : list) { future.get(); } + // allow proper shutdown so that resources aren't leaked + doReturn(originalBrokerService).when(pulsar).getBrokerService(); } /** From f598d95324a5e9966f7e2687828203aa077b60cc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 09:13:51 +0200 Subject: [PATCH 44/56] Refactor restarting broker with different config --- .../auth/MockedPulsarServiceBaseTest.java | 7 +++ .../client/api/BrokerServiceLookupTest.java | 58 +++++++++---------- 2 files changed, 36 insertions(+), 29 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index 11ab325324f5b..5be5c7544524f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -366,7 +366,14 @@ protected void afterPulsarStart(PulsarService pulsar) throws Exception { * @throws Exception if an error occurs */ protected void restartBroker() throws Exception { + restartBroker(null); + } + + protected void restartBroker(Consumer configurationChanger) throws Exception { stopBroker(); + if (configurationChanger != null) { + configurationChanger.accept(conf); + } startBroker(); if (pulsarClient == null) { pulsarClient = newPulsarClient(lookupUrl.toString(), 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index 2d3f235ee141f..1974aeaf6aead 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -407,11 +407,11 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { @Cleanup PulsarClient pulsarClient2 = PulsarClient.builder().serviceUrl(brokerServiceUrl.toString()).build(); - // enable authorization: so, broker can validate cluster and redirect if finds different cluster - pulsar.getConfiguration().setAuthorizationEnabled(true); // restart broker with authorization enabled: it initialize AuthorizationService - stopBroker(); - startBroker(); + restartBroker(conf -> { + // enable authorization: so, broker can validate cluster and redirect if finds different cluster + conf.setAuthorizationEnabled(true); + }); LoadManager loadManager2 = spy(pulsar2.getLoadManager().get()); Field loadManagerField = NamespaceService.class.getDeclaredField("loadManager"); @@ -570,18 +570,18 @@ public void testWebserviceServiceTls() throws Exception { PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(conf2); PulsarService pulsar2 = pulsarTestContext2.getPulsarService(); - // restart broker1 with tls enabled - conf.setBrokerServicePortTls(Optional.of(0)); - conf.setWebServicePortTls(Optional.of(0)); - conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); - conf.setTlsRequireTrustedClientCertOnConnect(true); - conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); - conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); - conf.setNumExecutorThreadPoolSize(5); - // Not in use, and because TLS is not configured, it will fail to start - conf.setSystemTopicEnabled(false); - stopBroker(); - startBroker(); + restartBroker(conf -> { + // restart broker1 with tls enabled + conf.setBrokerServicePortTls(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); + conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + conf.setTlsRequireTrustedClientCertOnConnect(true); + conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); + conf.setNumExecutorThreadPoolSize(5); + // Not in use, and because TLS is not configured, it will fail to start + conf.setSystemTopicEnabled(false); + }); pulsar.getLoadManager().get().writeLoadReportOnZookeeper(); pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); @@ -786,9 +786,9 @@ public void testModularLoadManagerSplitBundle() throws Exception { conf2.setLoadBalancerNamespaceBundleMaxTopics(1); // configure broker-1 with ModularLoadManager - stopBroker(); - conf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); - startBroker(); + restartBroker(conf -> { + conf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); + }); @Cleanup PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(conf2); @@ -908,11 +908,11 @@ public void testSkipSplitBundleIfOnlyOneBroker() throws Exception { final String topicName2 = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp_"); try { // configure broker with ModularLoadManager. - stopBroker(); - conf.setDefaultNumberOfNamespaceBundles(1); - conf.setLoadBalancerNamespaceBundleMaxTopics(1); - conf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); - startBroker(); + restartBroker(conf -> { + conf.setDefaultNumberOfNamespaceBundles(1); + conf.setLoadBalancerNamespaceBundleMaxTopics(1); + conf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); + }); final ModularLoadManagerWrapper modularLoadManagerWrapper = (ModularLoadManagerWrapper) pulsar.getLoadManager().get(); final ModularLoadManagerImpl modularLoadManager = @@ -1065,11 +1065,11 @@ public void testPartitionedMetadataWithDeprecatedVersion() throws Exception { admin.namespaces().createNamespace(property + "/" + cluster + "/" + namespace); admin.topics().createPartitionedTopic(dest.toString(), totalPartitions); - stopBroker(); - conf.setBrokerServicePortTls(Optional.empty()); - conf.setWebServicePortTls(Optional.empty()); - conf.setClientLibraryVersionCheckEnabled(true); - startBroker(); + restartBroker(conf -> { + conf.setBrokerServicePortTls(Optional.empty()); + conf.setWebServicePortTls(Optional.empty()); + conf.setClientLibraryVersionCheckEnabled(true); + }); URI brokerServiceUrl = new URI(pulsar.getSafeWebServiceAddress()); From 358d057e1940433dea6599873b58cd5aa3c9f8bc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 10:05:37 +0200 Subject: [PATCH 45/56] Improve MockZooKeeper shutdown --- .../main/java/org/apache/zookeeper/MockZooKeeper.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index bff694da1eeb0..a84177f4f3426 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -23,6 +23,7 @@ import com.google.common.collect.Multimaps; import com.google.common.collect.SetMultimap; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.MoreExecutors; import io.netty.util.concurrent.DefaultThreadFactory; import java.util.ArrayList; import java.util.Collection; @@ -233,6 +234,9 @@ private T runInExecutorReturningValue(Callable task) throws InterruptedEx private T runInExecutorReturningValue(Callable task, boolean allowRunningInCurrentThread) throws InterruptedException, KeeperException { + if (isStopped()) { + throw new KeeperException.ConnectionLossException(); + } if (allowRunningInCurrentThread && inExecutorThreadLocal.get()) { try { return task.call(); @@ -1096,12 +1100,7 @@ public void shutdown() throws InterruptedException { watchers.clear(); persistentWatchers.clear(); }); - try { - executor.shutdownNow(); - executor.awaitTermination(5, TimeUnit.SECONDS); - } catch (InterruptedException ex) { - log.error("MockZooKeeper shutdown had error", ex); - } + MoreExecutors.shutdownAndAwaitTermination(executor, 10, TimeUnit.SECONDS); } } From c3f24b15b3526b01fb88bd2fee6d736cbbe781c5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 10:07:33 +0200 Subject: [PATCH 46/56] Remove unnecessary parameter --- .../java/org/apache/zookeeper/MockZooKeeper.java | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index a84177f4f3426..355eb3dd5a23c 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -228,16 +228,12 @@ public String create(String path, byte[] data, List acl, CreateMode createM return runInExecutorReturningValue(() -> internalCreate(path, data, createMode)); } - private T runInExecutorReturningValue(Callable task) throws InterruptedException, KeeperException { - return runInExecutorReturningValue(task, true); - } - - private T runInExecutorReturningValue(Callable task, boolean allowRunningInCurrentThread) + private T runInExecutorReturningValue(Callable task) throws InterruptedException, KeeperException { if (isStopped()) { throw new KeeperException.ConnectionLossException(); } - if (allowRunningInCurrentThread && inExecutorThreadLocal.get()) { + if (inExecutorThreadLocal.get()) { try { return task.call(); } catch (Exception e) { @@ -277,11 +273,7 @@ private T runInExecutorReturningValue(Callable task, boolean allowRunning } private void runInExecutorAsync(Runnable runnable) { - runInExecutorAsync(runnable, true); - } - - private void runInExecutorAsync(Runnable runnable, boolean allowRunningInCurrentThread) { - if (allowRunningInCurrentThread && inExecutorThreadLocal.get()) { + if (inExecutorThreadLocal.get()) { try { runnable.run(); } catch (Throwable t) { From 9b58676b41f73747fb1812dbfc8e4d635e5d5c7b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 10:11:06 +0200 Subject: [PATCH 47/56] Reject tasks if stopped --- .../src/main/java/org/apache/zookeeper/MockZooKeeper.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 355eb3dd5a23c..69e66dc373e59 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -38,6 +38,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -273,6 +274,9 @@ private T runInExecutorReturningValue(Callable task) } private void runInExecutorAsync(Runnable runnable) { + if (isStopped()) { + throw new RejectedExecutionException("MockZooKeeper is stopped"); + } if (inExecutorThreadLocal.get()) { try { runnable.run(); From 70f63219624861eea857c20e1e1156c2a204bdca Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 10:14:50 +0200 Subject: [PATCH 48/56] Improve exception message --- .../org/apache/pulsar/common/naming/NamespaceBundles.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java index 3ee365cdd4571..27c73edc6b597 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java @@ -108,9 +108,11 @@ public int size() { public void validateBundle(NamespaceBundle nsBundle) throws Exception { int idx = Arrays.binarySearch(partitions, nsBundle.getLowerEndpoint()); - checkArgument(idx >= 0, "Cannot find bundle in the bundles list"); - checkArgument(nsBundle.getUpperEndpoint().equals(bundles.get(idx).getUpperEndpoint()), - "Invalid upper boundary for bundle"); + checkArgument(idx >= 0, "Cannot find bundle %s in the bundles list", nsBundle); + NamespaceBundle foundBundle = bundles.get(idx); + Long upperEndpoint = foundBundle.getUpperEndpoint(); + checkArgument(nsBundle.getUpperEndpoint().equals(upperEndpoint), + "Invalid upper boundary for bundle %s. Expected upper boundary of %s", nsBundle, foundBundle); } public NamespaceBundle getFullBundle() { From 2c45034ab091139b14f3beb843caf4a33a137e2e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 10:22:08 +0200 Subject: [PATCH 49/56] Fix shutdown --- .../src/main/java/org/apache/zookeeper/MockZooKeeper.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 69e66dc373e59..e777c481ce1ec 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -38,6 +38,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -1091,11 +1092,16 @@ public synchronized void close() throws InterruptedException { public void shutdown() throws InterruptedException { if (stopped.compareAndSet(false, true)) { - runInExecutorSync(() -> { + Future shutdownTask = executor.submit(() -> { tree.clear(); watchers.clear(); persistentWatchers.clear(); }); + try { + shutdownTask.get(); + } catch (ExecutionException e) { + log.error("Error shutting down", e); + } MoreExecutors.shutdownAndAwaitTermination(executor, 10, TimeUnit.SECONDS); } } From a631efaa876cab9b3f6aec6605a64fdcccaf3aca Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 10:40:25 +0200 Subject: [PATCH 50/56] Avoid unnecessary broker restarts in BrokerServiceLookupTest --- .../client/api/BrokerServiceLookupTest.java | 73 +++++++++---------- .../client/api/ProducerConsumerBase.java | 2 +- 2 files changed, 34 insertions(+), 41 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index 1974aeaf6aead..d0cb4266ae10a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -147,7 +147,7 @@ public void applyTestName(Method method) { testName = method.getName() + " with " + (useTestZookeeper ? "TestZKServer" : "MockZooKeeper"); } - @BeforeMethod + @BeforeMethod(dependsOnMethods = "setTestMethodName") @Override protected void setup() throws Exception { conf.setDefaultNumberOfNamespaceBundles(1); @@ -156,6 +156,38 @@ protected void setup() throws Exception { producerBaseSetup(); } + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + switch (methodName) { + case "testMultipleBrokerDifferentClusterLookup" -> { + conf.setAuthenticationEnabled(true); + } + case "testWebserviceServiceTls" -> { + // broker1 with tls enabled + conf.setBrokerServicePortTls(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); + conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + conf.setTlsRequireTrustedClientCertOnConnect(true); + conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); + conf.setNumExecutorThreadPoolSize(5); + // Not in use, and because TLS is not configured, it will fail to start + conf.setSystemTopicEnabled(false); + } + case "testSkipSplitBundleIfOnlyOneBroker" -> { + conf.setDefaultNumberOfNamespaceBundles(1); + conf.setLoadBalancerNamespaceBundleMaxTopics(1); + conf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); + } + case "testPartitionedMetadataWithDeprecatedVersion" -> { + conf.setBrokerServicePortTls(Optional.empty()); + conf.setWebServicePortTls(Optional.empty()); + conf.setClientLibraryVersionCheckEnabled(true); + } + } + } + @AfterMethod(alwaysRun = true) @Override protected void cleanup() throws Exception { @@ -407,12 +439,6 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { @Cleanup PulsarClient pulsarClient2 = PulsarClient.builder().serviceUrl(brokerServiceUrl.toString()).build(); - // restart broker with authorization enabled: it initialize AuthorizationService - restartBroker(conf -> { - // enable authorization: so, broker can validate cluster and redirect if finds different cluster - conf.setAuthorizationEnabled(true); - }); - LoadManager loadManager2 = spy(pulsar2.getLoadManager().get()); Field loadManagerField = NamespaceService.class.getDeclaredField("loadManager"); loadManagerField.setAccessible(true); @@ -449,10 +475,6 @@ public void testMultipleBrokerDifferentClusterLookup() throws Exception { consumer.acknowledgeCumulative(msg); consumer.close(); producer.close(); - - // disable authorization - pulsar.getConfiguration().setAuthorizationEnabled(false); - loadManager2 = null; } /** @@ -570,18 +592,6 @@ public void testWebserviceServiceTls() throws Exception { PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(conf2); PulsarService pulsar2 = pulsarTestContext2.getPulsarService(); - restartBroker(conf -> { - // restart broker1 with tls enabled - conf.setBrokerServicePortTls(Optional.of(0)); - conf.setWebServicePortTls(Optional.of(0)); - conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); - conf.setTlsRequireTrustedClientCertOnConnect(true); - conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); - conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); - conf.setNumExecutorThreadPoolSize(5); - // Not in use, and because TLS is not configured, it will fail to start - conf.setSystemTopicEnabled(false); - }); pulsar.getLoadManager().get().writeLoadReportOnZookeeper(); pulsar2.getLoadManager().get().writeLoadReportOnZookeeper(); @@ -785,11 +795,6 @@ public void testModularLoadManagerSplitBundle() throws Exception { conf2.setLoadBalancerAutoUnloadSplitBundlesEnabled(true); conf2.setLoadBalancerNamespaceBundleMaxTopics(1); - // configure broker-1 with ModularLoadManager - restartBroker(conf -> { - conf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); - }); - @Cleanup PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(conf2); PulsarService pulsar2 = pulsarTestContext2.getPulsarService(); @@ -907,12 +912,6 @@ public void testSkipSplitBundleIfOnlyOneBroker() throws Exception { final String topicName1 = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp_"); final String topicName2 = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp_"); try { - // configure broker with ModularLoadManager. - restartBroker(conf -> { - conf.setDefaultNumberOfNamespaceBundles(1); - conf.setLoadBalancerNamespaceBundleMaxTopics(1); - conf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); - }); final ModularLoadManagerWrapper modularLoadManagerWrapper = (ModularLoadManagerWrapper) pulsar.getLoadManager().get(); final ModularLoadManagerImpl modularLoadManager = @@ -1065,12 +1064,6 @@ public void testPartitionedMetadataWithDeprecatedVersion() throws Exception { admin.namespaces().createNamespace(property + "/" + cluster + "/" + namespace); admin.topics().createPartitionedTopic(dest.toString(), totalPartitions); - restartBroker(conf -> { - conf.setBrokerServicePortTls(Optional.empty()); - conf.setWebServicePortTls(Optional.empty()); - conf.setClientLibraryVersionCheckEnabled(true); - }); - URI brokerServiceUrl = new URI(pulsar.getSafeWebServiceAddress()); URL url = brokerServiceUrl.toURL(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java index 0cf2e49d35bee..01d2f107dffe0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java @@ -40,7 +40,7 @@ public abstract class ProducerConsumerBase extends MockedPulsarServiceBaseTest { protected String methodName; @BeforeMethod(alwaysRun = true) - public void beforeMethod(Method m) throws Exception { + public void setTestMethodName(Method m) throws Exception { methodName = m.getName(); } From 2c7b04200ffd321068cd72c595cd86d61db63665 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 13:30:21 +0200 Subject: [PATCH 51/56] Properly implement reusing TestZKServer in another context - ensures that it has a separate session --- .../broker/testcontext/PulsarTestContext.java | 61 +++++++++++-------- 1 file changed, 37 insertions(+), 24 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java index de8b24edd7888..f8205a2e939a3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java @@ -35,6 +35,7 @@ import lombok.Builder; import lombok.Getter; import lombok.Singular; +import lombok.SneakyThrows; import lombok.ToString; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BookKeeper; @@ -162,6 +163,10 @@ public class PulsarTestContext implements AutoCloseable { private final MockZooKeeper mockZooKeeperGlobal; + private final TestZKServer testZKServer; + + private final TestZKServer testZKServerGlobal; + private final SpyConfig spyConfig; private final boolean startable; @@ -410,6 +415,11 @@ public Builder reuseMockBookkeeperAndMetadataStores(PulsarTestContext otherConte if (otherContext.getMockZooKeeperGlobal() != null) { mockZooKeeperGlobal(otherContext.getMockZooKeeperGlobal()); } + } else if (otherContext.getTestZKServer() != null) { + testZKServer(otherContext.getTestZKServer()); + if (otherContext.getTestZKServerGlobal() != null) { + testZKServerGlobal(otherContext.getTestZKServerGlobal()); + } } else { localMetadataStore(NonClosingProxyHandler.createNonClosingProxy(otherContext.getLocalMetadataStore(), MetadataStoreExtended.class @@ -509,20 +519,10 @@ public Builder withTestZookeeper() { */ public Builder withTestZookeeper(boolean useSeparateGlobalZk) { try { - TestZKServer localZk = createTestZookeeper(); - MetadataStoreExtended localStore = - createTestZookeeperMetadataStore(localZk, MetadataStoreConfig.METADATA_STORE); - localMetadataStore(localStore); - MetadataStoreExtended configStore; + testZKServer(createTestZookeeper()); if (useSeparateGlobalZk) { - TestZKServer globalZk = createTestZookeeper(); - configStore = createTestZookeeperMetadataStore(globalZk, - MetadataStoreConfig.CONFIGURATION_METADATA_STORE); - } else { - configStore = - createTestZookeeperMetadataStore(localZk, MetadataStoreConfig.CONFIGURATION_METADATA_STORE); + testZKServerGlobal(createTestZookeeper()); } - configurationMetadataStore(configStore); } catch (Exception e) { throw new RuntimeException(e); } @@ -539,18 +539,6 @@ private TestZKServer createTestZookeeper() throws Exception { return testZKServer; } - private MetadataStoreExtended createTestZookeeperMetadataStore(TestZKServer zkServer, - String metadataStoreName) { - try { - MetadataStoreExtended store = MetadataStoreExtended.create("zk:" + zkServer.getConnectionString(), - MetadataStoreConfig.builder().metadataStoreName(metadataStoreName).build()); - registerCloseable(store); - return store; - } catch (MetadataStoreException e) { - throw new RuntimeException(e); - } - } - /** * Applicable only when PulsarTestContext is not startable. This will configure mocks * for PulsarTestResources and related classes. @@ -738,6 +726,20 @@ private void initializeCommonPulsarServices(SpyConfig spyConfig) { configurationMetadataStore(mockZookeeperMetadataStore); } } + } else if (super.testZKServer != null) { + MetadataStoreExtended testZookeeperMetadataStore = + createTestZookeeperMetadataStore(super.testZKServer, MetadataStoreConfig.METADATA_STORE); + if (super.localMetadataStore == null) { + localMetadataStore(testZookeeperMetadataStore); + } + if (super.configurationMetadataStore == null) { + if (super.testZKServerGlobal != null) { + configurationMetadataStore(createTestZookeeperMetadataStore(super.testZKServerGlobal, + MetadataStoreConfig.CONFIGURATION_METADATA_STORE)); + } else { + configurationMetadataStore(testZookeeperMetadataStore); + } + } } else { try { MetadataStoreExtended store = MetadataStoreFactoryImpl.createExtended("memory:local", @@ -782,6 +784,17 @@ private MetadataStoreExtended createMockZookeeperMetadataStore(MockZooKeeper moc return nonClosingProxy; } + @SneakyThrows + private MetadataStoreExtended createTestZookeeperMetadataStore(TestZKServer zkServer, + String metadataStoreName) { + MetadataStoreExtended store = MetadataStoreExtended.create("zk:" + zkServer.getConnectionString(), + MetadataStoreConfig.builder().metadataStoreName(metadataStoreName).build()); + registerCloseable(store); + MetadataStoreExtended nonClosingProxy = + NonClosingProxyHandler.createNonClosingProxy(store, MetadataStoreExtended.class); + return nonClosingProxy; + } + protected abstract void initializePulsarServices(SpyConfig spyConfig, Builder builder); } From e219043090d0206986bf79024e25110387f093a3 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 14:26:23 +0200 Subject: [PATCH 52/56] Remove unnecessary synchronization --- .../src/main/java/org/apache/zookeeper/MockZooKeeper.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index e777c481ce1ec..b38117415899b 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -20,7 +20,6 @@ import com.google.common.collect.HashMultimap; import com.google.common.collect.Maps; -import com.google.common.collect.Multimaps; import com.google.common.collect.SetMultimap; import com.google.common.collect.Sets; import com.google.common.util.concurrent.MoreExecutors; @@ -194,8 +193,7 @@ private static MockZooKeeper createMockZooKeeperInstance(int readOpDelayMs) { private void init() { tree = Maps.newTreeMap(); this.executor = Executors.newSingleThreadExecutor(new DefaultThreadFactory("mock-zookeeper")); - SetMultimap w = HashMultimap.create(); - watchers = Multimaps.synchronizedSetMultimap(w); + watchers = HashMultimap.create(); stopped = new AtomicBoolean(false); alwaysFail = new AtomicReference<>(KeeperException.Code.OK); failures = new CopyOnWriteArrayList<>(); @@ -1199,7 +1197,7 @@ public void deleteEphemeralNodes(long sessionId) { } - public synchronized void deleteWatchers(long sessionId) { + public void deleteWatchers(long sessionId) { runInExecutorSync(() -> { // remove all persistent watchers for the session persistentWatchers.removeIf(w -> w.sessionId == sessionId); From aa6274ff94d0c20b3df52b7dfd2afce84f3fc937 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 15:03:16 +0200 Subject: [PATCH 53/56] Fix parent handling --- .../org/apache/zookeeper/MockZooKeeper.java | 31 +++++++++---------- 1 file changed, 14 insertions(+), 17 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index b38117415899b..d74bc2577666b 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -66,6 +66,7 @@ public class MockZooKeeper extends ZooKeeper { // ephemeralOwner value for persistent nodes private static final long NOT_EPHEMERAL = 0L; + private static final String ROOT_PATH = "/"; @AllArgsConstructor private static class MockZNode { @@ -192,6 +193,7 @@ private static MockZooKeeper createMockZooKeeperInstance(int readOpDelayMs) { private void init() { tree = Maps.newTreeMap(); + tree.put(ROOT_PATH, MockZNode.of(new byte[0], 0, NOT_EPHEMERAL)); this.executor = Executors.newSingleThreadExecutor(new DefaultThreadFactory("mock-zookeeper")); watchers = HashMultimap.create(); stopped = new AtomicBoolean(false); @@ -327,7 +329,7 @@ private String internalCreate(String path, byte[] data, CreateMode createMode) t throw new KeeperException.NodeExistsException(path); } - if (!parent.isEmpty() && !tree.containsKey(parent)) { + if (!tree.containsKey(parent)) { throw new KeeperException.NoNodeException(parent); } @@ -345,7 +347,7 @@ private String internalCreate(String path, byte[] data, CreateMode createMode) t tree.put(path, createMockZNode(data, createMode)); toNotifyCreate.addAll(getWatchers(path)); - if (!parent.isEmpty()) { + if (!ROOT_PATH.equals(parent)) { toNotifyParent.addAll(getWatchers(parent)); } watchers.removeAll(path); @@ -374,7 +376,8 @@ private String internalCreate(String path, byte[] data, CreateMode createMode) t } private static String getParentName(String path) { - return path.substring(0, path.lastIndexOf("/")); + String parentName = path.substring(0, path.lastIndexOf('/')); + return parentName.length() > 0 ? parentName : "/"; } private static String getNodeName(String path) { @@ -426,7 +429,7 @@ public void create(final String path, final byte[] data, final List acl, Cr final Set toNotifyParent = Sets.newHashSet(); final String parent = getParentName(path); - if (!parent.isEmpty()) { + if (!ROOT_PATH.equals(parent)) { toNotifyParent.addAll(getWatchers(parent)); } @@ -444,7 +447,7 @@ public void create(final String path, final byte[] data, final List acl, Cr cb.processResult(KeeperException.Code.CONNECTIONLOSS.intValue(), path, ctx, null); } else if (tree.containsKey(path)) { cb.processResult(KeeperException.Code.NODEEXISTS.intValue(), path, ctx, null); - } else if (!parent.isEmpty() && !tree.containsKey(parent)) { + } else if (!tree.containsKey(parent)) { runNotifications(() -> { toNotifyParent.forEach(watcher -> watcher .process(new WatchedEvent(EventType.NodeChildrenChanged, KeeperState.SyncConnected, @@ -452,9 +455,7 @@ public void create(final String path, final byte[] data, final List acl, Cr }); cb.processResult(KeeperException.Code.NONODE.intValue(), path, ctx, null); } else { - if (!parent.isEmpty()) { - tree.get(parent).getChildren().add(getNodeName(name)); - } + tree.get(parent).getChildren().add(getNodeName(name)); tree.put(name, createMockZNode(data, createMode)); watchers.removeAll(name); cb.processResult(0, path, ctx, name); @@ -873,15 +874,13 @@ private void internalDelete(String path, int version) throws KeeperException { parent = getParentName(path); tree.remove(path); - if (!parent.isEmpty()) { - tree.get(parent).getChildren().remove(getNodeName(path)); - } + tree.get(parent).getChildren().remove(getNodeName(path)); toNotifyDelete = Sets.newHashSet(); toNotifyDelete.addAll(getWatchers(path)); toNotifyParent = Sets.newHashSet(); - if (!parent.isEmpty()) { + if (!ROOT_PATH.equals(parent)) { toNotifyParent.addAll(getWatchers(parent)); } @@ -911,8 +910,8 @@ public void delete(final String path, int version, final VoidCallback cb, final toNotifyDelete.addAll(getWatchers(path)); final Set toNotifyParent = Sets.newHashSet(); - final String parent = path.substring(0, path.lastIndexOf('/')); - if (!parent.isEmpty()) { + final String parent = getParentName(path); + if (!ROOT_PATH.equals(parent)) { toNotifyParent.addAll(getWatchers(parent)); } watchers.removeAll(path); @@ -936,9 +935,7 @@ public void delete(final String path, int version, final VoidCallback cb, final } tree.remove(path); - if (!parent.isEmpty()) { - tree.get(parent).getChildren().remove(getNodeName(path)); - } + tree.get(parent).getChildren().remove(getNodeName(path)); cb.processResult(0, path, ctx); runNotifications(() -> { From d0a0505b15119d8b7b4210779fd3e66c9ca80fbe Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 15:09:06 +0200 Subject: [PATCH 54/56] Refactor --- .../java/org/apache/zookeeper/MockZooKeeper.java | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index d74bc2577666b..e124699ee1383 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -329,21 +329,19 @@ private String internalCreate(String path, byte[] data, CreateMode createMode) t throw new KeeperException.NodeExistsException(path); } - if (!tree.containsKey(parent)) { + MockZNode parentNode = tree.get(parent); + + if (parentNode == null) { throw new KeeperException.NoNodeException(parent); } - MockZNode parentNode = tree.get(parent); - if (createMode.isSequential()) { - int parentVersion = tree.get(parent).getVersion(); + int parentVersion = parentNode.getVersion(); path = path + parentVersion; parentNode.updateVersion(); } - if (parentNode != null) { - parentNode.getChildren().add(getNodeName(path)); - } + parentNode.getChildren().add(getNodeName(path)); tree.put(path, createMockZNode(data, createMode)); toNotifyCreate.addAll(getWatchers(path)); From 4ceecadf2bb1898a095e145655fb72e3b0071175 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 15:36:08 +0200 Subject: [PATCH 55/56] Fix pulsar-metadata test resource leaks --- .../bookkeeper/replication/BookKeeperClusterTestCase.java | 4 ++++ .../src/test/java/org/apache/pulsar/metadata/CounterTest.java | 1 + .../org/apache/pulsar/metadata/MetadataStoreExtendedTest.java | 1 + 3 files changed, 6 insertions(+) diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookKeeperClusterTestCase.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookKeeperClusterTestCase.java index ccbdb8cef64c5..9724d2d6ff65d 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookKeeperClusterTestCase.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookKeeperClusterTestCase.java @@ -212,6 +212,7 @@ public void tearDown() throws Exception { try { // cleanup for metrics. metadataStore.close(); + metadataStore = null; stopZKCluster(); } catch (Exception e) { LOG.error("Got Exception while trying to stop ZKCluster", e); @@ -256,6 +257,9 @@ private static void callCloseables(List closeables) { protected void startZKCluster() throws Exception { zkUtil.startCluster(); zkc = zkUtil.getZooKeeperClient(); + if (metadataStore != null) { + metadataStore.close(); + } metadataStore = new FaultInjectionMetadataStore( MetadataStoreExtended.create(zkUtil.getZooKeeperConnectString(), MetadataStoreConfig.builder().metadataStoreName("metastore-" + getClass().getSimpleName()).build())); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java index f6d1d7553099b..bd068539cc549 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/CounterTest.java @@ -70,6 +70,7 @@ public void testCounterDoesNotAutoReset(String provider, Supplier urlSup return; } String metadataUrl = urlSupplier.get(); + @Cleanup MetadataStoreExtended store1 = MetadataStoreExtended.create(metadataUrl, MetadataStoreConfig.builder().build()); CoordinationService cs1 = new CoordinationServiceImpl(store1); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java index b71511aabceae..30fbd9b836e92 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreExtendedTest.java @@ -71,6 +71,7 @@ public void sequentialKeys(String provider, Supplier urlSupplier) throws @Test(dataProvider = "impl") public void testPersistentOrEphemeralPut(String provider, Supplier urlSupplier) throws Exception { final String key1 = newKey(); + @Cleanup MetadataStoreExtended store = MetadataStoreExtended.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); store.put(key1, "value-1".getBytes(), Optional.empty(), EnumSet.noneOf(CreateOption.class)).join(); var value = store.get(key1).join().get(); From 06e745b34058ea00863953264968dca5652cbddd Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Feb 2025 16:18:15 +0200 Subject: [PATCH 56/56] Fix one more leak --- .../test/java/org/apache/pulsar/metadata/MetadataStoreTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java index 5dbb585dae199..a7b1dcf6bf02b 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java @@ -551,6 +551,7 @@ public void testOxiaLoadConfigFromFile() throws Exception { builder.configFilePath("src/test/resources/oxia_client.conf"); MetadataStoreConfig config = builder.build(); + @Cleanup OxiaMetadataStore store = (OxiaMetadataStore) MetadataStoreFactory.create(oxia, config); var client = (AsyncOxiaClient) WhiteboxImpl.getInternalState(store, "client"); var sessionManager = (SessionManager) WhiteboxImpl.getInternalState(client, "sessionManager");