1
0

[HUDI-3469] Refactor HoodieTestDataGenerator to provide for reproducible Builds (#4866)

This commit is contained in:
Alexey Kudinkin
2022-03-01 22:15:26 -08:00
committed by GitHub
parent 3b2da9f138
commit 85f47b53df
13 changed files with 159 additions and 115 deletions

View File

@@ -19,6 +19,17 @@
package org.apache.hudi.common.testutils;
import org.apache.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericArray;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.fs.FSUtils;
@@ -34,29 +45,22 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.AvroOrcUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericArray;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.orc.TypeDescription;
import java.io.IOException;
import java.io.Serializable;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.sql.Date;
import java.time.Instant;
import java.time.LocalDateTime;
import java.time.ZoneOffset;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@@ -140,7 +144,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
public static final TypeDescription ORC_TRIP_SCHEMA = AvroOrcUtils.createOrcSchema(new Schema.Parser().parse(TRIP_SCHEMA));
public static final Schema FLATTENED_AVRO_SCHEMA = new Schema.Parser().parse(TRIP_FLATTENED_SCHEMA);
private static final Random RAND = new Random(46474747);
private final Random rand;
//Maintains all the existing keys schema wise
private final Map<String, Map<Integer, KeyPartition>> existingKeysBySchema;
@@ -148,27 +152,58 @@ public class HoodieTestDataGenerator implements AutoCloseable {
//maintains the count of existing keys schema wise
private Map<String, Integer> numKeysBySchema;
public HoodieTestDataGenerator(long seed) {
this(seed, DEFAULT_PARTITION_PATHS, new HashMap<>());
}
public HoodieTestDataGenerator(long seed, String[] partitionPaths, Map<Integer, KeyPartition> keyPartitionMap) {
this.rand = new Random(seed);
this.partitionPaths = Arrays.copyOf(partitionPaths, partitionPaths.length);
this.existingKeysBySchema = new HashMap<>();
this.existingKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, keyPartitionMap);
this.numKeysBySchema = new HashMap<>();
this.numKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, keyPartitionMap.size());
logger.info(String.format("Test DataGenerator's seed (%s)", seed));
}
//////////////////////////////////////////////////////////////////////////////////
// DEPRECATED API
//////////////////////////////////////////////////////////////////////////////////
@Deprecated
public HoodieTestDataGenerator(String[] partitionPaths) {
this(partitionPaths, new HashMap<>());
}
@Deprecated
public HoodieTestDataGenerator() {
this(DEFAULT_PARTITION_PATHS);
}
@Deprecated
public HoodieTestDataGenerator(String[] partitionPaths, Map<Integer, KeyPartition> keyPartitionMap) {
this.partitionPaths = Arrays.copyOf(partitionPaths, partitionPaths.length);
this.existingKeysBySchema = new HashMap<>();
existingKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, keyPartitionMap);
numKeysBySchema = new HashMap<>();
numKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, keyPartitionMap.size());
// NOTE: This used as a workaround to make sure that new instantiations of the generator
// always return "new" random values.
// Caveat is that if 2 successive invocations are made w/in the timespan that is smaller
// than the resolution of {@code nanoTime}, then this will produce identical results
this(System.nanoTime(), partitionPaths, keyPartitionMap);
}
/**
* @deprecated please use non-static version
*/
public static void writePartitionMetadataDeprecated(FileSystem fs, String[] partitionPaths, String basePath) {
new HoodieTestDataGenerator().writePartitionMetadata(fs, partitionPaths, basePath);
}
//////////////////////////////////////////////////////////////////////////////////
/**
* @implNote {@link HoodieTestDataGenerator} is supposed to just generate records with schemas. Leave HoodieTable files (metafile, basefile, logfile, etc) to {@link HoodieTestTable}.
* @deprecated Use {@link HoodieTestTable#withPartitionMetaFiles(java.lang.String...)} instead.
*/
public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths, String basePath) {
public void writePartitionMetadata(FileSystem fs, String[] partitionPaths, String basePath) {
for (String partitionPath : partitionPaths) {
new HoodiePartitionMetadata(fs, "000", new Path(basePath), new Path(basePath, partitionPath)).trySave(0);
}
@@ -198,7 +233,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
* @param instantTime Instant time to use.
* @return Raw paylaod of a test record.
*/
public static RawTripTestPayload generateRandomValue(HoodieKey key, String instantTime) throws IOException {
public RawTripTestPayload generateRandomValue(HoodieKey key, String instantTime) throws IOException {
return generateRandomValue(key, instantTime, false);
}
@@ -212,12 +247,12 @@ public class HoodieTestDataGenerator implements AutoCloseable {
* @return Raw paylaod of a test record.
* @throws IOException
*/
public static RawTripTestPayload generateRandomValue(
private RawTripTestPayload generateRandomValue(
HoodieKey key, String instantTime, boolean isFlattened) throws IOException {
return generateRandomValue(key, instantTime, isFlattened, 0);
}
public static RawTripTestPayload generateRandomValue(
private RawTripTestPayload generateRandomValue(
HoodieKey key, String instantTime, boolean isFlattened, int ts) throws IOException {
GenericRecord rec = generateGenericRecord(
key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, ts,
@@ -241,7 +276,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
/**
* Generates a new avro record of the above schema format for a delete.
*/
public static RawTripTestPayload generateRandomDeleteValue(HoodieKey key, String instantTime) throws IOException {
private RawTripTestPayload generateRandomDeleteValue(HoodieKey key, String instantTime) throws IOException {
GenericRecord rec = generateGenericRecord(key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0,
true, false);
return new RawTripTestPayload(Option.of(rec.toString()), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA, true, 0L);
@@ -250,17 +285,17 @@ public class HoodieTestDataGenerator implements AutoCloseable {
/**
* Generates a new avro record of the above schema format, retaining the key if optionally provided.
*/
public static HoodieAvroPayload generateAvroPayload(HoodieKey key, String instantTime) {
private HoodieAvroPayload generateAvroPayload(HoodieKey key, String instantTime) {
GenericRecord rec = generateGenericRecord(key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0);
return new HoodieAvroPayload(Option.of(rec));
}
public static GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName,
long timestamp) {
public GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName,
long timestamp) {
return generateGenericRecord(rowKey, partitionPath, riderName, driverName, timestamp, false, false);
}
public static GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName,
public GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName,
long timestamp, boolean isDeleteRecord,
boolean isFlattened) {
GenericRecord rec = new GenericData.Record(isFlattened ? FLATTENED_AVRO_SCHEMA : AVRO_SCHEMA);
@@ -269,25 +304,25 @@ public class HoodieTestDataGenerator implements AutoCloseable {
rec.put("partition_path", partitionPath);
rec.put("rider", riderName);
rec.put("driver", driverName);
rec.put("begin_lat", RAND.nextDouble());
rec.put("begin_lon", RAND.nextDouble());
rec.put("end_lat", RAND.nextDouble());
rec.put("end_lon", RAND.nextDouble());
rec.put("begin_lat", rand.nextDouble());
rec.put("begin_lon", rand.nextDouble());
rec.put("end_lat", rand.nextDouble());
rec.put("end_lon", rand.nextDouble());
if (isFlattened) {
rec.put("fare", RAND.nextDouble() * 100);
rec.put("fare", rand.nextDouble() * 100);
rec.put("currency", "USD");
} else {
rec.put("distance_in_meters", RAND.nextInt());
rec.put("seconds_since_epoch", RAND.nextLong());
rec.put("weight", RAND.nextFloat());
rec.put("distance_in_meters", rand.nextInt());
rec.put("seconds_since_epoch", rand.nextLong());
rec.put("weight", rand.nextFloat());
byte[] bytes = "Canada".getBytes();
rec.put("nation", ByteBuffer.wrap(bytes));
long currentTimeMillis = System.currentTimeMillis();
Date date = new Date(currentTimeMillis);
rec.put("current_date", (int) date.toLocalDate().toEpochDay());
rec.put("current_ts", currentTimeMillis);
long randomMillis = genRandomTimeMillis(rand);
Instant instant = Instant.ofEpochMilli(randomMillis);
rec.put("current_date", (int) LocalDateTime.ofInstant(instant, ZoneOffset.UTC).toLocalDate().toEpochDay());
rec.put("current_ts", randomMillis);
BigDecimal bigDecimal = new BigDecimal(String.format("%5f", RAND.nextFloat()));
BigDecimal bigDecimal = new BigDecimal(String.format("%5f", rand.nextFloat()));
Schema decimalSchema = AVRO_SCHEMA.getField("height").schema();
Conversions.DecimalConversion decimalConversions = new Conversions.DecimalConversion();
GenericFixed genericFixed = decimalConversions.toFixed(bigDecimal, decimalSchema, LogicalTypes.decimal(10, 6));
@@ -296,14 +331,14 @@ public class HoodieTestDataGenerator implements AutoCloseable {
rec.put("city_to_state", Collections.singletonMap("LA", "CA"));
GenericRecord fareRecord = new GenericData.Record(AVRO_SCHEMA.getField("fare").schema());
fareRecord.put("amount", RAND.nextDouble() * 100);
fareRecord.put("amount", rand.nextDouble() * 100);
fareRecord.put("currency", "USD");
rec.put("fare", fareRecord);
GenericArray<GenericRecord> tipHistoryArray = new GenericData.Array<>(1, AVRO_SCHEMA.getField("tip_history").schema());
Schema tipSchema = new Schema.Parser().parse(AVRO_SCHEMA.getField("tip_history").schema().toString()).getElementType();
GenericRecord tipRecord = new GenericData.Record(tipSchema);
tipRecord.put("amount", RAND.nextDouble() * 100);
tipRecord.put("amount", rand.nextDouble() * 100);
tipRecord.put("currency", "USD");
tipHistoryArray.add(tipRecord);
rec.put("tip_history", tipHistoryArray);
@@ -326,7 +361,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
rec.put("timestamp", timestamp);
rec.put("rider", riderName);
rec.put("driver", driverName);
rec.put("fare", RAND.nextDouble() * 100);
rec.put("fare", rand.nextDouble() * 100);
rec.put("_hoodie_is_deleted", false);
return rec;
}
@@ -337,7 +372,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
rec.put("timestamp", timestamp);
rec.put("rider", riderName);
rec.put("driver", driverName);
rec.put("fare", RAND.nextDouble() * 100);
rec.put("fare", rand.nextDouble() * 100);
rec.put("_hoodie_is_deleted", false);
return rec;
}
@@ -347,7 +382,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
createCommitFile(basePath, instantTime, configuration, commitMetadata);
}
public static void createCommitFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) {
private static void createCommitFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) {
Arrays.asList(HoodieTimeline.makeCommitFileName(instantTime), HoodieTimeline.makeInflightCommitFileName(instantTime),
HoodieTimeline.makeRequestedCommitFileName(instantTime))
.forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata));
@@ -383,13 +418,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
}
}
public static void createReplaceFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) {
Arrays.asList(HoodieTimeline.makeReplaceFileName(instantTime), HoodieTimeline.makeInflightReplaceFileName(instantTime),
HoodieTimeline.makeRequestedReplaceFileName(instantTime))
.forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata));
}
public static void createPendingReplaceFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) {
private static void createPendingReplaceFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) {
Arrays.asList(HoodieTimeline.makeInflightReplaceFileName(instantTime),
HoodieTimeline.makeRequestedReplaceFileName(instantTime))
.forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata));
@@ -407,13 +436,6 @@ public class HoodieTestDataGenerator implements AutoCloseable {
createEmptyFile(basePath, commitFile, configuration);
}
public static void createCompactionRequestedFile(String basePath, String instantTime, Configuration configuration)
throws IOException {
Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
+ HoodieTimeline.makeRequestedCompactionFileName(instantTime));
createEmptyFile(basePath, commitFile, configuration);
}
private static void createEmptyFile(String basePath, Path filePath, Configuration configuration) throws IOException {
FileSystem fs = FSUtils.getFs(basePath, configuration);
FSDataOutputStream os = fs.create(filePath, true);
@@ -484,13 +506,13 @@ public class HoodieTestDataGenerator implements AutoCloseable {
}
public List<HoodieRecord> generateInsertsForPartition(String instantTime, Integer n, String partition) {
return generateInsertsStream(instantTime, n, false, TRIP_EXAMPLE_SCHEMA, false, () -> partition, () -> UUID.randomUUID().toString()).collect(Collectors.toList());
return generateInsertsStream(instantTime, n, false, TRIP_EXAMPLE_SCHEMA, false, () -> partition, () -> genPseudoRandomUUID(rand).toString()).collect(Collectors.toList());
}
public Stream<HoodieRecord> generateInsertsStream(String commitTime, Integer n, boolean isFlattened, String schemaStr, boolean containsAllPartitions) {
return generateInsertsStream(commitTime, n, isFlattened, schemaStr, containsAllPartitions,
() -> partitionPaths[RAND.nextInt(partitionPaths.length)],
() -> UUID.randomUUID().toString());
() -> partitionPaths[rand.nextInt(partitionPaths.length)],
() -> genPseudoRandomUUID(rand).toString());
}
/**
@@ -552,8 +574,8 @@ public class HoodieTestDataGenerator implements AutoCloseable {
List<HoodieRecord> inserts = new ArrayList<>();
int currSize = getNumExistingKeys(TRIP_EXAMPLE_SCHEMA);
for (int i = 0; i < limit; i++) {
String partitionPath = partitionPaths[RAND.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(genPseudoRandomUUID(rand).toString(), partitionPath);
HoodieRecord record = new HoodieAvroRecord(key, generateAvroPayload(key, instantTime));
inserts.add(record);
@@ -654,7 +676,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
for (int i = 0; i < n; i++) {
Map<Integer, KeyPartition> existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
KeyPartition kp = existingKeys.get(RAND.nextInt(numExistingKeys - 1));
KeyPartition kp = existingKeys.get(rand.nextInt(numExistingKeys - 1));
HoodieRecord record = generateUpdateRecord(kp.key, instantTime);
updates.add(record);
}
@@ -726,7 +748,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
}
return IntStream.range(0, n).boxed().map(i -> {
int index = numExistingKeys == 1 ? 0 : RAND.nextInt(numExistingKeys - 1);
int index = numExistingKeys == 1 ? 0 : rand.nextInt(numExistingKeys - 1);
KeyPartition kp = existingKeys.get(index);
// Find the available keyPartition starting from randomly chosen one.
while (used.contains(kp)) {
@@ -759,7 +781,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
List<HoodieKey> result = new ArrayList<>();
for (int i = 0; i < n; i++) {
int index = RAND.nextInt(numExistingKeys);
int index = rand.nextInt(numExistingKeys);
while (!existingKeys.containsKey(index)) {
index = (index + 1) % numExistingKeys;
}
@@ -791,7 +813,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
List<HoodieRecord> result = new ArrayList<>();
for (int i = 0; i < n; i++) {
int index = RAND.nextInt(numExistingKeys);
int index = rand.nextInt(numExistingKeys);
while (!existingKeys.containsKey(index)) {
index = (index + 1) % numExistingKeys;
}
@@ -841,8 +863,8 @@ public class HoodieTestDataGenerator implements AutoCloseable {
public List<GenericRecord> generateGenericRecords(int numRecords) {
List<GenericRecord> list = new ArrayList<>();
IntStream.range(0, numRecords).forEach(i -> {
list.add(generateGenericRecord(UUID.randomUUID().toString(), "0", UUID.randomUUID().toString(), UUID.randomUUID()
.toString(), RAND.nextLong()));
list.add(generateGenericRecord(genPseudoRandomUUID(rand).toString(), "0",
genPseudoRandomUUID(rand).toString(), genPseudoRandomUUID(rand).toString(), rand.nextLong()));
});
return list;
}
@@ -865,4 +887,31 @@ public class HoodieTestDataGenerator implements AutoCloseable {
public void close() {
existingKeysBySchema.clear();
}
private static long genRandomTimeMillis(Random r) {
// Fri Feb 13 15:31:30 PST 2009
long anchorTs = 1234567890L;
// NOTE: To provide for certainty and not generate overly random dates, we will limit
// dispersion to be w/in +/- 3 days from the anchor date
return anchorTs + r.nextLong() % 259200000L;
}
private static UUID genPseudoRandomUUID(Random r) {
byte[] bytes = new byte[16];
r.nextBytes(bytes);
bytes[6] &= 0x0f;
bytes[6] |= 0x40;
bytes[8] &= 0x3f;
bytes[8] |= 0x80;
try {
Constructor<UUID> ctor = UUID.class.getDeclaredConstructor(byte[].class);
ctor.setAccessible(true);
return ctor.newInstance((Object) bytes);
} catch (InvocationTargetException | InstantiationException | IllegalAccessException | NoSuchMethodException e) {
logger.info("Failed to generate pseudo-random UUID!");
throw new HoodieException(e);
}
}
}