Skip to content

Commit

Permalink
[fix][broker] Fix the retry mechanism in `MetadataCache#readModifyUpd…
Browse files Browse the repository at this point in the history
…ateOrCreate` (apache#23686)

## Motivation

The method `MetadataCache#readModifyUpdateOrCreate` should handle the BadVersionException by retrying the modification process, as already noted in the Java documentation: "The modify function can potentially be called multiple times due to concurrent updates."

Currently, `MetadataCache#readModifyUpdateOrCreate` does not catch the BadVersionException on the second attempt, allowing the exception to be passed to the caller. This issue can be easily reproduced by increasing concurrent futures in the test `MetadataCacheTest#readModifyUpdateBadVersionRetry`.

The current retry implementation is incorrect and lacks a backoff mechanism, which could lead to too many requests to the metadata store.

## Modification

- Correct the retry process in `MetadataCache#readModifyUpdateOrCreate` to ensure BadVersionException is caught during each retry.
- Implement a retry backoff mechanism in `MetadataCache#readModifyUpdateOrCreate` to manage the frequency of retries effectively.
- Add new config `retryBackoff` to the MetadataCacheConfig to control the MetadataCache retry backoff.
- Respective the `metadataStoreOperationTimeoutSeconds` for the MetadataCache retry
  • Loading branch information
RobertIndie authored Dec 14, 2024
1 parent 0845c21 commit 0ae3f9d
Show file tree
Hide file tree
Showing 7 changed files with 135 additions and 39 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.metadata.api.MetadataCache;
import org.apache.pulsar.metadata.api.MetadataCacheConfig;
import org.apache.pulsar.metadata.api.MetadataStore;
import org.apache.pulsar.metadata.api.MetadataStoreException;

Expand All @@ -58,13 +59,19 @@ public class BaseResources<T> {

public BaseResources(MetadataStore store, Class<T> clazz, int operationTimeoutSec) {
this.store = store;
this.cache = store.getMetadataCache(clazz);
this.cache = store.getMetadataCache(clazz, MetadataCacheConfig.builder()
.retryBackoff(MetadataCacheConfig.DEFAULT_RETRY_BACKOFF_BUILDER.setMandatoryStop(operationTimeoutSec,
TimeUnit.SECONDS))
.build());
this.operationTimeoutSec = operationTimeoutSec;
}

public BaseResources(MetadataStore store, TypeReference<T> typeRef, int operationTimeoutSec) {
this.store = store;
this.cache = store.getMetadataCache(typeRef);
this.cache = store.getMetadataCache(typeRef, MetadataCacheConfig.builder()
.retryBackoff(MetadataCacheConfig.DEFAULT_RETRY_BACKOFF_BUILDER.setMandatoryStop(operationTimeoutSec,
TimeUnit.SECONDS))
.build());
this.operationTimeoutSec = operationTimeoutSec;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
*/
package org.apache.pulsar.common.util;

import com.google.common.annotations.VisibleForTesting;
import java.time.Clock;
import java.util.Random;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -95,11 +94,6 @@ public void reset() {
this.mandatoryStopMade = false;
}

@VisibleForTesting
long getFirstBackoffTimeInMillis() {
return firstBackoffTimeInMillis;
}

public static boolean shouldBackoff(long initialTimestamp, TimeUnit unitInitial, int failedAttempts,
long defaultInterval, long maxBackoffInterval) {
long initialTimestampInNano = unitInitial.toNanos(initialTimestamp);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,7 @@ public void mandatoryStopTest() {

// would have been 1600 w/o the mandatory stop
assertTrue(withinTenPercentAndDecrementTimer(backoff, 400));
assertTrue(backoff.isMandatoryStopMade());
Mockito.when(mockClock.millis()).thenReturn(1900L);
assertTrue(withinTenPercentAndDecrementTimer(backoff, 3200));
Mockito.when(mockClock.millis()).thenReturn(3200L);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import lombok.Builder;
import lombok.Getter;
import lombok.ToString;
import org.apache.pulsar.common.util.BackoffBuilder;

/**
* The configuration builder for a {@link MetadataCache} config.
Expand All @@ -33,6 +34,10 @@
@ToString
public class MetadataCacheConfig<T> {
private static final long DEFAULT_CACHE_REFRESH_TIME_MILLIS = TimeUnit.MINUTES.toMillis(5);
public static final BackoffBuilder DEFAULT_RETRY_BACKOFF_BUILDER =
new BackoffBuilder().setInitialTime(5, TimeUnit.MILLISECONDS)
.setMax(3, TimeUnit.SECONDS)
.setMandatoryStop(30, TimeUnit.SECONDS);

/**
* Specifies that active entries are eligible for automatic refresh once a fixed duration has
Expand All @@ -57,4 +62,7 @@ public class MetadataCacheConfig<T> {
@Builder.Default
private final BiConsumer<String, Optional<CacheGetResult<T>>> asyncReloadConsumer = null;

@Builder.Default
private final BackoffBuilder retryBackoff = DEFAULT_RETRY_BACKOFF_BUILDER;

}
Original file line number Diff line number Diff line change
Expand Up @@ -30,13 +30,16 @@
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Supplier;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.common.concurrent.FutureUtils;
import org.apache.pulsar.common.util.Backoff;
import org.apache.pulsar.metadata.api.CacheGetResult;
import org.apache.pulsar.metadata.api.GetResult;
import org.apache.pulsar.metadata.api.MetadataCache;
Expand All @@ -58,25 +61,32 @@ public class MetadataCacheImpl<T> implements MetadataCache<T>, Consumer<Notifica
private final MetadataStore store;
private final MetadataStoreExtended storeExtended;
private final MetadataSerde<T> serde;
private final ScheduledExecutorService executor;
private final MetadataCacheConfig<T> cacheConfig;

private final AsyncLoadingCache<String, Optional<CacheGetResult<T>>> objCache;

public MetadataCacheImpl(MetadataStore store, TypeReference<T> typeRef, MetadataCacheConfig<T> cacheConfig) {
this(store, new JSONMetadataSerdeTypeRef<>(typeRef), cacheConfig);
public MetadataCacheImpl(MetadataStore store, TypeReference<T> typeRef, MetadataCacheConfig<T> cacheConfig,
ScheduledExecutorService executor) {
this(store, new JSONMetadataSerdeTypeRef<>(typeRef), cacheConfig, executor);
}

public MetadataCacheImpl(MetadataStore store, JavaType type, MetadataCacheConfig<T> cacheConfig) {
this(store, new JSONMetadataSerdeSimpleType<>(type), cacheConfig);
public MetadataCacheImpl(MetadataStore store, JavaType type, MetadataCacheConfig<T> cacheConfig,
ScheduledExecutorService executor) {
this(store, new JSONMetadataSerdeSimpleType<>(type), cacheConfig, executor);
}

public MetadataCacheImpl(MetadataStore store, MetadataSerde<T> serde, MetadataCacheConfig<T> cacheConfig) {
public MetadataCacheImpl(MetadataStore store, MetadataSerde<T> serde, MetadataCacheConfig<T> cacheConfig,
ScheduledExecutorService executor) {
this.store = store;
if (store instanceof MetadataStoreExtended) {
this.storeExtended = (MetadataStoreExtended) store;
} else {
this.storeExtended = null;
}
this.serde = serde;
this.cacheConfig = cacheConfig;
this.executor = executor;

Caffeine<Object, Object> cacheBuilder = Caffeine.newBuilder();
if (cacheConfig.getRefreshAfterWriteMillis() > 0) {
Expand Down Expand Up @@ -321,22 +331,34 @@ public void accept(Notification t) {
}
}

private CompletableFuture<T> executeWithRetry(Supplier<CompletableFuture<T>> op, String key) {
CompletableFuture<T> result = new CompletableFuture<>();
private void execute(Supplier<CompletableFuture<T>> op, String key, CompletableFuture<T> result, Backoff backoff) {
op.get().thenAccept(result::complete).exceptionally((ex) -> {
if (ex.getCause() instanceof BadVersionException) {
// if resource is updated by other than metadata-cache then metadata-cache will get bad-version
// exception. so, try to invalidate the cache and try one more time.
objCache.synchronous().invalidate(key);
op.get().thenAccept(result::complete).exceptionally((ex1) -> {
result.completeExceptionally(ex1.getCause());
long elapsed = System.currentTimeMillis() - backoff.getFirstBackoffTimeInMillis();
if (backoff.isMandatoryStopMade()) {
result.completeExceptionally(new TimeoutException(
String.format("Timeout to update key %s. Elapsed time: %d ms", key, elapsed)));
return null;
});
}
final var next = backoff.next();
log.info("Update key {} conflicts. Retrying in {} ms. Mandatory stop: {}. Elapsed time: {} ms", key,
next, backoff.isMandatoryStopMade(), elapsed);
executor.schedule(() -> execute(op, key, result, backoff), next,
TimeUnit.MILLISECONDS);
return null;
}
result.completeExceptionally(ex.getCause());
return null;
});
}

private CompletableFuture<T> executeWithRetry(Supplier<CompletableFuture<T>> op, String key) {
final var backoff = cacheConfig.getRetryBackoff().create();
CompletableFuture<T> result = new CompletableFuture<>();
execute(op, key, result, backoff);
return result;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -236,21 +236,21 @@ protected boolean shouldIgnoreEvent(MetadataEvent event, GetResult existingValue
@Override
public <T> MetadataCache<T> getMetadataCache(Class<T> clazz, MetadataCacheConfig cacheConfig) {
MetadataCacheImpl<T> metadataCache = new MetadataCacheImpl<T>(this,
TypeFactory.defaultInstance().constructSimpleType(clazz, null), cacheConfig);
TypeFactory.defaultInstance().constructSimpleType(clazz, null), cacheConfig, this.executor);
metadataCaches.add(metadataCache);
return metadataCache;
}

@Override
public <T> MetadataCache<T> getMetadataCache(TypeReference<T> typeRef, MetadataCacheConfig cacheConfig) {
MetadataCacheImpl<T> metadataCache = new MetadataCacheImpl<T>(this, typeRef, cacheConfig);
MetadataCacheImpl<T> metadataCache = new MetadataCacheImpl<T>(this, typeRef, cacheConfig, this.executor);
metadataCaches.add(metadataCache);
return metadataCache;
}

@Override
public <T> MetadataCache<T> getMetadataCache(MetadataSerde<T> serde, MetadataCacheConfig cacheConfig) {
MetadataCacheImpl<T> metadataCache = new MetadataCacheImpl<>(this, serde, cacheConfig);
MetadataCacheImpl<T> metadataCache = new MetadataCacheImpl<>(this, serde, cacheConfig, this.executor);
metadataCaches.add(metadataCache);
return metadataCache;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,9 @@
*/
package org.apache.pulsar.metadata;

import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.spy;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotEquals;
import static org.testng.Assert.assertNotSame;
Expand All @@ -26,7 +29,9 @@
import static org.testng.Assert.fail;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.core.type.TypeReference;
import com.github.benmanes.caffeine.cache.AsyncLoadingCache;
import java.io.IOException;
import java.lang.reflect.Field;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.EnumSet;
Expand All @@ -36,6 +41,8 @@
import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;
import lombok.AllArgsConstructor;
Expand All @@ -44,13 +51,16 @@
import lombok.NoArgsConstructor;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.common.util.BackoffBuilder;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.common.util.ObjectMapperFactory;
import org.apache.pulsar.metadata.api.CacheGetResult;
import org.apache.pulsar.metadata.api.MetadataCache;
import org.apache.pulsar.metadata.api.MetadataCacheConfig;
import org.apache.pulsar.metadata.api.MetadataSerde;
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.MetadataStoreException.AlreadyExistsException;
import org.apache.pulsar.metadata.api.MetadataStoreException.ContentDeserializationException;
import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException;
Expand All @@ -60,6 +70,7 @@
import org.apache.pulsar.metadata.api.extended.CreateOption;
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;

Expand Down Expand Up @@ -488,32 +499,74 @@ public void readModifyUpdate(String provider, Supplier<String> urlSupplier) thro
public void readModifyUpdateBadVersionRetry() throws Exception {
String url = zks.getConnectionString();
@Cleanup
MetadataStore sourceStore1 = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build());
@Cleanup
MetadataStore sourceStore2 = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build());
MetadataStore store = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build());

MetadataCache<MyClass> objCache1 = sourceStore1.getMetadataCache(MyClass.class);
MetadataCache<MyClass> objCache2 = sourceStore2.getMetadataCache(MyClass.class);
MetadataCache<MyClass> cache = store.getMetadataCache(MyClass.class);

String key1 = newKey();

MyClass value1 = new MyClass("a", 1);
objCache1.create(key1, value1).join();
assertEquals(objCache1.get(key1).join().get().b, 1);
cache.create(key1, value1).join();
assertEquals(cache.get(key1).join().get().b, 1);

CompletableFuture<MyClass> future1 = objCache1.readModifyUpdate(key1, v -> {
return new MyClass(v.a, v.b + 1);
});

CompletableFuture<MyClass> future2 = objCache2.readModifyUpdate(key1, v -> {
return new MyClass(v.a, v.b + 1);
});
final var futures = new ArrayList<CompletableFuture<MyClass>>();
final var sourceStores = new ArrayList<MetadataStore>();

MyClass myClass1 = future1.join();
assertEquals(myClass1.b, 2);
for (int i = 0; i < 20; i++) {
final var sourceStore = MetadataStoreFactory.create(url, 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)));
}
FutureUtil.waitForAll(futures).join();
for (var sourceStore : sourceStores) {
sourceStore.close();
}
}

