Skip to content

Commit b87c318

Browse files
Merge pull request #1275 from data-integrations/bugfix/PLUGIN-1660-Add-retry-for-snapshot-creation
PLUGIN-1660 - Add retry for create and delete snapshot.
2 parents 2c0b6a1 + 7389355 commit b87c318

File tree

3 files changed

+103
-11
lines changed

3 files changed

+103
-11
lines changed

src/main/java/io/cdap/plugin/gcp/publisher/source/PubSubDirectDStream.java

Lines changed: 27 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,7 @@ public class PubSubDirectDStream<T> extends InputDStream<T> implements Streaming
5555

5656
private static final Logger LOG = LoggerFactory.getLogger(PubSubDirectDStream.class);
5757
private static final String CDAP_PIPELINE = "cdap_pipeline";
58+
private static final int MAX_SNAPSHOT_ATTEMPTS = 3;
5859

5960
private final Credentials credentials;
6061
private final PubSubSubscriberConfig config;
@@ -64,6 +65,7 @@ public class PubSubDirectDStream<T> extends InputDStream<T> implements Streaming
6465
private final SerializableFunction<PubSubMessage, T> mappingFn;
6566
private final StreamingContext streamingContext;
6667
private final String pipeline;
68+
private final BackoffConfig backoffConfig;
6769

6870
private SubscriptionAdminClient subscriptionAdminClient;
6971
private ProjectSnapshotName currentSnapshotName;
@@ -82,6 +84,7 @@ public PubSubDirectDStream(io.cdap.cdap.etl.api.streaming.StreamingContext conte
8284
this.pipeline = context.getPipelineName();
8385
this.credentials = PubSubSubscriberUtil.createCredentials(config.getServiceAccount(),
8486
config.isServiceAccountFilePath());
87+
backoffConfig = BackoffConfig.defaultInstance();
8588
}
8689

8790
@Override
@@ -195,7 +198,7 @@ private String generateName(String subscription) {
195198
}
196199

