@@ -1093,19 +1093,55 @@ public Read<K, V> withTopicPartitions(List<TopicPartition> topicPartitions) {
10931093
10941094 /**
10951095 * Sets redistribute transform that hints to the runner to try to redistribute the work evenly.
1096+ *
1097+ * @return an updated {@link Read} transform.
10961098 */
10971099 public Read <K , V > withRedistribute () {
10981100 return toBuilder ().setRedistributed (true ).build ();
10991101 }
11001102
1103+ /**
1104+ * Hints to the runner that it can relax exactly-once processing guarantees, allowing duplicates
1105+ * in at-least-once processing mode of Kafka inputs.
1106+ *
1107+ * <p>Must be used with {@link KafkaIO#withRedistribute()}.
1108+ *
1109+ * <p>Not compatible with {@link KafkaIO#withOffsetDeduplication()}.
1110+ *
1111+ * @param allowDuplicates specifies whether to allow duplicates.
1112+ * @return an updated {@link Read} transform.
1113+ */
11011114 public Read <K , V > withAllowDuplicates (Boolean allowDuplicates ) {
11021115 return toBuilder ().setAllowDuplicates (allowDuplicates ).build ();
11031116 }
11041117
1118+ /**
1119+ * Redistributes Kafka messages into a distinct number of keys for processing in subsequent
1120+ * steps.
1121+ *
1122+ * <p>Specifying an explicit number of keys is generally recommended over redistributing into an
1123+ * unbounded key space.
1124+ *
1125+ * <p>Must be used with {@link KafkaIO#withRedistribute()}.
1126+ *
1127+ * @param redistributeNumKeys specifies the total number of keys for redistributing inputs.
1128+ * @return an updated {@link Read} transform.
1129+ */
11051130 public Read <K , V > withRedistributeNumKeys (int redistributeNumKeys ) {
11061131 return toBuilder ().setRedistributeNumKeys (redistributeNumKeys ).build ();
11071132 }
11081133
1134+ /**
1135+ * Hints to the runner to optimize the redistribute by minimizing the amount of data required
1136+ * for persistence as part of the redistribute operation.
1137+ *
1138+ * <p>Must be used with {@link KafkaIO#withRedistribute()}.
1139+ *
1140+ * <p>Not compatible with {@link KafkaIO#withAllowDuplicates()}.
1141+ *
1142+ * @param offsetDeduplication specifies whether to enable offset-based deduplication.
1143+ * @return an updated {@link Read} transform.
1144+ */
11091145 public Read <K , V > withOffsetDeduplication (Boolean offsetDeduplication ) {
11101146 return toBuilder ().setOffsetDeduplication (offsetDeduplication ).build ();
11111147 }
0 commit comments