@PublicEvolving public class SavepointReader extends Object
Modifier and Type | Method and Description |
---|---|
static SavepointReader |
read(StreamExecutionEnvironment env,
String path)
Loads an existing savepoint.
|
static SavepointReader |
read(StreamExecutionEnvironment env,
String path,
StateBackend stateBackend)
Loads an existing savepoint.
|
<K,V> DataStream<Tuple2<K,V>> |
readBroadcastState(OperatorIdentifier identifier,
String name,
TypeInformation<K> keyTypeInfo,
TypeInformation<V> valueTypeInfo)
Read operator
BroadcastState from a Savepoint . |
<K,V> DataStream<Tuple2<K,V>> |
readBroadcastState(OperatorIdentifier identifier,
String name,
TypeInformation<K> keyTypeInfo,
TypeInformation<V> valueTypeInfo,
TypeSerializer<K> keySerializer,
TypeSerializer<V> valueSerializer)
Read operator
BroadcastState from a Savepoint when a custom serializer was
used; e.g., a different serializer than the one returned by TypeInformation#createSerializer . |
<K,V> DataStream<Tuple2<K,V>> |
readBroadcastState(String uid,
String name,
TypeInformation<K> keyTypeInfo,
TypeInformation<V> valueTypeInfo)
|
<K,V> DataStream<Tuple2<K,V>> |
readBroadcastState(String uid,
String name,
TypeInformation<K> keyTypeInfo,
TypeInformation<V> valueTypeInfo,
TypeSerializer<K> keySerializer,
TypeSerializer<V> valueSerializer)
|
<K,OUT> DataStream<OUT> |
readKeyedState(OperatorIdentifier identifier,
KeyedStateReaderFunction<K,OUT> function)
Read keyed state from an operator in a
Savepoint . |
<K,OUT> DataStream<OUT> |
readKeyedState(OperatorIdentifier identifier,
KeyedStateReaderFunction<K,OUT> function,
TypeInformation<K> keyTypeInfo,
TypeInformation<OUT> outTypeInfo)
Read keyed state from an operator in a
Savepoint . |
<K,OUT> DataStream<OUT> |
readKeyedState(String uid,
KeyedStateReaderFunction<K,OUT> function)
Deprecated.
|
<K,OUT> DataStream<OUT> |
readKeyedState(String uid,
KeyedStateReaderFunction<K,OUT> function,
TypeInformation<K> keyTypeInfo,
TypeInformation<OUT> outTypeInfo)
|
<T> DataStream<T> |
readListState(OperatorIdentifier identifier,
String name,
TypeInformation<T> typeInfo)
Read operator
ListState from a Savepoint . |
<T> DataStream<T> |
readListState(OperatorIdentifier identifier,
String name,
TypeInformation<T> typeInfo,
TypeSerializer<T> serializer)
Read operator
ListState from a Savepoint when a custom serializer was used;
e.g., a different serializer than the one returned by TypeInformation#createSerializer . |
<T> DataStream<T> |
readListState(String uid,
String name,
TypeInformation<T> typeInfo)
Deprecated.
|
<T> DataStream<T> |
readListState(String uid,
String name,
TypeInformation<T> typeInfo,
TypeSerializer<T> serializer)
|
<T> DataStream<T> |
readUnionState(OperatorIdentifier identifier,
String name,
TypeInformation<T> typeInfo)
Read operator
UnionState from a Savepoint . |
<T> DataStream<T> |
readUnionState(OperatorIdentifier identifier,
String name,
TypeInformation<T> typeInfo,
TypeSerializer<T> serializer)
Read operator
UnionState from a Savepoint when a custom serializer was used;
e.g., a different serializer than the one returned by TypeInformation#createSerializer . |
<T> DataStream<T> |
readUnionState(String uid,
String name,
TypeInformation<T> typeInfo)
Deprecated.
|
<T> DataStream<T> |
readUnionState(String uid,
String name,
TypeInformation<T> typeInfo,
TypeSerializer<T> serializer)
|
<W extends Window> |
window(TypeSerializer<W> windowSerializer)
Read window state from an operator in a
Savepoint . |
<W extends Window> |
window(WindowAssigner<?,W> assigner)
Read window state from an operator in a
Savepoint . |
public static SavepointReader read(StreamExecutionEnvironment env, String path) throws IOException
env
- The execution environment used to transform the savepoint.path
- The path to an existing savepoint on disk.SavepointReader
.IOException
public static SavepointReader read(StreamExecutionEnvironment env, String path, StateBackend stateBackend) throws IOException
env
- The execution environment used to transform the savepoint.path
- The path to an existing savepoint on disk.stateBackend
- The state backend of the savepoint.SavepointReader
.IOException
@Deprecated public <T> DataStream<T> readListState(String uid, String name, TypeInformation<T> typeInfo) throws IOException
readListState(OperatorIdentifier, String, TypeInformation)
IOException
public <T> DataStream<T> readListState(OperatorIdentifier identifier, String name, TypeInformation<T> typeInfo) throws IOException
ListState
from a Savepoint
.T
- The type of the values that are in the list state.identifier
- The identifier of the operator.name
- The (unique) name for the state.typeInfo
- The type of the elements in the state.DataStream
representing the elements in state.IOException
- If the savepoint path is invalid or the uid does not exist.@Deprecated public <T> DataStream<T> readListState(String uid, String name, TypeInformation<T> typeInfo, TypeSerializer<T> serializer) throws IOException
IOException
public <T> DataStream<T> readListState(OperatorIdentifier identifier, String name, TypeInformation<T> typeInfo, TypeSerializer<T> serializer) throws IOException
ListState
from a Savepoint
when a custom serializer was used;
e.g., a different serializer than the one returned by TypeInformation#createSerializer
.T
- The type of the values that are in the list state.identifier
- The identifier of the operator.name
- The (unique) name for the state.typeInfo
- The type of the elements in the state.serializer
- The serializer used to write the elements into state.DataStream
representing the elements in state.IOException
- If the savepoint path is invalid or the uid does not exist.@Deprecated public <T> DataStream<T> readUnionState(String uid, String name, TypeInformation<T> typeInfo) throws IOException
readUnionState(OperatorIdentifier, String, TypeInformation)
IOException
public <T> DataStream<T> readUnionState(OperatorIdentifier identifier, String name, TypeInformation<T> typeInfo) throws IOException
UnionState
from a Savepoint
.T
- The type of the values that are in the union state.identifier
- The identifier of the operator.name
- The (unique) name for the state.typeInfo
- The type of the elements in the state.DataStream
representing the elements in state.IOException
- If the savepoint path is invalid or the uid does not exist.public <T> DataStream<T> readUnionState(String uid, String name, TypeInformation<T> typeInfo, TypeSerializer<T> serializer) throws IOException
IOException
public <T> DataStream<T> readUnionState(OperatorIdentifier identifier, String name, TypeInformation<T> typeInfo, TypeSerializer<T> serializer) throws IOException
UnionState
from a Savepoint
when a custom serializer was used;
e.g., a different serializer than the one returned by TypeInformation#createSerializer
.T
- The type of the values that are in the union state.identifier
- The identifier of the operator.name
- The (unique) name for the state.typeInfo
- The type of the elements in the state.serializer
- The serializer used to write the elements into state.DataStream
representing the elements in state.IOException
- If the savepoint path is invalid or the uid does not exist.@Deprecated public <K,V> DataStream<Tuple2<K,V>> readBroadcastState(String uid, String name, TypeInformation<K> keyTypeInfo, TypeInformation<V> valueTypeInfo) throws IOException
IOException
public <K,V> DataStream<Tuple2<K,V>> readBroadcastState(OperatorIdentifier identifier, String name, TypeInformation<K> keyTypeInfo, TypeInformation<V> valueTypeInfo) throws IOException
BroadcastState
from a Savepoint
.K
- The type of keys in state.V
- The type of values in state.identifier
- The identifier of the operator.name
- The (unique) name for the state.keyTypeInfo
- The type information for the keys in the state.valueTypeInfo
- The type information for the values in the state.DataStream
of key-value pairs from state.IOException
- If the savepoint does not contain the specified uid.@Deprecated public <K,V> DataStream<Tuple2<K,V>> readBroadcastState(String uid, String name, TypeInformation<K> keyTypeInfo, TypeInformation<V> valueTypeInfo, TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer) throws IOException
readBroadcastState(OperatorIdentifier, String, TypeInformation,
TypeInformation, TypeSerializer, TypeSerializer)
IOException
public <K,V> DataStream<Tuple2<K,V>> readBroadcastState(OperatorIdentifier identifier, String name, TypeInformation<K> keyTypeInfo, TypeInformation<V> valueTypeInfo, TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer) throws IOException
BroadcastState
from a Savepoint
when a custom serializer was
used; e.g., a different serializer than the one returned by TypeInformation#createSerializer
.K
- The type of keys in state.V
- The type of values in state.identifier
- The identifier of the operator.name
- The (unique) name for the state.keyTypeInfo
- The type information for the keys in the state.valueTypeInfo
- The type information for the values in the state.keySerializer
- The type serializer used to write keys into the state.valueSerializer
- The type serializer used to write values into the state.DataStream
of key-value pairs from state.IOException
- If the savepoint path is invalid or the uid does not exist.@Deprecated public <K,OUT> DataStream<OUT> readKeyedState(String uid, KeyedStateReaderFunction<K,OUT> function) throws IOException
readKeyedState(OperatorIdentifier, KeyedStateReaderFunction)
IOException
public <K,OUT> DataStream<OUT> readKeyedState(OperatorIdentifier identifier, KeyedStateReaderFunction<K,OUT> function) throws IOException
Savepoint
.K
- The type of the key in state.OUT
- The output type of the transform function.identifier
- The identifier of the operator.function
- The KeyedStateReaderFunction
that is called for each key in state.DataStream
of objects read from keyed state.IOException
- If the savepoint does not contain operator state with the given uid.@Deprecated public <K,OUT> DataStream<OUT> readKeyedState(String uid, KeyedStateReaderFunction<K,OUT> function, TypeInformation<K> keyTypeInfo, TypeInformation<OUT> outTypeInfo) throws IOException
readKeyedState(OperatorIdentifier, KeyedStateReaderFunction,
TypeInformation, TypeInformation)
IOException
public <K,OUT> DataStream<OUT> readKeyedState(OperatorIdentifier identifier, KeyedStateReaderFunction<K,OUT> function, TypeInformation<K> keyTypeInfo, TypeInformation<OUT> outTypeInfo) throws IOException
Savepoint
.K
- The type of the key in state.OUT
- The output type of the transform function.identifier
- The identifier of the operator.function
- The KeyedStateReaderFunction
that is called for each key in state.keyTypeInfo
- The type information of the key in state.outTypeInfo
- The type information of the output of the transform reader function.DataStream
of objects read from keyed state.IOException
- If the savepoint does not contain operator state with the given uid.public <W extends Window> WindowSavepointReader<W> window(WindowAssigner<?,W> assigner)
Savepoint
. This method supports reading from
any type of window.assigner
- The WindowAssigner
used to write out the operator.WindowSavepointReader
.public <W extends Window> WindowSavepointReader<W> window(TypeSerializer<W> windowSerializer)
Savepoint
. This method supports reading from
any type of window.windowSerializer
- The serializer used for the window type.WindowSavepointReader
.Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.