-
Notifications
You must be signed in to change notification settings - Fork 65
/
FlinkPravegaWriter.java
216 lines (185 loc) · 8.15 KB
/
FlinkPravegaWriter.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
/**
* Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved.
*
* 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
*/
package io.pravega.connectors.flink;
import io.pravega.client.ClientFactory;
import io.pravega.client.stream.EventStreamWriter;
import io.pravega.client.stream.EventWriterConfig;
import io.pravega.client.stream.Serializer;
import com.google.common.base.Preconditions;
import lombok.extern.slf4j.Slf4j;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
import org.apache.flink.streaming.util.serialization.SerializationSchema;
import java.io.IOException;
import java.io.Serializable;
import java.net.URI;
import java.nio.ByteBuffer;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
/**
* Flink sink implementation for writing into pravega storage.
*
* @param <T> The type of the event to be written.
*/
@Slf4j
public class FlinkPravegaWriter<T> extends RichSinkFunction<T> implements CheckpointedFunction, Serializable {
private static final long serialVersionUID = 1L;
// The supplied event serializer.
private final SerializationSchema<T> serializationSchema;
// The router used to partition events within a stream.
private final PravegaEventRouter<T> eventRouter;
// The pravega controller endpoint.
private final URI controllerURI;
// The scope name of the destination stream.
private final String scopeName;
// The pravega stream name to write events to.
private final String streamName;
// The sink's mode of operation. This is used to provide different guarantees for the written events.
private PravegaWriterMode writerMode = PravegaWriterMode.ATLEAST_ONCE;
// Following runtime parameters are populated when the flink sub-tasks are executed at the task managers.
// The pravega writer client.
private transient EventStreamWriter<T> pravegaWriter = null;
// The event serializer implementation for the pravega writer.
private transient Serializer<T> eventSerializer = null;
// Error which will be detected asynchronously and reported to flink.
private transient AtomicReference<Throwable> writeError = null;
// Used to track confirmation from all writes to ensure guaranteed writes.
private transient AtomicInteger pendingWritesCount = null;
// Thread pool for handling callbacks from write events.
private transient ExecutorService executorService = null;
/**
* The flink pravega writer instance which can be added as a sink to a flink job.
*
* @param controllerURI The pravega controller endpoint address.
* @param scope The destination stream's scope name.
* @param streamName The destination stream Name.
* @param serializationSchema The implementation for serializing every event into pravega's storage format.
* @param router The implementation to extract the partition key from the event.
*/
public FlinkPravegaWriter(final URI controllerURI, final String scope, final String streamName,
final SerializationSchema<T> serializationSchema, final PravegaEventRouter<T> router) {
Preconditions.checkNotNull(controllerURI);
Preconditions.checkNotNull(scope);
Preconditions.checkNotNull(streamName);
Preconditions.checkNotNull(serializationSchema);
Preconditions.checkNotNull(router);
this.controllerURI = controllerURI;
this.scopeName = scope;
this.streamName = streamName;
this.serializationSchema = serializationSchema;
this.eventRouter = router;
}
/**
* Gets the associated event router.
*/
public PravegaEventRouter<T> getEventRouter() {
return this.eventRouter;
}
/**
* Set this writer's operating mode.
*
* @param writerMode The mode of operation.
*/
public void setPravegaWriterMode(PravegaWriterMode writerMode) {
Preconditions.checkNotNull(writerMode);
this.writerMode = writerMode;
}
@Override
public void open(Configuration parameters) throws Exception {
this.eventSerializer = new Serializer<T>() {
@Override
public ByteBuffer serialize(T event) {
return ByteBuffer.wrap(serializationSchema.serialize(event));
}
@Override
public T deserialize(ByteBuffer serializedValue) {
throw new IllegalStateException("deserialize() called for a serializer");
}
};
this.writeError = new AtomicReference<>(null);
this.pendingWritesCount = new AtomicInteger(0);
ClientFactory clientFactory = ClientFactory.withScope(this.scopeName, this.controllerURI);
this.pravegaWriter = clientFactory.createEventWriter(
this.streamName,
this.eventSerializer,
EventWriterConfig.builder().build());
this.executorService = Executors.newFixedThreadPool(5);
log.info("Initialized pravega writer for stream: {}/{} with controller URI: {}", this.scopeName,
this.streamName, this.controllerURI);
}
@Override
public void close() throws Exception {
if (this.writerMode == PravegaWriterMode.ATLEAST_ONCE) {
flushAndVerify();
}
this.pravegaWriter.close();
}
@Override
public void invoke(T event) throws Exception {
if (this.writerMode == PravegaWriterMode.ATLEAST_ONCE) {
checkWriteError();
}
this.pendingWritesCount.incrementAndGet();
final CompletableFuture<Void> future = this.pravegaWriter.writeEvent(this.eventRouter.getRoutingKey(event), event);
if (writerMode == PravegaWriterMode.ATLEAST_ONCE) {
future.whenCompleteAsync(
(result, e) -> {
if (e == null) {
synchronized (this) {
pendingWritesCount.decrementAndGet();
this.notify();
}
} else {
log.warn("Detected a write failure: {}", e);
// We will record only the first error detected, since this will mostly likely help with
// finding the root cause. Storing all errors will not be feasible.
writeError.compareAndSet(null, e);
}
},
executorService
);
}
}
@Override
public void snapshotState(FunctionSnapshotContext context) throws Exception {
if (this.writerMode == PravegaWriterMode.ATLEAST_ONCE) {
log.debug("Snapshot triggered, wait for all pending writes to complete");
flushAndVerify();
}
}
@Override
public void initializeState(FunctionInitializationContext context) throws Exception {
// Nothing to restore.
}
// Wait until all pending writes are completed and throw any errors detected.
private void flushAndVerify() throws Exception {
this.pravegaWriter.flush();
// Wait until all errors, if any, have been recorded.
synchronized (this) {
while (this.pendingWritesCount.get() > 0) {
this.wait();
}
}
// Verify that no events have been lost so far.
checkWriteError();
}
private void checkWriteError() throws Exception {
Throwable error = this.writeError.getAndSet(null);
if (error != null) {
throw new IOException("Write failure", error);
}
}
}