1
0

Reformatting code per Google Code Style all over

This commit is contained in:
Vinoth Chandar
2017-11-12 22:54:56 -08:00
committed by vinoth chandar
parent 5a62480a92
commit e45679f5e2
254 changed files with 21580 additions and 21108 deletions

View File

@@ -29,15 +29,6 @@ import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
import org.apache.spark.SparkConf;
import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
@@ -49,6 +40,12 @@ import java.util.Iterator;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
/**
* Utility methods to aid testing inside the HoodieClient module.
@@ -56,133 +53,142 @@ import java.util.stream.Collectors;
public class HoodieClientTestUtils {
public static List<WriteStatus> collectStatuses(Iterator<List<WriteStatus>> statusListItr) {
List<WriteStatus> statuses = new ArrayList<>();
while (statusListItr.hasNext()) {
statuses.addAll(statusListItr.next());
}
return statuses;
}
public static Set<String> getRecordKeys(List<HoodieRecord> hoodieRecords) {
Set<String> keys = new HashSet<>();
for (HoodieRecord rec: hoodieRecords) {
keys.add(rec.getRecordKey());
}
return keys;
}
private static void fakeMetaFile(String basePath, String commitTime, String suffix) throws IOException {
String parentPath = basePath + "/"+ HoodieTableMetaClient.METAFOLDER_NAME;
new File(parentPath).mkdirs();
new File(parentPath + "/" + commitTime + suffix).createNewFile();
}
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTimeline.COMMIT_EXTENSION);
}
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTimeline.INFLIGHT_EXTENSION);
}
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId) throws Exception {
fakeDataFile(basePath, partitionPath, commitTime, fileId, 0);
}
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId, long length) throws Exception {
String parentPath = String.format("%s/%s", basePath, partitionPath);
new File(parentPath).mkdirs();
String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId));
new File(path).createNewFile();
new RandomAccessFile(path, "rw").setLength(length);
}
public static SparkConf getSparkConfForTest(String appName) {
SparkConf sparkConf = new SparkConf()
.setAppName(appName)
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.setMaster("local[1]");
return HoodieReadClient.addHoodieSupport(sparkConf);
}
public static HashMap<String, String> getLatestFileIDsToFullPath(String basePath,
HoodieTimeline commitTimeline,
List<HoodieInstant> commitsToReturn) throws IOException {
HashMap<String, String> fileIdToFullPath = new HashMap<>();
for (HoodieInstant commit : commitsToReturn) {
HoodieCommitMetadata metadata =
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
}
return fileIdToFullPath;
}
public static Dataset<Row> readCommit(String basePath,
SQLContext sqlContext,
HoodieTimeline commitTimeline,
String commitTime) {
HoodieInstant commitInstant =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!commitTimeline.containsInstant(commitInstant)) {
new HoodieException("No commit exists at " + commitTime);
}
try {
HashMap<String, String> paths = getLatestFileIDsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant));
return sqlContext.read()
.parquet(paths.values().toArray(new String[paths.size()]))
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime));
} catch (Exception e) {
throw new HoodieException("Error reading commit " + commitTime, e);
}
}
/**
* Obtain all new data written into the Hoodie dataset since the given timestamp.
*/
public static Dataset<Row> readSince(String basePath,
SQLContext sqlContext,
HoodieTimeline commitTimeline,
String lastCommitTime) {
List<HoodieInstant> commitsToReturn =
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE)
.getInstants().collect(Collectors.toList());
try {
// Go over the commit metadata, and obtain the new files that need to be read.
HashMap<String, String> fileIdToFullPath = getLatestFileIDsToFullPath(basePath, commitTimeline, commitsToReturn);
return sqlContext.read()
.parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]))
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime));
} catch (IOException e) {
throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e);
}
}
/**
* Reads the paths under the a hoodie dataset out as a DataFrame
*/
public static Dataset<Row> read(String basePath,
SQLContext sqlContext,
FileSystem fs,
String... paths) {
List<String> filteredPaths = new ArrayList<>();
try {
HoodieTable hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
for (String path : paths) {
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(hoodieTable.getMetaClient(),
hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path)));
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(
Collectors.toList());
for (HoodieDataFile file : latestFiles) {
filteredPaths.add(file.getPath());
}
}
return sqlContext.read()
.parquet(filteredPaths.toArray(new String[filteredPaths.size()]));
} catch (Exception e) {
throw new HoodieException("Error reading hoodie dataset as a dataframe", e);
public static List<WriteStatus> collectStatuses(Iterator<List<WriteStatus>> statusListItr) {
List<WriteStatus> statuses = new ArrayList<>();
while (statusListItr.hasNext()) {
statuses.addAll(statusListItr.next());
}
return statuses;
}
public static Set<String> getRecordKeys(List<HoodieRecord> hoodieRecords) {
Set<String> keys = new HashSet<>();
for (HoodieRecord rec : hoodieRecords) {
keys.add(rec.getRecordKey());
}
return keys;
}
private static void fakeMetaFile(String basePath, String commitTime, String suffix)
throws IOException {
String parentPath = basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME;
new File(parentPath).mkdirs();
new File(parentPath + "/" + commitTime + suffix).createNewFile();
}
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTimeline.COMMIT_EXTENSION);
}
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTimeline.INFLIGHT_EXTENSION);
}
public static void fakeDataFile(String basePath, String partitionPath, String commitTime,
String fileId) throws Exception {
fakeDataFile(basePath, partitionPath, commitTime, fileId, 0);
}
public static void fakeDataFile(String basePath, String partitionPath, String commitTime,
String fileId, long length) throws Exception {
String parentPath = String.format("%s/%s", basePath, partitionPath);
new File(parentPath).mkdirs();
String path = String
.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId));
new File(path).createNewFile();
new RandomAccessFile(path, "rw").setLength(length);
}
public static SparkConf getSparkConfForTest(String appName) {
SparkConf sparkConf = new SparkConf()
.setAppName(appName)
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.setMaster("local[1]");
return HoodieReadClient.addHoodieSupport(sparkConf);
}
public static HashMap<String, String> getLatestFileIDsToFullPath(String basePath,
HoodieTimeline commitTimeline,
List<HoodieInstant> commitsToReturn) throws IOException {
HashMap<String, String> fileIdToFullPath = new HashMap<>();
for (HoodieInstant commit : commitsToReturn) {
HoodieCommitMetadata metadata =
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
}
return fileIdToFullPath;
}
public static Dataset<Row> readCommit(String basePath,
SQLContext sqlContext,
HoodieTimeline commitTimeline,
String commitTime) {
HoodieInstant commitInstant =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!commitTimeline.containsInstant(commitInstant)) {
new HoodieException("No commit exists at " + commitTime);
}
try {
HashMap<String, String> paths = getLatestFileIDsToFullPath(basePath, commitTimeline,
Arrays.asList(commitInstant));
return sqlContext.read()
.parquet(paths.values().toArray(new String[paths.size()]))
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime));
} catch (Exception e) {
throw new HoodieException("Error reading commit " + commitTime, e);
}
}
/**
* Obtain all new data written into the Hoodie dataset since the given timestamp.
*/
public static Dataset<Row> readSince(String basePath,
SQLContext sqlContext,
HoodieTimeline commitTimeline,
String lastCommitTime) {
List<HoodieInstant> commitsToReturn =
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE)
.getInstants().collect(Collectors.toList());
try {
// Go over the commit metadata, and obtain the new files that need to be read.
HashMap<String, String> fileIdToFullPath = getLatestFileIDsToFullPath(basePath,
commitTimeline, commitsToReturn);
return sqlContext.read()
.parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]))
.filter(
String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime));
} catch (IOException e) {
throw new HoodieException(
"Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e);
}
}
/**
* Reads the paths under the a hoodie dataset out as a DataFrame
*/
public static Dataset<Row> read(String basePath,
SQLContext sqlContext,
FileSystem fs,
String... paths) {
List<String> filteredPaths = new ArrayList<>();
try {
HoodieTable hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
for (String path : paths) {
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(
hoodieTable.getMetaClient(),
hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path)));
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(
Collectors.toList());
for (HoodieDataFile file : latestFiles) {
filteredPaths.add(file.getPath());
}
}
return sqlContext.read()
.parquet(filteredPaths.toArray(new String[filteredPaths.size()]));
} catch (Exception e) {
throw new HoodieException("Error reading hoodie dataset as a dataframe", e);
}
}
}

