[GitHub] eliaslevy commented on a change in pull request #6379: [FLINK-9637] Add public user documentation for state TTL feature

2018-07-29 Thread GitBox
eliaslevy 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_r205991168
 
 

 ##
 File path: docs/dev/stream/state/state.md
 ##
 @@ -266,6 +266,92 @@ 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 based on the best effort.
+Depending on configuration, the expired state can become unavailable for read 
access
+even if it is not cleaned up yet. In this case it behaves as if it does not 
exist any more.
+
+The collection types of state support TTL on entry level: 
+separate list elements and map entries expire independently. 
+
+The behaviour of state with TTL firstly should be configured by building 
`StateTtlConfiguration`:
+
+
+
+{% highlight java %}
+StateTtlConfiguration ttlConfig = StateTtlConfiguration
+.newBuilder(Time.seconds(1))
+.setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite)
+
.setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired)
+.build();
+{% endhighlight %}
+
+
+
+{% highlight scala %}
+val ttlConfig = StateTtlConfiguration
+.newBuilder(Time.seconds(1))
+.setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite)
+
.setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired)
+.build()
+{% endhighlight %}
+
+
+
+It has several options to consider. 
+The first parameter of `newBuilder` method is mandatory, it is a value of 
time-to-live itself.
+
+The update type configures when the time-to-live of state value is prolonged 
(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.
+
+The TTL can be enabled in descriptor for any type of state:
+
+
+
+{% highlight java %}
+StateTtlConfiguration ttlConfig = 
StateTtlConfiguration.newBuilder(Time.seconds(1)).build();
+ValueStateDescriptor stateDescriptor = new 
ValueStateDescriptor<>("text state", String.class);
+stateDescriptor.enableTimeToLive(ttlConfig);
+{% endhighlight %}
+
+
+
+{% highlight scala %}
+val ttlConfig = StateTtlConfiguration.newBuilder(Time.seconds(1)).build()
+val stateDescriptor = new ValueStateDescriptor[String]("text state", 
classOf[String])
+stateDescriptor.enableTimeToLive(ttlConfig)
+{% endhighlight %}
+
+
+
+**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.
+
+- As of current implementation the state storage is cleaned up of expired 
value 
+only on its explicit read access per key, e.g. calling `ValueState.value()`. 
+This might change in future releases, e.g. additional strategies might be 
added in background to speed up cleanup.
+
+- Only *processing time* scale is currently supported for TTL.
 
 Review comment:
   Only TTLs in reference to *processing time* are currently supported.


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


With regards,
Apache Git Services


[GitHub] eliaslevy commented on a change in pull request #6379: [FLINK-9637] Add public user documentation for state TTL feature

2018-07-29 Thread GitBox
eliaslevy 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_r205991075
 
 

 ##
 File path: docs/dev/stream/state/state.md
 ##
 @@ -266,6 +266,92 @@ 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 based on the best effort.
+Depending on configuration, the expired state can become unavailable for read 
access
+even if it is not cleaned up yet. In this case it behaves as if it does not 
exist any more.
+
+The collection types of state support TTL on entry level: 
+separate list elements and map entries expire independently. 
+
+The behaviour of state with TTL firstly should be configured by building 
`StateTtlConfiguration`:
+
+
+
+{% highlight java %}
+StateTtlConfiguration ttlConfig = StateTtlConfiguration
+.newBuilder(Time.seconds(1))
+.setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite)
+
.setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired)
+.build();
+{% endhighlight %}
+
+
+
+{% highlight scala %}
+val ttlConfig = StateTtlConfiguration
+.newBuilder(Time.seconds(1))
+.setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite)
+
.setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired)
+.build()
+{% endhighlight %}
+
+
+
+It has several options to consider. 
+The first parameter of `newBuilder` method is mandatory, it is a value of 
time-to-live itself.
+
+The update type configures when the time-to-live of state value is prolonged 
(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.
+
+The TTL can be enabled in descriptor for any type of state:
+
+
+
+{% highlight java %}
+StateTtlConfiguration ttlConfig = 
StateTtlConfiguration.newBuilder(Time.seconds(1)).build();
+ValueStateDescriptor stateDescriptor = new 
ValueStateDescriptor<>("text state", String.class);
+stateDescriptor.enableTimeToLive(ttlConfig);
+{% endhighlight %}
+
+
+
+{% highlight scala %}
+val ttlConfig = StateTtlConfiguration.newBuilder(Time.seconds(1)).build()
+val stateDescriptor = new ValueStateDescriptor[String]("text state", 
classOf[String])
+stateDescriptor.enableTimeToLive(ttlConfig)
+{% endhighlight %}
+
+
+
+**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.
+
+- As of current implementation the state storage is cleaned up of expired 
value 
+only on its explicit read access per key, e.g. calling `ValueState.value()`. 
+This might change in future releases, e.g. additional strategies might be 
added in background to speed up cleanup.
 
 Review comment:
   In the current implementation, expired values are only removed when they are 
read explicitly, e.g. by calling `ValueState.value()`. Note that this means 
that under the current implementation if expired state is not read, it won't be 
removed, possibly leading to ever growing state. This might change in future 
releases. Additional strategies might be added that clean up expired state in 
the background. 
   
   BTW, is this true given FLINK-9938?


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


With regards,
Apache Git Services


[GitHub] eliaslevy commented on a change in pull request #6379: [FLINK-9637] Add public user documentation for state TTL feature

2018-07-29 Thread GitBox
eliaslevy 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_r205990823
 
 

 ##
 File path: docs/dev/stream/state/state.md
 ##
 @@ -266,6 +266,92 @@ 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 based on the best effort.
+Depending on configuration, the expired state can become unavailable for read 
access
+even if it is not cleaned up yet. In this case it behaves as if it does not 
exist any more.
+
+The collection types of state support TTL on entry level: 
+separate list elements and map entries expire independently. 
+
+The behaviour of state with TTL firstly should be configured by building 
`StateTtlConfiguration`:
+
+
+
+{% highlight java %}
+StateTtlConfiguration ttlConfig = StateTtlConfiguration
+.newBuilder(Time.seconds(1))
+.setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite)
+
.setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired)
+.build();
+{% endhighlight %}
+
+
+
+{% highlight scala %}
+val ttlConfig = StateTtlConfiguration
+.newBuilder(Time.seconds(1))
+.setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite)
+
.setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired)
+.build()
+{% endhighlight %}
+
+
+
+It has several options to consider. 
+The first parameter of `newBuilder` method is mandatory, it is a value of 
time-to-live itself.
+
+The update type configures when the time-to-live of state value is prolonged 
(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.
+
+The TTL can be enabled in descriptor for any type of state:
 
 Review comment:
   TTL functionality can be enabled in the descriptor of any type of state:


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


With regards,
Apache Git Services


[GitHub] eliaslevy commented on a change in pull request #6379: [FLINK-9637] Add public user documentation for state TTL feature

2018-07-29 Thread GitBox
eliaslevy 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_r205990785
 
 

 ##
 File path: docs/dev/stream/state/state.md
 ##
 @@ -266,6 +266,92 @@ 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 based on the best effort.
+Depending on configuration, the expired state can become unavailable for read 
access
+even if it is not cleaned up yet. In this case it behaves as if it does not 
exist any more.
+
+The collection types of state support TTL on entry level: 
+separate list elements and map entries expire independently. 
+
+The behaviour of state with TTL firstly should be configured by building 
`StateTtlConfiguration`:
+
+
+
+{% highlight java %}
+StateTtlConfiguration ttlConfig = StateTtlConfiguration
+.newBuilder(Time.seconds(1))
+.setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite)
+
.setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired)
+.build();
+{% endhighlight %}
+
+
+
+{% highlight scala %}
+val ttlConfig = StateTtlConfiguration
+.newBuilder(Time.seconds(1))
+.setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite)
+
.setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired)
+.build()
+{% endhighlight %}
+
+
+
+It has several options to consider. 
+The first parameter of `newBuilder` method is mandatory, it is a value of 
time-to-live itself.
+
+The update type configures when the time-to-live of state value is prolonged 
(default `OnCreateAndWrite`):
 
 Review comment:
   The update type configures when the state TTL is refreshed (default 
`OnCreateAndWrite`):


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


