Package org.apache.flink.state.forst
Class ForStReducingState<K,N,V>
- java.lang.Object
-
- org.apache.flink.runtime.state.v2.AbstractKeyedState<K,N,V>
-
- org.apache.flink.runtime.state.v2.AbstractReducingState<K,N,V>
-
- org.apache.flink.state.forst.ForStReducingState<K,N,V>
-
- Type Parameters:
K
- The type of the key.N
- The type of the namespace.V
- The type of the value.
- All Implemented Interfaces:
AggregatingState<V,V>
,AppendingState<V,V,V>
,MergingState<V,V,V>
,ReducingState<V>
,State
,InternalAggregatingState<K,N,V,V,V>
,InternalAppendingState<K,N,V,V,V,V>
,InternalKeyedState<K,N,V>
,InternalMergingState<K,N,V,V,V,V>
,InternalPartitionedState<N>
,InternalReducingState<K,N,V>
,ForStInnerTable<K,N,V>
public class ForStReducingState<K,N,V> extends AbstractReducingState<K,N,V> implements ForStInnerTable<K,N,V>
TheAbstractReducingState
implement for ForStDB.
-
-
Field Summary
-
Fields inherited from class org.apache.flink.runtime.state.v2.AbstractReducingState
reduceFunction
-
Fields inherited from class org.apache.flink.runtime.state.v2.AbstractKeyedState
stateRequestHandler
-
-
Constructor Summary
Constructors Constructor Description ForStReducingState(StateRequestHandler stateRequestHandler, org.forstdb.ColumnFamilyHandle columnFamily, ReducingStateDescriptor<V> reducingStateDescriptor, Supplier<SerializedCompositeKeyBuilder<K>> serializedKeyBuilderInitializer, N defaultNamespace, Supplier<TypeSerializer<N>> namespaceSerializerInitializer, Supplier<DataOutputSerializer> valueSerializerViewInitializer, Supplier<DataInputDeserializer> valueDeserializerViewInitializer)
-
Method Summary
All Methods Instance Methods Concrete Methods Modifier and Type Method Description ForStDBGetRequest<K,N,V,V>
buildDBGetRequest(StateRequest<?,?,?,?> stateRequest)
Build aForStDBGetRequest
that belong to thisForStInnerTable
with the given stateRequest.ForStDBPutRequest<K,N,V>
buildDBPutRequest(StateRequest<?,?,?,?> stateRequest)
Build aForStDBPutRequest
that belong toForStInnerTable
with the given stateRequest.V
deserializeValue(byte[] valueBytes)
Deserialize the given bytes value to POJO value.org.forstdb.ColumnFamilyHandle
getColumnFamilyHandle()
Get the columnFamily handle corresponding to table.byte[]
serializeKey(ContextKey<K,N> contextKey)
Serialize the given key to bytes.byte[]
serializeValue(V value)
Serialize the given value to the outputView.-
Methods inherited from class org.apache.flink.runtime.state.v2.AbstractReducingState
add, asyncAdd, asyncGet, asyncGetInternal, asyncMergeNamespaces, asyncUpdateInternal, get, getInternal, mergeNamespaces, updateInternal
-
Methods inherited from class org.apache.flink.runtime.state.v2.AbstractKeyedState
asyncClear, clear, getStateDescriptor, getStateRequestHandler, getValueSerializer, handleRequest, handleRequestSync, setCurrentNamespace
-
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
-
Methods inherited from interface org.apache.flink.runtime.state.v2.internal.InternalPartitionedState
setCurrentNamespace
-
Methods inherited from interface org.apache.flink.api.common.state.v2.State
asyncClear, clear
-
-
-
-
Constructor Detail
-
ForStReducingState
public ForStReducingState(StateRequestHandler stateRequestHandler, org.forstdb.ColumnFamilyHandle columnFamily, ReducingStateDescriptor<V> reducingStateDescriptor, Supplier<SerializedCompositeKeyBuilder<K>> serializedKeyBuilderInitializer, N defaultNamespace, Supplier<TypeSerializer<N>> namespaceSerializerInitializer, Supplier<DataOutputSerializer> valueSerializerViewInitializer, Supplier<DataInputDeserializer> valueDeserializerViewInitializer)
-
-
Method Detail
-
getColumnFamilyHandle
public org.forstdb.ColumnFamilyHandle getColumnFamilyHandle()
Description copied from interface:ForStInnerTable
Get the columnFamily handle corresponding to table.- Specified by:
getColumnFamilyHandle
in interfaceForStInnerTable<K,N,V>
-
serializeKey
public byte[] serializeKey(ContextKey<K,N> contextKey) throws IOException
Description copied from interface:ForStInnerTable
Serialize the given key to bytes.- Specified by:
serializeKey
in interfaceForStInnerTable<K,N,V>
- Parameters:
contextKey
- the key to be serialized.- Returns:
- the key bytes
- Throws:
IOException
- Thrown if the serialization encountered an I/O related error.
-
serializeValue
public byte[] serializeValue(V value) throws IOException
Description copied from interface:ForStInnerTable
Serialize the given value to the outputView.- Specified by:
serializeValue
in interfaceForStInnerTable<K,N,V>
- Parameters:
value
- the value to be serialized.- Returns:
- the value bytes
- Throws:
IOException
- Thrown if the serialization encountered an I/O related error.
-
deserializeValue
public V deserializeValue(byte[] valueBytes) throws IOException
Description copied from interface:ForStInnerTable
Deserialize the given bytes value to POJO value.- Specified by:
deserializeValue
in interfaceForStInnerTable<K,N,V>
- Parameters:
valueBytes
- the value bytes to be deserialized.- Returns:
- the deserialized POJO value
- Throws:
IOException
- Thrown if the deserialization encountered an I/O related error.
-
buildDBGetRequest
public ForStDBGetRequest<K,N,V,V> buildDBGetRequest(StateRequest<?,?,?,?> stateRequest)
Description copied from interface:ForStInnerTable
Build aForStDBGetRequest
that belong to thisForStInnerTable
with the given stateRequest.- Specified by:
buildDBGetRequest
in interfaceForStInnerTable<K,N,V>
- Parameters:
stateRequest
- The given stateRequest.- Returns:
- The corresponding ForSt GetRequest.
-
buildDBPutRequest
public ForStDBPutRequest<K,N,V> buildDBPutRequest(StateRequest<?,?,?,?> stateRequest)
Description copied from interface:ForStInnerTable
Build aForStDBPutRequest
that belong toForStInnerTable
with the given stateRequest.- Specified by:
buildDBPutRequest
in interfaceForStInnerTable<K,N,V>
- Parameters:
stateRequest
- The given stateRequest.- Returns:
- The corresponding ForSt PutRequest.
-
-