-
Notifications
You must be signed in to change notification settings - Fork 21
/
PerSubscriptionPartitionSdf.java
110 lines (101 loc) · 4.49 KB
/
PerSubscriptionPartitionSdf.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
/*
* Copyright 2020 Google LLC
*
* Licensed 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 com.google.cloud.pubsublite.beam;
import com.google.cloud.pubsublite.Offset;
import com.google.cloud.pubsublite.internal.ExtractStatus;
import com.google.cloud.pubsublite.internal.wire.Committer;
import com.google.cloud.pubsublite.proto.SequencedMessage;
import org.apache.beam.sdk.io.range.OffsetRange;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.SerializableBiFunction;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.MonotonicallyIncreasing;
import org.joda.time.Duration;
import org.joda.time.Instant;
class PerSubscriptionPartitionSdf extends DoFn<SubscriptionPartition, SequencedMessage> {
private final Duration maxSleepTime;
private final SubscriptionPartitionProcessorFactory processorFactory;
private final SerializableFunction<SubscriptionPartition, InitialOffsetReader>
offsetReaderFactory;
private final SerializableBiFunction<
SubscriptionPartition, OffsetRange, RestrictionTracker<OffsetRange, OffsetByteProgress>>
trackerFactory;
private final SerializableFunction<SubscriptionPartition, Committer> committerFactory;
PerSubscriptionPartitionSdf(
Duration maxSleepTime,
SerializableFunction<SubscriptionPartition, InitialOffsetReader> offsetReaderFactory,
SerializableBiFunction<
SubscriptionPartition,
OffsetRange,
RestrictionTracker<OffsetRange, OffsetByteProgress>>
trackerFactory,
SubscriptionPartitionProcessorFactory processorFactory,
SerializableFunction<SubscriptionPartition, Committer> committerFactory) {
this.maxSleepTime = maxSleepTime;
this.processorFactory = processorFactory;
this.offsetReaderFactory = offsetReaderFactory;
this.trackerFactory = trackerFactory;
this.committerFactory = committerFactory;
}
@GetInitialWatermarkEstimatorState
public Instant getInitialWatermarkState() {
return Instant.EPOCH;
}
@NewWatermarkEstimator
public MonotonicallyIncreasing newWatermarkEstimator(@WatermarkEstimatorState Instant state) {
return new MonotonicallyIncreasing(state);
}
@ProcessElement
public ProcessContinuation processElement(
RestrictionTracker<OffsetRange, OffsetByteProgress> tracker,
@Element SubscriptionPartition subscriptionPartition,
OutputReceiver<SequencedMessage> receiver)
throws Exception {
try (SubscriptionPartitionProcessor processor =
processorFactory.newProcessor(subscriptionPartition, tracker, receiver)) {
processor.start();
ProcessContinuation result = processor.waitForCompletion(maxSleepTime);
processor
.lastClaimed()
.ifPresent(
lastClaimedOffset -> {
Committer committer = committerFactory.apply(subscriptionPartition);
committer.startAsync().awaitRunning();
// Commit the next-to-deliver offset.
try {
committer.commitOffset(Offset.of(lastClaimedOffset.value() + 1)).get();
} catch (Exception e) {
throw ExtractStatus.toCanonical(e).underlying;
}
committer.stopAsync().awaitTerminated();
});
return result;
}
}
@GetInitialRestriction
public OffsetRange getInitialRestriction(@Element SubscriptionPartition subscriptionPartition) {
try (InitialOffsetReader reader = offsetReaderFactory.apply(subscriptionPartition)) {
Offset offset = reader.read();
return new OffsetRange(offset.value(), Long.MAX_VALUE /* open interval */);
}
}
@NewTracker
public RestrictionTracker<OffsetRange, OffsetByteProgress> newTracker(
@Element SubscriptionPartition subscriptionPartition, @Restriction OffsetRange range) {
return trackerFactory.apply(subscriptionPartition, range);
}
}