[HUDI-2656] Generalize HoodieIndex for flexible record data type (#3893)
Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
@@ -18,15 +18,12 @@
|
||||
|
||||
package org.apache.hudi.integ.testsuite;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
@@ -45,6 +42,8 @@ import org.apache.hudi.integ.testsuite.dag.nodes.ScheduleCompactNode;
|
||||
import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
@@ -52,6 +51,8 @@ import org.apache.spark.rdd.RDD;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
@@ -128,7 +129,7 @@ public class HoodieTestSuiteWriter implements Serializable {
|
||||
Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> nextBatch = fetchSource();
|
||||
lastCheckpoint = Option.of(nextBatch.getValue().getLeft());
|
||||
JavaRDD<HoodieRecord> inputRDD = nextBatch.getRight().getRight();
|
||||
return inputRDD.map(r -> (GenericRecord) r.getData()
|
||||
return inputRDD.map(r -> (GenericRecord) ((HoodieAvroRecord) r).getData()
|
||||
.getInsertValue(new Schema.Parser().parse(schema)).get()).rdd();
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user