[
https://issues.apache.org/jira/browse/FLINK-5163?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715162#comment-15715162
]
ASF GitHub Bot commented on FLINK-5163:
---------------------------------------
Github user aljoscha commented on a diff in the pull request:
https://github.com/apache/flink/pull/2871#discussion_r90639653
--- Diff:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/FromElementsFunction.java
---
@@ -62,6 +73,9 @@
/** Flag to make the source cancelable */
private volatile boolean isRunning = true;
+ private transient ListState<Integer> checkpointedState;
+
+ private static final String FROM_ELEMENT_STATE_NAME =
"from-element-state";
--- End diff --
Instead of this you could also have
```
private static final ListStateDescriptor<Integer> STATE_DESCRIPTOR =
new ListStateDescriptor<>("from-elements-state",
IntSerializer.INSTANCE);
```
and then use this descriptor later directly instead of initialising with
this field.
That's just a personal style nitpick. Your version is also fine. 😃
> Make the production functions rescalable (apart from the Rolling/Bucketing
> Sinks)
> ---------------------------------------------------------------------------------
>
> Key: FLINK-5163
> URL: https://issues.apache.org/jira/browse/FLINK-5163
> Project: Flink
> Issue Type: Improvement
> Affects Versions: 1.2.0
> Reporter: Kostas Kloudas
> Assignee: Kostas Kloudas
> Fix For: 1.2.0
>
>
> This issue targets porting all the functions in the production code to the
> new state abstractions. These functions are:
> 1) StatefulSequenceSource
> 2) MessageAcknowledgingSourceBase
> 3) FromElementsFunction
> 4) ContinuousFileMonitoringFunction
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)