Skip to content

Commit 68199af

Browse files
X-czhreswqa
authored andcommitted
[FLINK-36905][doc] Update Chinese doc on serialization to reflect the latest changes in Flink 2.0
1 parent eb59884 commit 68199af

File tree

3 files changed

+139
-222
lines changed

3 files changed

+139
-222
lines changed

Diff for: docs/content.zh/docs/dev/datastream/fault-tolerance/serialization/custom_serialization.md

+12-25
Original file line numberDiff line numberDiff line change
@@ -42,8 +42,6 @@ to specify the state's name, as well as information about the type of the state.
4242
It is also possible to completely bypass this and let Flink use your own custom serializer to serialize managed states,
4343
simply by directly instantiating the `StateDescriptor` with your own `TypeSerializer` implementation:
4444

45-
{{< tabs "ee215ff6-2e21-4a40-a1b4-7f114560546f" >}}
46-
{{< tab "Java" >}}
4745
```java
4846
public class CustomTypeSerializer extends TypeSerializer<Tuple2<String, Integer>> {...};
4947

@@ -54,20 +52,6 @@ ListStateDescriptor<Tuple2<String, Integer>> descriptor =
5452

5553
checkpointedState = getRuntimeContext().getListState(descriptor);
5654
```
57-
{{< /tab >}}
58-
{{< tab "Scala" >}}
59-
```scala
60-
class CustomTypeSerializer extends TypeSerializer[(String, Integer)] {...}
61-
62-
val descriptor = new ListStateDescriptor[(String, Integer)](
63-
"state-name",
64-
new CustomTypeSerializer)
65-
)
66-
67-
checkpointedState = getRuntimeContext.getListState(descriptor)
68-
```
69-
{{< /tab >}}
70-
{{< /tabs >}}
7155

7256
## State serializers and schema evolution
7357

@@ -151,7 +135,7 @@ To wrap up, this section concludes how Flink, or more specifically the state bac
151135
abstractions. The interaction is slightly different depending on the state backend, but this is orthogonal
152136
to the implementation of state serializers and their serializer snapshots.
153137

154-
#### Off-heap state backends (e.g. `RocksDBStateBackend`)
138+
#### Off-heap state backends (e.g. `EmbeddedRocksDBStateBackend`)
155139

156140
1. **Register new state with a state serializer that has schema _A_**
157141
- the registered `TypeSerializer` for the state is used to read / write state on every state access.
@@ -172,7 +156,7 @@ to the implementation of state serializers and their serializer snapshots.
172156
of the accessed state is migrated all-together before processing continues.
173157
- If the resolution signals incompatibility, then the state access fails with an exception.
174158

175-
#### Heap state backends (e.g. `MemoryStateBackend`, `FsStateBackend`)
159+
#### Heap state backends (e.g. `HashMapStateBackend`)
176160

177161
1. **Register new state with a state serializer that has schema _A_**
178162
- the registered `TypeSerializer` is maintained by the state backend.
@@ -284,6 +268,7 @@ public class MapSerializerSnapshot<K, V> extends CompositeTypeSerializerSnapshot
284268
}
285269
```
286270

271+
287272
When implementing a new serializer snapshot as a subclass of `CompositeTypeSerializerSnapshot`,
288273
the following three methods must be implemented:
289274
* `#getCurrentOuterSnapshotVersion()`: This method defines the version of
@@ -311,6 +296,7 @@ has outer snapshot information, then all three methods must be implemented.
311296
Below is an example of how the `CompositeTypeSerializerSnapshot` is used for composite serializer snapshots
312297
that do have outer snapshot information, using Flink's `GenericArraySerializer` as an example:
313298

299+
314300
```java
315301
public final class GenericArraySerializerSnapshot<C> extends CompositeTypeSerializerSnapshot<C[], GenericArraySerializer> {
316302

@@ -365,6 +351,7 @@ public final class GenericArraySerializerSnapshot<C> extends CompositeTypeSerial
365351
}
366352
```
367353

354+
368355
There are two important things to notice in the above code snippet. First of all, since this
369356
`CompositeTypeSerializerSnapshot` implementation has outer snapshot information that is written as part of the snapshot,
370357
the outer snapshot version, as defined by `getCurrentOuterSnapshotVersion()`, must be upticked whenever the
@@ -449,18 +436,18 @@ migrate from the old abstractions. The steps to do this is as follows:
449436

450437
This section is a guide for a method migration from the serializer snapshots that existed before Flink 1.19.
451438

452-
Before Flink 1.19, when using a customized serializer to process data, the schema compatibility in the old serializer
453-
(maybe in Flink library) has to meet the future need.
454-
Or else TypeSerializerSnapshot#resolveSchemaCompatibility(TypeSerializer<T> newSerializer) of the old serializer has to be modified.
455-
There are no ways to specify the compatibility with the old serializer in the new serializer, which also makes scheme evolution
439+
Before Flink 1.19, when using a customized serializer to process data, the schema compatibility in the old serializer
440+
(maybe in Flink library) has to meet the future need.
441+
Or else TypeSerializerSnapshot#resolveSchemaCompatibility(TypeSerializer<T> newSerializer) of the old serializer has to be modified.
442+
There are no ways to specify the compatibility with the old serializer in the new serializer, which also makes scheme evolution
456443
not supported in some scenarios.
457444

