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

@@ -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)
}
}