forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
/
CompositeSerializer.java
320 lines (277 loc) · 11.4 KB
/
CompositeSerializer.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
/*
* 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.flink.api.common.typeutils;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.util.Preconditions;
import javax.annotation.Nonnull;
import java.io.IOException;
import java.io.Serializable;
import java.util.Arrays;
import java.util.List;
/**
* Base class for composite serializers.
*
* <p>This class serializes a composite type using array of its field serializers.
* Fields are indexed the same way as their serializers.
*
* @param <T> type of custom serialized value
*/
public abstract class CompositeSerializer<T> extends TypeSerializer<T> {
private static final long serialVersionUID = 1L;
/** Serializers for fields which constitute T. */
protected final TypeSerializer<Object>[] fieldSerializers;
final PrecomputedParameters precomputed;
/** Can be used for user facing constructor. */
@SuppressWarnings("unchecked")
protected CompositeSerializer(boolean immutableTargetType, TypeSerializer<?> ... fieldSerializers) {
this(
PrecomputedParameters.precompute(immutableTargetType, (TypeSerializer<Object>[]) fieldSerializers),
fieldSerializers);
}
/** Can be used in createSerializerInstance for internal operations. */
@SuppressWarnings("unchecked")
protected CompositeSerializer(PrecomputedParameters precomputed, TypeSerializer<?> ... fieldSerializers) {
this.fieldSerializers = (TypeSerializer<Object>[]) fieldSerializers;
this.precomputed = precomputed;
}
/** Create new instance from its fields. */
public abstract T createInstance(@Nonnull Object ... values);
/** Modify field of existing instance. Supported only by mutable types. */
protected abstract void setField(@Nonnull T value, int index, Object fieldValue);
/** Get field of existing instance. */
protected abstract Object getField(@Nonnull T value, int index);
/** Factory for concrete serializer. */
protected abstract CompositeSerializer<T> createSerializerInstance(
PrecomputedParameters precomputed,
TypeSerializer<?> ... originalSerializers);
@Override
public CompositeSerializer<T> duplicate() {
return precomputed.stateful ?
createSerializerInstance(precomputed, duplicateFieldSerializers(fieldSerializers)) : this;
}
private static TypeSerializer[] duplicateFieldSerializers(TypeSerializer<Object>[] fieldSerializers) {
TypeSerializer[] duplicatedSerializers = new TypeSerializer[fieldSerializers.length];
for (int index = 0; index < fieldSerializers.length; index++) {
duplicatedSerializers[index] = fieldSerializers[index].duplicate();
assert duplicatedSerializers[index] != null;
}
return duplicatedSerializers;
}
@Override
public boolean isImmutableType() {
return precomputed.immutable;
}
@Override
public T createInstance() {
Object[] fields = new Object[fieldSerializers.length];
for (int index = 0; index < fieldSerializers.length; index++) {
fields[index] = fieldSerializers[index].createInstance();
}
return createInstance(fields);
}
@Override
public T copy(T from) {
Preconditions.checkNotNull(from);
if (isImmutableType()) {
return from;
}
Object[] fields = new Object[fieldSerializers.length];
for (int index = 0; index < fieldSerializers.length; index++) {
fields[index] = fieldSerializers[index].copy(getField(from, index));
}
return createInstance(fields);
}
@Override
public T copy(T from, T reuse) {
Preconditions.checkNotNull(from);
Preconditions.checkNotNull(reuse);
if (isImmutableType()) {
return from;
}
Object[] fields = new Object[fieldSerializers.length];
for (int index = 0; index < fieldSerializers.length; index++) {
fields[index] = fieldSerializers[index].copy(getField(from, index), getField(reuse, index));
}
return createInstanceWithReuse(fields, reuse);
}
@Override
public int getLength() {
return precomputed.length;
}
@Override
public void serialize(T record, DataOutputView target) throws IOException {
Preconditions.checkNotNull(record);
Preconditions.checkNotNull(target);
for (int index = 0; index < fieldSerializers.length; index++) {
fieldSerializers[index].serialize(getField(record, index), target);
}
}
@Override
public T deserialize(DataInputView source) throws IOException {
Preconditions.checkNotNull(source);
Object[] fields = new Object[fieldSerializers.length];
for (int i = 0; i < fieldSerializers.length; i++) {
fields[i] = fieldSerializers[i].deserialize(source);
}
return createInstance(fields);
}
@Override
public T deserialize(T reuse, DataInputView source) throws IOException {
Preconditions.checkNotNull(reuse);
Preconditions.checkNotNull(source);
Object[] fields = new Object[fieldSerializers.length];
for (int index = 0; index < fieldSerializers.length; index++) {
fields[index] = fieldSerializers[index].deserialize(getField(reuse, index), source);
}
return precomputed.immutable ? createInstance(fields) : createInstanceWithReuse(fields, reuse);
}
private T createInstanceWithReuse(Object[] fields, T reuse) {
for (int index = 0; index < fields.length; index++) {
setField(reuse, index, fields[index]);
}
return reuse;
}
@Override
public void copy(DataInputView source, DataOutputView target) throws IOException {
Preconditions.checkNotNull(source);
Preconditions.checkNotNull(target);
for (TypeSerializer typeSerializer : fieldSerializers) {
typeSerializer.copy(source, target);
}
}
@Override
public int hashCode() {
return 31 * Boolean.hashCode(precomputed.immutableTargetType) + Arrays.hashCode(fieldSerializers);
}
@SuppressWarnings("EqualsWhichDoesntCheckParameterClass")
@Override
public boolean equals(Object obj) {
if (canEqual(obj)) {
CompositeSerializer<?> other = (CompositeSerializer<?>) obj;
return precomputed.immutable == other.precomputed.immutable
&& Arrays.equals(fieldSerializers, other.fieldSerializers);
}
return false;
}
@Override
public boolean canEqual(Object obj) {
// as this is an abstract class, we allow equality only between instances of the same class
return obj != null && getClass().equals(obj.getClass());
}
@Override
public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
// We can not remove this method, as long as we support restoring into CompositeTypeSerializerConfigSnapshot.
// Previously (pre 1.8), multiple composite serializers were using this class directly as their snapshot class.
if (configSnapshot instanceof ConfigSnapshot) {
List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> previousSerializersAndConfigs =
((CompositeTypeSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
if (previousSerializersAndConfigs.size() == fieldSerializers.length) {
return ensureFieldCompatibility(previousSerializersAndConfigs);
}
}
return CompatibilityResult.requiresMigration();
}
@SuppressWarnings("unchecked")
private CompatibilityResult<T> ensureFieldCompatibility(
List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> previousSerializersAndConfigs) {
TypeSerializer<Object>[] convertSerializers = new TypeSerializer[fieldSerializers.length];
boolean requiresMigration = false;
for (int index = 0; index < previousSerializersAndConfigs.size(); index++) {
CompatibilityResult<Object> compatResult =
resolveFieldCompatibility(previousSerializersAndConfigs, index);
if (compatResult.isRequiresMigration()) {
requiresMigration = true;
if (compatResult.getConvertDeserializer() != null) {
convertSerializers[index] = new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer());
} else {
return CompatibilityResult.requiresMigration();
}
}
}
return requiresMigration ? createMigrationCompatResult(convertSerializers) : CompatibilityResult.compatible();
}
private CompatibilityResult<Object> resolveFieldCompatibility(
List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> previousSerializersAndConfigs, int index) {
return CompatibilityUtil.resolveCompatibilityResult(
previousSerializersAndConfigs.get(index).f0, UnloadableDummyTypeSerializer.class,
previousSerializersAndConfigs.get(index).f1, fieldSerializers[index]);
}
private CompatibilityResult<T> createMigrationCompatResult(TypeSerializer<Object>[] convertSerializers) {
PrecomputedParameters precomputed =
PrecomputedParameters.precompute(this.precomputed.immutableTargetType, convertSerializers);
return CompatibilityResult.requiresMigration(createSerializerInstance(precomputed, convertSerializers));
}
/** This class holds composite serializer parameters which can be precomputed in advanced for better performance. */
protected static class PrecomputedParameters implements Serializable {
private static final long serialVersionUID = 1L;
/** Whether target type is immutable. */
final boolean immutableTargetType;
/** Whether target type and its fields are immutable. */
final boolean immutable;
/** Byte length of target object in serialized form. */
private final int length;
/** Whether any field serializer is stateful. */
final boolean stateful;
private PrecomputedParameters(boolean immutableTargetType, boolean immutable, int length, boolean stateful) {
this.immutableTargetType = immutableTargetType;
this.immutable = immutable;
this.length = length;
this.stateful = stateful;
}
static PrecomputedParameters precompute(
boolean immutableTargetType,
TypeSerializer<Object>[] fieldSerializers) {
Preconditions.checkNotNull(fieldSerializers);
int totalLength = 0;
boolean fieldsImmutable = true;
boolean stateful = false;
for (TypeSerializer<Object> fieldSerializer : fieldSerializers) {
Preconditions.checkNotNull(fieldSerializer);
if (fieldSerializer != fieldSerializer.duplicate()) {
stateful = true;
}
if (!fieldSerializer.isImmutableType()) {
fieldsImmutable = false;
}
if (fieldSerializer.getLength() < 0) {
totalLength = -1;
}
totalLength = totalLength >= 0 ? totalLength + fieldSerializer.getLength() : totalLength;
}
return new PrecomputedParameters(immutableTargetType, fieldsImmutable, totalLength, stateful);
}
}
/** Snapshot field serializers of composite type. */
@Deprecated
public static class ConfigSnapshot extends CompositeTypeSerializerConfigSnapshot {
private static final int VERSION = 0;
/** This empty nullary constructor is required for deserializing the configuration. */
@SuppressWarnings("unused")
public ConfigSnapshot() {
}
ConfigSnapshot(@Nonnull TypeSerializer<?>... nestedSerializers) {
super(nestedSerializers);
}
@Override
public int getVersion() {
return VERSION;
}
}
}