You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/11/28 08:56:25 UTC

[GitHub] [pulsar] Jason918 opened a new pull request #13005: [metadata] add DelayInjectionMetadataStore

Jason918 opened a new pull request #13005:
URL: https://github.com/apache/pulsar/pull/13005


   <!--
   ### Contribution Checklist
     
     - Name the pull request in the form "[Issue XYZ][component] Title of the pull request", where *XYZ* should be replaced by the actual issue number.
       Skip *Issue XYZ* if there is no associated github issue for this pull request.
       Skip *component* if you are unsure about which is the best component. E.g. `[docs] Fix typo in produce method`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ### Motivation
   
   Currently, we have `FaultInjectionMetadataStore` which can only inject errors for metadata store. 
   but there are some race conditions in metadata store, like #13004, and we don't have useful tools to address these. 
   
   ### Modifications
   
   Add DelayInjectionMetadataStore with two key features. 
   
   1.  Add a random delay (range can be configured ) in each metadata operation.
   2. Add a condition delay with target operation and path, this type of delay will be removed once fired.
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   
   
   This change added tests and can be verified as follows:
     - DelayInjectionMetadataStoreTest
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API: (no)
     - The schema: (no)
     - The default values of configurations: (no)
     - The wire protocol: (no)
     - The rest endpoints: (no)
     - The admin cli options: (no)
     - Anything that affects deployment: (no)
   
   ### Documentation
   
   Check the box below and label this PR (if you have committer privilege).
   
   Need to update docs? 
     
   - [x] `doc` 
     Doc provided in DelayInjectionMetadataStore.java
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on pull request #13005: [metadata] add DelayInjectionMetadataStore

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #13005:
URL: https://github.com/apache/pulsar/pull/13005#issuecomment-982319426


   @merlimat PTAL. I think we can discover more race condition issues in current metadata implementation with this DelayInjectionMetadataStore, 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on pull request #13005: [metadata] add DelayInjectionMetadataStore

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #13005:
URL: https://github.com/apache/pulsar/pull/13005#issuecomment-981656190


   @eolivelli Hi~ Please help take a look. Thank you so much.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #13005: [metadata] add DelayInjectionMetadataStore

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #13005:
URL: https://github.com/apache/pulsar/pull/13005#discussion_r757866877



