- Introduced concept of converters to be able to serde generic datatype for SpillableMap - Fixed/Added configs to Hoodie Configs - Changed HoodieMergeHandle to start using SpillableMap
73 lines
2.3 KiB
Java
73 lines
2.3 KiB
Java
/*
|
|
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
|
*
|
|
* Licensed 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 com.uber.hoodie;
|
|
|
|
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
|
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
|
import java.io.IOException;
|
|
import java.util.Optional;
|
|
import org.apache.avro.Schema;
|
|
import org.apache.avro.generic.GenericRecord;
|
|
import org.apache.avro.generic.IndexedRecord;
|
|
|
|
/**
|
|
* Default payload used for delta streamer.
|
|
*
|
|
* 1. preCombine - Picks the latest delta record for a key, based on an ordering field 2.
|
|
* combineAndGetUpdateValue/getInsertValue - Simply overwrites storage with latest delta record
|
|
*/
|
|
public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements
|
|
HoodieRecordPayload<OverwriteWithLatestAvroPayload> {
|
|
|
|
/**
|
|
*
|
|
* @param record
|
|
* @param orderingVal
|
|
*/
|
|
public OverwriteWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) {
|
|
super(record, orderingVal);
|
|
}
|
|
|
|
public OverwriteWithLatestAvroPayload(Optional<GenericRecord> record) {
|
|
this(record.get(), (record1) -> 0); // natural order
|
|
}
|
|
|
|
@Override
|
|
public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload another) {
|
|
// pick the payload with greatest ordering value
|
|
if (another.orderingVal.compareTo(orderingVal) > 0) {
|
|
return another;
|
|
} else {
|
|
return this;
|
|
}
|
|
}
|
|
|
|
@Override
|
|
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
|
|
throws IOException {
|
|
// combining strategy here trivially ignores currentValue on disk and writes this record
|
|
return getInsertValue(schema);
|
|
}
|
|
|
|
@Override
|
|
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
|
|
return Optional.of(HoodieAvroUtils.rewriteRecord(record, schema));
|
|
}
|
|
}
|