/
SubscribeTransform.java
145 lines (130 loc) · 5.57 KB
/
SubscribeTransform.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
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
/*
* 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 static com.google.cloud.pubsublite.internal.ExtractStatus.toCanonical;
import static com.google.cloud.pubsublite.internal.UncheckedApiPreconditions.checkArgument;
import com.google.api.gax.rpc.ApiException;
import com.google.cloud.pubsublite.AdminClient;
import com.google.cloud.pubsublite.AdminClientSettings;
import com.google.cloud.pubsublite.Partition;
import com.google.cloud.pubsublite.TopicPath;
import com.google.cloud.pubsublite.internal.wire.Committer;
import com.google.cloud.pubsublite.internal.wire.Subscriber;
import com.google.cloud.pubsublite.proto.SequencedMessage;
import com.google.common.base.Stopwatch;
import com.google.common.math.LongMath;
import java.util.List;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import org.apache.beam.sdk.io.range.OffsetRange;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.joda.time.Duration;
class SubscribeTransform extends PTransform<PBegin, PCollection<SequencedMessage>> {
private static final Duration MAX_SLEEP_TIME = Duration.standardMinutes(1);
private final SubscriberOptions options;
SubscribeTransform(SubscriberOptions options) {
this.options = options;
}
private void checkSubscription(SubscriptionPartition subscriptionPartition) throws ApiException {
checkArgument(subscriptionPartition.subscription().equals(options.subscriptionPath()));
}
private Subscriber newSubscriber(Partition partition, Consumer<List<SequencedMessage>> consumer) {
try {
return options
.getSubscriberFactory(partition)
.newSubscriber(
messages ->
consumer.accept(
messages.stream()
.map(message -> message.toProto())
.collect(Collectors.toList())));
} catch (Throwable t) {
throw toCanonical(t).underlying;
}
}
private SubscriptionPartitionProcessor newPartitionProcessor(
SubscriptionPartition subscriptionPartition,
RestrictionTracker<OffsetRange, OffsetByteProgress> tracker,
OutputReceiver<SequencedMessage> receiver)
throws ApiException {
checkSubscription(subscriptionPartition);
return new SubscriptionPartitionProcessorImpl(
tracker,
receiver,
consumer -> newSubscriber(subscriptionPartition.partition(), consumer),
options.flowControlSettings());
}
private RestrictionTracker<OffsetRange, OffsetByteProgress> newRestrictionTracker(
SubscriptionPartition subscriptionPartition, OffsetRange initial) {
checkSubscription(subscriptionPartition);
return new OffsetByteRangeTracker(
initial,
options.getBacklogReader(subscriptionPartition.partition()),
Stopwatch.createUnstarted(),
MAX_SLEEP_TIME.multipliedBy(3).dividedBy(4),
LongMath.saturatedMultiply(options.flowControlSettings().bytesOutstanding(), 10));
}
private InitialOffsetReader newInitialOffsetReader(SubscriptionPartition subscriptionPartition) {
checkSubscription(subscriptionPartition);
return options.getInitialOffsetReader(subscriptionPartition.partition());
}
private Committer newCommitter(SubscriptionPartition subscriptionPartition) {
checkSubscription(subscriptionPartition);
return options.getCommitter(subscriptionPartition.partition());
}
private TopicPath getTopicPath() {
try (AdminClient admin =
AdminClient.create(
AdminClientSettings.newBuilder()
.setRegion(options.subscriptionPath().location().region())
.build())) {
return TopicPath.parse(admin.getSubscription(options.subscriptionPath()).get().getTopic());
} catch (Throwable t) {
throw toCanonical(t).underlying;
}
}
@Override
public PCollection<SequencedMessage> expand(PBegin input) {
PCollection<SubscriptionPartition> subscriptionPartitions;
if (options.partitions().isEmpty()) {
subscriptionPartitions =
input.apply(new SubscriptionPartitionLoader(getTopicPath(), options.subscriptionPath()));
} else {
subscriptionPartitions =
input.apply(
Create.of(
options.partitions().stream()
.map(
partition ->
SubscriptionPartition.of(options.subscriptionPath(), partition))
.collect(Collectors.toList())));
}
return subscriptionPartitions.apply(
ParDo.of(
new PerSubscriptionPartitionSdf(
MAX_SLEEP_TIME,
this::newInitialOffsetReader,
this::newRestrictionTracker,
this::newPartitionProcessor,
this::newCommitter)));
}
}