|
33 | 33 | import org.apache.fluss.metadata.TablePath; |
34 | 34 | import org.apache.fluss.row.GenericRow; |
35 | 35 | import org.apache.fluss.row.InternalRow; |
| 36 | +import org.apache.fluss.types.DataTypes; |
36 | 37 | import org.apache.fluss.types.RowType; |
37 | 38 |
|
| 39 | +import org.apache.flink.api.common.eventtime.Watermark; |
| 40 | +import org.apache.flink.api.common.eventtime.WatermarkGenerator; |
| 41 | +import org.apache.flink.api.common.eventtime.WatermarkOutput; |
38 | 42 | import org.apache.flink.api.common.eventtime.WatermarkStrategy; |
| 43 | +import org.apache.flink.api.common.functions.MapFunction; |
| 44 | +import org.apache.flink.api.common.typeinfo.TypeInformation; |
39 | 45 | import org.apache.flink.streaming.api.datastream.DataStreamSource; |
40 | 46 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; |
| 47 | +import org.apache.flink.streaming.api.functions.ProcessFunction; |
| 48 | +import org.apache.flink.streaming.api.operators.StreamMap; |
| 49 | +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; |
41 | 50 | import org.apache.flink.table.data.GenericRowData; |
42 | 51 | import org.apache.flink.table.data.RowData; |
43 | 52 | import org.apache.flink.table.data.StringData; |
44 | 53 | import org.apache.flink.types.RowKind; |
| 54 | +import org.apache.flink.util.Collector; |
45 | 55 | import org.junit.jupiter.api.BeforeEach; |
46 | 56 | import org.junit.jupiter.api.Test; |
47 | 57 |
|
@@ -266,6 +276,149 @@ public void testTableLogSourceWithProjectionPushdown() throws Exception { |
266 | 276 | assertThat(collectedElements).hasSameElementsAs(expectedOutput); |
267 | 277 | } |
268 | 278 |
|
| 279 | + /** Verifies that event-time timestamps are correctly assigned via WatermarkStrategy. */ |
| 280 | + @Test |
| 281 | + void testTimestamp() throws Exception { |
| 282 | + // 1. Create Fluss log table |
| 283 | + String tableName = "wm_timestamp_test"; |
| 284 | + TablePath tablePath = TablePath.of(DEFAULT_DB, tableName); |
| 285 | + Schema schema = |
| 286 | + Schema.newBuilder() |
| 287 | + .column("id", DataTypes.INT()) |
| 288 | + .column("name", DataTypes.STRING()) |
| 289 | + .column("event_time", DataTypes.BIGINT()) |
| 290 | + .build(); |
| 291 | + createTable(tablePath, TableDescriptor.builder().schema(schema).distributedBy(1).build()); |
| 292 | + |
| 293 | + // 2. Write 3 records with known event_time values |
| 294 | + final long currentTimestamp = System.currentTimeMillis(); |
| 295 | + List<InternalRow> rows = |
| 296 | + Arrays.asList( |
| 297 | + row(1, "name1", currentTimestamp + 1L), |
| 298 | + row(2, "name2", currentTimestamp + 2L), |
| 299 | + row(3, "name3", currentTimestamp + 3L)); |
| 300 | + writeRows(conn, tablePath, rows, true); |
| 301 | + |
| 302 | + // 3. Build FlussSource and apply WatermarkStrategy with TimestampAssigner |
| 303 | + FlussSource<RowData> source = |
| 304 | + FlussSource.<RowData>builder() |
| 305 | + .setBootstrapServers(bootstrapServers) |
| 306 | + .setDatabase(DEFAULT_DB) |
| 307 | + .setTable(tableName) |
| 308 | + .setStartingOffsets(OffsetsInitializer.earliest()) |
| 309 | + .setDeserializationSchema(new RowDataDeserializationSchema()) |
| 310 | + .build(); |
| 311 | + |
| 312 | + env.setParallelism(1); |
| 313 | + DataStreamSource<RowData> stream = |
| 314 | + env.fromSource( |
| 315 | + source, |
| 316 | + WatermarkStrategy.<RowData>noWatermarks() |
| 317 | + .withTimestampAssigner( |
| 318 | + (rowData, ts) -> rowData.getLong(2)), // event_time column |
| 319 | + "testTimestamp"); |
| 320 | + |
| 321 | + // Verify that the timestamp and watermark are working fine. |
| 322 | + List<Long> result = |
| 323 | + stream.transform( |
| 324 | + "timestampVerifier", |
| 325 | + TypeInformation.of(Long.class), |
| 326 | + new WatermarkVerifyingOperator(v -> v.getLong(2))) |
| 327 | + .executeAndCollect(3); |
| 328 | + assertThat(result) |
| 329 | + .containsExactlyInAnyOrder( |
| 330 | + currentTimestamp + 1L, currentTimestamp + 2L, currentTimestamp + 3L); |
| 331 | + } |
| 332 | + |
| 333 | + /** Verifies per-bucket (per-split) watermark multiplexing correctness. */ |
| 334 | + @Test |
| 335 | + void testPerBucketWatermark() throws Exception { |
| 336 | + // 1. Create 2-bucket Fluss log table |
| 337 | + String tableName = "wm_per_bucket_test"; |
| 338 | + TablePath tablePath = TablePath.of(DEFAULT_DB, tableName); |
| 339 | + Schema schema = |
| 340 | + Schema.newBuilder() |
| 341 | + .column("id", DataTypes.INT()) |
| 342 | + .column("name", DataTypes.STRING()) |
| 343 | + .column("ts", DataTypes.BIGINT()) |
| 344 | + .build(); |
| 345 | + createTable(tablePath, TableDescriptor.builder().schema(schema).distributedBy(2).build()); |
| 346 | + |
| 347 | + // 2. Write 6 records with interleaved timestamps |
| 348 | + List<InternalRow> rows = |
| 349 | + Arrays.asList( |
| 350 | + row(1, "a", 100L), |
| 351 | + row(2, "b", 150L), |
| 352 | + row(3, "c", 200L), |
| 353 | + row(4, "d", 250L), |
| 354 | + row(5, "e", 300L), |
| 355 | + row(6, "f", 350L)); |
| 356 | + writeRows(conn, tablePath, rows, true); |
| 357 | + |
| 358 | + // 3. Build FlussSource and apply per-split WatermarkStrategy |
| 359 | + FlussSource<RowData> source = |
| 360 | + FlussSource.<RowData>builder() |
| 361 | + .setBootstrapServers(bootstrapServers) |
| 362 | + .setDatabase(DEFAULT_DB) |
| 363 | + .setTable(tableName) |
| 364 | + .setStartingOffsets(OffsetsInitializer.earliest()) |
| 365 | + .setDeserializationSchema(new RowDataDeserializationSchema()) |
| 366 | + .build(); |
| 367 | + |
| 368 | + env.setParallelism(1); |
| 369 | + |
| 370 | + // 4. Assert per-split watermark ordering via ProcessFunction |
| 371 | + env.fromSource( |
| 372 | + source, |
| 373 | + WatermarkStrategy.forGenerator(ctx -> new OnEventWatermarkGenerator()) |
| 374 | + .withTimestampAssigner( |
| 375 | + (rowData, ts) -> rowData.getLong(2)), // ts column |
| 376 | + "testPerPartitionWatermark") |
| 377 | + .process( |
| 378 | + new ProcessFunction<RowData, Object>() { |
| 379 | + @Override |
| 380 | + public void processElement( |
| 381 | + RowData value, |
| 382 | + ProcessFunction<RowData, Object>.Context ctx, |
| 383 | + Collector<Object> out) { |
| 384 | + assertThat(ctx.timestamp()) |
| 385 | + .as( |
| 386 | + "Event time should never behind watermark " |
| 387 | + + "because of per-split watermark multiplexing logic") |
| 388 | + .isGreaterThanOrEqualTo( |
| 389 | + ctx.timerService().currentWatermark()); |
| 390 | + out.collect(ctx.timestamp()); |
| 391 | + } |
| 392 | + }) |
| 393 | + .executeAndCollect(6); |
| 394 | + } |
| 395 | + |
| 396 | + /** A StreamMap that verifies the watermark logic. */ |
| 397 | + private static class WatermarkVerifyingOperator extends StreamMap<RowData, Long> { |
| 398 | + |
| 399 | + private static final long serialVersionUID = 1L; |
| 400 | + |
| 401 | + public WatermarkVerifyingOperator(MapFunction<RowData, Long> mapper) { |
| 402 | + super(mapper); |
| 403 | + } |
| 404 | + |
| 405 | + @Override |
| 406 | + public void processElement(StreamRecord<RowData> element) { |
| 407 | + output.collect(new StreamRecord<>(element.getTimestamp())); |
| 408 | + } |
| 409 | + } |
| 410 | + |
| 411 | + /** A WatermarkGenerator that emits a watermark equal to the event timestamp on each event. */ |
| 412 | + private static class OnEventWatermarkGenerator implements WatermarkGenerator<RowData> { |
| 413 | + @Override |
| 414 | + public void onEvent(RowData event, long eventTimestamp, WatermarkOutput output) { |
| 415 | + output.emitWatermark(new Watermark(eventTimestamp)); |
| 416 | + } |
| 417 | + |
| 418 | + @Override |
| 419 | + public void onPeriodicEmit(WatermarkOutput output) {} |
| 420 | + } |
| 421 | + |
269 | 422 | private static RowData createRowData( |
270 | 423 | Long orderId, Long itemId, Integer amount, String address, RowKind rowKind) { |
271 | 424 | GenericRowData row = new GenericRowData(4); |
|
0 commit comments