[HUDI-2798] Fix flink query operation fields (#4041)
This commit is contained in:
@@ -250,11 +250,17 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
|||||||
+ ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
|
+ ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord) {
|
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, GenericRecord oldRecord, Option<IndexedRecord> indexedRecord) {
|
||||||
|
boolean isDelete = false;
|
||||||
if (indexedRecord.isPresent()) {
|
if (indexedRecord.isPresent()) {
|
||||||
updatedRecordsWritten++;
|
updatedRecordsWritten++;
|
||||||
|
GenericRecord record = (GenericRecord) indexedRecord.get();
|
||||||
|
if (oldRecord != record) {
|
||||||
|
// the incoming record is chosen
|
||||||
|
isDelete = HoodieOperation.isDelete(hoodieRecord.getOperation());
|
||||||
}
|
}
|
||||||
return writeRecord(hoodieRecord, indexedRecord);
|
}
|
||||||
|
return writeRecord(hoodieRecord, indexedRecord, isDelete);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void writeInsertRecord(HoodieRecord<T> hoodieRecord) throws IOException {
|
protected void writeInsertRecord(HoodieRecord<T> hoodieRecord) throws IOException {
|
||||||
@@ -264,12 +270,16 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
|||||||
if (insertRecord.isPresent() && insertRecord.get().equals(IGNORE_RECORD)) {
|
if (insertRecord.isPresent() && insertRecord.get().equals(IGNORE_RECORD)) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
if (writeRecord(hoodieRecord, insertRecord)) {
|
if (writeRecord(hoodieRecord, insertRecord, HoodieOperation.isDelete(hoodieRecord.getOperation()))) {
|
||||||
insertRecordsWritten++;
|
insertRecordsWritten++;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord) {
|
protected boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord) {
|
||||||
|
return writeRecord(hoodieRecord, indexedRecord, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord, boolean isDelete) {
|
||||||
Option recordMetadata = hoodieRecord.getData().getMetadata();
|
Option recordMetadata = hoodieRecord.getData().getMetadata();
|
||||||
if (!partitionPath.equals(hoodieRecord.getPartitionPath())) {
|
if (!partitionPath.equals(hoodieRecord.getPartitionPath())) {
|
||||||
HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: "
|
HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: "
|
||||||
@@ -277,11 +287,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
|||||||
writeStatus.markFailure(hoodieRecord, failureEx, recordMetadata);
|
writeStatus.markFailure(hoodieRecord, failureEx, recordMetadata);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
if (HoodieOperation.isDelete(hoodieRecord.getOperation())) {
|
|
||||||
indexedRecord = Option.empty();
|
|
||||||
}
|
|
||||||
try {
|
try {
|
||||||
if (indexedRecord.isPresent()) {
|
if (indexedRecord.isPresent() && !isDelete) {
|
||||||
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
||||||
IndexedRecord recordWithMetadataInSchema = rewriteRecord((GenericRecord) indexedRecord.get());
|
IndexedRecord recordWithMetadataInSchema = rewriteRecord((GenericRecord) indexedRecord.get());
|
||||||
fileWriter.writeAvroWithMetadata(recordWithMetadataInSchema, hoodieRecord);
|
fileWriter.writeAvroWithMetadata(recordWithMetadataInSchema, hoodieRecord);
|
||||||
@@ -321,7 +328,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
|||||||
if (combinedAvroRecord.isPresent() && combinedAvroRecord.get().equals(IGNORE_RECORD)) {
|
if (combinedAvroRecord.isPresent() && combinedAvroRecord.get().equals(IGNORE_RECORD)) {
|
||||||
// If it is an IGNORE_RECORD, just copy the old record, and do not update the new record.
|
// If it is an IGNORE_RECORD, just copy the old record, and do not update the new record.
|
||||||
copyOldRecord = true;
|
copyOldRecord = true;
|
||||||
} else if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
} else if (writeUpdateRecord(hoodieRecord, oldRecord, combinedAvroRecord)) {
|
||||||
/*
|
/*
|
||||||
* ONLY WHEN 1) we have an update for this key AND 2) We are able to successfully
|
* ONLY WHEN 1) we have an update for this key AND 2) We are able to successfully
|
||||||
* write the the combined new
|
* write the the combined new
|
||||||
|
|||||||
@@ -179,6 +179,7 @@ public abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
|
|||||||
.withSpillableMapBasePath(config.getSpillableMapBasePath())
|
.withSpillableMapBasePath(config.getSpillableMapBasePath())
|
||||||
.withDiskMapType(config.getCommonConfig().getSpillableDiskMapType())
|
.withDiskMapType(config.getCommonConfig().getSpillableDiskMapType())
|
||||||
.withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
|
.withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
|
||||||
|
.withOperationField(config.allowOperationMetadataField())
|
||||||
.build();
|
.build();
|
||||||
if (!scanner.iterator().hasNext()) {
|
if (!scanner.iterator().hasNext()) {
|
||||||
scanner.close();
|
scanner.close();
|
||||||
|
|||||||
@@ -44,6 +44,7 @@ import javax.annotation.Nullable;
|
|||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
@@ -156,6 +157,24 @@ public class IncrementalInputSplits implements Serializable {
|
|||||||
}
|
}
|
||||||
|
|
||||||
String tableName = conf.getString(FlinkOptions.TABLE_NAME);
|
String tableName = conf.getString(FlinkOptions.TABLE_NAME);
|
||||||
|
|
||||||
|
Set<String> writePartitions;
|
||||||
|
final FileStatus[] fileStatuses;
|
||||||
|
|
||||||
|
if (instantRange == null) {
|
||||||
|
// reading from the earliest, scans the partitions and files directly.
|
||||||
|
FileIndex fileIndex = FileIndex.instance(new org.apache.hadoop.fs.Path(path.toUri()), conf);
|
||||||
|
if (this.requiredPartitions != null) {
|
||||||
|
// apply partition push down
|
||||||
|
fileIndex.setPartitionPaths(this.requiredPartitions);
|
||||||
|
}
|
||||||
|
writePartitions = new HashSet<>(fileIndex.getOrBuildPartitionPaths());
|
||||||
|
if (writePartitions.size() == 0) {
|
||||||
|
LOG.warn("No partitions found for reading in user provided path.");
|
||||||
|
return Result.EMPTY;
|
||||||
|
}
|
||||||
|
fileStatuses = fileIndex.getFilesInPartitions();
|
||||||
|
} else {
|
||||||
List<HoodieCommitMetadata> activeMetadataList = instants.stream()
|
List<HoodieCommitMetadata> activeMetadataList = instants.stream()
|
||||||
.map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, commitTimeline)).collect(Collectors.toList());
|
.map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, commitTimeline)).collect(Collectors.toList());
|
||||||
List<HoodieCommitMetadata> archivedMetadataList = getArchivedMetadata(metaClient, instantRange, commitTimeline, tableName);
|
List<HoodieCommitMetadata> archivedMetadataList = getArchivedMetadata(metaClient, instantRange, commitTimeline, tableName);
|
||||||
@@ -171,13 +190,19 @@ public class IncrementalInputSplits implements Serializable {
|
|||||||
? mergeList(archivedMetadataList, activeMetadataList)
|
? mergeList(archivedMetadataList, activeMetadataList)
|
||||||
: activeMetadataList;
|
: activeMetadataList;
|
||||||
|
|
||||||
Set<String> writePartitions = HoodieInputFormatUtils.getWritePartitionPaths(metadataList);
|
writePartitions = HoodieInputFormatUtils.getWritePartitionPaths(metadataList);
|
||||||
// apply partition push down
|
// apply partition push down
|
||||||
if (this.requiredPartitions != null) {
|
if (this.requiredPartitions != null) {
|
||||||
writePartitions = writePartitions.stream()
|
writePartitions = writePartitions.stream()
|
||||||
.filter(this.requiredPartitions::contains).collect(Collectors.toSet());
|
.filter(this.requiredPartitions::contains).collect(Collectors.toSet());
|
||||||
}
|
}
|
||||||
FileStatus[] fileStatuses = WriteProfiles.getWritePathsOfInstants(path, hadoopConf, metadataList, metaClient.getTableType());
|
if (writePartitions.size() == 0) {
|
||||||
|
LOG.warn("No partitions found for reading in user provided path.");
|
||||||
|
return Result.EMPTY;
|
||||||
|
}
|
||||||
|
fileStatuses = WriteProfiles.getWritePathsOfInstants(path, hadoopConf, metadataList, metaClient.getTableType());
|
||||||
|
}
|
||||||
|
|
||||||
if (fileStatuses.length == 0) {
|
if (fileStatuses.length == 0) {
|
||||||
LOG.warn("No files found for reading in user provided path.");
|
LOG.warn("No files found for reading in user provided path.");
|
||||||
return Result.EMPTY;
|
return Result.EMPTY;
|
||||||
|
|||||||
@@ -683,13 +683,18 @@ public class MergeOnReadInputFormat
|
|||||||
// deleted
|
// deleted
|
||||||
continue;
|
continue;
|
||||||
} else {
|
} else {
|
||||||
|
final RowKind rowKind = FormatUtils.getRowKindSafely(mergedAvroRecord.get(), this.operationPos);
|
||||||
|
if (!emitDelete && rowKind == RowKind.DELETE) {
|
||||||
|
// deleted
|
||||||
|
continue;
|
||||||
|
}
|
||||||
GenericRecord avroRecord = buildAvroRecordBySchema(
|
GenericRecord avroRecord = buildAvroRecordBySchema(
|
||||||
mergedAvroRecord.get(),
|
mergedAvroRecord.get(),
|
||||||
requiredSchema,
|
requiredSchema,
|
||||||
requiredPos,
|
requiredPos,
|
||||||
recordBuilder);
|
recordBuilder);
|
||||||
this.currentRecord = (RowData) avroToRowDataConverter.convert(avroRecord);
|
this.currentRecord = (RowData) avroToRowDataConverter.convert(avroRecord);
|
||||||
FormatUtils.setRowKind(this.currentRecord, mergedAvroRecord.get(), this.operationPos);
|
this.currentRecord.setRowKind(rowKind);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@@ -746,9 +751,6 @@ public class MergeOnReadInputFormat
|
|||||||
RowData curRow,
|
RowData curRow,
|
||||||
String curKey) throws IOException {
|
String curKey) throws IOException {
|
||||||
final HoodieRecord<?> record = scanner.getRecords().get(curKey);
|
final HoodieRecord<?> record = scanner.getRecords().get(curKey);
|
||||||
if (!emitDelete && HoodieOperation.isDelete(record.getOperation())) {
|
|
||||||
return Option.empty();
|
|
||||||
}
|
|
||||||
GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow);
|
GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow);
|
||||||
return record.getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema);
|
return record.getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -39,6 +39,7 @@ import org.junit.jupiter.api.Test;
|
|||||||
import org.junit.jupiter.api.io.TempDir;
|
import org.junit.jupiter.api.io.TempDir;
|
||||||
import org.junit.jupiter.params.ParameterizedTest;
|
import org.junit.jupiter.params.ParameterizedTest;
|
||||||
import org.junit.jupiter.params.provider.EnumSource;
|
import org.junit.jupiter.params.provider.EnumSource;
|
||||||
|
import org.junit.jupiter.params.provider.ValueSource;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
@@ -221,8 +222,9 @@ public class TestInputFormat {
|
|||||||
assertThat(actual2, is(expected2));
|
assertThat(actual2, is(expected2));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
void testReadBaseAndLogFilesWithDisorderUpdateDelete() throws Exception {
|
@ValueSource(booleans = {true, false})
|
||||||
|
void testReadBaseAndLogFilesWithDisorderUpdateDelete(boolean compact) throws Exception {
|
||||||
Map<String, String> options = new HashMap<>();
|
Map<String, String> options = new HashMap<>();
|
||||||
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), "true");
|
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), "true");
|
||||||
beforeEach(HoodieTableType.MERGE_ON_READ, options);
|
beforeEach(HoodieTableType.MERGE_ON_READ, options);
|
||||||
@@ -233,7 +235,7 @@ public class TestInputFormat {
|
|||||||
TestData.writeData(TestData.DATA_SET_SINGLE_INSERT, conf);
|
TestData.writeData(TestData.DATA_SET_SINGLE_INSERT, conf);
|
||||||
|
|
||||||
// write another commit using logs and read again.
|
// write another commit using logs and read again.
|
||||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false);
|
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, compact);
|
||||||
TestData.writeData(TestData.DATA_SET_DISORDER_UPDATE_DELETE, conf);
|
TestData.writeData(TestData.DATA_SET_DISORDER_UPDATE_DELETE, conf);
|
||||||
|
|
||||||
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
||||||
@@ -242,9 +244,11 @@ public class TestInputFormat {
|
|||||||
// when isEmitDelete is false.
|
// when isEmitDelete is false.
|
||||||
List<RowData> result1 = readData(inputFormat);
|
List<RowData> result1 = readData(inputFormat);
|
||||||
|
|
||||||
|
final String rowKind = compact ? "I" : "U";
|
||||||
|
final String expected = "[+" + rowKind + "[id1, Danny, 22, 1970-01-01T00:00:00.004, par1]]";
|
||||||
|
|
||||||
final String actual1 = TestData.rowDataToString(result1);
|
final String actual1 = TestData.rowDataToString(result1);
|
||||||
final String expected1 = "[+U[id1, Danny, 22, 1970-01-01T00:00:00.004, par1]]";
|
assertThat(actual1, is(expected));
|
||||||
assertThat(actual1, is(expected1));
|
|
||||||
|
|
||||||
// refresh the input format and set isEmitDelete to true.
|
// refresh the input format and set isEmitDelete to true.
|
||||||
this.tableSource.reset();
|
this.tableSource.reset();
|
||||||
@@ -254,8 +258,7 @@ public class TestInputFormat {
|
|||||||
List<RowData> result2 = readData(inputFormat);
|
List<RowData> result2 = readData(inputFormat);
|
||||||
|
|
||||||
final String actual2 = TestData.rowDataToString(result2);
|
final String actual2 = TestData.rowDataToString(result2);
|
||||||
final String expected2 = "[+U[id1, Danny, 22, 1970-01-01T00:00:00.004, par1]]";
|
assertThat(actual2, is(expected));
|
||||||
assertThat(actual2, is(expected2));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|||||||
Reference in New Issue
Block a user