With regards,
Apache Git Services


[GitHub] eliaslevy commented on a change in pull request #6379: [FLINK-9637] Add public user documentation for state TTL feature

2018-07-29 Thread GitBox
eliaslevy 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_r205990730
 
 

 ##
 File path: docs/dev/stream/state/state.md
 ##
 @@ -266,6 +266,92 @@ 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 based on the best effort.
+Depending on configuration, the expired state can become unavailable for read 
access
+even if it is not cleaned up yet. In this case it behaves as if it does not 
exist any more.
+
+The collection types of state support TTL on entry level: 
+separate list elements and map entries expire independently. 
+
+The behaviour of state with TTL firstly should be configured by building 
`StateTtlConfiguration`:
+
+
+
+{% highlight java %}
+StateTtlConfiguration ttlConfig = StateTtlConfiguration
+.newBuilder(Time.seconds(1))
+.setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite)
+
.setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired)
+.build();
+{% endhighlight %}
+
+
+
+{% highlight scala %}
+val ttlConfig = StateTtlConfiguration
+.newBuilder(Time.seconds(1))
+.setTtlUpdateType(StateTtlConfiguration.TtlUpdateType.OnCreateAndWrite)
+
.setStateVisibility(StateTtlConfiguration.TtlStateVisibility.NeverReturnExpired)
+.build()
+{% endhighlight %}
+
+
+
+It has several options to consider. 
+The first parameter of `newBuilder` method is mandatory, it is a value of 
time-to-live itself.
 
 Review comment:
   The first parameter of `newBuilder` method is mandatory, it is the 
