1
0

[REVERT] "[HUDI-1058] Make delete marker configurable (#1819)" (#1914)

This reverts commit 433d7d2c98.
This commit is contained in:
Sivabalan Narayanan
2020-08-04 18:20:38 -04:00
committed by GitHub
parent 539621bd33
commit ab11ba43e1
14 changed files with 43 additions and 266 deletions

View File

@@ -27,7 +27,6 @@ import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.StringUtils;
@@ -208,20 +207,11 @@ public class DataSourceUtils {
/**
* Create a payload class via reflection, passing in an ordering/precombine value.
*/
public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record,
Comparable orderingVal,
String deleteMarkerField) throws IOException {
public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record, Comparable orderingVal)
throws IOException {
try {
HoodieRecordPayload payload = null;
if (payloadClass.equals(OverwriteWithLatestAvroPayload.class.getName())) {
payload = (OverwriteWithLatestAvroPayload) ReflectionUtils.loadClass(payloadClass,
new Class<?>[]{GenericRecord.class, Comparable.class, String.class},
record, orderingVal, deleteMarkerField);
} else {
payload = (HoodieRecordPayload) ReflectionUtils.loadClass(payloadClass,
new Class<?>[]{GenericRecord.class, Comparable.class}, record, orderingVal);
}
return payload;
return (HoodieRecordPayload) ReflectionUtils.loadClass(payloadClass,
new Class<?>[] {GenericRecord.class, Comparable.class}, record, orderingVal);
} catch (Throwable e) {
throw new IOException("Could not create payload for class: " + payloadClass, e);
}
@@ -277,9 +267,8 @@ public class DataSourceUtils {
}
public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal, HoodieKey hKey,
String payloadClass,
String deleteMarkerField) throws IOException {
HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal, deleteMarkerField);
String payloadClass) throws IOException {
HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal);
return new HoodieRecord<>(hKey, payload);
}

View File

@@ -70,7 +70,7 @@ public class SparkParquetBootstrapDataProvider extends FullRecordBootstrapDataPr
gr, props.getString("hoodie.datasource.write.precombine.field"), false);
try {
return DataSourceUtils.createHoodieRecord(gr, orderingVal, keyGenerator.getKey(gr),
props.getString("hoodie.datasource.write.payload.class"), "_hoodie_is_deleted");
props.getString("hoodie.datasource.write.payload.class"));
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}

View File

@@ -184,13 +184,6 @@ object DataSourceWriteOptions {
val PAYLOAD_CLASS_OPT_KEY = "hoodie.datasource.write.payload.class"
val DEFAULT_PAYLOAD_OPT_VAL = classOf[OverwriteWithLatestAvroPayload].getName
/**
* Field used in OverwriteWithLatestAvroPayload combineAndGetUpdateValue, When two records have the same
* key value, we will check if the new record is deleted by the delete field.
*/
val DELETE_FIELD_OPT_KEY = "hoodie.datasource.write.delete.field"
val DEFAULT_DELETE_FIELD_OPT_VAL = "_hoodie_is_deleted"
/**
* Record key field. Value to be used as the `recordKey` component of `HoodieKey`. Actual value
* will be obtained by invoking .toString() on the field value. Nested fields can be specified using

View File

@@ -111,9 +111,7 @@ private[hudi] object HoodieSparkSqlWriter {
val orderingVal = HoodieAvroUtils.getNestedFieldVal(gr, parameters(PRECOMBINE_FIELD_OPT_KEY), false)
.asInstanceOf[Comparable[_]]
DataSourceUtils.createHoodieRecord(gr,
orderingVal, keyGenerator.getKey(gr),
parameters(PAYLOAD_CLASS_OPT_KEY),
parameters(DELETE_FIELD_OPT_KEY))
orderingVal, keyGenerator.getKey(gr), parameters(PAYLOAD_CLASS_OPT_KEY))
}).toJavaRDD()
// Handle various save modes
@@ -206,7 +204,6 @@ private[hudi] object HoodieSparkSqlWriter {
TABLE_TYPE_OPT_KEY -> DEFAULT_TABLE_TYPE_OPT_VAL,
PRECOMBINE_FIELD_OPT_KEY -> DEFAULT_PRECOMBINE_FIELD_OPT_VAL,
PAYLOAD_CLASS_OPT_KEY -> DEFAULT_PAYLOAD_OPT_VAL,
DELETE_FIELD_OPT_KEY -> DEFAULT_DELETE_FIELD_OPT_VAL,
RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL,
PARTITIONPATH_FIELD_OPT_KEY -> DEFAULT_PARTITIONPATH_FIELD_OPT_VAL,
KEYGENERATOR_CLASS_OPT_KEY -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,

View File

@@ -100,52 +100,6 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
}
}
test("test OverwriteWithLatestAvroPayload with user defined delete field") {
val session = SparkSession.builder()
.appName("test_append_mode")
.master("local[2]")
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.getOrCreate()
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path1")
try {
val sqlContext = session.sqlContext
val hoodieFooTableName = "hoodie_foo_tbl"
val keyField = "id"
val deleteMarkerField = "delete_field"
//create a new table
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
"hoodie.insert.shuffle.parallelism" -> "2",
"hoodie.upsert.shuffle.parallelism" -> "2",
DELETE_FIELD_OPT_KEY -> deleteMarkerField,
RECORDKEY_FIELD_OPT_KEY -> keyField)
val fooTableParams = HoodieSparkSqlWriter.parametersWithWriteDefaults(fooTableModifier)
val id1 = UUID.randomUUID().toString
val dataFrame = session.createDataFrame(Seq(
(id1, 1, false)
)) toDF(keyField, "ts", deleteMarkerField)
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, dataFrame)
val recordCount1 = sqlContext.read.format("org.apache.hudi").load(path.toString + "/*/*.parquet").count
assert(recordCount1 == 1, "result should be 1, but get " + recordCount1)
val dataFrame2 = session.createDataFrame(Seq(
(id1, 2, true)
)) toDF(keyField, "ts", deleteMarkerField)
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, dataFrame2)
val recordCount2 = sqlContext.read.format("org.apache.hudi").load(path.toString + "/*/*.parquet").count()
assert(recordCount2 == 0, "result should be 0, but get " + recordCount2)
} finally {
session.stop()
FileUtils.deleteDirectory(path.toFile)
}
}
case class Test(uuid: String, ts: Long)
}