起源
事情的起源是在1.6.2的版本上,钉友发现了一个Bug,在TTL state snapshot的时,此处会抛IllegalArgumentException:
protected CompositeSerializer<TtlValue<T>> createSerializerInstance(
PrecomputedParameters precomputed,
TypeSerializer<?> ... originalSerializers) {
Preconditions.checkNotNull(originalSerializers);
//异常,length为1,原因在紧接的一行中
Preconditions.checkArgument(originalSerializers.length == 2);
//duplicate的时候只取了fieldSerializer,丢弃了TTL中timestamp的LongSerializer
return new TtlSerializer<>(precomputed, (TypeSerializer<T>) originalSerializers[1]);
}
上述BUG已经在Flink 1.6.3版本修复,在定位过程中反复跟踪和阅读了State的注册和使用源码,在此进行记录。
State 存储结构
存储结构层次如下:
- 在AbstractKeyedStateBackend中以StateDescriptor name为key存储State
// From AbstractKeyedStateBackend.java private final HashMap<String, InternalKvState<K, ?, ?>> keyValueStatesByName;
- InternalKvState中以三元组的形式存储保存数据,具体保存方式和store的类型相关,以heap方式为例:
//From AbstractHeapState.java protected final StateTable<K, N, SV> stateTable;
- StateTable的具体实现方式有两种,CopyOnWriteStateTable和NestedMapsStateTable,两者的主要区别是一种是flat的方式存储,一种是嵌套map的方式存储。重点看看CopyOnWriteStateTable的数据保存方式:
此处有两个StateTableEntry的数据,和CopyOnWriteTable的实现相关,不用太关注。// From CopyOnWriteStateTable.java /** * This is the primary entry array (hash directory) of the state table. If no incremental rehash is ongoing, this * is the only used table. **/ private StateTableEntry<K, N, S>[] primaryTable; /** * We maintain a secondary entry array while performing an incremental rehash. The purpose is to slowly migrate * entries from the primary table to this resized table array. When all entries are migrated, this becomes the new * primary table. */ private StateTableEntry<K, N, S>[] incrementalRehashTable;
- StateTableEntry就是state的封装类了,其中还包括一些hash、冲突链next指针等信息,和CopyOnWriteTable的实现强相关,是一种类hashMap的处理。
需要强调的是不同的State在具体的实现上有些差异。
State 注册过程
1. 注册时机
StreamTask在run前会先open所有的operator,operator的open方法中,我们通常会通过StateDescription 来初始化State。没错,就在此时注册。一个MapState注册的调用栈:
2. 关键过程
-
静态注册StateFactory
//From TtlStateFactory @SuppressWarnings("deprecation") private Map<Class<? extends StateDescriptor>, SupplierWithException<IS, Exception>> createStateFactories() { return Stream.of( Tuple2.of(ValueStateDescriptor.class, (SupplierWithException<IS, Exception>) this::createValueState), Tuple2.of(ListStateDescriptor.class, (SupplierWithException<IS, Exception>) this::createListState), Tuple2.of(MapStateDescriptor.class, (SupplierWithException<IS, Exception>) this::createMapState), Tuple2.of(ReducingStateDescriptor.class, (SupplierWithException<IS, Exception>) this::createReducingState), Tuple2.of(AggregatingStateDescriptor.class, (SupplierWithException<IS, Exception>) this::createAggregatingState), Tuple2.of(FoldingStateDescriptor.class, (SupplierWithException<IS, Exception>) this::createFoldingState) ).collect(Collectors.toMap(t -> t.f0, t -> t.f1)); }
-
创建 KeyedState
// From AbstractKeyedStateBackend.java public <N, S extends State, V> S getOrCreateKeyedState( final TypeSerializer<N> namespaceSerializer, StateDescriptor<S, V> stateDescriptor) throws Exception { checkNotNull(namespaceSerializer, "Namespace serializer"); checkNotNull(keySerializer, "State key serializer has not been configured in the config. " + "This operation cannot use partitioned state."); InternalKvState<K, ?, ?> kvState = keyValueStatesByName.get(stateDescriptor.getName()); if (kvState == null) { if (!stateDescriptor.isSerializerInitialized()) { // 将Tyepinfo转为Serializer,上面一句判空应该是冗余的 stateDescriptor.initializeSerializerUnlessSet(executionConfig); } // 通过KeyedStateFactory创建KVState // 用TTLStateFactory进行包装,未enale TTL的情况,就直接使用this kvState = TtlStateFactory.createStateAndWrapWithTtlIfEnabled( namespaceSerializer, stateDescriptor, this, ttlTimeProvider); //以Descriptor的名称为key keyValueStatesByName.put(stateDescriptor.getName(), kvState); publishQueryableStateIfEnabled(stateDescriptor, kvState); } return (S) kvState; }
-
以MapState为例看具体创建代码
private <UK, UV> IS createMapState() throws Exception { MapStateDescriptor<UK, UV> mapStateDesc = (MapStateDescriptor<UK, UV>) stateDesc; //这里重新包装了descriptor,是因为除了mapstate之外,还需要记录timemstap,用于判断TTL MapStateDescriptor<UK, TtlValue<UV>> ttlDescriptor = new MapStateDescriptor<>( stateDesc.getName(), mapStateDesc.getKeySerializer(), new TtlSerializer<>(mapStateDesc.getValueSerializer())); //创建MapState return (IS) new TtlMapState<>( originalStateFactory.createInternalState(namespaceSerializer, ttlDescriptor, getSnapshotTransformFactory()), ttlConfig, timeProvider, mapStateDesc.getSerializer()); }
-
对于heap backend,创建internalState的code:
// From heapKeyedStateBackend.java public <N, SV, SEV, S extends State, IS extends S> IS createInternalState( @Nonnull TypeSerializer<N> namespaceSerializer, @Nonnull StateDescriptor<S, SV> stateDesc, @Nonnull StateSnapshotTransformFactory<SEV> snapshotTransformFactory) throws Exception { //STATE_FACTORIES的定义见后文 StateFactory stateFactory = STATE_FACTORIES.get(stateDesc.getClass()); if (stateFactory == null) { String message = String.format("State %s is not supported by %s", stateDesc.getClass(), this.getClass()); throw new FlinkRuntimeException(message); } StateTable<K, N, SV> stateTable = tryRegisterStateTable( namespaceSerializer, stateDesc, getStateSnapshotTransformer(stateDesc, snapshotTransformFactory)); return stateFactory.createState(stateDesc, stateTable, keySerializer); } // 注册state private <N, V> StateTable<K, N, V> tryRegisterStateTable( TypeSerializer<N> namespaceSerializer, StateDescriptor<?, V> stateDesc, StateSnapshotTransformer<V> snapshotTransformer) throws StateMigrationException { @SuppressWarnings("unchecked") // 以descriptor的名称为key, StateTable为value StateTable<K, N, V> stateTable = (StateTable<K, N, V>) registeredKVStates.get(stateDesc.getName()); RegisteredKeyValueStateBackendMetaInfo<N, V> newMetaInfo; if (stateTable != null) { @SuppressWarnings("unchecked") StateMetaInfoSnapshot restoredMetaInfoSnapshot = restoredStateMetaInfo.get( StateUID.of(stateDesc.getName(), StateMetaInfoSnapshot.BackendStateType.KEY_VALUE)); Preconditions.checkState( restoredMetaInfoSnapshot != null, "Requested to check compatibility of a restored RegisteredKeyedBackendStateMetaInfo," + " but its corresponding restored snapshot cannot be found."); newMetaInfo = RegisteredKeyValueStateBackendMetaInfo.resolveKvStateCompatibility( restoredMetaInfoSnapshot, namespaceSerializer, stateDesc, snapshotTransformer); stateTable.setMetaInfo(newMetaInfo); } else { newMetaInfo = new RegisteredKeyValueStateBackendMetaInfo<>( stateDesc.getType(), stateDesc.getName(), namespaceSerializer, stateDesc.getSerializer(), snapshotTransformer); // 创建StateTable stateTable = snapshotStrategy.newStateTable(newMetaInfo); registeredKVStates.put(stateDesc.getName(), stateTable); } return stateTable; }
链路有点长,需要点耐心,希望上述记录可以作为看该过程代码的roadmap。