##########
File path: pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/DelayInjectionMetadataStoreTest.java
##########
@@ -0,0 +1,73 @@
+/**
+ * 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.impl;
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import org.apache.pulsar.metadata.api.GetResult;
+import org.apache.pulsar.metadata.api.MetadataStoreConfig;
+import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class DelayInjectionMetadataStoreTest {
+    @Test
+    public void testRandomDelay() throws Exception {
+        MetadataStoreExtended innerStore =
+                MetadataStoreFactoryImpl.createExtended("memory://local", MetadataStoreConfig.builder().build());
+        DelayInjectionMetadataStore store = new DelayInjectionMetadataStore(innerStore);
+
+        store.setMaxRandomDelayMills(200);
+        store.setMinRandomDelayMills(100);
+        byte[] data = new byte[]{1, 2, 3};
+        long start = System.currentTimeMillis();
+        store.put("/data", data, Optional.empty()).get();
+        Assert.assertTrue(System.currentTimeMillis() - start >= 100);
+        start = System.currentTimeMillis();
+        Assert.assertEquals(store.get("/data").get().get().getValue(), data);
+        Assert.assertTrue(System.currentTimeMillis() - start >= 100);
+    }
+
+    @Test
+    public void testConditionDelay() throws Exception {
+        MetadataStoreExtended innerStore =
+                MetadataStoreFactoryImpl.createExtended("memory://local", MetadataStoreConfig.builder().build());
+        DelayInjectionMetadataStore store = new DelayInjectionMetadataStore(innerStore);
+
+        CompletableFuture<Void> delay = new CompletableFuture<>();
+        store.delayConditional(delay, (operationType, path) -> {
+            return operationType == DelayInjectionMetadataStore.OperationType.GET && path.equals("/data");
+        });
+
+        byte[] data = new byte[]{1, 2, 3};
+        store.put("/data", data, Optional.empty()).get(); //NO delay.
+
+        long start = System.currentTimeMillis();
+        CompletableFuture<Optional<GetResult>> getFuture = store.get("/data");
+        Thread.sleep(1000);

Review comment:
       Will this test be flaky?

##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/DelayInjectionMetadataStore.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiPredicate;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.pulsar.metadata.api.GetResult;
+import org.apache.pulsar.metadata.api.MetadataCache;
+import org.apache.pulsar.metadata.api.MetadataSerde;
+import org.apache.pulsar.metadata.api.Notification;
+import org.apache.pulsar.metadata.api.Stat;
+import org.apache.pulsar.metadata.api.extended.CreateOption;
+import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
+import org.apache.pulsar.metadata.api.extended.SessionEvent;
+
+/**
+ * Add possibility to inject delays during tests that interact with MetadataStore.
+ * This class provides two kinds of delays:
+ * 1. Add a random delay (set by {@link #setMinRandomDelayMills} and {@link #setMaxRandomDelayMills}) in each
+ * metadata operation.
+ * 2. Add a condition delay with target operation and path, this type of delay will be removed once fired. See
+ * {@link #delayConditional}
+ */
+@Slf4j
+public class DelayInjectionMetadataStore implements MetadataStoreExtended {
+
+    private final MetadataStoreExtended store;
+    private final CopyOnWriteArrayList<Delay> delays;
+    private final AtomicInteger maxRandomDelayMills = new AtomicInteger();
+    private final AtomicInteger minRandomDelayMills = new AtomicInteger(0);
+    private final List<Consumer<SessionEvent>> sessionListeners = new CopyOnWriteArrayList<>();
+    private final ScheduledExecutorService scheduler;
+
+    public enum OperationType {
+        GET,
+        GET_CHILDREN,
+        EXISTS,
+        PUT,
+        DELETE,
+    }
+
+    @Data
+    private static class Delay {
+        private final CompletableFuture<Void> delay;
+        private final BiPredicate<OperationType, String> predicate;
+    }
+
+    public DelayInjectionMetadataStore(MetadataStoreExtended store) {
+        this.store = store;
+        this.delays = new CopyOnWriteArrayList<>();
+        scheduler = Executors.newSingleThreadScheduledExecutor(
+                new DefaultThreadFactory("metadata-store-delay-injection"));
+    }
+
+
+    private CompletableFuture<Void> getRandomDelayStage() {
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        scheduler.schedule(() -> {
+            future.complete(null);
+        }, RandomUtils.nextInt(minRandomDelayMills.get(), maxRandomDelayMills.get()), TimeUnit.MILLISECONDS);
+        return future;
+    }
+
+    CompletableFuture injectRandomDelay(Supplier<CompletableFuture<?>> op) {
+        return getRandomDelayStage() //pre delay
+                .thenCompose((ignore) -> op.get()
+                        .thenCompose(result -> getRandomDelayStage() //post delay.
+                                .thenApply(v -> result)));
+    }
+
+    @Override
+    public CompletableFuture<Optional<GetResult>> get(String path) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(()->store.get(path));
+        }
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.GET, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.get(path));
+        }
+        return store.get(path);
+    }
+
+    @Override
+    public CompletableFuture<List<String>> getChildren(String path) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.getChildren(path));
+        }
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.GET_CHILDREN, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.getChildren(path));
+        }
+        return store.getChildren(path);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> exists(String path) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.exists(path));
+        }
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.EXISTS, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.exists(path));
+        }
+        return store.exists(path);
+    }
+
+    @Override
+    public CompletableFuture<Stat> put(String path, byte[] value, Optional<Long> expectedVersion) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.put(path, value, expectedVersion));
+        }
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.PUT, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.put(path, value, expectedVersion));
+        }
+        return store.put(path, value, expectedVersion);
+    }
+
+    @Override
+    public CompletableFuture<Stat> put(String path, byte[] value, Optional<Long> expectedVersion,
+                                       EnumSet<CreateOption> options) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.put(path, value, expectedVersion, options));
+        }
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.PUT, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.put(path, value, expectedVersion, options));
+        }
+        return store.put(path, value, expectedVersion, options);
+    }
+
+    @Override
+    public CompletableFuture<Void> delete(String path, Optional<Long> expectedVersion) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.delete(path, expectedVersion));
+        }
+
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.DELETE, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.delete(path, expectedVersion));
+        }
+        return store.delete(path, expectedVersion);
+    }
+
+    @Override
+    public CompletableFuture<Void> deleteRecursive(String path) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.deleteRecursive(path));
+        }
+
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.DELETE, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.deleteRecursive(path));
+        }
+        return store.deleteRecursive(path);
+    }
+
+    @Override
+    public void registerListener(Consumer<Notification> listener) {
+        store.registerListener(listener);
+    }
+
+    @Override
+    public <T> MetadataCache<T> getMetadataCache(Class<T> clazz) {
+        return store.getMetadataCache(clazz);
+    }
+
+    @Override
+    public <T> MetadataCache<T> getMetadataCache(TypeReference<T> typeRef) {
+        return store.getMetadataCache(typeRef);
+    }
+
+    @Override
+    public <T> MetadataCache<T> getMetadataCache(MetadataSerde<T> serde) {
+        return store.getMetadataCache(serde);
+    }
+
+    @Override
+    public void registerSessionListener(Consumer<SessionEvent> listener) {
+        store.registerSessionListener(listener);
+        sessionListeners.add(listener);
+    }
+
+    @Override
+    public void close() throws Exception {
+        store.close();
+    }
+
+    public void setMaxRandomDelayMills(int maxRandomDelayMills) {
+        this.maxRandomDelayMills.set(maxRandomDelayMills);
+    }
+
+    public void setMinRandomDelayMills(int minRandomDelayMills) {
+        this.minRandomDelayMills.set(minRandomDelayMills);
+    }
+
+    public void delayConditional(CompletableFuture<Void> wait, BiPredicate<OperationType, String> predicate) {
+        delays.add(new Delay(wait, predicate));
+    }
+
+    public void triggerSessionEvent(SessionEvent event) {
+        sessionListeners.forEach(l -> l.accept(event));
+    }
+
+    private Optional<CompletableFuture<Void>> programmedDelays(OperationType op, String path) {
+        Optional<Delay> failure = delays.stream()
+                .filter(f -> f.predicate.test(op, path))
+                .findFirst();
+
+        failure.ifPresent(delays::remove);

Review comment:
       This operation is not thread safe.
   If two threads enter this method they could use the same CompletareFuture leading to unpredictable results.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on pull request #13005: [metadata] add DelayInjectionMetadataStore

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #13005:
URL: https://github.com/apache/pulsar/pull/13005#issuecomment-981321983


   /pulsarbot run-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 closed pull request #13005: [metadata] add DelayInjectionMetadataStore

Posted by GitBox <gi...@apache.org>.
Jason918 closed pull request #13005:
URL: https://github.com/apache/pulsar/pull/13005


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on a change in pull request #13005: [metadata] add DelayInjectionMetadataStore

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #13005:
URL: https://github.com/apache/pulsar/pull/13005#discussion_r757903118



##########
File path: pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/DelayInjectionMetadataStoreTest.java
##########
@@ -0,0 +1,73 @@
+/**
+ * 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.impl;
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import org.apache.pulsar.metadata.api.GetResult;
+import org.apache.pulsar.metadata.api.MetadataStoreConfig;
+import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class DelayInjectionMetadataStoreTest {
+    @Test
+    public void testRandomDelay() throws Exception {
+        MetadataStoreExtended innerStore =
+                MetadataStoreFactoryImpl.createExtended("memory://local", MetadataStoreConfig.builder().build());
+        DelayInjectionMetadataStore store = new DelayInjectionMetadataStore(innerStore);
+
+        store.setMaxRandomDelayMills(200);
+        store.setMinRandomDelayMills(100);
+        byte[] data = new byte[]{1, 2, 3};
+        long start = System.currentTimeMillis();
+        store.put("/data", data, Optional.empty()).get();
+        Assert.assertTrue(System.currentTimeMillis() - start >= 100);
+        start = System.currentTimeMillis();
+        Assert.assertEquals(store.get("/data").get().get().getValue(), data);
+        Assert.assertTrue(System.currentTimeMillis() - start >= 100);
+    }
+
+    @Test
+    public void testConditionDelay() throws Exception {
+        MetadataStoreExtended innerStore =
+                MetadataStoreFactoryImpl.createExtended("memory://local", MetadataStoreConfig.builder().build());
+        DelayInjectionMetadataStore store = new DelayInjectionMetadataStore(innerStore);
+
+        CompletableFuture<Void> delay = new CompletableFuture<>();
+        store.delayConditional(delay, (operationType, path) -> {
+            return operationType == DelayInjectionMetadataStore.OperationType.GET && path.equals("/data");
+        });
+
+        byte[] data = new byte[]{1, 2, 3};
+        store.put("/data", data, Optional.empty()).get(); //NO delay.
+
+        long start = System.currentTimeMillis();
+        CompletableFuture<Optional<GetResult>> getFuture = store.get("/data");
+        Thread.sleep(1000);

Review comment:
       I don't think so. 
   But after a second thought, I changed it to a scheduled task, to make this test more convincing.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 commented on a change in pull request #13005: [metadata] add DelayInjectionMetadataStore

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #13005:
URL: https://github.com/apache/pulsar/pull/13005#discussion_r757901794



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/DelayInjectionMetadataStore.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiPredicate;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.pulsar.metadata.api.GetResult;
+import org.apache.pulsar.metadata.api.MetadataCache;
+import org.apache.pulsar.metadata.api.MetadataSerde;
+import org.apache.pulsar.metadata.api.Notification;
+import org.apache.pulsar.metadata.api.Stat;
+import org.apache.pulsar.metadata.api.extended.CreateOption;
+import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
+import org.apache.pulsar.metadata.api.extended.SessionEvent;
+
+/**
+ * Add possibility to inject delays during tests that interact with MetadataStore.
+ * This class provides two kinds of delays:
+ * 1. Add a random delay (set by {@link #setMinRandomDelayMills} and {@link #setMaxRandomDelayMills}) in each
+ * metadata operation.
+ * 2. Add a condition delay with target operation and path, this type of delay will be removed once fired. See
+ * {@link #delayConditional}
+ */
+@Slf4j
+public class DelayInjectionMetadataStore implements MetadataStoreExtended {
+
+    private final MetadataStoreExtended store;
+    private final CopyOnWriteArrayList<Delay> delays;
+    private final AtomicInteger maxRandomDelayMills = new AtomicInteger();
+    private final AtomicInteger minRandomDelayMills = new AtomicInteger(0);
+    private final List<Consumer<SessionEvent>> sessionListeners = new CopyOnWriteArrayList<>();
+    private final ScheduledExecutorService scheduler;
+
+    public enum OperationType {
+        GET,
+        GET_CHILDREN,
+        EXISTS,
+        PUT,
+        DELETE,
+    }
+
+    @Data
+    private static class Delay {
+        private final CompletableFuture<Void> delay;
+        private final BiPredicate<OperationType, String> predicate;
+    }
+
+    public DelayInjectionMetadataStore(MetadataStoreExtended store) {
+        this.store = store;
+        this.delays = new CopyOnWriteArrayList<>();
+        scheduler = Executors.newSingleThreadScheduledExecutor(
+                new DefaultThreadFactory("metadata-store-delay-injection"));
+    }
+
+
+    private CompletableFuture<Void> getRandomDelayStage() {
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        scheduler.schedule(() -> {
+            future.complete(null);
+        }, RandomUtils.nextInt(minRandomDelayMills.get(), maxRandomDelayMills.get()), TimeUnit.MILLISECONDS);
+        return future;
+    }
+
+    CompletableFuture injectRandomDelay(Supplier<CompletableFuture<?>> op) {
+        return getRandomDelayStage() //pre delay
+                .thenCompose((ignore) -> op.get()
+                        .thenCompose(result -> getRandomDelayStage() //post delay.
+                                .thenApply(v -> result)));
+    }
+
+    @Override
+    public CompletableFuture<Optional<GetResult>> get(String path) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(()->store.get(path));
+        }
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.GET, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.get(path));
+        }
+        return store.get(path);
+    }
+
+    @Override
+    public CompletableFuture<List<String>> getChildren(String path) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.getChildren(path));
+        }
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.GET_CHILDREN, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.getChildren(path));
+        }
+        return store.getChildren(path);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> exists(String path) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.exists(path));
+        }
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.EXISTS, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.exists(path));
+        }
+        return store.exists(path);
+    }
+
+    @Override
+    public CompletableFuture<Stat> put(String path, byte[] value, Optional<Long> expectedVersion) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.put(path, value, expectedVersion));
+        }
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.PUT, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.put(path, value, expectedVersion));
+        }
+        return store.put(path, value, expectedVersion);
+    }
+
+    @Override
+    public CompletableFuture<Stat> put(String path, byte[] value, Optional<Long> expectedVersion,
+                                       EnumSet<CreateOption> options) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.put(path, value, expectedVersion, options));
+        }
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.PUT, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.put(path, value, expectedVersion, options));
+        }
+        return store.put(path, value, expectedVersion, options);
+    }
+
+    @Override
+    public CompletableFuture<Void> delete(String path, Optional<Long> expectedVersion) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.delete(path, expectedVersion));
+        }
+
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.DELETE, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.delete(path, expectedVersion));
+        }
+        return store.delete(path, expectedVersion);
+    }
+
+    @Override
+    public CompletableFuture<Void> deleteRecursive(String path) {
+        if (maxRandomDelayMills.get() > 0) {
+            return injectRandomDelay(() -> store.deleteRecursive(path));
+        }
+
+        Optional<CompletableFuture<Void>> delay = programmedDelays(OperationType.DELETE, path);
+        if (delay.isPresent()) {
+            return delay.get().thenCompose(__ -> store.deleteRecursive(path));
+        }
+        return store.deleteRecursive(path);
+    }
+
+    @Override
+    public void registerListener(Consumer<Notification> listener) {
+        store.registerListener(listener);
+    }
+
+    @Override
+    public <T> MetadataCache<T> getMetadataCache(Class<T> clazz) {
+        return store.getMetadataCache(clazz);
+    }
+
+    @Override
+    public <T> MetadataCache<T> getMetadataCache(TypeReference<T> typeRef) {
+        return store.getMetadataCache(typeRef);
+    }
+
+    @Override
+    public <T> MetadataCache<T> getMetadataCache(MetadataSerde<T> serde) {
+        return store.getMetadataCache(serde);
+    }
+
+    @Override
+    public void registerSessionListener(Consumer<SessionEvent> listener) {
+        store.registerSessionListener(listener);
+        sessionListeners.add(listener);
+    }
+
+    @Override
+    public void close() throws Exception {
+        store.close();
+    }
+
+    public void setMaxRandomDelayMills(int maxRandomDelayMills) {
+        this.maxRandomDelayMills.set(maxRandomDelayMills);
+    }
+
+    public void setMinRandomDelayMills(int minRandomDelayMills) {
+        this.minRandomDelayMills.set(minRandomDelayMills);
+    }
+
+    public void delayConditional(CompletableFuture<Void> wait, BiPredicate<OperationType, String> predicate) {
+        delays.add(new Delay(wait, predicate));
+    }
+
+    public void triggerSessionEvent(SessionEvent event) {
+        sessionListeners.forEach(l -> l.accept(event));
+    }
+
+    private Optional<CompletableFuture<Void>> programmedDelays(OperationType op, String path) {
+        Optional<Delay> failure = delays.stream()
+                .filter(f -> f.predicate.test(op, path))
+                .findFirst();
+
+        failure.ifPresent(delays::remove);

Review comment:
       Great catch. 
   This will leads to an extra and unexpected delay.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Jason918 closed pull request #13005: [metadata] add DelayInjectionMetadataStore

Posted by GitBox <gi...@apache.org>.
Jason918 closed pull request #13005:
URL: https://github.com/apache/pulsar/pull/13005


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org