@@ -655,6 +655,14 @@ public static <K, V> WriteRecords<K, V> writeRecords() {
655655
656656 ///////////////////////// Read Support \\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\
657657
658+ /**
659+ * Default number of keys to redistribute Kafka inputs into.
660+ *
661+ * <p>This value is used when {@link Read#withRedistribute()} is used without {@link
662+ * Read#withRedistributeNumKeys(int redistributeNumKeys)}.
663+ */
664+ private static final int DEFAULT_REDISTRIBUTE_NUM_KEYS = 32768 ;
665+
658666 /**
659667 * A {@link PTransform} to read from Kafka topics. See {@link KafkaIO} for more information on
660668 * usage and configuration.
@@ -1099,7 +1107,11 @@ public Read<K, V> withTopicPartitions(List<TopicPartition> topicPartitions) {
10991107 * @return an updated {@link Read} transform.
11001108 */
11011109 public Read <K , V > withRedistribute () {
1102- return toBuilder ().setRedistributed (true ).build ();
1110+ Builder <K , V > builder = toBuilder ().setRedistributed (true );
1111+ if (getRedistributeNumKeys () == 0 ) {
1112+ builder = builder .setRedistributeNumKeys (DEFAULT_REDISTRIBUTE_NUM_KEYS );
1113+ }
1114+ return builder .build ();
11031115 }
11041116
11051117 /**
@@ -1121,10 +1133,11 @@ public Read<K, V> withAllowDuplicates(Boolean allowDuplicates) {
11211133 * Redistributes Kafka messages into a distinct number of keys for processing in subsequent
11221134 * steps.
11231135 *
1124- * <p>Specifying an explicit number of keys is generally recommended over redistributing into an
1125- * unbounded key space.
1136+ * <p>If unset, defaults to {@link KafkaIO#DEFAULT_REDISTRIBUTE_NUM_KEYS}.
11261137 *
1127- * <p>Must be used with {@link KafkaIO#withRedistribute()}.
1138+ * <p>Use zero to disable bucketing into a distinct number of keys.
1139+ *
1140+ * <p>Must be used with {@link Read#withRedistribute()}.
11281141 *
11291142 * @param redistributeNumKeys specifies the total number of keys for redistributing inputs.
11301143 * @return an updated {@link Read} transform.
@@ -2667,13 +2680,30 @@ public ReadSourceDescriptors<K, V> withProcessingTime() {
26672680
26682681 /** Enable Redistribute. */
26692682 public ReadSourceDescriptors <K , V > withRedistribute () {
2670- return toBuilder ().setRedistribute (true ).build ();
2683+ Builder <K , V > builder = toBuilder ().setRedistribute (true );
2684+ if (getRedistributeNumKeys () == 0 ) {
2685+ builder = builder .setRedistributeNumKeys (DEFAULT_REDISTRIBUTE_NUM_KEYS );
2686+ }
2687+ return builder .build ();
26712688 }
26722689
26732690 public ReadSourceDescriptors <K , V > withAllowDuplicates () {
26742691 return toBuilder ().setAllowDuplicates (true ).build ();
26752692 }
26762693
2694+ /**
2695+ * Redistributes Kafka messages into a distinct number of keys for processing in subsequent
2696+ * steps.
2697+ *
2698+ * <p>If unset, defaults to {@link KafkaIO#DEFAULT_REDISTRIBUTE_NUM_KEYS}.
2699+ *
2700+ * <p>Use zero to disable bucketing into a distinct number of keys.
2701+ *
2702+ * <p>Must be used with {@link ReadSourceDescriptors#withRedistribute()}.
2703+ *
2704+ * @param redistributeNumKeys specifies the total number of keys for redistributing inputs.
2705+ * @return an updated {@link Read} transform.
2706+ */
26772707 public ReadSourceDescriptors <K , V > withRedistributeNumKeys (int redistributeNumKeys ) {
26782708 return toBuilder ().setRedistributeNumKeys (redistributeNumKeys ).build ();
26792709 }
0 commit comments