[ https://issues.apache.org/jira/browse/FLINK-9637?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16571281#comment-16571281 ]
ASF GitHub Bot commented on FLINK-9637: --------------------------------------- twalthr commented on a change in pull request #6379: [FLINK-9637] Add public user documentation for state TTL feature URL: https://github.com/apache/flink/pull/6379#discussion_r208143373 ########## File path: docs/dev/stream/state/state.md ########## @@ -266,6 +266,101 @@ a `ValueState`. Once the count reaches 2 it will emit the average and clear the we start over from `0`. Note that this would keep a different state value for each different input key if we had tuples with different values in the first field. +### State time-to-live (TTL) + +A time-to-live (TTL) can be assigned to the keyed state value. +In this case it will expire after the configured TTL +and its stored value will be cleaned up on the best effort basis which is discussed in details later. + +The state collection types support per-entry TTLs: list elements and map entries expire independently. + +To use state TTL you must first build a `StateTtlConfiguration` object: + +<div class="codetabs" markdown="1"> +<div data-lang="java" markdown="1"> +{% highlight java %} +StateTtlConfiguration ttlConfig = StateTtlConfiguration + .newBuilder(Time.seconds(1)) + .setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite) + .setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired) + .build(); +{% endhighlight %} +</div> + +<div data-lang="scala" markdown="1"> +{% highlight scala %} +val ttlConfig = StateTtlConfiguration + .newBuilder(Time.seconds(1)) + .setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite) + .setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired) + .build() +{% endhighlight %} +</div> +</div> + +It has several options to consider. +The first parameter of `newBuilder` method is mandatory, it is the time-to-live value. + +The update type configures when the state TTL is refreshed (default `OnCreateAndWrite`): + + - `StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite` - only on creation and write access, + - `StateTtlConfiguration.TtlUpdateType.OnReadAndWrite` - also on read access. + +The state visibility configures whether the expired value is returned on read access +if it is not cleaned up yet (default `NeverReturnExpired`): + + - `StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired` - expired value is never returned, + - `StateTtlConfiguration.TtlStateVisibility.ReturnExpiredIfNotCleanedUp` - returned if still available. + + In case of `NeverReturnExpired`, the expired state behaves as if it does not exist any more, + even if it has yet to be removed. The option can be useful for the use cases + where data has to become unavailable for read access strictly after TTL, + e.g. application working with privacy sensitive data. + +Another option `ReturnExpiredIfNotCleanedUp` allows to return the expired state before its cleanup. + +TTL functionality can be enabled in the descriptor of any type of state: + +<div class="codetabs" markdown="1"> +<div data-lang="java" markdown="1"> +{% highlight java %} +StateTtlConfiguration ttlConfig = StateTtlConfiguration.newBuilder(Time.seconds(1)).build(); +ValueStateDescriptor<String> stateDescriptor = new ValueStateDescriptor<>("text state", String.class); +stateDescriptor.enableTimeToLive(ttlConfig); +{% endhighlight %} +</div> + +<div data-lang="scala" markdown="1"> +{% highlight scala %} +val ttlConfig = StateTtlConfiguration.newBuilder(Time.seconds(1)).build() +val stateDescriptor = new ValueStateDescriptor[String]("text state", classOf[String]) +stateDescriptor.enableTimeToLive(ttlConfig) +{% endhighlight %} +</div> +</div> + +**Notes:** + +- The state backends store the timestamp of last modification along with the user value, +which means that enabling this feature increases consumption of state storage. +Heap state backend stores an additional java object with a reference to the user state object +and a primitive long in memory. RocksDB state backend adds 8 bytes per stored value, list entry or map entry. Review comment: `The RocksDB state backend` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Add public user documentation for TTL feature > --------------------------------------------- > > Key: FLINK-9637 > URL: https://issues.apache.org/jira/browse/FLINK-9637 > Project: Flink > Issue Type: Sub-task > Components: State Backends, Checkpointing > Affects Versions: 1.6.0 > Reporter: Andrey Zagrebin > Assignee: Andrey Zagrebin > Priority: Major > Labels: pull-request-available > Fix For: 1.6.0 > > -- This message was sent by Atlassian JIRA (v7.6.3#76005)