View File

@@ -16,9 +16,16 @@
package com.uber.hoodie.common;
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.GenericRecordBuilder;
@@ -30,69 +37,64 @@ import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
/**
* Utility methods to aid in testing MergeOnRead (workaround for HoodieReadClient for MOR)
*/
public class HoodieMergeOnReadTestUtils {
public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths) throws IOException {
JobConf jobConf = new JobConf();
Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA));
HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat();
setPropsForInputFormat(inputFormat, jobConf, schema);
return inputPaths.stream().map(path -> {
setInputPath(jobConf, path);
List<GenericRecord> records = new ArrayList<>();
try {
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
RecordReader recordReader = inputFormat.getRecordReader(splits.get(0), jobConf, null);
Void key = (Void) recordReader.createKey();
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
while (recordReader.next(key, writable)) {
GenericRecordBuilder newRecord = new GenericRecordBuilder(schema);
// writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno]
Writable[] values = writable.get();
schema.getFields().forEach(field -> {
newRecord.set(field, values[2]);
});
records.add(newRecord.build());
}
} catch (IOException ie) {
ie.printStackTrace();
}
return records;
}).reduce((a, b) -> {
a.addAll(b);
return a;
}).get();
}
public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths)
throws IOException {
JobConf jobConf = new JobConf();
Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA));
HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat();
setPropsForInputFormat(inputFormat, jobConf, schema);
return inputPaths.stream().map(path -> {
setInputPath(jobConf, path);
List<GenericRecord> records = new ArrayList<>();
try {
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
RecordReader recordReader = inputFormat.getRecordReader(splits.get(0), jobConf, null);
Void key = (Void) recordReader.createKey();
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
while (recordReader.next(key, writable)) {
GenericRecordBuilder newRecord = new GenericRecordBuilder(schema);
// writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno]
Writable[] values = writable.get();
schema.getFields().forEach(field -> {
newRecord.set(field, values[2]);
});
records.add(newRecord.build());
}
} catch (IOException ie) {
ie.printStackTrace();
}
return records;
}).reduce((a, b) -> {
a.addAll(b);
return a;
}).get();
}
private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, Schema schema) {
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
Configuration conf = FSUtils.getFs().getConf();
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set("partition_columns", "datestr");
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
conf.set("partition_columns", "datestr");
inputFormat.setConf(conf);
jobConf.addResource(conf);
}
private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf,
Schema schema) {
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos()))
.collect(Collectors.joining(","));
Configuration conf = FSUtils.getFs().getConf();
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set("partition_columns", "datestr");
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
conf.set("partition_columns", "datestr");
inputFormat.setConf(conf);
jobConf.addResource(conf);
}
private static void setInputPath(JobConf jobConf, String inputPath) {
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("map.input.dir", inputPath);
}
private static void setInputPath(JobConf jobConf, String inputPath) {
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("map.input.dir", inputPath);
}
}

