[HUDI-1738] Emit deletes for flink MOR table streaming read (#2742)
Current we did a soft delete for DELETE row data when writes into hoodie table. For streaming read of MOR table, the Flink reader detects the delete records and still emit them if the record key semantics are still kept. This is useful and actually a must for streaming ETL pipeline incremental computation.
This commit is contained in:
@@ -137,7 +137,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpointCompleteWithRetry() {
|
||||
public void testCheckpointCompleteWithException() {
|
||||
final CompletableFuture<byte[]> future = new CompletableFuture<>();
|
||||
coordinator.checkpointCoordinator(1, future);
|
||||
String inflightInstant = coordinator.getInstant();
|
||||
@@ -149,7 +149,9 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
coordinator.handleEventFromOperator(0, event);
|
||||
assertThrows(HoodieException.class,
|
||||
() -> coordinator.notifyCheckpointComplete(1),
|
||||
"Try 3 to commit instant");
|
||||
"org.apache.hudi.exception.HoodieException: Instant [20210330153432] has a complete checkpoint [1],\n"
|
||||
+ "but the coordinator has not received full write success events,\n"
|
||||
+ "rolls back the instant and rethrow");
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -260,15 +260,17 @@ public class TestStreamReadOperator {
|
||||
TestConfigurations.ROW_TYPE,
|
||||
tableAvroSchema.toString(),
|
||||
AvroSchemaConverter.convertToSchema(TestConfigurations.ROW_TYPE).toString(),
|
||||
Collections.emptyList());
|
||||
Collections.emptyList(),
|
||||
new String[0]);
|
||||
Path[] paths = FilePathUtils.getReadPaths(new Path(basePath), conf, hadoopConf, partitionKeys);
|
||||
MergeOnReadInputFormat inputFormat = new MergeOnReadInputFormat(
|
||||
conf,
|
||||
FilePathUtils.toFlinkPaths(paths),
|
||||
hoodieTableState,
|
||||
rowDataType.getChildren(),
|
||||
"default",
|
||||
1000L);
|
||||
MergeOnReadInputFormat inputFormat = MergeOnReadInputFormat.builder()
|
||||
.config(conf)
|
||||
.paths(FilePathUtils.toFlinkPaths(paths))
|
||||
.tableState(hoodieTableState)
|
||||
.fieldTypes(rowDataType.getChildren())
|
||||
.defaultPartName("default").limit(1000L)
|
||||
.emitDelete(true)
|
||||
.build();
|
||||
|
||||
OneInputStreamOperatorFactory<MergeOnReadInputSplit, RowData> factory = StreamReadOperator.factory(inputFormat);
|
||||
OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness = new OneInputStreamOperatorTestHarness<>(
|
||||
|
||||
@@ -208,6 +208,41 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
"some commits should be cleaned");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testStreamReadWithDeletes() throws Exception {
|
||||
// create filesystem table named source
|
||||
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.TABLE_NAME, "t1");
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
|
||||
// write one commit
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
// write another commit with deletes
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_DELETE, conf);
|
||||
|
||||
String latestCommit = StreamerUtil.createWriteClient(conf, null)
|
||||
.getLastCompletedInstant(FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
|
||||
options.put(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
options.put(FlinkOptions.READ_AS_STREAMING.key(), "true");
|
||||
options.put(FlinkOptions.READ_STREAMING_CHECK_INTERVAL.key(), "2");
|
||||
options.put(FlinkOptions.READ_STREAMING_START_COMMIT.key(), latestCommit);
|
||||
String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
|
||||
streamTableEnv.executeSql(hoodieTableDDL);
|
||||
|
||||
List<Row> result = execSelectSql(streamTableEnv, "select * from t1", 10);
|
||||
final String expected = "["
|
||||
+ "id1,Danny,24,1970-01-01T00:00:00.001,par1, "
|
||||
+ "id2,Stephen,34,1970-01-01T00:00:00.002,par1, "
|
||||
+ "id3,null,null,null,null, "
|
||||
+ "id5,null,null,null,null, "
|
||||
+ "id9,null,null,null,null]";
|
||||
assertRowsEquals(result, expected);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = ExecMode.class)
|
||||
void testWriteAndRead(ExecMode execMode) {
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.table.format;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.table.HoodieTableSource;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
@@ -43,6 +44,7 @@ import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
|
||||
@@ -151,6 +153,29 @@ public class TestInputFormat {
|
||||
assertThat(actual, is(expected));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReadWithDeletes() throws Exception {
|
||||
beforeEach(HoodieTableType.MERGE_ON_READ);
|
||||
|
||||
// write another commit to read again
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_DELETE, conf);
|
||||
|
||||
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat, instanceOf(MergeOnReadInputFormat.class));
|
||||
((MergeOnReadInputFormat) inputFormat).isEmitDelete(true);
|
||||
|
||||
List<RowData> result = readData(inputFormat);
|
||||
|
||||
final String actual = TestData.rowDataToString(result);
|
||||
final String expected = "["
|
||||
+ "id1,Danny,24,1970-01-01T00:00:00.001,par1, "
|
||||
+ "id2,Stephen,34,1970-01-01T00:00:00.002,par1, "
|
||||
+ "id3,null,null,null,null, "
|
||||
+ "id5,null,null,null,null, "
|
||||
+ "id9,null,null,null,null]";
|
||||
assertThat(actual, is(expected));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = HoodieTableType.class)
|
||||
void testReadWithPartitionPrune(HoodieTableType tableType) throws Exception {
|
||||
|
||||
@@ -224,6 +224,10 @@ public class TestData {
|
||||
funcWrapper.close();
|
||||
}
|
||||
|
||||
private static String toStringSafely(Object obj) {
|
||||
return obj == null ? "null" : obj.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the {@code rows} using field at index 0 and asserts
|
||||
* it equals with the expected string {@code expected}.
|
||||
@@ -233,7 +237,7 @@ public class TestData {
|
||||
*/
|
||||
public static void assertRowsEquals(List<Row> rows, String expected) {
|
||||
String rowsString = rows.stream()
|
||||
.sorted(Comparator.comparing(o -> o.getField(0).toString()))
|
||||
.sorted(Comparator.comparing(o -> toStringSafely(o.getField(0))))
|
||||
.collect(Collectors.toList()).toString();
|
||||
assertThat(rowsString, is(expected));
|
||||
}
|
||||
@@ -247,7 +251,7 @@ public class TestData {
|
||||
*/
|
||||
public static void assertRowsEquals(List<Row> rows, List<RowData> expected) {
|
||||
String rowsString = rows.stream()
|
||||
.sorted(Comparator.comparing(o -> o.getField(0).toString()))
|
||||
.sorted(Comparator.comparing(o -> toStringSafely(o.getField(0))))
|
||||
.collect(Collectors.toList()).toString();
|
||||
assertThat(rowsString, is(rowDataToString(expected)));
|
||||
}
|
||||
|
||||
@@ -0,0 +1,107 @@
|
||||
/*
|
||||
* 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.hudi.utils;
|
||||
|
||||
import org.apache.hudi.keygen.KeyGenUtils;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.RowDataToAvroConverters;
|
||||
import org.apache.hudi.util.StringToRowDataConverter;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.data.DecimalData;
|
||||
import org.apache.flink.table.data.GenericRowData;
|
||||
import org.apache.flink.table.data.TimestampData;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalTime;
|
||||
import java.time.temporal.ChronoField;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
|
||||
/**
|
||||
* Test cases for {@link StringToRowDataConverter}.
|
||||
*/
|
||||
public class TestStringToRowDataConverter {
|
||||
@Test
|
||||
void testConvert() {
|
||||
String[] fields = new String[] {"1.1", "3.4", "2021-03-30", "56669000", "1617119069000", "12345.67"};
|
||||
LogicalType[] fieldTypes = new LogicalType[] {
|
||||
DataTypes.FLOAT().getLogicalType(),
|
||||
DataTypes.DOUBLE().getLogicalType(),
|
||||
DataTypes.DATE().getLogicalType(),
|
||||
DataTypes.TIME(3).getLogicalType(),
|
||||
DataTypes.TIMESTAMP().getLogicalType(),
|
||||
DataTypes.DECIMAL(7, 2).getLogicalType()
|
||||
};
|
||||
StringToRowDataConverter converter = new StringToRowDataConverter(fieldTypes);
|
||||
Object[] converted = converter.convert(fields);
|
||||
Object[] expected = new Object[] {
|
||||
1.1f, 3.4D, (int) LocalDate.parse("2021-03-30").toEpochDay(),
|
||||
LocalTime.parse("15:44:29").get(ChronoField.MILLI_OF_DAY),
|
||||
TimestampData.fromEpochMillis(Instant.parse("2021-03-30T15:44:29Z").toEpochMilli()),
|
||||
DecimalData.fromBigDecimal(new BigDecimal("12345.67"), 7, 2)
|
||||
};
|
||||
assertArrayEquals(expected, converted);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRowDataToAvroStringToRowData() {
|
||||
GenericRowData rowData = new GenericRowData(6);
|
||||
rowData.setField(0, 1.1f);
|
||||
rowData.setField(1, 3.4D);
|
||||
rowData.setField(2, (int) LocalDate.parse("2021-03-30").toEpochDay());
|
||||
rowData.setField(3, LocalTime.parse("15:44:29").get(ChronoField.MILLI_OF_DAY));
|
||||
rowData.setField(4, TimestampData.fromEpochMillis(Instant.parse("2021-03-30T15:44:29Z").toEpochMilli()));
|
||||
rowData.setField(5, DecimalData.fromBigDecimal(new BigDecimal("12345.67"), 7, 2));
|
||||
|
||||
DataType dataType = DataTypes.ROW(
|
||||
DataTypes.FIELD("f_float", DataTypes.FLOAT()),
|
||||
DataTypes.FIELD("f_double", DataTypes.DOUBLE()),
|
||||
DataTypes.FIELD("f_date", DataTypes.DATE()),
|
||||
DataTypes.FIELD("f_time", DataTypes.TIME(3)),
|
||||
DataTypes.FIELD("f_timestamp", DataTypes.TIMESTAMP(3)),
|
||||
DataTypes.FIELD("f_decimal", DataTypes.DECIMAL(7, 2))
|
||||
);
|
||||
RowType rowType = (RowType) dataType.getLogicalType();
|
||||
RowDataToAvroConverters.RowDataToAvroConverter converter =
|
||||
RowDataToAvroConverters.createConverter(rowType);
|
||||
GenericRecord avroRecord =
|
||||
(GenericRecord) converter.convert(AvroSchemaConverter.convertToSchema(rowType), rowData);
|
||||
StringToRowDataConverter stringToRowDataConverter =
|
||||
new StringToRowDataConverter(rowType.getChildren().toArray(new LogicalType[0]));
|
||||
final String recordKey = KeyGenUtils.getRecordKey(avroRecord, rowType.getFieldNames());
|
||||
final String[] recordKeys = KeyGenUtils.extractRecordKeys(recordKey);
|
||||
Object[] convertedKeys = stringToRowDataConverter.convert(recordKeys);
|
||||
|
||||
GenericRowData converted = new GenericRowData(6);
|
||||
for (int i = 0; i < 6; i++) {
|
||||
converted.setField(i, convertedKeys[i]);
|
||||
}
|
||||
assertThat(converted, is(rowData));
|
||||
}
|
||||
}
|
||||
@@ -143,14 +143,9 @@ public class CollectSinkTableFactory implements DynamicTableSinkFactory {
|
||||
|
||||
@Override
|
||||
public void invoke(RowData value, SinkFunction.Context context) {
|
||||
if (value.getRowKind() == RowKind.INSERT) {
|
||||
Row row = (Row) converter.toExternal(value);
|
||||
assert row != null;
|
||||
RESULT.get(taskID).add(row);
|
||||
} else {
|
||||
throw new RuntimeException(
|
||||
"CollectSinkFunction received " + value.getRowKind() + " messages.");
|
||||
}
|
||||
Row row = (Row) converter.toExternal(value);
|
||||
assert row != null;
|
||||
RESULT.get(taskID).add(row);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
Reference in New Issue
Block a user