1
0

Implement Savepoints and required metadata timeline - Part 2

This commit is contained in:
Prasanna Rajaperumal
2017-03-13 23:09:29 -07:00
parent 6f36e1eaaf
commit d83b671ada
7 changed files with 135 additions and 29 deletions

View File

@@ -3,7 +3,18 @@
"name": "HoodieSavepointMetadata",
"fields": [
{"name": "savepointedBy", "type": "string"},
{"name": "savepointedAt", "type": "string"},
{"name": "comments", "type": "string"}
{"name": "savepointedAt", "type": "long"},
{"name": "comments", "type": "string"},
{"name": "partitionMetadata", "type": {
"type" : "map", "values" : {
"type": "record",
"name": "HoodieSavepointPartitionMetadata",
"fields": [
{"name": "partitionPath", "type": "string"},
{"name": "savepointDataFile", "type": {"type": "array", "items": "string"}}
]
}
}
}
]
}

View File

@@ -24,9 +24,11 @@ import com.uber.hoodie.avro.model.HoodieCleanPartitionMetadata;
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
import com.uber.hoodie.avro.model.HoodieRollbackPartitionMetadata;
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
import com.uber.hoodie.avro.model.HoodieSavepointPartitionMetadata;
import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.HoodieRollbackStat;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.exception.HoodieIOException;
@@ -57,7 +59,9 @@ import java.io.ByteArrayOutputStream;
import java.io.FileWriter;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
public class AvroUtils {
@@ -129,6 +133,20 @@ public class AvroUtils {
totalDeleted, commits, partitionMetadataBuilder.build());
}
public static HoodieSavepointMetadata convertSavepointMetadata(String user, String comment,
Map<String, List<String>> latestFiles) {
ImmutableMap.Builder<String, HoodieSavepointPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.<String, HoodieSavepointPartitionMetadata>builder();
for (Map.Entry<String, List<String>> stat : latestFiles.entrySet()) {
HoodieSavepointPartitionMetadata metadata =
new HoodieSavepointPartitionMetadata(stat.getKey(), stat.getValue());
partitionMetadataBuilder.put(stat.getKey(), metadata);
}
return new HoodieSavepointMetadata(user, System.currentTimeMillis(), comment,
partitionMetadataBuilder.build());
}
public static Optional<byte[]> serializeCleanMetadata(HoodieCleanMetadata metadata)
throws IOException {
return serializeAvroMetadata(metadata, HoodieCleanMetadata.class);
@@ -160,6 +178,11 @@ public class AvroUtils {
return deserializeAvroMetadata(bytes, HoodieCleanMetadata.class);
}
public static HoodieSavepointMetadata deserializeHoodieSavepointMetadata(byte[] bytes)
throws IOException {
return deserializeAvroMetadata(bytes, HoodieSavepointMetadata.class);
}
public static <T extends SpecificRecordBase> T deserializeAvroMetadata(byte[] bytes,
Class<T> clazz) throws IOException {
DatumReader<T> reader = new SpecificDatumReader<>(clazz);