Skip to content
Closed
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
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,30 @@ public static String[] extractRecordKeys(String recordKey) {
}).toArray(String[]::new);
}

/**
* Extracts the partition fields in strings out of the given partitionPath,
* this is the reverse operation of {@link #getPartitionPath(GenericRecord record, String partitionPathField,
* boolean hiveStylePartitioning, boolean encodePartitionPath, boolean consistentLogicalTimestampEnabled)}.
*
* @see SimpleAvroKeyGenerator
* @see org.apache.hudi.keygen.ComplexAvroKeyGenerator
*/
public static String[] extractPartitionPath(String partitionPath, boolean hiveStylePartitioning, boolean encodePartitionPath) {
String[] fields = partitionPath.split(DEFAULT_PARTITION_PATH_SEPARATOR);

return Arrays.stream(fields).map(field -> {
String partitionVal = field;
if (hiveStylePartitioning) {
final String[] partitionArray = field.split("=");
partitionVal = partitionArray.length == 1 ? partitionArray[0] : partitionArray[1];
}
if (encodePartitionPath) {
partitionVal = PartitionPathEncodeUtils.unescapePathName(partitionVal);
}
return partitionVal;
}).toArray(String[]::new);
}

public static String getRecordKey(GenericRecord record, List<String> recordKeyFields, boolean consistentLogicalTimestampEnabled) {
boolean keyIsNullEmpty = true;
StringBuilder recordKey = new StringBuilder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -335,7 +335,7 @@ private ClosableIterator<RowData> getLogFileIterator(MergeOnReadInputSplit split
// projections. For e.g, if the pk fields are [a, b] but user only select a,
// then the pk semantics is lost.
final boolean pkSemanticLost = Arrays.stream(pkOffset).anyMatch(offset -> offset == -1);
final LogicalType[] pkTypes = pkSemanticLost ? null : tableState.getPkTypes(pkOffset);
final LogicalType[] pkTypes = pkSemanticLost ? null : tableState.getColumnTypes(pkOffset);
final StringToRowDataConverter converter = pkSemanticLost ? null : new StringToRowDataConverter(pkTypes);

return new ClosableIterator<RowData>() {
Expand All @@ -345,10 +345,12 @@ private ClosableIterator<RowData> getLogFileIterator(MergeOnReadInputSplit split
public boolean hasNext() {
while (logRecordsKeyIterator.hasNext()) {
String curAvroKey = logRecordsKeyIterator.next();
Option<IndexedRecord> curAvroRecord = null;
Option<IndexedRecord> curAvroRecord;
Comparable preCombineValue;
final HoodieAvroRecord<?> hoodieRecord = (HoodieAvroRecord) scanner.getRecords().get(curAvroKey);
try {
curAvroRecord = hoodieRecord.getData().getInsertValue(tableSchema);
preCombineValue = hoodieRecord.getData().getOrderingValue();
} catch (IOException e) {
throw new HoodieException("Get avro insert value error for key: " + curAvroKey, e);
}
Expand All @@ -363,6 +365,25 @@ public boolean hasNext() {
for (int i = 0; i < pkOffset.length; i++) {
delete.setField(pkOffset[i], converted[i]);
}
final int preCombineOffset = tableState.getRequiredPosition(conf.getString(FlinkOptions.PRECOMBINE_FIELD));
if (preCombineOffset != -1) {
final AvroToRowDataConverters.AvroToRowDataConverter preCombineConverter =
AvroToRowDataConverters.createConverter(tableState.getRequiredRowType().getTypeAt(preCombineOffset));
delete.setField(preCombineOffset, preCombineConverter.convert(preCombineValue));
}
final int[] partitionOffset = tableState.getColumnsOffsetsInRequired(conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(","));
final boolean partitionSemanticLost = Arrays.stream(partitionOffset).anyMatch(offset -> offset == -1);
final LogicalType[] partitionTypes = partitionSemanticLost ? null : tableState.getColumnTypes(partitionOffset);
final StringToRowDataConverter partitionConverter = partitionSemanticLost ? null : new StringToRowDataConverter(partitionTypes);
if (!partitionSemanticLost) {
final String partitionPath = hoodieRecord.getPartitionPath();
final String[] partitionFields =
KeyGenUtils.extractPartitionPath(partitionPath, conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING), conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING));
final Object[] partitionConverted = partitionConverter.convert(partitionFields);
for (int i = 0; i < partitionOffset.length; i++) {
delete.setField(partitionOffset[i], partitionConverted[i]);
}
}
delete.setRowKind(RowKind.DELETE);

this.currentRecord = delete;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,11 @@ public int[] getRequiredPositions() {
.toArray();
}

public int getRequiredPosition(String fieldName) {
final List<String> fieldNames = requiredRowType.getFieldNames();
return fieldNames.indexOf(fieldName);
}

/**
* Get the primary key positions in required row type.
*/
Expand All @@ -102,16 +107,27 @@ public int[] getPkOffsetsInRequired() {
}

/**
* Returns the primary key fields logical type with given offsets.
* Get column positions in required row type.
*/
public int[] getColumnsOffsetsInRequired(String[] columns) {
final List<String> fieldNames = requiredRowType.getFieldNames();
return Arrays.stream(columns)
.map(fieldNames::indexOf)
.mapToInt(i -> i)
.toArray();
}

/**
* Returns the fields logical type with given offsets.
*
* @param pkOffsets the pk offsets in required row type
* @return pk field logical types
* @param offsets the offsets in required row type
* @return field logical types
* @see #getPkOffsetsInRequired()
*/
public LogicalType[] getPkTypes(int[] pkOffsets) {
public LogicalType[] getColumnTypes(int[] offsets) {
final LogicalType[] requiredTypes = requiredRowType.getFields().stream()
.map(RowType.RowField::getType).toArray(LogicalType[]::new);
return Arrays.stream(pkOffsets).mapToObj(offset -> requiredTypes[offset])
return Arrays.stream(offsets).mapToObj(offset -> requiredTypes[offset])
.toArray(LogicalType[]::new);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.configuration.HadoopConfigurations;
import org.apache.hudi.keygen.constant.KeyGeneratorType;
import org.apache.hudi.table.HoodieTableSource;
import org.apache.hudi.table.format.cow.CopyOnWriteInputFormat;
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
Expand All @@ -48,6 +49,7 @@
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
Expand Down Expand Up @@ -290,6 +292,33 @@ void testReadWithDeletesMOR() throws Exception {
assertThat(actual, is(expected));
}

@Test
void testReadWithDeletesMORChangeLogDisabled() throws Exception {
Map<String, String> options = new HashMap<>();
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), "false");
options.put(FlinkOptions.PARTITION_PATH_FIELD.key(), "partition,name");
options.put(FlinkOptions.KEYGEN_TYPE.key(), KeyGeneratorType.COMPLEX.name());
beforeEach(HoodieTableType.MERGE_ON_READ, options);

// 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 = "["
+ "+I[id1, Danny, 24, 1970-01-01T00:00:00.001, par1], "
+ "+I[id2, Stephen, 34, 1970-01-01T00:00:00.002, par1], "
+ "-D[id3, Julian, null, 1970-01-01T00:00:00.003, par2], "
+ "-D[id5, Sophia, null, 1970-01-01T00:00:00.005, par3], "
+ "-D[id9, Jane, null, 1970-01-01T00:00:00.006, par3]]";
assertThat(actual, is(expected));
}

@Test
void testReadWithDeletesCOW() throws Exception {
beforeEach(HoodieTableType.COPY_ON_WRITE);
Expand Down Expand Up @@ -626,7 +655,7 @@ private HoodieTableSource getTableSource(Configuration conf) {
return new HoodieTableSource(
TestConfigurations.TABLE_SCHEMA,
new Path(tempFile.getAbsolutePath()),
Collections.singletonList("partition"),
Arrays.asList(conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(",")),
"default",
conf);
}
Expand Down