2121import org .apache .fluss .config .Configuration ;
2222import org .apache .fluss .flink .adapter .SinkAdapter ;
2323import org .apache .fluss .flink .sink .serializer .FlussSerializationSchema ;
24+ import org .apache .fluss .flink .sink .shuffle .DataStatisticsOperatorFactory ;
25+ import org .apache .fluss .flink .sink .shuffle .DistributionMode ;
26+ import org .apache .fluss .flink .sink .shuffle .StatisticsOrRecord ;
27+ import org .apache .fluss .flink .sink .shuffle .StatisticsOrRecordChannelComputer ;
28+ import org .apache .fluss .flink .sink .shuffle .StatisticsOrRecordTypeInformation ;
2429import org .apache .fluss .flink .sink .writer .AppendSinkWriter ;
2530import org .apache .fluss .flink .sink .writer .FlinkSinkWriter ;
2631import org .apache .fluss .flink .sink .writer .UpsertSinkWriter ;
2732import org .apache .fluss .metadata .DataLakeFormat ;
2833import org .apache .fluss .metadata .TablePath ;
2934
35+ import org .apache .flink .api .common .functions .FlatMapFunction ;
3036import org .apache .flink .api .common .operators .MailboxExecutor ;
37+ import org .apache .flink .api .common .typeinfo .TypeInformation ;
3138import org .apache .flink .api .connector .sink2 .SinkWriter ;
3239import org .apache .flink .metrics .groups .SinkWriterMetricGroup ;
3340import org .apache .flink .runtime .metrics .groups .InternalSinkWriterMetricGroup ;
3441import org .apache .flink .streaming .api .connector .sink2 .SupportsPreWriteTopology ;
3542import org .apache .flink .streaming .api .datastream .DataStream ;
43+ import org .apache .flink .streaming .api .datastream .SingleOutputStreamOperator ;
3644import org .apache .flink .table .types .logical .RowType ;
3745
3846import javax .annotation .Nullable ;
@@ -87,8 +95,9 @@ static class AppendSinkWriterBuilder<InputT>
8795 private final List <String > bucketKeys ;
8896 private final List <String > partitionKeys ;
8997 private final @ Nullable DataLakeFormat lakeFormat ;
90- private final boolean shuffleByBucketId ;
98+ private final DistributionMode shuffleMode ;
9199 private final FlussSerializationSchema <InputT > flussSerializationSchema ;
100+ private final @ Nullable TypeInformation <InputT > rowTypeInformation ;
92101
93102 public AppendSinkWriterBuilder (
94103 TablePath tablePath ,
@@ -98,17 +107,19 @@ public AppendSinkWriterBuilder(
98107 List <String > bucketKeys ,
99108 List <String > partitionKeys ,
100109 @ Nullable DataLakeFormat lakeFormat ,
101- boolean shuffleByBucketId ,
102- FlussSerializationSchema <InputT > flussSerializationSchema ) {
110+ DistributionMode shuffleMode ,
111+ FlussSerializationSchema <InputT > flussSerializationSchema ,
112+ @ Nullable TypeInformation <InputT > rowTypeInformation ) {
103113 this .tablePath = tablePath ;
104114 this .flussConfig = flussConfig ;
105115 this .tableRowType = tableRowType ;
106116 this .numBucket = numBucket ;
107117 this .bucketKeys = bucketKeys ;
108118 this .partitionKeys = partitionKeys ;
109119 this .lakeFormat = lakeFormat ;
110- this .shuffleByBucketId = shuffleByBucketId ;
120+ this .shuffleMode = shuffleMode ;
111121 this .flussSerializationSchema = flussSerializationSchema ;
122+ this .rowTypeInformation = rowTypeInformation ;
112123 }
113124
114125 @ Override
@@ -123,20 +134,76 @@ public AppendSinkWriter<InputT> createWriter(MailboxExecutor mailboxExecutor) {
123134
124135 @ Override
125136 public DataStream <InputT > addPreWriteTopology (DataStream <InputT > input ) {
126- // For append only sink, we will do bucket shuffle only if bucket keys are not empty.
127- if (!bucketKeys .isEmpty () && shuffleByBucketId ) {
128- return partition (
129- input ,
130- new FlinkRowDataChannelComputer <>(
131- toFlussRowType (tableRowType ),
132- bucketKeys ,
133- partitionKeys ,
134- lakeFormat ,
135- numBucket ,
136- flussSerializationSchema ),
137- input .getParallelism ());
138- } else {
139- return input ;
137+ switch (shuffleMode ) {
138+ case BUCKET_SHUFFLE :
139+ if (!bucketKeys .isEmpty ()) {
140+ return partition (
141+ input ,
142+ new FlinkRowDataChannelComputer <>(
143+ toFlussRowType (tableRowType ),
144+ bucketKeys ,
145+ partitionKeys ,
146+ lakeFormat ,
147+ numBucket ,
148+ flussSerializationSchema ),
149+ input .getParallelism ());
150+ }
151+ return input ;
152+ case NONE :
153+ return input ;
154+ case DYNAMIC_SHUFFLE :
155+ if (partitionKeys .isEmpty ()) {
156+ throw new UnsupportedOperationException (
157+ "DYNAMIC_SHUFFLE is only supported for partition tables" );
158+ }
159+
160+ if (rowTypeInformation == null ) {
161+ throw new UnsupportedOperationException (
162+ "RowTypeInformation is required for DYNAMIC_SHUFFLE mode." );
163+ }
164+ TypeInformation <StatisticsOrRecord <InputT >> statisticsOrRecordTypeInformation =
165+ new StatisticsOrRecordTypeInformation <>(rowTypeInformation );
166+ SingleOutputStreamOperator <StatisticsOrRecord <InputT >> shuffleStream =
167+ input .transform (
168+ "Dynamic shuffle data statistics" ,
169+ statisticsOrRecordTypeInformation ,
170+ new DataStatisticsOperatorFactory <>(
171+ toFlussRowType (tableRowType ),
172+ partitionKeys ,
173+ flussSerializationSchema ))
174+ .uid ("Dynamic shuffle data statistics" + tablePath )
175+ // Set the parallelism same as input operator to encourage
176+ // chaining
177+ .setParallelism (input .getParallelism ());
178+
179+ return partition (
180+ shuffleStream ,
181+ new StatisticsOrRecordChannelComputer <>(
182+ toFlussRowType (tableRowType ),
183+ bucketKeys ,
184+ partitionKeys ,
185+ numBucket ,
186+ lakeFormat ,
187+ flussSerializationSchema ),
188+ input .getParallelism ())
189+ .flatMap (
190+ (FlatMapFunction <StatisticsOrRecord <InputT >, InputT >)
191+ (statisticsOrRecord , out ) -> {
192+ if (statisticsOrRecord .hasRecord ()) {
193+ out .collect (statisticsOrRecord .record ());
194+ }
195+ })
196+ .uid ("flat map" + tablePath )
197+ // To promote operator chaining with the downstream writer operator,
198+ // setting slot sharing group and the parallelism as default, {@link
199+ // SinkTransformationTranslator} will set the parallelism same as sink
200+ // transformation.
201+ .slotSharingGroup ("shuffle-partition-custom-group" )
202+ .returns (rowTypeInformation );
203+
204+ default :
205+ throw new UnsupportedOperationException (
206+ "Unsupported distribution mode: " + shuffleMode );
140207 }
141208 }
142209 }
@@ -155,7 +222,7 @@ static class UpsertSinkWriterBuilder<InputT>
155222 private final List <String > bucketKeys ;
156223 private final List <String > partitionKeys ;
157224 private final @ Nullable DataLakeFormat lakeFormat ;
158- private final boolean shuffleByBucketId ;
225+ private final DistributionMode shuffleMode ;
159226 private final FlussSerializationSchema <InputT > flussSerializationSchema ;
160227
161228 UpsertSinkWriterBuilder (
@@ -167,7 +234,7 @@ static class UpsertSinkWriterBuilder<InputT>
167234 List <String > bucketKeys ,
168235 List <String > partitionKeys ,
169236 @ Nullable DataLakeFormat lakeFormat ,
170- boolean shuffleByBucketId ,
237+ DistributionMode shuffleMode ,
171238 FlussSerializationSchema <InputT > flussSerializationSchema ) {
172239 this .tablePath = tablePath ;
173240 this .flussConfig = flussConfig ;
@@ -177,7 +244,7 @@ static class UpsertSinkWriterBuilder<InputT>
177244 this .bucketKeys = bucketKeys ;
178245 this .partitionKeys = partitionKeys ;
179246 this .lakeFormat = lakeFormat ;
180- this .shuffleByBucketId = shuffleByBucketId ;
247+ this .shuffleMode = shuffleMode ;
181248 this .flussSerializationSchema = flussSerializationSchema ;
182249 }
183250
@@ -194,8 +261,9 @@ public UpsertSinkWriter<InputT> createWriter(MailboxExecutor mailboxExecutor) {
194261
195262 @ Override
196263 public DataStream <InputT > addPreWriteTopology (DataStream <InputT > input ) {
197- return shuffleByBucketId
198- ? partition (
264+ switch (shuffleMode ) {
265+ case BUCKET_SHUFFLE :
266+ return partition (
199267 input ,
200268 new FlinkRowDataChannelComputer <>(
201269 toFlussRowType (tableRowType ),
@@ -204,8 +272,13 @@ public DataStream<InputT> addPreWriteTopology(DataStream<InputT> input) {
204272 lakeFormat ,
205273 numBucket ,
206274 flussSerializationSchema ),
207- input .getParallelism ())
208- : input ;
275+ input .getParallelism ());
276+ case NONE :
277+ return input ;
278+ default :
279+ throw new UnsupportedOperationException (
280+ "Unsupported distribution mode: " + shuffleMode );
281+ }
209282 }
210283 }
211284}
0 commit comments