forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-35356][State] State descriptor and implementation for async re…
…ducing state
- Loading branch information
Showing
13 changed files
with
351 additions
and
15 deletions.
There are no files selected for viewing
38 changes: 38 additions & 0 deletions
38
flink-core-api/src/main/java/org/apache/flink/api/common/state/v2/ReducingState.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 | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,38 @@ | ||
/* | ||
* 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.v2; | ||
|
||
import org.apache.flink.annotation.Experimental; | ||
|
||
/** | ||
* {@link State} interface for reducing state. Elements can be added to the state, they will be | ||
* combined using a reduce function. The current state can be inspected. | ||
* | ||
* <p>The state is accessed and modified by user functions, and checkpointed consistently by the | ||
* system as part of the distributed snapshots. | ||
* | ||
* <p>The state is only accessible by functions applied on a {@code KeyedStream}. The key is | ||
* automatically supplied by the system, so the function always sees the value mapped to the key of | ||
* the current element. That way, the system can handle stream and state partitioning consistently | ||
* together. | ||
* | ||
* @param <T> Type of the value in the operator state | ||
*/ | ||
@Experimental | ||
public interface ReducingState<T> extends MergingState<T, T> {} |
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
53 changes: 53 additions & 0 deletions
53
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/InternalReducingState.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 | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,53 @@ | ||
/* | ||
* 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.v2; | ||
|
||
import org.apache.flink.api.common.functions.ReduceFunction; | ||
import org.apache.flink.api.common.state.v2.ReducingState; | ||
import org.apache.flink.api.common.state.v2.StateFuture; | ||
import org.apache.flink.runtime.asyncprocessing.StateRequestHandler; | ||
import org.apache.flink.runtime.asyncprocessing.StateRequestType; | ||
|
||
/** | ||
* A default implementation of {@link ReducingState} which delegates all async requests to {@link | ||
* StateRequestHandler}. | ||
* | ||
* @param <K> The type of key the state is associated to. | ||
* @param <V> The type of values kept internally in state. | ||
*/ | ||
public class InternalReducingState<K, V> extends InternalKeyedState<K, V> | ||
implements ReducingState<V> { | ||
|
||
protected final ReduceFunction<V> reduceFunction; | ||
|
||
public InternalReducingState( | ||
StateRequestHandler stateRequestHandler, ReducingStateDescriptor<V> stateDescriptor) { | ||
super(stateRequestHandler, stateDescriptor); | ||
this.reduceFunction = stateDescriptor.getReduceFunction(); | ||
} | ||
|
||
@Override | ||
public StateFuture<V> asyncGet() { | ||
return handleRequest(StateRequestType.REDUCING_GET, null); | ||
} | ||
|
||
@Override | ||
public StateFuture<Void> asyncAdd(V value) { | ||
return handleRequest(StateRequestType.REDUCING_ADD, value); | ||
} | ||
} |
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
74 changes: 74 additions & 0 deletions
74
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/ReducingStateDescriptor.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 | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,74 @@ | ||
/* | ||
* 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.v2; | ||
|
||
import org.apache.flink.api.common.functions.ReduceFunction; | ||
import org.apache.flink.api.common.serialization.SerializerConfig; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
|
||
import static org.apache.flink.util.Preconditions.checkNotNull; | ||
|
||
/** | ||
* {@link StateDescriptor} for {@link org.apache.flink.api.common.state.v2.ReducingState}. | ||
* | ||
* @param <T> The type of the values that can be added to the state. | ||
*/ | ||
public class ReducingStateDescriptor<T> extends StateDescriptor<T> { | ||
|
||
private final ReduceFunction<T> reduceFunction; | ||
|
||
/** | ||
* Creates a new {@code ReducingStateDescriptor} with the given name and default value. | ||
* | ||
* @param name The (unique) name for the state. | ||
* @param reduceFunction The {@code ReduceFunction} used to aggregate the state. | ||
* @param typeInfo The type of the values in the state. | ||
*/ | ||
public ReducingStateDescriptor( | ||
String name, ReduceFunction<T> reduceFunction, TypeInformation<T> typeInfo) { | ||
super(name, typeInfo, null); | ||
this.reduceFunction = checkNotNull(reduceFunction); | ||
} | ||
|
||
/** | ||
* Creates a new {@code ReducingStateDescriptor} with the given name and default value. | ||
* | ||
* @param name The (unique) name for the state. | ||
* @param reduceFunction The {@code ReduceFunction} used to aggregate the state. | ||
* @param typeInfo The type of the values in the state. | ||
*/ | ||
public ReducingStateDescriptor( | ||
String name, | ||
ReduceFunction<T> reduceFunction, | ||
TypeInformation<T> typeInfo, | ||
SerializerConfig serializerConfig) { | ||
super(name, typeInfo, serializerConfig); | ||
this.reduceFunction = checkNotNull(reduceFunction); | ||
} | ||
|
||
/** Returns the reduce function to be used for the reducing state. */ | ||
public ReduceFunction<T> getReduceFunction() { | ||
return reduceFunction; | ||
} | ||
|
||
@Override | ||
public Type getType() { | ||
return Type.REDUCING; | ||
} | ||
} |
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 | Diff line number | Diff line change |
---|---|---|
|
@@ -47,7 +47,6 @@ public enum Type { | |
VALUE, | ||
LIST, | ||
REDUCING, | ||
FOLDING, | ||
AGGREGATING, | ||
MAP | ||
} | ||
|
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
49 changes: 49 additions & 0 deletions
49
flink-runtime/src/test/java/org/apache/flink/runtime/state/v2/InternalReducingStateTest.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 | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,49 @@ | ||
/* | ||
* 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.v2; | ||
|
||
import org.apache.flink.api.common.functions.ReduceFunction; | ||
import org.apache.flink.api.common.typeinfo.BasicTypeInfo; | ||
import org.apache.flink.runtime.asyncprocessing.StateRequestType; | ||
|
||
import org.junit.jupiter.api.Test; | ||
|
||
/** Tests for {@link InternalReducingState}. */ | ||
public class InternalReducingStateTest extends InternalKeyedStateTestBase { | ||
|
||
@Test | ||
@SuppressWarnings({"unchecked"}) | ||
public void testEachOperation() { | ||
ReduceFunction<Integer> reducer = Integer::sum; | ||
ReducingStateDescriptor<Integer> descriptor = | ||
new ReducingStateDescriptor<>("testState", reducer, BasicTypeInfo.INT_TYPE_INFO); | ||
InternalReducingState<String, Integer> reducingState = | ||
new InternalReducingState<>(aec, descriptor); | ||
aec.setCurrentContext(aec.buildContext("test", "test")); | ||
|
||
reducingState.asyncClear(); | ||
validateRequestRun(reducingState, StateRequestType.CLEAR, null); | ||
|
||
reducingState.asyncGet(); | ||
validateRequestRun(reducingState, StateRequestType.REDUCING_GET, null); | ||
|
||
reducingState.asyncAdd(1); | ||
validateRequestRun(reducingState, StateRequestType.REDUCING_ADD, 1); | ||
} | ||
} |
Oops, something went wrong.