time-to-live value.


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


With regards,
Apache Git Services


[GitHub] eliaslevy commented on a change in pull request #6379: [FLINK-9637] Add public user documentation for state TTL feature

2018-07-29 Thread GitBox
eliaslevy 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_r205990700
 
 

 ##
 File path: docs/dev/stream/state/state.md
 ##
 @@ -266,6 +266,92 @@ 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 based on the best effort.
+Depending on configuration, the expired state can become unavailable for read 
access
+even if it is not cleaned up yet. In this case it behaves as if it does not 
exist any more.
+
+The collection types of state support TTL on entry level: 
+separate list elements and map entries expire independently. 
+
+The behaviour of state with TTL firstly should be configured by building 
`StateTtlConfiguration`:
 
 Review comment:
   To use state TTL you must first build a `StateTtlConfiguration` object:


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


With regards,
Apache Git Services


[GitHub] eliaslevy commented on a change in pull request #6379: [FLINK-9637] Add public user documentation for state TTL feature

2018-07-29 Thread GitBox
eliaslevy 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_r205990622
 
 

 ##
 File path: docs/dev/stream/state/state.md
 ##
 @@ -266,6 +266,92 @@ 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 based on the best effort.
+Depending on configuration, the expired state can become unavailable for read 
access
+even if it is not cleaned up yet. In this case it behaves as if it does not 
exist any more.
+
+The collection types of state support TTL on entry level: 
+separate list elements and map entries expire independently. 
 
 Review comment:
   The state collection types support per-entry TTLs: list elements and map 
entries expire independently.


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


With regards,
Apache Git Services


[GitHub] eliaslevy commented on a change in pull request #6379: [FLINK-9637] Add public user documentation for state TTL feature

2018-07-29 Thread GitBox
eliaslevy 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_r205990514
 
 

 ##
 File path: docs/dev/stream/state/state.md
 ##
 @@ -266,6 +266,92 @@ 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 based on the best effort.
+Depending on configuration, the expired state can become unavailable for read 
access
+even if it is not cleaned up yet. In this case it behaves as if it does not 
exist any more.
 
 Review comment:
   s/even if it is not cleaned up yet/even if it has yet to be removed/
   s/In this case it behaves as if it does not exist any more/In that case, it 
behaves as if it no longer exists/


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


With regards,
Apache Git Services


[GitHub] eliaslevy commented on a change in pull request #6379: [FLINK-9637] Add public user documentation for state TTL feature

2018-07-29 Thread GitBox
eliaslevy 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_r205990412
 
 

 ##
 File path: docs/dev/stream/state/state.md
 ##
 @@ -266,6 +266,92 @@ 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 based on the best effort.
 
 Review comment:
   s/based on the best effort/on a best effort basis/


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


With regards,
Apache Git Services