@Override public StateTag<StateT> asKind(StateKind kind) { return new SimpleStateTag<>(id.asKind(kind), spec); }
/** Create a state spec that supporting for {@link java.util.Set} like access patterns. */ public static <T> StateTag<SetState<T>> set(String id, Coder<T> elemCoder) { return new SimpleStateTag<>(new StructuredId(id), StateSpecs.set(elemCoder)); }
/** * Create a state tag for values that use a {@link CombineFn} to automatically merge multiple * {@code InputT}s into a single {@code OutputT}. * * <p>This determines the {@code Coder<AccumT>} from the given {@code Coder<InputT>}, and should * only be used to initialize static values. */ public static <InputT, AccumT, OutputT> StateTag<CombiningState<InputT, AccumT, OutputT>> combiningValueFromInputInternal( String id, Coder<InputT> inputCoder, CombineFn<InputT, AccumT, OutputT> combineFn) { return new SimpleStateTag<>( new StructuredId(id), StateSpecs.combiningFromInputInternal(inputCoder, combineFn)); }
/** Create a state spec that supporting for {@link java.util.Map} like access patterns. */ public static <K, V> StateTag<MapState<K, V>> map( String id, Coder<K> keyCoder, Coder<V> valueCoder) { return new SimpleStateTag<>(new StructuredId(id), StateSpecs.map(keyCoder, valueCoder)); }
/** * Create a state tag that is optimized for adding values frequently, and occasionally retrieving * all the values that have been added. */ public static <T> StateTag<BagState<T>> bag(String id, Coder<T> elemCoder) { return new SimpleStateTag<>(new StructuredId(id), StateSpecs.bag(elemCoder)); }
/** Create a state tag for the given id and spec. */ public static <StateT extends State> StateTag<StateT> tagForSpec( String id, StateSpec<StateT> spec) { return new SimpleStateTag<>(new StructuredId(id), spec); }
public static <InputT, AccumT, OutputT> StateTag<BagState<AccumT>> convertToBagTagInternal( StateTag<CombiningState<InputT, AccumT, OutputT>> combiningTag) { return new SimpleStateTag<>( new StructuredId(combiningTag.getId()), StateSpecs.convertToBagSpecInternal(combiningTag.getSpec())); }
/** Create a state tag for holding the watermark. */ public static <W extends BoundedWindow> StateTag<WatermarkHoldState> watermarkStateInternal( String id, TimestampCombiner timestampCombiner) { return new SimpleStateTag<>( new StructuredId(id), StateSpecs.watermarkStateInternal(timestampCombiner)); }
/** Create a simple state tag for values of type {@code T}. */ public static <T> StateTag<ValueState<T>> value(String id, Coder<T> valueCoder) { return new SimpleStateTag<>(new StructuredId(id), StateSpecs.value(valueCoder)); }
/** * Create a state tag for values that use a {@link CombineFn} to automatically merge multiple * {@code InputT}s into a single {@code OutputT}. */ public static <InputT, AccumT, OutputT> StateTag<CombiningState<InputT, AccumT, OutputT>> combiningValue( String id, Coder<AccumT> accumCoder, CombineFn<InputT, AccumT, OutputT> combineFn) { return new SimpleStateTag<>(new StructuredId(id), StateSpecs.combining(accumCoder, combineFn)); }
/** * Create a state tag for values that use a {@link CombineFnWithContext} to automatically merge * multiple {@code InputT}s into a single {@code OutputT}. */ public static <InputT, AccumT, OutputT> StateTag<CombiningState<InputT, AccumT, OutputT>> combiningValueWithContext( String id, Coder<AccumT> accumCoder, CombineFnWithContext<InputT, AccumT, OutputT> combineFn) { return new SimpleStateTag<>(new StructuredId(id), StateSpecs.combining(accumCoder, combineFn)); }