Skip to content

Commit b502e5b

Browse files
committed
fix
1 parent c191346 commit b502e5b

File tree

2 files changed

+8
-31
lines changed

2 files changed

+8
-31
lines changed

paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -148,7 +148,7 @@ public void testSnapshotsTableWithRecordCount() throws Exception {
148148
sql(
149149
"SELECT snapshot_id, total_record_count, delta_record_count, changelog_record_count FROM T$snapshots");
150150
assertThat(result)
151-
.containsExactlyInAnyOrder(Row.of(1L, 1L, 1L, 0L), Row.of(2L, 2L, 1L, 0L));
151+
.containsExactlyInAnyOrder(Row.of(1L, 1L, 1L, null), Row.of(2L, 2L, 1L, null));
152152
}
153153

154154
@Test

paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java

Lines changed: 7 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@
2727
import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList;
2828

2929
import org.apache.flink.table.api.StatementSet;
30-
import org.apache.flink.table.api.ValidationException;
3130
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
3231
import org.apache.flink.types.Row;
3332
import org.apache.flink.types.RowKind;
@@ -227,7 +226,7 @@ public void testContinuousLatest() throws Exception {
227226

228227
BlockingIterator<Row, Row> iterator =
229228
BlockingIterator.of(
230-
streamSqlIter("SELECT * FROM T1 /*+ OPTIONS('log.scan'='latest') */"));
229+
streamSqlIter("SELECT * FROM T1 /*+ OPTIONS('scan.mode'='latest') */"));
231230

232231
batchSql("INSERT INTO T1 VALUES ('7', '8', '9'), ('10', '11', '12')");
233232
assertThat(iterator.collect(2))
@@ -255,7 +254,7 @@ public void testContinuousLatestStartingFromEmpty() throws Exception {
255254
@Test
256255
public void testContinuousFromTimestamp() throws Exception {
257256
String sql =
258-
"SELECT * FROM T1 /*+ OPTIONS('log.scan'='from-timestamp', 'log.scan.timestamp-millis'='%s') */";
257+
"SELECT * FROM T1 /*+ OPTIONS('scan.mode'='from-timestamp', 'scan.timestamp-millis'='%s') */";
259258

260259
// empty table
261260
BlockingIterator<Row, Row> iterator = BlockingIterator.of(streamSqlIter(sql, 0));
@@ -323,31 +322,31 @@ public void testLackStartupTimestamp() {
323322
assertThatThrownBy(
324323
() ->
325324
streamSqlIter(
326-
"SELECT * FROM T1 /*+ OPTIONS('log.scan'='from-timestamp') */"))
325+
"SELECT * FROM T1 /*+ OPTIONS('scan.mode'='from-timestamp') */"))
327326
.hasCauseInstanceOf(IllegalArgumentException.class)
328327
.hasRootCauseMessage(
329328
"must set only one key in [scan.timestamp-millis,scan.timestamp] when you use from-timestamp for scan.mode");
330329
}
331330

332331
@Test
333332
public void testConfigureStartupTimestamp() throws Exception {
334-
// Configure 'log.scan.timestamp-millis' without 'log.scan'.
333+
// Configure 'scan.timestamp-millis' without 'scan.mode'.
335334
BlockingIterator<Row, Row> iterator =
336335
BlockingIterator.of(
337336
streamSqlIter(
338-
"SELECT * FROM T1 /*+ OPTIONS('log.scan.timestamp-millis'='%s') */",
337+
"SELECT * FROM T1 /*+ OPTIONS('scan.timestamp-millis'='%s') */",
339338
0));
340339
batchSql("INSERT INTO T1 VALUES ('1', '2', '3'), ('4', '5', '6')");
341340
batchSql("INSERT INTO T1 VALUES ('7', '8', '9'), ('10', '11', '12')");
342341
assertThat(iterator.collect(2))
343342
.containsExactlyInAnyOrder(Row.of("1", "2", "3"), Row.of("4", "5", "6"));
344343
iterator.close();
345344

346-
// Configure 'log.scan.timestamp-millis' with 'log.scan=latest'.
345+
// Configure 'scan.timestamp-millis' with 'scan.mode=latest'.
347346
assertThatThrownBy(
348347
() ->
349348
streamSqlIter(
350-
"SELECT * FROM T1 /*+ OPTIONS('log.scan'='latest', 'log.scan.timestamp-millis'='%s') */",
349+
"SELECT * FROM T1 /*+ OPTIONS('scan.mode'='latest', 'scan.timestamp-millis'='%s') */",
351350
0))
352351
.hasCauseInstanceOf(IllegalArgumentException.class)
353352
.hasRootCauseMessage(
@@ -466,28 +465,6 @@ public void testIgnoreOverwrite() throws Exception {
466465
iterator.close();
467466
}
468467

469-
@Test
470-
public void testUnsupportedUpsert() {
471-
assertThatThrownBy(
472-
() ->
473-
streamSqlIter(
474-
"SELECT * FROM T1 /*+ OPTIONS('log.changelog-mode'='upsert') */"))
475-
.hasCauseInstanceOf(ValidationException.class)
476-
.hasRootCauseMessage(
477-
"File store continuous reading does not support upsert changelog mode.");
478-
}
479-
480-
@Test
481-
public void testUnsupportedEventual() {
482-
assertThatThrownBy(
483-
() ->
484-
streamSqlIter(
485-
"SELECT * FROM T1 /*+ OPTIONS('log.consistency'='eventual') */"))
486-
.hasCauseInstanceOf(ValidationException.class)
487-
.hasRootCauseMessage(
488-
"File store continuous reading does not support eventual consistency mode.");
489-
}
490-
491468
@Test
492469
public void testFlinkMemoryPool() {
493470
// Check if the configuration is effective

0 commit comments

Comments
 (0)