-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
Join.java
468 lines (419 loc) · 19.7 KB
/
Join.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
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
/*
* 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.extensions.joinlibrary;
import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkNotNull;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.join.CoGbkResult;
import org.apache.beam.sdk.transforms.join.CoGroupByKey;
import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TupleTag;
/**
* Utility class with different versions of joins. All methods join two collections of key/value
* pairs (KV).
*/
public class Join {
/**
* PTransform representing an inner join of two collections of KV elements.
*
* @param <K> Type of the key for both collections
* @param <V1> Type of the values for the left collection.
* @param <V2> Type of the values for the right collection.
*/
public static class InnerJoin<K, V1, V2>
extends PTransform<PCollection<KV<K, V1>>, PCollection<KV<K, KV<V1, V2>>>> {
private transient PCollection<KV<K, V2>> rightCollection;
private InnerJoin(PCollection<KV<K, V2>> rightCollection) {
this.rightCollection = rightCollection;
}
public static <K, V1, V2> InnerJoin<K, V1, V2> with(PCollection<KV<K, V2>> rightCollection) {
return new InnerJoin<>(rightCollection);
}
@Override
public PCollection<KV<K, KV<V1, V2>>> expand(PCollection<KV<K, V1>> leftCollection) {
checkNotNull(leftCollection);
checkNotNull(rightCollection);
final TupleTag<V1> v1Tuple = new TupleTag<>();
final TupleTag<V2> v2Tuple = new TupleTag<>();
PCollection<KV<K, CoGbkResult>> coGbkResultCollection =
KeyedPCollectionTuple.of(v1Tuple, leftCollection)
.and(v2Tuple, rightCollection)
.apply("CoGBK", CoGroupByKey.create());
return coGbkResultCollection
.apply(
"Join",
ParDo.of(
new DoFn<KV<K, CoGbkResult>, KV<K, KV<V1, V2>>>() {
@ProcessElement
public void processElement(ProcessContext c) {
KV<K, CoGbkResult> e = c.element();
Iterable<V1> leftValuesIterable = e.getValue().getAll(v1Tuple);
Iterable<V2> rightValuesIterable = e.getValue().getAll(v2Tuple);
for (V1 leftValue : leftValuesIterable) {
for (V2 rightValue : rightValuesIterable) {
c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue)));
}
}
}
}))
.setCoder(
KvCoder.of(
((KvCoder) leftCollection.getCoder()).getKeyCoder(),
KvCoder.of(
((KvCoder) leftCollection.getCoder()).getValueCoder(),
((KvCoder) rightCollection.getCoder()).getValueCoder())));
}
}
/**
* PTransform representing a left outer join of two collections of KV elements.
*
* @param <K> Type of the key for both collections
* @param <V1> Type of the values for the left collection.
* @param <V2> Type of the values for the right collection.
*/
public static class LeftOuterJoin<K, V1, V2>
extends PTransform<PCollection<KV<K, V1>>, PCollection<KV<K, KV<V1, V2>>>> {
private transient PCollection<KV<K, V2>> rightCollection;
private V2 nullValue;
private LeftOuterJoin(PCollection<KV<K, V2>> rightCollection, V2 nullValue) {
this.rightCollection = rightCollection;
this.nullValue = nullValue;
}
public static <K, V1, V2> LeftOuterJoin<K, V1, V2> with(
PCollection<KV<K, V2>> rightCollection, V2 nullValue) {
return new LeftOuterJoin<>(rightCollection, nullValue);
}
@Override
public PCollection<KV<K, KV<V1, V2>>> expand(PCollection<KV<K, V1>> leftCollection) {
checkNotNull(leftCollection);
checkNotNull(rightCollection);
checkNotNull(nullValue);
final TupleTag<V1> v1Tuple = new TupleTag<>();
final TupleTag<V2> v2Tuple = new TupleTag<>();
PCollection<KV<K, CoGbkResult>> coGbkResultCollection =
KeyedPCollectionTuple.of(v1Tuple, leftCollection)
.and(v2Tuple, rightCollection)
.apply("CoGBK", CoGroupByKey.create());
return coGbkResultCollection
.apply(
"Join",
ParDo.of(
new DoFn<KV<K, CoGbkResult>, KV<K, KV<V1, V2>>>() {
@ProcessElement
public void processElement(ProcessContext c) {
KV<K, CoGbkResult> e = c.element();
Iterable<V1> leftValuesIterable = e.getValue().getAll(v1Tuple);
Iterable<V2> rightValuesIterable = e.getValue().getAll(v2Tuple);
for (V1 leftValue : leftValuesIterable) {
if (rightValuesIterable.iterator().hasNext()) {
for (V2 rightValue : rightValuesIterable) {
c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue)));
}
} else {
c.output(KV.of(e.getKey(), KV.of(leftValue, nullValue)));
}
}
}
}))
.setCoder(
KvCoder.of(
((KvCoder) leftCollection.getCoder()).getKeyCoder(),
KvCoder.of(
((KvCoder) leftCollection.getCoder()).getValueCoder(),
((KvCoder) rightCollection.getCoder()).getValueCoder())));
}
}
/**
* PTransform representing a right outer join of two collections of KV elements.
*
* @param <K> Type of the key for both collections
* @param <V1> Type of the values for the left collection.
* @param <V2> Type of the values for the right collection.
*/
public static class RightOuterJoin<K, V1, V2>
extends PTransform<PCollection<KV<K, V1>>, PCollection<KV<K, KV<V1, V2>>>> {
private transient PCollection<KV<K, V2>> rightCollection;
private V1 nullValue;
private RightOuterJoin(PCollection<KV<K, V2>> rightCollection, V1 nullValue) {
this.rightCollection = rightCollection;
this.nullValue = nullValue;
}
public static <K, V1, V2> RightOuterJoin<K, V1, V2> with(
PCollection<KV<K, V2>> rightCollection, V1 nullValue) {
return new RightOuterJoin<>(rightCollection, nullValue);
}
@Override
public PCollection<KV<K, KV<V1, V2>>> expand(PCollection<KV<K, V1>> leftCollection) {
checkNotNull(leftCollection);
checkNotNull(rightCollection);
checkNotNull(nullValue);
final TupleTag<V1> v1Tuple = new TupleTag<>();
final TupleTag<V2> v2Tuple = new TupleTag<>();
PCollection<KV<K, CoGbkResult>> coGbkResultCollection =
KeyedPCollectionTuple.of(v1Tuple, leftCollection)
.and(v2Tuple, rightCollection)
.apply("CoGBK", CoGroupByKey.create());
return coGbkResultCollection
.apply(
"Join",
ParDo.of(
new DoFn<KV<K, CoGbkResult>, KV<K, KV<V1, V2>>>() {
@ProcessElement
public void processElement(ProcessContext c) {
KV<K, CoGbkResult> e = c.element();
Iterable<V1> leftValuesIterable = e.getValue().getAll(v1Tuple);
Iterable<V2> rightValuesIterable = e.getValue().getAll(v2Tuple);
for (V2 rightValue : rightValuesIterable) {
if (leftValuesIterable.iterator().hasNext()) {
for (V1 leftValue : leftValuesIterable) {
c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue)));
}
} else {
c.output(KV.of(e.getKey(), KV.of(nullValue, rightValue)));
}
}
}
}))
.setCoder(
KvCoder.of(
((KvCoder) leftCollection.getCoder()).getKeyCoder(),
KvCoder.of(
((KvCoder) leftCollection.getCoder()).getValueCoder(),
((KvCoder) rightCollection.getCoder()).getValueCoder())));
}
}
/**
* PTransform representing a full outer join of two collections of KV elements.
*
* @param <K> Type of the key for both collections
* @param <V1> Type of the values for the left collection.
* @param <V2> Type of the values for the right collection.
*/
public static class FullOuterJoin<K, V1, V2>
extends PTransform<PCollection<KV<K, V1>>, PCollection<KV<K, KV<V1, V2>>>> {
private transient PCollection<KV<K, V2>> rightCollection;
private V1 leftNullValue;
private V2 rightNullValue;
private FullOuterJoin(
PCollection<KV<K, V2>> rightCollection, V1 leftNullValue, V2 rightNullValue) {
this.rightCollection = rightCollection;
this.leftNullValue = leftNullValue;
this.rightNullValue = rightNullValue;
}
public static <K, V1, V2> FullOuterJoin<K, V1, V2> with(
PCollection<KV<K, V2>> rightCollection, V1 leftNullValue, V2 rightNullValue) {
return new FullOuterJoin<>(rightCollection, leftNullValue, rightNullValue);
}
@Override
public PCollection<KV<K, KV<V1, V2>>> expand(PCollection<KV<K, V1>> leftCollection) {
checkNotNull(leftCollection);
checkNotNull(rightCollection);
checkNotNull(leftNullValue);
checkNotNull(rightNullValue);
final TupleTag<V1> v1Tuple = new TupleTag<>();
final TupleTag<V2> v2Tuple = new TupleTag<>();
PCollection<KV<K, CoGbkResult>> coGbkResultCollection =
KeyedPCollectionTuple.of(v1Tuple, leftCollection)
.and(v2Tuple, rightCollection)
.apply("CoGBK", CoGroupByKey.create());
return coGbkResultCollection
.apply(
"Join",
ParDo.of(
new DoFn<KV<K, CoGbkResult>, KV<K, KV<V1, V2>>>() {
@ProcessElement
public void processElement(ProcessContext c) {
KV<K, CoGbkResult> e = c.element();
Iterable<V1> leftValuesIterable = e.getValue().getAll(v1Tuple);
Iterable<V2> rightValuesIterable = e.getValue().getAll(v2Tuple);
if (leftValuesIterable.iterator().hasNext()
&& rightValuesIterable.iterator().hasNext()) {
for (V2 rightValue : rightValuesIterable) {
for (V1 leftValue : leftValuesIterable) {
c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue)));
}
}
} else if (leftValuesIterable.iterator().hasNext()
&& !rightValuesIterable.iterator().hasNext()) {
for (V1 leftValue : leftValuesIterable) {
c.output(KV.of(e.getKey(), KV.of(leftValue, rightNullValue)));
}
} else if (!leftValuesIterable.iterator().hasNext()
&& rightValuesIterable.iterator().hasNext()) {
for (V2 rightValue : rightValuesIterable) {
c.output(KV.of(e.getKey(), KV.of(leftNullValue, rightValue)));
}
}
}
}))
.setCoder(
KvCoder.of(
((KvCoder) leftCollection.getCoder()).getKeyCoder(),
KvCoder.of(
((KvCoder) leftCollection.getCoder()).getValueCoder(),
((KvCoder) rightCollection.getCoder()).getValueCoder())));
}
}
/**
* Inner join of two collections of KV elements.
*
* @param leftCollection Left side collection to join.
* @param rightCollection Right side collection to join.
* @param <K> Type of the key for both collections
* @param <V1> Type of the values for the left collection.
* @param <V2> Type of the values for the right collection.
* @return A joined collection of KV where Key is the key and value is a KV where Key is of type
* V1 and Value is type V2.
*/
public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> innerJoin(
final PCollection<KV<K, V1>> leftCollection, final PCollection<KV<K, V2>> rightCollection) {
return innerJoin("InnerJoin", leftCollection, rightCollection);
}
/**
* Inner join of two collections of KV elements.
*
* @param name Name of the PTransform.
* @param leftCollection Left side collection to join.
* @param rightCollection Right side collection to join.
* @param <K> Type of the key for both collections
* @param <V1> Type of the values for the left collection.
* @param <V2> Type of the values for the right collection.
* @return A joined collection of KV where Key is the key and value is a KV where Key is of type
* V1 and Value is type V2.
*/
public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> innerJoin(
final String name,
final PCollection<KV<K, V1>> leftCollection,
final PCollection<KV<K, V2>> rightCollection) {
return leftCollection.apply(name, InnerJoin.with(rightCollection));
}
/**
* Left Outer Join of two collections of KV elements.
*
* @param name Name of the PTransform.
* @param leftCollection Left side collection to join.
* @param rightCollection Right side collection to join.
* @param nullValue Value to use as null value when right side do not match left side.
* @param <K> Type of the key for both collections
* @param <V1> Type of the values for the left collection.
* @param <V2> Type of the values for the right collection.
* @return A joined collection of KV where Key is the key and value is a KV where Key is of type
* V1 and Value is type V2. Values that should be null or empty is replaced with nullValue.
*/
public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> leftOuterJoin(
final String name,
final PCollection<KV<K, V1>> leftCollection,
final PCollection<KV<K, V2>> rightCollection,
final V2 nullValue) {
return leftCollection.apply(name, LeftOuterJoin.with(rightCollection, nullValue));
}
public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> leftOuterJoin(
final PCollection<KV<K, V1>> leftCollection,
final PCollection<KV<K, V2>> rightCollection,
final V2 nullValue) {
return leftOuterJoin("LeftOuterJoin", leftCollection, rightCollection, nullValue);
}
/**
* Right Outer Join of two collections of KV elements.
*
* @param name Name of the PTransform.
* @param leftCollection Left side collection to join.
* @param rightCollection Right side collection to join.
* @param nullValue Value to use as null value when left side do not match right side.
* @param <K> Type of the key for both collections
* @param <V1> Type of the values for the left collection.
* @param <V2> Type of the values for the right collection.
* @return A joined collection of KV where Key is the key and value is a KV where Key is of type
* V1 and Value is type V2. Values that should be null or empty is replaced with nullValue.
*/
public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> rightOuterJoin(
final String name,
final PCollection<KV<K, V1>> leftCollection,
final PCollection<KV<K, V2>> rightCollection,
final V1 nullValue) {
return leftCollection.apply(name, RightOuterJoin.with(rightCollection, nullValue));
}
/**
* Right Outer Join of two collections of KV elements.
*
* @param leftCollection Left side collection to join.
* @param rightCollection Right side collection to join.
* @param nullValue Value to use as null value when left side do not match right side.
* @param <K> Type of the key for both collections
* @param <V1> Type of the values for the left collection.
* @param <V2> Type of the values for the right collection.
* @return A joined collection of KV where Key is the key and value is a KV where Key is of type
* V1 and Value is type V2. Values that should be null or empty is replaced with nullValue.
*/
public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> rightOuterJoin(
final PCollection<KV<K, V1>> leftCollection,
final PCollection<KV<K, V2>> rightCollection,
final V1 nullValue) {
return rightOuterJoin("RightOuterJoin", leftCollection, rightCollection, nullValue);
}
/**
* Full Outer Join of two collections of KV elements.
*
* @param name Name of the PTransform.
* @param leftCollection Left side collection to join.
* @param rightCollection Right side collection to join.
* @param leftNullValue Value to use as null value when left side do not match right side.
* @param rightNullValue Value to use as null value when right side do not match right side.
* @param <K> Type of the key for both collections
* @param <V1> Type of the values for the left collection.
* @param <V2> Type of the values for the right collection.
* @return A joined collection of KV where Key is the key and value is a KV where Key is of type
* V1 and Value is type V2. Values that should be null or empty is replaced with
* leftNullValue/rightNullValue.
*/
public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> fullOuterJoin(
final String name,
final PCollection<KV<K, V1>> leftCollection,
final PCollection<KV<K, V2>> rightCollection,
final V1 leftNullValue,
final V2 rightNullValue) {
return leftCollection.apply(
name, FullOuterJoin.with(rightCollection, leftNullValue, rightNullValue));
}
/**
* Full Outer Join of two collections of KV elements.
*
* @param leftCollection Left side collection to join.
* @param rightCollection Right side collection to join.
* @param leftNullValue Value to use as null value when left side do not match right side.
* @param rightNullValue Value to use as null value when right side do not match right side.
* @param <K> Type of the key for both collections
* @param <V1> Type of the values for the left collection.
* @param <V2> Type of the values for the right collection.
* @return A joined collection of KV where Key is the key and value is a KV where Key is of type
* V1 and Value is type V2. Values that should be null or empty is replaced with
* leftNullValue/rightNullValue.
*/
public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> fullOuterJoin(
final PCollection<KV<K, V1>> leftCollection,
final PCollection<KV<K, V2>> rightCollection,
final V1 leftNullValue,
final V2 rightNullValue) {
return fullOuterJoin(
"FullOuterJoin", leftCollection, rightCollection, leftNullValue, rightNullValue);
}
}