-
Notifications
You must be signed in to change notification settings - Fork 1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[flink] fix delete record exception after partial-update merge engine configuration to lookup changelog producer #4345
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -120,7 +120,6 @@ public void add(KeyValue kv) { | |
currentDeleteRow = true; | ||
row = new GenericRow(getters.length); | ||
} | ||
// ignore -U records | ||
return; | ||
} | ||
|
||
|
@@ -254,10 +253,9 @@ public KeyValue getResult() { | |
if (reused == null) { | ||
reused = new KeyValue(); | ||
} | ||
if (currentDeleteRow) { | ||
return null; | ||
} | ||
return reused.replace(currentKey, latestSequenceNumber, RowKind.INSERT, row); | ||
|
||
RowKind rowKind = currentDeleteRow ? RowKind.DELETE : RowKind.INSERT; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why here don't return null could avoid NPE in : There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Before the change, it returned null. exception: Caused by: java.io.IOException: java.util.concurrent.ExecutionException: java.lang.NullPointerException |
||
return reused.replace(currentKey, latestSequenceNumber, rowKind, row); | ||
} | ||
|
||
public static MergeFunctionFactory<KeyValue> factory( | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -623,11 +623,12 @@ public void testIgnoreDelete(boolean localMerge) throws Exception { | |
} | ||
|
||
@Test | ||
public void testRemoveRecordOnDelete() { | ||
public void testRemoveRecordOnDelete() throws Exception { | ||
sql( | ||
"CREATE TABLE remove_record_on_delete (pk INT PRIMARY KEY NOT ENFORCED, a STRING, b STRING) WITH (" | ||
+ " 'merge-engine' = 'partial-update'," | ||
+ " 'partial-update.remove-record-on-delete' = 'true'" | ||
+ " 'partial-update.remove-record-on-delete' = 'true'," | ||
+ " 'changelog-producer' = 'lookup'" | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can you add a new case for lookup? |
||
+ ")"); | ||
|
||
sql("INSERT INTO remove_record_on_delete VALUES (1, CAST (NULL AS STRING), 'apple')"); | ||
|
@@ -645,5 +646,18 @@ public void testRemoveRecordOnDelete() { | |
// batch read | ||
assertThat(sql("SELECT * FROM remove_record_on_delete")) | ||
.containsExactlyInAnyOrder(Row.of(1, "A", "apache")); | ||
|
||
// streaming read results has -U | ||
BlockingIterator<Row, Row> iterator = | ||
streamSqlBlockIter( | ||
"SELECT * FROM remove_record_on_delete /*+ OPTIONS('scan.timestamp-millis' = '0') */"); | ||
assertThat(iterator.collect(5)) | ||
.containsExactly( | ||
Row.ofKind(RowKind.INSERT, 1, null, "apple"), | ||
Row.ofKind(RowKind.DELETE, 1, null, "apple"), | ||
Row.ofKind(RowKind.INSERT, 1, null, "apache"), | ||
Row.ofKind(RowKind.UPDATE_BEFORE, 1, null, "apache"), | ||
Row.ofKind(RowKind.UPDATE_AFTER, 1, "A", "apache")); | ||
iterator.close(); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why delete this?