[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:
@@ -20,6 +20,7 @@ package org.apache.hudi.examples.common;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
@@ -126,7 +127,7 @@ public class HoodieExampleDataGenerator<T extends HoodieRecordPayload<T>> {
|
||||
kp.partitionPath = partitionPath;
|
||||
existingKeys.put(currSize + i, kp);
|
||||
numExistingKeys++;
|
||||
return new HoodieRecord<>(key, generateRandomValue(key, commitTime));
|
||||
return new HoodieAvroRecord<>(key, generateRandomValue(key, commitTime));
|
||||
});
|
||||
}
|
||||
|
||||
@@ -149,7 +150,7 @@ public class HoodieExampleDataGenerator<T extends HoodieRecordPayload<T>> {
|
||||
}
|
||||
|
||||
public HoodieRecord<T> generateUpdateRecord(HoodieKey key, String commitTime) {
|
||||
return new HoodieRecord<>(key, generateRandomValue(key, commitTime));
|
||||
return new HoodieAvroRecord<>(key, generateRandomValue(key, commitTime));
|
||||
}
|
||||
|
||||
private Option<String> convertToString(HoodieRecord<T> record) {
|
||||
|
||||
@@ -18,11 +18,11 @@
|
||||
|
||||
package org.apache.hudi.examples.java;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.client.HoodieJavaWriteClient;
|
||||
import org.apache.hudi.client.common.HoodieJavaEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
@@ -33,6 +33,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.examples.common.HoodieExampleDataGenerator;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -95,7 +96,7 @@ public class HoodieJavaWriteClientExample {
|
||||
List<HoodieRecord<HoodieAvroPayload>> records = dataGen.generateInserts(newCommitTime, 10);
|
||||
List<HoodieRecord<HoodieAvroPayload>> recordsSoFar = new ArrayList<>(records);
|
||||
List<HoodieRecord<HoodieAvroPayload>> writeRecords =
|
||||
recordsSoFar.stream().map(r -> new HoodieRecord<HoodieAvroPayload>(r)).collect(Collectors.toList());
|
||||
recordsSoFar.stream().map(r -> new HoodieAvroRecord<HoodieAvroPayload>(r)).collect(Collectors.toList());
|
||||
client.insert(writeRecords, newCommitTime);
|
||||
|
||||
// updates
|
||||
@@ -105,7 +106,7 @@ public class HoodieJavaWriteClientExample {
|
||||
records.addAll(toBeUpdated);
|
||||
recordsSoFar.addAll(toBeUpdated);
|
||||
writeRecords =
|
||||
recordsSoFar.stream().map(r -> new HoodieRecord<HoodieAvroPayload>(r)).collect(Collectors.toList());
|
||||
recordsSoFar.stream().map(r -> new HoodieAvroRecord<HoodieAvroPayload>(r)).collect(Collectors.toList());
|
||||
client.upsert(writeRecords, newCommitTime);
|
||||
|
||||
// Delete
|
||||
|
||||
Reference in New Issue
Block a user