View File

@@ -16,17 +16,21 @@
package com.uber.hoodie.common;
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.Random;
import java.util.UUID;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
@@ -34,15 +38,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import java.util.Random;
import java.util.UUID;
/**
* Class to be used in tests to keep generating test inserts and updates against a corpus.
*
@@ -51,153 +46,164 @@ import java.util.UUID;
public class HoodieTestDataGenerator {
static class KeyPartition {
HoodieKey key;
String partitionPath;
HoodieKey key;
String partitionPath;
}
public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\","
+ "\"name\": \"triprec\","
+ "\"fields\": [ "
+ "{\"name\": \"timestamp\",\"type\": \"double\"},"
+ "{\"name\": \"_row_key\", \"type\": \"string\"},"
+ "{\"name\": \"rider\", \"type\": \"string\"},"
+ "{\"name\": \"driver\", \"type\": \"string\"},"
+ "{\"name\": \"begin_lat\", \"type\": \"double\"},"
+ "{\"name\": \"begin_lon\", \"type\": \"double\"},"
+ "{\"name\": \"end_lat\", \"type\": \"double\"},"
+ "{\"name\": \"end_lon\", \"type\": \"double\"},"
+ "{\"name\":\"fare\",\"type\": \"double\"}]}";
// based on examination of sample file, the schema produces the following per record size
public static final int SIZE_PER_RECORD = 50 * 1024;
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
public static 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);
}
}
public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\","
+ "\"name\": \"triprec\","
+ "\"fields\": [ "
+ "{\"name\": \"timestamp\",\"type\": \"double\"},"
+ "{\"name\": \"_row_key\", \"type\": \"string\"},"
+ "{\"name\": \"rider\", \"type\": \"string\"},"
+ "{\"name\": \"driver\", \"type\": \"string\"},"
+ "{\"name\": \"begin_lat\", \"type\": \"double\"},"
+ "{\"name\": \"begin_lon\", \"type\": \"double\"},"
+ "{\"name\": \"end_lat\", \"type\": \"double\"},"
+ "{\"name\": \"end_lon\", \"type\": \"double\"},"
+ "{\"name\":\"fare\",\"type\": \"double\"}]}";
private List<KeyPartition> existingKeysList = new ArrayList<>();
public static Schema avroSchema = HoodieAvroUtils
.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA));
private static Random rand = new Random(46474747);
private String[] partitionPaths = DEFAULT_PARTITION_PATHS;
// based on examination of sample file, the schema produces the following per record size
public static final int SIZE_PER_RECORD = 50 * 1024;
public HoodieTestDataGenerator(String[] partitionPaths) {
this.partitionPaths = partitionPaths;
}
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
public HoodieTestDataGenerator() {
this(new String[]{"2016/03/15", "2015/03/16", "2015/03/17"});
}
public static 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);
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list of
* existing keys.
*/
public List<HoodieRecord> generateInserts(String commitTime, int n) throws IOException {
List<HoodieRecord> inserts = new ArrayList<>();
for (int i = 0; i < n; i++) {
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime));
inserts.add(record);
KeyPartition kp = new KeyPartition();
kp.key = key;
kp.partitionPath = partitionPath;
existingKeysList.add(kp);
}
return inserts;
}
private List<KeyPartition> existingKeysList = new ArrayList<>();
public static Schema avroSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA));
private static Random rand = new Random(46474747);
private String[] partitionPaths = DEFAULT_PARTITION_PATHS;
public List<HoodieRecord> generateDeletes(String commitTime, int n) throws IOException {
List<HoodieRecord> inserts = generateInserts(commitTime, n);
return generateDeletesFromExistingRecords(inserts);
}
public List<HoodieRecord> generateDeletesFromExistingRecords(List<HoodieRecord> existingRecords)
throws IOException {
List<HoodieRecord> deletes = new ArrayList<>();
for (HoodieRecord existingRecord : existingRecords) {
HoodieRecord record = generateDeleteRecord(existingRecord);
deletes.add(record);
public HoodieTestDataGenerator(String[] partitionPaths) {
this.partitionPaths = partitionPaths;
}
return deletes;
}
public HoodieTestDataGenerator() {
this(new String[]{"2016/03/15", "2015/03/16", "2015/03/17"});
public HoodieRecord generateDeleteRecord(HoodieRecord existingRecord) throws IOException {
HoodieKey key = existingRecord.getKey();
TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(),
key.getPartitionPath(), null, true);
return new HoodieRecord(key, payload);
}
public List<HoodieRecord> generateUpdates(String commitTime, List<HoodieRecord> baseRecords)
throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (HoodieRecord baseRecord : baseRecords) {
HoodieRecord record = new HoodieRecord(baseRecord.getKey(),
generateRandomValue(baseRecord.getKey(), commitTime));
updates.add(record);
}
return updates;
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list
* of existing keys.
*/
public List<HoodieRecord> generateInserts(String commitTime, int n) throws IOException {
List<HoodieRecord> inserts = new ArrayList<>();
for (int i = 0; i < n; i++) {
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime));
inserts.add(record);
KeyPartition kp = new KeyPartition();
kp.key = key;
kp.partitionPath = partitionPath;
existingKeysList.add(kp);
}
return inserts;
/**
* Generates new updates, randomly distributed across the keys above.
*/
public List<HoodieRecord> generateUpdates(String commitTime, int n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (int i = 0; i < n; i++) {
KeyPartition kp = existingKeysList.get(rand.nextInt(existingKeysList.size() - 1));
HoodieRecord record = new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime));
updates.add(record);
}
return updates;
}
public List<HoodieRecord> generateDeletes(String commitTime, int n) throws IOException {
List<HoodieRecord> inserts = generateInserts(commitTime, n);
return generateDeletesFromExistingRecords(inserts);
}
public List<HoodieRecord> generateDeletesFromExistingRecords(List<HoodieRecord> existingRecords) throws IOException {
List<HoodieRecord> deletes = new ArrayList<>();
for (HoodieRecord existingRecord: existingRecords) {
HoodieRecord record = generateDeleteRecord(existingRecord);
deletes.add(record);
}
return deletes;
}
public HoodieRecord generateDeleteRecord(HoodieRecord existingRecord) throws IOException {
HoodieKey key = existingRecord.getKey();
TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(), key.getPartitionPath(), null, true);
return new HoodieRecord(key, payload);
}
public List<HoodieRecord> generateUpdates(String commitTime, List<HoodieRecord> baseRecords) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (HoodieRecord baseRecord: baseRecords) {
HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateRandomValue(baseRecord.getKey(), commitTime));
updates.add(record);
}
return updates;
}
/**
* Generates new updates, randomly distributed across the keys above.
*/
public List<HoodieRecord> generateUpdates(String commitTime, int n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (int i = 0; i < n; i++) {
KeyPartition kp = existingKeysList.get(rand.nextInt(existingKeysList.size() - 1));
HoodieRecord record = new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime));
updates.add(record);
}
return updates;
}
/**
* Generates a new avro record of the above schema format, retaining the key if optionally
* provided.
*/
public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) throws IOException {
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime,
"driver-" + commitTime, 0.0);
HoodieAvroUtils.addCommitMetadataToRecord(rec, commitTime, "-1");
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
}
public static GenericRecord generateGenericRecord(String rowKey, String riderName,
String driverName, double timestamp) {
GenericRecord rec = new GenericData.Record(avroSchema);
rec.put("_row_key", rowKey);
rec.put("timestamp", timestamp);
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("fare", rand.nextDouble() * 100);
return rec;
}
public static void createCommitFile(String basePath, String commitTime) throws IOException {
Path commitFile =
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime));
FileSystem fs = FSUtils.getFs();
FSDataOutputStream os = fs.create(commitFile, true);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
try {
// Write empty commit metadata
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(
StandardCharsets.UTF_8)));
} finally {
os.close();
}
/**
* Generates a new avro record of the above schema format, retaining the key if optionally
* provided.
*/
public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime)
throws IOException {
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime,
"driver-" + commitTime, 0.0);
HoodieAvroUtils.addCommitMetadataToRecord(rec, commitTime, "-1");
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(),
TRIP_EXAMPLE_SCHEMA);
}
public static GenericRecord generateGenericRecord(String rowKey, String riderName,
String driverName, double timestamp) {
GenericRecord rec = new GenericData.Record(avroSchema);
rec.put("_row_key", rowKey);
rec.put("timestamp", timestamp);
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("fare", rand.nextDouble() * 100);
return rec;
}
public static void createCommitFile(String basePath, String commitTime) throws IOException {
Path commitFile =
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeCommitFileName(commitTime));
FileSystem fs = FSUtils.getFs();
FSDataOutputStream os = fs.create(commitFile, true);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
try {
// Write empty commit metadata
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(
StandardCharsets.UTF_8)));
} finally {
os.close();
}
}
public static void createSavepointFile(String basePath, String commitTime) throws IOException {
Path commitFile =
@@ -215,7 +221,7 @@ public class HoodieTestDataGenerator {
}
}
public String[] getPartitionPaths() {
return partitionPaths;
}
public String[] getPartitionPaths() {
return partitionPaths;
}
}

