Skip to content

Conversation

@openinx
Copy link
Member

@openinx openinx commented Nov 5, 2021

This is trying to address issue: #3431

@github-actions github-actions bot added the flink label Nov 5, 2021
}

@Override
public void setRowPositionSupplier(Supplier<Long> posSupplier) {
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it's a critical bug which we've missed to set the posSupplier to the reader since #1222. But I'm still not sure why this TestFlinkInputFormatReaderDeletes did not cover the avro case.

Let me dig it into deeper and try to provide unit tests in the TestFlinkInputFormatReaderDeletes to address this thing.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Others from Chinese Iceberg group also encountered this bug, they are stacktraces are the following:

Caused by: java.lang.NullPointerException
  at org.apache.iceberg.data.DeleteFilter.pos(DeleteFilter.java:108) ~[iceberg-flink-runtime-0.12.1-20211109.jar:?]
  at org.apache.iceberg.deletes.Deletes$PositionSetDeleteFilter.shouldKeep(Deletes.java:157) ~[iceberg-flink-runtime-0.12.1-20211109.jar:?]
  at org.apache.iceberg.util.Filter$Iterator.shouldKeep(Filter.java:49) ~[iceberg-flink-runtime-0.12.1-20211109.jar:?]
  at org.apache.iceberg.io.FilterIterator.advance(FilterIterator.java:67) ~[iceberg-flink-runtime-0.12.1-20211109.jar:?]
  at org.apache.iceberg.io.FilterIterator.hasNext(FilterIterator.java:50) ~[iceberg-flink-runtime-0.12.1-20211109.jar:?]
  at org.apache.iceberg.io.FilterIterator.advance(FilterIterator.java:65) ~[iceberg-flink-runtime-0.12.1-20211109.jar:?]
  at org.apache.iceberg.io.FilterIterator.hasNext(FilterIterator.java:50) ~[iceberg-flink-runtime-0.12.1-20211109.jar:?]
  at org.apache.iceberg.flink.source.DataIterator.updateCurrentIterator(DataIterator.java:100) ~[iceberg-flink-runtime-0.12.1-20211109.jar:?]
  at org.apache.iceberg.flink.source.DataIterator.hasNext(DataIterator.java:84) ~[iceberg-flink-runtime-0.12.1-20211109.jar:?]
  at org.apache.iceberg.flink.source.FlinkInputFormat.reachedEnd(FlinkInputFormat.java:104) ~[iceberg-flink-runtime-0.12.1-20211109.jar:?]
  at org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction.run(InputFormatSourceFunction.java:89) ~[flink-dist_2.11-1.12.1.jar:1.12.1]
  at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:110) ~[flink-dist_2.11-1.12.1.jar:1.12.1]
  at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:66) ~[flink-dist_2.11-1.12.1.jar:1.12.1]
  at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:241) ~[flink-dist_2.11-1.12.1.jar:1.12.1]

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we separate this fix into other issue? This doesn't seem to be related to the unittest.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, that's exactly what I plan to do. Thanks @Reo-LEI .

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Created separate PR for this: #3540

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Merged #3540.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As the separate PR #3550 has been merged, Let's update this PR.


@Test
public void testV1SkipToRemoveMetaColumn() throws IOException {
testSkipToRemoveMetaColumn(1);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why don't we move formatVersion to test parameters too?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @stevenzwu for the comment. I did not use the parameterized formatVersion approach before because not all of the unit tests are need to be addressed in both v1 and v2 tables. For example, the testV2RemoveMetaColumn should only be checked for v2 tables. If we use the parameterized formatVersion, then there will introduce more unit tests that be ignored because we will usually provide a line in testV2RemoveMetaColumn as : Assume.assumeTrue("xxx", formatVersion==2).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This make sense.

List<RowData> results = Lists.newArrayList();
TestHelpers.readRowData(input, rowData -> {
// If project to remove the meta columns, it will get a RowDataProjection.
Assert.assertTrue(rowData instanceof GenericRowData);
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For the v1 table, we don't need to remove the meta columns because we don't even add them. So all the output rows should be an instance of GenericRowData.

List<RowData> results = Lists.newArrayList();
TestHelpers.readRowData(input, rowData -> {
// If project to remove the meta columns, it will get a RowDataProjection.
Assert.assertTrue(rowData instanceof RowDataProjection);
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For v2 table with eq-deletes or pos-deletes to merge, we will add _pos, file_path, is_deleted meta columns. For flink engine, we will need to project the row data to remove the extra meta columns for users. The RowDataProjection was introduced since here https://github.com/apache/iceberg/pull/3240/files#diff-aed36b23cf7e6110f84bd3b34307240d2174b2be2edb987dfcd22ff15ab75f72R81-R83

Copy link
Contributor

@Reo-LEI Reo-LEI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this PR is look good to me overall. Thanks @openinx add this tests.

// Create the table with given format version.
String location = folder.getRoot().getAbsolutePath();
Table table = SimpleDataUtil.createTable(location,
ImmutableMap.of("format-version", String.valueOf(formatVersion)),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: I think we should use TableProperties.FORMAT_VERSION here.

@openinx
Copy link
Member Author

openinx commented Nov 22, 2021

@rdblue , Is there any chance to take a look at this PR ?

@rdblue
Copy link
Contributor

rdblue commented Nov 22, 2021

Sure, I'll take a look. Can you give me more background on meta-columns and when they are excluded?

@openinx
Copy link
Member Author

openinx commented Nov 23, 2021

@rdblue , I think this conversation is a good material to read for this PR's background: https://github.com/apache/iceberg/pull/3240/files#r736304982

input.open(s);
try {
while (!input.reachedEnd()) {
RowData row = input.nextRecord(null);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: This is a new addition, but can we add an in-line column on the null to indicate that it's reuse like /** resuse **/? I always forget.

@rdblue rdblue merged commit 1778185 into apache:master Dec 10, 2021
Copy link
Contributor

@kbendick kbendick left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This LGTM

@rdblue
Copy link
Contributor

rdblue commented Dec 10, 2021

Thanks, @openinx! I think we also need to port these tests to 1.14 as well.

@kbendick
Copy link
Contributor

Oh woops. I clicked request changes when I meant to click approve. Sorry about that! It was merged anyway 🙂

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants