|
23 | 23 | import org.apache.flink.table.annotation.ArgumentTrait;
|
24 | 24 | import org.apache.flink.table.annotation.DataTypeHint;
|
25 | 25 | import org.apache.flink.table.annotation.FunctionHint;
|
| 26 | +import org.apache.flink.table.annotation.StateHint; |
26 | 27 | import org.apache.flink.table.catalog.DataTypeFactory;
|
27 | 28 | import org.apache.flink.table.types.extraction.TypeInferenceExtractor;
|
28 | 29 | import org.apache.flink.table.types.inference.TypeInference;
|
|
144 | 145 | * <pre>{@code
|
145 | 146 | * // Function with explicit table argument type of row
|
146 | 147 | * class MyPTF extends ProcessTableFunction<String> {
|
147 |
| - * public void eval(Context ctx, @ArgumentHint(value = ArgumentTrait.TABLE_AS_SET, type = "ROW < s STRING >") Row t) { |
| 148 | + * public void eval(Context ctx, @ArgumentHint(value = ArgumentTrait.TABLE_AS_SET, type = @DataTypeHint("ROW < s STRING >")) Row t) { |
148 | 149 | * TableSemantics semantics = ctx.tableSemanticsFor("t");
|
149 | 150 | * // Always returns "ROW < s STRING >"
|
150 | 151 | * semantics.dataType();
|
|
194 | 195 | * }
|
195 | 196 | * }</pre>
|
196 | 197 | *
|
| 198 | + * <h1>State</h1> |
| 199 | + * |
| 200 | + * <p>A PTF that takes set semantic tables can be stateful. Intermediate results can be buffered, |
| 201 | + * cached, aggregated, or simply stored for repeated access. A function can have one or more state |
| 202 | + * entries which are managed by the framework. Flink takes care of storing and restoring those |
| 203 | + * during failures or restarts (i.e. Flink managed state). |
| 204 | + * |
| 205 | + * <p>A state entry is partitioned by a key and cannot be accessed globally. The partitioning (or a |
| 206 | + * single partition in case of no partitioning) is defined by the corresponding function call. In |
| 207 | + * other words: Similar to how a virtual processor has access only to a portion of the entire table, |
| 208 | + * a PTF has access only to a portion of the entire state defined by the PARTITION BY clause. In |
| 209 | + * Flink, this concept is also known as keyed state. |
| 210 | + * |
| 211 | + * <p>State entries can be added as a mutable parameter to the eval() method. In order to |
| 212 | + * distinguish them from call arguments, they must be declared before any other argument, but after |
| 213 | + * an optional {@link Context} parameter. Furthermore, they must be annotated either via {@link |
| 214 | + * StateHint} or declared as part of {@link FunctionHint#state()}. |
| 215 | + * |
| 216 | + * <p>For read and write access, only row or structured types (i.e. POJOs with default constructor) |
| 217 | + * qualify as a data type. If no state is present, all fields are set to null (in case of a row |
| 218 | + * type) or fields are set to their default value (in case of a structured type). For state |
| 219 | + * efficiency, it is recommended to keep all fields nullable. |
| 220 | + * |
| 221 | + * <pre>{@code |
| 222 | + * // a function that counts and stores its intermediate result in the CountState object |
| 223 | + * // which will be persisted by Flink |
| 224 | + * class CountingFunction extends ProcessTableFunction<String> { |
| 225 | + * public static class CountState { |
| 226 | + * public long count = 0L; |
| 227 | + * } |
| 228 | + * |
| 229 | + * public void eval(@StateHint CountState memory, @ArgumentHint(TABLE_AS_SET) Row input) { |
| 230 | + * memory.count++; |
| 231 | + * collect("Seen rows: " + memory.count); |
| 232 | + * } |
| 233 | + * } |
| 234 | + * |
| 235 | + * // a function that waits for a second event coming in |
| 236 | + * class CountingFunction extends ProcessTableFunction<String> { |
| 237 | + * public static class SeenState { |
| 238 | + * public String first; |
| 239 | + * } |
| 240 | + * |
| 241 | + * public void eval(@StateHint SeenState memory, @ArgumentHint(TABLE_AS_SET) Row input) { |
| 242 | + * if (memory.first == null) { |
| 243 | + * memory.first = input.toString(); |
| 244 | + * } else { |
| 245 | + * collect("Event 1: " + memory.first + " and Event 2: " + input.toString()); |
| 246 | + * } |
| 247 | + * } |
| 248 | + * } |
| 249 | + * |
| 250 | + * // a function that uses Row for state |
| 251 | + * class CountingFunction extends ProcessTableFunction<String> { |
| 252 | + * public void eval(@StateHint(type = @DataTypeHint("ROW < count BIGINT >")) Row memory, @ArgumentHint(TABLE_AS_SET) Row input) { |
| 253 | + * Long newCount = 1L; |
| 254 | + * if (memory.getField("count") != null) { |
| 255 | + * newCount += memory.getFieldAs("count"); |
| 256 | + * } |
| 257 | + * memory.setField("count", newCount); |
| 258 | + * collect("Seen rows: " + newCount); |
| 259 | + * } |
| 260 | + * } |
| 261 | + * }</pre> |
| 262 | + * |
| 263 | + * <h2>Efficiency and Design Principles</h2> |
| 264 | + * |
| 265 | + * <p>A stateful function also means that data layout and data retention should be well thought |
| 266 | + * through. An ever-growing state can happen by an unlimited number of partitions (i.e. an open |
| 267 | + * keyspace) or even within a partition. Consider setting a {@link StateHint#ttl()} or call {@link |
| 268 | + * Context#clearAllState()} eventually: |
| 269 | + * |
| 270 | + * <pre>{@code |
| 271 | + * // a function that waits for a second event coming in BUT with better state efficiency |
| 272 | + * class CountingFunction extends ProcessTableFunction<String> { |
| 273 | + * public static class SeenState { |
| 274 | + * public String first; |
| 275 | + * } |
| 276 | + * |
| 277 | + * public void eval(Context ctx, @StateHint(ttl = "1 day") SeenState memory, @ArgumentHint(TABLE_AS_SET) Row input) { |
| 278 | + * if (memory.first == null) { |
| 279 | + * memory.first = input.toString(); |
| 280 | + * } else { |
| 281 | + * collect("Event 1: " + memory.first + " and Event 2: " + input.toString()); |
| 282 | + * ctx.clearAllState(); |
| 283 | + * } |
| 284 | + * } |
| 285 | + * } |
| 286 | + * }</pre> |
| 287 | + * |
197 | 288 | * @param <T> The type of the output row. Either an explicit composite type or an atomic type that
|
198 | 289 | * is implicitly wrapped into a row consisting of one field.
|
199 | 290 | */
|
@@ -241,8 +332,28 @@ public interface Context {
|
241 | 332 | /**
|
242 | 333 | * Returns additional information about the semantics of a table argument.
|
243 | 334 | *
|
244 |
| - * @param argName name of the table argument |
| 335 | + * @param argName name of the table argument; either reflectively extracted or manually |
| 336 | + * defined via {@link ArgumentHint#name()}. |
245 | 337 | */
|
246 | 338 | TableSemantics tableSemanticsFor(String argName);
|
| 339 | + |
| 340 | + /** |
| 341 | + * Clears the given state entry within the virtual partition once the eval() method returns. |
| 342 | + * |
| 343 | + * <p>Semantically this is equal to setting all fields of the state entry to null shortly |
| 344 | + * before the eval() method returns. |
| 345 | + * |
| 346 | + * @param stateName name of the state entry; either reflectively extracted or manually |
| 347 | + * defined via {@link StateHint#name()}. |
| 348 | + */ |
| 349 | + void clearState(String stateName); |
| 350 | + |
| 351 | + /** |
| 352 | + * Clears all state entries within the virtual partition once the eval() method returns. |
| 353 | + * |
| 354 | + * <p>Semantically this is equal to calling {@link #clearState(String)} on all state |
| 355 | + * entries. |
| 356 | + */ |
| 357 | + void clearAllState(); |
247 | 358 | }
|
248 | 359 | }
|
0 commit comments