@@ -73,7 +73,7 @@ public interface MessageStream<M> {
7373 * should be retained in the filtered {@link MessageStream}.
7474 *
7575 * @param filterFn the predicate to filter messages from this {@link MessageStream}.
76- * @return the transformed {@link MessageStream}
76+ * @return the filtered {@link MessageStream}
7777 */
7878 MessageStream <M > filter (FilterFunction <? super M > filterFn );
7979
@@ -105,15 +105,19 @@ public interface MessageStream<M> {
105105 * <p>
106106 * Use the {@link org.apache.samza.operators.windows.Windows} helper methods to create the appropriate windows.
107107 * <p>
108- * <b>Warning:</b> As of version 0.13.0, messages in windows are kept in memory and will be lost during restarts.
108+ * The {@code id} must be unique for each operator in this application. It is used as part of the unique ID
109+ * for any state stores and streams created by this operator (the full ID also contains the job name, job id and
110+ * operator type). If the application logic is changed, this ID must be reused in the new operator to retain
111+ * state from the previous version, and changed for the new operator to discard the state from the previous version.
109112 *
110113 * @param window the window to group and process messages from this {@link MessageStream}
114+ * @param id the unique id of this operator in this application
111115 * @param <K> the type of key in the message in this {@link MessageStream}. If a key is specified,
112116 * panes are emitted per-key.
113117 * @param <WV> the type of value in the {@link WindowPane} in the transformed {@link MessageStream}
114- * @return the transformed {@link MessageStream}
118+ * @return the windowed {@link MessageStream}
115119 */
116- <K , WV > MessageStream <WindowPane <K , WV >> window (Window <M , K , WV > window );
120+ <K , WV > MessageStream <WindowPane <K , WV >> window (Window <M , K , WV > window , String id );
117121
118122 /**
119123 * Joins this {@link MessageStream} with another {@link MessageStream} using the provided
@@ -124,22 +128,27 @@ public interface MessageStream<M> {
124128 * <p>
125129 * Both inputs being joined must have the same number of partitions, and should be partitioned by the join key.
126130 * <p>
127- * <b>Warning:</b> As of version 0.13.0, messages in joins are kept in memory and will be lost during restarts.
131+ * The {@code id} must be unique for each operator in this application. It is used as part of the unique ID
132+ * for any state stores and streams created by this operator (the full ID also contains the job name, job id and
133+ * operator type). If the application logic is changed, this ID must be reused in the new operator to retain
134+ * state from the previous version, and changed for the new operator to discard the state from the previous version.
128135 *
129136 * @param otherStream the other {@link MessageStream} to be joined with
130137 * @param joinFn the function to join messages from this and the other {@link MessageStream}
131- * @param ttl the ttl for messages in each stream
132138 * @param keySerde the serde for the join key
133139 * @param messageSerde the serde for messages in this stream
134140 * @param otherMessageSerde the serde for messages in the other stream
141+ * @param ttl the ttl for messages in each stream
142+ * @param id the unique id of this operator in this application
135143 * @param <K> the type of join key
136144 * @param <OM> the type of messages in the other stream
137145 * @param <JM> the type of messages resulting from the {@code joinFn}
138146 * @return the joined {@link MessageStream}
139147 */
140148 <K , OM , JM > MessageStream <JM > join (MessageStream <OM > otherStream ,
141149 JoinFunction <? extends K , ? super M , ? super OM , ? extends JM > joinFn ,
142- Serde <K > keySerde , Serde <M > messageSerde , Serde <OM > otherMessageSerde , Duration ttl );
150+ Serde <K > keySerde , Serde <M > messageSerde , Serde <OM > otherMessageSerde ,
151+ Duration ttl , String id );
143152
144153 /**
145154 * Merges all {@code otherStreams} with this {@link MessageStream}.
@@ -186,26 +195,34 @@ static <T> MessageStream<T> mergeAll(Collection<? extends MessageStream<? extend
186195 * configuration, if present.
187196 * Else, the number of partitions is set to to the max of number of partitions for all input and output streams
188197 * (excluding intermediate streams).
198+ * <p>
199+ * The {@code id} must be unique for each operator in this application. It is used as part of the unique ID
200+ * for any state stores and streams created by this operator (the full ID also contains the job name, job id and
201+ * operator type). If the application logic is changed, this ID must be reused in the new operator to retain
202+ * state from the previous version, and changed for the new operator to discard the state from the previous version.
189203 *
190- * @param <K> the type of output key
191- * @param <V> the type of output value
192204 * @param keyExtractor the {@link Function} to extract the message and partition key from the input message
193205 * @param valueExtractor the {@link Function} to extract the value from the input message
194206 * @param serde the {@link KVSerde} to use for (de)serializing the key and value.
207+ * @param id the unique id of this operator in this application
208+ * @param <K> the type of output key
209+ * @param <V> the type of output value
195210 * @return the repartitioned {@link MessageStream}
196211 */
197212 <K , V > MessageStream <KV <K , V >> partitionBy (Function <? super M , ? extends K > keyExtractor ,
198- Function <? super M , ? extends V > valueExtractor , KVSerde <K , V > serde );
213+ Function <? super M , ? extends V > valueExtractor , KVSerde <K , V > serde , String id );
214+
199215
200216 /**
201- * Same as calling {@link #partitionBy(Function, Function, KVSerde)} with a null KVSerde.
217+ * Same as calling {@link #partitionBy(Function, Function, KVSerde, String )} with a null KVSerde.
202218 *
203219 * @param keyExtractor the {@link Function} to extract the message and partition key from the input message
204220 * @param valueExtractor the {@link Function} to extract the value from the input message
221+ * @param id the unique id of this operator in this application
205222 * @param <K> the type of output key
206223 * @param <V> the type of output value
207224 * @return the repartitioned {@link MessageStream}
208225 */
209226 <K , V > MessageStream <KV <K , V >> partitionBy (Function <? super M , ? extends K > keyExtractor ,
210- Function <? super M , ? extends V > valueExtractor );
227+ Function <? super M , ? extends V > valueExtractor , String id );
211228}
0 commit comments