Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[streaming] Initial rework of the operator state interfaces
- Loading branch information
Showing
38 changed files
with
1,244 additions
and
269 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
67 changes: 67 additions & 0 deletions
67
flink-core/src/main/java/org/apache/flink/api/common/state/OperatorState.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,67 @@ | |||
/* | |||
* Licensed to the Apache Software Foundation (ASF) under one | |||
* or more contributor license agreements. See the NOTICE file | |||
* distributed with this work for additional information | |||
* regarding copyright ownership. The ASF licenses this file | |||
* to you under the Apache License, Version 2.0 (the | |||
* "License"); you may not use this file except in compliance | |||
* with the License. You may obtain a copy of the License at | |||
* | |||
* http://www.apache.org/licenses/LICENSE-2.0 | |||
* | |||
* Unless required by applicable law or agreed to in writing, software | |||
* distributed under the License is distributed on an "AS IS" BASIS, | |||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | |||
* See the License for the specific language governing permissions and | |||
* limitations under the License. | |||
*/ | |||
|
|||
package org.apache.flink.api.common.state; | |||
|
|||
import org.apache.flink.api.common.functions.MapFunction; | |||
|
|||
/** | |||
* Base class for all streaming operator states. It can represent both | |||
* partitioned (when state partitioning is defined in the program) or | |||
* non-partitioned user states. | |||
* | |||
* State can be accessed and manipulated using the {@link #getState()} and | |||
* {@link #updateState(T)} methods. These calls are only valid in the | |||
* transformation call the operator represents, for instance inside | |||
* {@link MapFunction#map()} and invalid in | |||
* {@link #open(org.apache.flink.configuration.Configuration)} or | |||
* {@link #close()}. | |||
* | |||
* @param <T> | |||
* Type of the operator state | |||
*/ | |||
public interface OperatorState<T> { | |||
|
|||
/** | |||
* Gets the current state for the operator. When the state is not | |||
* partitioned the returned state is the same for all inputs. If state | |||
* partitioning is applied the state returned depends on the current | |||
* operator input, as the operator maintains an independent state for each | |||
* partitions. | |||
* | |||
* <p> | |||
* {@link #getState()} returns <code>null</code> if there is no state stored | |||
* in the operator. This is the expected behaviour before initializing the | |||
* state with {@link #updateState(T)}. | |||
* </p> | |||
* | |||
* @return The operator state corresponding to the current input. | |||
*/ | |||
T getState(); | |||
|
|||
/** | |||
* Updates the operator state accessible by {@link #getState()} to the given | |||
* value. The next time {@link #getState()} is called (for the same state | |||
* partition) the returned state will represent the updated value. | |||
* | |||
* @param state | |||
* The updated state. | |||
*/ | |||
void updateState(T state); | |||
|
|||
} |
73 changes: 73 additions & 0 deletions
73
flink-core/src/main/java/org/apache/flink/api/common/state/StateCheckpointer.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,73 @@ | |||
/* | |||
* Licensed to the Apache Software Foundation (ASF) under one | |||
* or more contributor license agreements. See the NOTICE file | |||
* distributed with this work for additional information | |||
* regarding copyright ownership. The ASF licenses this file | |||
* to you under the Apache License, Version 2.0 (the | |||
* "License"); you may not use this file except in compliance | |||
* with the License. You may obtain a copy of the License at | |||
* | |||
* http://www.apache.org/licenses/LICENSE-2.0 | |||
* | |||
* Unless required by applicable law or agreed to in writing, software | |||
* distributed under the License is distributed on an "AS IS" BASIS, | |||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | |||
* See the License for the specific language governing permissions and | |||
* limitations under the License. | |||
*/ | |||
|
|||
package org.apache.flink.api.common.state; | |||
|
|||
import java.io.Serializable; | |||
|
|||
/** | |||
* Basic interface for creating {@link OperatorState} snapshots in stateful | |||
* streaming programs. | |||
* | |||
* The user needs to implement the {@link #snapshotState(S, long, long)} and | |||
* {@link #restoreState(C)} methods that will be called to create and restore | |||
* state snapshots of the given states. | |||
* | |||
* <p> | |||
* Note that the {@link OperatorState} is <i>synchronously</i> checkpointed. | |||
* While the state is written, the state cannot be accessed or modified so the | |||
* function needs not return a copy of its state, but may return a reference to | |||
* its state. | |||
* </p> | |||
* | |||
* @param <S> | |||
* Type of the operator state. | |||
* @param <C> | |||
* Type of the snapshot that will be persisted. | |||
*/ | |||
public interface StateCheckpointer<S, C extends Serializable> { | |||
|
|||
/** | |||
* Takes a snapshot of a given operator state. The snapshot returned will be | |||
* persisted in the state backend for this job and restored upon failure. | |||
* This method is called for all state partitions in case of partitioned | |||
* state when creating a checkpoint. | |||
* | |||
* @param state | |||
* The state for which the snapshot needs to be taken | |||
* @param checkpointId | |||
* The ID of the checkpoint. | |||
* @param checkpointTimestamp | |||
* The timestamp of the checkpoint, as derived by | |||
* System.currentTimeMillis() on the JobManager. | |||
* | |||
* @return A snapshot of the operator state. | |||
*/ | |||
public C snapshotState(S state, long checkpointId, long checkpointTimestamp); | |||
|
|||
/** | |||
* Restores the operator states from a given snapshot. The restores state | |||
* will be loaded back to the function. In case of partitioned state, each | |||
* partition is restored independently. | |||
* | |||
* @param stateSnapshot | |||
* The state snapshot that needs to be restored. | |||
* @return The state corresponding to the snapshot. | |||
*/ | |||
public S restoreState(C stateSnapshot); | |||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
47 changes: 47 additions & 0 deletions
47
flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionedStateHandle.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,47 @@ | |||
/* | |||
* Licensed to the Apache Software Foundation (ASF) under one | |||
* or more contributor license agreements. See the NOTICE file | |||
* distributed with this work for additional information | |||
* regarding copyright ownership. The ASF licenses this file | |||
* to you under the Apache License, Version 2.0 (the | |||
* "License"); you may not use this file except in compliance | |||
* with the License. You may obtain a copy of the License at | |||
* | |||
* http://www.apache.org/licenses/LICENSE-2.0 | |||
* | |||
* Unless required by applicable law or agreed to in writing, software | |||
* distributed under the License is distributed on an "AS IS" BASIS, | |||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | |||
* See the License for the specific language governing permissions and | |||
* limitations under the License. | |||
*/ | |||
|
|||
package org.apache.flink.runtime.state; | |||
|
|||
import java.io.Serializable; | |||
import java.util.Map; | |||
|
|||
public class PartitionedStateHandle implements | |||
StateHandle<Map<Serializable, StateHandle<Serializable>>> { | |||
|
|||
private static final long serialVersionUID = 7505365403501402100L; | |||
|
|||
Map<Serializable, StateHandle<Serializable>> handles; | |||
|
|||
public PartitionedStateHandle(Map<Serializable, StateHandle<Serializable>> handles) { | |||
this.handles = handles; | |||
} | |||
|
|||
@Override | |||
public Map<Serializable, StateHandle<Serializable>> getState() throws Exception { | |||
return handles; | |||
} | |||
|
|||
@Override | |||
public void discardState() throws Exception { | |||
for (StateHandle<Serializable> handle : handles.values()) { | |||
handle.discardState(); | |||
} | |||
} | |||
|
|||
} |
47 changes: 47 additions & 0 deletions
47
flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionedStateStore.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,47 @@ | |||
/* | |||
* Licensed to the Apache Software Foundation (ASF) under one | |||
* or more contributor license agreements. See the NOTICE file | |||
* distributed with this work for additional information | |||
* regarding copyright ownership. The ASF licenses this file | |||
* to you under the Apache License, Version 2.0 (the | |||
* "License"); you may not use this file except in compliance | |||
* with the License. You may obtain a copy of the License at | |||
* | |||
* http://www.apache.org/licenses/LICENSE-2.0 | |||
* | |||
* Unless required by applicable law or agreed to in writing, software | |||
* distributed under the License is distributed on an "AS IS" BASIS, | |||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | |||
* See the License for the specific language governing permissions and | |||
* limitations under the License. | |||
*/ | |||
|
|||
package org.apache.flink.runtime.state; | |||
|
|||
import java.io.Serializable; | |||
import java.util.Map; | |||
|
|||
/** | |||
* Interface for storing and accessing partitioned state. The interface is | |||
* designed in a way that allows implementations for lazily state access. | |||
* | |||
* @param <S> | |||
* Type of the state. | |||
* @param <C> | |||
* Type of the state snapshot. | |||
*/ | |||
public interface PartitionedStateStore<S, C extends Serializable> { | |||
|
|||
S getStateForKey(Serializable key) throws Exception; | |||
|
|||
void setStateForKey(Serializable key, S state); | |||
|
|||
Map<Serializable, S> getPartitionedState() throws Exception; | |||
|
|||
Map<Serializable, StateHandle<C>> snapshotStates(long checkpointId, long checkpointTimestamp) throws Exception; | |||
|
|||
void restoreStates(Map<Serializable, StateHandle<C>> snapshots) throws Exception; | |||
|
|||
boolean containsKey(Serializable key); | |||
|
|||
} |
Oops, something went wrong.