197200
private void createSubscriptionIfNotPresent() throws IOException, InterruptedException {
198-
PubSubSubscriberUtil.createSubscription(() -> true, BackoffConfig.defaultInstance(),
201+
PubSubSubscriberUtil.createSubscription(() -> true, backoffConfig,
199202
ProjectSubscriptionName.format(config.getProject(),
200203
config.getSubscription()),
201204
TopicName.format(config.getProject(), config.getTopic()),
@@ -243,15 +246,21 @@ private ProjectSnapshotName fetchSnapShot(String subscriptionId,
243246

244247
private Snapshot createSnapshot(ProjectSnapshotName projectSnapshotName,
245248
ProjectSubscriptionName projectSubscriptionName) {
246-
// Creation takes around 3.5 s .
247249
LOG.debug("Creating snapshot {} for subscription {} in Pub/Sub .", projectSnapshotName.toString(),
248250
projectSubscriptionName.toString());
249-
CreateSnapshotRequest request = CreateSnapshotRequest.newBuilder()
250-
.setName(projectSnapshotName.toString())
251-
.setSubscription(projectSubscriptionName.toString())
252-
.putAllLabels(Collections.singletonMap(CDAP_PIPELINE, getLabelValue(pipeline)))
253-
.build();
254-
return subscriptionAdminClient.createSnapshot(request);
251+
try {
252+
return PubSubSubscriberUtil.callWithRetry(() -> {
253+
// Creation takes around 3.5 s .
254+
CreateSnapshotRequest request = CreateSnapshotRequest.newBuilder()
255+
.setName(projectSnapshotName.toString())
256+
.setSubscription(projectSubscriptionName.toString())
257+
.putAllLabels(Collections.singletonMap(CDAP_PIPELINE, getLabelValue(pipeline)))
258+
.build();
259+
return subscriptionAdminClient.createSnapshot(request);
260+
}, backoffConfig, MAX_SNAPSHOT_ATTEMPTS);
261+
} catch (Exception e) {
262+
throw new RuntimeException(e);
263+
}
255264
}
256265

257266
@VisibleForTesting
@@ -268,9 +277,16 @@ static String getLabelValue(String pipeline) {
268277
}
269278

270279
private void deleteSnapshot(ProjectSnapshotName projectSnapshotName) {
271-
// Deletion takes around 2.5 s .
272-
// TODO - Consider making this asynchronous
273-
subscriptionAdminClient.deleteSnapshot(projectSnapshotName);
280+
try {
281+
PubSubSubscriberUtil.callWithRetry(() -> {
282+
// Deletion takes around 2.5 s .
283+
// TODO - Consider making this asynchronous
284+
subscriptionAdminClient.deleteSnapshot(projectSnapshotName);
285+
return null;
286+
}, backoffConfig, MAX_SNAPSHOT_ATTEMPTS);
287+
} catch (Exception e) {
288+
throw new RuntimeException(e);
289+
}
274290
}
275291

276292
private void saveSnapshotAsState(Snapshot snapshot, String subscription,

src/main/java/io/cdap/plugin/gcp/publisher/source/PubSubSubscriberUtil.java

Lines changed: 30 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -227,6 +227,36 @@ public static Credentials createCredentials(String serviceAccount, boolean servi
227227
}
228228
}
229229

230+
/**
231+
* Call provided supplier with retries.
232+
*
233+
* @param supplier The supplier to be invoked
234+
* @param backoffConfig {@link BackoffConfig} for the retries
235+
* @param maxAttempts Integer indicating max number of attempts
236+
* @param <T>
237+
* @return Value returned by supplier
238+
* @throws Exception Any exception that is not retryable or exceeded retry
239+
*/
240+
public static <T> T callWithRetry(Supplier<T> supplier,
241+
BackoffConfig backoffConfig, int maxAttempts) throws Exception {
242+
int backoff = backoffConfig.getInitialBackoffMs();
243+
ApiException lastApiException = null;
244+
while (maxAttempts-- > 0) {
245+
try {
246+
return supplier.get();
247+
} catch (ApiException ae) {
248+
lastApiException = ae;
249+
//Retry if the exception is retryable.
250+
if (PubSubSubscriberUtil.isApiExceptionRetryable(ae)) {
251+
backoff = PubSubSubscriberUtil.sleepAndIncreaseBackoff(() -> true, backoff, backoffConfig);
252+
continue;
253+
}
254+
throw ae;
255+
}
256+
}
257+
throw new RuntimeException(lastApiException);
258+
}
259+
230260
private static SubscriptionAdminClient buildSubscriptionAdminClient(Credentials credentials) throws IOException {
231261
SubscriptionAdminSettings.Builder builder = SubscriptionAdminSettings.newBuilder();
232262
if (credentials != null) {
Lines changed: 46 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,46 @@
1+
package io.cdap.plugin.gcp.publisher.source;
2+
3+
import com.google.api.gax.grpc.GrpcStatusCode;
4+
import com.google.api.gax.rpc.InternalException;
5+
import com.google.api.gax.rpc.NotFoundException;
6+
import io.grpc.Status;
7+
import io.grpc.StatusRuntimeException;
8+
import org.junit.Assert;
9+
import org.junit.Test;
10+
import org.mockito.Mockito;
11+
12+
import java.util.function.Supplier;
13+
14+
/**
15+
* Tests for {@link PubSubSubscriberUtil}
16+
*/
17+
public class PubSubSubscriberUtilTest {
18+
@Test(expected = NotFoundException.class)
19+
public void testCallWithRetryNonRetryable() throws Exception {
20+
PubSubSubscriberUtil.callWithRetry(() -> {
21+
throw new NotFoundException(new RuntimeException("subscription not found"),
22+
GrpcStatusCode.of(Status.Code.NOT_FOUND), false);
23+
}, BackoffConfig.defaultInstance(), 3);
24+
}
25+
26+
@Test(expected = RuntimeException.class)
27+
public void testCallWithRetryMaxRetry() throws Exception {
28+
PubSubSubscriberUtil.callWithRetry(() -> {
29+
throw new InternalException(new StatusRuntimeException(Status.INTERNAL),
30+
GrpcStatusCode.of(Status.Code.INTERNAL), false);
31+
}, BackoffConfig.defaultInstance(), 3);
32+
}
33+
34+
@Test
35+
public void testCallWithRetrySuccess() throws Exception {
36+
InternalException internalException = new InternalException(new StatusRuntimeException(Status.INTERNAL),
37+
GrpcStatusCode.of(Status.Code.INTERNAL), false);
38+
Supplier<String> testSupplier = Mockito.mock(Supplier.class);
39+
String returnValue = "success";
40+
Mockito.when(testSupplier.get()).thenThrow(internalException).thenThrow(internalException)
41+
.thenReturn(returnValue);
42+
43+
String result = PubSubSubscriberUtil.callWithRetry(testSupplier, BackoffConfig.defaultInstance(), 3);
44+
Assert.assertSame(returnValue, result);
45+
}
46+
}

0 commit comments

Comments
 (0)