42
42
43
43
import lombok .extern .slf4j .Slf4j ;
44
44
import org .apache .pulsar .client .admin .PulsarAdmin ;
45
+ import org .apache .pulsar .client .api .CryptoKeyReader ;
45
46
import org .apache .pulsar .client .api .MessageId ;
46
47
import org .apache .pulsar .client .api .MessageRouter ;
47
48
import org .apache .pulsar .client .api .MessageRoutingMode ;
62
63
import java .io .IOException ;
63
64
import java .util .ArrayList ;
64
65
import java .util .HashMap ;
66
+ import java .util .HashSet ;
65
67
import java .util .List ;
66
68
import java .util .Map ;
67
69
import java .util .Optional ;
68
70
import java .util .Properties ;
71
+ import java .util .Set ;
69
72
import java .util .concurrent .CompletableFuture ;
70
73
import java .util .concurrent .ConcurrentHashMap ;
71
74
import java .util .concurrent .ExecutionException ;
86
89
@ Slf4j
87
90
abstract class FlinkPulsarSinkBase <T > extends TwoPhaseCommitSinkFunction <T , FlinkPulsarSinkBase .PulsarTransactionState <T >, Void > implements CheckpointedFunction {
88
91
92
+ public static class Config <T > {
93
+ private String adminUrl ;
94
+ private Optional <String > defaultTopicName ;
95
+ private ClientConfigurationData clientConf ;
96
+ private Properties properties ;
97
+ private PulsarSerializationSchema <T > serializationSchema ;
98
+ private MessageRouter messageRouter ;
99
+ private PulsarSinkSemantic semantic = PulsarSinkSemantic .AT_LEAST_ONCE ;
100
+ private CryptoKeyReader cryptoKeyReader ;
101
+ private Set <String > encryptionKeys = new HashSet <>();
102
+
103
+ public Config <T > withAdminUrl (final String adminUrl ) {
104
+ this .adminUrl = adminUrl ;
105
+ return this ;
106
+ }
107
+
108
+ public Config <T > withDefaultTopicName (final String defaultTopicName ) {
109
+ this .defaultTopicName = Optional .ofNullable (defaultTopicName );
110
+ return this ;
111
+ }
112
+
113
+ public Config <T > withClientConf (ClientConfigurationData clientConf ) {
114
+ this .clientConf = clientConf ;
115
+ return this ;
116
+ }
117
+
118
+ public Config <T > withProperties (final Properties properties ) {
119
+ this .properties = properties ;
120
+ return this ;
121
+ }
122
+
123
+ public Config <T > withSerializationSchema (final PulsarSerializationSchema <T > serializationSchema ) {
124
+ this .serializationSchema = serializationSchema ;
125
+ return this ;
126
+ }
127
+
128
+ public Config <T > withMessageRouter (final MessageRouter messageRouter ) {
129
+ this .messageRouter = messageRouter ;
130
+ return this ;
131
+ }
132
+
133
+ public Config <T > withSemantic (final PulsarSinkSemantic semantic ) {
134
+ this .semantic = semantic ;
135
+ return this ;
136
+ }
137
+
138
+ public Config <T > withCryptoKeyReader (final CryptoKeyReader cryptoKeyReader ) {
139
+ this .cryptoKeyReader = cryptoKeyReader ;
140
+ return this ;
141
+ }
142
+
143
+ public Config <T > withEncryptionKeys (final Set <String > encryptionKeys ) {
144
+ this .encryptionKeys = encryptionKeys ;
145
+ return this ;
146
+ }
147
+
148
+ }
149
+
89
150
protected String adminUrl ;
90
151
91
152
protected ClientConfigurationData clientConfigurationData ;
@@ -143,6 +204,10 @@ abstract class FlinkPulsarSinkBase<T> extends TwoPhaseCommitSinkFunction<T, Flin
143
204
144
205
protected transient Map <String , Producer <T >> topic2Producer ;
145
206
207
+ private final CryptoKeyReader cryptoKeyReader ;
208
+
209
+ private final Set <String > encryptionKeys ;
210
+
146
211
public FlinkPulsarSinkBase (
147
212
String adminUrl ,
148
213
Optional <String > defaultTopicName ,
@@ -153,34 +218,29 @@ public FlinkPulsarSinkBase(
153
218
this (adminUrl , defaultTopicName , clientConf , properties , serializationSchema , messageRouter , PulsarSinkSemantic .AT_LEAST_ONCE );
154
219
}
155
220
156
- public FlinkPulsarSinkBase (
157
- String adminUrl ,
158
- Optional <String > defaultTopicName ,
159
- ClientConfigurationData clientConf ,
160
- Properties properties ,
161
- PulsarSerializationSchema <T > serializationSchema ,
162
- MessageRouter messageRouter ,
163
- PulsarSinkSemantic semantic ) {
221
+ public FlinkPulsarSinkBase (final Config <T > config ) {
164
222
super (new TransactionStateSerializer (), VoidSerializer .INSTANCE );
165
223
166
- this .adminUrl = checkNotNull (adminUrl );
167
- this .semantic = semantic ;
224
+ this .adminUrl = checkNotNull (config .adminUrl );
225
+ this .semantic = config .semantic ;
226
+ this .cryptoKeyReader = config .cryptoKeyReader ;
227
+ this .encryptionKeys = config .encryptionKeys ;
168
228
169
- if (defaultTopicName .isPresent ()) {
229
+ if (config . defaultTopicName .isPresent ()) {
170
230
this .forcedTopic = true ;
171
- this .defaultTopic = defaultTopicName .get ();
231
+ this .defaultTopic = config . defaultTopicName .get ();
172
232
} else {
173
233
this .forcedTopic = false ;
174
234
this .defaultTopic = null ;
175
235
}
176
236
177
- this .serializationSchema = serializationSchema ;
237
+ this .serializationSchema = config . serializationSchema ;
178
238
179
- this .messageRouter = messageRouter ;
239
+ this .messageRouter = config . messageRouter ;
180
240
181
- this .clientConfigurationData = clientConf ;
241
+ this .clientConfigurationData = config . clientConf ;
182
242
183
- this .properties = checkNotNull (properties );
243
+ this .properties = checkNotNull (config . properties );
184
244
185
245
this .caseInsensitiveParams =
186
246
SourceSinkUtils .toCaceInsensitiveParams (Maps .fromProperties (properties ));
@@ -216,6 +276,25 @@ public FlinkPulsarSinkBase(
216
276
}
217
277
}
218
278
279
+ public FlinkPulsarSinkBase (
280
+ String adminUrl ,
281
+ Optional <String > defaultTopicName ,
282
+ ClientConfigurationData clientConf ,
283
+ Properties properties ,
284
+ PulsarSerializationSchema <T > serializationSchema ,
285
+ MessageRouter messageRouter ,
286
+ PulsarSinkSemantic semantic ) {
287
+ this (new Config <T >()
288
+ .withAdminUrl (adminUrl )
289
+ .withDefaultTopicName (defaultTopicName .orElse (null ))
290
+ .withClientConf (clientConf )
291
+ .withProperties (properties )
292
+ .withSerializationSchema (serializationSchema )
293
+ .withMessageRouter (messageRouter )
294
+ .withSemantic (semantic )
295
+ );
296
+ }
297
+
219
298
public FlinkPulsarSinkBase (
220
299
String serviceUrl ,
221
300
String adminUrl ,
@@ -340,6 +419,12 @@ protected Producer<T> createProducer(
340
419
// maximizing the throughput
341
420
.batchingMaxBytes (5 * 1024 * 1024 )
342
421
.loadConf (producerConf );
422
+ if (cryptoKeyReader != null ){
423
+ builder .cryptoKeyReader (cryptoKeyReader );
424
+ for (final String encryptionKey : this .encryptionKeys ) {
425
+ builder .addEncryptionKey (encryptionKey );
426
+ }
427
+ }
343
428
if (messageRouter == null ) {
344
429
return builder .create ();
345
430
} else {
0 commit comments