-
Notifications
You must be signed in to change notification settings - Fork 21
/
SubscriptionPartitionLoader.java
101 lines (94 loc) · 3.99 KB
/
SubscriptionPartitionLoader.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
/*
* 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.common.base.Preconditions.checkArgument;
import com.google.cloud.pubsublite.Partition;
import com.google.cloud.pubsublite.PartitionLookupUtils;
import com.google.cloud.pubsublite.SubscriptionPath;
import com.google.cloud.pubsublite.TopicPath;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.Watch;
import org.apache.beam.sdk.transforms.Watch.Growth.PollFn;
import org.apache.beam.sdk.transforms.Watch.Growth.PollResult;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.joda.time.Duration;
import org.joda.time.Instant;
class SubscriptionPartitionLoader extends PTransform<PBegin, PCollection<SubscriptionPartition>> {
private final TopicPath topic;
private final SubscriptionPath subscription;
private final SerializableFunction<TopicPath, Integer> getPartitionCount;
private final Duration pollDuration;
private final boolean terminate;
SubscriptionPartitionLoader(TopicPath topic, SubscriptionPath subscription) {
this(
topic,
subscription,
PartitionLookupUtils::numPartitions,
Duration.standardMinutes(1),
false);
}
@VisibleForTesting
SubscriptionPartitionLoader(
TopicPath topic,
SubscriptionPath subscription,
SerializableFunction<TopicPath, Integer> getPartitionCount,
Duration pollDuration,
boolean terminate) {
this.topic = topic;
this.subscription = subscription;
this.getPartitionCount = getPartitionCount;
this.pollDuration = pollDuration;
this.terminate = terminate;
}
@Override
public PCollection<SubscriptionPartition> expand(PBegin input) {
PCollection<TopicPath> start = input.apply(Create.of(ImmutableList.of(topic)));
PCollection<KV<TopicPath, Partition>> partitions =
start.apply(
Watch.growthOf(
new PollFn<TopicPath, Partition>() {
@Override
public PollResult<Partition> apply(TopicPath element, Context c) {
checkArgument(element.equals(topic));
int partitionCount = getPartitionCount.apply(element);
List<Partition> partitions =
IntStream.range(0, partitionCount)
.mapToObj(Partition::of)
.collect(Collectors.toList());
return PollResult.incomplete(Instant.now(), partitions);
}
})
.withPollInterval(pollDuration)
.withTerminationPerInput(
terminate
? Watch.Growth.afterTotalOf(pollDuration.multipliedBy(10))
: Watch.Growth.never()));
return partitions.apply(
MapElements.into(TypeDescriptor.of(SubscriptionPartition.class))
.via(kv -> SubscriptionPartition.of(subscription, kv.getValue())));
}
}