Package org.apache.flink.state.forst
Class ForStAggregatingState<K,N,IN,ACC,OUT>
- java.lang.Object
-
- org.apache.flink.runtime.state.v2.AbstractKeyedState<K,N,ACC>
-
- org.apache.flink.runtime.state.v2.AbstractAggregatingState<K,N,IN,ACC,OUT>
-
- org.apache.flink.state.forst.ForStAggregatingState<K,N,IN,ACC,OUT>
-
- Type Parameters:
K
- type of keyIN
- type of inputACC
- type of aggregate stateOUT
- type of output
- All Implemented Interfaces:
AggregatingState<IN,OUT>
,AppendingState<IN,OUT,OUT>
,MergingState<IN,OUT,OUT>
,State
,InternalAggregatingState<K,N,IN,ACC,OUT>
,InternalAppendingState<K,N,IN,ACC,OUT,OUT>
,InternalKeyedState<K,N,ACC>
,InternalMergingState<K,N,IN,ACC,OUT,OUT>
,InternalPartitionedState<N>
,ForStInnerTable<K,N,ACC>
public class ForStAggregatingState<K,N,IN,ACC,OUT> extends AbstractAggregatingState<K,N,IN,ACC,OUT> implements ForStInnerTable<K,N,ACC>
The implementation ofAggregatingState
for ForStDB.
-
-
Field Summary
-
Fields inherited from class org.apache.flink.runtime.state.v2.AbstractAggregatingState
aggregateFunction
-
Fields inherited from class org.apache.flink.runtime.state.v2.AbstractKeyedState
stateRequestHandler
-
-
Constructor Summary
Constructors Constructor Description ForStAggregatingState(AggregatingStateDescriptor<IN,ACC,OUT> stateDescriptor, StateRequestHandler stateRequestHandler, org.forstdb.ColumnFamilyHandle columnFamily, 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,ACC,?>
buildDBGetRequest(StateRequest<?,?,?,?> stateRequest)
Build aForStDBGetRequest
that belong to thisForStInnerTable
with the given stateRequest.ForStDBPutRequest<?,?,?>
buildDBPutRequest(StateRequest<?,?,?,?> stateRequest)
Build aForStDBPutRequest
that belong toForStInnerTable
with the given stateRequest.ACC
deserializeValue(byte[] value)
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(ACC value)
Serialize the given value to the outputView.-
Methods inherited from class org.apache.flink.runtime.state.v2.AbstractAggregatingState
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
-
ForStAggregatingState
public ForStAggregatingState(AggregatingStateDescriptor<IN,ACC,OUT> stateDescriptor, StateRequestHandler stateRequestHandler, org.forstdb.ColumnFamilyHandle columnFamily, 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,IN>
-
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,IN>
- 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(ACC value) throws IOException
Description copied from interface:ForStInnerTable
Serialize the given value to the outputView.- Specified by:
serializeValue
in interfaceForStInnerTable<K,N,IN>
- 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 ACC deserializeValue(byte[] value) throws IOException
Description copied from interface:ForStInnerTable
Deserialize the given bytes value to POJO value.- Specified by:
deserializeValue
in interfaceForStInnerTable<K,N,IN>
- Parameters:
value
- 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,ACC,?> 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,IN>
- Parameters:
stateRequest
- The given stateRequest.- Returns:
- The corresponding ForSt GetRequest.
-
buildDBPutRequest
public ForStDBPutRequest<?,?,?> buildDBPutRequest(StateRequest<?,?,?,?> stateRequest)
Description copied from interface:ForStInnerTable
Build aForStDBPutRequest
that belong toForStInnerTable
with the given stateRequest.- Specified by:
buildDBPutRequest
in interfaceForStInnerTable<K,N,IN>
- Parameters:
stateRequest
- The given stateRequest.- Returns:
- The corresponding ForSt PutRequest.
-
-