Skip to content
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: Add null check to writers to prevent resurrecting null values #12049

Merged
merged 2 commits into from
Feb 5, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
28 changes: 28 additions & 0 deletions data/src/test/java/org/apache/iceberg/data/DataTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -27,10 +27,12 @@
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.nio.file.Path;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Stream;
import org.apache.iceberg.Schema;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.types.Types;
Expand All @@ -50,6 +52,8 @@ public abstract class DataTest {

protected abstract void writeAndValidate(Schema schema) throws IOException;

protected abstract void writeAndValidate(Schema schema, List<Record> data) throws IOException;

protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException {
throw new UnsupportedEncodingException(
"Cannot run test, writeAndValidate(Schema, Schema) is not implemented");
Expand All @@ -59,6 +63,10 @@ protected boolean supportsDefaultValues() {
return false;
}

protected boolean allowsWritingNullValuesForRequiredFields() {
return false;
}

protected static final StructType SUPPORTED_PRIMITIVES =
StructType.of(
required(100, "id", LongType.get()),
Expand Down Expand Up @@ -486,4 +494,24 @@ public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defau

writeAndValidate(writeSchema, readSchema);
}

@Test
public void testWriteNullValueForRequiredType() throws Exception {
Schema schema =
new Schema(
required(0, "id", LongType.get()), required(1, "string", Types.StringType.get()));

GenericRecord genericRecord = GenericRecord.create(schema);
genericRecord.set(0, 42L);
genericRecord.set(1, null);

if (allowsWritingNullValuesForRequiredFields()) {
writeAndValidate(schema, ImmutableList.of(genericRecord));
} else {
assertThatThrownBy(
// The actual exception depends on the implementation, e.g.
// NullPointerException or IllegalArgumentException.
() -> writeAndValidate(schema, ImmutableList.of(genericRecord)));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,14 +35,26 @@
import org.apache.iceberg.relocated.com.google.common.collect.Lists;

public class TestGenericData extends DataTest {

@Override
protected void writeAndValidate(Schema schema) throws IOException {
writeAndValidate(schema, schema);
}

@Override
protected void writeAndValidate(Schema writeSchema, List<Record> expectedData)
throws IOException {
writeAndValidate(writeSchema, writeSchema, expectedData);
}

@Override
protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException {
List<Record> expected = RandomGenericData.generate(writeSchema, 100, 0L);
List<Record> data = RandomGenericData.generate(writeSchema, 100, 0L);
writeAndValidate(writeSchema, expectedSchema, data);
}

private void writeAndValidate(Schema writeSchema, Schema expectedSchema, List<Record> expected)
throws IOException {

File testFile = File.createTempFile("junit", null, temp.toFile());
assertThat(testFile.delete()).isTrue();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,13 +54,24 @@

public class TestGenericData extends DataTest {

/** Orc writers don't have notion of non-null / required fields. */
@Override
protected boolean allowsWritingNullValuesForRequiredFields() {
return true;
}

@Override
protected void writeAndValidate(Schema schema) throws IOException {
List<Record> expected = RandomGenericData.generate(schema, 100, 0L);

writeAndValidateRecords(schema, expected);
}

@Override
protected void writeAndValidate(Schema schema, List<Record> expectedData) throws IOException {
writeAndValidateRecords(schema, expectedData);
}

@Test
public void writeAndValidateRepeatingRecords() throws IOException {
Schema structSchema =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,14 +46,25 @@
import org.junit.jupiter.api.Test;

public class TestGenericData extends DataTest {

@Override
protected void writeAndValidate(Schema schema) throws IOException {
writeAndValidate(schema, schema);
}

@Override
protected void writeAndValidate(Schema schema, List<Record> expected) throws IOException {
writeAndValidate(schema, schema, expected);
}

@Override
protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException {
List<Record> expected = RandomGenericData.generate(writeSchema, 100, 12228L);
List<Record> data = RandomGenericData.generate(writeSchema, 100, 12228L);
writeAndValidate(writeSchema, expectedSchema, data);
}

private void writeAndValidate(Schema writeSchema, Schema expectedSchema, List<Record> expected)
throws IOException {

File testFile = File.createTempFile("junit", null, temp.toFile());
assertThat(testFile.delete()).isTrue();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,11 @@ public class TestParquetEncryptionWithWriteSupport extends DataTest {
@Override
protected void writeAndValidate(Schema schema) throws IOException {
List<Record> expected = RandomGenericData.generate(schema, 100, 0L);
writeAndValidate(schema, expected);
}

@Override
protected void writeAndValidate(Schema schema, List<Record> expected) throws IOException {

File testFile = File.createTempFile("junit", null, temp.toFile());
assertThat(testFile.delete()).isTrue();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iceberg.flink;

import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.LogicalType;

public class FlinkRowData {

private FlinkRowData() {}

public static RowData.FieldGetter createFieldGetter(LogicalType fieldType, int fieldPos) {
RowData.FieldGetter flinkFieldGetter = RowData.createFieldGetter(fieldType, fieldPos);
return rowData -> {
// Be sure to check for null values, even if the field is required. Flink
// RowData.createFieldGetter(..) does not null-check optional / nullable types. Without this
// explicit null check, the null flag of BinaryRowData will be ignored and random bytes will
// be parsed as actual values. This will produce incorrect writes instead of failing with a
// NullPointerException.
if (!fieldType.isNullable() && rowData.isNullAt(fieldPos)) {
return null;
}
return flinkFieldGetter.getFieldOrNull(rowData);
};
}
stevenzwu marked this conversation as resolved.
Show resolved Hide resolved
}
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ public <T> T get(int pos, Class<T> javaClass) {
return javaClass.cast(getters[pos].get(rowData, pos));
}

Object value = RowData.createFieldGetter(types[pos], pos).getFieldOrNull(rowData);
Object value = FlinkRowData.createFieldGetter(types[pos], pos).getFieldOrNull(rowData);
return javaClass.cast(value);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.iceberg.FieldMetrics;
import org.apache.iceberg.data.orc.GenericOrcWriters;
import org.apache.iceberg.flink.FlinkRowData;
import org.apache.iceberg.orc.OrcValueWriter;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
Expand Down Expand Up @@ -298,7 +299,7 @@ static class RowDataWriter extends GenericOrcWriters.StructWriter<RowData> {

this.fieldGetters = Lists.newArrayListWithExpectedSize(types.size());
for (int i = 0; i < types.size(); i++) {
fieldGetters.add(RowData.createFieldGetter(types.get(i), i));
fieldGetters.add(FlinkRowData.createFieldGetter(types.get(i), i));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.flink.table.types.logical.RowType.RowField;
import org.apache.flink.table.types.logical.SmallIntType;
import org.apache.flink.table.types.logical.TinyIntType;
import org.apache.iceberg.flink.FlinkRowData;
import org.apache.iceberg.parquet.ParquetValueReaders;
import org.apache.iceberg.parquet.ParquetValueWriter;
import org.apache.iceberg.parquet.ParquetValueWriters;
Expand Down Expand Up @@ -492,7 +493,7 @@ private static class RowDataWriter extends ParquetValueWriters.StructWriter<RowD
super(writers);
fieldGetter = new RowData.FieldGetter[types.size()];
for (int i = 0; i < types.size(); i += 1) {
fieldGetter[i] = RowData.createFieldGetter(types.get(i), i);
fieldGetter[i] = FlinkRowData.createFieldGetter(types.get(i), i);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.flink.table.data.TimestampData;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.iceberg.avro.ValueWriter;
import org.apache.iceberg.flink.FlinkRowData;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.util.DecimalUtil;

Expand Down Expand Up @@ -229,7 +230,7 @@ private RowWriter(List<ValueWriter<?>> writers, List<LogicalType> types) {
this.getters = new RowData.FieldGetter[writers.size()];
for (int i = 0; i < writers.size(); i += 1) {
this.writers[i] = writers.get(i);
this.getters[i] = RowData.createFieldGetter(types.get(i), i);
this.getters[i] = FlinkRowData.createFieldGetter(types.get(i), i);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.flink.types.RowKind;
import org.apache.flink.util.StringUtils;
import org.apache.iceberg.Schema;
import org.apache.iceberg.flink.FlinkRowData;
import org.apache.iceberg.flink.FlinkSchemaUtil;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
Expand Down Expand Up @@ -135,7 +136,7 @@ private static RowData.FieldGetter createFieldGetter(
projectField,
rowField);

return RowData.createFieldGetter(rowType.getTypeAt(position), position);
return FlinkRowData.createFieldGetter(rowType.getTypeAt(position), position);

case LIST:
Types.ListType projectedList = projectField.type().asListType();
Expand All @@ -150,10 +151,10 @@ private static RowData.FieldGetter createFieldGetter(
projectField,
rowField);

return RowData.createFieldGetter(rowType.getTypeAt(position), position);
return FlinkRowData.createFieldGetter(rowType.getTypeAt(position), position);

default:
return RowData.createFieldGetter(rowType.getTypeAt(position), position);
return FlinkRowData.createFieldGetter(rowType.getTypeAt(position), position);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.runtime.typeutils.InternalSerializers;
import org.apache.flink.table.types.logical.RowType;
import org.apache.iceberg.flink.FlinkRowData;
import org.apache.iceberg.flink.data.RowDataUtil;

class RowDataRecordFactory implements RecordFactory<RowData> {
Expand All @@ -45,7 +46,7 @@ static TypeSerializer[] createFieldSerializers(RowType rowType) {
static RowData.FieldGetter[] createFieldGetters(RowType rowType) {
RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()];
for (int i = 0; i < rowType.getFieldCount(); ++i) {
fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i);
fieldGetters[i] = FlinkRowData.createFieldGetter(rowType.getTypeAt(i), i);
}

return fieldGetters;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ public static RowData copyRowData(RowData from, RowType rowType) {
.toArray(TypeSerializer[]::new);
RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()];
for (int i = 0; i < rowType.getFieldCount(); ++i) {
fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i);
fieldGetters[i] = FlinkRowData.createFieldGetter(rowType.getTypeAt(i), i);
}

return RowDataUtil.clone(from, null, rowType, fieldSerializers, fieldGetters);
Expand Down Expand Up @@ -190,17 +190,7 @@ public static void assertRowData(
for (int i = 0; i < types.size(); i += 1) {
LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
Object expected = expectedRecord.get(i, Object.class);
// The RowData.createFieldGetter won't return null for the required field. But in the
// projection case, if we are
// projecting a nested required field from an optional struct, then we should give a null for
// the projected field
// if the outer struct value is null. So we need to check the nullable for actualRowData here.
// For more details
// please see issue #2738.
Object actual =
actualRowData.isNullAt(i)
? null
: RowData.createFieldGetter(logicalType, i).getFieldOrNull(actualRowData);
Object actual = FlinkRowData.createFieldGetter(logicalType, i).getFieldOrNull(actualRowData);
assertEquals(types.get(i), logicalType, expected, actual);
}
}
Expand Down Expand Up @@ -602,7 +592,7 @@ public static void assertEquals(ContentFile<?> expected, ContentFile<?> actual)
assertThat(actual).isNotNull();
assertThat(actual.specId()).as("SpecId").isEqualTo(expected.specId());
assertThat(actual.content()).as("Content").isEqualTo(expected.content());
assertThat(actual.path()).as("Path").isEqualTo(expected.path());
assertThat(actual.location()).as("Location").isEqualTo(expected.location());
assertThat(actual.format()).as("Format").isEqualTo(expected.format());
assertThat(actual.partition().size())
.as("Partition size")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,13 +67,13 @@ public abstract class AbstractTestFlinkAvroReaderWriter extends DataTest {
@Override
protected void writeAndValidate(Schema schema) throws IOException {
List<Record> expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1991L);
writeAndValidate(schema, expectedRecords, NUM_RECORDS);
writeAndValidate(schema, expectedRecords);
}

protected abstract Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema);

private void writeAndValidate(Schema schema, List<Record> expectedRecords, int numRecord)
throws IOException {
@Override
protected void writeAndValidate(Schema schema, List<Record> expectedRecords) throws IOException {
RowType flinkSchema = FlinkSchemaUtil.convert(schema);
List<RowData> expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords));

Expand All @@ -93,7 +93,7 @@ private void writeAndValidate(Schema schema, List<Record> expectedRecords, int n
try (CloseableIterable<RowData> reader = createAvroReadBuilder(recordsFile, schema).build()) {
Iterator<Record> expected = expectedRecords.iterator();
Iterator<RowData> rows = reader.iterator();
for (int i = 0; i < numRecord; i++) {
for (int i = 0; i < expectedRecords.size(); i++) {
assertThat(rows).hasNext();
TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next());
}
Expand All @@ -120,7 +120,7 @@ private void writeAndValidate(Schema schema, List<Record> expectedRecords, int n
.build()) {
Iterator<RowData> expected = expectedRows.iterator();
Iterator<Record> records = reader.iterator();
for (int i = 0; i < numRecord; i += 1) {
for (int i = 0; i < expectedRecords.size(); i += 1) {
assertThat(records).hasNext();
TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next());
}
Expand Down Expand Up @@ -177,6 +177,6 @@ public void testNumericTypes() throws IOException {
1643811742000L,
10.24d));

writeAndValidate(SCHEMA_NUM_TYPE, expected, 2);
writeAndValidate(SCHEMA_NUM_TYPE, expected);
}
}
Loading