1
0

Add empty payload class to support deletes via apache spark

This commit is contained in:
lyogev
2019-04-11 15:14:53 +03:00
committed by n3nash
parent 243c58f77c
commit 9ef51deb84
2 changed files with 63 additions and 1 deletions

View File

@@ -0,0 +1,48 @@
/*
* Copyright (c) 2019 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 java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
/**
* Empty payload used for deletions
*/
public class EmptyHoodieRecordPayload implements HoodieRecordPayload<EmptyHoodieRecordPayload> {
public EmptyHoodieRecordPayload(GenericRecord record, Comparable orderingVal) { }
@Override
public EmptyHoodieRecordPayload preCombine(EmptyHoodieRecordPayload another) {
return another;
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) {
return Optional.empty();
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) {
return Optional.empty();
}
}

View File

@@ -16,9 +16,11 @@
*
*/
import java.util.Optional
import com.uber.hoodie.common.util.{SchemaTestUtil, TypedProperties}
import com.uber.hoodie.exception.HoodieException
import com.uber.hoodie.{DataSourceWriteOptions, OverwriteWithLatestAvroPayload, SimpleKeyGenerator}
import com.uber.hoodie.{DataSourceWriteOptions, EmptyHoodieRecordPayload, OverwriteWithLatestAvroPayload, SimpleKeyGenerator}
import org.apache.avro.generic.GenericRecord
import org.junit.Assert._
import org.junit.{Before, Test}
@@ -114,4 +116,16 @@ class DataSourceDefaultsTest extends AssertionsForJUnit {
val combinedGR21 = combinedPayload21.getInsertValue(schema).get().asInstanceOf[GenericRecord]
assertEquals("field2", combinedGR21.get("field1").toString)
}
@Test def testEmptyHoodieRecordPayload() = {
val emptyPayload1 = new EmptyHoodieRecordPayload(baseRecord, 1)
val laterRecord = SchemaTestUtil
.generateAvroRecordFromJson(schema, 2, "001", "f1")
val emptyPayload2 = new EmptyHoodieRecordPayload(laterRecord, 2)
// it will provide an empty record
val combinedPayload12 = emptyPayload1.preCombine(emptyPayload2)
val combined12 = combinedPayload12.getInsertValue(schema)
assertEquals(Optional.empty(), combined12)
}
}