forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
/
SimplifiedKinesisClient.java
283 lines (261 loc) · 11.5 KB
/
SimplifiedKinesisClient.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
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.beam.sdk.io.kinesis2;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletableFuture;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
import org.joda.time.Instant;
import org.joda.time.Minutes;
import software.amazon.awssdk.core.exception.SdkClientException;
import software.amazon.awssdk.core.exception.SdkServiceException;
import software.amazon.awssdk.services.cloudwatch.CloudWatchClient;
import software.amazon.awssdk.services.cloudwatch.model.Datapoint;
import software.amazon.awssdk.services.cloudwatch.model.Dimension;
import software.amazon.awssdk.services.cloudwatch.model.GetMetricStatisticsRequest;
import software.amazon.awssdk.services.cloudwatch.model.GetMetricStatisticsResponse;
import software.amazon.awssdk.services.cloudwatch.model.Statistic;
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
import software.amazon.awssdk.services.kinesis.KinesisClient;
import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest;
import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException;
import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest;
import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest;
import software.amazon.awssdk.services.kinesis.model.LimitExceededException;
import software.amazon.awssdk.services.kinesis.model.ProvisionedThroughputExceededException;
import software.amazon.awssdk.services.kinesis.model.Record;
import software.amazon.awssdk.services.kinesis.model.Shard;
import software.amazon.awssdk.services.kinesis.model.ShardIteratorType;
import software.amazon.awssdk.services.kinesis.model.StreamDescription;
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest;
import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler;
import software.amazon.kinesis.retrieval.AggregatorUtil;
import software.amazon.kinesis.retrieval.KinesisClientRecord;
/** Wraps {@link KinesisClient} class providing much simpler interface and proper error handling. */
class SimplifiedKinesisClient {
private static final String KINESIS_NAMESPACE = "AWS/Kinesis";
private static final String INCOMING_RECORDS_METRIC = "IncomingBytes";
private static final int PERIOD_GRANULARITY_IN_SECONDS = 60;
private static final String STREAM_NAME_DIMENSION = "StreamName";
private final KinesisClient kinesis;
private final KinesisAsyncClient kinesisAsync;
private final CloudWatchClient cloudWatch;
private final Integer limit;
public SimplifiedKinesisClient(
KinesisClient kinesis,
KinesisAsyncClient kinesisAsync,
CloudWatchClient cloudWatch,
Integer limit) {
this.kinesis = checkNotNull(kinesis, "kinesis");
this.kinesisAsync = checkNotNull(kinesisAsync, "kinesisAsync");
this.cloudWatch = checkNotNull(cloudWatch, "cloudWatch");
this.limit = limit;
}
public static SimplifiedKinesisClient from(AWSClientsProvider provider, Integer limit) {
return new SimplifiedKinesisClient(
provider.getKinesisClient(),
provider.getKinesisAsyncClient(),
provider.getCloudWatchClient(),
limit);
}
public CompletableFuture<Void> subscribeToShard(
final String consumerArn,
final String shardId,
final ShardIteratorType shardIteratorType,
final String startingSequenceNumber,
final Instant timestamp,
final SubscribeToShardResponseHandler.Visitor visitor,
final Consumer<Throwable> onError)
throws TransientKinesisException {
SubscribeToShardRequest request =
SubscribeToShardRequest.builder()
.consumerARN(consumerArn)
.shardId(shardId)
.startingPosition(
s ->
s.type(shardIteratorType)
.sequenceNumber(startingSequenceNumber)
.timestamp(TimeUtil.toJava(timestamp)))
.build();
SubscribeToShardResponseHandler responseHandler =
SubscribeToShardResponseHandler.builder().subscriber(visitor).onError(onError).build();
return wrapExceptions(() -> kinesisAsync.subscribeToShard(request, responseHandler));
}
public String getShardIterator(
final String streamName,
final String shardId,
final ShardIteratorType shardIteratorType,
final String startingSequenceNumber,
final Instant timestamp)
throws TransientKinesisException {
return wrapExceptions(
() ->
kinesis
.getShardIterator(
GetShardIteratorRequest.builder()
.streamName(streamName)
.shardId(shardId)
.shardIteratorType(shardIteratorType)
.startingSequenceNumber(startingSequenceNumber)
.timestamp(TimeUtil.toJava(timestamp))
.build())
.shardIterator());
}
public List<Shard> listShards(final String streamName) throws TransientKinesisException {
return wrapExceptions(
() -> {
List<Shard> shards = Lists.newArrayList();
String lastShardId = null;
StreamDescription description;
do {
description =
kinesis
.describeStream(
DescribeStreamRequest.builder()
.streamName(streamName)
.exclusiveStartShardId(lastShardId)
.build())
.streamDescription();
shards.addAll(description.shards());
lastShardId = shards.get(shards.size() - 1).shardId();
} while (description.hasMoreShards());
return shards;
});
}
/**
* Gets records from Kinesis and deaggregates them if needed.
*
* @return list of deaggregated records
* @throws TransientKinesisException - in case of recoverable situation
*/
public GetKinesisRecordsResult getRecords(String shardIterator, String streamName, String shardId)
throws TransientKinesisException {
return getRecords(shardIterator, streamName, shardId, limit);
}
/**
* Gets records from Kinesis and deaggregates them if needed.
*
* @return list of deaggregated records
* @throws TransientKinesisException - in case of recoverable situation
*/
public GetKinesisRecordsResult getRecords(
final String shardIterator,
final String streamName,
final String shardId,
final Integer limit)
throws TransientKinesisException {
return wrapExceptions(
() -> {
GetRecordsResponse response =
kinesis.getRecords(
GetRecordsRequest.builder().shardIterator(shardIterator).limit(limit).build());
List<Record> records = response.records();
return new GetKinesisRecordsResult(
deaggregate(records),
response.nextShardIterator(),
response.millisBehindLatest(),
streamName,
shardId);
});
}
public static List<KinesisClientRecord> deaggregate(List<Record> records) {
return records.isEmpty()
? ImmutableList.of()
: new AggregatorUtil()
.deaggregate(
records.stream().map(KinesisClientRecord::fromRecord).collect(Collectors.toList()));
}
/**
* Gets total size in bytes of all events that remain in Kinesis stream after specified instant.
*
* @return total size in bytes of all Kinesis events after specified instant
*/
public long getBacklogBytes(String streamName, Instant countSince)
throws TransientKinesisException {
return getBacklogBytes(streamName, countSince, new Instant());
}
/**
* Gets total size in bytes of all events that remain in Kinesis stream between specified
* instants.
*
* @return total size in bytes of all Kinesis events after specified instant
*/
public long getBacklogBytes(
final String streamName, final Instant countSince, final Instant countTo)
throws TransientKinesisException {
return wrapExceptions(
() -> {
Minutes period = Minutes.minutesBetween(countSince, countTo);
if (period.isLessThan(Minutes.ONE)) {
return 0L;
}
GetMetricStatisticsRequest request =
createMetricStatisticsRequest(streamName, countSince, countTo, period);
long totalSizeInBytes = 0;
GetMetricStatisticsResponse response = cloudWatch.getMetricStatistics(request);
for (Datapoint point : response.datapoints()) {
totalSizeInBytes += point.sum().longValue();
}
return totalSizeInBytes;
});
}
GetMetricStatisticsRequest createMetricStatisticsRequest(
String streamName, Instant countSince, Instant countTo, Minutes period) {
return GetMetricStatisticsRequest.builder()
.namespace(KINESIS_NAMESPACE)
.metricName(INCOMING_RECORDS_METRIC)
.period(period.getMinutes() * PERIOD_GRANULARITY_IN_SECONDS)
.startTime(TimeUtil.toJava(countSince))
.endTime(TimeUtil.toJava(countTo))
.statistics(Statistic.SUM)
.dimensions(Dimension.builder().name(STREAM_NAME_DIMENSION).value(streamName).build())
.build();
}
/**
* Wraps Amazon specific exceptions into more friendly format.
*
* @throws TransientKinesisException - in case of recoverable situation, i.e. the request rate is
* too high, Kinesis remote service failed, network issue, etc.
* @throws ExpiredIteratorException - if iterator needs to be refreshed
* @throws RuntimeException - in all other cases
*/
private <T> T wrapExceptions(Callable<T> callable) throws TransientKinesisException {
try {
return callable.call();
} catch (ExpiredIteratorException e) {
throw e;
} catch (LimitExceededException | ProvisionedThroughputExceededException e) {
throw new TransientKinesisException(
"Too many requests to Kinesis. Wait some time and retry.", e);
} catch (SdkServiceException e) {
throw new TransientKinesisException("Kinesis backend failed. Wait some time and retry.", e);
} catch (SdkClientException e) {
if (e.retryable()) {
throw new TransientKinesisException("Retryable client failure", e);
}
throw new RuntimeException("Not retryable client failure", e);
} catch (Exception e) {
throw new RuntimeException("Unknown kinesis failure, when trying to reach kinesis", e);
}
}
}