Skip to content

Commit

Permalink
Flink: Add null check to writers to prevent resurrecting null values
Browse files Browse the repository at this point in the history
Flink's BinaryRowData uses a magic byte to indicate null values in the backing
byte arrays. Flink's internal RowData#createFieldGetter method which Iceberg
uses, only adds a null check whenever a type is nullable. We map Iceberg's
optional attribute to nullable, but Iceberg's required attribute to
non-nullable. The latter creates an issue when the user, by mistake, nulls a
field. The resulting RowData field will then be interpreted as actual data
because the null field is not checked. This yields random values which should
have been null and produced an error in the writer.

The solution is to always check if a field is nullable before attempting to read
data from it.
  • Loading branch information
mxm committed Jan 31, 2025
1 parent bdcd9c3 commit 42c1b86
Show file tree
Hide file tree
Showing 20 changed files with 204 additions and 21 deletions.
22 changes: 22 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 Down Expand Up @@ -486,4 +490,22 @@ public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defau

writeAndValidate(writeSchema, readSchema);
}

@Test
public void testWriteNullValueForRequiredType() {
Assumptions.assumeThat(supportsDefaultValues()).isTrue();

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);

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 @@ -61,6 +61,11 @@ protected void writeAndValidate(Schema schema) throws IOException {
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
Expand Up @@ -250,4 +250,9 @@ protected void writeAndValidate(Schema schema) throws IOException {
protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException {
writeAndValidate(RandomGenericData.generate(writeSchema, 100, 0L), writeSchema, expectedSchema);
}

@Override
protected void writeAndValidate(Schema schema, List<Record> expectedData) throws IOException {
writeAndValidate(expectedData, schema, schema);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -250,4 +250,9 @@ protected void writeAndValidate(Schema schema) throws IOException {
protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException {
writeAndValidate(RandomGenericData.generate(writeSchema, 100, 0L), writeSchema, expectedSchema);
}

@Override
protected void writeAndValidate(Schema schema, List<Record> expectedData) throws IOException {
writeAndValidate(expectedData, schema, schema);
}
}
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);
};
}
}
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 @@ -200,7 +200,7 @@ public static void assertRowData(
Object actual =
actualRowData.isNullAt(i)
? null
: RowData.createFieldGetter(logicalType, i).getFieldOrNull(actualRowData);
: FlinkRowData.createFieldGetter(logicalType, i).getFieldOrNull(actualRowData);
assertEquals(types.get(i), logicalType, expected, actual);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@
import org.apache.iceberg.data.GenericRecord;
import org.apache.iceberg.data.RandomGenericData;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.data.avro.DataReader;
import org.apache.iceberg.data.avro.DataWriter;
import org.apache.iceberg.data.avro.PlannedDataReader;
import org.apache.iceberg.flink.FlinkSchemaUtil;
import org.apache.iceberg.flink.TestHelpers;
import org.apache.iceberg.io.CloseableIterable;
Expand Down 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 @@ -116,11 +116,11 @@ private void writeAndValidate(Schema schema, List<Record> expectedRecords, int n
try (CloseableIterable<Record> reader =
Avro.read(Files.localInput(rowDataFile))
.project(schema)
.createResolvingReader(PlannedDataReader::create)
.createReaderFunc(DataReader::create)
.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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -45,8 +45,13 @@ public class TestFlinkOrcReaderWriter extends DataTest {

@Override
protected void writeAndValidate(Schema schema) throws IOException {
RowType flinkSchema = FlinkSchemaUtil.convert(schema);
List<Record> expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1990L);
writeAndValidate(schema, expectedRecords);
}

@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));

File recordsFile = File.createTempFile("junit", null, temp.toFile());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -250,4 +250,9 @@ protected void writeAndValidate(Schema schema) throws IOException {
protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException {
writeAndValidate(RandomGenericData.generate(writeSchema, 100, 0L), writeSchema, expectedSchema);
}

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

0 comments on commit 42c1b86

Please sign in to comment.