[HUDI-115] Adding DefaultHoodieRecordPayload to honor ordering with combineAndGetUpdateValue (#2311)
* Added ability to pass in `properties` to payload methods, so they can perform table/record specific merges * Added default methods so existing payload classes are backwards compatible. * Adding DefaultHoodiePayload to honor ordering while merging two records * Fixing default payload based on feedback
This commit is contained in:
committed by
GitHub
parent
5388c7f7a3
commit
33d338f392
@@ -442,7 +442,7 @@ public class HoodieAvroUtils {
|
||||
* @param fieldValue avro field value
|
||||
* @return field value either converted (for certain data types) or as it is.
|
||||
*/
|
||||
private static Object convertValueForSpecificDataTypes(Schema fieldSchema, Object fieldValue) {
|
||||
public static Object convertValueForSpecificDataTypes(Schema fieldSchema, Object fieldValue) {
|
||||
if (fieldSchema == null) {
|
||||
return fieldValue;
|
||||
}
|
||||
|
||||
@@ -0,0 +1,82 @@
|
||||
/*
|
||||
* 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.model;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldVal;
|
||||
|
||||
/**
|
||||
* {@link HoodieRecordPayload} impl that honors ordering field in both preCombine and combineAndGetUpdateValue.
|
||||
* <p>
|
||||
* 1. preCombine - Picks the latest delta record for a key, based on an ordering field 2. combineAndGetUpdateValue/getInsertValue - Chooses the latest record based on ordering field value.
|
||||
*/
|
||||
public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
|
||||
|
||||
public DefaultHoodieRecordPayload(GenericRecord record, Comparable orderingVal) {
|
||||
super(record, orderingVal);
|
||||
}
|
||||
|
||||
public DefaultHoodieRecordPayload(Option<GenericRecord> record) {
|
||||
this(record.isPresent() ? record.get() : null, (record1) -> 0); // natural order
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) throws IOException {
|
||||
if (recordBytes.length == 0) {
|
||||
return Option.empty();
|
||||
}
|
||||
GenericRecord incomingRecord = bytesToAvro(recordBytes, schema);
|
||||
/*
|
||||
* Combining strategy here returns currentValue on disk if incoming record is older.
|
||||
* The incoming record can be either a delete (sent as an upsert with _hoodie_is_deleted set to true)
|
||||
* or an insert/update record. In any case, if it is older than the record in disk, the currentValue
|
||||
* in disk is returned (to be rewritten with new commit time).
|
||||
*
|
||||
* NOTE: Deletes sent via EmptyHoodieRecordPayload and/or Delete operation type do not hit this code path
|
||||
* and need to be dealt with separately.
|
||||
*/
|
||||
Object persistedOrderingVal = getNestedFieldVal((GenericRecord) currentValue, properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), true);
|
||||
Comparable incomingOrderingVal = (Comparable) getNestedFieldVal(incomingRecord, properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), false);
|
||||
|
||||
// Null check is needed here to support schema evolution. The record in storage may be from old schema where
|
||||
// the new ordering column might not be present and hence returns null.
|
||||
if (persistedOrderingVal != null && ((Comparable) persistedOrderingVal).compareTo(incomingOrderingVal) > 0) {
|
||||
return Option.of(currentValue);
|
||||
}
|
||||
|
||||
/*
|
||||
* We reached a point where the value is disk is older than the incoming record.
|
||||
* Now check if the incoming record is a delete record.
|
||||
*/
|
||||
if (isDeleteRecord(incomingRecord)) {
|
||||
return Option.empty();
|
||||
} else {
|
||||
return Option.of(incomingRecord);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* 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.model;
|
||||
|
||||
/**
|
||||
* Holds payload properties that implementation of {@link HoodieRecordPayload} can leverage.
|
||||
* Since both payload classes and HoodiePayloadConfig needs to access these props, storing it here in hudi-common.
|
||||
*/
|
||||
public class HoodiePayloadProps {
|
||||
|
||||
// payload ordering field. This could be used to merge incoming record with that in storage. Implementations of
|
||||
// {@link HoodieRecordPayload} can leverage if required.
|
||||
public static final String PAYLOAD_ORDERING_FIELD_PROP = "hoodie.payload.ordering.field";
|
||||
public static String DEFAULT_PAYLOAD_ORDERING_FIELD_VAL = "ts";
|
||||
|
||||
}
|
||||
@@ -29,47 +29,84 @@ import org.apache.avro.generic.IndexedRecord;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Every Hoodie table has an implementation of the <code>HoodieRecordPayload</code> This abstracts out callbacks which
|
||||
* depend on record specific logic.
|
||||
* Every Hoodie table has an implementation of the <code>HoodieRecordPayload</code> This abstracts out callbacks which depend on record specific logic.
|
||||
*/
|
||||
@PublicAPIClass(maturity = ApiMaturityLevel.STABLE)
|
||||
public interface HoodieRecordPayload<T extends HoodieRecordPayload> extends Serializable {
|
||||
|
||||
/**
|
||||
* When more than one HoodieRecord have the same HoodieKey, this function combines them before attempting to
|
||||
* insert/upsert (if combining turned on in HoodieClientConfig).
|
||||
* This method is deprecated. Please use this {@link #preCombine(HoodieRecordPayload, Properties)} method.
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
T preCombine(T another);
|
||||
|
||||
/**
|
||||
* This methods lets you write custom merging/combining logic to produce new values as a function of current value on
|
||||
* storage and whats contained in this object.
|
||||
* <p>
|
||||
* eg: 1) You are updating counters, you may want to add counts to currentValue and write back updated counts 2) You
|
||||
* may be reading DB redo logs, and merge them with current image for a database row on storage
|
||||
*
|
||||
* @param currentValue Current value in storage, to merge/combine this payload with
|
||||
* @param schema Schema used for record
|
||||
* @return new combined/merged value to be written back to storage. EMPTY to skip writing this record.
|
||||
* When more than one HoodieRecord have the same HoodieKey, this function combines them before attempting to insert/upsert by taking in a property map.
|
||||
* Implementation can leverage the property to decide their business logic to do preCombine.
|
||||
* @param another instance of another {@link HoodieRecordPayload} to be combined with.
|
||||
* @param properties Payload related properties. For example pass the ordering field(s) name to extract from value in storage.
|
||||
* @return the combined value
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
default T preCombine(T another, Properties properties) {
|
||||
return preCombine(another);
|
||||
}
|
||||
|
||||
/**
|
||||
* This methods is deprecated. Please refer to {@link #combineAndGetUpdateValue(IndexedRecord, Schema, Properties)} for java docs.
|
||||
*/
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException;
|
||||
|
||||
/**
|
||||
* Generates an avro record out of the given HoodieRecordPayload, to be written out to storage. Called when writing a
|
||||
* new value for the given HoodieKey, wherein there is no existing record in storage to be combined against. (i.e
|
||||
* insert) Return EMPTY to skip writing this record.
|
||||
* This methods lets you write custom merging/combining logic to produce new values as a function of current value on storage and whats contained
|
||||
* in this object. Implementations can leverage properties if required.
|
||||
* <p>
|
||||
* eg:
|
||||
* 1) You are updating counters, you may want to add counts to currentValue and write back updated counts
|
||||
* 2) You may be reading DB redo logs, and merge them with current image for a database row on storage
|
||||
* </p>
|
||||
*
|
||||
* @param currentValue Current value in storage, to merge/combine this payload with
|
||||
* @param schema Schema used for record
|
||||
* @param properties Payload related properties. For example pass the ordering field(s) name to extract from value in storage.
|
||||
* @return new combined/merged value to be written back to storage. EMPTY to skip writing this record.
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
default Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) throws IOException {
|
||||
return combineAndGetUpdateValue(currentValue, schema);
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is deprecated. Refer to {@link #getInsertValue(Schema, Properties)} for java docs.
|
||||
* @param schema Schema used for record
|
||||
* @return the {@link IndexedRecord} to be inserted.
|
||||
*/
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
Option<IndexedRecord> getInsertValue(Schema schema) throws IOException;
|
||||
|
||||
/**
|
||||
* This method can be used to extract some metadata from HoodieRecordPayload. The metadata is passed to
|
||||
* {@code WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()} in order to compute some aggregate metrics
|
||||
* using the metadata in the context of a write success or failure.
|
||||
* Generates an avro record out of the given HoodieRecordPayload, to be written out to storage. Called when writing a new value for the given
|
||||
* HoodieKey, wherein there is no existing record in storage to be combined against. (i.e insert) Return EMPTY to skip writing this record.
|
||||
* Implementations can leverage properties if required.
|
||||
* @param schema Schema used for record
|
||||
* @param properties Payload related properties. For example pass the ordering field(s) name to extract from value in storage.
|
||||
* @return the {@link IndexedRecord} to be inserted.
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
default Option<IndexedRecord> getInsertValue(Schema schema, Properties properties) throws IOException {
|
||||
return getInsertValue(schema);
|
||||
}
|
||||
|
||||
/**
|
||||
* This method can be used to extract some metadata from HoodieRecordPayload. The metadata is passed to {@code WriteStatus.markSuccess()} and
|
||||
* {@code WriteStatus.markFailure()} in order to compute some aggregate metrics using the metadata in the context of a write success or failure.
|
||||
* @return the metadata in the form of Map<String, String> if any.
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
default Option<Map<String, String>> getMetadata() {
|
||||
|
||||
@@ -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.common.model;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Type;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
||||
/**
|
||||
* Unit tests {@link DefaultHoodieRecordPayload}.
|
||||
*/
|
||||
public class TestDefaultHoodieRecordPayload {
|
||||
|
||||
private Schema schema;
|
||||
private Properties props;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
schema = Schema.createRecord(Arrays.asList(
|
||||
new Schema.Field("id", Schema.create(Schema.Type.STRING), "", null),
|
||||
new Schema.Field("partition", Schema.create(Schema.Type.STRING), "", null),
|
||||
new Schema.Field("ts", Schema.create(Schema.Type.LONG), "", null),
|
||||
new Schema.Field("_hoodie_is_deleted", Schema.create(Type.BOOLEAN), "", false)
|
||||
));
|
||||
props = new Properties();
|
||||
props.setProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP, "ts");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testActiveRecords() throws IOException {
|
||||
GenericRecord record1 = new GenericData.Record(schema);
|
||||
record1.put("id", "1");
|
||||
record1.put("partition", "partition0");
|
||||
record1.put("ts", 0L);
|
||||
record1.put("_hoodie_is_deleted", false);
|
||||
|
||||
GenericRecord record2 = new GenericData.Record(schema);
|
||||
record2.put("id", "2");
|
||||
record2.put("partition", "partition1");
|
||||
record2.put("ts", 1L);
|
||||
record2.put("_hoodie_is_deleted", false);
|
||||
|
||||
DefaultHoodieRecordPayload payload1 = new DefaultHoodieRecordPayload(record1, 1);
|
||||
DefaultHoodieRecordPayload payload2 = new DefaultHoodieRecordPayload(record2, 2);
|
||||
assertEquals(payload1.preCombine(payload2, props), payload2);
|
||||
assertEquals(payload2.preCombine(payload1, props), payload2);
|
||||
|
||||
assertEquals(record1, payload1.getInsertValue(schema).get());
|
||||
assertEquals(record2, payload2.getInsertValue(schema).get());
|
||||
|
||||
assertEquals(payload1.combineAndGetUpdateValue(record2, schema, props).get(), record2);
|
||||
assertEquals(payload2.combineAndGetUpdateValue(record1, schema, props).get(), record2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeletedRecord() throws IOException {
|
||||
GenericRecord record1 = new GenericData.Record(schema);
|
||||
record1.put("id", "1");
|
||||
record1.put("partition", "partition0");
|
||||
record1.put("ts", 0L);
|
||||
record1.put("_hoodie_is_deleted", false);
|
||||
|
||||
GenericRecord delRecord1 = new GenericData.Record(schema);
|
||||
delRecord1.put("id", "2");
|
||||
delRecord1.put("partition", "partition1");
|
||||
delRecord1.put("ts", 1L);
|
||||
delRecord1.put("_hoodie_is_deleted", true);
|
||||
|
||||
DefaultHoodieRecordPayload payload1 = new DefaultHoodieRecordPayload(record1, 1);
|
||||
DefaultHoodieRecordPayload payload2 = new DefaultHoodieRecordPayload(delRecord1, 2);
|
||||
assertEquals(payload1.preCombine(payload2, props), payload2);
|
||||
assertEquals(payload2.preCombine(payload1, props), payload2);
|
||||
|
||||
assertEquals(record1, payload1.getInsertValue(schema).get());
|
||||
assertFalse(payload2.getInsertValue(schema).isPresent());
|
||||
|
||||
assertEquals(payload1.combineAndGetUpdateValue(delRecord1, schema, props).get(), delRecord1);
|
||||
assertFalse(payload2.combineAndGetUpdateValue(record1, schema, props).isPresent());
|
||||
}
|
||||
|
||||
}
|
||||
Reference in New Issue
Block a user