alpinegizmo commented on a change in pull request #13384:
URL: https://github.com/apache/flink/pull/13384#discussion_r489314761
##########
File path: docs/dev/libs/state_processor_api.md
##########
@@ -345,6 +345,44 @@ Along with reading registered state values, each key has
access to a `Context` w
{% panel **Note:** When using a `KeyedStateReaderFunction`, all state
descriptors must be registered eagerly inside of open. Any attempt to call a
`RuntimeContext#get*State` will result in a `RuntimeException`. %}
+### Window State
+
+The state processor api supports reading state from a [window operator]({{
site.baseurl }}/dev/stream/operators/windows.html).
+This includes both time based windows along with other types, pre-aggregation,
non-aggregated, and windows with evictors.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+
+ExecutionEnvironment batchEnv = ExecutionEnvironment.getExecutionEnvironment();
+ExistingSavepoint savepoint = Savepoint.load(batchEnv,
"hdfs://checkpoint-dir", new MemoryStateBackend());
+
+int count = savepoint
+ // The timeWindow method supports reading from any type of time based
window, including but not limited to
+ // Tumbling, Sliding, and Session windows for both event time and
processing time.
+ .timeWindow()
Review comment:
When you say "not limited to", what are you thinking of?
Am I right in thinking that non-keyed time windows are not handled -- i.e.,
timeWindowAll? What about global windows?
##########
File path: docs/dev/libs/state_processor_api.md
##########
@@ -345,6 +345,44 @@ Along with reading registered state values, each key has
access to a `Context` w
{% panel **Note:** When using a `KeyedStateReaderFunction`, all state
descriptors must be registered eagerly inside of open. Any attempt to call a
`RuntimeContext#get*State` will result in a `RuntimeException`. %}
+### Window State
+
+The state processor api supports reading state from a [window operator]({{
site.baseurl }}/dev/stream/operators/windows.html).
+This includes both time based windows along with other types, pre-aggregation,
non-aggregated, and windows with evictors.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+
+ExecutionEnvironment batchEnv = ExecutionEnvironment.getExecutionEnvironment();
+ExistingSavepoint savepoint = Savepoint.load(batchEnv,
"hdfs://checkpoint-dir", new MemoryStateBackend());
+
+int count = savepoint
+ // The timeWindow method supports reading from any type of time based
window, including but not limited to
+ // Tumbling, Sliding, and Session windows for both event time and
processing time.
+ .timeWindow()
Review comment:
Also, I'll just note that it has been proposed to deprecate timeWindow
from the datastream API. That doesn't have to affect this, of course.
##########
File path: docs/dev/libs/state_processor_api.md
##########
@@ -345,6 +345,44 @@ Along with reading registered state values, each key has
access to a `Context` w
{% panel **Note:** When using a `KeyedStateReaderFunction`, all state
descriptors must be registered eagerly inside of open. Any attempt to call a
`RuntimeContext#get*State` will result in a `RuntimeException`. %}
+### Window State
+
+The state processor api supports reading state from a [window operator]({{
site.baseurl }}/dev/stream/operators/windows.html).
+This includes both time based windows along with other types, pre-aggregation,
non-aggregated, and windows with evictors.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+
+ExecutionEnvironment batchEnv = ExecutionEnvironment.getExecutionEnvironment();
+ExistingSavepoint savepoint = Savepoint.load(batchEnv,
"hdfs://checkpoint-dir", new MemoryStateBackend());
+
+int count = savepoint
+ // The timeWindow method supports reading from any type of time based
window, including but not limited to
+ // Tumbling, Sliding, and Session windows for both event time and
processing time.
+ .timeWindow()
+ .reduce(uid, new ReduceSum(), Types.INT, Types.INT)
+ .count();
Review comment:
I don't understand this example. Where does the count() method come
from, and what is being counted?
If I wanted to inspect or modify the window contents, how would I do that?
Can I write window state, or only read it?
##########
File path:
flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWindowReaderITCase.java
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.state.api;
+
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.state.api.functions.WindowReaderFunction;
+import org.apache.flink.state.api.utils.AggregateSum;
+import org.apache.flink.state.api.utils.ReduceSum;
+import org.apache.flink.state.api.utils.SavepointTestBase;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import
org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.windowing.evictors.Evictor;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.operators.windowing.TimestampedValue;
+import org.apache.flink.util.Collector;
+
+import org.hamcrest.Matchers;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * IT Case for reading window operator state.
+ */
+public abstract class SavepointWindowReaderITCase<B extends StateBackend>
extends SavepointTestBase {
+ private static final String uid = "stateful-operator";
+
+ private static final Integer[] numbers = { 1, 2, 3 };
+
+ protected abstract B getStateBackend();
+
+ @Test
+ public void testReduceWindowStateReader() throws Exception {
+ String savepointPath = takeSavepoint(numbers, source -> {
+ StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+
env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+ env.setStateBackend(getStateBackend());
+ env.setParallelism(4);
+
+ env
+ .addSource(source)
+ .rebalance()
+ .assignTimestampsAndWatermarks(WatermarkStrategy
+ .<Integer>noWatermarks()
+ .withTimestampAssigner((event,
timestamp) -> 0))
+ .keyBy(id -> id)
+ .timeWindow(Time.milliseconds(10))
+ .reduce(new ReduceSum())
+ .uid(uid)
+ .addSink(new DiscardingSink<>());
+
+ return env;
+ });
+
+ ExecutionEnvironment batchEnv =
ExecutionEnvironment.getExecutionEnvironment();
+ ExistingSavepoint savepoint = Savepoint.load(batchEnv,
savepointPath, getStateBackend());
+
+ List<Integer> results = savepoint
+ .timeWindow()
+ .reduce(uid, new ReduceSum(), Types.INT, Types.INT)
+ .collect();
+
+ Assert.assertThat("Unexpected results from keyed state",
results, Matchers.containsInAnyOrder(numbers));
Review comment:
Been trying to use this test as an example, to see if I can understand
what's going on. Why would the savepoint contain all of the numbers? I would
expect that at whatever point the savepoint is taken, each window pane would
only contain the result of reducing whichever numbers have been processed so
far. Are you assuming that the 3 numbers/keys are each distributed to a
different parallel instance, and that's why all of them appear in the state?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]