1
0

[HUDI-2987] Update all deprecated calls to new apis in HoodieRecordPayload (#4681)

This commit is contained in:
Sivabalan Narayanan
2022-02-10 19:19:33 -05:00
committed by GitHub
parent 2fe7a3a41f
commit ba4e732ba7
8 changed files with 68 additions and 21 deletions

View File

@@ -31,6 +31,7 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Properties;
/**
* Provides support for seamlessly applying changes captured via Debezium for PostgresDB.
@@ -71,6 +72,19 @@ public class PostgresDebeziumAvroPayload extends AbstractDebeziumAvroPayload {
return insertSourceLSN < currentSourceLSN;
}
@Override
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) throws IOException {
// Specific to Postgres: If the updated record has TOASTED columns,
// we will need to keep the previous value for those columns
// see https://debezium.io/documentation/reference/connectors/postgresql.html#postgresql-toasted-values
Option<IndexedRecord> insertOrDeleteRecord = super.combineAndGetUpdateValue(currentValue, schema, properties);
if (insertOrDeleteRecord.isPresent()) {
mergeToastedValuesIfPresent(insertOrDeleteRecord.get(), currentValue);
}
return insertOrDeleteRecord;
}
@Override
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException {
// Specific to Postgres: If the updated record has TOASTED columns,

View File

@@ -1,91 +0,0 @@
/*
* 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.common.table.log;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import java.io.IOException;
import java.util.Iterator;
import java.util.stream.StreamSupport;
/**
* Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
*/
public class HoodieFileSliceReader<T extends HoodieRecordPayload> implements Iterator<HoodieRecord<T>> {
private final Iterator<HoodieRecord<T>> recordsIterator;
public static HoodieFileSliceReader getFileSliceReader(
Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
if (baseFileReader.isPresent()) {
Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
while (baseIterator.hasNext()) {
GenericRecord record = (GenericRecord) baseIterator.next();
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = transform(
record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
scanner.processNextRecord(hoodieRecord);
}
return new HoodieFileSliceReader(scanner.iterator());
} else {
Iterable<HoodieRecord<? extends HoodieRecordPayload>> iterable = () -> scanner.iterator();
return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false)
.map(e -> {
try {
GenericRecord record = (GenericRecord) e.getData().getInsertValue(schema).get();
return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
} catch (IOException io) {
throw new HoodieIOException("Error while creating reader for file slice with no base file.", io);
}
}).iterator());
}
}
private static HoodieRecord<? extends HoodieRecordPayload> transform(
GenericRecord record, HoodieMergedLogRecordScanner scanner, String payloadClass,
String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) {
return simpleKeyGenFieldsOpt.isPresent()
? SpillableMapUtils.convertToHoodieRecordPayload(record,
payloadClass, preCombineField, simpleKeyGenFieldsOpt.get(), scanner.isWithOperationField(), Option.empty())
: SpillableMapUtils.convertToHoodieRecordPayload(record,
payloadClass, preCombineField, scanner.isWithOperationField(), scanner.getPartitionName());
}
private HoodieFileSliceReader(Iterator<HoodieRecord<T>> recordsItr) {
this.recordsIterator = recordsItr;
}
@Override
public boolean hasNext() {
return recordsIterator.hasNext();
}
@Override
public HoodieRecord<T> next() {
return recordsIterator.next();
}
}

View File

@@ -52,6 +52,7 @@ import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -67,7 +68,7 @@ import static org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_L
* <p>
* METADATA_TYPE_PARTITION_LIST (1):
* -- List of all partitions. There is a single such record
* -- key = @{@link HoodieTableMetadata.RECORDKEY_PARTITION_LIST}
* -- key = @{@link HoodieTableMetadata#RECORDKEY_PARTITION_LIST}
* <p>
* METADATA_TYPE_FILE_LIST (2):
* -- List of all files in a partition. There is one such record for each partition
@@ -289,14 +290,19 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
}
@Override
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRecord, Schema schema) throws IOException {
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRecord, Schema schema, Properties properties) throws IOException {
HoodieMetadataPayload anotherPayload = new HoodieMetadataPayload(Option.of((GenericRecord) oldRecord));
HoodieRecordPayload combinedPayload = preCombine(anotherPayload);
return combinedPayload.getInsertValue(schema);
return combinedPayload.getInsertValue(schema, properties);
}
@Override
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRecord, Schema schema) throws IOException {
return combineAndGetUpdateValue(oldRecord, schema, new Properties());
}
@Override
public Option<IndexedRecord> getInsertValue(Schema schema, Properties properties) throws IOException {
if (key == null) {
return Option.empty();
}
@@ -306,6 +312,11 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
return Option.of(record);
}
@Override
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return getInsertValue(schema, new Properties());
}
/**
* Returns the list of filenames added as part of this record.
*/