|
| 1 | +/* |
| 2 | + * Copyright 2017-2025 O2 Czech Republic, a.s. |
| 3 | + * |
| 4 | + * Licensed under the Apache License, Version 2.0 (the "License"); |
| 5 | + * you may not use this file except in compliance with the License. |
| 6 | + * You may obtain a copy of the License at |
| 7 | + * |
| 8 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 9 | + * |
| 10 | + * Unless required by applicable law or agreed to in writing, software |
| 11 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 12 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 13 | + * See the License for the specific language governing permissions and |
| 14 | + * limitations under the License. |
| 15 | + */ |
| 16 | +package cz.o2.proxima.beam.core.transform.retract; |
| 17 | + |
| 18 | +import cz.o2.proxima.core.functional.UnaryFunction; |
| 19 | +import cz.o2.proxima.core.util.ExceptionUtils; |
| 20 | +import java.util.Map; |
| 21 | +import lombok.AccessLevel; |
| 22 | +import lombok.Getter; |
| 23 | +import org.apache.beam.sdk.Pipeline; |
| 24 | +import org.apache.beam.sdk.coders.Coder; |
| 25 | +import org.apache.beam.sdk.coders.KvCoder; |
| 26 | +import org.apache.beam.sdk.transforms.MapElements; |
| 27 | +import org.apache.beam.sdk.transforms.PTransform; |
| 28 | +import org.apache.beam.sdk.values.KV; |
| 29 | +import org.apache.beam.sdk.values.PCollection; |
| 30 | +import org.apache.beam.sdk.values.PValue; |
| 31 | +import org.apache.beam.sdk.values.PValueBase; |
| 32 | +import org.apache.beam.sdk.values.TupleTag; |
| 33 | +import org.apache.beam.sdk.values.TypeDescriptor; |
| 34 | +import org.apache.beam.sdk.values.TypeParameter; |
| 35 | + |
| 36 | +public class KeyedRetractPCollection<K, V> extends PValueBase implements PValue { |
| 37 | + |
| 38 | + private final PCollection<RetractElement<V>> unwrapped; |
| 39 | + private final TupleTag<RetractElement<V>> tag; |
| 40 | + @Getter private final TypeDescriptor<K> keyDescriptor; |
| 41 | + @Getter private final TypeDescriptor<V> valueDescriptor; |
| 42 | + |
| 43 | + @Getter(AccessLevel.PACKAGE) |
| 44 | + private final UnaryFunction<V, K> keyExtractor; |
| 45 | + |
| 46 | + KeyedRetractPCollection( |
| 47 | + PCollection<RetractElement<V>> parent, |
| 48 | + Pipeline pipeline, |
| 49 | + TypeDescriptor<V> valueType, |
| 50 | + TypeDescriptor<K> keyType, |
| 51 | + UnaryFunction<V, K> keyExtractor, |
| 52 | + TupleTag<RetractElement<V>> tag) { |
| 53 | + |
| 54 | + super(pipeline); |
| 55 | + this.unwrapped = parent; |
| 56 | + this.keyDescriptor = keyType; |
| 57 | + this.valueDescriptor = valueType; |
| 58 | + this.tag = tag; |
| 59 | + this.keyExtractor = keyExtractor; |
| 60 | + } |
| 61 | + |
| 62 | + public <OUT extends PValue> OUT apply(PTransform<KeyedRetractPCollection<K, V>, OUT> transform) { |
| 63 | + return Pipeline.applyTransform(this, transform); |
| 64 | + } |
| 65 | + |
| 66 | + public PCollection<RetractElement<KV<K, V>>> unwrapped() { |
| 67 | + TypeDescriptor<RetractElement<KV<K, V>>> type = |
| 68 | + getRetractElementTypeDescriptor(keyDescriptor, valueDescriptor); |
| 69 | + |
| 70 | + Coder<K> keyCoder = |
| 71 | + ExceptionUtils.uncheckedFactory( |
| 72 | + () -> unwrapped.getPipeline().getCoderRegistry().getCoder(keyDescriptor)); |
| 73 | + Coder<V> valueCoder = |
| 74 | + ExceptionUtils.uncheckedFactory( |
| 75 | + () -> unwrapped.getPipeline().getCoderRegistry().getCoder(valueDescriptor)); |
| 76 | + KvCoder<K, V> kvCoder = KvCoder.of(keyCoder, valueCoder); |
| 77 | + Coder<RetractElement<KV<K, V>>> outputCoder = RetractElement.Coder.of(kvCoder); |
| 78 | + return unwrapped.apply(mapToUnwrapped(keyExtractor, type)).setCoder(outputCoder); |
| 79 | + } |
| 80 | + |
| 81 | + public PCollection<RetractElement<V>> unwrappedValues() { |
| 82 | + return unwrapped; |
| 83 | + } |
| 84 | + |
| 85 | + private static <K, V> MapElements<RetractElement<V>, RetractElement<KV<K, V>>> mapToUnwrapped( |
| 86 | + UnaryFunction<V, K> keyExtractor, TypeDescriptor<RetractElement<KV<K, V>>> type) { |
| 87 | + |
| 88 | + return MapElements.into(type) |
| 89 | + .via( |
| 90 | + e -> |
| 91 | + e.isAddition() |
| 92 | + ? RetractElement.ofAddition( |
| 93 | + KV.of(keyExtractor.apply(e.getValue()), e.getValue()), e.getSeqId()) |
| 94 | + : RetractElement.ofDeletion( |
| 95 | + KV.of(keyExtractor.apply(e.getValue()), e.getValue()), e.getSeqId())); |
| 96 | + } |
| 97 | + |
| 98 | + private static <K, V> |
| 99 | + MapElements<RetractElement<V>, RetractElement<KV<K, V>>> mapToUnwrappedValue( |
| 100 | + UnaryFunction<V, K> keyExtractor, TypeDescriptor<RetractElement<KV<K, V>>> type) { |
| 101 | + |
| 102 | + return MapElements.into(type) |
| 103 | + .via( |
| 104 | + e -> |
| 105 | + e.isAddition() |
| 106 | + ? RetractElement.ofAddition( |
| 107 | + KV.of(keyExtractor.apply(e.getValue()), e.getValue()), e.getSeqId()) |
| 108 | + : RetractElement.ofDeletion( |
| 109 | + KV.of(keyExtractor.apply(e.getValue()), e.getValue()), e.getSeqId())); |
| 110 | + } |
| 111 | + |
| 112 | + private static <K, V> TypeDescriptor<RetractElement<KV<K, V>>> getRetractElementTypeDescriptor( |
| 113 | + TypeDescriptor<K> keyDesc, TypeDescriptor<V> valueDesc) { |
| 114 | + |
| 115 | + return new TypeDescriptor<RetractElement<KV<K, V>>>() {}.where( |
| 116 | + new TypeParameter<>() {}, keyDesc) |
| 117 | + .where(new TypeParameter<>() {}, valueDesc); |
| 118 | + } |
| 119 | + |
| 120 | + @Override |
| 121 | + public Map<TupleTag<?>, PValue> expand() { |
| 122 | + return Map.of(tag, unwrapped); |
| 123 | + } |
| 124 | +} |
0 commit comments