View File

@@ -17,174 +17,182 @@
package com.uber.hoodie.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.avro.MercifulJsonConverter;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.StringWriter;
import java.util.HashMap;
import java.util.List;
import java.util.Map.Entry;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.IOUtils;
import java.io.*;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.zip.Deflater;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.IOUtils;
/**
* Example row change event based on some example data used by testcases. The data avro schema is
* src/test/resources/schema1.
*/
public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayload> {
private transient static final ObjectMapper mapper = new ObjectMapper();
private String partitionPath;
private String rowKey;
private byte[] jsonDataCompressed;
private int dataSize;
private boolean isDeleted;
public TestRawTripPayload(Optional<String> jsonData, String rowKey, String partitionPath,
String schemaStr, Boolean isDeleted) throws IOException {
if(jsonData.isPresent()) {
this.jsonDataCompressed = compressData(jsonData.get());
this.dataSize = jsonData.get().length();
}
this.rowKey = rowKey;
this.partitionPath = partitionPath;
this.isDeleted = isDeleted;
private transient static final ObjectMapper mapper = new ObjectMapper();
private String partitionPath;
private String rowKey;
private byte[] jsonDataCompressed;
private int dataSize;
private boolean isDeleted;
public TestRawTripPayload(Optional<String> jsonData, String rowKey, String partitionPath,
String schemaStr, Boolean isDeleted) throws IOException {
if (jsonData.isPresent()) {
this.jsonDataCompressed = compressData(jsonData.get());
this.dataSize = jsonData.get().length();
}
this.rowKey = rowKey;
this.partitionPath = partitionPath;
this.isDeleted = isDeleted;
}
public TestRawTripPayload(String jsonData, String rowKey, String partitionPath,
String schemaStr)throws IOException {
this(Optional.of(jsonData), rowKey, partitionPath, schemaStr, false);
public TestRawTripPayload(String jsonData, String rowKey, String partitionPath,
String schemaStr) throws IOException {
this(Optional.of(jsonData), rowKey, partitionPath, schemaStr, false);
}
public TestRawTripPayload(String jsonData) throws IOException {
this.jsonDataCompressed = compressData(jsonData);
this.dataSize = jsonData.length();
Map<String, Object> jsonRecordMap = mapper.readValue(jsonData, Map.class);
this.rowKey = jsonRecordMap.get("_row_key").toString();
this.partitionPath = jsonRecordMap.get("time").toString().split("T")[0].replace("-", "/");
this.isDeleted = false;
}
public String getPartitionPath() {
return partitionPath;
}
@Override
public TestRawTripPayload preCombine(TestRawTripPayload another) {
return another;
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema)
throws IOException {
return this.getInsertValue(schema);
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
if (isDeleted) {
return Optional.empty();
} else {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return Optional.of(jsonConverter.convert(getJsonData()));
}
}
public TestRawTripPayload(String jsonData) throws IOException {
this.jsonDataCompressed = compressData(jsonData);
this.dataSize = jsonData.length();
Map<String, Object> jsonRecordMap = mapper.readValue(jsonData, Map.class);
this.rowKey = jsonRecordMap.get("_row_key").toString();
this.partitionPath = jsonRecordMap.get("time").toString().split("T")[0].replace("-", "/");
this.isDeleted = false;
}
public String getPartitionPath() {
return partitionPath;
@Override
public Optional<Map<String, String>> getMetadata() {
// Let's assume we want to count the number of input row change events
// that are processed. Let the time-bucket for this row change event be 1506582000.
Map<String, String> metadataMap = new HashMap<>();
metadataMap.put("InputRecordCount_1506582000", "2");
return Optional.of(metadataMap);
}
public String getRowKey() {
return rowKey;
}
public String getJsonData() throws IOException {
return unCompressData(jsonDataCompressed);
}
private byte[] compressData(String jsonData) throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DeflaterOutputStream dos =
new DeflaterOutputStream(baos, new Deflater(Deflater.BEST_COMPRESSION), true);
try {
dos.write(jsonData.getBytes());
} finally {
dos.flush();
dos.close();
}
return baos.toByteArray();
}
@Override public TestRawTripPayload preCombine(TestRawTripPayload another) {
return another;
}
private String unCompressData(byte[] data) throws IOException {
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
StringWriter sw = new StringWriter(dataSize);
IOUtils.copy(iis, sw);
return sw.toString();
}
@Override public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
return this.getInsertValue(schema);
}
/**
* A custom {@link WriteStatus} that merges passed metadata key value map to {@code
* WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()}.
*/
public static class MetadataMergeWriteStatus extends WriteStatus {
@Override public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
if(isDeleted){
return Optional.empty();
} else {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return Optional.of(jsonConverter.convert(getJsonData()));
}
private Map<String, String> mergedMetadataMap = new HashMap<>();
@Override
public void markSuccess(HoodieRecord record, Optional<Map<String, String>> recordMetadata) {
super.markSuccess(record, recordMetadata);
if (recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
@Override
public Optional<Map<String, String>> getMetadata() {
// Let's assume we want to count the number of input row change events
// that are processed. Let the time-bucket for this row change event be 1506582000.
Map<String, String> metadataMap = new HashMap<>();
metadataMap.put("InputRecordCount_1506582000", "2");
return Optional.of(metadataMap);
public void markFailure(HoodieRecord record, Throwable t,
Optional<Map<String, String>> recordMetadata) {
super.markFailure(record, t, recordMetadata);
if (recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
public String getRowKey() {
return rowKey;
public static Map<String, String> mergeMetadataForWriteStatuses(
List<WriteStatus> writeStatuses) {
Map<String, String> allWriteStatusMergedMetadataMap = new HashMap<>();
for (WriteStatus writeStatus : writeStatuses) {
MetadataMergeWriteStatus.mergeMetadataMaps(
((MetadataMergeWriteStatus) writeStatus).getMergedMetadataMap(),
allWriteStatusMergedMetadataMap);
}
return allWriteStatusMergedMetadataMap;
}
public String getJsonData() throws IOException {
return unCompressData(jsonDataCompressed);
private static void mergeMetadataMaps(Map<String, String> mergeFromMap,
Map<String, String> mergeToMap) {
for (Entry<String, String> entry : mergeFromMap.entrySet()) {
String key = entry.getKey();
if (!mergeToMap.containsKey(key)) {
mergeToMap.put(key, "0");
}
mergeToMap
.put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key)));
}
}
private byte[] compressData(String jsonData) throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DeflaterOutputStream dos =
new DeflaterOutputStream(baos, new Deflater(Deflater.BEST_COMPRESSION), true);
try {
dos.write(jsonData.getBytes());
} finally {
dos.flush();
dos.close();
}
return baos.toByteArray();
private Map<String, String> getMergedMetadataMap() {
return mergedMetadataMap;
}
private String unCompressData(byte[] data) throws IOException {
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
StringWriter sw = new StringWriter(dataSize);
IOUtils.copy(iis, sw);
return sw.toString();
}
/**
* A custom {@link WriteStatus} that merges passed metadata key value map
* to {@code WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()}.
*/
public static class MetadataMergeWriteStatus extends WriteStatus {
private Map<String, String> mergedMetadataMap = new HashMap<>();
@Override
public void markSuccess(HoodieRecord record, Optional<Map<String, String>> recordMetadata) {
super.markSuccess(record, recordMetadata);
if(recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
@Override
public void markFailure(HoodieRecord record, Throwable t,
Optional<Map<String, String>> recordMetadata) {
super.markFailure(record, t, recordMetadata);
if(recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
public static Map<String, String> mergeMetadataForWriteStatuses(List<WriteStatus> writeStatuses) {
Map<String, String> allWriteStatusMergedMetadataMap = new HashMap<>();
for (WriteStatus writeStatus : writeStatuses) {
MetadataMergeWriteStatus.mergeMetadataMaps(
((MetadataMergeWriteStatus)writeStatus).getMergedMetadataMap(),
allWriteStatusMergedMetadataMap);
}
return allWriteStatusMergedMetadataMap;
}
private static void mergeMetadataMaps(Map<String, String> mergeFromMap, Map<String, String> mergeToMap) {
for (Entry<String, String> entry : mergeFromMap.entrySet()) {
String key = entry.getKey();
if(!mergeToMap.containsKey(key)) {
mergeToMap.put(key, "0");
}
mergeToMap
.put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key)));
}
}
private Map<String, String> getMergedMetadataMap() {
return mergedMetadataMap;
}
private static String addStrsAsInt(String a, String b) {
return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b));
}
private static String addStrsAsInt(String a, String b) {
return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b));
}
}
}