MyClass myClass2 = future2.join();
assertEquals(myClass2.b, 3);
@Test
public void readModifyUpdateOrCreateRetryTimeout() throws Exception {
String url = zks.getConnectionString();
@Cleanup
MetadataStore store = MetadataStoreFactory.create(url, MetadataStoreConfig.builder().build());

MetadataCache<MyClass> cache = store.getMetadataCache(MyClass.class, MetadataCacheConfig.builder()
.retryBackoff(new BackoffBuilder()
.setInitialTime(5, TimeUnit.MILLISECONDS)
.setMax(1, TimeUnit.SECONDS)
.setMandatoryStop(3, TimeUnit.SECONDS)).build());

Field metadataCacheField = cache.getClass().getDeclaredField("objCache");
metadataCacheField.setAccessible(true);
var objCache = metadataCacheField.get(cache);
var spyObjCache = (AsyncLoadingCache<?, ?>) spy(objCache);
doAnswer((Answer<CompletableFuture<MyClass>>) invocation -> CompletableFuture.failedFuture(
new MetadataStoreException.BadVersionException(""))).when(spyObjCache).get(any());
metadataCacheField.set(cache, spyObjCache);

// Test three times to ensure that the retry works each time.
for (int i = 0; i < 3; i++) {
var start = System.currentTimeMillis();
boolean timeouted = false;
try {
cache.readModifyUpdateOrCreate(newKey(), Optional::get).join();
} catch (CompletionException e) {
if (e.getCause() instanceof TimeoutException) {
var elapsed = System.currentTimeMillis() - start;
// Since we reduce the wait time by a random amount for each retry, the total elapsed time should be
// mandatoryStopTime - maxTime * 0.9, which is 2900ms.
assertTrue(elapsed >= 2900L,
"The elapsed time should be greater than the timeout. But now it's " + elapsed);
// The elapsed time should be less than the timeout. The 1.5 factor allows for some extra time.
assertTrue(elapsed < 3000L * 1.5,
"The retry should have been stopped after the timeout. But now it's " + elapsed);
timeouted = true;
} else {
fail("Should have failed with TimeoutException, but failed with " + e.getCause());
}
}
assertTrue(timeouted, "Should have failed with TimeoutException, but succeeded");
}
}

@Test(dataProvider = "impl")
Expand Down Expand Up @@ -647,4 +700,15 @@ public void testAsyncReloadConsumer(String provider, Supplier<String> urlSupplie
refreshed.contains(value2);
});
}

@Test
public void testDefaultMetadataCacheConfig() {
final var config = MetadataCacheConfig.builder().build();
assertEquals(config.getRefreshAfterWriteMillis(), TimeUnit.MINUTES.toMillis(5));
assertEquals(config.getExpireAfterWriteMillis(), TimeUnit.MINUTES.toMillis(10));
final var backoff = config.getRetryBackoff().create();
assertEquals(backoff.getInitial(), 5);
assertEquals(backoff.getMax(), 3000);
assertEquals(backoff.getMandatoryStop(), 30_000);
}
}

0 comments on commit 0ae3f9d

Please sign in to comment.