458-
So from Flink 1.19, the direction of resolving schema compatibility has been reversed. The old method
459-
`TypeSerializerSnapshot#resolveSchemaCompatibility(TypeSerializer newSerializer)` is now removed and needs to be replaced with
445+
So from Flink 1.19, the direction of resolving schema compatibility has been reversed. The old method
446+
`TypeSerializerSnapshot#resolveSchemaCompatibility(TypeSerializer newSerializer)` is now removed and needs to be replaced with
460447
`TypeSerializerSnapshot#resolveSchemaCompatibility(TypeSerializerSnapshot oldSerializerSnapshot)`.
461448
To make this transition, follow these steps:
462449

463-
1. Implement the `TypeSerializerSnapshot#resolveSchemaCompatibility(TypeSerializerSnapshot oldSerializerSnapshot)` whose logic
450+
1. Implement the `TypeSerializerSnapshot#resolveSchemaCompatibility(TypeSerializerSnapshot oldSerializerSnapshot)` whose logic
464451
should be same as the original `TypeSerializerSnapshot#resolveSchemaCompatibility(TypeSerializer newSerializer)`.
465452
2. Remove the old method `TypeSerializerSnapshot#resolveSchemaCompatibility(TypeSerializer newSerializer)`.
466453

Diff for: docs/content.zh/docs/dev/datastream/fault-tolerance/serialization/custom_serializers.md renamed to docs/content.zh/docs/dev/datastream/fault-tolerance/serialization/third_party_serializers.md

+22-19
Original file line numberDiff line numberDiff line change
@@ -29,34 +29,37 @@ under the License.
2929

3030
如果在 Flink 程序中使用了 Flink 类型序列化器无法进行序列化的用户自定义类型,Flink 会回退到通用的 Kryo 序列化器。
3131
可以使用 Kryo 注册自己的序列化器或序列化系统,比如 Google Protobuf 或 Apache Thrift。
32-
使用方法是在 Flink 程序中的 `ExecutionConfig` 注册类类型以及序列化器。
32+
使用方法是在 Flink 程序中使用配置 [pipeline.serialization-config]({{< ref "docs/deployment/config#pipeline-serialization-config" >}})
33+
注册类类型以及序列化器:
3334

34-
```java
35-
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
36-
37-
// 为类型注册序列化器类
38-
env.getConfig().registerTypeWithKryoSerializer(MyCustomType.class, MyCustomSerializer.class);
39-
40-
// 为类型注册序列化器实例
41-
MySerializer mySerializer = new MySerializer();
42-
env.getConfig().registerTypeWithKryoSerializer(MyCustomType.class, mySerializer);
35+
```yaml
36+
pipeline.serialization-config:
37+
- org.example.MyCustomType: {type: kryo, kryo-type: registered, class: org.example.MyCustomSerializer}
4338
```
4439
45-
需要确保你的自定义序列化器继承了 Kryo 的序列化器类。
46-
对于 Google Protobuf 或 Apache Thrift,这一点已经为你做好了:
40+
你也可以使用代码设置:
4741
4842
```java
43+
Configuration config = new Configuration();
4944

50-
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
45+
// register the class of the serializer as serializer for a type
46+
config.set(PipelineOptions.SERIALIZATION_CONFIG,
47+
List.of("org.example.MyCustomType: {type: kryo, kryo-type: registered, class: org.example.MyCustomSerializer}"));
5148

52-
// 使用 Kryo 注册 Google Protobuf 序列化器
53-
env.getConfig().registerTypeWithKryoSerializer(MyCustomType.class, ProtobufSerializer.class);
49+
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config);
50+
```
5451

55-
// 注册 Apache Thrift 序列化器为标准序列化器
56-
// TBaseSerializer 需要初始化为默认的 kryo 序列化器
57-
env.getConfig().addDefaultKryoSerializer(MyCustomType.class, TBaseSerializer.class);
52+
需要确保你的自定义序列化器继承了 Kryo 的序列化器类。
53+
对于 Google Protobuf 或 Apache Thrift,这一点已经为你做好了:
5854

59-
```
55+
```yaml
56+
pipeline.serialization-config:
57+
# register the Google Protobuf serializer with Kryo
58+
- org.example.MyCustomProtobufType: {type: kryo, kryo-type: registered, class: com.twitter.chill.protobuf.ProtobufSerializer}
59+
# register the serializer included with Apache Thrift as the standard serializer
60+
# TBaseSerializer states it should be initialized as a default Kryo serializer
61+
- org.example.MyCustomThriftType: {type: kryo, kryo-type: default, class: com.twitter.chill.thrift.TBaseSerializer}
62+
````
6063

6164
为了使上面的例子正常工作,需要在 Maven 项目文件中(pom.xml)包含必要的依赖。
6265
为 Apache Thrift 添加以下依赖:

0 commit comments

Comments
 (0)