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

@@ -15,128 +15,130 @@
~ limitations under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>hoodie</artifactId>
<groupId>com.uber.hoodie</groupId>
<version>0.4.1-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>hoodie</artifactId>
<groupId>com.uber.hoodie</groupId>
<version>0.4.1-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>hoodie-common</artifactId>
<artifactId>hoodie-common</artifactId>
<build>
<plugins>
<plugin>
<groupId>org.jacoco</groupId>
<artifactId>jacoco-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<version>2.5</version>
<executions>
<execution>
<goals>
<goal>test-jar</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<configuration>
<imports>
<!-- import avro files -->
<import>${basedir}/src/main/avro/HoodieCommitMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieSavePointMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieCompactionMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieCleanMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieRollbackMetadata.avsc</import>
</imports>
</configuration>
</plugin>
</plugins>
</build>
<build>
<plugins>
<plugin>
<groupId>org.jacoco</groupId>
<artifactId>jacoco-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<version>2.5</version>
<executions>
<execution>
<goals>
<goal>test-jar</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<configuration>
<imports>
<!-- import avro files -->
<import>${basedir}/src/main/avro/HoodieCommitMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieSavePointMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieCompactionMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieCleanMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieRollbackMetadata.avsc</import>
</imports>
</configuration>
</plugin>
</plugins>
</build>
<dependencies>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<exclusions>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<version>${junit.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
</dependency>
<dependency>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-mapper-asl</artifactId>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-avro</artifactId>
<version>${parquet.version}</version>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-all</artifactId>
<version>1.10.19</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
<classifier>tests</classifier>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<classifier>tests</classifier>
</dependency>
<dependency>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-core-asl</artifactId>
<version>1.9.13</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
<groupId>com.esotericsoftware</groupId>
<artifactId>kryo</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<exclusions>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>
<dependencies>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<exclusions>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<version>${junit.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
</dependency>
<dependency>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-mapper-asl</artifactId>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-avro</artifactId>
<version>${parquet.version}</version>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-all</artifactId>
<version>1.10.19</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
<classifier>tests</classifier>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<classifier>tests</classifier>
</dependency>
<dependency>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-core-asl</artifactId>
<version>1.9.13</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
<groupId>com.esotericsoftware</groupId>
<artifactId>kryo</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<exclusions>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>
</project>

View File

@@ -17,60 +17,59 @@
package com.uber.hoodie.avro;
import com.uber.hoodie.common.BloomFilter;
import java.util.HashMap;
import org.apache.avro.Schema;
import org.apache.parquet.avro.AvroWriteSupport;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.schema.MessageType;
import java.util.HashMap;
/**
* Wrap AvroWriterSupport for plugging in the bloom filter.
*/
public class HoodieAvroWriteSupport extends AvroWriteSupport {
private BloomFilter bloomFilter;
private String minRecordKey;
private String maxRecordKey;
private BloomFilter bloomFilter;
private String minRecordKey;
private String maxRecordKey;
public final static String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY =
"com.uber.hoodie.bloomfilter";
public final static String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key";
public final static String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key";
public final static String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY =
"com.uber.hoodie.bloomfilter";
public final static String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key";
public final static String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key";
public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, BloomFilter bloomFilter) {
super(schema, avroSchema);
this.bloomFilter = bloomFilter;
public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, BloomFilter bloomFilter) {
super(schema, avroSchema);
this.bloomFilter = bloomFilter;
}
@Override
public WriteSupport.FinalizedWriteContext finalizeWrite() {
HashMap<String, String> extraMetaData = new HashMap<>();
if (bloomFilter != null) {
extraMetaData
.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilter.serializeToString());
if (minRecordKey != null && maxRecordKey != null) {
extraMetaData.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey);
extraMetaData.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey);
}
}
return new WriteSupport.FinalizedWriteContext(extraMetaData);
}
public void add(String recordKey) {
this.bloomFilter.add(recordKey);
if (minRecordKey != null) {
minRecordKey = minRecordKey.compareTo(recordKey) <= 0 ? minRecordKey : recordKey;
} else {
minRecordKey = recordKey;
}
@Override
public WriteSupport.FinalizedWriteContext finalizeWrite() {
HashMap<String, String> extraMetaData = new HashMap<>();
if (bloomFilter != null) {
extraMetaData
.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilter.serializeToString());
if (minRecordKey != null && maxRecordKey != null) {
extraMetaData.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey);
extraMetaData.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey);
}
}
return new WriteSupport.FinalizedWriteContext(extraMetaData);
}
public void add(String recordKey) {
this.bloomFilter.add(recordKey);
if (minRecordKey != null) {
minRecordKey = minRecordKey.compareTo(recordKey) <= 0 ? minRecordKey : recordKey;
} else {
minRecordKey = recordKey;
}
if (maxRecordKey != null) {
maxRecordKey = maxRecordKey.compareTo(recordKey) >= 0 ? maxRecordKey : recordKey;
} else {
maxRecordKey = recordKey;
}
if (maxRecordKey != null) {
maxRecordKey = maxRecordKey.compareTo(recordKey) >= 0 ? maxRecordKey : recordKey;
} else {
maxRecordKey = recordKey;
}
}
}

View File

@@ -17,151 +17,148 @@
package com.uber.hoodie.avro;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.GenericRecordBuilder;
import org.codehaus.jackson.map.ObjectMapper;
/**
* Marjority of this is copied from
* https://github.com/jwills/avro-json/blob/master/src/main/java/com/cloudera/science/avro/common/JsonConverter.java
* Marjority of this is copied from https://github.com/jwills/avro-json/blob/master/src/main/java/com/cloudera/science/avro/common/JsonConverter.java
* Adjusted for expected behavior of our use cases
*/
public class MercifulJsonConverter {
private final ObjectMapper mapper = new ObjectMapper();
private final Schema baseSchema;
public MercifulJsonConverter(Schema schema) {
this.baseSchema = schema;
private final ObjectMapper mapper = new ObjectMapper();
private final Schema baseSchema;
public MercifulJsonConverter(Schema schema) {
this.baseSchema = schema;
}
public GenericRecord convert(String json) throws IOException {
try {
return convert(mapper.readValue(json, Map.class), baseSchema);
} catch (IOException e) {
throw new IOException("Failed to parse as Json: " + json + "\n\n" + e.getMessage());
}
}
private GenericRecord convert(Map<String, Object> raw, Schema schema)
throws IOException {
GenericRecord result = new GenericData.Record(schema);
for (Schema.Field f : schema.getFields()) {
String name = f.name();
Object rawValue = raw.get(name);
if (rawValue != null) {
result.put(f.pos(), typeConvert(rawValue, name, f.schema()));
}
}
return result;
}
public GenericRecord convert(String json) throws IOException {
try {
return convert(mapper.readValue(json, Map.class), baseSchema);
} catch (IOException e) {
throw new IOException("Failed to parse as Json: " + json + "\n\n" + e.getMessage());
private Object typeConvert(Object value, String name, Schema schema) throws IOException {
if (isOptional(schema)) {
if (value == null) {
return null;
} else {
schema = getNonNull(schema);
}
} else if (value == null) {
// Always fail on null for non-nullable schemas
throw new JsonConversionException(null, name, schema);
}
switch (schema.getType()) {
case BOOLEAN:
if (value instanceof Boolean) {
return (Boolean) value;
}
}
private GenericRecord convert(Map<String, Object> raw, Schema schema)
throws IOException {
GenericRecord result = new GenericData.Record(schema);
for (Schema.Field f : schema.getFields()) {
String name = f.name();
Object rawValue = raw.get(name);
if (rawValue != null) {
result.put(f.pos(), typeConvert(rawValue, name, f.schema()));
}
break;
case DOUBLE:
if (value instanceof Number) {
return ((Number) value).doubleValue();
}
return result;
}
private Object typeConvert(Object value, String name, Schema schema) throws IOException {
if (isOptional(schema)) {
if (value == null) {
return null;
} else {
schema = getNonNull(schema);
}
} else if (value == null) {
// Always fail on null for non-nullable schemas
throw new JsonConversionException(null, name, schema);
break;
case FLOAT:
if (value instanceof Number) {
return ((Number) value).floatValue();
}
switch (schema.getType()) {
case BOOLEAN:
if (value instanceof Boolean) {
return (Boolean) value;
}
break;
case DOUBLE:
if (value instanceof Number) {
return ((Number) value).doubleValue();
}
break;
case FLOAT:
if (value instanceof Number) {
return ((Number) value).floatValue();
}
break;
case INT:
if (value instanceof Number) {
return ((Number) value).intValue();
}
break;
case LONG:
if (value instanceof Number) {
return ((Number) value).longValue();
}
break;
case STRING:
return value.toString();
case ENUM:
if (schema.getEnumSymbols().contains(value.toString())) {
return new GenericData.EnumSymbol(schema, value.toString());
}
throw new JsonConversionException(String.format("Symbol %s not in enum", value.toString()),
schema.getFullName(), schema);
case RECORD:
return convert((Map<String, Object>) value, schema);
case ARRAY:
Schema elementSchema = schema.getElementType();
List listRes = new ArrayList();
for (Object v : (List) value) {
listRes.add(typeConvert(v, name, elementSchema));
}
return listRes;
case MAP:
Schema valueSchema = schema.getValueType();
Map<String, Object> mapRes = new HashMap<String, Object>();
for (Map.Entry<String, Object> v : ((Map<String, Object>) value).entrySet()) {
mapRes.put(v.getKey(), typeConvert(v.getValue(), name, valueSchema));
}
return mapRes;
default:
throw new IllegalArgumentException(
"JsonConverter cannot handle type: " + schema.getType());
break;
case INT:
if (value instanceof Number) {
return ((Number) value).intValue();
}
throw new JsonConversionException(value, name, schema);
}
private boolean isOptional(Schema schema) {
return schema.getType().equals(Schema.Type.UNION) &&
schema.getTypes().size() == 2 &&
(schema.getTypes().get(0).getType().equals(Schema.Type.NULL) ||
schema.getTypes().get(1).getType().equals(Schema.Type.NULL));
}
private Schema getNonNull(Schema schema) {
List<Schema> types = schema.getTypes();
return types.get(0).getType().equals(Schema.Type.NULL) ? types.get(1) : types.get(0);
}
public static class JsonConversionException extends RuntimeException {
private Object value;
private String fieldName;
private Schema schema;
public JsonConversionException(Object value, String fieldName, Schema schema) {
this.value = value;
this.fieldName = fieldName;
this.schema = schema;
break;
case LONG:
if (value instanceof Number) {
return ((Number) value).longValue();
}
@Override
public String toString() {
return String.format("Type conversion error for field %s, %s for %s",
fieldName, value, schema);
break;
case STRING:
return value.toString();
case ENUM:
if (schema.getEnumSymbols().contains(value.toString())) {
return new GenericData.EnumSymbol(schema, value.toString());
}
throw new JsonConversionException(String.format("Symbol %s not in enum", value.toString()),
schema.getFullName(), schema);
case RECORD:
return convert((Map<String, Object>) value, schema);
case ARRAY:
Schema elementSchema = schema.getElementType();
List listRes = new ArrayList();
for (Object v : (List) value) {
listRes.add(typeConvert(v, name, elementSchema));
}
return listRes;
case MAP:
Schema valueSchema = schema.getValueType();
Map<String, Object> mapRes = new HashMap<String, Object>();
for (Map.Entry<String, Object> v : ((Map<String, Object>) value).entrySet()) {
mapRes.put(v.getKey(), typeConvert(v.getValue(), name, valueSchema));
}
return mapRes;
default:
throw new IllegalArgumentException(
"JsonConverter cannot handle type: " + schema.getType());
}
throw new JsonConversionException(value, name, schema);
}
private boolean isOptional(Schema schema) {
return schema.getType().equals(Schema.Type.UNION) &&
schema.getTypes().size() == 2 &&
(schema.getTypes().get(0).getType().equals(Schema.Type.NULL) ||
schema.getTypes().get(1).getType().equals(Schema.Type.NULL));
}
private Schema getNonNull(Schema schema) {
List<Schema> types = schema.getTypes();
return types.get(0).getType().equals(Schema.Type.NULL) ? types.get(1) : types.get(0);
}
public static class JsonConversionException extends RuntimeException {
private Object value;
private String fieldName;
private Schema schema;
public JsonConversionException(Object value, String fieldName, Schema schema) {
this.value = value;
this.fieldName = fieldName;
this.schema = schema;
}
@Override
public String toString() {
return String.format("Type conversion error for field %s, %s for %s",
fieldName, value, schema);
}
}
}

View File

@@ -17,84 +17,86 @@
package com.uber.hoodie.common;
import com.uber.hoodie.exception.HoodieIndexException;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import javax.xml.bind.DatatypeConverter;
import org.apache.commons.io.output.ByteArrayOutputStream;
import org.apache.hadoop.util.bloom.Key;
import org.apache.hadoop.util.hash.Hash;
import javax.xml.bind.DatatypeConverter;
import java.io.*;
import java.nio.charset.StandardCharsets;
/**
* A Bloom filter implementation built on top of {@link org.apache.hadoop.util.bloom.BloomFilter}.
*/
public class BloomFilter {
/**
* Used in computing the optimal Bloom filter size. This approximately equals 0.480453.
*/
public static final double LOG2_SQUARED = Math.log(2) * Math.log(2);
private org.apache.hadoop.util.bloom.BloomFilter filter = null;
/**
* Used in computing the optimal Bloom filter size. This approximately equals 0.480453.
*/
public static final double LOG2_SQUARED = Math.log(2) * Math.log(2);
public BloomFilter(int numEntries, double errorRate) {
this(numEntries, errorRate, Hash.MURMUR_HASH);
private org.apache.hadoop.util.bloom.BloomFilter filter = null;
public BloomFilter(int numEntries, double errorRate) {
this(numEntries, errorRate, Hash.MURMUR_HASH);
}
/**
* Create a new Bloom filter with the given configurations.
*/
public BloomFilter(int numEntries, double errorRate, int hashType) {
// Bit size
int bitSize = (int) Math.ceil(numEntries * (-Math.log(errorRate) / LOG2_SQUARED));
// Number of the hash functions
int numHashs = (int) Math.ceil(Math.log(2) * bitSize / numEntries);
// The filter
this.filter = new org.apache.hadoop.util.bloom.BloomFilter(bitSize, numHashs, hashType);
}
/**
* Create the bloom filter from serialized string.
*/
public BloomFilter(String filterStr) {
this.filter = new org.apache.hadoop.util.bloom.BloomFilter();
byte[] bytes = DatatypeConverter.parseBase64Binary(filterStr);
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes));
try {
this.filter.readFields(dis);
dis.close();
} catch (IOException e) {
throw new HoodieIndexException("Could not deserialize BloomFilter instance", e);
}
}
/**
* Create a new Bloom filter with the given configurations.
*/
public BloomFilter(int numEntries, double errorRate, int hashType) {
// Bit size
int bitSize = (int) Math.ceil(numEntries * (-Math.log(errorRate) / LOG2_SQUARED));
// Number of the hash functions
int numHashs = (int) Math.ceil(Math.log(2) * bitSize / numEntries);
// The filter
this.filter = new org.apache.hadoop.util.bloom.BloomFilter(bitSize, numHashs, hashType);
public void add(String key) {
if (key == null) {
throw new NullPointerException("Key cannot by null");
}
filter.add(new Key(key.getBytes(StandardCharsets.UTF_8)));
}
/**
* Create the bloom filter from serialized string.
*/
public BloomFilter(String filterStr) {
this.filter = new org.apache.hadoop.util.bloom.BloomFilter();
byte[] bytes = DatatypeConverter.parseBase64Binary(filterStr);
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes));
try {
this.filter.readFields(dis);
dis.close();
} catch (IOException e) {
throw new HoodieIndexException("Could not deserialize BloomFilter instance", e);
}
public boolean mightContain(String key) {
if (key == null) {
throw new NullPointerException("Key cannot by null");
}
return filter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8)));
}
public void add(String key) {
if (key == null) {
throw new NullPointerException("Key cannot by null");
}
filter.add(new Key(key.getBytes(StandardCharsets.UTF_8)));
}
public boolean mightContain(String key) {
if (key == null) {
throw new NullPointerException("Key cannot by null");
}
return filter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8)));
}
/**
* Serialize the bloom filter as a string.
*/
public String serializeToString() {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(baos);
try {
filter.write(dos);
byte[] bytes = baos.toByteArray();
dos.close();
return DatatypeConverter.printBase64Binary(bytes);
} catch (IOException e) {
throw new HoodieIndexException("Could not serialize BloomFilter instance", e);
}
/**
* Serialize the bloom filter as a string.
*/
public String serializeToString() {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(baos);
try {
filter.write(dos);
byte[] bytes = baos.toByteArray();
dos.close();
return DatatypeConverter.printBase64Binary(bytes);
} catch (IOException e) {
throw new HoodieIndexException("Could not serialize BloomFilter instance", e);
}
}
}

View File

@@ -18,7 +18,6 @@ package com.uber.hoodie.common;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import java.io.Serializable;
import java.util.List;
import java.util.Optional;
@@ -27,100 +26,102 @@ import java.util.Optional;
* Collects stats about a single partition clean operation
*/
public class HoodieCleanStat implements Serializable {
// Policy used
private final HoodieCleaningPolicy policy;
// Partition path cleaned
private final String partitionPath;
// The patterns that were generated for the delete operation
private final List<String> deletePathPatterns;
private final List<String> successDeleteFiles;
// Files that could not be deleted
private final List<String> failedDeleteFiles;
// Earliest commit that was retained in this clean
private final String earliestCommitToRetain;
public HoodieCleanStat(HoodieCleaningPolicy policy, String partitionPath,
List<String> deletePathPatterns, List<String> successDeleteFiles,
List<String> failedDeleteFiles, String earliestCommitToRetain) {
this.policy = policy;
this.partitionPath = partitionPath;
this.deletePathPatterns = deletePathPatterns;
this.successDeleteFiles = successDeleteFiles;
this.failedDeleteFiles = failedDeleteFiles;
this.earliestCommitToRetain = earliestCommitToRetain;
// Policy used
private final HoodieCleaningPolicy policy;
// Partition path cleaned
private final String partitionPath;
// The patterns that were generated for the delete operation
private final List<String> deletePathPatterns;
private final List<String> successDeleteFiles;
// Files that could not be deleted
private final List<String> failedDeleteFiles;
// Earliest commit that was retained in this clean
private final String earliestCommitToRetain;
public HoodieCleanStat(HoodieCleaningPolicy policy, String partitionPath,
List<String> deletePathPatterns, List<String> successDeleteFiles,
List<String> failedDeleteFiles, String earliestCommitToRetain) {
this.policy = policy;
this.partitionPath = partitionPath;
this.deletePathPatterns = deletePathPatterns;
this.successDeleteFiles = successDeleteFiles;
this.failedDeleteFiles = failedDeleteFiles;
this.earliestCommitToRetain = earliestCommitToRetain;
}
public HoodieCleaningPolicy getPolicy() {
return policy;
}
public String getPartitionPath() {
return partitionPath;
}
public List<String> getDeletePathPatterns() {
return deletePathPatterns;
}
public List<String> getSuccessDeleteFiles() {
return successDeleteFiles;
}
public List<String> getFailedDeleteFiles() {
return failedDeleteFiles;
}
public String getEarliestCommitToRetain() {
return earliestCommitToRetain;
}
public static HoodieCleanStat.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private HoodieCleaningPolicy policy;
private List<String> deletePathPatterns;
private List<String> successDeleteFiles;
private List<String> failedDeleteFiles;
private String partitionPath;
private String earliestCommitToRetain;
public Builder withPolicy(HoodieCleaningPolicy policy) {
this.policy = policy;
return this;
}
public HoodieCleaningPolicy getPolicy() {
return policy;
public Builder withDeletePathPattern(List<String> deletePathPatterns) {
this.deletePathPatterns = deletePathPatterns;
return this;
}
public String getPartitionPath() {
return partitionPath;
public Builder withSuccessfulDeletes(List<String> successDeleteFiles) {
this.successDeleteFiles = successDeleteFiles;
return this;
}
public List<String> getDeletePathPatterns() {
return deletePathPatterns;
public Builder withFailedDeletes(List<String> failedDeleteFiles) {
this.failedDeleteFiles = failedDeleteFiles;
return this;
}
public List<String> getSuccessDeleteFiles() {
return successDeleteFiles;
public Builder withPartitionPath(String partitionPath) {
this.partitionPath = partitionPath;
return this;
}
public List<String> getFailedDeleteFiles() {
return failedDeleteFiles;
public Builder withEarliestCommitRetained(Optional<HoodieInstant> earliestCommitToRetain) {
this.earliestCommitToRetain = (earliestCommitToRetain.isPresent()) ?
earliestCommitToRetain.get().getTimestamp() :
"-1";
return this;
}
public String getEarliestCommitToRetain() {
return earliestCommitToRetain;
}
public static HoodieCleanStat.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private HoodieCleaningPolicy policy;
private List<String> deletePathPatterns;
private List<String> successDeleteFiles;
private List<String> failedDeleteFiles;
private String partitionPath;
private String earliestCommitToRetain;
public Builder withPolicy(HoodieCleaningPolicy policy) {
this.policy = policy;
return this;
}
public Builder withDeletePathPattern(List<String> deletePathPatterns) {
this.deletePathPatterns = deletePathPatterns;
return this;
}
public Builder withSuccessfulDeletes(List<String> successDeleteFiles) {
this.successDeleteFiles = successDeleteFiles;
return this;
}
public Builder withFailedDeletes(List<String> failedDeleteFiles) {
this.failedDeleteFiles= failedDeleteFiles;
return this;
}
public Builder withPartitionPath(String partitionPath) {
this.partitionPath = partitionPath;
return this;
}
public Builder withEarliestCommitRetained(Optional<HoodieInstant> earliestCommitToRetain) {
this.earliestCommitToRetain = (earliestCommitToRetain.isPresent()) ?
earliestCommitToRetain.get().getTimestamp() :
"-1";
return this;
}
public HoodieCleanStat build() {
return new HoodieCleanStat(policy, partitionPath, deletePathPatterns,
successDeleteFiles, failedDeleteFiles, earliestCommitToRetain);
}
public HoodieCleanStat build() {
return new HoodieCleanStat(policy, partitionPath, deletePathPatterns,
successDeleteFiles, failedDeleteFiles, earliestCommitToRetain);
}
}
}

View File

@@ -19,13 +19,6 @@ package com.uber.hoodie.common;
import com.uber.hoodie.avro.MercifulJsonConverter;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.exception.HoodieException;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.IOUtils;
import org.codehaus.jackson.JsonNode;
import org.codehaus.jackson.map.ObjectMapper;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
@@ -34,75 +27,85 @@ 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;
import org.codehaus.jackson.JsonNode;
import org.codehaus.jackson.map.ObjectMapper;
public class HoodieJsonPayload implements HoodieRecordPayload<HoodieJsonPayload> {
private byte[] jsonDataCompressed;
private int dataSize;
public HoodieJsonPayload(String json) throws IOException {
this.jsonDataCompressed = compressData(json);
this.dataSize = json.length();
private byte[] jsonDataCompressed;
private int dataSize;
public HoodieJsonPayload(String json) throws IOException {
this.jsonDataCompressed = compressData(json);
this.dataSize = json.length();
}
@Override
public HoodieJsonPayload preCombine(HoodieJsonPayload another) {
return this;
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema)
throws IOException {
return getInsertValue(schema);
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return Optional.of(jsonConverter.convert(getJsonData()));
}
private String getJsonData() throws IOException {
return unCompressData(jsonDataCompressed);
}
private byte[] compressData(String jsonData) throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
Deflater deflater = new Deflater(Deflater.BEST_COMPRESSION);
DeflaterOutputStream dos =
new DeflaterOutputStream(baos, deflater, true);
try {
dos.write(jsonData.getBytes());
} finally {
dos.flush();
dos.close();
// Its important to call this.
// Deflater takes off-heap native memory and does not release until GC kicks in
deflater.end();
}
return baos.toByteArray();
}
@Override public HoodieJsonPayload preCombine(HoodieJsonPayload another) {
return this;
private String unCompressData(byte[] data) throws IOException {
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
try {
StringWriter sw = new StringWriter(dataSize);
IOUtils.copy(iis, sw);
return sw.toString();
} finally {
iis.close();
}
}
@Override public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
return getInsertValue(schema);
private String getFieldFromJsonOrFail(String field) throws IOException {
JsonNode node = new ObjectMapper().readTree(getJsonData());
if (!node.has(field)) {
throw new HoodieException("Field :" + field + " not found in payload => " + node.toString());
}
return node.get(field).getTextValue();
}
@Override public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return Optional.of(jsonConverter.convert(getJsonData()));
}
public String getRowKey(String keyColumnField) throws IOException {
return getFieldFromJsonOrFail(keyColumnField);
}
private String getJsonData() throws IOException {
return unCompressData(jsonDataCompressed);
}
private byte[] compressData(String jsonData) throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
Deflater deflater = new Deflater(Deflater.BEST_COMPRESSION);
DeflaterOutputStream dos =
new DeflaterOutputStream(baos, deflater, true);
try {
dos.write(jsonData.getBytes());
} finally {
dos.flush();
dos.close();
// Its important to call this.
// Deflater takes off-heap native memory and does not release until GC kicks in
deflater.end();
}
return baos.toByteArray();
}
private String unCompressData(byte[] data) throws IOException {
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
try {
StringWriter sw = new StringWriter(dataSize);
IOUtils.copy(iis, sw);
return sw.toString();
} finally {
iis.close();
}
}
private String getFieldFromJsonOrFail(String field) throws IOException {
JsonNode node = new ObjectMapper().readTree(getJsonData());
if(!node.has(field)) {
throw new HoodieException("Field :" + field + " not found in payload => " + node.toString());
}
return node.get(field).getTextValue();
}
public String getRowKey(String keyColumnField) throws IOException {
return getFieldFromJsonOrFail(keyColumnField);
}
public String getPartitionPath(String partitionPathField) throws IOException {
return getFieldFromJsonOrFail(partitionPathField);
}
public String getPartitionPath(String partitionPathField) throws IOException {
return getFieldFromJsonOrFail(partitionPathField);
}
}

View File

@@ -16,81 +16,82 @@
package com.uber.hoodie.common;
import org.apache.hadoop.fs.FileStatus;
import java.io.File;
import java.io.Serializable;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus;
/**
* Collects stats about a single partition clean operation
*/
public class HoodieRollbackStat implements Serializable {
// Partition path
private final String partitionPath;
private final List<String> successDeleteFiles;
// Files that could not be deleted
private final List<String> failedDeleteFiles;
// Count of HoodieLogFile to commandBlocks written for a particular rollback
private final Map<FileStatus, Long> commandBlocksCount;
public HoodieRollbackStat(String partitionPath, List<String> successDeleteFiles,
List<String> failedDeleteFiles, Map<FileStatus, Long> commandBlocksCount) {
this.partitionPath = partitionPath;
this.successDeleteFiles = successDeleteFiles;
this.failedDeleteFiles = failedDeleteFiles;
this.commandBlocksCount = commandBlocksCount;
// Partition path
private final String partitionPath;
private final List<String> successDeleteFiles;
// Files that could not be deleted
private final List<String> failedDeleteFiles;
// Count of HoodieLogFile to commandBlocks written for a particular rollback
private final Map<FileStatus, Long> commandBlocksCount;
public HoodieRollbackStat(String partitionPath, List<String> successDeleteFiles,
List<String> failedDeleteFiles, Map<FileStatus, Long> commandBlocksCount) {
this.partitionPath = partitionPath;
this.successDeleteFiles = successDeleteFiles;
this.failedDeleteFiles = failedDeleteFiles;
this.commandBlocksCount = commandBlocksCount;
}
public Map<FileStatus, Long> getCommandBlocksCount() {
return commandBlocksCount;
}
public String getPartitionPath() {
return partitionPath;
}
public List<String> getSuccessDeleteFiles() {
return successDeleteFiles;
}
public List<String> getFailedDeleteFiles() {
return failedDeleteFiles;
}
public static HoodieRollbackStat.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private List<String> successDeleteFiles;
private List<String> failedDeleteFiles;
private Map<FileStatus, Long> commandBlocksCount;
private String partitionPath;
public Builder withDeletedFileResults(Map<FileStatus, Boolean> deletedFiles) {
//noinspection Convert2MethodRef
successDeleteFiles = deletedFiles.entrySet().stream().filter(s -> s.getValue())
.map(s -> s.getKey().getPath().toString()).collect(Collectors.toList());
failedDeleteFiles = deletedFiles.entrySet().stream().filter(s -> !s.getValue())
.map(s -> s.getKey().getPath().toString()).collect(Collectors.toList());
return this;
}
public Map<FileStatus, Long> getCommandBlocksCount() {
return commandBlocksCount;
public Builder withRollbackBlockAppendResults(Map<FileStatus, Long> commandBlocksCount) {
this.commandBlocksCount = commandBlocksCount;
return this;
}
public String getPartitionPath() {
return partitionPath;
public Builder withPartitionPath(String partitionPath) {
this.partitionPath = partitionPath;
return this;
}
public List<String> getSuccessDeleteFiles() {
return successDeleteFiles;
}
public List<String> getFailedDeleteFiles() {
return failedDeleteFiles;
}
public static HoodieRollbackStat.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private List<String> successDeleteFiles;
private List<String> failedDeleteFiles;
private Map<FileStatus, Long> commandBlocksCount;
private String partitionPath;
public Builder withDeletedFileResults(Map<FileStatus, Boolean> deletedFiles) {
//noinspection Convert2MethodRef
successDeleteFiles = deletedFiles.entrySet().stream().filter(s -> s.getValue())
.map(s -> s.getKey().getPath().toString()).collect(Collectors.toList());
failedDeleteFiles = deletedFiles.entrySet().stream().filter(s -> !s.getValue())
.map(s -> s.getKey().getPath().toString()).collect(Collectors.toList());
return this;
}
public Builder withRollbackBlockAppendResults(Map<FileStatus, Long> commandBlocksCount) {
this.commandBlocksCount = commandBlocksCount;
return this;
}
public Builder withPartitionPath(String partitionPath) {
this.partitionPath = partitionPath;
return this;
}
public HoodieRollbackStat build() {
return new HoodieRollbackStat(partitionPath, successDeleteFiles, failedDeleteFiles, commandBlocksCount);
}
public HoodieRollbackStat build() {
return new HoodieRollbackStat(partitionPath, successDeleteFiles, failedDeleteFiles,
commandBlocksCount);
}
}
}

View File

@@ -17,5 +17,5 @@
package com.uber.hoodie.common.model;
public enum ActionType {
commit, savepoint, compaction, clean, rollback;
commit, savepoint, compaction, clean, rollback;
}

View File

@@ -17,13 +17,7 @@
package com.uber.hoodie.common.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.uber.hoodie.common.util.FSUtils;
import java.io.Serializable;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.Path;
@JsonIgnoreProperties(ignoreUnknown = true)
public class CompactionWriteStat implements Serializable {
@@ -34,7 +28,8 @@ public class CompactionWriteStat implements Serializable {
private long totalLogFiles;
private long totalRecordsToBeUpdate;
public CompactionWriteStat(HoodieWriteStat writeStat, String partitionPath, long totalLogFiles, long totalLogRecords,
public CompactionWriteStat(HoodieWriteStat writeStat, String partitionPath, long totalLogFiles,
long totalLogRecords,
long totalRecordsToUpdate) {
this.writeStat = writeStat;
this.partitionPath = partitionPath;
@@ -58,6 +53,7 @@ public class CompactionWriteStat implements Serializable {
public long getTotalRecordsToBeUpdate() {
return totalRecordsToBeUpdate;
}
public HoodieWriteStat getHoodieWriteStat() {
return writeStat;
}

View File

@@ -19,79 +19,75 @@
package com.uber.hoodie.common.model;
import java.io.Serializable;
import java.util.List;
import java.util.Optional;
import java.util.TreeSet;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Within a file group, a slice is a combination of data file written at a commit time
* and list of log files, containing changes to the data file from that commit time
* Within a file group, a slice is a combination of data file written at a commit time and list of
* log files, containing changes to the data file from that commit time
*/
public class FileSlice implements Serializable {
/**
* id of the slice
*/
private String fileId;
/**
* id of the slice
*/
private String fileId;
/**
* Point in the timeline, at which the slice was created
*/
private String baseCommitTime;
/**
* Point in the timeline, at which the slice was created
*/
private String baseCommitTime;
/**
* data file, with the compacted data, for this slice
*
*/
private HoodieDataFile dataFile;
/**
* data file, with the compacted data, for this slice
*/
private HoodieDataFile dataFile;
/**
* List of appendable log files with real time data
* - Sorted with greater log version first
* - Always empty for copy_on_write storage.
*/
private final TreeSet<HoodieLogFile> logFiles;
/**
* List of appendable log files with real time data - Sorted with greater log version first -
* Always empty for copy_on_write storage.
*/
private final TreeSet<HoodieLogFile> logFiles;
public FileSlice(String baseCommitTime, String fileId) {
this.fileId = fileId;
this.baseCommitTime = baseCommitTime;
this.dataFile = null;
this.logFiles = new TreeSet<>(HoodieLogFile.getLogVersionComparator());
}
public FileSlice(String baseCommitTime, String fileId) {
this.fileId = fileId;
this.baseCommitTime = baseCommitTime;
this.dataFile = null;
this.logFiles = new TreeSet<>(HoodieLogFile.getLogVersionComparator());
}
public void setDataFile(HoodieDataFile dataFile) {
this.dataFile = dataFile;
}
public void setDataFile(HoodieDataFile dataFile) {
this.dataFile = dataFile;
}
public void addLogFile(HoodieLogFile logFile) {
this.logFiles.add(logFile);
}
public void addLogFile(HoodieLogFile logFile) {
this.logFiles.add(logFile);
}
public Stream<HoodieLogFile> getLogFiles() {
return logFiles.stream();
}
public Stream<HoodieLogFile> getLogFiles() {
return logFiles.stream();
}
public String getBaseCommitTime() {
return baseCommitTime;
}
public String getBaseCommitTime() {
return baseCommitTime;
}
public String getFileId() {
return fileId;
}
public String getFileId() {
return fileId;
}
public Optional<HoodieDataFile> getDataFile() {
return Optional.ofNullable(dataFile);
}
public Optional<HoodieDataFile> getDataFile() {
return Optional.ofNullable(dataFile);
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("FileSlice {");
sb.append("baseCommitTime=").append(baseCommitTime);
sb.append(", dataFile='").append(dataFile).append('\'');
sb.append(", logFiles='").append(logFiles).append('\'');
sb.append('}');
return sb.toString();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("FileSlice {");
sb.append("baseCommitTime=").append(baseCommitTime);
sb.append(", dataFile='").append(dataFile).append('\'');
sb.append(", logFiles='").append(logFiles).append('\'');
sb.append('}');
return sb.toString();
}
}

View File

@@ -23,18 +23,18 @@ import org.apache.hadoop.fs.Path;
public class HoodieArchivedLogFile extends HoodieLogFile {
public static final String ARCHIVE_EXTENSION = ".archive";
public static final String ARCHIVE_EXTENSION = ".archive";
public HoodieArchivedLogFile(FileStatus fileStatus) {
super(fileStatus);
}
public HoodieArchivedLogFile(FileStatus fileStatus) {
super(fileStatus);
}
public HoodieArchivedLogFile(Path logPath) {
super(logPath);
}
public HoodieArchivedLogFile(Path logPath) {
super(logPath);
}
@Override
public String toString() {
return "HoodieArchivedLogFile {" + super.getPath() + '}';
}
@Override
public String toString() {
return "HoodieArchivedLogFile {" + super.getPath() + '}';
}
}

View File

@@ -17,40 +17,37 @@
package com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import java.io.IOException;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import java.io.IOException;
/**
* This is a payload to wrap a existing Hoodie Avro Record.
* Useful to create a HoodieRecord over existing GenericRecords in a hoodie datasets (useful in compactions)
*
* This is a payload to wrap a existing Hoodie Avro Record. Useful to create a HoodieRecord over
* existing GenericRecords in a hoodie datasets (useful in compactions)
*/
public class HoodieAvroPayload implements HoodieRecordPayload<HoodieAvroPayload> {
private final Optional<GenericRecord> record;
public HoodieAvroPayload(Optional<GenericRecord> record) {
this.record = record;
}
private final Optional<GenericRecord> record;
@Override
public HoodieAvroPayload preCombine(HoodieAvroPayload another) {
return this;
}
public HoodieAvroPayload(Optional<GenericRecord> record) {
this.record = record;
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException {
return getInsertValue(schema);
}
@Override
public HoodieAvroPayload preCombine(HoodieAvroPayload another) {
return this;
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return record.map(r -> HoodieAvroUtils.rewriteRecord(r, schema));
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException {
return getInsertValue(schema);
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return record.map(r -> HoodieAvroUtils.rewriteRecord(r, schema));
}
}

View File

@@ -17,6 +17,6 @@
package com.uber.hoodie.common.model;
public enum HoodieCleaningPolicy {
KEEP_LATEST_FILE_VERSIONS,
KEEP_LATEST_COMMITS
KEEP_LATEST_FILE_VERSIONS,
KEEP_LATEST_COMMITS
}

View File

@@ -17,8 +17,13 @@
package com.uber.hoodie.common.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.databind.DeserializationFeature;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -27,196 +32,195 @@ import org.codehaus.jackson.annotate.JsonMethod;
import org.codehaus.jackson.map.DeserializationConfig.Feature;
import org.codehaus.jackson.map.ObjectMapper;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* All the metadata that gets stored along with a commit.
*/
@JsonIgnoreProperties(ignoreUnknown = true)
public class HoodieCommitMetadata implements Serializable {
private static volatile Logger log = LogManager.getLogger(HoodieCommitMetadata.class);
protected Map<String, List<HoodieWriteStat>> partitionToWriteStats;
private Map<String, String> extraMetadataMap;
private static volatile Logger log = LogManager.getLogger(HoodieCommitMetadata.class);
protected Map<String, List<HoodieWriteStat>> partitionToWriteStats;
public HoodieCommitMetadata() {
extraMetadataMap = new HashMap<>();
partitionToWriteStats = new HashMap<>();
private Map<String, String> extraMetadataMap;
public HoodieCommitMetadata() {
extraMetadataMap = new HashMap<>();
partitionToWriteStats = new HashMap<>();
}
public void addWriteStat(String partitionPath, HoodieWriteStat stat) {
if (!partitionToWriteStats.containsKey(partitionPath)) {
partitionToWriteStats.put(partitionPath, new ArrayList<>());
}
partitionToWriteStats.get(partitionPath).add(stat);
}
public void addWriteStat(String partitionPath, HoodieWriteStat stat) {
if (!partitionToWriteStats.containsKey(partitionPath)) {
partitionToWriteStats.put(partitionPath, new ArrayList<>());
public void addMetadata(String metaKey, String value) {
extraMetadataMap.put(metaKey, value);
}
public List<HoodieWriteStat> getWriteStats(String partitionPath) {
return partitionToWriteStats.get(partitionPath);
}
public Map<String, String> getExtraMetadata() {
return extraMetadataMap;
}
public Map<String, List<HoodieWriteStat>> getPartitionToWriteStats() {
return partitionToWriteStats;
}
public String getMetadata(String metaKey) {
return extraMetadataMap.get(metaKey);
}
public HashMap<String, String> getFileIdAndRelativePaths() {
HashMap<String, String> filePaths = new HashMap<>();
// list all partitions paths
for (Map.Entry<String, List<HoodieWriteStat>> entry : getPartitionToWriteStats().entrySet()) {
for (HoodieWriteStat stat : entry.getValue()) {
filePaths.put(stat.getFileId(), stat.getPath());
}
}
return filePaths;
}
public HashMap<String, String> getFileIdAndFullPaths(String basePath) {
HashMap<String, String> fullPaths = new HashMap<>();
for (Map.Entry<String, String> entry : getFileIdAndRelativePaths().entrySet()) {
String fullPath =
(entry.getValue() != null) ? (new Path(basePath, entry.getValue())).toString() : null;
fullPaths.put(entry.getKey(), fullPath);
}
return fullPaths;
}
public String toJsonString() throws IOException {
if (partitionToWriteStats.containsKey(null)) {
log.info("partition path is null for " + partitionToWriteStats.get(null));
partitionToWriteStats.remove(null);
}
ObjectMapper mapper = new ObjectMapper();
mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY);
return mapper.defaultPrettyPrintingWriter().writeValueAsString(this);
}
public static HoodieCommitMetadata fromJsonString(String jsonStr) throws IOException {
if (jsonStr == null || jsonStr.isEmpty()) {
// For empty commit file (no data or somethings bad happen).
return new HoodieCommitMetadata();
}
ObjectMapper mapper = new ObjectMapper();
mapper.configure(Feature.FAIL_ON_UNKNOWN_PROPERTIES, false);
mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY);
return mapper.readValue(jsonStr, HoodieCommitMetadata.class);
}
// Here the functions are named "fetch" instead of "get", to get avoid of the json conversion.
public long fetchTotalPartitionsWritten() {
return partitionToWriteStats.size();
}
public long fetchTotalFilesInsert() {
long totalFilesInsert = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
if (stat.getPrevCommit() != null && stat.getPrevCommit().equals("null")) {
totalFilesInsert++;
}
partitionToWriteStats.get(partitionPath).add(stat);
}
}
return totalFilesInsert;
}
public void addMetadata(String metaKey, String value) {
extraMetadataMap.put(metaKey, value);
}
public List<HoodieWriteStat> getWriteStats(String partitionPath) {
return partitionToWriteStats.get(partitionPath);
}
public Map<String, String> getExtraMetadata() { return extraMetadataMap; }
public Map<String, List<HoodieWriteStat>> getPartitionToWriteStats() {
return partitionToWriteStats;
}
public String getMetadata(String metaKey) {
return extraMetadataMap.get(metaKey);
}
public HashMap<String, String> getFileIdAndRelativePaths() {
HashMap<String, String> filePaths = new HashMap<>();
// list all partitions paths
for (Map.Entry<String, List<HoodieWriteStat>> entry: getPartitionToWriteStats().entrySet()) {
for (HoodieWriteStat stat: entry.getValue()) {
filePaths.put(stat.getFileId(), stat.getPath());
}
public long fetchTotalFilesUpdated() {
long totalFilesUpdated = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
if (stat.getPrevCommit() != null && !stat.getPrevCommit().equals("null")) {
totalFilesUpdated++;
}
return filePaths;
}
}
return totalFilesUpdated;
}
public HashMap<String, String> getFileIdAndFullPaths(String basePath) {
HashMap<String, String> fullPaths = new HashMap<>();
for (Map.Entry<String, String> entry: getFileIdAndRelativePaths().entrySet()) {
String fullPath = (entry.getValue() != null) ? (new Path(basePath, entry.getValue())).toString() : null;
fullPaths.put(entry.getKey(), fullPath);
} return fullPaths;
public long fetchTotalUpdateRecordsWritten() {
long totalUpdateRecordsWritten = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
totalUpdateRecordsWritten += stat.getNumUpdateWrites();
}
}
return totalUpdateRecordsWritten;
}
public String toJsonString() throws IOException {
if(partitionToWriteStats.containsKey(null)) {
log.info("partition path is null for " + partitionToWriteStats.get(null));
partitionToWriteStats.remove(null);
public long fetchTotalInsertRecordsWritten() {
long totalInsertRecordsWritten = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
if (stat.getPrevCommit() != null && stat.getPrevCommit().equals("null")) {
totalInsertRecordsWritten += stat.getNumWrites();
}
ObjectMapper mapper = new ObjectMapper();
mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY);
return mapper.defaultPrettyPrintingWriter().writeValueAsString(this);
}
}
return totalInsertRecordsWritten;
}
public long fetchTotalRecordsWritten() {
long totalRecordsWritten = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
totalRecordsWritten += stat.getNumWrites();
}
}
return totalRecordsWritten;
}
public long fetchTotalBytesWritten() {
long totalBytesWritten = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
totalBytesWritten += stat.getTotalWriteBytes();
}
}
return totalBytesWritten;
}
public long fetchTotalWriteErrors() {
long totalWriteErrors = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
totalWriteErrors += stat.getTotalWriteErrors();
}
}
return totalWriteErrors;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
public static HoodieCommitMetadata fromJsonString(String jsonStr) throws IOException {
if (jsonStr == null || jsonStr.isEmpty()) {
// For empty commit file (no data or somethings bad happen).
return new HoodieCommitMetadata();
}
ObjectMapper mapper = new ObjectMapper();
mapper.configure(Feature.FAIL_ON_UNKNOWN_PROPERTIES, false);
mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY);
return mapper.readValue(jsonStr, HoodieCommitMetadata.class);
}
HoodieCommitMetadata that = (HoodieCommitMetadata) o;
// Here the functions are named "fetch" instead of "get", to get avoid of the json conversion.
public long fetchTotalPartitionsWritten() {
return partitionToWriteStats.size();
}
return partitionToWriteStats != null ?
partitionToWriteStats.equals(that.partitionToWriteStats) :
that.partitionToWriteStats == null;
public long fetchTotalFilesInsert() {
long totalFilesInsert = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
if (stat.getPrevCommit() != null && stat.getPrevCommit().equals("null")) {
totalFilesInsert ++;
}
}
}
return totalFilesInsert;
}
}
public long fetchTotalFilesUpdated() {
long totalFilesUpdated = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
if (stat.getPrevCommit() != null && !stat.getPrevCommit().equals("null")) {
totalFilesUpdated ++;
}
}
}
return totalFilesUpdated;
}
@Override
public int hashCode() {
return partitionToWriteStats != null ? partitionToWriteStats.hashCode() : 0;
}
public long fetchTotalUpdateRecordsWritten() {
long totalUpdateRecordsWritten = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
totalUpdateRecordsWritten += stat.getNumUpdateWrites();
}
}
return totalUpdateRecordsWritten;
}
public long fetchTotalInsertRecordsWritten() {
long totalInsertRecordsWritten = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
if (stat.getPrevCommit() != null && stat.getPrevCommit().equals("null")) {
totalInsertRecordsWritten += stat.getNumWrites();
}
}
}
return totalInsertRecordsWritten;
}
public long fetchTotalRecordsWritten() {
long totalRecordsWritten = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
totalRecordsWritten += stat.getNumWrites();
}
}
return totalRecordsWritten;
}
public long fetchTotalBytesWritten() {
long totalBytesWritten = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
totalBytesWritten += stat.getTotalWriteBytes();
}
}
return totalBytesWritten;
}
public long fetchTotalWriteErrors() {
long totalWriteErrors = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
totalWriteErrors += stat.getTotalWriteErrors();
}
}
return totalWriteErrors;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieCommitMetadata that = (HoodieCommitMetadata) o;
return partitionToWriteStats != null ?
partitionToWriteStats.equals(that.partitionToWriteStats) :
that.partitionToWriteStats == null;
}
@Override
public int hashCode() {
return partitionToWriteStats != null ? partitionToWriteStats.hashCode() : 0;
}
public static HoodieCommitMetadata fromBytes(byte[] bytes) throws IOException {
return fromJsonString(new String(bytes, Charset.forName("utf-8")));
}
public static HoodieCommitMetadata fromBytes(byte[] bytes) throws IOException {
return fromJsonString(new String(bytes, Charset.forName("utf-8")));
}
}

View File

@@ -16,15 +16,12 @@
package com.uber.hoodie.common.model;
import com.google.common.collect.Maps;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.BinaryOperator;
import java.util.function.Supplier;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.codehaus.jackson.annotate.JsonAutoDetect;
@@ -33,9 +30,11 @@ import org.codehaus.jackson.map.DeserializationConfig.Feature;
import org.codehaus.jackson.map.ObjectMapper;
/**
* Place holder for the compaction specific meta-data, uses all the details used in a normal HoodieCommitMetadata
* Place holder for the compaction specific meta-data, uses all the details used in a normal
* HoodieCommitMetadata
*/
public class HoodieCompactionMetadata extends HoodieCommitMetadata {
private static volatile Logger log = LogManager.getLogger(HoodieCompactionMetadata.class);
protected HashMap<String, List<CompactionWriteStat>> partitionToCompactionWriteStats;
@@ -60,7 +59,7 @@ public class HoodieCompactionMetadata extends HoodieCommitMetadata {
}
public String toJsonString() throws IOException {
if(partitionToCompactionWriteStats.containsKey(null)) {
if (partitionToCompactionWriteStats.containsKey(null)) {
log.info("partition path is null for " + partitionToCompactionWriteStats.get(null));
partitionToCompactionWriteStats.remove(null);
}

View File

@@ -17,56 +17,54 @@
package com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import java.io.Serializable;
import java.util.Comparator;
import org.apache.hadoop.fs.FileStatus;
public class HoodieDataFile implements Serializable {
private FileStatus fileStatus;
public HoodieDataFile(FileStatus fileStatus) {
this.fileStatus = fileStatus;
}
private FileStatus fileStatus;
public String getFileId() {
return FSUtils.getFileId(fileStatus.getPath().getName());
}
public HoodieDataFile(FileStatus fileStatus) {
this.fileStatus = fileStatus;
}
public String getCommitTime() {
return FSUtils.getCommitTime(fileStatus.getPath().getName());
}
public String getFileId() {
return FSUtils.getFileId(fileStatus.getPath().getName());
}
public String getPath() {
return fileStatus.getPath().toString();
}
public String getCommitTime() {
return FSUtils.getCommitTime(fileStatus.getPath().getName());
}
public String getFileName() {
return fileStatus.getPath().getName();
}
public String getPath() {
return fileStatus.getPath().toString();
}
public FileStatus getFileStatus() {
return fileStatus;
}
public String getFileName() {
return fileStatus.getPath().getName();
}
public static Comparator<HoodieDataFile> getCommitTimeComparator() {
return (o1, o2) -> {
// reverse the order
return o2.getCommitTime().compareTo(o1.getCommitTime());
};
}
public FileStatus getFileStatus() {
return fileStatus;
}
public long getFileSize() {
return fileStatus.getLen();
}
public static Comparator<HoodieDataFile> getCommitTimeComparator() {
return (o1, o2) -> {
// reverse the order
return o2.getCommitTime().compareTo(o1.getCommitTime());
};
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieDataFile {");
sb.append("fileStatus=").append(fileStatus);
sb.append('}');
return sb.toString();
}
public long getFileSize() {
return fileStatus.getLen();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieDataFile {");
sb.append("fileStatus=").append(fileStatus);
sb.append('}');
return sb.toString();
}
}

View File

@@ -24,22 +24,22 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
@JsonIgnoreProperties(ignoreUnknown = true)
public class HoodieDeltaWriteStat extends HoodieWriteStat {
private int logVersion;
private long logOffset;
private int logVersion;
private long logOffset;
public void setLogVersion(int logVersion) {
this.logVersion = logVersion;
}
public void setLogVersion(int logVersion) {
this.logVersion = logVersion;
}
public int getLogVersion() {
return logVersion;
}
public int getLogVersion() {
return logVersion;
}
public void setLogOffset(long logOffset) {
this.logOffset = logOffset;
}
public void setLogOffset(long logOffset) {
this.logOffset = logOffset;
}
public long getLogOffset() {
return logOffset;
}
public long getLogOffset() {
return logOffset;
}
}

View File

@@ -17,15 +17,15 @@
package com.uber.hoodie.common.model;
public enum HoodieFileFormat {
PARQUET(".parquet"), HOODIE_LOG(".log");
PARQUET(".parquet"), HOODIE_LOG(".log");
private final String extension;
private final String extension;
HoodieFileFormat(String extension) {
this.extension = extension;
}
HoodieFileFormat(String extension) {
this.extension = extension;
}
public String getFileExtension() {
return extension;
}
public String getFileExtension() {
return extension;
}
}

View File

@@ -20,9 +20,6 @@ package com.uber.hoodie.common.model;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import org.apache.commons.lang3.tuple.Pair;
import java.io.Serializable;
import java.util.Comparator;
import java.util.List;
@@ -35,212 +32,184 @@ import java.util.stream.Stream;
*/
public class HoodieFileGroup implements Serializable {
public static Comparator<String> getReverseCommitTimeComparator() {
return (o1, o2) -> {
// reverse the order
return o2.compareTo(o1);
};
public static Comparator<String> getReverseCommitTimeComparator() {
return (o1, o2) -> {
// reverse the order
return o2.compareTo(o1);
};
}
/**
* Partition containing the file group.
*/
private final String partitionPath;
/**
* uniquely identifies the file group
*/
private final String id;
/**
* Slices of files in this group, sorted with greater commit first.
*/
private final TreeMap<String, FileSlice> fileSlices;
/**
* Timeline, based on which all getter work
*/
private final HoodieTimeline timeline;
/**
* The last completed instant, that acts as a high watermark for all getters
*/
private final Optional<HoodieInstant> lastInstant;
public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) {
this.partitionPath = partitionPath;
this.id = id;
this.fileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator());
this.timeline = timeline;
this.lastInstant = timeline.lastInstant();
}
/**
* Add a new datafile into the file group
*/
public void addDataFile(HoodieDataFile dataFile) {
if (!fileSlices.containsKey(dataFile.getCommitTime())) {
fileSlices.put(dataFile.getCommitTime(), new FileSlice(dataFile.getCommitTime(), id));
}
fileSlices.get(dataFile.getCommitTime()).setDataFile(dataFile);
}
/**
* Partition containing the file group.
*/
private final String partitionPath;
/**
* uniquely identifies the file group
*/
private final String id;
/**
* Slices of files in this group, sorted with greater commit first.
*/
private final TreeMap<String, FileSlice> fileSlices;
/**
* Timeline, based on which all getter work
*/
private final HoodieTimeline timeline;
/**
* The last completed instant, that acts as a high watermark for all
* getters
*/
private final Optional<HoodieInstant> lastInstant;
public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) {
this.partitionPath = partitionPath;
this.id = id;
this.fileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator());
this.timeline = timeline;
this.lastInstant = timeline.lastInstant();
/**
* Add a new log file into the group
*/
public void addLogFile(HoodieLogFile logFile) {
if (!fileSlices.containsKey(logFile.getBaseCommitTime())) {
fileSlices.put(logFile.getBaseCommitTime(), new FileSlice(logFile.getBaseCommitTime(), id));
}
fileSlices.get(logFile.getBaseCommitTime()).addLogFile(logFile);
}
/**
* Add a new datafile into the file group
*
* @param dataFile
*/
public void addDataFile(HoodieDataFile dataFile) {
if (!fileSlices.containsKey(dataFile.getCommitTime())) {
fileSlices.put(dataFile.getCommitTime(), new FileSlice(dataFile.getCommitTime(), id));
}
fileSlices.get(dataFile.getCommitTime()).setDataFile(dataFile);
public String getId() {
return id;
}
public String getPartitionPath() {
return partitionPath;
}
/**
* A FileSlice is considered committed, if one of the following is true - There is a committed
* data file - There are some log files, that are based off a commit or delta commit
*/
private boolean isFileSliceCommitted(FileSlice slice) {
String maxCommitTime = lastInstant.get().getTimestamp();
return timeline.containsOrBeforeTimelineStarts(slice.getBaseCommitTime()) &&
HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL);
}
/**
* Provides a stream of committed file slices, sorted reverse base commit time.
*/
public Stream<FileSlice> getAllFileSlices() {
if (!timeline.empty()) {
return fileSlices.entrySet().stream()
.map(sliceEntry -> sliceEntry.getValue())
.filter(slice -> isFileSliceCommitted(slice));
}
return Stream.empty();
}
/**
* Add a new log file into the group
*
* @param logFile
*/
public void addLogFile(HoodieLogFile logFile) {
if (!fileSlices.containsKey(logFile.getBaseCommitTime())) {
fileSlices.put(logFile.getBaseCommitTime(), new FileSlice(logFile.getBaseCommitTime(), id));
}
fileSlices.get(logFile.getBaseCommitTime()).addLogFile(logFile);
/**
* Gets the latest slice - this can contain either
*
* - just the log files without data file - (or) data file with 0 or more log files
*/
public Optional<FileSlice> getLatestFileSlice() {
// there should always be one
return getAllFileSlices().findFirst();
}
/**
* Obtain the latest file slice, upto a commitTime i.e <= maxCommitTime
*/
public Optional<FileSlice> getLatestFileSliceBeforeOrOn(String maxCommitTime) {
return getAllFileSlices()
.filter(slice ->
HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL))
.findFirst();
}
public Optional<FileSlice> getLatestFileSliceInRange(List<String> commitRange) {
return getAllFileSlices()
.filter(slice -> commitRange.contains(slice.getBaseCommitTime()))
.findFirst();
}
/**
* Stream of committed data files, sorted reverse commit time
*/
public Stream<HoodieDataFile> getAllDataFiles() {
return getAllFileSlices()
.filter(slice -> slice.getDataFile().isPresent())
.map(slice -> slice.getDataFile().get());
}
/**
* Get the latest committed data file
*/
public Optional<HoodieDataFile> getLatestDataFile() {
return getAllDataFiles().findFirst();
}
/**
* Get the latest data file, that is <= max commit time
*/
public Optional<HoodieDataFile> getLatestDataFileBeforeOrOn(String maxCommitTime) {
return getAllDataFiles()
.filter(dataFile ->
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL))
.findFirst();
}
/**
* Get the latest data file, that is contained within the provided commit range.
*/
public Optional<HoodieDataFile> getLatestDataFileInRange(List<String> commitRange) {
return getAllDataFiles()
.filter(dataFile -> commitRange.contains(dataFile.getCommitTime()))
.findFirst();
}
/**
* Obtain the latest log file (based on latest committed data file), currently being appended to
*
* @return logfile if present, empty if no log file has been opened already.
*/
public Optional<HoodieLogFile> getLatestLogFile() {
Optional<FileSlice> latestSlice = getLatestFileSlice();
if (latestSlice.isPresent() && latestSlice.get().getLogFiles().count() > 0) {
return latestSlice.get().getLogFiles().findFirst();
}
return Optional.empty();
}
public String getId() {
return id;
}
public String getPartitionPath() {
return partitionPath;
}
/**
* A FileSlice is considered committed, if one of the following is true
* - There is a committed data file
* - There are some log files, that are based off a commit or delta commit
*
* @param slice
* @return
*/
private boolean isFileSliceCommitted(FileSlice slice) {
String maxCommitTime = lastInstant.get().getTimestamp();
return timeline.containsOrBeforeTimelineStarts(slice.getBaseCommitTime()) &&
HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL);
}
/**
* Provides a stream of committed file slices, sorted reverse base commit time.
*
* @return
*/
public Stream<FileSlice> getAllFileSlices() {
if (!timeline.empty()) {
return fileSlices.entrySet().stream()
.map(sliceEntry -> sliceEntry.getValue())
.filter(slice -> isFileSliceCommitted(slice));
}
return Stream.empty();
}
/**
* Gets the latest slice - this can contain either
*
* - just the log files without data file
* - (or) data file with 0 or more log files
*
* @return
*/
public Optional<FileSlice> getLatestFileSlice() {
// there should always be one
return getAllFileSlices().findFirst();
}
/**
* Obtain the latest file slice, upto a commitTime i.e <= maxCommitTime
*
* @param maxCommitTime
* @return
*/
public Optional<FileSlice> getLatestFileSliceBeforeOrOn(String maxCommitTime) {
return getAllFileSlices()
.filter(slice ->
HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL))
.findFirst();
}
public Optional<FileSlice> getLatestFileSliceInRange(List<String> commitRange) {
return getAllFileSlices()
.filter(slice -> commitRange.contains(slice.getBaseCommitTime()))
.findFirst();
}
/**
* Stream of committed data files, sorted reverse commit time
*
* @return
*/
public Stream<HoodieDataFile> getAllDataFiles() {
return getAllFileSlices()
.filter(slice -> slice.getDataFile().isPresent())
.map(slice -> slice.getDataFile().get());
}
/**
* Get the latest committed data file
*
* @return
*/
public Optional<HoodieDataFile> getLatestDataFile() {
return getAllDataFiles().findFirst();
}
/**
* Get the latest data file, that is <= max commit time
*
* @param maxCommitTime
* @return
*/
public Optional<HoodieDataFile> getLatestDataFileBeforeOrOn(String maxCommitTime) {
return getAllDataFiles()
.filter(dataFile ->
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL))
.findFirst();
}
/**
* Get the latest data file, that is contained within the provided commit range.
*
* @param commitRange
* @return
*/
public Optional<HoodieDataFile> getLatestDataFileInRange(List<String> commitRange) {
return getAllDataFiles()
.filter(dataFile -> commitRange.contains(dataFile.getCommitTime()))
.findFirst();
}
/**
* Obtain the latest log file (based on latest committed data file),
* currently being appended to
*
* @return logfile if present, empty if no log file has been opened already.
*/
public Optional<HoodieLogFile> getLatestLogFile() {
Optional<FileSlice> latestSlice = getLatestFileSlice();
if (latestSlice.isPresent() && latestSlice.get().getLogFiles().count() > 0) {
return latestSlice.get().getLogFiles().findFirst();
}
return Optional.empty();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieFileGroup {");
sb.append("id=").append(id);
sb.append(", fileSlices='").append(fileSlices).append('\'');
sb.append('}');
return sb.toString();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieFileGroup {");
sb.append("id=").append(id);
sb.append(", fileSlices='").append(fileSlices).append('\'');
sb.append('}');
return sb.toString();
}
}

View File

@@ -17,57 +17,58 @@
package com.uber.hoodie.common.model;
import com.google.common.base.Objects;
import java.io.Serializable;
/**
* HoodieKey consists of
*
* - recordKey : a recordKey that acts as primary key for a record
* - partitionPath : path to the partition that contains the record
* - recordKey : a recordKey that acts as primary key for a record - partitionPath : path to the
* partition that contains the record
*/
public class HoodieKey implements Serializable {
private final String recordKey;
private final String recordKey;
private final String partitionPath;
private final String partitionPath;
public HoodieKey(String recordKey, String partitionPath) {
this.recordKey = recordKey;
this.partitionPath = partitionPath;
public HoodieKey(String recordKey, String partitionPath) {
this.recordKey = recordKey;
this.partitionPath = partitionPath;
}
public String getRecordKey() {
return recordKey;
}
public String getPartitionPath() {
return partitionPath;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
public String getRecordKey() {
return recordKey;
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieKey otherKey = (HoodieKey) o;
return Objects.equal(recordKey, otherKey.recordKey) &&
Objects.equal(partitionPath, otherKey.partitionPath);
}
public String getPartitionPath() {
return partitionPath;
}
@Override
public int hashCode() {
return Objects.hashCode(recordKey, partitionPath);
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieKey otherKey = (HoodieKey) o;
return Objects.equal(recordKey, otherKey.recordKey) &&
Objects.equal(partitionPath, otherKey.partitionPath);
}
@Override
public int hashCode() {
return Objects.hashCode(recordKey, partitionPath);
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieKey {");
sb.append(" recordKey=").append(recordKey);
sb.append(" partitionPath=").append(partitionPath);
sb.append('}');
return sb.toString();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieKey {");
sb.append(" recordKey=").append(recordKey);
sb.append(" partitionPath=").append(partitionPath);
sb.append('}');
return sb.toString();
}
}

View File

@@ -19,13 +19,13 @@
package com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.io.Serializable;
import java.util.Comparator;
import java.util.Optional;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
/**
* Abstracts a single log file. Contains methods to extract metadata like the fileId, version and
@@ -34,73 +34,74 @@ import java.util.Optional;
* Also contains logic to roll-over the log file
*/
public class HoodieLogFile implements Serializable {
public static final String DELTA_EXTENSION = ".log";
private final Path path;
private Optional<FileStatus> fileStatus;
public static final String DELTA_EXTENSION = ".log";
public HoodieLogFile(FileStatus fileStatus) {
this(fileStatus.getPath());
this.fileStatus = Optional.of(fileStatus);
}
private final Path path;
private Optional<FileStatus> fileStatus;
public HoodieLogFile(Path logPath) {
this.path = logPath;
this.fileStatus = Optional.empty();
}
public HoodieLogFile(FileStatus fileStatus) {
this(fileStatus.getPath());
this.fileStatus = Optional.of(fileStatus);
}
public String getFileId() {
return FSUtils.getFileIdFromLogPath(path);
}
public HoodieLogFile(Path logPath) {
this.path = logPath;
this.fileStatus = Optional.empty();
}
public String getBaseCommitTime() {
return FSUtils.getBaseCommitTimeFromLogPath(path);
}
public String getFileId() {
return FSUtils.getFileIdFromLogPath(path);
}
public int getLogVersion() {
return FSUtils.getFileVersionFromLog(path);
}
public String getBaseCommitTime() {
return FSUtils.getBaseCommitTimeFromLogPath(path);
}
public String getFileExtension() {
return FSUtils.getFileExtensionFromLog(path);
}
public int getLogVersion() {
return FSUtils.getFileVersionFromLog(path);
}
public Path getPath() {
return path;
}
public String getFileExtension() {
return FSUtils.getFileExtensionFromLog(path);
}
public String getFileName() {
return path.getName();
}
public Path getPath() {
return path;
}
public Optional<FileStatus> getFileStatus() {
return fileStatus;
}
public String getFileName() {
return path.getName();
}
public Optional<Long> getFileSize() {
return fileStatus.map(FileStatus::getLen);
}
public Optional<FileStatus> getFileStatus() {
return fileStatus;
}
public HoodieLogFile rollOver(FileSystem fs) throws IOException {
String fileId = getFileId();
String baseCommitTime = getBaseCommitTime();
String extension = "." + FSUtils.getFileExtensionFromLog(path);
int newVersion = FSUtils
.computeNextLogVersion(fs, path.getParent(), fileId,
extension, baseCommitTime);
return new HoodieLogFile(new Path(path.getParent(),
FSUtils.makeLogFileName(fileId, extension, baseCommitTime, newVersion)));
}
public Optional<Long> getFileSize() {
return fileStatus.map(FileStatus::getLen);
}
public static Comparator<HoodieLogFile> getLogVersionComparator() {
return (o1, o2) -> {
// reverse the order
return new Integer(o2.getLogVersion()).compareTo(o1.getLogVersion());
};
}
public HoodieLogFile rollOver(FileSystem fs) throws IOException {
String fileId = getFileId();
String baseCommitTime = getBaseCommitTime();
String extension = "." + FSUtils.getFileExtensionFromLog(path);
int newVersion = FSUtils
.computeNextLogVersion(fs, path.getParent(), fileId,
extension, baseCommitTime);
return new HoodieLogFile(new Path(path.getParent(),
FSUtils.makeLogFileName(fileId, extension, baseCommitTime, newVersion)));
}
@Override
public String toString() {
return "HoodieLogFile {" + path + '}';
}
public static Comparator<HoodieLogFile> getLogVersionComparator() {
return (o1, o2) -> {
// reverse the order
return new Integer(o2.getLogVersion()).compareTo(o1.getLogVersion());
};
}
@Override
public String toString() {
return "HoodieLogFile {" + path + '}';
}
}

View File

@@ -17,7 +17,8 @@
package com.uber.hoodie.common.model;
import com.uber.hoodie.exception.HoodieException;
import java.io.IOException;
import java.util.Properties;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -25,117 +26,119 @@ import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Properties;
/**
* The metadata that goes into the meta file in each partition
*/
public class HoodiePartitionMetadata {
public static final String HOODIE_PARTITION_METAFILE = ".hoodie_partition_metadata";
public static final String PARTITION_DEPTH_KEY = "partitionDepth";
public static final String COMMIT_TIME_KEY = "commitTime";
public static final String HOODIE_PARTITION_METAFILE = ".hoodie_partition_metadata";
public static final String PARTITION_DEPTH_KEY = "partitionDepth";
public static final String COMMIT_TIME_KEY = "commitTime";
/**
* Contents of the metadata
*/
private final Properties props;
/**
* Contents of the metadata
*/
private final Properties props;
/**
* Path to the partition, about which we have the metadata
*/
private final Path partitionPath;
/**
* Path to the partition, about which we have the metadata
*/
private final Path partitionPath;
private final FileSystem fs;
private final FileSystem fs;
private static Logger log = LogManager.getLogger(HoodiePartitionMetadata.class);
private static Logger log = LogManager.getLogger(HoodiePartitionMetadata.class);
/**
* Construct metadata from existing partition
*/
public HoodiePartitionMetadata(FileSystem fs, Path partitionPath) {
this.fs = fs;
this.props = new Properties();
this.partitionPath = partitionPath;
/**
* Construct metadata from existing partition
*/
public HoodiePartitionMetadata(FileSystem fs, Path partitionPath) {
this.fs = fs;
this.props = new Properties();
this.partitionPath = partitionPath;
}
/**
* Construct metadata object to be written out.
*/
public HoodiePartitionMetadata(FileSystem fs, String commitTime, Path basePath,
Path partitionPath) {
this(fs, partitionPath);
props.setProperty(COMMIT_TIME_KEY, commitTime);
props
.setProperty(PARTITION_DEPTH_KEY, String.valueOf(partitionPath.depth() - basePath.depth()));
}
public int getPartitionDepth() {
if (!props.containsKey(PARTITION_DEPTH_KEY)) {
throw new HoodieException("Could not find partitionDepth in partition metafile");
}
return Integer.parseInt(props.getProperty(PARTITION_DEPTH_KEY));
}
/**
* Construct metadata object to be written out.
*/
public HoodiePartitionMetadata(FileSystem fs, String commitTime, Path basePath, Path partitionPath) {
this(fs, partitionPath);
props.setProperty(COMMIT_TIME_KEY, commitTime);
props.setProperty(PARTITION_DEPTH_KEY, String.valueOf(partitionPath.depth() - basePath.depth()));
}
/**
* Write the metadata safely into partition atomically.
*/
public void trySave(int taskPartitionId) {
Path tmpMetaPath = new Path(partitionPath,
HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE + "_" + taskPartitionId);
Path metaPath = new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
boolean metafileExists = false;
public int getPartitionDepth() {
if (!props.containsKey(PARTITION_DEPTH_KEY)) {
throw new HoodieException("Could not find partitionDepth in partition metafile");
}
return Integer.parseInt(props.getProperty(PARTITION_DEPTH_KEY));
}
/**
* Write the metadata safely into partition atomically.
*
* @param taskPartitionId
*/
public void trySave(int taskPartitionId) {
Path tmpMetaPath = new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE + "_" + taskPartitionId);
Path metaPath = new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
boolean metafileExists = false;
try {
metafileExists = fs.exists(metaPath);
if (!metafileExists) {
// write to temporary file
FSDataOutputStream os = fs.create(tmpMetaPath, true);
props.store(os, "partition metadata");
os.hsync();
os.hflush();
os.close();
// move to actual path
fs.rename(tmpMetaPath, metaPath);
}
} catch (IOException ioe) {
log.warn(
"Error trying to save partition metadata (this is okay, as long as atleast 1 of these succced), "
+
partitionPath, ioe);
} finally {
if (!metafileExists) {
try {
metafileExists = fs.exists(metaPath);
if (!metafileExists) {
// write to temporary file
FSDataOutputStream os = fs.create(tmpMetaPath, true);
props.store(os, "partition metadata");
os.hsync();
os.hflush();
os.close();
// move to actual path
fs.rename(tmpMetaPath, metaPath);
}
// clean up tmp file, if still lying around
if (fs.exists(tmpMetaPath)) {
fs.delete(tmpMetaPath, false);
}
} catch (IOException ioe) {
log.warn("Error trying to save partition metadata (this is okay, as long as atleast 1 of these succced), " +
partitionPath, ioe);
} finally {
if (!metafileExists) {
try {
// clean up tmp file, if still lying around
if (fs.exists(tmpMetaPath)) {
fs.delete(tmpMetaPath, false);
}
} catch (IOException ioe) {
log.warn("Error trying to clean up temporary files for " + partitionPath, ioe);
}
}
log.warn("Error trying to clean up temporary files for " + partitionPath, ioe);
}
}
}
}
/**
* Read out the metadata for this partition
*/
public void readFromFS() {
try {
Path metaFile = new Path(partitionPath, HOODIE_PARTITION_METAFILE);
FSDataInputStream is = fs.open(metaFile);
props.load(is);
} catch (IOException ioe) {
throw new HoodieException("Error reading Hoodie partition metadata for " + partitionPath, ioe);
}
/**
* Read out the metadata for this partition
*/
public void readFromFS() {
try {
Path metaFile = new Path(partitionPath, HOODIE_PARTITION_METAFILE);
FSDataInputStream is = fs.open(metaFile);
props.load(is);
} catch (IOException ioe) {
throw new HoodieException("Error reading Hoodie partition metadata for " + partitionPath,
ioe);
}
}
// methods related to partition meta data
public static boolean hasPartitionMetadata(FileSystem fs, Path partitionPath) {
try {
return fs.exists(new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE));
} catch (IOException ioe) {
throw new HoodieException("Error checking Hoodie partition metadata for " + partitionPath, ioe);
}
// methods related to partition meta data
public static boolean hasPartitionMetadata(FileSystem fs, Path partitionPath) {
try {
return fs.exists(new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE));
} catch (IOException ioe) {
throw new HoodieException("Error checking Hoodie partition metadata for " + partitionPath,
ioe);
}
}
}

View File

@@ -17,7 +17,6 @@
package com.uber.hoodie.common.model;
import com.google.common.base.Objects;
import java.io.Serializable;
import java.util.Optional;
@@ -26,129 +25,131 @@ import java.util.Optional;
*/
public class HoodieRecord<T extends HoodieRecordPayload> implements Serializable {
public static String COMMIT_TIME_METADATA_FIELD = "_hoodie_commit_time";
public static String COMMIT_SEQNO_METADATA_FIELD = "_hoodie_commit_seqno";
public static String RECORD_KEY_METADATA_FIELD = "_hoodie_record_key";
public static String PARTITION_PATH_METADATA_FIELD = "_hoodie_partition_path";
public static String FILENAME_METADATA_FIELD = "_hoodie_file_name";
public static String COMMIT_TIME_METADATA_FIELD = "_hoodie_commit_time";
public static String COMMIT_SEQNO_METADATA_FIELD = "_hoodie_commit_seqno";
public static String RECORD_KEY_METADATA_FIELD = "_hoodie_record_key";
public static String PARTITION_PATH_METADATA_FIELD = "_hoodie_partition_path";
public static String FILENAME_METADATA_FIELD = "_hoodie_file_name";
/**
* Identifies the record across the table
*/
private HoodieKey key;
/**
* Identifies the record across the table
*/
private HoodieKey key;
/**
* Actual payload of the record
*/
private T data;
/**
* Actual payload of the record
*/
private T data;
/**
* Current location of record on storage. Filled in by looking up index
*/
private HoodieRecordLocation currentLocation;
/**
* Current location of record on storage. Filled in by looking up index
*/
private HoodieRecordLocation currentLocation;
/**
* New location of record on storage, after written
*/
private HoodieRecordLocation newLocation;
/**
* New location of record on storage, after written
*/
private HoodieRecordLocation newLocation;
public HoodieRecord(HoodieKey key, T data) {
this.key = key;
this.data = data;
this.currentLocation = null;
this.newLocation = null;
public HoodieRecord(HoodieKey key, T data) {
this.key = key;
this.data = data;
this.currentLocation = null;
this.newLocation = null;
}
public HoodieKey getKey() {
return key;
}
public T getData() {
if (data == null) {
throw new IllegalStateException("Payload already deflated for record.");
}
return data;
}
public HoodieKey getKey() {
return key;
/**
* Release the actual payload, to ease memory pressure. To be called after the record has been
* written to storage. Once deflated, cannot be inflated.
*/
public void deflate() {
this.data = null;
}
/**
* Sets the current currentLocation of the record. This should happen exactly-once
*/
public HoodieRecord setCurrentLocation(HoodieRecordLocation location) {
assert currentLocation == null;
this.currentLocation = location;
return this;
}
public HoodieRecordLocation getCurrentLocation() {
return currentLocation;
}
/**
* Sets the new currentLocation of the record, after being written. This again should happen
* exactly-once.
*/
public HoodieRecord setNewLocation(HoodieRecordLocation location) {
assert newLocation == null;
this.newLocation = location;
return this;
}
public Optional<HoodieRecordLocation> getNewLocation() {
return Optional.of(this.newLocation);
}
public boolean isCurrentLocationKnown() {
return this.currentLocation != null;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
public T getData() {
if (data == null) {
throw new IllegalStateException("Payload already deflated for record.");
}
return data;
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieRecord that = (HoodieRecord) o;
return Objects.equal(key, that.key) &&
Objects.equal(data, that.data) &&
Objects.equal(currentLocation, that.currentLocation) &&
Objects.equal(newLocation, that.newLocation);
}
/**
* Release the actual payload, to ease memory pressure. To be called after the record
* has been written to storage. Once deflated, cannot be inflated.
*/
public void deflate() {
this.data = null;
}
@Override
public int hashCode() {
return Objects.hashCode(key, data, currentLocation, newLocation);
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieRecord{");
sb.append("key=").append(key);
sb.append(", currentLocation='").append(currentLocation).append('\'');
sb.append(", newLocation='").append(newLocation).append('\'');
sb.append('}');
return sb.toString();
}
/**
* Sets the current currentLocation of the record. This should happen exactly-once
*/
public HoodieRecord setCurrentLocation(HoodieRecordLocation location) {
assert currentLocation == null;
this.currentLocation = location;
return this;
}
public static String generateSequenceId(String commitTime, int partitionId, long recordIndex) {
return commitTime + "_" + partitionId + "_" + recordIndex;
}
public HoodieRecordLocation getCurrentLocation() {
return currentLocation;
}
public String getPartitionPath() {
assert key != null;
return key.getPartitionPath();
}
/**
* Sets the new currentLocation of the record, after being written. This again should happen
* exactly-once.
*/
public HoodieRecord setNewLocation(HoodieRecordLocation location) {
assert newLocation == null;
this.newLocation = location;
return this;
}
public Optional<HoodieRecordLocation> getNewLocation() {
return Optional.of(this.newLocation);
}
public boolean isCurrentLocationKnown() {
return this.currentLocation != null;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieRecord that = (HoodieRecord) o;
return Objects.equal(key, that.key) &&
Objects.equal(data, that.data) &&
Objects.equal(currentLocation, that.currentLocation) &&
Objects.equal(newLocation, that.newLocation);
}
@Override
public int hashCode() {
return Objects.hashCode(key, data, currentLocation, newLocation);
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieRecord{");
sb.append("key=").append(key);
sb.append(", currentLocation='").append(currentLocation).append('\'');
sb.append(", newLocation='").append(newLocation).append('\'');
sb.append('}');
return sb.toString();
}
public static String generateSequenceId(String commitTime, int partitionId, long recordIndex) {
return commitTime + "_" + partitionId + "_" + recordIndex;
}
public String getPartitionPath() {
assert key != null;
return key.getPartitionPath();
}
public String getRecordKey() {
assert key != null;
return key.getRecordKey();
}
public String getRecordKey() {
assert key != null;
return key.getRecordKey();
}
}

View File

@@ -17,7 +17,6 @@
package com.uber.hoodie.common.model;
import com.google.common.base.Objects;
import java.io.Serializable;
/**
@@ -26,44 +25,46 @@ import java.io.Serializable;
*/
public class HoodieRecordLocation implements Serializable {
private final String commitTime;
private final String fileId;
private final String commitTime;
private final String fileId;
public HoodieRecordLocation(String commitTime, String fileId) {
this.commitTime = commitTime;
this.fileId = fileId;
}
public HoodieRecordLocation(String commitTime, String fileId) {
this.commitTime = commitTime;
this.fileId = fileId;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieRecordLocation otherLoc = (HoodieRecordLocation) o;
return Objects.equal(commitTime, otherLoc.commitTime) &&
Objects.equal(fileId, otherLoc.fileId);
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieRecordLocation otherLoc = (HoodieRecordLocation) o;
return Objects.equal(commitTime, otherLoc.commitTime) &&
Objects.equal(fileId, otherLoc.fileId);
}
@Override
public int hashCode() {
return Objects.hashCode(commitTime, fileId);
}
@Override
public int hashCode() {
return Objects.hashCode(commitTime, fileId);
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieRecordLocation {");
sb.append("commitTime=").append(commitTime).append(", ");
sb.append("fileId=").append(fileId);
sb.append('}');
return sb.toString();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieRecordLocation {");
sb.append("commitTime=").append(commitTime).append(", ");
sb.append("fileId=").append(fileId);
sb.append('}');
return sb.toString();
}
public String getCommitTime() {
return commitTime;
}
public String getCommitTime() {
return commitTime;
}
public String getFileId() {
return fileId;
}
public String getFileId() {
return fileId;
}
}

View File

@@ -16,54 +16,55 @@
package com.uber.hoodie.common.model;
import java.io.IOException;
import java.io.Serializable;
import java.util.Map;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import java.io.IOException;
import java.io.Serializable;
import java.util.Optional;
/**
* Every Hoodie dataset has an implementation of the <code>HoodieRecordPayload</code>
* This abstracts out callbacks which depend on record specific logic
* Every Hoodie dataset has an implementation of the <code>HoodieRecordPayload</code> This abstracts
* out callbacks which depend on record specific logic
*/
public interface HoodieRecordPayload<T extends HoodieRecordPayload> extends Serializable {
/**
* When more than one HoodieRecord have the same HoodieKey, this function combines them
* before attempting to insert/upsert (if combining turned on in HoodieClientConfig)
*/
T preCombine(T another);
/**
*
* This methods lets you write custom merging/combining logic to produce new values
* as a function of current value on storage and whats contained in this object.
*
* eg:
* 1) You are updating counters, you may want to add counts to currentValue and write back updated counts
* 2) You may be reading DB redo logs, and merge them with current image for a database row on storage
*
* @param currentValue Current value in storage, to merge/combine this payload with
* @param schema Schema used for record
* @return new combined/merged value to be written back to storage. EMPTY to skip writing this record.
*/
Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException;
/**
* When more than one HoodieRecord have the same HoodieKey, this function combines them before
* attempting to insert/upsert (if combining turned on in HoodieClientConfig)
*/
T preCombine(T another);
/**
* Generates an avro record out of the given HoodieRecordPayload, to be written out to storage.
* Called when writing a new value for the given HoodieKey, wherein there is no existing record in
* storage to be combined against. (i.e insert)
* Return EMPTY to skip writing this record.
*/
Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException;
/**
* This methods lets you write custom merging/combining logic to produce new values as a function
* of current value on storage and whats contained in this object.
*
* eg: 1) You are updating counters, you may want to add counts to currentValue and write back
* updated counts 2) You may be reading DB redo logs, and merge them with current image for a
* database row on storage
*
* @param currentValue Current value in storage, to merge/combine this payload with
* @param schema Schema used for record
* @return new combined/merged value to be written back to storage. EMPTY to skip writing this
* record.
*/
Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException;
/**
* This method can be used to extract some metadata from HoodieRecordPayload. The metadata is passed
* to {@code WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()} in order to compute
* some aggregate metrics using the metadata in the context of a write success or failure.
*/
default Optional<Map<String, String>> getMetadata() {
return Optional.empty();
}
/**
* Generates an avro record out of the given HoodieRecordPayload, to be written out to storage.
* Called when writing a new value for the given HoodieKey, wherein there is no existing record in
* storage to be combined against. (i.e insert) Return EMPTY to skip writing this record.
*/
Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException;
/**
* This method can be used to extract some metadata from HoodieRecordPayload. The metadata is
* passed to {@code WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()} in order to
* compute some aggregate metrics using the metadata in the context of a write success or
* failure.
*/
default Optional<Map<String, String>> getMetadata() {
return Optional.empty();
}
}

View File

@@ -31,5 +31,5 @@ package com.uber.hoodie.common.model;
* SIMPLE_LSM - A simple 2 level LSM tree.
*/
public enum HoodieTableType {
COPY_ON_WRITE, MERGE_ON_READ
COPY_ON_WRITE, MERGE_ON_READ
}

View File

@@ -17,8 +17,6 @@
package com.uber.hoodie.common.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.uber.hoodie.common.util.FSUtils;
import java.io.Serializable;
/**
@@ -27,143 +25,153 @@ import java.io.Serializable;
@JsonIgnoreProperties(ignoreUnknown = true)
public class HoodieWriteStat implements Serializable {
public static final String NULL_COMMIT = "null";
public static final String NULL_COMMIT = "null";
/**
* Id of the file being written
*/
private String fileId;
/**
* Id of the file being written
*/
private String fileId;
/**
* Relative path to the file from the base path
*/
private String path;
/**
* Relative path to the file from the base path
*/
private String path;
/**
* The previous version of the file. (null if this is the first version. i.e insert)
*/
private String prevCommit;
/**
* The previous version of the file. (null if this is the first version. i.e insert)
*/
private String prevCommit;
/**
* Total number of records written for this file.
* - for updates, its the entire number of records in the file
* - for inserts, its the actual number of records inserted.
*/
private long numWrites;
/**
* Total number of records written for this file. - for updates, its the entire number of records
* in the file - for inserts, its the actual number of records inserted.
*/
private long numWrites;
/**
* Total number of records deleted.
*/
private long numDeletes;
/**
* Total number of records deleted.
*/
private long numDeletes;
/**
* Total number of records actually changed. (0 for inserts)
*/
private long numUpdateWrites;
/**
* Total number of records actually changed. (0 for inserts)
*/
private long numUpdateWrites;
/**
* Total size of file written
*/
private long totalWriteBytes;
/**
* Total size of file written
*/
private long totalWriteBytes;
/**
* Total number of records, that were n't able to be written due to errors.
*/
private long totalWriteErrors;
/**
* Total number of records, that were n't able to be written due to errors.
*/
private long totalWriteErrors;
public HoodieWriteStat() {
// called by jackson json lib
public HoodieWriteStat() {
// called by jackson json lib
}
public void setFileId(String fileId) {
this.fileId = fileId;
}
public void setPath(String path) {
this.path = path;
}
public void setPrevCommit(String prevCommit) {
this.prevCommit = prevCommit;
}
public void setNumWrites(long numWrites) {
this.numWrites = numWrites;
}
public void setNumDeletes(long numDeletes) {
this.numDeletes = numDeletes;
}
public void setNumUpdateWrites(long numUpdateWrites) {
this.numUpdateWrites = numUpdateWrites;
}
public long getTotalWriteBytes() {
return totalWriteBytes;
}
public void setTotalWriteBytes(long totalWriteBytes) {
this.totalWriteBytes = totalWriteBytes;
}
public long getTotalWriteErrors() {
return totalWriteErrors;
}
public void setTotalWriteErrors(long totalWriteErrors) {
this.totalWriteErrors = totalWriteErrors;
}
public String getPrevCommit() {
return prevCommit;
}
public long getNumWrites() {
return numWrites;
}
public long getNumDeletes() {
return numDeletes;
}
public long getNumUpdateWrites() {
return numUpdateWrites;
}
public String getFileId() {
return fileId;
}
public String getPath() {
return path;
}
@Override
public String toString() {
return new StringBuilder()
.append("HoodieWriteStat {")
.append("path=" + path)
.append(", prevCommit='" + prevCommit + '\'')
.append(", numWrites=" + numWrites)
.append(", numDeletes=" + numDeletes)
.append(", numUpdateWrites=" + numUpdateWrites)
.append(", numWriteBytes=" + totalWriteBytes)
.append('}')
.toString();
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
public void setFileId(String fileId) {
this.fileId = fileId;
HoodieWriteStat that = (HoodieWriteStat) o;
if (!path.equals(that.path)) {
return false;
}
return prevCommit.equals(that.prevCommit);
public void setPath(String path) { this.path = path; }
}
public void setPrevCommit(String prevCommit) {
this.prevCommit = prevCommit;
}
public void setNumWrites(long numWrites) {
this.numWrites = numWrites;
}
public void setNumDeletes(long numDeletes) {
this.numDeletes = numDeletes;
}
public void setNumUpdateWrites(long numUpdateWrites) {
this.numUpdateWrites = numUpdateWrites;
}
public long getTotalWriteBytes() {
return totalWriteBytes;
}
public void setTotalWriteBytes(long totalWriteBytes) {
this.totalWriteBytes = totalWriteBytes;
}
public long getTotalWriteErrors() { return totalWriteErrors; }
public void setTotalWriteErrors(long totalWriteErrors) { this.totalWriteErrors = totalWriteErrors; }
public String getPrevCommit() {
return prevCommit;
}
public long getNumWrites() {
return numWrites;
}
public long getNumDeletes() {
return numDeletes;
}
public long getNumUpdateWrites() {
return numUpdateWrites;
}
public String getFileId() {
return fileId;
}
public String getPath() { return path; }
@Override
public String toString() {
return new StringBuilder()
.append("HoodieWriteStat {")
.append("path=" + path)
.append(", prevCommit='" + prevCommit + '\'')
.append(", numWrites=" + numWrites)
.append(", numDeletes=" + numDeletes)
.append(", numUpdateWrites=" + numUpdateWrites)
.append(", numWriteBytes=" + totalWriteBytes)
.append('}')
.toString();
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieWriteStat that = (HoodieWriteStat) o;
if (!path.equals(that.path))
return false;
return prevCommit.equals(that.prevCommit);
}
@Override
public int hashCode() {
int result = path.hashCode();
result = 31 * result + prevCommit.hashCode();
return result;
}
@Override
public int hashCode() {
int result = path.hashCode();
result = 31 * result + prevCommit.hashCode();
return result;
}
}

View File

@@ -19,8 +19,11 @@ package com.uber.hoodie.common.table;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieFileFormat;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.io.Serializable;
import java.util.Date;
import java.util.Properties;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -28,144 +31,134 @@ import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.io.Serializable;
import java.util.Date;
import java.util.Properties;
/**
* Configurations on the Hoodie Table like type of ingestion, storage formats, hive table name etc
* Configurations are loaded from hoodie.properties, these properties are usually set during initializing a path as hoodie base path
* and never changes during the lifetime of a hoodie dataset.
* Configurations are loaded from hoodie.properties, these properties are usually set during
* initializing a path as hoodie base path and never changes during the lifetime of a hoodie
* dataset.
*
* @see HoodieTableMetaClient
* @since 0.3.0
*/
public class HoodieTableConfig implements Serializable {
private final transient static Logger log = LogManager.getLogger(HoodieTableConfig.class);
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME =
"hoodie.table.ro.file.format";
public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME =
"hoodie.table.rt.file.format";
public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class";
private final transient static Logger log = LogManager.getLogger(HoodieTableConfig.class);
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET;
public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
private Properties props;
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME =
"hoodie.table.ro.file.format";
public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME =
"hoodie.table.rt.file.format";
public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class";
public HoodieTableConfig(FileSystem fs, String metaPath) {
Properties props = new Properties();
Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
log.info("Loading dataset properties from " + propertyPath);
try {
try (FSDataInputStream inputStream = fs.open(propertyPath)) {
props.load(inputStream);
}
} catch (IOException e) {
throw new HoodieIOException("Could not load Hoodie properties from " + propertyPath, e);
}
this.props = props;
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET;
public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
private Properties props;
public HoodieTableConfig(FileSystem fs, String metaPath) {
Properties props = new Properties();
Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
log.info("Loading dataset properties from " + propertyPath);
try {
try (FSDataInputStream inputStream = fs.open(propertyPath)) {
props.load(inputStream);
}
} catch (IOException e) {
throw new HoodieIOException("Could not load Hoodie properties from " + propertyPath, e);
}
this.props = props;
}
/**
* For serailizing and de-serializing
* @deprecated
*/
public HoodieTableConfig() {
/**
* For serailizing and de-serializing
*
* @deprecated
*/
public HoodieTableConfig() {
}
/**
* Initialize the hoodie meta directory and any necessary files inside the meta (including the
* hoodie.properties)
*/
public static void createHoodieProperties(FileSystem fs, Path metadataFolder,
Properties properties) throws IOException {
if (!fs.exists(metadataFolder)) {
fs.mkdirs(metadataFolder);
}
/**
* Initialize the hoodie meta directory and any necessary files inside the meta (including the hoodie.properties)
*
* @param metadataFolder
* @param properties
* @throws IOException
*/
public static void createHoodieProperties(FileSystem fs, Path metadataFolder,
Properties properties) throws IOException {
if (!fs.exists(metadataFolder)) {
fs.mkdirs(metadataFolder);
}
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
FSDataOutputStream outputStream = fs.create(propertyPath);
try {
if (!properties.containsKey(HOODIE_TABLE_NAME_PROP_NAME)) {
throw new IllegalArgumentException(
HOODIE_TABLE_NAME_PROP_NAME + " property needs to be specified");
}
if (!properties.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
properties.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name());
}
if (properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME) == HoodieTableType.MERGE_ON_READ.name()
&& !properties.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME)) {
properties.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS);
}
properties
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
} finally {
outputStream.close();
}
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
FSDataOutputStream outputStream = fs.create(propertyPath);
try {
if (!properties.containsKey(HOODIE_TABLE_NAME_PROP_NAME)) {
throw new IllegalArgumentException(
HOODIE_TABLE_NAME_PROP_NAME + " property needs to be specified");
}
if (!properties.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
properties.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name());
}
if (properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME) == HoodieTableType.MERGE_ON_READ
.name()
&& !properties.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME)) {
properties.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS);
}
properties
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
} finally {
outputStream.close();
}
}
/**
* Read the table type from the table properties and if not found, return the default
*
* @return
*/
public HoodieTableType getTableType() {
if (props.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
return HoodieTableType.valueOf(props.getProperty(HOODIE_TABLE_TYPE_PROP_NAME));
}
return DEFAULT_TABLE_TYPE;
/**
* Read the table type from the table properties and if not found, return the default
*/
public HoodieTableType getTableType() {
if (props.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
return HoodieTableType.valueOf(props.getProperty(HOODIE_TABLE_TYPE_PROP_NAME));
}
return DEFAULT_TABLE_TYPE;
}
/**
* Read the payload class for HoodieRecords from the table properties
*
* @return
*/
public String getPayloadClass() {
return props.getProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS);
}
/**
* Read the payload class for HoodieRecords from the table properties
*/
public String getPayloadClass() {
return props.getProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS);
}
/**
* Read the table name
*
* @return
*/
public String getTableName() {
return props.getProperty(HOODIE_TABLE_NAME_PROP_NAME);
}
/**
* Read the table name
*/
public String getTableName() {
return props.getProperty(HOODIE_TABLE_NAME_PROP_NAME);
}
/**
* Get the Read Optimized Storage Format
*
* @return HoodieFileFormat for the Read Optimized Storage format
*/
public HoodieFileFormat getROFileFormat() {
if (props.containsKey(HOODIE_RO_FILE_FORMAT_PROP_NAME)) {
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RO_FILE_FORMAT_PROP_NAME));
}
return DEFAULT_RO_FILE_FORMAT;
/**
* Get the Read Optimized Storage Format
*
* @return HoodieFileFormat for the Read Optimized Storage format
*/
public HoodieFileFormat getROFileFormat() {
if (props.containsKey(HOODIE_RO_FILE_FORMAT_PROP_NAME)) {
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RO_FILE_FORMAT_PROP_NAME));
}
return DEFAULT_RO_FILE_FORMAT;
}
/**
* Get the Read Optimized Storage Format
*
* @return HoodieFileFormat for the Read Optimized Storage format
*/
public HoodieFileFormat getRTFileFormat() {
if (props.containsKey(HOODIE_RT_FILE_FORMAT_PROP_NAME)) {
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RT_FILE_FORMAT_PROP_NAME));
}
return DEFAULT_RT_FILE_FORMAT;
/**
* Get the Read Optimized Storage Format
*
* @return HoodieFileFormat for the Read Optimized Storage format
*/
public HoodieFileFormat getRTFileFormat() {
if (props.containsKey(HOODIE_RT_FILE_FORMAT_PROP_NAME)) {
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RT_FILE_FORMAT_PROP_NAME));
}
return DEFAULT_RT_FILE_FORMAT;
}
}

View File

@@ -21,6 +21,11 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.DatasetNotFoundException;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.util.Objects;
import java.util.Properties;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -28,225 +33,205 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.Externalizable;
import java.io.File;
import java.io.IOException;
import java.io.ObjectInput;
import java.io.ObjectInputStream;
import java.io.ObjectOutput;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import java.util.Objects;
import java.util.Properties;
/**
* <code>HoodieTableMetaClient</code> allows to access meta-data about a hoodie table
* It returns meta-data about commits, savepoints, compactions, cleanups as a <code>HoodieTimeline</code>
* Create an instance of the <code>HoodieTableMetaClient</code> with FileSystem and basePath to start getting the meta-data.
* <p>
* All the timelines are computed lazily, once computed the timeline is cached and never refreshed.
* Use the <code>HoodieTimeline.reload()</code> to refresh timelines.
* <code>HoodieTableMetaClient</code> allows to access meta-data about a hoodie table It returns
* meta-data about commits, savepoints, compactions, cleanups as a <code>HoodieTimeline</code>
* Create an instance of the <code>HoodieTableMetaClient</code> with FileSystem and basePath to
* start getting the meta-data. <p> All the timelines are computed lazily, once computed the
* timeline is cached and never refreshed. Use the <code>HoodieTimeline.reload()</code> to refresh
* timelines.
*
* @see HoodieTimeline
* @since 0.3.0
*/
public class HoodieTableMetaClient implements Serializable {
private final transient static Logger log = LogManager.getLogger(HoodieTableMetaClient.class);
public static String METAFOLDER_NAME = ".hoodie";
private String basePath;
private transient FileSystem fs;
private String metaPath;
private HoodieTableType tableType;
private HoodieTableConfig tableConfig;
private HoodieActiveTimeline activeTimeline;
private HoodieArchivedTimeline archivedTimeline;
private final transient static Logger log = LogManager.getLogger(HoodieTableMetaClient.class);
public static String METAFOLDER_NAME = ".hoodie";
public HoodieTableMetaClient(FileSystem fs, String basePath) throws DatasetNotFoundException {
// Do not load any timeline by default
this(fs, basePath, false);
private String basePath;
private transient FileSystem fs;
private String metaPath;
private HoodieTableType tableType;
private HoodieTableConfig tableConfig;
private HoodieActiveTimeline activeTimeline;
private HoodieArchivedTimeline archivedTimeline;
public HoodieTableMetaClient(FileSystem fs, String basePath) throws DatasetNotFoundException {
// Do not load any timeline by default
this(fs, basePath, false);
}
public HoodieTableMetaClient(FileSystem fs, String basePath, boolean loadActiveTimelineOnLoad)
throws DatasetNotFoundException {
log.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath;
this.fs = fs;
Path basePathDir = new Path(this.basePath);
this.metaPath = basePath + File.separator + METAFOLDER_NAME;
Path metaPathDir = new Path(this.metaPath);
DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir);
this.tableConfig = new HoodieTableConfig(fs, metaPath);
this.tableType = tableConfig.getTableType();
log.info("Finished Loading Table of type " + tableType + " from " + basePath);
if (loadActiveTimelineOnLoad) {
log.info("Loading Active commit timeline for " + basePath);
getActiveTimeline();
}
}
public HoodieTableMetaClient(FileSystem fs, String basePath, boolean loadActiveTimelineOnLoad)
throws DatasetNotFoundException {
log.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath;
this.fs = fs;
Path basePathDir = new Path(this.basePath);
this.metaPath = basePath + File.separator + METAFOLDER_NAME;
Path metaPathDir = new Path(this.metaPath);
DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir);
this.tableConfig = new HoodieTableConfig(fs, metaPath);
this.tableType = tableConfig.getTableType();
log.info("Finished Loading Table of type " + tableType + " from " + basePath);
if (loadActiveTimelineOnLoad) {
log.info("Loading Active commit timeline for " + basePath);
getActiveTimeline();
}
}
/**
* For serailizing and de-serializing
*
* @deprecated
*/
public HoodieTableMetaClient() {
}
/**
* For serailizing and de-serializing
*
* @deprecated
*/
public HoodieTableMetaClient() {
}
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
private void writeObject(java.io.ObjectOutputStream out)
throws IOException {
out.defaultWriteObject();
}
private void writeObject(java.io.ObjectOutputStream out)
throws IOException {
out.defaultWriteObject();
}
/**
* @return Base path
*/
public String getBasePath() {
return basePath;
}
/**
* @return Base path
*/
public String getBasePath() {
return basePath;
}
/**
* @return Hoodie Table Type
*/
public HoodieTableType getTableType() {
return tableType;
}
/**
* @return Hoodie Table Type
*/
public HoodieTableType getTableType() {
return tableType;
}
/**
* @return Meta path
*/
public String getMetaPath() {
return metaPath;
}
/**
* @return Meta path
*/
public String getMetaPath() {
return metaPath;
}
/**
* @return Table Config
*/
public HoodieTableConfig getTableConfig() {
return tableConfig;
}
/**
* @return Table Config
*/
public HoodieTableConfig getTableConfig() {
return tableConfig;
}
/**
* Get the FS implementation for this table
*/
public FileSystem getFs() {
return fs;
}
/**
* Get the FS implementation for this table
* @return
*/
public FileSystem getFs() {
return fs;
/**
* Get the active instants as a timeline
*
* @return Active instants timeline
*/
public synchronized HoodieActiveTimeline getActiveTimeline() {
if (activeTimeline == null) {
activeTimeline = new HoodieActiveTimeline(fs, metaPath);
}
return activeTimeline;
}
/**
* Get the active instants as a timeline
*
* @return Active instants timeline
* @throws IOException
*/
public synchronized HoodieActiveTimeline getActiveTimeline() {
if (activeTimeline == null) {
activeTimeline = new HoodieActiveTimeline(fs, metaPath);
}
return activeTimeline;
/**
* Get the archived commits as a timeline. This is costly operation, as all data from the archived
* files are read. This should not be used, unless for historical debugging purposes
*
* @return Active commit timeline
*/
public synchronized HoodieArchivedTimeline getArchivedTimeline() {
if (archivedTimeline == null) {
archivedTimeline = new HoodieArchivedTimeline(fs, metaPath);
}
return archivedTimeline;
}
/**
* Get the archived commits as a timeline. This is costly operation, as all data from the
* archived files are read. This should not be used, unless for historical debugging purposes
*
* @return Active commit timeline
* @throws IOException
*/
public synchronized HoodieArchivedTimeline getArchivedTimeline() {
if (archivedTimeline == null) {
archivedTimeline = new HoodieArchivedTimeline(fs, metaPath);
}
return archivedTimeline;
/**
* Helper method to initialize a given path, as a given storage type and table name
*/
public static HoodieTableMetaClient initTableType(FileSystem fs, String basePath,
HoodieTableType tableType, String tableName, String payloadClassName) throws IOException {
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
if (tableType == HoodieTableType.MERGE_ON_READ) {
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
}
return HoodieTableMetaClient.initializePathAsHoodieDataset(fs, basePath, properties);
}
/**
* Helper method to initialize a given path, as a given storage type and table name
*
* @param fs
* @param basePath
* @param tableType
* @param tableName
* @return
* @throws IOException
*/
public static HoodieTableMetaClient initTableType(FileSystem fs, String basePath, HoodieTableType tableType, String tableName, String payloadClassName) throws IOException {
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
if(tableType == HoodieTableType.MERGE_ON_READ) {
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
}
return HoodieTableMetaClient.initializePathAsHoodieDataset(fs, basePath, properties);
/**
* Helper method to initialize a given path as a hoodie dataset with configs passed in as as
* Properties
*
* @return Instance of HoodieTableMetaClient
*/
public static HoodieTableMetaClient initializePathAsHoodieDataset(FileSystem fs,
String basePath, Properties props) throws IOException {
log.info("Initializing " + basePath + " as hoodie dataset " + basePath);
Path basePathDir = new Path(basePath);
if (!fs.exists(basePathDir)) {
fs.mkdirs(basePathDir);
}
Path metaPathDir = new Path(basePath, METAFOLDER_NAME);
if (!fs.exists(metaPathDir)) {
fs.mkdirs(metaPathDir);
}
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType()
+ " from " + basePath);
return metaClient;
}
/**
* Helper method to initialize a given path as a hoodie dataset with configs passed in as as Properties
*
* @param fs
* @param basePath
* @param props
* @return Instance of HoodieTableMetaClient
* @throws IOException
*/
public static HoodieTableMetaClient initializePathAsHoodieDataset(FileSystem fs,
String basePath, Properties props) throws IOException {
log.info("Initializing " + basePath + " as hoodie dataset " + basePath);
Path basePathDir = new Path(basePath);
if (!fs.exists(basePathDir)) {
fs.mkdirs(basePathDir);
}
Path metaPathDir = new Path(basePath, METAFOLDER_NAME);
if (!fs.exists(metaPathDir)) {
fs.mkdirs(metaPathDir);
}
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType()
+ " from " + basePath);
return metaClient;
}
// HELPER METHODS TO CREATE META FILE NAMES
public static FileStatus[] scanFiles(FileSystem fs, Path metaPath, PathFilter nameFilter)
throws IOException {
return fs.listStatus(metaPath, nameFilter);
}
// HELPER METHODS TO CREATE META FILE NAMES
public static FileStatus[] scanFiles(FileSystem fs, Path metaPath, PathFilter nameFilter)
throws IOException {
return fs.listStatus(metaPath, nameFilter);
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieTableMetaClient that = (HoodieTableMetaClient) o;
return Objects.equals(basePath, that.basePath) && tableType == that.tableType;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieTableMetaClient that = (HoodieTableMetaClient) o;
return Objects.equals(basePath, that.basePath) && tableType == that.tableType;
}
@Override
public int hashCode() {
return Objects.hash(basePath, tableType);
}
@Override
public int hashCode() {
return Objects.hash(basePath, tableType);
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieTableMetaClient{");
sb.append("basePath='").append(basePath).append('\'');
sb.append(", metaPath='").append(metaPath).append('\'');
sb.append(", tableType=").append(tableType);
sb.append('}');
return sb.toString();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieTableMetaClient{");
sb.append("basePath='").append(basePath).append('\'');
sb.append(", metaPath='").append(metaPath).append('\'');
sb.append(", tableType=").append(tableType);
sb.append('}');
return sb.toString();
}
}

View File

@@ -18,19 +18,16 @@ package com.uber.hoodie.common.table;
import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import java.io.Serializable;
import java.util.Optional;
import java.util.function.BiPredicate;
import java.util.stream.Stream;
/**
* HoodieTimeline is a view of meta-data instants in the hoodie dataset.
* Instants are specific points in time represented as HoodieInstant.
* <p>
* Timelines are immutable once created and operations create new instance of
* timelines which filter on the instants and this can be chained.
* HoodieTimeline is a view of meta-data instants in the hoodie dataset. Instants are specific
* points in time represented as HoodieInstant. <p> Timelines are immutable once created and
* operations create new instance of timelines which filter on the instants and this can be
* chained.
*
* @see com.uber.hoodie.common.table.HoodieTableMetaClient
* @see HoodieDefaultTimeline
@@ -38,205 +35,195 @@ import java.util.stream.Stream;
* @since 0.3.0
*/
public interface HoodieTimeline extends Serializable {
String COMMIT_ACTION = "commit";
String DELTA_COMMIT_ACTION = "deltacommit";
String CLEAN_ACTION = "clean";
String ROLLBACK_ACTION = "rollback";
String SAVEPOINT_ACTION = "savepoint";
String COMPACTION_ACTION = "compaction";
String INFLIGHT_EXTENSION = ".inflight";
String COMMIT_EXTENSION = "." + COMMIT_ACTION;
String DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION;
String CLEAN_EXTENSION = "." + CLEAN_ACTION;
String ROLLBACK_EXTENSION = "." + ROLLBACK_ACTION;
String SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION;
String COMPACTION_EXTENSION = "." + COMPACTION_ACTION;
//this is to preserve backwards compatibility on commit in-flight filenames
String INFLIGHT_COMMIT_EXTENSION = INFLIGHT_EXTENSION;
String INFLIGHT_DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_CLEAN_EXTENSION = "." + CLEAN_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_ROLLBACK_EXTENSION = "." + ROLLBACK_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_COMPACTION_EXTENSION = "." + COMPACTION_ACTION + INFLIGHT_EXTENSION;
String COMMIT_ACTION = "commit";
String DELTA_COMMIT_ACTION = "deltacommit";
String CLEAN_ACTION = "clean";
String ROLLBACK_ACTION = "rollback";
String SAVEPOINT_ACTION = "savepoint";
String COMPACTION_ACTION = "compaction";
String INFLIGHT_EXTENSION = ".inflight";
/**
* Filter this timeline to just include the in-flights
*
* @return New instance of HoodieTimeline with just in-flights
*/
HoodieTimeline filterInflights();
String COMMIT_EXTENSION = "." + COMMIT_ACTION;
String DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION;
String CLEAN_EXTENSION = "." + CLEAN_ACTION;
String ROLLBACK_EXTENSION = "." + ROLLBACK_ACTION;
String SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION;
String COMPACTION_EXTENSION = "." + COMPACTION_ACTION;
//this is to preserve backwards compatibility on commit in-flight filenames
String INFLIGHT_COMMIT_EXTENSION = INFLIGHT_EXTENSION;
String INFLIGHT_DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_CLEAN_EXTENSION = "." + CLEAN_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_ROLLBACK_EXTENSION = "." + ROLLBACK_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_COMPACTION_EXTENSION = "." + COMPACTION_ACTION + INFLIGHT_EXTENSION;
/**
* Filter this timeline to just include the completed instants
*
* @return New instance of HoodieTimeline with just completed instants
*/
HoodieTimeline filterCompletedInstants();
/**
* Filter this timeline to just include the in-flights
*
* @return New instance of HoodieTimeline with just in-flights
*/
HoodieTimeline filterInflights();
/**
* Filter this timeline to just include the completed instants
*
* @return New instance of HoodieTimeline with just completed instants
*/
HoodieTimeline filterCompletedInstants();
/**
* Create a new Timeline with instants after startTs and before or on endTs
*
* @param startTs
* @param endTs
*/
HoodieTimeline findInstantsInRange(String startTs, String endTs);
/**
* Create a new Timeline with instants after startTs and before or on endTs
*/
HoodieTimeline findInstantsInRange(String startTs, String endTs);
/**
* Create a new Timeline with all the instants after startTs
*
* @param commitTime
* @param numCommits
*/
HoodieTimeline findInstantsAfter(String commitTime, int numCommits);
/**
* Create a new Timeline with all the instants after startTs
*/
HoodieTimeline findInstantsAfter(String commitTime, int numCommits);
/**
* If the timeline has any instants
*
* @return true if timeline is empty
*/
boolean empty();
/**
* If the timeline has any instants
*
* @return true if timeline is empty
*/
boolean empty();
/**
* @return total number of completed instants
*/
int countInstants();
/**
* @return total number of completed instants
*/
int countInstants();
/**
* @return first completed instant if available
*/
Optional<HoodieInstant> firstInstant();
/**
* @return first completed instant if available
*/
Optional<HoodieInstant> firstInstant();
/**
* @param n
* @return nth completed instant from the first completed instant
*/
Optional<HoodieInstant> nthInstant(int n);
/**
* @return nth completed instant from the first completed instant
*/
Optional<HoodieInstant> nthInstant(int n);
/**
* @return last completed instant if available
*/
Optional<HoodieInstant> lastInstant();
/**
* @return last completed instant if available
*/
Optional<HoodieInstant> lastInstant();
/**
* @param n
* @return nth completed instant going back from the last completed instant
*/
Optional<HoodieInstant> nthFromLastInstant(int n);
/**
* @return nth completed instant going back from the last completed instant
*/
Optional<HoodieInstant> nthFromLastInstant(int n);
/**
* @return true if the passed instant is present as a completed instant on the timeline
*/
boolean containsInstant(HoodieInstant instant);
/**
* @return true if the passed instant is present as a completed instant on the timeline
*/
boolean containsInstant(HoodieInstant instant);
/**
* @return true if the passed instant is present as a completed instant on the timeline or
* if the instant is before the first completed instant in the timeline
*/
boolean containsOrBeforeTimelineStarts(String ts);
/**
* @return true if the passed instant is present as a completed instant on the timeline or if the
* instant is before the first completed instant in the timeline
*/
boolean containsOrBeforeTimelineStarts(String ts);
/**
* @return Get the stream of completed instants
*/
Stream<HoodieInstant> getInstants();
/**
* @return Get the stream of completed instants
*/
Stream<HoodieInstant> getInstants();
/**
* @return true if the passed in instant is before the first completed instant in the timeline
*/
boolean isBeforeTimelineStarts(String ts);
/**
* @return true if the passed in instant is before the first completed instant in the timeline
*/
boolean isBeforeTimelineStarts(String ts);
/**
* Read the completed instant details
*
* @param instant
* @return
*/
Optional<byte[]> getInstantDetails(HoodieInstant instant);
/**
* Read the completed instant details
*/
Optional<byte[]> getInstantDetails(HoodieInstant instant);
/**
* Helper methods to compare instants
**/
BiPredicate<String, String> GREATER_OR_EQUAL =
(commit1, commit2) -> commit1.compareTo(commit2) >= 0;
BiPredicate<String, String> GREATER = (commit1, commit2) -> commit1.compareTo(commit2) > 0;
BiPredicate<String, String> LESSER_OR_EQUAL =
(commit1, commit2) -> commit1.compareTo(commit2) <= 0;
BiPredicate<String, String> LESSER = (commit1, commit2) -> commit1.compareTo(commit2) < 0;
/**
* Helper methods to compare instants
**/
BiPredicate<String, String> GREATER_OR_EQUAL =
(commit1, commit2) -> commit1.compareTo(commit2) >= 0;
BiPredicate<String, String> GREATER = (commit1, commit2) -> commit1.compareTo(commit2) > 0;
BiPredicate<String, String> LESSER_OR_EQUAL =
(commit1, commit2) -> commit1.compareTo(commit2) <= 0;
BiPredicate<String, String> LESSER = (commit1, commit2) -> commit1.compareTo(commit2) < 0;
static boolean compareTimestamps(String commit1, String commit2,
BiPredicate<String, String> predicateToApply) {
return predicateToApply.test(commit1, commit2);
}
static boolean compareTimestamps(String commit1, String commit2,
BiPredicate<String, String> predicateToApply) {
return predicateToApply.test(commit1, commit2);
}
static HoodieInstant getCompletedInstant(final HoodieInstant instant) {
return new HoodieInstant(false, instant.getAction(), instant.getTimestamp());
}
static HoodieInstant getCompletedInstant(final HoodieInstant instant) {
return new HoodieInstant(false, instant.getAction(), instant.getTimestamp());
}
static HoodieInstant getInflightInstant(final HoodieInstant instant) {
return new HoodieInstant(true, instant.getAction(), instant.getTimestamp());
}
static HoodieInstant getInflightInstant(final HoodieInstant instant) {
return new HoodieInstant(true, instant.getAction(), instant.getTimestamp());
}
static String makeCommitFileName(String commitTime) {
return commitTime + HoodieTimeline.COMMIT_EXTENSION;
}
static String makeCommitFileName(String commitTime) {
return commitTime + HoodieTimeline.COMMIT_EXTENSION;
}
static String makeInflightCommitFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
}
static String makeInflightCommitFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
}
static String makeCleanerFileName(String instant) {
return instant + HoodieTimeline.CLEAN_EXTENSION;
}
static String makeCleanerFileName(String instant) {
return instant + HoodieTimeline.CLEAN_EXTENSION;
}
static String makeInflightCleanerFileName(String instant) {
return instant + HoodieTimeline.INFLIGHT_CLEAN_EXTENSION;
}
static String makeInflightCleanerFileName(String instant) {
return instant + HoodieTimeline.INFLIGHT_CLEAN_EXTENSION;
}
static String makeRollbackFileName(String instant) {
return instant + HoodieTimeline.ROLLBACK_EXTENSION;
}
static String makeRollbackFileName(String instant) {
return instant + HoodieTimeline.ROLLBACK_EXTENSION;
}
static String makeInflightRollbackFileName(String instant) {
return instant + HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION;
}
static String makeInflightRollbackFileName(String instant) {
return instant + HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION;
}
static String makeInflightSavePointFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION;
}
static String makeInflightSavePointFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION;
}
static String makeSavePointFileName(String commitTime) {
return commitTime + HoodieTimeline.SAVEPOINT_EXTENSION;
}
static String makeSavePointFileName(String commitTime) {
return commitTime + HoodieTimeline.SAVEPOINT_EXTENSION;
}
static String makeInflightCompactionFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION;
}
static String makeInflightCompactionFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION;
}
static String makeCompactionFileName(String commitTime) {
return commitTime + HoodieTimeline.COMPACTION_EXTENSION;
}
static String makeCompactionFileName(String commitTime) {
return commitTime + HoodieTimeline.COMPACTION_EXTENSION;
}
static String makeInflightDeltaFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION;
}
static String makeInflightDeltaFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION;
}
static String makeDeltaFileName(String commitTime) {
return commitTime + HoodieTimeline.DELTA_COMMIT_EXTENSION;
}
static String makeDeltaFileName(String commitTime) {
return commitTime + HoodieTimeline.DELTA_COMMIT_EXTENSION;
}
static String getCommitFromCommitFile(String commitFileName) {
return commitFileName.split("\\.")[0];
}
static String getCommitFromCommitFile(String commitFileName) {
return commitFileName.split("\\.")[0];
}
static String makeFileNameAsComplete(String fileName) {
return fileName.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
}
static String makeFileNameAsComplete(String fileName) {
return fileName.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
}
static String makeFileNameAsInflight(String fileName) {
return fileName + HoodieTimeline.INFLIGHT_EXTENSION;
}
static String makeFileNameAsInflight(String fileName) {
return fileName + HoodieTimeline.INFLIGHT_EXTENSION;
}
}

View File

@@ -19,13 +19,7 @@ package com.uber.hoodie.common.table;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieFileGroup;
import com.uber.hoodie.common.model.HoodieLogFile;
import org.apache.hadoop.fs.FileStatus;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.function.Predicate;
import java.util.stream.Stream;
/**
@@ -35,70 +29,70 @@ import java.util.stream.Stream;
*/
public interface TableFileSystemView {
/**
* ReadOptimizedView - methods to provide a view of columnar data files only.
*/
interface ReadOptimizedView {
/**
* Stream all the latest data files in the given partition
*/
Stream<HoodieDataFile> getLatestDataFiles(String partitionPath);
/**
* Stream all the latest data files, in the file system view
*/
Stream<HoodieDataFile> getLatestDataFiles();
/**
* Stream all the latest version data files in the given partition with precondition that
* commitTime(file) before maxCommitTime
*/
Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
String maxCommitTime);
/**
* Stream all the latest data files pass
*/
Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn);
/**
* Stream all the data file versions grouped by FileId for a given partition
*/
Stream<HoodieDataFile> getAllDataFiles(String partitionPath);
}
/**
* ReadOptimizedView - methods to provide a view of columnar data files only.
*/
interface ReadOptimizedView {
/**
* RealtimeView - methods to access a combination of columnar data files + log files with real time data.
* Stream all the latest data files in the given partition
*/
interface RealtimeView {
/**
* Stream all the latest file slices in the given partition
*/
Stream<FileSlice> getLatestFileSlices(String partitionPath);
/**
* Stream all the latest file slices in the given partition with precondition that
* commitTime(file) before maxCommitTime
*/
Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath,
String maxCommitTime);
/**
* Stream all the latest file slices, in the given range
*/
Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn);
/**
* Stream all the file slices for a given partition, latest or not.
*/
Stream<FileSlice> getAllFileSlices(String partitionPath);
}
Stream<HoodieDataFile> getLatestDataFiles(String partitionPath);
/**
* Stream all the file groups for a given partition
*
* @param partitionPath
* @return
* Stream all the latest data files, in the file system view
*/
Stream<HoodieFileGroup> getAllFileGroups(String partitionPath);
Stream<HoodieDataFile> getLatestDataFiles();
/**
* Stream all the latest version data files in the given partition with precondition that
* commitTime(file) before maxCommitTime
*/
Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
String maxCommitTime);
/**
* Stream all the latest data files pass
*/
Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn);
/**
* Stream all the data file versions grouped by FileId for a given partition
*/
Stream<HoodieDataFile> getAllDataFiles(String partitionPath);
}
/**
* RealtimeView - methods to access a combination of columnar data files + log files with real
* time data.
*/
interface RealtimeView {
/**
* Stream all the latest file slices in the given partition
*/
Stream<FileSlice> getLatestFileSlices(String partitionPath);
/**
* Stream all the latest file slices in the given partition with precondition that
* commitTime(file) before maxCommitTime
*/
Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath,
String maxCommitTime);
/**
* Stream all the latest file slices, in the given range
*/
Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn);
/**
* Stream all the file slices for a given partition, latest or not.
*/
Stream<FileSlice> getAllFileSlices(String partitionPath);
}
/**
* Stream all the file groups for a given partition
*/
Stream<HoodieFileGroup> getAllFileGroups(String partitionPath);
}

View File

@@ -16,6 +16,9 @@
package com.uber.hoodie.common.table.log;
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK;
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.LogMetadataType.INSTANT_TIME;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieLogFile;
@@ -29,14 +32,6 @@ import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.Arrays;
@@ -48,17 +43,22 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK;
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.LogMetadataType.INSTANT_TIME;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged
* list of records which will be used as a lookup table when merging the base columnar file
* with the redo log file.
*
* Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of
* records which will be used as a lookup table when merging the base columnar file with the redo
* log file.
*/
public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<? extends HoodieRecordPayload>> {
public class HoodieCompactedLogRecordScanner implements
Iterable<HoodieRecord<? extends HoodieRecordPayload>> {
private final static Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class);
// Final list of compacted/merged records to iterate
@@ -80,10 +80,10 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<?
Deque<HoodieLogBlock> lastBlocks = new ArrayDeque<>();
public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths,
Schema readerSchema, String latestInstantTime) {
Schema readerSchema, String latestInstantTime) {
this.readerSchema = readerSchema;
this.latestInstantTime = latestInstantTime;
this.hoodieTableMetaClient = new HoodieTableMetaClient(fs, basePath);
this.hoodieTableMetaClient = new HoodieTableMetaClient(fs, basePath);
// load class from the payload fully qualified class name
this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass();
@@ -91,18 +91,18 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<?
Map<String, HoodieRecord<? extends HoodieRecordPayload>> records = Maps.newHashMap();
// iterate over the paths
Iterator<String> logFilePathsItr = logFilePaths.iterator();
while(logFilePathsItr.hasNext()) {
while (logFilePathsItr.hasNext()) {
HoodieLogFile logFile = new HoodieLogFile(new Path(logFilePathsItr.next()));
log.info("Scanning log file " + logFile.getPath());
totalLogFiles.incrementAndGet();
try {
// Use the HoodieLogFormatReader to iterate through the blocks in the log file
HoodieLogFormatReader reader = new HoodieLogFormatReader(fs, logFile, readerSchema, true);
while(reader.hasNext()) {
while (reader.hasNext()) {
HoodieLogBlock r = reader.next();
String blockInstantTime = r.getLogMetadata().get(INSTANT_TIME);
if(!HoodieTimeline.compareTimestamps(blockInstantTime, this.latestInstantTime,
HoodieTimeline.LESSER_OR_EQUAL)) {
if (!HoodieTimeline.compareTimestamps(blockInstantTime, this.latestInstantTime,
HoodieTimeline.LESSER_OR_EQUAL)) {
//hit a block with instant time greater than should be processed, stop processing further
break;
}
@@ -117,7 +117,7 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<?
case DELETE_BLOCK:
log.info("Reading a delete block from file " + logFile.getPath());
String lastBlockInstantTime = lastBlocks.peek().getLogMetadata().get(INSTANT_TIME);
if(!lastBlockInstantTime.equals(blockInstantTime)) {
if (!lastBlockInstantTime.equals(blockInstantTime)) {
// Block with the keys listed as to be deleted, data and delete blocks written in different batches
// so it is safe to merge
// This is a delete block, so lets merge any records from previous data block
@@ -130,7 +130,8 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<?
log.info("Reading a command block from file " + logFile.getPath());
// This is a command block - take appropriate action based on the command
HoodieCommandBlock commandBlock = (HoodieCommandBlock) r;
String targetInstantForCommandBlock = r.getLogMetadata().get(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME);
String targetInstantForCommandBlock = r.getLogMetadata()
.get(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME);
switch (commandBlock.getType()) { // there can be different types of command blocks
case ROLLBACK_PREVIOUS_BLOCK:
// Rollback the last read log block
@@ -139,15 +140,16 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<?
// the rollback operation itself
HoodieLogBlock lastBlock = lastBlocks.peek();
if (lastBlock != null && lastBlock.getBlockType() != CORRUPT_BLOCK &&
targetInstantForCommandBlock.contentEquals(lastBlock.getLogMetadata().get(INSTANT_TIME))) {
targetInstantForCommandBlock
.contentEquals(lastBlock.getLogMetadata().get(INSTANT_TIME))) {
log.info("Rolling back the last log block read in " + logFile.getPath());
lastBlocks.pop();
} else if(lastBlock != null && lastBlock.getBlockType() == CORRUPT_BLOCK) {
} else if (lastBlock != null && lastBlock.getBlockType() == CORRUPT_BLOCK) {
// handle corrupt blocks separately since they may not have metadata
log.info("Rolling back the last corrupted log block read in " + logFile.getPath());
log.info(
"Rolling back the last corrupted log block read in " + logFile.getPath());
lastBlocks.pop();
}
else {
} else {
log.warn("Invalid or extra rollback command block in " + logFile.getPath());
}
break;
@@ -165,7 +167,7 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<?
throw new HoodieIOException("IOException when reading log file " + logFile);
}
// merge the last read block when all the blocks are done reading
if(!lastBlocks.isEmpty()) {
if (!lastBlocks.isEmpty()) {
log.info("Merging the final data blocks in " + logFile.getPath());
merge(records, lastBlocks);
}
@@ -175,32 +177,33 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<?
}
/**
* Iterate over the GenericRecord in the block, read the hoodie key and partition path
* and merge with the application specific payload if the same key was found before
* Sufficient to just merge the log records since the base data is merged on previous compaction
*
* @param dataBlock
* Iterate over the GenericRecord in the block, read the hoodie key and partition path and merge
* with the application specific payload if the same key was found before Sufficient to just merge
* the log records since the base data is merged on previous compaction
*/
private Map<String, HoodieRecord<? extends HoodieRecordPayload>> loadRecordsFromBlock(HoodieAvroDataBlock dataBlock) {
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordsFromLastBlock = Maps.newHashMap();
private Map<String, HoodieRecord<? extends HoodieRecordPayload>> loadRecordsFromBlock(
HoodieAvroDataBlock dataBlock) {
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordsFromLastBlock = Maps
.newHashMap();
List<IndexedRecord> recs = dataBlock.getRecords();
totalLogRecords.addAndGet(recs.size());
recs.forEach(rec -> {
String key = ((GenericRecord) rec).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)
.toString();
.toString();
String partitionPath =
((GenericRecord) rec).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)
.toString();
((GenericRecord) rec).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)
.toString();
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = new HoodieRecord<>(
new HoodieKey(key, partitionPath),
ReflectionUtils.loadPayload(this.payloadClassFQN, new Object[]{Optional.of(rec)}, Optional.class));
new HoodieKey(key, partitionPath),
ReflectionUtils
.loadPayload(this.payloadClassFQN, new Object[]{Optional.of(rec)}, Optional.class));
if (recordsFromLastBlock.containsKey(key)) {
// Merge and store the merged record
HoodieRecordPayload combinedValue = recordsFromLastBlock.get(key).getData()
.preCombine(hoodieRecord.getData());
.preCombine(hoodieRecord.getData());
recordsFromLastBlock
.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()),
combinedValue));
.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()),
combinedValue));
} else {
// Put the record as is
recordsFromLastBlock.put(key, hoodieRecord);
@@ -211,12 +214,9 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<?
/**
* Merge the last seen log blocks with the accumulated records
*
* @param records
* @param lastBlocks
*/
private void merge(Map<String, HoodieRecord<? extends HoodieRecordPayload>> records,
Deque<HoodieLogBlock> lastBlocks) {
Deque<HoodieLogBlock> lastBlocks) {
while (!lastBlocks.isEmpty()) {
HoodieLogBlock lastBlock = lastBlocks.pop();
switch (lastBlock.getBlockType()) {
@@ -234,19 +234,16 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<?
/**
* Merge the records read from a single data block with the accumulated records
*
* @param records
* @param recordsFromLastBlock
*/
private void merge(Map<String, HoodieRecord<? extends HoodieRecordPayload>> records,
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordsFromLastBlock) {
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordsFromLastBlock) {
recordsFromLastBlock.forEach((key, hoodieRecord) -> {
if (records.containsKey(key)) {
// Merge and store the merged record
HoodieRecordPayload combinedValue = records.get(key).getData()
.preCombine(hoodieRecord.getData());
.preCombine(hoodieRecord.getData());
records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()),
combinedValue));
combinedValue));
} else {
// Put the record as is
records.put(key, hoodieRecord);

View File

@@ -29,31 +29,36 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* File Format for Hoodie Log Files.
* The File Format consists of blocks each seperated with a MAGIC sync marker.
* A Block can either be a Data block, Command block or Delete Block.
* Data Block - Contains log records serialized as Avro Binary Format
* Command Block - Specific commands like RoLLBACK_PREVIOUS-BLOCK - Tombstone for the previously written block
* Delete Block - List of keys to delete - tombstone for keys
* File Format for Hoodie Log Files. The File Format consists of blocks each seperated with a MAGIC
* sync marker. A Block can either be a Data block, Command block or Delete Block. Data Block -
* Contains log records serialized as Avro Binary Format Command Block - Specific commands like
* RoLLBACK_PREVIOUS-BLOCK - Tombstone for the previously written block Delete Block - List of keys
* to delete - tombstone for keys
*/
public interface HoodieLogFormat {
/**
* Magic 4 bytes we put at the start of every block in the log file. Sync marker.
* We could make this file specific (generate a random 4 byte magic and stick it in the file header), but this I think is suffice for now - PR
* Magic 4 bytes we put at the start of every block in the log file. Sync marker. We could make
* this file specific (generate a random 4 byte magic and stick it in the file header), but this I
* think is suffice for now - PR
*/
byte [] MAGIC = new byte [] {'H', 'U', 'D', 'I'};
byte[] MAGIC = new byte[]{'H', 'U', 'D', 'I'};
/**
* Writer interface to allow appending block to this file format
*/
interface Writer extends Closeable {
/** @return the path to this {@link HoodieLogFormat} */
/**
* @return the path to this {@link HoodieLogFormat}
*/
HoodieLogFile getLogFile();
/**
* Append Block returns a new Writer if the log is rolled
*/
Writer appendBlock(HoodieLogBlock block) throws IOException, InterruptedException;
long getCurrentSize() throws IOException;
}
@@ -61,7 +66,10 @@ public interface HoodieLogFormat {
* Reader interface which is an Iterator of HoodieLogBlock
*/
interface Reader extends Closeable, Iterator<HoodieLogBlock> {
/** @return the path to this {@link HoodieLogFormat} */
/**
* @return the path to this {@link HoodieLogFormat}
*/
HoodieLogFile getLogFile();
}
@@ -70,6 +78,7 @@ public interface HoodieLogFormat {
* Builder class to construct the default log format writer
*/
class WriterBuilder {
private final static Logger log = LogManager.getLogger(WriterBuilder.class);
// Default max log file size 512 MB
public static final long DEFAULT_SIZE_THRESHOLD = 512 * 1024 * 1024L;
@@ -187,7 +196,8 @@ public interface HoodieLogFormat {
return new WriterBuilder();
}
static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, boolean readMetadata)
static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema,
boolean readMetadata)
throws IOException {
return new HoodieLogFormatReader(fs, logFile, readerSchema, readMetadata);
}

View File

@@ -17,7 +17,6 @@
package com.uber.hoodie.common.table.log;
import com.google.common.base.Preconditions;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
@@ -38,11 +37,12 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Scans a log file and provides block level iterator on the log file
* Loads the entire block contents in memory
* Can emit either a DataBlock, CommandBlock, DeleteBlock or CorruptBlock (if one is found)
* Scans a log file and provides block level iterator on the log file Loads the entire block
* contents in memory Can emit either a DataBlock, CommandBlock, DeleteBlock or CorruptBlock (if one
* is found)
*/
public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
private static final int DEFAULT_BUFFER_SIZE = 4096;
private final static Logger log = LogManager.getLogger(HoodieLogFormatReader.class);
@@ -53,14 +53,16 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
private HoodieLogBlock nextBlock = null;
private boolean readMetadata = true;
HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, boolean readMetadata) throws IOException {
HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
boolean readMetadata) throws IOException {
this.inputStream = fs.open(logFile.getPath(), bufferSize);
this.logFile = logFile;
this.readerSchema = readerSchema;
this.readMetadata = readMetadata;
}
HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, boolean readMetadata) throws IOException {
HoodieLogFormatReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema,
boolean readMetadata) throws IOException {
this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readMetadata);
}
@@ -83,7 +85,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
// Skip blocksize in the stream and we should either find a sync marker (start of the next block) or EOF
// If we did not find either of it, then this block is a corrupted block.
boolean isCorrupted = isBlockCorrupt(blocksize);
if(isCorrupted) {
if (isCorrupted) {
return createCorruptBlock();
}
@@ -140,7 +142,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
}
private long scanForNextAvailableBlockOffset() throws IOException {
while(true) {
while (true) {
long currentPos = inputStream.getPos();
try {
boolean isEOF = readMagic();
@@ -191,7 +193,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
@Override
public HoodieLogBlock next() {
if(nextBlock == null) {
if (nextBlock == null) {
// may be hasNext is not called
hasNext();
}

View File

@@ -16,8 +16,6 @@
package com.uber.hoodie.common.table.log;
import com.google.common.base.Preconditions;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.HoodieLogFormat.WriterBuilder;
@@ -35,8 +33,8 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* HoodieLogFormatWriter can be used to append blocks to a log file
* Use HoodieLogFormat.WriterBuilder to construct
* HoodieLogFormatWriter can be used to append blocks to a log file Use
* HoodieLogFormat.WriterBuilder to construct
*/
public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
@@ -58,7 +56,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
* @param sizeThreshold
*/
HoodieLogFormatWriter(FileSystem fs, HoodieLogFile logFile, Integer bufferSize,
Short replication, Long sizeThreshold)
Short replication, Long sizeThreshold)
throws IOException, InterruptedException {
this.fs = fs;
this.logFile = logFile;
@@ -157,8 +155,9 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
}
public long getCurrentSize() throws IOException {
if(output == null) {
throw new IllegalStateException("Cannot get current size as the underlying stream has been closed already");
if (output == null) {
throw new IllegalStateException(
"Cannot get current size as the underlying stream has been closed already");
}
return output.getPos();
}

View File

@@ -18,6 +18,14 @@ package com.uber.hoodie.common.table.log.block;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
@@ -27,30 +35,18 @@ import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.Encoder;
import org.apache.avro.io.EncoderFactory;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
/**
* DataBlock contains a list of records serialized using Avro.
* The Datablock contains
* 1. Compressed Writer Schema length
* 2. Compressed Writer Schema content
* 3. Total number of records in the block
* 4. Size of a record
* 5. Actual avro serialized content of the record
* DataBlock contains a list of records serialized using Avro. The Datablock contains 1. Compressed
* Writer Schema length 2. Compressed Writer Schema content 3. Total number of records in the block
* 4. Size of a record 5. Actual avro serialized content of the record
*/
public class HoodieAvroDataBlock extends HoodieLogBlock {
private List<IndexedRecord> records;
private Schema schema;
public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema, Map<LogMetadataType, String> metadata) {
public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema,
Map<LogMetadataType, String> metadata) {
super(metadata);
this.records = records;
this.schema = schema;
@@ -76,7 +72,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
DataOutputStream output = new DataOutputStream(baos);
// 1. Write out metadata
if(super.getLogMetadata() != null) {
if (super.getLogMetadata() != null) {
output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
}
@@ -117,12 +113,13 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
return HoodieLogBlockType.AVRO_DATA_BLOCK;
}
public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema, boolean readMetadata) throws IOException {
public static HoodieLogBlock fromBytes(byte[] content, Schema readerSchema, boolean readMetadata)
throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
Map<LogMetadataType, String> metadata = null;
// 1. Read the metadata written out, if applicable
if(readMetadata) {
if (readMetadata) {
metadata = HoodieLogBlock.getLogMetadata(dis);
}
// 1. Read the schema written out
@@ -131,7 +128,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
dis.readFully(compressedSchema, 0, schemaLength);
Schema writerSchema = new Schema.Parser().parse(HoodieAvroUtils.decompress(compressedSchema));
if(readerSchema == null) {
if (readerSchema == null) {
readerSchema = writerSchema;
}
@@ -141,7 +138,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
List<IndexedRecord> records = new ArrayList<>(totalRecords);
// 3. Read the content
for(int i=0;i<totalRecords;i++) {
for (int i = 0; i < totalRecords; i++) {
// TODO - avoid bytes copy
int recordLength = dis.readInt();
byte[] recordData = new byte[recordLength];

View File

@@ -32,7 +32,8 @@ public class HoodieCommandBlock extends HoodieLogBlock {
public enum HoodieCommandBlockTypeEnum {ROLLBACK_PREVIOUS_BLOCK}
public HoodieCommandBlock(HoodieCommandBlockTypeEnum type, Map<LogMetadataType, String> metadata) {
public HoodieCommandBlock(HoodieCommandBlockTypeEnum type,
Map<LogMetadataType, String> metadata) {
super(metadata);
this.type = type;
}
@@ -46,7 +47,7 @@ public class HoodieCommandBlock extends HoodieLogBlock {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
if(super.getLogMetadata() != null) {
if (super.getLogMetadata() != null) {
output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
}
output.writeInt(type.ordinal());
@@ -66,7 +67,7 @@ public class HoodieCommandBlock extends HoodieLogBlock {
public static HoodieLogBlock fromBytes(byte[] content, boolean readMetadata) throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
Map<LogMetadataType, String> metadata = null;
if(readMetadata) {
if (readMetadata) {
metadata = HoodieLogBlock.getLogMetadata(dis);
}
int ordinal = dis.readInt();

View File

@@ -44,7 +44,7 @@ public class HoodieCorruptBlock extends HoodieLogBlock {
public byte[] getBytes() throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
if(super.getLogMetadata() != null) {
if (super.getLogMetadata() != null) {
output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
}
output.write(corruptedBytes);
@@ -60,20 +60,21 @@ public class HoodieCorruptBlock extends HoodieLogBlock {
return corruptedBytes;
}
public static HoodieLogBlock fromBytes(byte[] content, int blockSize, boolean readMetadata) throws IOException {
public static HoodieLogBlock fromBytes(byte[] content, int blockSize, boolean readMetadata)
throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
Map<LogMetadataType, String> metadata = null;
int bytesRemaining = blockSize;
if(readMetadata) {
if (readMetadata) {
try { //attempt to read metadata
metadata = HoodieLogBlock.getLogMetadata(dis);
bytesRemaining = blockSize - HoodieLogBlock.getLogMetadataBytes(metadata).length;
} catch(IOException e) {
} catch (IOException e) {
// unable to read metadata, possibly corrupted
metadata = null;
}
}
byte [] corruptedBytes = new byte[bytesRemaining];
byte[] corruptedBytes = new byte[bytesRemaining];
dis.readFully(corruptedBytes);
return new HoodieCorruptBlock(corruptedBytes, metadata);
}

View File

@@ -16,8 +16,6 @@
package com.uber.hoodie.common.table.log.block;
import org.apache.commons.lang3.StringUtils;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
@@ -25,6 +23,7 @@ import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.Map;
import org.apache.commons.lang3.StringUtils;
/**
* Delete block contains a list of keys to be deleted from scanning the blocks so far
@@ -46,10 +45,10 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
public byte[] getBytes() throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
if(super.getLogMetadata() != null) {
if (super.getLogMetadata() != null) {
output.write(HoodieLogBlock.getLogMetadataBytes(super.getLogMetadata()));
}
byte [] bytesToWrite = StringUtils.join(keysToDelete, ',').getBytes(Charset.forName("utf-8"));
byte[] bytesToWrite = StringUtils.join(keysToDelete, ',').getBytes(Charset.forName("utf-8"));
output.writeInt(bytesToWrite.length);
output.write(bytesToWrite);
return baos.toByteArray();
@@ -67,11 +66,11 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
public static HoodieLogBlock fromBytes(byte[] content, boolean readMetadata) throws IOException {
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(content));
Map<LogMetadataType, String> metadata = null;
if(readMetadata) {
if (readMetadata) {
metadata = HoodieLogBlock.getLogMetadata(dis);
}
int dataLength = dis.readInt();
byte [] data = new byte[dataLength];
byte[] data = new byte[dataLength];
dis.readFully(data);
return new HoodieDeleteBlock(new String(data).split(","), metadata);
}

View File

@@ -18,7 +18,6 @@ package com.uber.hoodie.common.table.log.block;
import com.google.common.collect.Maps;
import com.uber.hoodie.exception.HoodieException;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
@@ -34,6 +33,7 @@ public abstract class HoodieLogBlock {
public byte[] getBytes() throws IOException {
throw new HoodieException("No implementation was provided");
}
public HoodieLogBlockType getBlockType() {
throw new HoodieException("No implementation was provided");
}
@@ -42,8 +42,8 @@ public abstract class HoodieLogBlock {
private Map<LogMetadataType, String> logMetadata;
/**
* Type of the log block
* WARNING: This enum is serialized as the ordinal. Only add new enums at the end.
* Type of the log block WARNING: This enum is serialized as the ordinal. Only add new enums at
* the end.
*/
public enum HoodieLogBlockType {
COMMAND_BLOCK,
@@ -53,8 +53,8 @@ public abstract class HoodieLogBlock {
}
/**
* Metadata abstraction for a HoodieLogBlock
* WARNING : This enum is serialized as the ordinal. Only add new enums at the end.
* Metadata abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal.
* Only add new enums at the end.
*/
public enum LogMetadataType {
INSTANT_TIME,
@@ -70,21 +70,17 @@ public abstract class HoodieLogBlock {
}
/**
* Convert log metadata to bytes
* 1. Write size of metadata
* 2. Write enum ordinal
* 3. Write actual bytes
* @param metadata
* @return
* @throws IOException
* Convert log metadata to bytes 1. Write size of metadata 2. Write enum ordinal 3. Write actual
* bytes
*/
public static byte [] getLogMetadataBytes(Map<LogMetadataType, String> metadata) throws IOException {
public static byte[] getLogMetadataBytes(Map<LogMetadataType, String> metadata)
throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
output.writeInt(metadata.size());
for(Map.Entry<LogMetadataType, String> entry : metadata.entrySet()) {
for (Map.Entry<LogMetadataType, String> entry : metadata.entrySet()) {
output.writeInt(entry.getKey().ordinal());
byte [] bytes = entry.getValue().getBytes();
byte[] bytes = entry.getValue().getBytes();
output.writeInt(bytes.length);
output.write(bytes);
}
@@ -92,13 +88,10 @@ public abstract class HoodieLogBlock {
}
/**
* Convert bytes to LogMetadata, follow the same order as
* {@link HoodieLogBlock#getLogMetadataBytes}
* @param dis
* @return
* @throws IOException
* Convert bytes to LogMetadata, follow the same order as {@link HoodieLogBlock#getLogMetadataBytes}
*/
public static Map<LogMetadataType, String> getLogMetadata(DataInputStream dis) throws IOException {
public static Map<LogMetadataType, String> getLogMetadata(DataInputStream dis)
throws IOException {
Map<LogMetadataType, String> metadata = Maps.newHashMap();
// 1. Read the metadata written out
@@ -113,7 +106,7 @@ public abstract class HoodieLogBlock {
metadataCount--;
}
return metadata;
} catch(EOFException eof) {
} catch (EOFException eof) {
throw new IOException("Could not read metadata fields ", eof);
}
}

View File

@@ -22,7 +22,18 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.io.Serializable;
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Comparator;
import java.util.Date;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -31,297 +42,273 @@ import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.io.Serializable;
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Comparator;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Represents the Active Timeline for the HoodieDataset. Instants for the last 12 hours (configurable)
* is in the ActiveTimeline and the rest are Archived. ActiveTimeline is a special timeline
* that allows for creation of instants on the timeline.
* <p></p>
* The timeline is not automatically reloaded on any mutation operation, clients have to manually call reload()
* so that they can chain multiple mutations to the timeline and then call reload() once.
* <p></p>
* This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized.
* Represents the Active Timeline for the HoodieDataset. Instants for the last 12 hours
* (configurable) is in the ActiveTimeline and the rest are Archived. ActiveTimeline is a special
* timeline that allows for creation of instants on the timeline. <p></p> The timeline is not
* automatically reloaded on any mutation operation, clients have to manually call reload() so that
* they can chain multiple mutations to the timeline and then call reload() once. <p></p> This class
* can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized.
*/
public class HoodieActiveTimeline extends HoodieDefaultTimeline {
public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
private String metaPath;
private transient FileSystem fs;
private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
private String metaPath;
private transient FileSystem fs;
/**
* Returns next commit time in the {@link #COMMIT_FORMATTER} format.
* @return
*/
public static String createNewCommitTime() {
return HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
/**
* Returns next commit time in the {@link #COMMIT_FORMATTER} format.
*/
public static String createNewCommitTime() {
return HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
}
protected HoodieActiveTimeline(FileSystem fs, String metaPath, String[] includedExtensions) {
// Filter all the filter in the metapath and include only the extensions passed and
// convert them into HoodieInstant
try {
this.instants =
Arrays.stream(HoodieTableMetaClient.scanFiles(fs, new Path(metaPath), path -> {
// Include only the meta files with extensions that needs to be included
String extension = FSUtils.getFileExtension(path.getName());
return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension));
})).sorted(Comparator.comparing(
// Sort the meta-data by the instant time (first part of the file name)
fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName())))
// create HoodieInstantMarkers from FileStatus, which extracts properties
.map(HoodieInstant::new).collect(Collectors.toList());
log.info("Loaded instants " + instants);
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.fs = fs;
this.metaPath = metaPath;
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
}
protected HoodieActiveTimeline(FileSystem fs, String metaPath, String[] includedExtensions) {
// Filter all the filter in the metapath and include only the extensions passed and
// convert them into HoodieInstant
try {
this.instants =
Arrays.stream(HoodieTableMetaClient.scanFiles(fs, new Path(metaPath), path -> {
// Include only the meta files with extensions that needs to be included
String extension = FSUtils.getFileExtension(path.getName());
return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension));
})).sorted(Comparator.comparing(
// Sort the meta-data by the instant time (first part of the file name)
fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName())))
// create HoodieInstantMarkers from FileStatus, which extracts properties
.map(HoodieInstant::new).collect(Collectors.toList());
log.info("Loaded instants " + instants);
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.fs = fs;
this.metaPath = metaPath;
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
public HoodieActiveTimeline(FileSystem fs, String metaPath) {
this(fs, metaPath,
new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
INFLIGHT_DELTA_COMMIT_EXTENSION, COMPACTION_EXTENSION,
INFLIGHT_COMPACTION_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION,
CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION});
}
/**
* For serialization and de-serialization only.
*
* @deprecated
*/
public HoodieActiveTimeline() {
}
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
/**
* Get all instants (commits, delta commits, compactions) that produce new data, in the active
* timeline *
*/
public HoodieTimeline getCommitsAndCompactionsTimeline() {
return getTimelineOfActions(
Sets.newHashSet(COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION));
}
/**
* Get all instants (commits, delta commits, compactions, clean, savepoint, rollback) that result
* in actions, in the active timeline *
*/
public HoodieTimeline getAllCommitsTimeline() {
return getTimelineOfActions(
Sets.newHashSet(COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION,
SAVEPOINT_ACTION, ROLLBACK_ACTION));
}
/**
* Get only pure commits (inflight and completed) in the active timeline
*/
public HoodieTimeline getCommitTimeline() {
return getTimelineOfActions(Sets.newHashSet(COMMIT_ACTION));
}
/**
* Get only the delta commits (inflight and completed) in the active timeline
*/
public HoodieTimeline getDeltaCommitTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(DELTA_COMMIT_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the commits (inflight and completed) in the compaction timeline
*/
public HoodieTimeline getCompactionTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(COMPACTION_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get a timeline of a specific set of actions. useful to create a merged timeline of multiple
* actions
*
* @param actions actions allowed in the timeline
*/
public HoodieTimeline getTimelineOfActions(Set<String> actions) {
return new HoodieDefaultTimeline(instants.stream().filter(s -> actions.contains(s.getAction())),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the cleaner action (inflight and completed) in the active timeline
*/
public HoodieTimeline getCleanerTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(CLEAN_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the rollback action (inflight and completed) in the active timeline
*/
public HoodieTimeline getRollbackTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(ROLLBACK_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the save point action (inflight and completed) in the active timeline
*/
public HoodieTimeline getSavePointTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(SAVEPOINT_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
protected Stream<HoodieInstant> filterInstantsByAction(String action) {
return instants.stream().filter(s -> s.getAction().equals(action));
}
public void createInflight(HoodieInstant instant) {
log.info("Creating a new in-flight instant " + instant);
// Create the in-flight file
createFileInMetaPath(instant.getFileName(), Optional.empty());
}
public void saveAsComplete(HoodieInstant instant, Optional<byte[]> data) {
log.info("Marking instant complete " + instant);
Preconditions.checkArgument(instant.isInflight(),
"Could not mark an already completed instant as complete again " + instant);
moveInflightToComplete(instant, HoodieTimeline.getCompletedInstant(instant), data);
log.info("Completed " + instant);
}
public void revertToInflight(HoodieInstant instant) {
log.info("Reverting instant to inflight " + instant);
moveCompleteToInflight(instant, HoodieTimeline.getInflightInstant(instant));
log.info("Reverted " + instant + " to inflight");
}
public void deleteInflight(HoodieInstant instant) {
log.info("Deleting in-flight " + instant);
Path inFlightCommitFilePath = new Path(metaPath, instant.getFileName());
try {
boolean result = fs.delete(inFlightCommitFilePath, false);
if (result) {
log.info("Removed in-flight " + instant);
} else {
throw new HoodieIOException("Could not delete in-flight instant " + instant);
}
} catch (IOException e) {
throw new HoodieIOException(
"Could not remove inflight commit " + inFlightCommitFilePath, e);
}
}
public HoodieActiveTimeline(FileSystem fs, String metaPath) {
this(fs, metaPath,
new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
INFLIGHT_DELTA_COMMIT_EXTENSION, COMPACTION_EXTENSION,
INFLIGHT_COMPACTION_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION,
CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION});
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
Path detailPath = new Path(metaPath, instant.getFileName());
return readDataFromPath(detailPath);
}
protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant completed,
Optional<byte[]> data) {
Path commitFilePath = new Path(metaPath, completed.getFileName());
try {
// open a new file and write the commit metadata in
Path inflightCommitFile = new Path(metaPath, inflight.getFileName());
createFileInMetaPath(inflight.getFileName(), data);
boolean success = fs.rename(inflightCommitFile, commitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + inflightCommitFile + " to " + commitFilePath);
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete " + inflight, e);
}
}
/**
* For serialization and de-serialization only.
* @deprecated
*/
public HoodieActiveTimeline() {
}
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
/**
* Get all instants (commits, delta commits, compactions) that produce new data, in the active timeline
**
* @return
*/
public HoodieTimeline getCommitsAndCompactionsTimeline() {
return getTimelineOfActions(Sets.newHashSet(COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION));
}
/**
* Get all instants (commits, delta commits, compactions, clean, savepoint, rollback) that result in actions, in the active timeline
**
* @return
*/
public HoodieTimeline getAllCommitsTimeline() {
return getTimelineOfActions(Sets.newHashSet(COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION));
}
/**
* Get only pure commits (inflight and completed) in the active timeline
*
* @return
*/
public HoodieTimeline getCommitTimeline() {
return getTimelineOfActions(Sets.newHashSet(COMMIT_ACTION));
}
/**
* Get only the delta commits (inflight and completed) in the active timeline
*
* @return
*/
public HoodieTimeline getDeltaCommitTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(DELTA_COMMIT_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the commits (inflight and completed) in the compaction timeline
*
* @return
*/
public HoodieTimeline getCompactionTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(COMPACTION_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get a timeline of a specific set of actions. useful to create a merged timeline of multiple actions
*
* @param actions actions allowed in the timeline
* @return
*/
public HoodieTimeline getTimelineOfActions(Set<String> actions) {
return new HoodieDefaultTimeline(instants.stream().filter(s -> actions.contains(s.getAction())),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the cleaner action (inflight and completed) in the active timeline
*
* @return
*/
public HoodieTimeline getCleanerTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(CLEAN_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the rollback action (inflight and completed) in the active timeline
*
* @return
*/
public HoodieTimeline getRollbackTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(ROLLBACK_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the save point action (inflight and completed) in the active timeline
*
* @return
*/
public HoodieTimeline getSavePointTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(SAVEPOINT_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
protected Stream<HoodieInstant> filterInstantsByAction(String action) {
return instants.stream().filter(s -> s.getAction().equals(action));
}
public void createInflight(HoodieInstant instant) {
log.info("Creating a new in-flight instant " + instant);
// Create the in-flight file
createFileInMetaPath(instant.getFileName(), Optional.empty());
}
public void saveAsComplete(HoodieInstant instant, Optional<byte[]> data) {
log.info("Marking instant complete " + instant);
Preconditions.checkArgument(instant.isInflight(),
"Could not mark an already completed instant as complete again " + instant);
moveInflightToComplete(instant, HoodieTimeline.getCompletedInstant(instant), data);
log.info("Completed " + instant);
}
public void revertToInflight(HoodieInstant instant) {
log.info("Reverting instant to inflight " + instant);
moveCompleteToInflight(instant, HoodieTimeline.getInflightInstant(instant));
log.info("Reverted " + instant + " to inflight");
}
public void deleteInflight(HoodieInstant instant) {
log.info("Deleting in-flight " + instant);
Path inFlightCommitFilePath = new Path(metaPath, instant.getFileName());
try {
boolean result = fs.delete(inFlightCommitFilePath, false);
if (result) {
log.info("Removed in-flight " + instant);
} else {
throw new HoodieIOException("Could not delete in-flight instant " + instant);
}
} catch (IOException e) {
throw new HoodieIOException(
"Could not remove inflight commit " + inFlightCommitFilePath, e);
}
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
Path detailPath = new Path(metaPath, instant.getFileName());
return readDataFromPath(detailPath);
}
protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant completed,
Optional<byte[]> data) {
protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
Path inFlightCommitFilePath = new Path(metaPath, inflight.getFileName());
try {
if (!fs.exists(inFlightCommitFilePath)) {
Path commitFilePath = new Path(metaPath, completed.getFileName());
try {
// open a new file and write the commit metadata in
Path inflightCommitFile = new Path(metaPath, inflight.getFileName());
createFileInMetaPath(inflight.getFileName(), data);
boolean success = fs.rename(inflightCommitFile, commitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + inflightCommitFile + " to " + commitFilePath);
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete " + inflight, e);
boolean success = fs.rename(commitFilePath, inFlightCommitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
}
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete revert " + completed, e);
}
}
protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
Path inFlightCommitFilePath = new Path(metaPath, inflight.getFileName());
try {
if (!fs.exists(inFlightCommitFilePath)) {
Path commitFilePath = new Path(metaPath, completed.getFileName());
boolean success = fs.rename(commitFilePath, inFlightCommitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
}
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete revert " + completed, e);
public void saveToInflight(HoodieInstant instant, Optional<byte[]> content) {
createFileInMetaPath(instant.getFileName(), content);
}
protected void createFileInMetaPath(String filename, Optional<byte[]> content) {
Path fullPath = new Path(metaPath, filename);
try {
if (!content.isPresent()) {
if (fs.createNewFile(fullPath)) {
log.info("Created a new file in meta path: " + fullPath);
return;
}
} else {
FSDataOutputStream fsout = fs.create(fullPath, true);
fsout.write(content.get());
fsout.close();
return;
}
throw new HoodieIOException("Failed to create file " + fullPath);
} catch (IOException e) {
throw new HoodieIOException("Failed to create file " + fullPath, e);
}
}
public void saveToInflight(HoodieInstant instant, Optional<byte[]> content) {
createFileInMetaPath(instant.getFileName(), content);
protected Optional<byte[]> readDataFromPath(Path detailPath) {
try (FSDataInputStream is = fs.open(detailPath)) {
return Optional.of(IOUtils.toByteArray(is));
} catch (IOException e) {
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
}
}
protected void createFileInMetaPath(String filename, Optional<byte[]> content) {
Path fullPath = new Path(metaPath, filename);
try {
if (!content.isPresent()) {
if (fs.createNewFile(fullPath)) {
log.info("Created a new file in meta path: " + fullPath);
return;
}
} else {
FSDataOutputStream fsout = fs.create(fullPath, true);
fsout.write(content.get());
fsout.close();
return;
}
throw new HoodieIOException("Failed to create file " + fullPath);
} catch (IOException e) {
throw new HoodieIOException("Failed to create file " + fullPath, e);
}
}
protected Optional<byte[]> readDataFromPath(Path detailPath) {
try (FSDataInputStream is = fs.open(detailPath)) {
return Optional.of(IOUtils.toByteArray(is));
} catch (IOException e) {
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
}
}
public HoodieActiveTimeline reload() {
return new HoodieActiveTimeline(fs, metaPath);
}
public HoodieActiveTimeline reload() {
return new HoodieActiveTimeline(fs, metaPath);
}
}

View File

@@ -19,13 +19,6 @@ package com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.io.Serializable;
import java.util.Arrays;
@@ -34,79 +27,85 @@ import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Represents the Archived Timeline for the HoodieDataset. Instants for the last 12 hours (configurable)
* is in the ActiveTimeline and the rest are in ArchivedTimeline.
* <p></p>
* Instants are read from the archive file during initialization and never refreshed. To refresh, clients
* need to call reload()
* <p></p>
* This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized.
* Represents the Archived Timeline for the HoodieDataset. Instants for the last 12 hours
* (configurable) is in the ActiveTimeline and the rest are in ArchivedTimeline. <p></p> Instants
* are read from the archive file during initialization and never refreshed. To refresh, clients
* need to call reload() <p></p> This class can be serialized and de-serialized and on
* de-serialization the FileSystem is re-initialized.
*/
public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits";
private transient FileSystem fs;
private String metaPath;
private Map<String, byte[]> readCommits = new HashMap<>();
private final transient static Logger log = LogManager.getLogger(HoodieArchivedTimeline.class);
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits";
private transient FileSystem fs;
private String metaPath;
private Map<String, byte[]> readCommits = new HashMap<>();
public HoodieArchivedTimeline(FileSystem fs, String metaPath) {
// Read back the commits to make sure
Path archiveLogPath = getArchiveLogPath(metaPath);
try (SequenceFile.Reader reader =
new SequenceFile.Reader(fs.getConf(), SequenceFile.Reader.file(archiveLogPath))) {
Text key = new Text();
Text val = new Text();
while (reader.next(key, val)) {
// TODO - limit the number of commits loaded in memory. this could get very large.
// This is okay because only tooling will load the archived commit timeline today
readCommits.put(key.toString(), Arrays.copyOf(val.getBytes(), val.getLength()));
}
this.instants = readCommits.keySet().stream().map(
s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)).collect(
Collectors.toList());
} catch (IOException e) {
throw new HoodieIOException(
"Could not load archived commit timeline from path " + archiveLogPath, e);
}
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
this.fs = fs;
this.metaPath = metaPath;
private final transient static Logger log = LogManager.getLogger(HoodieArchivedTimeline.class);
public HoodieArchivedTimeline(FileSystem fs, String metaPath) {
// Read back the commits to make sure
Path archiveLogPath = getArchiveLogPath(metaPath);
try (SequenceFile.Reader reader =
new SequenceFile.Reader(fs.getConf(), SequenceFile.Reader.file(archiveLogPath))) {
Text key = new Text();
Text val = new Text();
while (reader.next(key, val)) {
// TODO - limit the number of commits loaded in memory. this could get very large.
// This is okay because only tooling will load the archived commit timeline today
readCommits.put(key.toString(), Arrays.copyOf(val.getBytes(), val.getLength()));
}
this.instants = readCommits.keySet().stream().map(
s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)).collect(
Collectors.toList());
} catch (IOException e) {
throw new HoodieIOException(
"Could not load archived commit timeline from path " + archiveLogPath, e);
}
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
this.fs = fs;
this.metaPath = metaPath;
}
/**
* For serialization and de-serialization only.
* @deprecated
*/
public HoodieArchivedTimeline() {
}
/**
* For serialization and de-serialization only.
*
* @deprecated
*/
public HoodieArchivedTimeline() {
}
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
public static Path getArchiveLogPath(String metaPath) {
return new Path(metaPath, HOODIE_COMMIT_ARCHIVE_LOG_FILE);
}
public static Path getArchiveLogPath(String metaPath) {
return new Path(metaPath, HOODIE_COMMIT_ARCHIVE_LOG_FILE);
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
return Optional.ofNullable(readCommits.get(instant.getTimestamp()));
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
return Optional.ofNullable(readCommits.get(instant.getTimestamp()));
}
public HoodieArchivedTimeline reload() {
return new HoodieArchivedTimeline(fs, metaPath);
}
public HoodieArchivedTimeline reload() {
return new HoodieArchivedTimeline(fs, metaPath);
}
}

View File

@@ -17,135 +17,136 @@
package com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.util.List;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* HoodieDefaultTimeline is a default implementation of the HoodieTimeline.
* It provides methods to inspect a List[HoodieInstant]. Function to get the details of the instant
* is passed in as a lamdba.
* HoodieDefaultTimeline is a default implementation of the HoodieTimeline. It provides methods to
* inspect a List[HoodieInstant]. Function to get the details of the instant is passed in as a
* lamdba.
*
* @see HoodieTimeline
*/
public class HoodieDefaultTimeline implements HoodieTimeline {
private final transient static Logger log = LogManager.getLogger(HoodieDefaultTimeline.class);
protected Function<HoodieInstant, Optional<byte[]>> details;
protected List<HoodieInstant> instants;
private final transient static Logger log = LogManager.getLogger(HoodieDefaultTimeline.class);
public HoodieDefaultTimeline(Stream<HoodieInstant> instants,
Function<HoodieInstant, Optional<byte[]>> details) {
this.instants = instants.collect(Collectors.toList());
this.details = details;
}
protected Function<HoodieInstant, Optional<byte[]>> details;
protected List<HoodieInstant> instants;
/**
* For serailizing and de-serializing
*
* @deprecated
*/
public HoodieDefaultTimeline() {
}
public HoodieDefaultTimeline(Stream<HoodieInstant> instants,
Function<HoodieInstant, Optional<byte[]>> details) {
this.instants = instants.collect(Collectors.toList());
this.details = details;
}
public HoodieTimeline filterInflights() {
return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isInflight),
details);
}
/**
* For serailizing and de-serializing
*
* @deprecated
*/
public HoodieDefaultTimeline() {
}
public HoodieTimeline filterCompletedInstants() {
return new HoodieDefaultTimeline(instants.stream().filter(s -> !s.isInflight()), details);
}
public HoodieTimeline filterInflights() {
return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isInflight),
details);
}
@Override
public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) {
return new HoodieDefaultTimeline(instants.stream().filter(
s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), startTs, GREATER) &&
HoodieTimeline.compareTimestamps(
public HoodieTimeline filterCompletedInstants() {
return new HoodieDefaultTimeline(instants.stream().filter(s -> !s.isInflight()), details);
}
@Override
public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) {
return new HoodieDefaultTimeline(instants.stream().filter(
s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), startTs, GREATER) &&
HoodieTimeline.compareTimestamps(
s.getTimestamp(), endTs, LESSER_OR_EQUAL)), details);
}
}
@Override
public HoodieDefaultTimeline findInstantsAfter(String commitTime, int numCommits) {
return new HoodieDefaultTimeline(
instants.stream().filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), commitTime, GREATER))
.limit(numCommits), details);
}
@Override
public HoodieDefaultTimeline findInstantsAfter(String commitTime, int numCommits) {
return new HoodieDefaultTimeline(
instants.stream()
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), commitTime, GREATER))
.limit(numCommits), details);
}
@Override
public boolean empty() {
return !instants.stream().findFirst().isPresent();
}
@Override
public boolean empty() {
return !instants.stream().findFirst().isPresent();
}
@Override
public int countInstants() {
return new Long(instants.stream().count()).intValue();
}
@Override
public int countInstants() {
return new Long(instants.stream().count()).intValue();
}
@Override
public Optional<HoodieInstant> firstInstant() {
return instants.stream().findFirst();
}
@Override
public Optional<HoodieInstant> firstInstant() {
return instants.stream().findFirst();
}
@Override
public Optional<HoodieInstant> nthInstant(int n) {
if (empty() || n >= countInstants()) {
return Optional.empty();
}
return Optional.of(instants.get(n));
@Override
public Optional<HoodieInstant> nthInstant(int n) {
if (empty() || n >= countInstants()) {
return Optional.empty();
}
return Optional.of(instants.get(n));
}
@Override
public Optional<HoodieInstant> lastInstant() {
return empty() ? Optional.empty() : nthInstant(countInstants() - 1);
}
@Override
public Optional<HoodieInstant> lastInstant() {
return empty() ? Optional.empty() : nthInstant(countInstants() - 1);
}
@Override
public Optional<HoodieInstant> nthFromLastInstant(int n) {
if (countInstants() < n + 1) {
return Optional.empty();
}
return nthInstant(countInstants() - 1 - n);
@Override
public Optional<HoodieInstant> nthFromLastInstant(int n) {
if (countInstants() < n + 1) {
return Optional.empty();
}
return nthInstant(countInstants() - 1 - n);
}
@Override
public boolean containsInstant(HoodieInstant instant) {
return instants.stream().anyMatch(s -> s.equals(instant));
}
@Override
public boolean containsInstant(HoodieInstant instant) {
return instants.stream().anyMatch(s -> s.equals(instant));
}
@Override
public boolean containsOrBeforeTimelineStarts(String instant) {
return instants.stream().anyMatch(s -> s.getTimestamp().equals(instant))
|| isBeforeTimelineStarts(instant);
}
@Override
public boolean containsOrBeforeTimelineStarts(String instant) {
return instants.stream().anyMatch(s -> s.getTimestamp().equals(instant))
|| isBeforeTimelineStarts(instant);
}
@Override
public Stream<HoodieInstant> getInstants() {
return instants.stream();
}
@Override
public Stream<HoodieInstant> getInstants() {
return instants.stream();
}
@Override
public boolean isBeforeTimelineStarts(String instant) {
Optional<HoodieInstant> firstCommit = firstInstant();
return firstCommit.isPresent() &&
HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER);
}
@Override
public boolean isBeforeTimelineStarts(String instant) {
Optional<HoodieInstant> firstCommit = firstInstant();
return firstCommit.isPresent() &&
HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER);
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
return details.apply(instant);
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
return details.apply(instant);
}
@Override
public String toString() {
return this.getClass().getName() + ": " + instants.stream().map(Object::toString)
.collect(Collectors.joining(","));
}
@Override
public String toString() {
return this.getClass().getName() + ": " + instants.stream().map(Object::toString)
.collect(Collectors.joining(","));
}
}

View File

@@ -16,118 +16,117 @@
package com.uber.hoodie.common.table.timeline;
import com.google.common.io.Files;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileStatus;
import java.io.Serializable;
import java.util.Objects;
import org.apache.hadoop.fs.FileStatus;
/**
* A Hoodie Instant represents a action done on a hoodie dataset.
* All actions start with a inflight instant and then create a completed instant after done.
* A Hoodie Instant represents a action done on a hoodie dataset. All actions start with a inflight
* instant and then create a completed instant after done.
*
* @see HoodieTimeline
*/
public class HoodieInstant implements Serializable {
private boolean isInflight = false;
private String action;
private String timestamp;
/**
* Load the instant from the meta FileStatus
* @param fileStatus
*/
public HoodieInstant(FileStatus fileStatus) {
// First read the instant timestamp. [==>20170101193025<==].commit
String fileName = fileStatus.getPath().getName();
String fileExtension = FSUtils.getFileExtension(fileName);
timestamp = fileName.replace(fileExtension, "");
private boolean isInflight = false;
private String action;
private String timestamp;
// Next read the action for this marker
action = fileExtension.replaceFirst(".", "");
if(action.equals("inflight")) {
// This is to support backwards compatibility on how in-flight commit files were written
// General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
action = "commit";
isInflight = true;
} else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
isInflight = true;
action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
}
/**
* Load the instant from the meta FileStatus
*/
public HoodieInstant(FileStatus fileStatus) {
// First read the instant timestamp. [==>20170101193025<==].commit
String fileName = fileStatus.getPath().getName();
String fileExtension = FSUtils.getFileExtension(fileName);
timestamp = fileName.replace(fileExtension, "");
// Next read the action for this marker
action = fileExtension.replaceFirst(".", "");
if (action.equals("inflight")) {
// This is to support backwards compatibility on how in-flight commit files were written
// General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
action = "commit";
isInflight = true;
} else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
isInflight = true;
action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
}
}
public HoodieInstant(boolean isInflight, String action, String timestamp) {
this.isInflight = isInflight;
this.action = action;
this.timestamp = timestamp;
}
public HoodieInstant(boolean isInflight, String action, String timestamp) {
this.isInflight = isInflight;
this.action = action;
this.timestamp = timestamp;
}
public boolean isInflight() {
return isInflight;
}
public boolean isInflight() {
return isInflight;
}
public String getAction() {
return action;
}
public String getAction() {
return action;
}
public String getTimestamp() {
return timestamp;
}
public String getTimestamp() {
return timestamp;
}
/**
* Get the filename for this instant
* @return
*/
public String getFileName() {
if (HoodieTimeline.COMMIT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCommitFileName(timestamp) :
HoodieTimeline.makeCommitFileName(timestamp);
} else if (HoodieTimeline.CLEAN_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCleanerFileName(timestamp) :
HoodieTimeline.makeCleanerFileName(timestamp);
} else if (HoodieTimeline.ROLLBACK_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightRollbackFileName(timestamp) :
HoodieTimeline.makeRollbackFileName(timestamp);
} else if (HoodieTimeline.SAVEPOINT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightSavePointFileName(timestamp) :
HoodieTimeline.makeSavePointFileName(timestamp);
} else if (HoodieTimeline.COMPACTION_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCompactionFileName(timestamp) :
HoodieTimeline.makeCompactionFileName(timestamp);
} else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightDeltaFileName(timestamp) :
HoodieTimeline.makeDeltaFileName(timestamp);
}
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
/**
* Get the filename for this instant
*/
public String getFileName() {
if (HoodieTimeline.COMMIT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCommitFileName(timestamp) :
HoodieTimeline.makeCommitFileName(timestamp);
} else if (HoodieTimeline.CLEAN_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCleanerFileName(timestamp) :
HoodieTimeline.makeCleanerFileName(timestamp);
} else if (HoodieTimeline.ROLLBACK_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightRollbackFileName(timestamp) :
HoodieTimeline.makeRollbackFileName(timestamp);
} else if (HoodieTimeline.SAVEPOINT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightSavePointFileName(timestamp) :
HoodieTimeline.makeSavePointFileName(timestamp);
} else if (HoodieTimeline.COMPACTION_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCompactionFileName(timestamp) :
HoodieTimeline.makeCompactionFileName(timestamp);
} else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightDeltaFileName(timestamp) :
HoodieTimeline.makeDeltaFileName(timestamp);
}
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieInstant that = (HoodieInstant) o;
return isInflight == that.isInflight &&
Objects.equals(action, that.action) &&
Objects.equals(timestamp, that.timestamp);
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieInstant that = (HoodieInstant) o;
return isInflight == that.isInflight &&
Objects.equals(action, that.action) &&
Objects.equals(timestamp, that.timestamp);
}
@Override
public int hashCode() {
return Objects.hash(isInflight, action, timestamp);
}
@Override
public int hashCode() {
return Objects.hash(isInflight, action, timestamp);
}
@Override
public String toString() {
return "[" + ((isInflight) ? "==>" : "") + timestamp + "__" + action + "]";
}
@Override
public String toString() {
return "[" + ((isInflight) ? "==>" : "") + timestamp + "__" + action + "]";
}
}

View File

@@ -19,18 +19,12 @@ package com.uber.hoodie.common.table.view;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieFileGroup;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
@@ -44,6 +38,10 @@ import java.util.Set;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
/**
* Common abstract implementation for multiple TableFileSystemView Implementations. 2 possible
@@ -54,8 +52,9 @@ import java.util.stream.Stream;
* @see TableFileSystemView
* @since 0.3.0
*/
public class HoodieTableFileSystemView implements TableFileSystemView, TableFileSystemView.ReadOptimizedView,
TableFileSystemView.RealtimeView, Serializable {
public class HoodieTableFileSystemView implements TableFileSystemView,
TableFileSystemView.ReadOptimizedView,
TableFileSystemView.RealtimeView, Serializable {
protected HoodieTableMetaClient metaClient;
protected transient FileSystem fs;
@@ -69,12 +68,9 @@ public class HoodieTableFileSystemView implements TableFileSystemView, TableFile
/**
* Create a file system view, as of the given timeline
*
* @param metaClient
* @param visibleActiveTimeline
*/
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline) {
HoodieTimeline visibleActiveTimeline) {
this.metaClient = metaClient;
this.fs = metaClient.getFs();
this.visibleActiveTimeline = visibleActiveTimeline;
@@ -85,14 +81,10 @@ public class HoodieTableFileSystemView implements TableFileSystemView, TableFile
/**
* Create a file system view, as of the given timeline, with the provided file statuses.
*
* @param metaClient
* @param visibleActiveTimeline
* @param fileStatuses
*/
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline,
FileStatus[] fileStatuses) {
HoodieTimeline visibleActiveTimeline,
FileStatus[] fileStatuses) {
this(metaClient, visibleActiveTimeline);
addFilesToView(fileStatuses);
}
@@ -104,44 +96,44 @@ public class HoodieTableFileSystemView implements TableFileSystemView, TableFile
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
private void writeObject(java.io.ObjectOutputStream out)
throws IOException {
throws IOException {
out.defaultWriteObject();
}
/**
* Adds the provided statuses into the file system view, and also caches it inside this object.
*
* @param statuses
* @return
*/
private List<HoodieFileGroup> addFilesToView(FileStatus[] statuses) {
Map<Pair<String, String>, List<HoodieDataFile>> dataFiles = convertFileStatusesToDataFiles(statuses)
.collect(Collectors.groupingBy((dataFile) -> {
String partitionPathStr = FSUtils.getRelativePartitionPath(
new Path(metaClient.getBasePath()),
dataFile.getFileStatus().getPath().getParent());
return Pair.of(partitionPathStr , dataFile.getFileId());
}));
Map<Pair<String, String>, List<HoodieLogFile>> logFiles = convertFileStatusesToLogFiles(statuses)
.collect(Collectors.groupingBy((logFile) -> {
String partitionPathStr = FSUtils.getRelativePartitionPath(
new Path(metaClient.getBasePath()),
logFile.getPath().getParent());
return Pair.of(partitionPathStr , logFile.getFileId());
}));
Map<Pair<String, String>, List<HoodieDataFile>> dataFiles = convertFileStatusesToDataFiles(
statuses)
.collect(Collectors.groupingBy((dataFile) -> {
String partitionPathStr = FSUtils.getRelativePartitionPath(
new Path(metaClient.getBasePath()),
dataFile.getFileStatus().getPath().getParent());
return Pair.of(partitionPathStr, dataFile.getFileId());
}));
Map<Pair<String, String>, List<HoodieLogFile>> logFiles = convertFileStatusesToLogFiles(
statuses)
.collect(Collectors.groupingBy((logFile) -> {
String partitionPathStr = FSUtils.getRelativePartitionPath(
new Path(metaClient.getBasePath()),
logFile.getPath().getParent());
return Pair.of(partitionPathStr, logFile.getFileId());
}));
Set<Pair<String, String>> fileIdSet = new HashSet<>(dataFiles.keySet());
fileIdSet.addAll(logFiles.keySet());
List<HoodieFileGroup> fileGroups = new ArrayList<>();
fileIdSet.forEach(pair -> {
HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), pair.getValue(), visibleActiveTimeline);
HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), pair.getValue(),
visibleActiveTimeline);
if (dataFiles.containsKey(pair)) {
dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile));
}
@@ -165,90 +157,93 @@ public class HoodieTableFileSystemView implements TableFileSystemView, TableFile
private Stream<HoodieDataFile> convertFileStatusesToDataFiles(FileStatus[] statuses) {
Predicate<FileStatus> roFilePredicate = fileStatus ->
fileStatus.getPath().getName().contains(metaClient.getTableConfig().getROFileFormat().getFileExtension());
fileStatus.getPath().getName()
.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension());
return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new);
}
private Stream<HoodieLogFile> convertFileStatusesToLogFiles(FileStatus[] statuses) {
Predicate<FileStatus> rtFilePredicate = fileStatus ->
fileStatus.getPath().getName().contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension());
fileStatus.getPath().getName()
.contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension());
return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new);
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles(final String partitionPath) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestDataFile())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
.map(fileGroup -> fileGroup.getLatestDataFile())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles() {
return fileGroupMap.values().stream()
.map(fileGroup -> fileGroup.getLatestDataFile())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
.map(fileGroup -> fileGroup.getLatestDataFile())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
String maxCommitTime) {
String maxCommitTime) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestDataFileBeforeOrOn(maxCommitTime))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
.map(fileGroup -> fileGroup.getLatestDataFileBeforeOrOn(maxCommitTime))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
return fileGroupMap.values().stream()
.map(fileGroup -> fileGroup.getLatestDataFileInRange(commitsToReturn))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
return fileGroupMap.values().stream()
.map(fileGroup -> fileGroup.getLatestDataFileInRange(commitsToReturn))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getAllDataFiles())
.flatMap(dataFileList -> dataFileList);
.map(fileGroup -> fileGroup.getAllDataFiles())
.flatMap(dataFileList -> dataFileList);
}
@Override
public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestFileSlice())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
.map(fileGroup -> fileGroup.getLatestFileSlice())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath, String maxCommitTime) {
public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath,
String maxCommitTime) {
return getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
.map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
return fileGroupMap.values().stream()
.map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
.map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
}
@Override
public Stream<FileSlice> getAllFileSlices(String partitionPath) {
return getAllFileGroups(partitionPath)
.map(group -> group.getAllFileSlices())
.flatMap(sliceList -> sliceList);
.map(group -> group.getAllFileSlices())
.flatMap(sliceList -> sliceList);
}
/**
* Given a partition path, obtain all filegroups within that. All methods, that work at the partition level
* go through this.
* Given a partition path, obtain all filegroups within that. All methods, that work at the
* partition level go through this.
*/
@Override
public Stream<HoodieFileGroup> getAllFileGroups(String partitionPathStr) {
@@ -266,7 +261,7 @@ public class HoodieTableFileSystemView implements TableFileSystemView, TableFile
return fileGroups.stream();
} catch (IOException e) {
throw new HoodieIOException(
"Failed to list data files in partition " + partitionPathStr, e);
"Failed to list data files in partition " + partitionPathStr, e);
}
}
}

View File

@@ -19,7 +19,6 @@ package com.uber.hoodie.common.util;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
import com.uber.hoodie.avro.model.HoodieCleanPartitionMetadata;
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
@@ -32,7 +31,11 @@ import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.DataFileWriter;
@@ -50,146 +53,140 @@ import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Optional;
public class AvroUtils {
public static List<HoodieRecord<HoodieAvroPayload>> loadFromFiles(FileSystem fs,
List<String> deltaFilePaths, Schema expectedSchema) {
List<HoodieRecord<HoodieAvroPayload>> loadedRecords = Lists.newArrayList();
deltaFilePaths.forEach(s -> {
List<HoodieRecord<HoodieAvroPayload>> records = loadFromFile(fs, s, expectedSchema);
loadedRecords.addAll(records);
});
return loadedRecords;
public static List<HoodieRecord<HoodieAvroPayload>> loadFromFiles(FileSystem fs,
List<String> deltaFilePaths, Schema expectedSchema) {
List<HoodieRecord<HoodieAvroPayload>> loadedRecords = Lists.newArrayList();
deltaFilePaths.forEach(s -> {
List<HoodieRecord<HoodieAvroPayload>> records = loadFromFile(fs, s, expectedSchema);
loadedRecords.addAll(records);
});
return loadedRecords;
}
public static List<HoodieRecord<HoodieAvroPayload>> loadFromFile(FileSystem fs,
String deltaFilePath, Schema expectedSchema) {
List<HoodieRecord<HoodieAvroPayload>> loadedRecords = Lists.newArrayList();
Path path = new Path(deltaFilePath);
try {
SeekableInput input = new FsInput(path, fs.getConf());
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<>();
// Set the expected schema to be the current schema to account for schema evolution
reader.setExpected(expectedSchema);
FileReader<GenericRecord> fileReader = DataFileReader.openReader(input, reader);
for (GenericRecord deltaRecord : fileReader) {
String key = deltaRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String partitionPath =
deltaRecord.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
loadedRecords.add(new HoodieRecord<>(new HoodieKey(key, partitionPath),
new HoodieAvroPayload(Optional.of(deltaRecord))));
}
fileReader.close(); // also closes underlying FsInput
} catch (IOException e) {
throw new HoodieIOException("Could not read avro records from path " + deltaFilePath,
e);
}
return loadedRecords;
}
public static List<HoodieRecord<HoodieAvroPayload>> loadFromFile(FileSystem fs,
String deltaFilePath, Schema expectedSchema) {
List<HoodieRecord<HoodieAvroPayload>> loadedRecords = Lists.newArrayList();
Path path = new Path(deltaFilePath);
try {
SeekableInput input = new FsInput(path, fs.getConf());
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<>();
// Set the expected schema to be the current schema to account for schema evolution
reader.setExpected(expectedSchema);
FileReader<GenericRecord> fileReader = DataFileReader.openReader(input, reader);
for (GenericRecord deltaRecord : fileReader) {
String key = deltaRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String partitionPath =
deltaRecord.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
loadedRecords.add(new HoodieRecord<>(new HoodieKey(key, partitionPath),
new HoodieAvroPayload(Optional.of(deltaRecord))));
}
fileReader.close(); // also closes underlying FsInput
} catch (IOException e) {
throw new HoodieIOException("Could not read avro records from path " + deltaFilePath,
e);
}
return loadedRecords;
public static HoodieCleanMetadata convertCleanMetadata(String startCleanTime,
Optional<Long> durationInMs, List<HoodieCleanStat> cleanStats) {
ImmutableMap.Builder<String, HoodieCleanPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.<String, HoodieCleanPartitionMetadata>builder();
int totalDeleted = 0;
String earliestCommitToRetain = null;
for (HoodieCleanStat stat : cleanStats) {
HoodieCleanPartitionMetadata metadata =
new HoodieCleanPartitionMetadata(stat.getPartitionPath(), stat.getPolicy().name(),
stat.getDeletePathPatterns(), stat.getSuccessDeleteFiles(),
stat.getDeletePathPatterns());
partitionMetadataBuilder.put(stat.getPartitionPath(), metadata);
totalDeleted += stat.getSuccessDeleteFiles().size();
if (earliestCommitToRetain == null) {
// This will be the same for all partitions
earliestCommitToRetain = stat.getEarliestCommitToRetain();
}
}
return new HoodieCleanMetadata(startCleanTime, durationInMs.orElseGet(() -> -1L),
totalDeleted, earliestCommitToRetain, partitionMetadataBuilder.build());
}
public static HoodieCleanMetadata convertCleanMetadata(String startCleanTime,
Optional<Long> durationInMs, List<HoodieCleanStat> cleanStats) {
ImmutableMap.Builder<String, HoodieCleanPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.<String, HoodieCleanPartitionMetadata>builder();
int totalDeleted = 0;
String earliestCommitToRetain = null;
for (HoodieCleanStat stat : cleanStats) {
HoodieCleanPartitionMetadata metadata =
new HoodieCleanPartitionMetadata(stat.getPartitionPath(), stat.getPolicy().name(),
stat.getDeletePathPatterns(), stat.getSuccessDeleteFiles(),
stat.getDeletePathPatterns());
partitionMetadataBuilder.put(stat.getPartitionPath(), metadata);
totalDeleted += stat.getSuccessDeleteFiles().size();
if (earliestCommitToRetain == null) {
// This will be the same for all partitions
earliestCommitToRetain = stat.getEarliestCommitToRetain();
}
}
return new HoodieCleanMetadata(startCleanTime, durationInMs.orElseGet(() -> -1L),
totalDeleted, earliestCommitToRetain, partitionMetadataBuilder.build());
public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime,
Optional<Long> durationInMs, List<String> commits, List<HoodieRollbackStat> stats) {
ImmutableMap.Builder<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.<String, HoodieRollbackPartitionMetadata>builder();
int totalDeleted = 0;
for (HoodieRollbackStat stat : stats) {
HoodieRollbackPartitionMetadata metadata =
new HoodieRollbackPartitionMetadata(stat.getPartitionPath(),
stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles());
partitionMetadataBuilder.put(stat.getPartitionPath(), metadata);
totalDeleted += stat.getSuccessDeleteFiles().size();
}
return new HoodieRollbackMetadata(startRollbackTime, durationInMs.orElseGet(() -> -1L),
totalDeleted, commits, partitionMetadataBuilder.build());
}
public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime,
Optional<Long> durationInMs, List<String> commits, List<HoodieRollbackStat> stats) {
ImmutableMap.Builder<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.<String, HoodieRollbackPartitionMetadata>builder();
int totalDeleted = 0;
for (HoodieRollbackStat stat : stats) {
HoodieRollbackPartitionMetadata metadata =
new HoodieRollbackPartitionMetadata(stat.getPartitionPath(),
stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles());
partitionMetadataBuilder.put(stat.getPartitionPath(), metadata);
totalDeleted += stat.getSuccessDeleteFiles().size();
}
return new HoodieRollbackMetadata(startRollbackTime, durationInMs.orElseGet(() -> -1L),
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 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);
}
public static Optional<byte[]> serializeCleanMetadata(HoodieCleanMetadata metadata)
throws IOException {
return serializeAvroMetadata(metadata, HoodieCleanMetadata.class);
}
public static Optional<byte[]> serializeSavepointMetadata(HoodieSavepointMetadata metadata)
throws IOException {
return serializeAvroMetadata(metadata, HoodieSavepointMetadata.class);
}
public static Optional<byte[]> serializeSavepointMetadata(HoodieSavepointMetadata metadata)
throws IOException {
return serializeAvroMetadata(metadata, HoodieSavepointMetadata.class);
}
public static Optional<byte[]> serializeRollbackMetadata(
HoodieRollbackMetadata rollbackMetadata) throws IOException {
return serializeAvroMetadata(rollbackMetadata, HoodieRollbackMetadata.class);
}
public static Optional<byte[]> serializeRollbackMetadata(
HoodieRollbackMetadata rollbackMetadata) throws IOException {
return serializeAvroMetadata(rollbackMetadata, HoodieRollbackMetadata.class);
}
public static <T extends SpecificRecordBase> Optional<byte[]> serializeAvroMetadata(T metadata,
Class<T> clazz) throws IOException {
DatumWriter<T> datumWriter = new SpecificDatumWriter<>(clazz);
DataFileWriter<T> fileWriter = new DataFileWriter<>(datumWriter);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
fileWriter.create(metadata.getSchema(), baos);
fileWriter.append(metadata);
fileWriter.flush();
return Optional.of(baos.toByteArray());
}
public static <T extends SpecificRecordBase> Optional<byte[]> serializeAvroMetadata(T metadata,
Class<T> clazz) throws IOException {
DatumWriter<T> datumWriter = new SpecificDatumWriter<>(clazz);
DataFileWriter<T> fileWriter = new DataFileWriter<>(datumWriter);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
fileWriter.create(metadata.getSchema(), baos);
fileWriter.append(metadata);
fileWriter.flush();
return Optional.of(baos.toByteArray());
}
public static HoodieCleanMetadata deserializeHoodieCleanMetadata(byte[] bytes)
throws IOException {
return deserializeAvroMetadata(bytes, HoodieCleanMetadata.class);
}
public static HoodieCleanMetadata deserializeHoodieCleanMetadata(byte[] bytes)
throws IOException {
return deserializeAvroMetadata(bytes, HoodieCleanMetadata.class);
}
public static HoodieSavepointMetadata deserializeHoodieSavepointMetadata(byte[] bytes)
throws IOException {
return deserializeAvroMetadata(bytes, HoodieSavepointMetadata.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);
FileReader<T> fileReader =
DataFileReader.openReader(new SeekableByteArrayInput(bytes), reader);
Preconditions
.checkArgument(fileReader.hasNext(), "Could not deserialize metadata of type " + clazz);
return fileReader.next();
}
public static <T extends SpecificRecordBase> T deserializeAvroMetadata(byte[] bytes,
Class<T> clazz) throws IOException {
DatumReader<T> reader = new SpecificDatumReader<>(clazz);
FileReader<T> fileReader =
DataFileReader.openReader(new SeekableByteArrayInput(bytes), reader);
Preconditions
.checkArgument(fileReader.hasNext(), "Could not deserialize metadata of type " + clazz);
return fileReader.next();
}
}

View File

@@ -23,16 +23,6 @@ import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.InvalidHoodiePathException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
@@ -43,319 +33,339 @@ import java.util.Optional;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Utility functions related to accessing the file storage
*/
public class FSUtils {
private static final Logger LOG = LogManager.getLogger(FSUtils.class);
// Log files are of this pattern - .b5068208-e1a4-11e6-bf01-fe55135034f3_20170101134598.log.1
private static final Pattern LOG_FILE_PATTERN = Pattern.compile("\\.(.*)_(.*)\\.(.*)\\.([0-9]*)");
private static final String LOG_FILE_PREFIX = ".";
private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10;
private static final long MIN_CLEAN_TO_KEEP = 10;
private static final long MIN_ROLLBACK_TO_KEEP = 10;
private static FileSystem fs;
private static final Logger LOG = LogManager.getLogger(FSUtils.class);
// Log files are of this pattern - .b5068208-e1a4-11e6-bf01-fe55135034f3_20170101134598.log.1
private static final Pattern LOG_FILE_PATTERN = Pattern.compile("\\.(.*)_(.*)\\.(.*)\\.([0-9]*)");
private static final String LOG_FILE_PREFIX = ".";
private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10;
private static final long MIN_CLEAN_TO_KEEP = 10;
private static final long MIN_ROLLBACK_TO_KEEP = 10;
private static FileSystem fs;
/**
* Only to be used for testing.
*/
@VisibleForTesting
public static void setFs(FileSystem fs) {
FSUtils.fs = fs;
/**
* Only to be used for testing.
*/
@VisibleForTesting
public static void setFs(FileSystem fs) {
FSUtils.fs = fs;
}
public static FileSystem getFs() {
if (fs != null) {
return fs;
}
public static FileSystem getFs() {
if (fs != null) {
return fs;
}
Configuration conf = new Configuration();
conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
FileSystem fs;
try {
fs = FileSystem.get(conf);
} catch (IOException e) {
throw new HoodieIOException("Failed to get instance of " + FileSystem.class.getName(),
e);
}
LOG.info(String.format("Hadoop Configuration: fs.defaultFS: [%s], Config:[%s], FileSystem: [%s]",
conf.getRaw("fs.defaultFS"), conf.toString(), fs.toString()));
return fs;
Configuration conf = new Configuration();
conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
FileSystem fs;
try {
fs = FileSystem.get(conf);
} catch (IOException e) {
throw new HoodieIOException("Failed to get instance of " + FileSystem.class.getName(),
e);
}
LOG.info(
String.format("Hadoop Configuration: fs.defaultFS: [%s], Config:[%s], FileSystem: [%s]",
conf.getRaw("fs.defaultFS"), conf.toString(), fs.toString()));
public static String makeDataFileName(String commitTime, int taskPartitionId, String fileId) {
return String.format("%s_%d_%s.parquet", fileId, taskPartitionId, commitTime);
return fs;
}
public static String makeDataFileName(String commitTime, int taskPartitionId, String fileId) {
return String.format("%s_%d_%s.parquet", fileId, taskPartitionId, commitTime);
}
public static String maskWithoutFileId(String commitTime, int taskPartitionId) {
return String.format("*_%s_%s.parquet", taskPartitionId, commitTime);
}
public static String maskWithoutTaskPartitionId(String commitTime, String fileId) {
return String.format("%s_*_%s.parquet", fileId, commitTime);
}
public static String maskWithOnlyCommitTime(String commitTime) {
return String.format("*_*_%s.parquet", commitTime);
}
public static String getCommitFromCommitFile(String commitFileName) {
return commitFileName.split("\\.")[0];
}
public static String getCommitTime(String fullFileName) {
return fullFileName.split("_")[2].split("\\.")[0];
}
public static long getFileSize(FileSystem fs, Path path) throws IOException {
return fs.getFileStatus(path).getLen();
}
public static String getFileId(String fullFileName) {
return fullFileName.split("_")[0];
}
/**
* Gets all partition paths assuming date partitioning (year, month, day) three levels down.
*/
public static List<String> getAllFoldersThreeLevelsDown(FileSystem fs, String basePath)
throws IOException {
List<String> datePartitions = new ArrayList<>();
FileStatus[] folders = fs.globStatus(new Path(basePath + "/*/*/*"));
for (FileStatus status : folders) {
Path path = status.getPath();
datePartitions.add(String.format("%s/%s/%s", path.getParent().getParent().getName(),
path.getParent().getName(), path.getName()));
}
return datePartitions;
}
public static String maskWithoutFileId(String commitTime, int taskPartitionId) {
return String.format("*_%s_%s.parquet", taskPartitionId, commitTime);
public static String getRelativePartitionPath(Path basePath, Path partitionPath) {
String partitionFullPath = partitionPath.toString();
int partitionStartIndex = partitionFullPath.lastIndexOf(basePath.getName());
return partitionFullPath.substring(partitionStartIndex + basePath.getName().length() + 1);
}
/**
* Obtain all the partition paths, that are present in this table, denoted by presence of {@link
* com.uber.hoodie.common.model.HoodiePartitionMetadata#HOODIE_PARTITION_METAFILE}
*/
public static List<String> getAllFoldersWithPartitionMetaFile(FileSystem fs, String basePathStr)
throws IOException {
List<String> partitions = new ArrayList<>();
Path basePath = new Path(basePathStr);
RemoteIterator<LocatedFileStatus> allFiles = fs.listFiles(new Path(basePathStr), true);
while (allFiles.hasNext()) {
Path filePath = allFiles.next().getPath();
if (filePath.getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)) {
partitions.add(getRelativePartitionPath(basePath, filePath.getParent()));
}
}
return partitions;
}
public static String maskWithoutTaskPartitionId(String commitTime, String fileId) {
return String.format("%s_*_%s.parquet", fileId, commitTime);
public static List<String> getAllPartitionPaths(FileSystem fs, String basePathStr,
boolean assumeDatePartitioning)
throws IOException {
if (assumeDatePartitioning) {
return getAllFoldersThreeLevelsDown(fs, basePathStr);
} else {
return getAllFoldersWithPartitionMetaFile(fs, basePathStr);
}
}
public static String maskWithOnlyCommitTime(String commitTime) {
return String.format("*_*_%s.parquet", commitTime);
public static String getFileExtension(String fullName) {
Preconditions.checkNotNull(fullName);
String fileName = (new File(fullName)).getName();
int dotIndex = fileName.indexOf('.');
return dotIndex == -1 ? "" : fileName.substring(dotIndex);
}
public static String getInstantTime(String name) {
return name.replace(getFileExtension(name), "");
}
/**
* Get the file extension from the log file
*/
public static String getFileExtensionFromLog(Path logPath) {
Matcher matcher = LOG_FILE_PATTERN.matcher(logPath.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(logPath, "LogFile");
}
return matcher.group(3);
}
public static String getCommitFromCommitFile(String commitFileName) {
return commitFileName.split("\\.")[0];
/**
* Get the first part of the file name in the log file. That will be the fileId. Log file do not
* have commitTime in the file name.
*/
public static String getFileIdFromLogPath(Path path) {
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(path, "LogFile");
}
return matcher.group(1);
}
public static String getCommitTime(String fullFileName) {
return fullFileName.split("_")[2].split("\\.")[0];
/**
* Get the first part of the file name in the log file. That will be the fileId. Log file do not
* have commitTime in the file name.
*/
public static String getBaseCommitTimeFromLogPath(Path path) {
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(path, "LogFile");
}
return matcher.group(2);
}
public static long getFileSize(FileSystem fs, Path path) throws IOException {
return fs.getFileStatus(path).getLen();
/**
* Get the last part of the file name in the log file and convert to int.
*/
public static int getFileVersionFromLog(Path logPath) {
Matcher matcher = LOG_FILE_PATTERN.matcher(logPath.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(logPath, "LogFile");
}
return Integer.parseInt(matcher.group(4));
}
public static String getFileId(String fullFileName) {
return fullFileName.split("_")[0];
public static String makeLogFileName(String fileId, String logFileExtension,
String baseCommitTime, int version) {
return LOG_FILE_PREFIX + String
.format("%s_%s%s.%d", fileId, baseCommitTime, logFileExtension, version);
}
public static String maskWithoutLogVersion(String commitTime, String fileId,
String logFileExtension) {
return LOG_FILE_PREFIX + String.format("%s_%s%s*", fileId, commitTime, logFileExtension);
}
/**
* Get the latest log file written from the list of log files passed in
*/
public static Optional<HoodieLogFile> getLatestLogFile(Stream<HoodieLogFile> logFiles) {
return logFiles.sorted(Comparator
.comparing(s -> s.getLogVersion(),
Comparator.reverseOrder())).findFirst();
}
/**
* Get all the log files for the passed in FileId in the partition path
*/
public static Stream<HoodieLogFile> getAllLogFiles(FileSystem fs, Path partitionPath,
final String fileId, final String logFileExtension, final String baseCommitTime)
throws IOException {
return Arrays.stream(fs.listStatus(partitionPath,
path -> path.getName().startsWith("." + fileId) && path.getName()
.contains(logFileExtension)))
.map(HoodieLogFile::new).filter(s -> s.getBaseCommitTime().equals(baseCommitTime));
}
/**
* Get the latest log version for the fileId in the partition path
*/
public static Optional<Integer> getLatestLogVersion(FileSystem fs, Path partitionPath,
final String fileId, final String logFileExtension, final String baseCommitTime)
throws IOException {
Optional<HoodieLogFile> latestLogFile =
getLatestLogFile(
getAllLogFiles(fs, partitionPath, fileId, logFileExtension, baseCommitTime));
if (latestLogFile.isPresent()) {
return Optional.of(latestLogFile.get().getLogVersion());
}
return Optional.empty();
}
public static int getCurrentLogVersion(FileSystem fs, Path partitionPath,
final String fileId, final String logFileExtension, final String baseCommitTime)
throws IOException {
Optional<Integer> currentVersion =
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
// handle potential overflow
return (currentVersion.isPresent()) ? currentVersion.get() : 1;
}
/**
* Gets all partition paths assuming date partitioning (year, month, day) three levels down.
*/
public static List<String> getAllFoldersThreeLevelsDown(FileSystem fs, String basePath) throws IOException {
List<String> datePartitions = new ArrayList<>();
FileStatus[] folders = fs.globStatus(new Path(basePath + "/*/*/*"));
for (FileStatus status : folders) {
Path path = status.getPath();
datePartitions.add(String.format("%s/%s/%s", path.getParent().getParent().getName(),
path.getParent().getName(), path.getName()));
}
return datePartitions;
/**
* computes the next log version for the specified fileId in the partition path
*/
public static int computeNextLogVersion(FileSystem fs, Path partitionPath, final String fileId,
final String logFileExtension, final String baseCommitTime) throws IOException {
Optional<Integer> currentVersion =
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
// handle potential overflow
return (currentVersion.isPresent()) ? currentVersion.get() + 1 : 1;
}
public static int getDefaultBufferSize(final FileSystem fs) {
return fs.getConf().getInt("io.file.buffer.size", 4096);
}
public static Short getDefaultReplication(FileSystem fs, Path path) {
return fs.getDefaultReplication(path);
}
public static Long getDefaultBlockSize(FileSystem fs, Path path) {
return fs.getDefaultBlockSize(path);
}
/**
* When a file was opened and the task died without closing the stream, another task executor
* cannot open because the existing lease will be active. We will try to recover the lease, from
* HDFS. If a data node went down, it takes about 10 minutes for the lease to be rocovered. But if
* the client dies, this should be instant.
*/
public static boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p)
throws IOException, InterruptedException {
LOG.info("Recover lease on dfs file " + p);
// initiate the recovery
boolean recovered = false;
for (int nbAttempt = 0; nbAttempt < MAX_ATTEMPTS_RECOVER_LEASE; nbAttempt++) {
LOG.info("Attempt " + nbAttempt + " to recover lease on dfs file " + p);
recovered = dfs.recoverLease(p);
if (recovered) {
break;
}
// Sleep for 1 second before trying again. Typically it takes about 2-3 seconds to recover under default settings
Thread.sleep(1000);
}
return recovered;
public static String getRelativePartitionPath(Path basePath, Path partitionPath) {
String partitionFullPath = partitionPath.toString();
int partitionStartIndex = partitionFullPath.lastIndexOf(basePath.getName());
return partitionFullPath.substring(partitionStartIndex + basePath.getName().length() + 1);
}
public static void deleteOlderCleanMetaFiles(FileSystem fs, String metaPath,
Stream<HoodieInstant> instants) {
//TODO - this should be archived when archival is made general for all meta-data
// skip MIN_CLEAN_TO_KEEP and delete rest
instants.skip(MIN_CLEAN_TO_KEEP).map(s -> {
try {
return fs.delete(new Path(metaPath, s.getFileName()), false);
} catch (IOException e) {
throw new HoodieIOException("Could not delete clean meta files" + s.getFileName(),
e);
}
});
}
public static void deleteOlderRollbackMetaFiles(FileSystem fs, String metaPath,
Stream<HoodieInstant> instants) {
//TODO - this should be archived when archival is made general for all meta-data
// skip MIN_ROLLBACK_TO_KEEP and delete rest
instants.skip(MIN_ROLLBACK_TO_KEEP).map(s -> {
try {
return fs.delete(new Path(metaPath, s.getFileName()), false);
} catch (IOException e) {
throw new HoodieIOException(
"Could not delete rollback meta files " + s.getFileName(), e);
}
});
}
public static void createPathIfNotExists(FileSystem fs, Path partitionPath) throws IOException {
if (!fs.exists(partitionPath)) {
fs.mkdirs(partitionPath);
}
}
/**
* Obtain all the partition paths, that are present in this table, denoted by presence of {@link
* com.uber.hoodie.common.model.HoodiePartitionMetadata#HOODIE_PARTITION_METAFILE}
*/
public static List<String> getAllFoldersWithPartitionMetaFile(FileSystem fs, String basePathStr)
throws IOException {
List<String> partitions = new ArrayList<>();
Path basePath = new Path(basePathStr);
RemoteIterator<LocatedFileStatus> allFiles = fs.listFiles(new Path(basePathStr), true);
while (allFiles.hasNext()) {
Path filePath = allFiles.next().getPath();
if (filePath.getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)) {
partitions.add(getRelativePartitionPath(basePath, filePath.getParent()));
}
}
return partitions;
}
public static List<String> getAllPartitionPaths(FileSystem fs, String basePathStr, boolean assumeDatePartitioning)
throws IOException {
if (assumeDatePartitioning) {
return getAllFoldersThreeLevelsDown(fs, basePathStr);
} else {
return getAllFoldersWithPartitionMetaFile(fs, basePathStr);
}
}
public static String getFileExtension(String fullName) {
Preconditions.checkNotNull(fullName);
String fileName = (new File(fullName)).getName();
int dotIndex = fileName.indexOf('.');
return dotIndex == -1 ? "" : fileName.substring(dotIndex);
}
public static String getInstantTime(String name) {
return name.replace(getFileExtension(name), "");
}
/**
* Get the file extension from the log file
*/
public static String getFileExtensionFromLog(Path logPath) {
Matcher matcher = LOG_FILE_PATTERN.matcher(logPath.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(logPath, "LogFile");
}
return matcher.group(3);
}
/**
* Get the first part of the file name in the log file. That will be the fileId. Log file do not
* have commitTime in the file name.
*/
public static String getFileIdFromLogPath(Path path) {
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(path, "LogFile");
}
return matcher.group(1);
}
/**
* Get the first part of the file name in the log file. That will be the fileId. Log file do not
* have commitTime in the file name.
*/
public static String getBaseCommitTimeFromLogPath(Path path) {
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(path, "LogFile");
}
return matcher.group(2);
}
/**
* Get the last part of the file name in the log file and convert to int.
*/
public static int getFileVersionFromLog(Path logPath) {
Matcher matcher = LOG_FILE_PATTERN.matcher(logPath.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(logPath, "LogFile");
}
return Integer.parseInt(matcher.group(4));
}
public static String makeLogFileName(String fileId, String logFileExtension,
String baseCommitTime, int version) {
return LOG_FILE_PREFIX + String.format("%s_%s%s.%d", fileId, baseCommitTime, logFileExtension, version);
}
public static String maskWithoutLogVersion(String commitTime, String fileId, String logFileExtension) {
return LOG_FILE_PREFIX + String.format("%s_%s%s*", fileId, commitTime, logFileExtension);
}
/**
* Get the latest log file written from the list of log files passed in
*/
public static Optional<HoodieLogFile> getLatestLogFile(Stream<HoodieLogFile> logFiles) {
return logFiles.sorted(Comparator
.comparing(s -> s.getLogVersion(),
Comparator.reverseOrder())).findFirst();
}
/**
* Get all the log files for the passed in FileId in the partition path
*/
public static Stream<HoodieLogFile> getAllLogFiles(FileSystem fs, Path partitionPath,
final String fileId, final String logFileExtension, final String baseCommitTime) throws IOException {
return Arrays.stream(fs.listStatus(partitionPath,
path -> path.getName().startsWith("." + fileId) && path.getName().contains(logFileExtension)))
.map(HoodieLogFile::new).filter(s -> s.getBaseCommitTime().equals(baseCommitTime));
}
/**
* Get the latest log version for the fileId in the partition path
*/
public static Optional<Integer> getLatestLogVersion(FileSystem fs, Path partitionPath,
final String fileId, final String logFileExtension, final String baseCommitTime) throws IOException {
Optional<HoodieLogFile> latestLogFile =
getLatestLogFile(getAllLogFiles(fs, partitionPath, fileId, logFileExtension, baseCommitTime));
if (latestLogFile.isPresent()) {
return Optional.of(latestLogFile.get().getLogVersion());
}
return Optional.empty();
}
public static int getCurrentLogVersion(FileSystem fs, Path partitionPath,
final String fileId, final String logFileExtension, final String baseCommitTime) throws IOException {
Optional<Integer> currentVersion =
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
// handle potential overflow
return (currentVersion.isPresent()) ? currentVersion.get() : 1;
}
/**
* computes the next log version for the specified fileId in the partition path
*/
public static int computeNextLogVersion(FileSystem fs, Path partitionPath, final String fileId,
final String logFileExtension, final String baseCommitTime) throws IOException {
Optional<Integer> currentVersion =
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
// handle potential overflow
return (currentVersion.isPresent()) ? currentVersion.get() + 1 : 1;
}
public static int getDefaultBufferSize(final FileSystem fs) {
return fs.getConf().getInt("io.file.buffer.size", 4096);
}
public static Short getDefaultReplication(FileSystem fs, Path path) {
return fs.getDefaultReplication(path);
}
public static Long getDefaultBlockSize(FileSystem fs, Path path) {
return fs.getDefaultBlockSize(path);
}
/**
* When a file was opened and the task died without closing the stream, another task executor
* cannot open because the existing lease will be active. We will try to recover the lease, from
* HDFS. If a data node went down, it takes about 10 minutes for the lease to be rocovered. But
* if the client dies, this should be instant.
*/
public static boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p)
throws IOException, InterruptedException {
LOG.info("Recover lease on dfs file " + p);
// initiate the recovery
boolean recovered = false;
for (int nbAttempt = 0; nbAttempt < MAX_ATTEMPTS_RECOVER_LEASE; nbAttempt++) {
LOG.info("Attempt " + nbAttempt + " to recover lease on dfs file " + p);
recovered = dfs.recoverLease(p);
if (recovered)
break;
// Sleep for 1 second before trying again. Typically it takes about 2-3 seconds to recover under default settings
Thread.sleep(1000);
}
return recovered;
}
public static void deleteOlderCleanMetaFiles(FileSystem fs, String metaPath,
Stream<HoodieInstant> instants) {
//TODO - this should be archived when archival is made general for all meta-data
// skip MIN_CLEAN_TO_KEEP and delete rest
instants.skip(MIN_CLEAN_TO_KEEP).map(s -> {
try {
return fs.delete(new Path(metaPath, s.getFileName()), false);
} catch (IOException e) {
throw new HoodieIOException("Could not delete clean meta files" + s.getFileName(),
e);
}
});
}
public static void deleteOlderRollbackMetaFiles(FileSystem fs, String metaPath,
Stream<HoodieInstant> instants) {
//TODO - this should be archived when archival is made general for all meta-data
// skip MIN_ROLLBACK_TO_KEEP and delete rest
instants.skip(MIN_ROLLBACK_TO_KEEP).map(s -> {
try {
return fs.delete(new Path(metaPath, s.getFileName()), false);
} catch (IOException e) {
throw new HoodieIOException(
"Could not delete rollback meta files " + s.getFileName(), e);
}
});
}
public static void createPathIfNotExists(FileSystem fs, Path partitionPath) throws IOException {
if(!fs.exists(partitionPath)) {
fs.mkdirs(partitionPath);
}
}
public static Long getSizeInMB(long sizeInBytes) {
return sizeInBytes / (1024 * 1024);
}
public static Long getSizeInMB(long sizeInBytes) {
return sizeInBytes / (1024 * 1024);
}
}

View File

@@ -17,156 +17,167 @@
package com.uber.hoodie.common.util;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.SchemaCompatabilityException;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
import org.apache.avro.Schema;
import org.apache.avro.generic.*;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.BinaryEncoder;
import org.apache.avro.io.Decoder;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.EncoderFactory;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
/**
* Helper class to do common stuff across Avro.
*/
public class HoodieAvroUtils {
// All metadata fields are optional strings.
private final static Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList(
Schema.create(Schema.Type.NULL),
Schema.create(Schema.Type.STRING)));
// All metadata fields are optional strings.
private final static Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList(
Schema.create(Schema.Type.NULL),
Schema.create(Schema.Type.STRING)));
private final static Schema RECORD_KEY_SCHEMA = initRecordKeySchema();
private final static Schema RECORD_KEY_SCHEMA = initRecordKeySchema();
/**
* Convert a given avro record to bytes
*/
public static byte[] avroToBytes(GenericRecord record) throws IOException {
GenericDatumWriter<GenericRecord> writer =
new GenericDatumWriter<>(record.getSchema());
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
writer.write(record, encoder);
encoder.flush();
out.close();
return out.toByteArray();
/**
* Convert a given avro record to bytes
*/
public static byte[] avroToBytes(GenericRecord record) throws IOException {
GenericDatumWriter<GenericRecord> writer =
new GenericDatumWriter<>(record.getSchema());
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
writer.write(record, encoder);
encoder.flush();
out.close();
return out.toByteArray();
}
/**
* Convert serialized bytes back into avro record
*/
public static GenericRecord bytesToAvro(byte[] bytes, Schema schema) throws IOException {
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<GenericRecord>(schema);
return reader.read(null, decoder);
}
/**
* Adds the Hoodie metadata fields to the given schema
*/
public static Schema addMetadataFields(Schema schema) {
List<Schema.Field> parentFields = new ArrayList<>();
Schema.Field commitTimeField = new Schema.Field(HoodieRecord.COMMIT_TIME_METADATA_FIELD,
METADATA_FIELD_SCHEMA, "", null);
Schema.Field commitSeqnoField = new Schema.Field(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD,
METADATA_FIELD_SCHEMA, "", null);
Schema.Field recordKeyField = new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD,
METADATA_FIELD_SCHEMA, "", null);
Schema.Field partitionPathField = new Schema.Field(HoodieRecord.PARTITION_PATH_METADATA_FIELD,
METADATA_FIELD_SCHEMA, "", null);
Schema.Field fileNameField = new Schema.Field(HoodieRecord.FILENAME_METADATA_FIELD,
METADATA_FIELD_SCHEMA, "", null);
parentFields.add(commitTimeField);
parentFields.add(commitSeqnoField);
parentFields.add(recordKeyField);
parentFields.add(partitionPathField);
parentFields.add(fileNameField);
for (Schema.Field field : schema.getFields()) {
parentFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), null));
}
/**
* Convert serialized bytes back into avro record
*/
public static GenericRecord bytesToAvro(byte[] bytes, Schema schema) throws IOException {
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<GenericRecord>(schema);
return reader.read(null, decoder);
Schema mergedSchema = Schema
.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false);
mergedSchema.setFields(parentFields);
return mergedSchema;
}
private static Schema initRecordKeySchema() {
Schema.Field recordKeyField = new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD,
METADATA_FIELD_SCHEMA, "", null);
Schema recordKeySchema = Schema.createRecord("HoodieRecordKey", "", "", false);
recordKeySchema.setFields(Arrays.asList(recordKeyField));
return recordKeySchema;
}
public static Schema getRecordKeySchema() {
return RECORD_KEY_SCHEMA;
}
public static GenericRecord addHoodieKeyToRecord(GenericRecord record, String recordKey,
String partitionPath, String fileName) {
record.put(HoodieRecord.FILENAME_METADATA_FIELD, fileName);
record.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, partitionPath);
record.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recordKey);
return record;
}
/**
* Adds the Hoodie commit metadata into the provided Generic Record.
*/
public static GenericRecord addCommitMetadataToRecord(GenericRecord record, String commitTime,
String commitSeqno) {
record.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime);
record.put(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, commitSeqno);
return record;
}
/**
* Given a avro record with a given schema, rewrites it into the new schema
*/
public static GenericRecord rewriteRecord(GenericRecord record, Schema newSchema) {
GenericRecord newRecord = new GenericData.Record(newSchema);
for (Schema.Field f : record.getSchema().getFields()) {
newRecord.put(f.name(), record.get(f.name()));
}
/**
* Adds the Hoodie metadata fields to the given schema
*/
public static Schema addMetadataFields(Schema schema) {
List<Schema.Field> parentFields = new ArrayList<>();
Schema.Field commitTimeField = new Schema.Field(HoodieRecord.COMMIT_TIME_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
Schema.Field commitSeqnoField = new Schema.Field(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
Schema.Field recordKeyField = new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
Schema.Field partitionPathField = new Schema.Field(HoodieRecord.PARTITION_PATH_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
Schema.Field fileNameField = new Schema.Field(HoodieRecord.FILENAME_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
parentFields.add(commitTimeField);
parentFields.add(commitSeqnoField);
parentFields.add(recordKeyField);
parentFields.add(partitionPathField);
parentFields.add(fileNameField);
for (Schema.Field field : schema.getFields()) {
parentFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), null));
}
Schema mergedSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false);
mergedSchema.setFields(parentFields);
return mergedSchema;
if (!new GenericData().validate(newSchema, newRecord)) {
throw new SchemaCompatabilityException(
"Unable to validate the rewritten record " + record + " against schema "
+ newSchema);
}
return newRecord;
}
private static Schema initRecordKeySchema() {
Schema.Field recordKeyField = new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
Schema recordKeySchema = Schema.createRecord("HoodieRecordKey", "", "", false);
recordKeySchema.setFields(Arrays.asList(recordKeyField));
return recordKeySchema;
public static byte[] compress(String text) {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
OutputStream out = new DeflaterOutputStream(baos);
out.write(text.getBytes("UTF-8"));
out.close();
} catch (IOException e) {
throw new HoodieIOException("IOException while compressing text " + text, e);
}
return baos.toByteArray();
}
public static Schema getRecordKeySchema() {
return RECORD_KEY_SCHEMA;
}
public static GenericRecord addHoodieKeyToRecord(GenericRecord record, String recordKey, String partitionPath, String fileName) {
record.put(HoodieRecord.FILENAME_METADATA_FIELD, fileName);
record.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, partitionPath);
record.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recordKey);
return record;
}
/**
* Adds the Hoodie commit metadata into the provided Generic Record.
*/
public static GenericRecord addCommitMetadataToRecord(GenericRecord record, String commitTime, String commitSeqno) {
record.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime);
record.put(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, commitSeqno);
return record;
}
/**
* Given a avro record with a given schema, rewrites it into the new schema
*/
public static GenericRecord rewriteRecord(GenericRecord record, Schema newSchema) {
GenericRecord newRecord = new GenericData.Record(newSchema);
for (Schema.Field f : record.getSchema().getFields()) {
newRecord.put(f.name(), record.get(f.name()));
}
if (!new GenericData().validate(newSchema, newRecord)) {
throw new SchemaCompatabilityException(
"Unable to validate the rewritten record " + record + " against schema "
+ newSchema);
}
return newRecord;
}
public static byte[] compress(String text) {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
OutputStream out = new DeflaterOutputStream(baos);
out.write(text.getBytes("UTF-8"));
out.close();
} catch (IOException e) {
throw new HoodieIOException("IOException while compressing text " + text, e);
}
return baos.toByteArray();
}
public static String decompress(byte[] bytes) {
InputStream in = new InflaterInputStream(new ByteArrayInputStream(bytes));
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
byte[] buffer = new byte[8192];
int len;
while((len = in.read(buffer))>0)
baos.write(buffer, 0, len);
return new String(baos.toByteArray(), "UTF-8");
} catch (IOException e) {
throw new HoodieIOException("IOException while decompressing text", e);
}
public static String decompress(byte[] bytes) {
InputStream in = new InflaterInputStream(new ByteArrayInputStream(bytes));
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
byte[] buffer = new byte[8192];
int len;
while ((len = in.read(buffer)) > 0) {
baos.write(buffer, 0, len);
}
return new String(baos.toByteArray(), "UTF-8");
} catch (IOException e) {
throw new HoodieIOException("IOException while decompressing text", e);
}
}
}

View File

@@ -17,10 +17,13 @@
package com.uber.hoodie.common.util;
public class NumericUtils {
public static String humanReadableByteCount(double bytes) {
if (bytes < 1024) return String.format("%.1f B", bytes);
int exp = (int) (Math.log(bytes) / Math.log(1024));
String pre = "KMGTPE".charAt(exp-1) + "";
return String.format("%.1f %sB", bytes / Math.pow(1024, exp), pre);
public static String humanReadableByteCount(double bytes) {
if (bytes < 1024) {
return String.format("%.1f B", bytes);
}
int exp = (int) (Math.log(bytes) / Math.log(1024));
String pre = "KMGTPE".charAt(exp - 1) + "";
return String.format("%.1f %sB", bytes / Math.pow(1024, exp), pre);
}
}

View File

@@ -16,14 +16,20 @@
package com.uber.hoodie.common.util;
import static com.uber.hoodie.common.util.FSUtils.getFs;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.MetadataNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
@@ -36,163 +42,144 @@ import org.apache.parquet.hadoop.ParquetReader;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
import org.apache.parquet.schema.MessageType;
import java.io.*;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static com.uber.hoodie.common.util.FSUtils.getFs;
/**
* Utility functions involving with parquet.
*/
public class ParquetUtils {
/**
* Read the rowKey list from the given parquet file.
*
* @param filePath The parquet file path.
*/
public static Set<String> readRowKeysFromParquet(Path filePath) {
Configuration conf = new Configuration();
conf.addResource(getFs().getConf());
Schema readSchema = HoodieAvroUtils.getRecordKeySchema();
AvroReadSupport.setAvroReadSchema(conf, readSchema);
AvroReadSupport.setRequestedProjection(conf, readSchema);
ParquetReader reader = null;
Set<String> rowKeys = new HashSet<>();
/**
* Read the rowKey list from the given parquet file.
*
* @param filePath The parquet file path.
*/
public static Set<String> readRowKeysFromParquet(Path filePath) {
Configuration conf = new Configuration();
conf.addResource(getFs().getConf());
Schema readSchema = HoodieAvroUtils.getRecordKeySchema();
AvroReadSupport.setAvroReadSchema(conf, readSchema);
AvroReadSupport.setRequestedProjection(conf, readSchema);
ParquetReader reader = null;
Set<String> rowKeys = new HashSet<>();
try {
reader = AvroParquetReader.builder(filePath).withConf(conf).build();
Object obj = reader.read();
while (obj != null) {
if (obj instanceof GenericRecord) {
rowKeys.add(((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString());
}
obj = reader.read();
}
} catch (IOException e) {
throw new HoodieIOException("Failed to read row keys from Parquet " + filePath, e);
} finally {
if (reader != null) {
try {
reader = AvroParquetReader.builder(filePath).withConf(conf).build();
Object obj = reader.read();
while (obj != null) {
if (obj instanceof GenericRecord) {
rowKeys.add(((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString());
}
obj = reader.read();
}
reader.close();
} catch (IOException e) {
throw new HoodieIOException("Failed to read row keys from Parquet " + filePath, e);
} finally {
if (reader != null) {
try {
reader.close();
} catch (IOException e) {
// ignore
}
}
// ignore
}
return rowKeys;
}
}
return rowKeys;
}
/**
*
* Read the metadata from a parquet file
*
* @param parquetFilePath
* @return
*/
public static ParquetMetadata readMetadata(Path parquetFilePath) {
return readMetadata(new Configuration(), parquetFilePath);
/**
* Read the metadata from a parquet file
*/
public static ParquetMetadata readMetadata(Path parquetFilePath) {
return readMetadata(new Configuration(), parquetFilePath);
}
public static ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) {
ParquetMetadata footer;
try {
// TODO(vc): Should we use the parallel reading version here?
footer = ParquetFileReader.readFooter(getFs().getConf(), parquetFilePath);
} catch (IOException e) {
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath,
e);
}
return footer;
}
public static ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) {
ParquetMetadata footer;
/**
* Get the schema of the given parquet file.
*/
public static MessageType readSchema(Path parquetFilePath) {
return readMetadata(parquetFilePath).getFileMetaData().getSchema();
}
private static List<String> readParquetFooter(Path parquetFilePath, String... footerNames) {
List<String> footerVals = new ArrayList<>();
ParquetMetadata footer = readMetadata(parquetFilePath);
Map<String, String> metadata = footer.getFileMetaData().getKeyValueMetaData();
for (String footerName : footerNames) {
if (metadata.containsKey(footerName)) {
footerVals.add(metadata.get(footerName));
} else {
throw new MetadataNotFoundException("Could not find index in Parquet footer. " +
"Looked for key " + footerName + " in " + parquetFilePath);
}
}
return footerVals;
}
public static Schema readAvroSchema(Path parquetFilePath) {
return new AvroSchemaConverter().convert(readSchema(parquetFilePath));
}
/**
* Read out the bloom filter from the parquet file meta data.
*/
public static BloomFilter readBloomFilterFromParquetMetadata(Path parquetFilePath) {
String footerVal = readParquetFooter(parquetFilePath,
HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY).get(0);
return new BloomFilter(footerVal);
}
public static String[] readMinMaxRecordKeys(Path parquetFilePath) {
List<String> minMaxKeys = readParquetFooter(parquetFilePath,
HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER,
HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER);
if (minMaxKeys.size() != 2) {
throw new HoodieException(String.format(
"Could not read min/max record key out of footer correctly from %s. read) : %s",
parquetFilePath, minMaxKeys));
}
return new String[]{minMaxKeys.get(0), minMaxKeys.get(1)};
}
/**
* NOTE: This literally reads the entire file contents, thus should be used with caution.
*/
public static List<GenericRecord> readAvroRecords(Path filePath) {
ParquetReader reader = null;
List<GenericRecord> records = new ArrayList<>();
try {
reader = AvroParquetReader.builder(filePath).build();
Object obj = reader.read();
while (obj != null) {
if (obj instanceof GenericRecord) {
records.add(((GenericRecord) obj));
}
obj = reader.read();
}
} catch (IOException e) {
throw new HoodieIOException("Failed to read avro records from Parquet " + filePath, e);
} finally {
if (reader != null) {
try {
// TODO(vc): Should we use the parallel reading version here?
footer = ParquetFileReader.readFooter(getFs().getConf(), parquetFilePath);
reader.close();
} catch (IOException e) {
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath,
e);
// ignore
}
return footer;
}
/**
* Get the schema of the given parquet file.
*
* @param parquetFilePath
* @return
*/
public static MessageType readSchema(Path parquetFilePath) {
return readMetadata(parquetFilePath).getFileMetaData().getSchema();
}
private static List<String> readParquetFooter(Path parquetFilePath, String... footerNames) {
List<String> footerVals = new ArrayList<>();
ParquetMetadata footer = readMetadata(parquetFilePath);
Map<String, String> metadata = footer.getFileMetaData().getKeyValueMetaData();
for (String footerName : footerNames) {
if (metadata.containsKey(footerName)) {
footerVals.add(metadata.get(footerName));
} else {
throw new MetadataNotFoundException("Could not find index in Parquet footer. " +
"Looked for key " + footerName + " in " + parquetFilePath);
}
}
return footerVals;
}
public static Schema readAvroSchema(Path parquetFilePath) {
return new AvroSchemaConverter().convert(readSchema(parquetFilePath));
}
/**
* Read out the bloom filter from the parquet file meta data.
*/
public static BloomFilter readBloomFilterFromParquetMetadata(Path parquetFilePath) {
String footerVal = readParquetFooter(parquetFilePath,
HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY).get(0);
return new BloomFilter(footerVal);
}
public static String[] readMinMaxRecordKeys(Path parquetFilePath) {
List<String> minMaxKeys = readParquetFooter(parquetFilePath, HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER,
HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER);
if (minMaxKeys.size() != 2) {
throw new HoodieException(String.format(
"Could not read min/max record key out of footer correctly from %s. read) : %s",
parquetFilePath, minMaxKeys));
}
return new String[]{minMaxKeys.get(0), minMaxKeys.get(1)};
}
/**
*
* NOTE: This literally reads the entire file contents, thus should be used with caution.
*
* @param filePath
* @return
*/
public static List<GenericRecord> readAvroRecords(Path filePath) {
ParquetReader reader = null;
List<GenericRecord> records = new ArrayList<>();
try {
reader = AvroParquetReader.builder(filePath).build();
Object obj = reader.read();
while (obj != null) {
if (obj instanceof GenericRecord) {
records.add(((GenericRecord) obj));
}
obj = reader.read();
}
} catch (IOException e) {
throw new HoodieIOException("Failed to read avro records from Parquet " + filePath, e);
} finally {
if (reader != null) {
try {
reader.close();
} catch (IOException e) {
// ignore
}
}
}
return records;
}
}
return records;
}
}

View File

@@ -17,22 +17,18 @@
package com.uber.hoodie.common.util;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.exception.HoodieException;
import org.apache.avro.generic.GenericRecord;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
public class ReflectionUtils {
private static Map<String, Class<?>> clazzCache = new HashMap<>();
public static <T> T loadClass(String fqcn) {
try {
if(clazzCache.get(fqcn) == null) {
if (clazzCache.get(fqcn) == null) {
Class<?> clazz = Class.<HoodieRecordPayload>forName(fqcn);
clazzCache.put(fqcn, clazz);
}
@@ -48,21 +44,17 @@ public class ReflectionUtils {
/**
* Instantiate a given class with a generic record payload
*
* @param recordPayloadClass
* @param payloadArgs
* @param <T>
* @return
*/
public static <T extends HoodieRecordPayload> T loadPayload(String recordPayloadClass,
Object [] payloadArgs,
Class<?> ... constructorArgTypes) {
Object[] payloadArgs,
Class<?>... constructorArgTypes) {
try {
if(clazzCache.get(recordPayloadClass) == null) {
if (clazzCache.get(recordPayloadClass) == null) {
Class<?> clazz = Class.<HoodieRecordPayload>forName(recordPayloadClass);
clazzCache.put(recordPayloadClass, clazz);
}
return (T) clazzCache.get(recordPayloadClass).getConstructor(constructorArgTypes).newInstance(payloadArgs);
return (T) clazzCache.get(recordPayloadClass).getConstructor(constructorArgTypes)
.newInstance(payloadArgs);
} catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
throw new HoodieException("Unable to instantiate payload class ", e);
} catch (ClassNotFoundException e) {

View File

@@ -16,43 +16,40 @@
package com.uber.hoodie.exception;
import java.io.IOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
/**
* <p>
* Exception thrown to indicate that a hoodie dataset was not found on the path provided
* <p>
* <p> Exception thrown to indicate that a hoodie dataset was not found on the path provided <p>
*/
public class DatasetNotFoundException extends HoodieException {
public DatasetNotFoundException(String basePath) {
super(getErrorMessage(basePath));
}
private static String getErrorMessage(String basePath) {
return "Hoodie dataset not found in path " + basePath;
}
public DatasetNotFoundException(String basePath) {
super(getErrorMessage(basePath));
}
public static void checkValidDataset(FileSystem fs, Path basePathDir, Path metaPathDir)
throws DatasetNotFoundException {
// Check if the base path is found
try {
if (!fs.exists(basePathDir) || !fs.isDirectory(basePathDir)) {
throw new DatasetNotFoundException(basePathDir.toString());
}
// Check if the meta path is found
if (!fs.exists(metaPathDir) || !fs.isDirectory(metaPathDir)) {
throw new DatasetNotFoundException(metaPathDir.toString());
}
} catch (IllegalArgumentException e) {
// if the base path is file:///, then we have a IllegalArgumentException
throw new DatasetNotFoundException(metaPathDir.toString());
}
catch (IOException e) {
throw new HoodieIOException(
"Could not check if dataset " + basePathDir + " is valid dataset", e);
}
private static String getErrorMessage(String basePath) {
return "Hoodie dataset not found in path " + basePath;
}
public static void checkValidDataset(FileSystem fs, Path basePathDir, Path metaPathDir)
throws DatasetNotFoundException {
// Check if the base path is found
try {
if (!fs.exists(basePathDir) || !fs.isDirectory(basePathDir)) {
throw new DatasetNotFoundException(basePathDir.toString());
}
// Check if the meta path is found
if (!fs.exists(metaPathDir) || !fs.isDirectory(metaPathDir)) {
throw new DatasetNotFoundException(metaPathDir.toString());
}
} catch (IllegalArgumentException e) {
// if the base path is file:///, then we have a IllegalArgumentException
throw new DatasetNotFoundException(metaPathDir.toString());
} catch (IOException e) {
throw new HoodieIOException(
"Could not check if dataset " + basePathDir + " is valid dataset", e);
}
}
}

View File

@@ -19,39 +19,34 @@ package com.uber.hoodie.exception;
import java.io.Serializable;
/**
* <p>
* Exception thrown for Hoodie failures. The root of
* the exception hierarchy.
* </p>
* <p>
* Hoodie Write/Read clients will throw this exception if
* any of its operations fail. This is a runtime (unchecked) exception.
* </p>
*
* <p> Exception thrown for Hoodie failures. The root of the exception hierarchy. </p> <p> Hoodie
* Write/Read clients will throw this exception if any of its operations fail. This is a runtime
* (unchecked) exception. </p>
*/
public class HoodieException extends RuntimeException implements Serializable {
public HoodieException() {
super();
}
public HoodieException(String message) {
super(message);
}
public HoodieException() {
super();
}
public HoodieException(String message, Throwable t) {
super(message, t);
}
public HoodieException(String message) {
super(message);
}
public HoodieException(Throwable t) {
super(t);
}
public HoodieException(String message, Throwable t) {
super(message, t);
}
protected static String format(String message, Object... args) {
String[] argStrings = new String[args.length];
for (int i = 0; i < args.length; i += 1) {
argStrings[i] = String.valueOf(args[i]);
}
return String.format(String.valueOf(message), (Object[]) argStrings);
public HoodieException(Throwable t) {
super(t);
}
protected static String format(String message, Object... args) {
String[] argStrings = new String[args.length];
for (int i = 0; i < args.length; i += 1) {
argStrings[i] = String.valueOf(args[i]);
}
return String.format(String.valueOf(message), (Object[]) argStrings);
}
}

View File

@@ -19,23 +19,22 @@ package com.uber.hoodie.exception;
import java.io.IOException;
/**
* <p>
* Exception thrown for dataset IO-related failures.
* </p>
* <p> Exception thrown for dataset IO-related failures. </p>
*/
public class HoodieIOException extends HoodieException {
private IOException ioException;
public HoodieIOException(String msg, IOException t) {
super(msg, t);
this.ioException = t;
}
private IOException ioException;
public HoodieIOException(String msg) {
super(msg);
}
public HoodieIOException(String msg, IOException t) {
super(msg, t);
this.ioException = t;
}
public IOException getIOException() {
return ioException;
}
public HoodieIOException(String msg) {
super(msg);
}
public IOException getIOException() {
return ioException;
}
}

View File

@@ -17,16 +17,15 @@
package com.uber.hoodie.exception;
/**
* <p>
* Exception thrown for HoodieIndex related errors.
* </p>
* <p> Exception thrown for HoodieIndex related errors. </p>
*/
public class HoodieIndexException extends HoodieException {
public HoodieIndexException(String msg) {
super(msg);
}
public HoodieIndexException(String msg, Throwable e) {
super(msg, e);
}
public HoodieIndexException(String msg) {
super(msg);
}
public HoodieIndexException(String msg, Throwable e) {
super(msg, e);
}
}

View File

@@ -17,7 +17,8 @@
package com.uber.hoodie.exception;
public class HoodieNotSupportedException extends HoodieException {
public HoodieNotSupportedException(String errorMsg) {
super(errorMsg);
}
public HoodieNotSupportedException(String errorMsg) {
super(errorMsg);
}
}

View File

@@ -19,17 +19,15 @@ package com.uber.hoodie.exception;
import com.uber.hoodie.common.model.HoodieRecord;
/**
* <p>
* Exception throws when indexing fails to locate the hoodie record.
* HoodieRecord current location and partition path does not match.
* This is an unrecoverable error
* </p>
* <p> Exception throws when indexing fails to locate the hoodie record. HoodieRecord current
* location and partition path does not match. This is an unrecoverable error </p>
*/
public class HoodieRecordMissingException extends HoodieException {
public HoodieRecordMissingException(HoodieRecord record) {
super(
"Record " + record.getRecordKey() + " with partition path " + record.getPartitionPath()
+ " in current location " + record.getCurrentLocation()
+ " is not found in the partition");
}
public HoodieRecordMissingException(HoodieRecord record) {
super(
"Record " + record.getRecordKey() + " with partition path " + record.getPartitionPath()
+ " in current location " + record.getCurrentLocation()
+ " is not found in the partition");
}
}

View File

@@ -17,16 +17,15 @@
package com.uber.hoodie.exception;
/**
* <p>
* Exception thrown to indicate that a hoodie dataset is invalid
* <p>
* <p> Exception thrown to indicate that a hoodie dataset is invalid <p>
*/
public class InvalidDatasetException extends HoodieException {
public InvalidDatasetException(String basePath) {
super(getErrorMessage(basePath));
}
private static String getErrorMessage(String basePath) {
return "Invalid Hoodie Dataset. " + basePath;
}
public InvalidDatasetException(String basePath) {
super(getErrorMessage(basePath));
}
private static String getErrorMessage(String basePath) {
return "Invalid Hoodie Dataset. " + basePath;
}
}

View File

@@ -19,7 +19,8 @@ package com.uber.hoodie.exception;
import org.apache.hadoop.fs.Path;
public class InvalidHoodiePathException extends HoodieException {
public InvalidHoodiePathException(Path path, String type) {
super("Invalid path " + path + " of type " + type);
}
public InvalidHoodiePathException(Path path, String type) {
super("Invalid path " + path + " of type " + type);
}
}

View File

@@ -22,11 +22,12 @@ package com.uber.hoodie.exception;
* Thrown when expected metadata is not found
*/
public class MetadataNotFoundException extends HoodieException {
public MetadataNotFoundException(String msg) {
super(msg);
}
public MetadataNotFoundException(String msg, Throwable e) {
super(msg, e);
}
public MetadataNotFoundException(String msg) {
super(msg);
}
public MetadataNotFoundException(String msg, Throwable e) {
super(msg, e);
}
}

View File

@@ -17,15 +17,16 @@
package com.uber.hoodie.exception;
public class SchemaCompatabilityException extends HoodieException {
public SchemaCompatabilityException(String message) {
super(message);
}
public SchemaCompatabilityException(String message, Throwable t) {
super(message, t);
}
public SchemaCompatabilityException(String message) {
super(message);
}
public SchemaCompatabilityException(Throwable t) {
super(t);
}
public SchemaCompatabilityException(String message, Throwable t) {
super(message, t);
}
public SchemaCompatabilityException(Throwable t) {
super(t);
}
}

View File

@@ -16,30 +16,30 @@
package com.uber.hoodie.common;
import java.io.IOException;
import org.junit.Test;
import java.io.*;
public class
TestBloomFilter {
@Test
public void testAddKey() {
BloomFilter filter = new BloomFilter(100, 0.0000001);
filter.add("key1");
assert (filter.mightContain("key1"));
}
@Test
public void testSerialize() throws IOException, ClassNotFoundException {
BloomFilter filter = new BloomFilter(1000, 0.0000001);
filter.add("key1");
filter.add("key2");
String filterStr = filter.serializeToString();
@Test
public void testAddKey() {
BloomFilter filter = new BloomFilter(100, 0.0000001);
filter.add("key1");
assert (filter.mightContain("key1"));
}
// Rebuild
BloomFilter newFilter = new BloomFilter(filterStr);
assert (newFilter.mightContain("key1"));
assert (newFilter.mightContain("key2"));
}
@Test
public void testSerialize() throws IOException, ClassNotFoundException {
BloomFilter filter = new BloomFilter(1000, 0.0000001);
filter.add("key1");
filter.add("key2");
String filterStr = filter.serializeToString();
// Rebuild
BloomFilter newFilter = new BloomFilter(filterStr);
assert (newFilter.mightContain("key1"));
assert (newFilter.mightContain("key2"));
}
}

View File

@@ -19,6 +19,8 @@ package com.uber.hoodie.common.minicluster;
import com.google.common.base.Preconditions;
import com.google.common.io.Files;
import java.io.File;
import java.io.IOException;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@@ -27,140 +29,129 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
/**
* An HDFS minicluster service implementation.
*/
public class HdfsTestService {
private static final Logger logger = LoggerFactory.getLogger(HdfsTestService.class);
private static final Logger logger = LoggerFactory.getLogger(HdfsTestService.class);
/**
* Configuration settings
*/
private Configuration hadoopConf;
private String workDir;
private String bindIP = "127.0.0.1";
private int namenodeRpcPort = 8020;
private int namenodeHttpPort = 50070;
private int datanodePort = 50010;
private int datanodeIpcPort = 50020;
private int datanodeHttpPort = 50075;
/**
* Configuration settings
*/
private Configuration hadoopConf;
private String workDir;
private String bindIP = "127.0.0.1";
private int namenodeRpcPort = 8020;
private int namenodeHttpPort = 50070;
private int datanodePort = 50010;
private int datanodeIpcPort = 50020;
private int datanodeHttpPort = 50075;
/**
* Embedded HDFS cluster
*/
private MiniDFSCluster miniDfsCluster;
/**
* Embedded HDFS cluster
*/
private MiniDFSCluster miniDfsCluster;
public HdfsTestService() {
hadoopConf = new Configuration();
workDir = Files.createTempDir().getAbsolutePath();
public HdfsTestService() {
hadoopConf = new Configuration();
workDir = Files.createTempDir().getAbsolutePath();
}
public Configuration getHadoopConf() {
return hadoopConf;
}
public MiniDFSCluster start(boolean format) throws IOException {
Preconditions
.checkState(workDir != null, "The work dir must be set before starting cluster.");
if (hadoopConf == null) {
hadoopConf = new Configuration();
}
public Configuration getHadoopConf() {
return hadoopConf;
// If clean, then remove the work dir so we can start fresh.
String localDFSLocation = getDFSLocation(workDir);
if (format) {
logger.info(
"Cleaning HDFS cluster data at: " + localDFSLocation + " and starting fresh.");
File file = new File(localDFSLocation);
FileUtils.deleteDirectory(file);
}
public MiniDFSCluster start(boolean format) throws IOException {
Preconditions
.checkState(workDir != null, "The work dir must be set before starting cluster.");
// Configure and start the HDFS cluster
// boolean format = shouldFormatDFSCluster(localDFSLocation, clean);
hadoopConf = configureDFSCluster(hadoopConf, localDFSLocation, bindIP, namenodeRpcPort,
namenodeHttpPort, datanodePort, datanodeIpcPort, datanodeHttpPort);
miniDfsCluster = new MiniDFSCluster.Builder(hadoopConf).numDataNodes(1).format(format)
.checkDataNodeAddrConfig(true).checkDataNodeHostConfig(true).build();
logger.info("HDFS Minicluster service started.");
return miniDfsCluster;
}
if (hadoopConf == null) {
hadoopConf = new Configuration();
}
public void stop() throws IOException {
miniDfsCluster.shutdown();
logger.info("HDFS Minicluster service shut down.");
miniDfsCluster = null;
hadoopConf = null;
}
// If clean, then remove the work dir so we can start fresh.
String localDFSLocation = getDFSLocation(workDir);
if (format) {
logger.info(
"Cleaning HDFS cluster data at: " + localDFSLocation + " and starting fresh.");
File file = new File(localDFSLocation);
FileUtils.deleteDirectory(file);
}
/**
* Get the location on the local FS where we store the HDFS data.
*
* @param baseFsLocation The base location on the local filesystem we have write access to create
* dirs.
* @return The location for HDFS data.
*/
private static String getDFSLocation(String baseFsLocation) {
return baseFsLocation + Path.SEPARATOR + "dfs";
}
// Configure and start the HDFS cluster
// boolean format = shouldFormatDFSCluster(localDFSLocation, clean);
hadoopConf = configureDFSCluster(hadoopConf, localDFSLocation, bindIP, namenodeRpcPort,
namenodeHttpPort, datanodePort, datanodeIpcPort, datanodeHttpPort);
miniDfsCluster = new MiniDFSCluster.Builder(hadoopConf).numDataNodes(1).format(format)
.checkDataNodeAddrConfig(true).checkDataNodeHostConfig(true).build();
logger.info("HDFS Minicluster service started.");
return miniDfsCluster;
/**
* Returns true if we should format the DFS Cluster. We'll format if clean is true, or if the
* dfsFsLocation does not exist.
*
* @param localDFSLocation The location on the local FS to hold the HDFS metadata and block data
* @param clean Specifies if we want to start a clean cluster
* @return Returns true if we should format a DFSCluster, otherwise false
*/
private static boolean shouldFormatDFSCluster(String localDFSLocation, boolean clean) {
boolean format = true;
File f = new File(localDFSLocation);
if (f.exists() && f.isDirectory() && !clean) {
format = false;
}
return format;
}
public void stop() throws IOException {
miniDfsCluster.shutdown();
logger.info("HDFS Minicluster service shut down.");
miniDfsCluster = null;
hadoopConf = null;
}
/**
* Configure the DFS Cluster before launching it.
*
* @param config The already created Hadoop configuration we'll further configure for HDFS
* @param localDFSLocation The location on the local filesystem where cluster data is stored
* @param bindIP An IP address we want to force the datanode and namenode to bind to.
* @return The updated Configuration object.
*/
private static Configuration configureDFSCluster(Configuration config, String localDFSLocation,
String bindIP, int namenodeRpcPort, int namenodeHttpPort, int datanodePort,
int datanodeIpcPort, int datanodeHttpPort) {
/**
* Get the location on the local FS where we store the HDFS data.
*
* @param baseFsLocation The base location on the local filesystem we have write access to
* create dirs.
* @return The location for HDFS data.
*/
private static String getDFSLocation(String baseFsLocation) {
return baseFsLocation + Path.SEPARATOR + "dfs";
}
/**
* Returns true if we should format the DFS Cluster. We'll format if clean is
* true, or if the dfsFsLocation does not exist.
*
* @param localDFSLocation The location on the local FS to hold the HDFS metadata and block
* data
* @param clean Specifies if we want to start a clean cluster
* @return Returns true if we should format a DFSCluster, otherwise false
*/
private static boolean shouldFormatDFSCluster(String localDFSLocation, boolean clean) {
boolean format = true;
File f = new File(localDFSLocation);
if (f.exists() && f.isDirectory() && !clean) {
format = false;
}
return format;
}
/**
* Configure the DFS Cluster before launching it.
*
* @param config The already created Hadoop configuration we'll further configure
* for HDFS
* @param localDFSLocation The location on the local filesystem where cluster data is stored
* @param bindIP An IP address we want to force the datanode and namenode to bind
* to.
* @param namenodeRpcPort
* @param namenodeHttpPort
* @param datanodePort
* @param datanodeIpcPort
* @param datanodeHttpPort
* @return The updated Configuration object.
*/
private static Configuration configureDFSCluster(Configuration config, String localDFSLocation,
String bindIP, int namenodeRpcPort, int namenodeHttpPort, int datanodePort,
int datanodeIpcPort, int datanodeHttpPort) {
logger.info("HDFS force binding to ip: " + bindIP);
config.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + bindIP + ":" + namenodeRpcPort);
config.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, bindIP + ":" + datanodePort);
config.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, bindIP + ":" + datanodeIpcPort);
config.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, bindIP + ":" + datanodeHttpPort);
// When a datanode registers with the namenode, the Namenode do a hostname
// check of the datanode which will fail on OpenShift due to reverse DNS
// issues with the internal IP addresses. This config disables that check,
// and will allow a datanode to connect regardless.
config.setBoolean("dfs.namenode.datanode.registration.ip-hostname-check", false);
config.set("hdfs.minidfs.basedir", localDFSLocation);
// allow current user to impersonate others
String user = System.getProperty("user.name");
config.set("hadoop.proxyuser." + user + ".groups", "*");
config.set("hadoop.proxyuser." + user + ".hosts", "*");
return config;
}
logger.info("HDFS force binding to ip: " + bindIP);
config.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + bindIP + ":" + namenodeRpcPort);
config.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, bindIP + ":" + datanodePort);
config.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, bindIP + ":" + datanodeIpcPort);
config.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, bindIP + ":" + datanodeHttpPort);
// When a datanode registers with the namenode, the Namenode do a hostname
// check of the datanode which will fail on OpenShift due to reverse DNS
// issues with the internal IP addresses. This config disables that check,
// and will allow a datanode to connect regardless.
config.setBoolean("dfs.namenode.datanode.registration.ip-hostname-check", false);
config.set("hdfs.minidfs.basedir", localDFSLocation);
// allow current user to impersonate others
String user = System.getProperty("user.name");
config.set("hadoop.proxyuser." + user + ".groups", "*");
config.set("hadoop.proxyuser." + user + ".hosts", "*");
return config;
}
}

View File

@@ -16,38 +16,38 @@
package com.uber.hoodie.common.minicluster;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.zookeeper.server.ZooKeeperServer;
import java.io.IOException;
public class MiniClusterUtil {
private static MiniDFSCluster dfsCluster;
private static ZooKeeperServer zkServer;
public static Configuration configuration;
public static FileSystem fileSystem;
public static void setUp() throws IOException, InterruptedException {
if (dfsCluster == null) {
HdfsTestService service = new HdfsTestService();
dfsCluster = service.start(true);
configuration = service.getHadoopConf();
}
if (zkServer == null) {
ZookeeperTestService zkService = new ZookeeperTestService(configuration);
zkServer = zkService.start();
}
fileSystem = FileSystem.get(configuration);
}
private static MiniDFSCluster dfsCluster;
private static ZooKeeperServer zkServer;
public static Configuration configuration;
public static FileSystem fileSystem;
public static void shutdown() {
if (dfsCluster != null) {
dfsCluster.shutdown();
}
if (zkServer != null) {
zkServer.shutdown();
}
public static void setUp() throws IOException, InterruptedException {
if (dfsCluster == null) {
HdfsTestService service = new HdfsTestService();
dfsCluster = service.start(true);
configuration = service.getHadoopConf();
}
if (zkServer == null) {
ZookeeperTestService zkService = new ZookeeperTestService(configuration);
zkServer = zkService.start();
}
fileSystem = FileSystem.get(configuration);
}
public static void shutdown() {
if (dfsCluster != null) {
dfsCluster.shutdown();
}
if (zkServer != null) {
zkServer.shutdown();
}
}
}

View File

@@ -18,14 +18,6 @@ package com.uber.hoodie.common.minicluster;
import com.google.common.base.Preconditions;
import com.google.common.io.Files;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.zookeeper.server.NIOServerCnxnFactory;
import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.server.persistence.FileTxnLog;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
@@ -34,208 +26,212 @@ import java.io.OutputStream;
import java.io.Reader;
import java.net.InetSocketAddress;
import java.net.Socket;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.zookeeper.server.NIOServerCnxnFactory;
import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.server.persistence.FileTxnLog;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A Zookeeper minicluster service implementation.
* <p/>
* This class was ripped from MiniZooKeeperCluster from the HBase tests. Changes
* made include:
* This class was ripped from MiniZooKeeperCluster from the HBase tests. Changes made include:
* <p/>
* 1. It will now only launch 1 zookeeper server.
* <p/>
* 2. It will only attempt to bind to the port specified, and will fail if it
* can't.
* 2. It will only attempt to bind to the port specified, and will fail if it can't.
* <p/>
* 3. The startup method now takes a bindAddress, which allows us to configure
* which IP the ZK server binds to. This was not configurable in the original
* class.
* 3. The startup method now takes a bindAddress, which allows us to configure which IP the ZK
* server binds to. This was not configurable in the original class.
* <p/>
* 4. The ZK cluster will re-use a data dir on the local filesystem if it
* already exists instead of blowing it away.
* 4. The ZK cluster will re-use a data dir on the local filesystem if it already exists instead of
* blowing it away.
*/
public class ZookeeperTestService {
private static final Logger logger = LoggerFactory.getLogger(ZookeeperTestService.class);
private static final Logger logger = LoggerFactory.getLogger(ZookeeperTestService.class);
private static final int TICK_TIME = 2000;
private static final int CONNECTION_TIMEOUT = 30000;
private static final int TICK_TIME = 2000;
private static final int CONNECTION_TIMEOUT = 30000;
/**
* Configuration settings
*/
private Configuration hadoopConf;
private String workDir;
private Integer clientPort = 2828;
private String bindIP = "127.0.0.1";
private Boolean clean = false;
private int tickTime = 0;
/**
* Configuration settings
*/
private Configuration hadoopConf;
private String workDir;
private Integer clientPort = 2828;
private String bindIP = "127.0.0.1";
private Boolean clean = false;
private int tickTime = 0;
/**
* Embedded ZooKeeper cluster
*/
private NIOServerCnxnFactory standaloneServerFactory;
private ZooKeeperServer zooKeeperServer;
private boolean started = false;
/**
* Embedded ZooKeeper cluster
*/
private NIOServerCnxnFactory standaloneServerFactory;
private ZooKeeperServer zooKeeperServer;
private boolean started = false;
public ZookeeperTestService(Configuration config) {
this.workDir = Files.createTempDir().getAbsolutePath();
this.hadoopConf = config;
public ZookeeperTestService(Configuration config) {
this.workDir = Files.createTempDir().getAbsolutePath();
this.hadoopConf = config;
}
public Configuration getHadoopConf() {
return hadoopConf;
}
public ZooKeeperServer start() throws IOException, InterruptedException {
Preconditions.checkState(workDir != null,
"The localBaseFsLocation must be set before starting cluster.");
setupTestEnv();
stop();
File dir = new File(workDir, "zookeeper").getAbsoluteFile();
recreateDir(dir, clean);
int tickTimeToUse;
if (this.tickTime > 0) {
tickTimeToUse = this.tickTime;
} else {
tickTimeToUse = TICK_TIME;
}
this.zooKeeperServer = new ZooKeeperServer(dir, dir, tickTimeToUse);
standaloneServerFactory = new NIOServerCnxnFactory();
// NOTE: Changed from the original, where InetSocketAddress was
// originally created to bind to the wildcard IP, we now configure it.
logger.info("Zookeeper force binding to: " + this.bindIP);
standaloneServerFactory.configure(new InetSocketAddress(bindIP, clientPort), 1000);
// Start up this ZK server
standaloneServerFactory.startup(zooKeeperServer);
String serverHostname;
if (bindIP.equals("0.0.0.0")) {
serverHostname = "localhost";
} else {
serverHostname = bindIP;
}
if (!waitForServerUp(serverHostname, clientPort, CONNECTION_TIMEOUT)) {
throw new IOException("Waiting for startup of standalone server");
}
public Configuration getHadoopConf() {
return hadoopConf;
started = true;
logger.info("Zookeeper Minicluster service started on client port: " + clientPort);
return zooKeeperServer;
}
public void stop() throws IOException {
if (!started) {
return;
}
public ZooKeeperServer start() throws IOException, InterruptedException {
Preconditions.checkState(workDir != null,
"The localBaseFsLocation must be set before starting cluster.");
setupTestEnv();
stop();
File dir = new File(workDir, "zookeeper").getAbsoluteFile();
recreateDir(dir, clean);
int tickTimeToUse;
if (this.tickTime > 0) {
tickTimeToUse = this.tickTime;
} else {
tickTimeToUse = TICK_TIME;
}
this.zooKeeperServer = new ZooKeeperServer(dir, dir, tickTimeToUse);
standaloneServerFactory = new NIOServerCnxnFactory();
// NOTE: Changed from the original, where InetSocketAddress was
// originally created to bind to the wildcard IP, we now configure it.
logger.info("Zookeeper force binding to: " + this.bindIP);
standaloneServerFactory.configure(new InetSocketAddress(bindIP, clientPort), 1000);
// Start up this ZK server
standaloneServerFactory.startup(zooKeeperServer);
String serverHostname;
if (bindIP.equals("0.0.0.0")) {
serverHostname = "localhost";
} else {
serverHostname = bindIP;
}
if (!waitForServerUp(serverHostname, clientPort, CONNECTION_TIMEOUT)) {
throw new IOException("Waiting for startup of standalone server");
}
started = true;
logger.info("Zookeeper Minicluster service started on client port: " + clientPort);
return zooKeeperServer;
standaloneServerFactory.shutdown();
if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
throw new IOException("Waiting for shutdown of standalone server");
}
public void stop() throws IOException {
if (!started) {
return;
}
// clear everything
started = false;
standaloneServerFactory = null;
zooKeeperServer = null;
standaloneServerFactory.shutdown();
if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
throw new IOException("Waiting for shutdown of standalone server");
}
logger.info("Zookeeper Minicluster service shut down.");
}
// clear everything
started = false;
standaloneServerFactory = null;
zooKeeperServer = null;
logger.info("Zookeeper Minicluster service shut down.");
private void recreateDir(File dir, boolean clean) throws IOException {
if (dir.exists() && clean) {
FileUtil.fullyDelete(dir);
} else if (dir.exists() && !clean) {
// the directory's exist, and we don't want to clean, so exit
return;
}
try {
dir.mkdirs();
} catch (SecurityException e) {
throw new IOException("creating dir: " + dir, e);
}
}
private void recreateDir(File dir, boolean clean) throws IOException {
if (dir.exists() && clean) {
FileUtil.fullyDelete(dir);
} else if (dir.exists() && !clean) {
// the directory's exist, and we don't want to clean, so exit
return;
}
// / XXX: From o.a.zk.t.ClientBase
private static void setupTestEnv() {
// during the tests we run with 100K prealloc in the logs.
// on windows systems prealloc of 64M was seen to take ~15seconds
// resulting in test failure (client timeout on first session).
// set env and directly in order to handle static init/gc issues
System.setProperty("zookeeper.preAllocSize", "100");
FileTxnLog.setPreallocSize(100 * 1024);
}
// XXX: From o.a.zk.t.ClientBase
private static boolean waitForServerDown(int port, long timeout) {
long start = System.currentTimeMillis();
while (true) {
try {
Socket sock = new Socket("localhost", port);
try {
dir.mkdirs();
} catch (SecurityException e) {
throw new IOException("creating dir: " + dir, e);
OutputStream outstream = sock.getOutputStream();
outstream.write("stat".getBytes());
outstream.flush();
} finally {
sock.close();
}
} catch (IOException e) {
return true;
}
if (System.currentTimeMillis() > start + timeout) {
break;
}
try {
Thread.sleep(250);
} catch (InterruptedException e) {
// ignore
}
}
return false;
}
// / XXX: From o.a.zk.t.ClientBase
private static void setupTestEnv() {
// during the tests we run with 100K prealloc in the logs.
// on windows systems prealloc of 64M was seen to take ~15seconds
// resulting in test failure (client timeout on first session).
// set env and directly in order to handle static init/gc issues
System.setProperty("zookeeper.preAllocSize", "100");
FileTxnLog.setPreallocSize(100 * 1024);
}
// XXX: From o.a.zk.t.ClientBase
private static boolean waitForServerUp(String hostname, int port, long timeout) {
long start = System.currentTimeMillis();
while (true) {
try {
Socket sock = new Socket(hostname, port);
BufferedReader reader = null;
try {
OutputStream outstream = sock.getOutputStream();
outstream.write("stat".getBytes());
outstream.flush();
// XXX: From o.a.zk.t.ClientBase
private static boolean waitForServerDown(int port, long timeout) {
long start = System.currentTimeMillis();
while (true) {
try {
Socket sock = new Socket("localhost", port);
try {
OutputStream outstream = sock.getOutputStream();
outstream.write("stat".getBytes());
outstream.flush();
} finally {
sock.close();
}
} catch (IOException e) {
return true;
}
if (System.currentTimeMillis() > start + timeout) {
break;
}
try {
Thread.sleep(250);
} catch (InterruptedException e) {
// ignore
}
Reader isr = new InputStreamReader(sock.getInputStream());
reader = new BufferedReader(isr);
String line = reader.readLine();
if (line != null && line.startsWith("Zookeeper version:")) {
return true;
}
} finally {
sock.close();
if (reader != null) {
reader.close();
}
}
return false;
}
// XXX: From o.a.zk.t.ClientBase
private static boolean waitForServerUp(String hostname, int port, long timeout) {
long start = System.currentTimeMillis();
while (true) {
try {
Socket sock = new Socket(hostname, port);
BufferedReader reader = null;
try {
OutputStream outstream = sock.getOutputStream();
outstream.write("stat".getBytes());
outstream.flush();
Reader isr = new InputStreamReader(sock.getInputStream());
reader = new BufferedReader(isr);
String line = reader.readLine();
if (line != null && line.startsWith("Zookeeper version:")) {
return true;
}
} finally {
sock.close();
if (reader != null) {
reader.close();
}
}
} catch (IOException e) {
// ignore as this is expected
logger.info("server " + hostname + ":" + port + " not up " + e);
}
if (System.currentTimeMillis() > start + timeout) {
break;
}
try {
Thread.sleep(250);
} catch (InterruptedException e) {
// ignore
}
}
return false;
} catch (IOException e) {
// ignore as this is expected
logger.info("server " + hostname + ":" + port + " not up " + e);
}
if (System.currentTimeMillis() > start + timeout) {
break;
}
try {
Thread.sleep(250);
} catch (InterruptedException e) {
// ignore
}
}
return false;
}
}

View File

@@ -16,6 +16,9 @@
package com.uber.hoodie.common.model;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
@@ -34,18 +37,6 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.util.StringUtils;
import org.junit.rules.TemporaryFolder;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;
@@ -64,224 +55,262 @@ import java.util.Random;
import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.util.StringUtils;
import org.junit.rules.TemporaryFolder;
public class HoodieTestUtils {
public static FileSystem fs = FSUtils.getFs();
public static final String TEST_EXTENSION = ".test";
public static final String RAW_TRIPS_TEST_NAME = "raw_trips";
public static final int DEFAULT_TASK_PARTITIONID = 1;
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
private static Random rand = new Random(46474747);
public static void resetFS() {
HoodieTestUtils.fs = FSUtils.getFs();
public static FileSystem fs = FSUtils.getFs();
public static final String TEST_EXTENSION = ".test";
public static final String RAW_TRIPS_TEST_NAME = "raw_trips";
public static final int DEFAULT_TASK_PARTITIONID = 1;
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
private static Random rand = new Random(46474747);
public static void resetFS() {
HoodieTestUtils.fs = FSUtils.getFs();
}
public static HoodieTableMetaClient init(String basePath) throws IOException {
return initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
}
public static HoodieTableMetaClient initTableType(String basePath, HoodieTableType tableType)
throws IOException {
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME,
HoodieAvroPayload.class.getName());
return HoodieTableMetaClient.initializePathAsHoodieDataset(fs, basePath, properties);
}
public static HoodieTableMetaClient initOnTemp() throws IOException {
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
String basePath = folder.getRoot().getAbsolutePath();
return HoodieTestUtils.init(basePath);
}
public static String makeNewCommitTime() {
return new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
}
public static final void createCommitFiles(String basePath, String... commitTimes)
throws IOException {
for (String commitTime : commitTimes) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeCommitFileName(commitTime)).createNewFile();
}
}
public static HoodieTableMetaClient init(String basePath) throws IOException {
return initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
public static final void createInflightCommitFiles(String basePath, String... commitTimes)
throws IOException {
for (String commitTime : commitTimes) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeInflightCommitFileName(commitTime)).createNewFile();
}
}
public static HoodieTableMetaClient initTableType(String basePath, HoodieTableType tableType) throws IOException {
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, HoodieAvroPayload.class.getName());
return HoodieTableMetaClient.initializePathAsHoodieDataset(fs, basePath, properties);
public static final String createNewDataFile(String basePath, String partitionPath,
String commitTime) throws IOException {
String fileID = UUID.randomUUID().toString();
return createDataFile(basePath, partitionPath, commitTime, fileID);
}
public static final String createDataFile(String basePath, String partitionPath,
String commitTime, String fileID) throws IOException {
String folderPath = basePath + "/" + partitionPath + "/";
new File(folderPath).mkdirs();
new File(folderPath + FSUtils.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID))
.createNewFile();
return fileID;
}
public static final String createNewLogFile(String basePath, String partitionPath,
String commitTime, String fileID, Optional<Integer> version) throws IOException {
String folderPath = basePath + "/" + partitionPath + "/";
boolean makeDir = fs.mkdirs(new Path(folderPath));
if (!makeDir) {
throw new IOException("cannot create directory for path " + folderPath);
}
public static HoodieTableMetaClient initOnTemp() throws IOException {
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
String basePath = folder.getRoot().getAbsolutePath();
return HoodieTestUtils.init(basePath);
boolean createFile = fs.createNewFile(new Path(folderPath + FSUtils
.makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_TASK_PARTITIONID))));
if (!createFile) {
throw new IOException(StringUtils
.format("cannot create data file for commit %s and fileId %s", commitTime, fileID));
}
return fileID;
}
public static String makeNewCommitTime() {
return new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
public static final void createCompactionCommitFiles(String basePath, String... commitTimes)
throws IOException {
for (String commitTime : commitTimes) {
boolean createFile = fs.createNewFile(new Path(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeCompactionFileName(commitTime)));
if (!createFile) {
throw new IOException("cannot create commit file for commit " + commitTime);
}
}
}
public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException {
for (String commitTime: commitTimes) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTimeline.makeCommitFileName(commitTime)).createNewFile();
}
public static final String getDataFilePath(String basePath, String partitionPath,
String commitTime, String fileID) throws IOException {
return basePath + "/" + partitionPath + "/" + FSUtils
.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID);
}
public static final String getLogFilePath(String basePath, String partitionPath,
String commitTime, String fileID, Optional<Integer> version) throws IOException {
return basePath + "/" + partitionPath + "/" + FSUtils
.makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_TASK_PARTITIONID));
}
public static final String getCommitFilePath(String basePath, String commitTime)
throws IOException {
return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime
+ HoodieTimeline.COMMIT_EXTENSION;
}
public static final boolean doesDataFileExist(String basePath, String partitionPath,
String commitTime, String fileID) throws IOException {
return new File(getDataFilePath(basePath, partitionPath, commitTime, fileID)).exists();
}
public static final boolean doesLogFileExist(String basePath, String partitionPath,
String commitTime, String fileID, Optional<Integer> version) throws IOException {
return new File(getLogFilePath(basePath, partitionPath, commitTime, fileID, version)).exists();
}
public static final boolean doesCommitExist(String basePath, String commitTime) {
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime
+ HoodieTimeline.COMMIT_EXTENSION).exists();
}
public static final boolean doesInflightExist(String basePath, String commitTime) {
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime
+ HoodieTimeline.INFLIGHT_EXTENSION).exists();
}
public static String makeInflightTestFileName(String instant) {
return instant + TEST_EXTENSION + HoodieTimeline.INFLIGHT_EXTENSION;
}
public static void createCleanFiles(String basePath, String commitTime) throws IOException {
Path commitFile =
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeCleanerFileName(commitTime));
FileSystem fs = FSUtils.getFs();
FSDataOutputStream os = fs.create(commitFile, true);
try {
HoodieCleanStat cleanStats = new HoodieCleanStat(
HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS,
DEFAULT_PARTITION_PATHS[rand.nextInt(DEFAULT_PARTITION_PATHS.length)],
new ArrayList<>(), new ArrayList<>(),
new ArrayList<>(), commitTime);
// Create the clean metadata
HoodieCleanMetadata cleanMetadata =
AvroUtils.convertCleanMetadata(commitTime, Optional.of(0L), Arrays.asList(cleanStats));
// Write empty clean metadata
os.write(AvroUtils.serializeCleanMetadata(cleanMetadata).get());
} finally {
os.close();
}
}
public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
for (String commitTime: commitTimes) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTimeline.makeInflightCommitFileName(commitTime)).createNewFile();
}
public static String makeTestFileName(String instant) {
return instant + TEST_EXTENSION;
}
public static String makeCommitFileName(String instant) {
return instant + ".commit";
}
public static void assertStreamEquals(String message, Stream<?> expected, Stream<?> actual) {
Iterator<?> iter1 = expected.iterator(), iter2 = actual.iterator();
while (iter1.hasNext() && iter2.hasNext()) {
assertEquals(message, iter1.next(), iter2.next());
}
assert !iter1.hasNext() && !iter2.hasNext();
}
public static final String createNewDataFile(String basePath, String partitionPath, String commitTime) throws IOException {
String fileID = UUID.randomUUID().toString();
return createDataFile(basePath, partitionPath, commitTime, fileID);
}
public static final String createDataFile(String basePath, String partitionPath, String commitTime, String fileID) throws IOException {
String folderPath = basePath + "/" + partitionPath + "/";
new File(folderPath).mkdirs();
new File(folderPath + FSUtils.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID)).createNewFile();
return fileID;
}
public static final String createNewLogFile(String basePath, String partitionPath, String commitTime, String fileID, Optional<Integer> version) throws IOException {
String folderPath = basePath + "/" + partitionPath + "/";
boolean makeDir = fs.mkdirs(new Path(folderPath));
if(!makeDir) {
throw new IOException("cannot create directory for path " + folderPath);
}
boolean createFile = fs.createNewFile(new Path(folderPath + FSUtils.makeLogFileName(fileID, ".log",commitTime, version.orElse(DEFAULT_TASK_PARTITIONID))));
if(!createFile) {
throw new IOException(StringUtils.format("cannot create data file for commit %s and fileId %s", commitTime, fileID));
}
return fileID;
}
public static final void createCompactionCommitFiles(String basePath, String... commitTimes) throws IOException {
for (String commitTime: commitTimes) {
boolean createFile = fs.createNewFile(new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTimeline.makeCompactionFileName(commitTime)));
if(!createFile) {
throw new IOException("cannot create commit file for commit " + commitTime);
}
}
}
public static final String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID) throws IOException {
return basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID);
}
public static final String getLogFilePath(String basePath, String partitionPath, String commitTime, String fileID, Optional<Integer> version) throws IOException {
return basePath + "/" + partitionPath + "/" + FSUtils.makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_TASK_PARTITIONID));
}
public static final String getCommitFilePath(String basePath, String commitTime) throws IOException {
return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION;
}
public static final boolean doesDataFileExist(String basePath, String partitionPath, String commitTime, String fileID) throws IOException {
return new File(getDataFilePath(basePath, partitionPath, commitTime, fileID)).exists();
}
public static final boolean doesLogFileExist(String basePath, String partitionPath, String commitTime, String fileID, Optional<Integer> version) throws IOException {
return new File(getLogFilePath(basePath, partitionPath, commitTime, fileID, version)).exists();
}
public static final boolean doesCommitExist(String basePath, String commitTime) {
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION).exists();
}
public static final boolean doesInflightExist(String basePath, String commitTime) {
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTimeline.INFLIGHT_EXTENSION).exists();
}
public static String makeInflightTestFileName(String instant) {
return instant + TEST_EXTENSION + HoodieTimeline.INFLIGHT_EXTENSION;
}
public static void createCleanFiles(String basePath, String commitTime) throws IOException {
Path commitFile =
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime));
FileSystem fs = FSUtils.getFs();
FSDataOutputStream os = fs.create(commitFile, true);
try {
HoodieCleanStat cleanStats = new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS,
DEFAULT_PARTITION_PATHS[rand.nextInt(DEFAULT_PARTITION_PATHS.length)],
new ArrayList<>(), new ArrayList<>(),
new ArrayList<>(), commitTime);
// Create the clean metadata
HoodieCleanMetadata cleanMetadata =
AvroUtils.convertCleanMetadata(commitTime, Optional.of(0L), Arrays.asList(cleanStats));
// Write empty clean metadata
os.write(AvroUtils.serializeCleanMetadata(cleanMetadata).get());
} finally {
os.close();
}
}
public static String makeTestFileName(String instant) {
return instant + TEST_EXTENSION;
}
public static String makeCommitFileName(String instant) {
return instant + ".commit";
}
public static void assertStreamEquals(String message, Stream<?> expected, Stream<?> actual) {
Iterator<?> iter1 = expected.iterator(), iter2 = actual.iterator();
while(iter1.hasNext() && iter2.hasNext())
assertEquals(message, iter1.next(), iter2.next());
assert !iter1.hasNext() && !iter2.hasNext();
}
public static <T extends Serializable> T serializeDeserialize(T object, Class<T> clazz)
throws IOException, ClassNotFoundException {
// Using Kyro as the default serializer in Spark Jobs
Kryo kryo = new Kryo();
kryo.register(HoodieTableMetaClient.class, new JavaSerializer());
ByteArrayOutputStream baos = new ByteArrayOutputStream();
Output output = new Output(baos);
kryo.writeObject(output, object);
output.close();
Input input = new Input(new ByteArrayInputStream(baos.toByteArray()));
T deseralizedObject = kryo.readObject(input, clazz);
input.close();
return deseralizedObject;
}
public static void writeRecordsToLogFiles(String basePath, Schema schema, List<HoodieRecord> updatedRecords) {
Map<HoodieRecordLocation, List<HoodieRecord>> groupedUpdated = updatedRecords.stream()
.collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation));
groupedUpdated.entrySet().forEach(s -> {
HoodieRecordLocation location = s.getKey();
String partitionPath = s.getValue().get(0).getPartitionPath();
Writer logWriter;
try {
logWriter = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(basePath, partitionPath))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId(location.getFileId())
.overBaseCommit(location.getCommitTime())
.withFs(fs).build();
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, location.getCommitTime());
logWriter.appendBlock(new HoodieAvroDataBlock(s.getValue().stream().map(r -> {
try {
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get();
HoodieAvroUtils.addHoodieKeyToRecord(val,
r.getRecordKey(),
r.getPartitionPath(),
"");
return (IndexedRecord) val;
} catch (IOException e) {
return null;
}
}).collect(Collectors.toList()), schema, metadata));
logWriter.close();
} catch (Exception e) {
fail(e.toString());
}
});
}
public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath)
throws IOException {
RemoteIterator<LocatedFileStatus> itr = fs.listFiles(new Path(basePath), true);
List<FileStatus> returns = Lists.newArrayList();
while(itr.hasNext()) {
LocatedFileStatus status = itr.next();
if(status.getPath().getName().contains(".parquet")) {
returns.add(status);
}
}
return returns.toArray(new FileStatus[returns.size()]);
public static <T extends Serializable> T serializeDeserialize(T object, Class<T> clazz)
throws IOException, ClassNotFoundException {
// Using Kyro as the default serializer in Spark Jobs
Kryo kryo = new Kryo();
kryo.register(HoodieTableMetaClient.class, new JavaSerializer());
ByteArrayOutputStream baos = new ByteArrayOutputStream();
Output output = new Output(baos);
kryo.writeObject(output, object);
output.close();
Input input = new Input(new ByteArrayInputStream(baos.toByteArray()));
T deseralizedObject = kryo.readObject(input, clazz);
input.close();
return deseralizedObject;
}
public static void writeRecordsToLogFiles(String basePath, Schema schema,
List<HoodieRecord> updatedRecords) {
Map<HoodieRecordLocation, List<HoodieRecord>> groupedUpdated = updatedRecords.stream()
.collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation));
groupedUpdated.entrySet().forEach(s -> {
HoodieRecordLocation location = s.getKey();
String partitionPath = s.getValue().get(0).getPartitionPath();
Writer logWriter;
try {
logWriter = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(basePath, partitionPath))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId(location.getFileId())
.overBaseCommit(location.getCommitTime())
.withFs(fs).build();
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, location.getCommitTime());
logWriter.appendBlock(new HoodieAvroDataBlock(s.getValue().stream().map(r -> {
try {
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get();
HoodieAvroUtils.addHoodieKeyToRecord(val,
r.getRecordKey(),
r.getPartitionPath(),
"");
return (IndexedRecord) val;
} catch (IOException e) {
return null;
}
}).collect(Collectors.toList()), schema, metadata));
logWriter.close();
} catch (Exception e) {
fail(e.toString());
}
});
}
public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath)
throws IOException {
RemoteIterator<LocatedFileStatus> itr = fs.listFiles(new Path(basePath), true);
List<FileStatus> returns = Lists.newArrayList();
while (itr.hasNext()) {
LocatedFileStatus status = itr.next();
if (status.getPath().getName().contains(".parquet")) {
returns.add(status);
}
}
return returns.toArray(new FileStatus[returns.size()]);
}
}

View File

@@ -16,11 +16,20 @@
package com.uber.hoodie.common.table;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.Lists;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.SequenceFile;
@@ -29,116 +38,106 @@ import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Collectors;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
public class HoodieTableMetaClientTest {
private HoodieTableMetaClient metaClient;
private String basePath;
@Before
public void init() throws IOException {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
metaClient = HoodieTestUtils.init(basePath);
}
private HoodieTableMetaClient metaClient;
private String basePath;
@Test
public void checkMetadata() {
assertEquals("Table name should be raw_trips", HoodieTestUtils.RAW_TRIPS_TEST_NAME,
metaClient.getTableConfig().getTableName());
assertEquals("Basepath should be the one assigned", basePath, metaClient.getBasePath());
assertEquals("Metapath should be ${basepath}/.hoodie", basePath + "/.hoodie",
metaClient.getMetaPath());
}
@Before
public void init() throws IOException {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
metaClient = HoodieTestUtils.init(basePath);
}
@Test
public void checkSerDe() throws IOException, ClassNotFoundException {
// check if this object is serialized and de-serialized, we are able to read from the file system
HoodieTableMetaClient deseralizedMetaClient =
HoodieTestUtils.serializeDeserialize(metaClient, HoodieTableMetaClient.class);
assertNotNull(deseralizedMetaClient);
HoodieActiveTimeline commitTimeline = deseralizedMetaClient.getActiveTimeline();
HoodieInstant instant =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
commitTimeline.createInflight(instant);
commitTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes()));
commitTimeline = commitTimeline.reload();
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
assertEquals("Commit should be 1 and completed", completedInstant,
commitTimeline.getInstants().findFirst().get());
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
commitTimeline.getInstantDetails(completedInstant).get());
}
@Test
public void checkMetadata() {
assertEquals("Table name should be raw_trips", HoodieTestUtils.RAW_TRIPS_TEST_NAME,
metaClient.getTableConfig().getTableName());
assertEquals("Basepath should be the one assigned", basePath, metaClient.getBasePath());
assertEquals("Metapath should be ${basepath}/.hoodie", basePath + "/.hoodie",
metaClient.getMetaPath());
}
@Test
public void checkCommitTimeline() throws IOException {
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
HoodieTimeline activeCommitTimeline = activeTimeline.getCommitTimeline();
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
@Test
public void checkSerDe() throws IOException, ClassNotFoundException {
// check if this object is serialized and de-serialized, we are able to read from the file system
HoodieTableMetaClient deseralizedMetaClient =
HoodieTestUtils.serializeDeserialize(metaClient, HoodieTableMetaClient.class);
assertNotNull(deseralizedMetaClient);
HoodieActiveTimeline commitTimeline = deseralizedMetaClient.getActiveTimeline();
HoodieInstant instant =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
commitTimeline.createInflight(instant);
commitTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes()));
commitTimeline = commitTimeline.reload();
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
assertEquals("Commit should be 1 and completed", completedInstant,
commitTimeline.getInstants().findFirst().get());
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
commitTimeline.getInstantDetails(completedInstant).get());
}
HoodieInstant instant =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
activeTimeline.createInflight(instant);
activeTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes()));
@Test
public void checkCommitTimeline() throws IOException {
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
HoodieTimeline activeCommitTimeline = activeTimeline.getCommitTimeline();
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
// Commit timeline should not auto-reload every time getActiveCommitTimeline(), it should be cached
activeTimeline = metaClient.getActiveTimeline();
activeCommitTimeline = activeTimeline.getCommitTimeline();
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
HoodieInstant instant =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
activeTimeline.createInflight(instant);
activeTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes()));
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
activeTimeline = activeTimeline.reload();
activeCommitTimeline = activeTimeline.getCommitTimeline();
assertFalse("Should be the 1 commit we made", activeCommitTimeline.empty());
assertEquals("Commit should be 1", completedInstant,
activeCommitTimeline.getInstants().findFirst().get());
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
activeCommitTimeline.getInstantDetails(completedInstant).get());
}
// Commit timeline should not auto-reload every time getActiveCommitTimeline(), it should be cached
activeTimeline = metaClient.getActiveTimeline();
activeCommitTimeline = activeTimeline.getCommitTimeline();
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
@Test
public void checkArchiveCommitTimeline() throws IOException {
Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getMetaPath());
SequenceFile.Writer writer = SequenceFile
.createWriter(HoodieTestUtils.fs.getConf(), SequenceFile.Writer.file(archiveLogPath),
SequenceFile.Writer.keyClass(Text.class),
SequenceFile.Writer.valueClass(Text.class));
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
activeTimeline = activeTimeline.reload();
activeCommitTimeline = activeTimeline.getCommitTimeline();
assertFalse("Should be the 1 commit we made", activeCommitTimeline.empty());
assertEquals("Commit should be 1", completedInstant,
activeCommitTimeline.getInstants().findFirst().get());
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
activeCommitTimeline.getInstantDetails(completedInstant).get());
}
writer.append(new Text("1"), new Text("data1"));
writer.append(new Text("2"), new Text("data2"));
writer.append(new Text("3"), new Text("data3"));
@Test
public void checkArchiveCommitTimeline() throws IOException {
Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getMetaPath());
SequenceFile.Writer writer = SequenceFile
.createWriter(HoodieTestUtils.fs.getConf(), SequenceFile.Writer.file(archiveLogPath),
SequenceFile.Writer.keyClass(Text.class),
SequenceFile.Writer.valueClass(Text.class));
IOUtils.closeStream(writer);
writer.append(new Text("1"), new Text("data1"));
writer.append(new Text("2"), new Text("data2"));
writer.append(new Text("3"), new Text("data3"));
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
IOUtils.closeStream(writer);
HoodieInstant instant1 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
HoodieInstant instant2 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2");
HoodieInstant instant3 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
assertEquals(Lists.newArrayList(instant1, instant2, instant3),
archivedTimeline.getInstants().collect(Collectors.toList()));
HoodieInstant instant1 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
HoodieInstant instant2 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2");
HoodieInstant instant3 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
assertArrayEquals(new Text("data1").getBytes(),
archivedTimeline.getInstantDetails(instant1).get());
assertArrayEquals(new Text("data2").getBytes(),
archivedTimeline.getInstantDetails(instant2).get());
assertArrayEquals(new Text("data3").getBytes(),
archivedTimeline.getInstantDetails(instant3).get());
}
assertEquals(Lists.newArrayList(instant1, instant2, instant3),
archivedTimeline.getInstants().collect(Collectors.toList()));
assertArrayEquals(new Text("data1").getBytes(),
archivedTimeline.getInstantDetails(instant1).get());
assertArrayEquals(new Text("data2").getBytes(),
archivedTimeline.getInstantDetails(instant2).get());
assertArrayEquals(new Text("data3").getBytes(),
archivedTimeline.getInstantDetails(instant3).get());
}
}

View File

@@ -16,13 +16,18 @@
package com.uber.hoodie.common.table.log;
import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.minicluster.MiniClusterUtil;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Reader;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
@@ -35,6 +40,15 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -48,22 +62,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@SuppressWarnings("Duplicates")
public class HoodieLogFormatTest {
@@ -140,7 +138,7 @@ public class HoodieLogFormatTest {
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema(), metadata);
getSimpleSchema(), metadata);
// Write out a block
writer = writer.appendBlock(dataBlock);
// Get the size of the block
@@ -170,7 +168,7 @@ public class HoodieLogFormatTest {
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema(), metadata);
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
writer.close();
@@ -222,7 +220,7 @@ public class HoodieLogFormatTest {
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema(), metadata);
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
// do not close this writer - this simulates a data note appending to a log dying without closing the file
@@ -254,11 +252,12 @@ public class HoodieLogFormatTest {
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema(), metadata);
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
Reader reader = HoodieLogFormat
.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("We wrote a block, we should be able to read it", reader.hasNext());
HoodieLogBlock nextBlock = reader.next();
assertEquals("The next block should be a data block", HoodieLogBlockType.AVRO_DATA_BLOCK,
@@ -281,7 +280,7 @@ public class HoodieLogFormatTest {
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
getSimpleSchema(), metadata);
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -304,7 +303,8 @@ public class HoodieLogFormatTest {
writer = writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
Reader reader = HoodieLogFormat
.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("First block should be available", reader.hasNext());
HoodieLogBlock nextBlock = reader.next();
HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock;
@@ -338,7 +338,7 @@ public class HoodieLogFormatTest {
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema(), metadata);
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -358,7 +358,8 @@ public class HoodieLogFormatTest {
outputStream.close();
// First round of reads - we should be able to read the first block and then EOF
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
Reader reader = HoodieLogFormat
.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("First block should be available", reader.hasNext());
reader.next();
assertTrue("We should have corrupted block next", reader.hasNext());
@@ -393,7 +394,8 @@ public class HoodieLogFormatTest {
writer.close();
// Second round of reads - we should be able to read the first and last block
reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
reader = HoodieLogFormat
.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("First block should be available", reader.hasNext());
reader.next();
assertTrue("We should get the 1st corrupted block next", reader.hasNext());
@@ -424,7 +426,7 @@ public class HoodieLogFormatTest {
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
@@ -438,7 +440,8 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles,
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "100");
assertEquals("", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
@@ -469,7 +472,7 @@ public class HoodieLogFormatTest {
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
@@ -493,9 +496,11 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles,
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "100");
assertEquals("We only read 200 records, but only 200 of them are valid", 200, scanner.getTotalLogRecords());
assertEquals("We only read 200 records, but only 200 of them are valid", 200,
scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records", 200, readKeys.size());
@@ -523,7 +528,7 @@ public class HoodieLogFormatTest {
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
schema, metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -561,7 +566,8 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles,
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "100");
assertEquals("We would read 200 records", 200,
scanner.getTotalLogRecords());
@@ -592,7 +598,7 @@ public class HoodieLogFormatTest {
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
@@ -609,7 +615,8 @@ public class HoodieLogFormatTest {
// Delete 50 keys
List<String> deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), metadata);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]),
metadata);
writer = writer.appendBlock(deleteBlock);
List<String> allLogFiles = FSUtils
@@ -617,7 +624,8 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles,
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "100");
assertEquals("We still would read 200 records", 200,
scanner.getTotalLogRecords());
@@ -632,25 +640,26 @@ public class HoodieLogFormatTest {
// Rollback the last block
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
readKeys.clear();
scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "100");
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size());
assertEquals("Stream collect should return all 200 records after rollback of delete", 200,
readKeys.size());
}
@Test
public void testAvroLogRecordReaderWithFailedRollbacks()
throws IOException, URISyntaxException, InterruptedException {
throws IOException, URISyntaxException, InterruptedException {
// Write a Data block and Delete block with same InstantTime (written in same batch)
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
@@ -658,7 +667,7 @@ public class HoodieLogFormatTest {
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
@@ -667,23 +676,24 @@ public class HoodieLogFormatTest {
writer = writer.appendBlock(dataBlock);
List<String> originalKeys = records1.stream()
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toList());
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toList());
// Delete 50 keys
List<String> deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), metadata);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]),
metadata);
writer = writer.appendBlock(deleteBlock);
// Attemp 1 : Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
try {
writer = writer.appendBlock(commandBlock);
// Say job failed, retry writing 2 rollback in the next rollback(..) attempt
throw new Exception("simulating failure");
} catch(Exception e) {
} catch (Exception e) {
// it's okay
}
// Attempt 2 : Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
@@ -691,14 +701,15 @@ public class HoodieLogFormatTest {
writer = writer.appendBlock(commandBlock);
List<String> allLogFiles = FSUtils
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles,
schema, "100");
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles,
schema, "100");
assertEquals("We would read 100 records", 100,
scanner.getTotalLogRecords());
scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(100);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -707,14 +718,14 @@ public class HoodieLogFormatTest {
@Test
public void testAvroLogRecordReaderWithInsertDeleteAndRollback()
throws IOException, URISyntaxException, InterruptedException {
throws IOException, URISyntaxException, InterruptedException {
// Write a Data block and Delete block with same InstantTime (written in same batch)
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
@@ -722,43 +733,45 @@ public class HoodieLogFormatTest {
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
schema, metadata);
writer = writer.appendBlock(dataBlock);
List<String> originalKeys = records1.stream()
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toList());
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toList());
// Delete 50 keys
List<String> deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), metadata);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]),
metadata);
writer = writer.appendBlock(deleteBlock);
// Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
writer = writer.appendBlock(commandBlock);
List<String> allLogFiles = FSUtils
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles, schema, "100");
allLogFiles, schema, "100");
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
scanner.getTotalLogRecords());
}
@Test
public void testAvroLogRecordReaderWithInvalidRollback() throws IOException, URISyntaxException, InterruptedException {
public void testAvroLogRecordReaderWithInvalidRollback()
throws IOException, URISyntaxException, InterruptedException {
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
@@ -766,23 +779,23 @@ public class HoodieLogFormatTest {
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write invalid rollback for a failed write (possible for in-flight commits)
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
List<String> allLogFiles = FSUtils
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
allLogFiles, schema, "100");
allLogFiles, schema, "100");
assertEquals("We still would read 100 records", 100,
scanner.getTotalLogRecords());
scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(100);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 150 records", 100, readKeys.size());

View File

@@ -16,11 +16,18 @@
package com.uber.hoodie.common.table.string;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Stream;
import org.apache.hadoop.fs.Path;
import org.junit.After;
import org.junit.Before;
@@ -28,109 +35,104 @@ import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Stream;
import static org.junit.Assert.*;
public class HoodieActiveTimelineTest {
private HoodieActiveTimeline timeline;
private HoodieTableMetaClient metaClient;
@Rule
public final ExpectedException exception = ExpectedException.none();
@Before
public void setUp() throws Exception {
this.metaClient = HoodieTestUtils.initOnTemp();
}
private HoodieActiveTimeline timeline;
private HoodieTableMetaClient metaClient;
@Rule
public final ExpectedException exception = ExpectedException.none();
@After
public void tearDown() throws Exception {
HoodieTestUtils.fs.delete(new Path(this.metaClient.getBasePath()), true);
}
@Before
public void setUp() throws Exception {
this.metaClient = HoodieTestUtils.initOnTemp();
}
@Test
public void testLoadingInstantsFromFiles() throws IOException {
HoodieInstant instant1 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
HoodieInstant instant2 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "3");
HoodieInstant instant3 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5");
HoodieInstant instant4 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "8");
HoodieInstant instant1_complete =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
HoodieInstant instant2_complete =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
HoodieInstant instant3_complete =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5");
HoodieInstant instant4_complete =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "8");
@After
public void tearDown() throws Exception {
HoodieTestUtils.fs.delete(new Path(this.metaClient.getBasePath()), true);
}
HoodieInstant instant5 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9");
@Test
public void testLoadingInstantsFromFiles() throws IOException {
HoodieInstant instant1 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
HoodieInstant instant2 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "3");
HoodieInstant instant3 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5");
HoodieInstant instant4 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "8");
HoodieInstant instant1_complete =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
HoodieInstant instant2_complete =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
HoodieInstant instant3_complete =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5");
HoodieInstant instant4_complete =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "8");
timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath());
timeline.saveAsComplete(instant1, Optional.empty());
timeline.saveAsComplete(instant2, Optional.empty());
timeline.saveAsComplete(instant3, Optional.empty());
timeline.saveAsComplete(instant4, Optional.empty());
timeline.createInflight(instant5);
timeline = timeline.reload();
HoodieInstant instant5 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9");
assertEquals("Total instants should be 5", 5, timeline.countInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream
.of(instant1_complete, instant2_complete, instant3_complete, instant4_complete,
instant5), timeline.getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream
.of(instant1_complete, instant2_complete, instant3_complete, instant4_complete,
instant5), timeline.getCommitTimeline().getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream",
Stream.of(instant1_complete, instant2_complete, instant3_complete, instant4_complete),
timeline.getCommitTimeline().filterCompletedInstants().getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream.of(instant5),
timeline.getCommitTimeline().filterInflights().getInstants());
}
timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath());
timeline.saveAsComplete(instant1, Optional.empty());
timeline.saveAsComplete(instant2, Optional.empty());
timeline.saveAsComplete(instant3, Optional.empty());
timeline.saveAsComplete(instant4, Optional.empty());
timeline.createInflight(instant5);
timeline = timeline.reload();
@Test
public void testTimelineOperationsBasic() throws Exception {
timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath());
assertTrue(timeline.empty());
assertEquals("", 0, timeline.countInstants());
assertEquals("", Optional.empty(), timeline.firstInstant());
assertEquals("", Optional.empty(), timeline.nthInstant(5));
assertEquals("", Optional.empty(), timeline.nthInstant(-1));
assertEquals("", Optional.empty(), timeline.lastInstant());
assertFalse("", timeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "01")));
}
assertEquals("Total instants should be 5", 5, timeline.countInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream
.of(instant1_complete, instant2_complete, instant3_complete, instant4_complete,
instant5), timeline.getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream
.of(instant1_complete, instant2_complete, instant3_complete, instant4_complete,
instant5), timeline.getCommitTimeline().getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream",
Stream.of(instant1_complete, instant2_complete, instant3_complete, instant4_complete),
timeline.getCommitTimeline().filterCompletedInstants().getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream.of(instant5),
timeline.getCommitTimeline().filterInflights().getInstants());
}
@Test
public void testTimelineOperations() throws Exception {
timeline = new MockHoodieTimeline(
Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"),
Stream.of("21", "23"));
HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"),
timeline.getCommitTimeline().filterCompletedInstants().findInstantsInRange("04", "11")
.getInstants().map(HoodieInstant::getTimestamp));
HoodieTestUtils.assertStreamEquals("", Stream.of("09", "11"),
timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2)
.getInstants().map(HoodieInstant::getTimestamp));
assertFalse(timeline.empty());
assertFalse(timeline.getCommitTimeline().filterInflights().empty());
assertEquals("", 12, timeline.countInstants());
HoodieTimeline activeCommitTimeline = timeline.getCommitTimeline().filterCompletedInstants();
assertEquals("", 10, activeCommitTimeline.countInstants());
@Test
public void testTimelineOperationsBasic() throws Exception {
timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath());
assertTrue(timeline.empty());
assertEquals("", 0, timeline.countInstants());
assertEquals("", Optional.empty(), timeline.firstInstant());
assertEquals("", Optional.empty(), timeline.nthInstant(5));
assertEquals("", Optional.empty(), timeline.nthInstant(-1));
assertEquals("", Optional.empty(), timeline.lastInstant());
assertFalse("", timeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "01")));
}
assertEquals("", "01", activeCommitTimeline.firstInstant().get().getTimestamp());
assertEquals("", "11", activeCommitTimeline.nthInstant(5).get().getTimestamp());
assertEquals("", "19", activeCommitTimeline.lastInstant().get().getTimestamp());
assertEquals("", "09", activeCommitTimeline.nthFromLastInstant(5).get().getTimestamp());
assertTrue("", activeCommitTimeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "09")));
assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
}
@Test
public void testTimelineOperations() throws Exception {
timeline = new MockHoodieTimeline(
Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"),
Stream.of("21", "23"));
HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"),
timeline.getCommitTimeline().filterCompletedInstants().findInstantsInRange("04", "11")
.getInstants().map(HoodieInstant::getTimestamp));
HoodieTestUtils.assertStreamEquals("", Stream.of("09", "11"),
timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2)
.getInstants().map(HoodieInstant::getTimestamp));
assertFalse(timeline.empty());
assertFalse(timeline.getCommitTimeline().filterInflights().empty());
assertEquals("", 12, timeline.countInstants());
HoodieTimeline activeCommitTimeline = timeline.getCommitTimeline().filterCompletedInstants();
assertEquals("", 10, activeCommitTimeline.countInstants());
assertEquals("", "01", activeCommitTimeline.firstInstant().get().getTimestamp());
assertEquals("", "11", activeCommitTimeline.nthInstant(5).get().getTimestamp());
assertEquals("", "19", activeCommitTimeline.lastInstant().get().getTimestamp());
assertEquals("", "09", activeCommitTimeline.nthFromLastInstant(5).get().getTimestamp());
assertTrue("", activeCommitTimeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "09")));
assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
}
}

View File

@@ -19,7 +19,6 @@ package com.uber.hoodie.common.table.string;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import java.io.IOException;
import java.util.Comparator;
import java.util.function.Function;
@@ -27,18 +26,19 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
public class MockHoodieTimeline extends HoodieActiveTimeline {
public MockHoodieTimeline(Stream<String> completed, Stream<String> inflights)
throws IOException {
super();
this.instants = Stream.concat(completed
.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)),
inflights.map(
s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s)))
.sorted(Comparator.comparing(new Function<HoodieInstant, String>() {
@Override
public String apply(HoodieInstant hoodieInstant) {
return hoodieInstant.getFileName();
}
})).collect(Collectors.toList());
}
public MockHoodieTimeline(Stream<String> completed, Stream<String> inflights)
throws IOException {
super();
this.instants = Stream.concat(completed
.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)),
inflights.map(
s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s)))
.sorted(Comparator.comparing(new Function<HoodieInstant, String>() {
@Override
public String apply(HoodieInstant hoodieInstant) {
return hoodieInstant.getFileName();
}
})).collect(Collectors.toList());
}
}

View File

@@ -16,9 +16,12 @@
package com.uber.hoodie.common.table.view;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieFileGroup;
@@ -30,499 +33,501 @@ import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.Assert.*;
@SuppressWarnings("ResultOfMethodCallIgnored")
public class HoodieTableFileSystemViewTest {
private HoodieTableMetaClient metaClient;
private String basePath;
private TableFileSystemView fsView;
private TableFileSystemView.ReadOptimizedView roView;
private TableFileSystemView.RealtimeView rtView;
@Before
public void init() throws IOException {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
metaClient = HoodieTestUtils.init(basePath);
fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
roView = (TableFileSystemView.ReadOptimizedView) fsView;
rtView = (TableFileSystemView.RealtimeView) fsView;
private HoodieTableMetaClient metaClient;
private String basePath;
private TableFileSystemView fsView;
private TableFileSystemView.ReadOptimizedView roView;
private TableFileSystemView.RealtimeView rtView;
@Before
public void init() throws IOException {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
metaClient = HoodieTestUtils.init(basePath);
fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
roView = (TableFileSystemView.ReadOptimizedView) fsView;
rtView = (TableFileSystemView.RealtimeView) fsView;
}
private void refreshFsView(FileStatus[] statuses) {
metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs, basePath, true);
if (statuses != null) {
fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(),
statuses);
} else {
fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
}
roView = (TableFileSystemView.ReadOptimizedView) fsView;
rtView = (TableFileSystemView.RealtimeView) fsView;
}
private void refreshFsView(FileStatus[] statuses) {
metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs, basePath, true);
if (statuses != null) {
fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(),
statuses);
} else {
fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
}
roView = (TableFileSystemView.ReadOptimizedView) fsView;
rtView = (TableFileSystemView.RealtimeView) fsView;
}
@Test
public void testGetLatestDataFilesForFileId() throws IOException {
String partitionPath = "2016/05/01";
new File(basePath + "/" + partitionPath).mkdirs();
String fileId = UUID.randomUUID().toString();
@Test
public void testGetLatestDataFilesForFileId() throws IOException {
String partitionPath = "2016/05/01";
new File(basePath + "/" + partitionPath).mkdirs();
String fileId = UUID.randomUUID().toString();
assertFalse("No commit, should not find any data file",
roView.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().isPresent());
assertFalse("No commit, should not find any data file",
roView.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().isPresent());
// Only one commit, but is not safe
String commitTime1 = "1";
String fileName1 = FSUtils.makeDataFileName(commitTime1, 1, fileId);
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
refreshFsView(null);
assertFalse("No commit, should not find any data file",
roView.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().isPresent());
// Only one commit, but is not safe
String commitTime1 = "1";
String fileName1 = FSUtils.makeDataFileName(commitTime1, 1, fileId);
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
refreshFsView(null);
assertFalse("No commit, should not find any data file",
roView.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().isPresent());
// Make this commit safe
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant1 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
commitTimeline.saveAsComplete(instant1, Optional.empty());
refreshFsView(null);
assertEquals("", fileName1, roView
.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().get()
.getFileName());
// Make this commit safe
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant1 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
commitTimeline.saveAsComplete(instant1, Optional.empty());
refreshFsView(null);
assertEquals("", fileName1, roView
.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().get()
.getFileName());
// Do another commit, but not safe
String commitTime2 = "2";
String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId);
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
refreshFsView(null);
assertEquals("", fileName1, roView
.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().get()
.getFileName());
// Do another commit, but not safe
String commitTime2 = "2";
String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId);
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
refreshFsView(null);
assertEquals("", fileName1, roView
.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().get()
.getFileName());
// Make it safe
HoodieInstant instant2 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2);
commitTimeline.saveAsComplete(instant2, Optional.empty());
refreshFsView(null);
assertEquals("", fileName2, roView
.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().get()
.getFileName());
}
// Make it safe
HoodieInstant instant2 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2);
commitTimeline.saveAsComplete(instant2, Optional.empty());
refreshFsView(null);
assertEquals("", fileName2, roView
.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().get()
.getFileName());
}
@Test
public void testStreamLatestVersionInPartition() throws IOException {
// Put some files in the partition
String fullPartitionPath = basePath + "/2016/05/01/";
new File(fullPartitionPath).mkdirs();
String commitTime1 = "1";
String commitTime2 = "2";
String commitTime3 = "3";
String commitTime4 = "4";
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
String fileId4 = UUID.randomUUID().toString();
@Test
public void testStreamLatestVersionInPartition() throws IOException {
// Put some files in the partition
String fullPartitionPath = basePath + "/2016/05/01/";
new File(fullPartitionPath).mkdirs();
String commitTime1 = "1";
String commitTime2 = "2";
String commitTime3 = "3";
String commitTime4 = "4";
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
String fileId4 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils
.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils
.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils
.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils
.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(11, statuses.length);
refreshFsView(null);
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
// Check files as of lastest commit.
List<FileSlice> allSlices = rtView.getAllFileSlices("2016/05/01").collect(Collectors.toList());
assertEquals(8, allSlices.size());
Map<String, Long> fileSliceMap = allSlices.stream().collect(Collectors.groupingBy(
slice -> slice.getFileId(), Collectors.counting()));
assertEquals(2, fileSliceMap.get(fileId1).longValue());
assertEquals(3, fileSliceMap.get(fileId2).longValue());
assertEquals(2, fileSliceMap.get(fileId3).longValue());
assertEquals(1, fileSliceMap.get(fileId4).longValue());
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(11, statuses.length);
refreshFsView(null);
// Check files as of lastest commit.
List<FileSlice> allSlices = rtView.getAllFileSlices("2016/05/01").collect(Collectors.toList());
assertEquals(8, allSlices.size());
Map<String, Long> fileSliceMap = allSlices.stream().collect(Collectors.groupingBy(
slice -> slice.getFileId(), Collectors.counting()));
assertEquals(2, fileSliceMap.get(fileId1).longValue());
assertEquals(3, fileSliceMap.get(fileId2).longValue());
assertEquals(2, fileSliceMap.get(fileId3).longValue());
assertEquals(1, fileSliceMap.get(fileId4).longValue());
List<HoodieDataFile> dataFileList =
roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4)
.collect(Collectors.toList());
assertEquals(3, dataFileList.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFileList) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
filenames = Sets.newHashSet();
List<HoodieLogFile> logFilesList =
rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4)
.map(slice -> slice.getLogFiles())
.flatMap(logFileList -> logFileList)
.collect(Collectors.toList());
assertEquals(logFilesList.size(), 4);
for (HoodieLogFile logFile: logFilesList) {
filenames.add(logFile.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)));
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1)));
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)));
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)));
// Reset the max commit time
List<HoodieDataFile> dataFiles =
roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3)
.collect(Collectors.toList());
assertEquals(dataFiles.size(), 3);
filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFiles) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
logFilesList =
rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3)
.map(slice -> slice.getLogFiles())
.flatMap(logFileList -> logFileList).collect(Collectors.toList());
assertEquals(logFilesList.size(), 1);
assertTrue(logFilesList.get(0).getFileName().equals(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)));
}
@Test
public void testStreamEveryVersionInPartition() throws IOException {
// Put some files in the partition
String fullPartitionPath = basePath + "/2016/05/01/";
new File(fullPartitionPath).mkdirs();
String commitTime1 = "1";
String commitTime2 = "2";
String commitTime3 = "3";
String commitTime4 = "4";
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(7, statuses.length);
refreshFsView(null);
List<HoodieFileGroup> fileGroups =
fsView.getAllFileGroups("2016/05/01").collect(Collectors.toList());
assertEquals(3, fileGroups.size());
for (HoodieFileGroup fileGroup : fileGroups) {
String fileId = fileGroup.getId();
Set<String> filenames = Sets.newHashSet();
fileGroup.getAllDataFiles().forEach(dataFile -> {
assertEquals("All same fileId should be grouped", fileId, dataFile.getFileId());
filenames.add(dataFile.getFileName());
});
if (fileId.equals(fileId1)) {
assertEquals(filenames,
Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId1),
FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
} else if (fileId.equals(fileId2)) {
assertEquals(filenames,
Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId2),
FSUtils.makeDataFileName(commitTime2, 1, fileId2),
FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
} else {
assertEquals(filenames,
Sets.newHashSet(FSUtils.makeDataFileName(commitTime3, 1, fileId3),
FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
}
}
}
@Test
public void streamLatestVersionInRange() throws IOException {
// Put some files in the partition
String fullPartitionPath = basePath + "/2016/05/01/";
new File(fullPartitionPath).mkdirs();
String commitTime1 = "1";
String commitTime2 = "2";
String commitTime3 = "3";
String commitTime4 = "4";
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(9, statuses.length);
refreshFsView(statuses);
List<HoodieDataFile> dataFiles = roView
.getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3))
List<HoodieDataFile> dataFileList =
roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4)
.collect(Collectors.toList());
assertEquals(3, dataFiles.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFiles) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
assertEquals(3, dataFileList.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFileList) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
filenames = Sets.newHashSet();
List<HoodieLogFile> logFilesList =
rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4)
.map(slice -> slice.getLogFiles())
.flatMap(logFileList -> logFileList)
.collect(Collectors.toList());
assertEquals(logFilesList.size(), 4);
for (HoodieLogFile logFile : logFilesList) {
filenames.add(logFile.getFileName());
}
assertTrue(filenames
.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)));
assertTrue(filenames
.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1)));
assertTrue(filenames
.contains(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)));
assertTrue(filenames
.contains(FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)));
List<FileSlice> slices = rtView
.getLatestFileSliceInRange(Lists.newArrayList(commitTime3, commitTime4))
.collect(Collectors.toList());
// Reset the max commit time
List<HoodieDataFile> dataFiles =
roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3)
.collect(Collectors.toList());
assertEquals(dataFiles.size(), 3);
filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFiles) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
logFilesList =
rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3)
.map(slice -> slice.getLogFiles())
.flatMap(logFileList -> logFileList).collect(Collectors.toList());
assertEquals(logFilesList.size(), 1);
assertTrue(logFilesList.get(0).getFileName()
.equals(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)));
}
@Test
public void testStreamEveryVersionInPartition() throws IOException {
// Put some files in the partition
String fullPartitionPath = basePath + "/2016/05/01/";
new File(fullPartitionPath).mkdirs();
String commitTime1 = "1";
String commitTime2 = "2";
String commitTime3 = "3";
String commitTime4 = "4";
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(7, statuses.length);
refreshFsView(null);
List<HoodieFileGroup> fileGroups =
fsView.getAllFileGroups("2016/05/01").collect(Collectors.toList());
assertEquals(3, fileGroups.size());
for (HoodieFileGroup fileGroup : fileGroups) {
String fileId = fileGroup.getId();
Set<String> filenames = Sets.newHashSet();
fileGroup.getAllDataFiles().forEach(dataFile -> {
assertEquals("All same fileId should be grouped", fileId, dataFile.getFileId());
filenames.add(dataFile.getFileName());
});
if (fileId.equals(fileId1)) {
assertEquals(filenames,
Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId1),
FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
} else if (fileId.equals(fileId2)) {
assertEquals(filenames,
Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId2),
FSUtils.makeDataFileName(commitTime2, 1, fileId2),
FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
} else {
assertEquals(filenames,
Sets.newHashSet(FSUtils.makeDataFileName(commitTime3, 1, fileId3),
FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
}
}
}
@Test
public void streamLatestVersionInRange() throws IOException {
// Put some files in the partition
String fullPartitionPath = basePath + "/2016/05/01/";
new File(fullPartitionPath).mkdirs();
String commitTime1 = "1";
String commitTime2 = "2";
String commitTime3 = "3";
String commitTime4 = "4";
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils
.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils
.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(9, statuses.length);
refreshFsView(statuses);
List<HoodieDataFile> dataFiles = roView
.getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3))
.collect(Collectors.toList());
assertEquals(3, dataFiles.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFiles) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
List<FileSlice> slices = rtView
.getLatestFileSliceInRange(Lists.newArrayList(commitTime3, commitTime4))
.collect(Collectors.toList());
assertEquals(3, slices.size());
for (FileSlice slice : slices) {
if (slice.getFileId().equals(fileId1)) {
assertEquals(slice.getBaseCommitTime(), commitTime3);
assertTrue(slice.getDataFile().isPresent());
assertEquals(slice.getLogFiles().count(), 0);
} else if (slice.getFileId().equals(fileId2)) {
assertEquals(slice.getBaseCommitTime(), commitTime4);
assertFalse(slice.getDataFile().isPresent());
assertEquals(slice.getLogFiles().count(), 1);
} else if (slice.getFileId().equals(fileId3)) {
assertEquals(slice.getBaseCommitTime(), commitTime4);
assertTrue(slice.getDataFile().isPresent());
assertEquals(slice.getLogFiles().count(), 0);
}
}
}
@Test
public void streamLatestVersionsBefore() throws IOException {
// Put some files in the partition
String partitionPath = "2016/05/01/";
String fullPartitionPath = basePath + "/" + partitionPath;
new File(fullPartitionPath).mkdirs();
String commitTime1 = "1";
String commitTime2 = "2";
String commitTime3 = "3";
String commitTime4 = "4";
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(7, statuses.length);
refreshFsView(null);
List<HoodieDataFile> dataFiles =
roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2)
.collect(Collectors.toList());
assertEquals(2, dataFiles.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFiles) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, 1, fileId2)));
}
@Test
public void streamLatestVersions() throws IOException {
// Put some files in the partition
String partitionPath = "2016/05/01/";
String fullPartitionPath = basePath + "/" + partitionPath;
new File(fullPartitionPath).mkdirs();
String commitTime1 = "1";
String commitTime2 = "2";
String commitTime3 = "3";
String commitTime4 = "4";
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils
.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils
.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils
.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(10, statuses.length);
refreshFsView(statuses);
List<HoodieFileGroup> fileGroups = fsView
.getAllFileGroups(partitionPath)
.collect(Collectors.toList());
assertEquals(3, fileGroups.size());
for (HoodieFileGroup fileGroup : fileGroups) {
List<FileSlice> slices = fileGroup.getAllFileSlices().collect(Collectors.toList());
if (fileGroup.getId().equals(fileId1)) {
assertEquals(2, slices.size());
assertEquals(commitTime4, slices.get(0).getBaseCommitTime());
assertEquals(commitTime1, slices.get(1).getBaseCommitTime());
} else if (fileGroup.getId().equals(fileId2)) {
assertEquals(3, slices.size());
for (FileSlice slice: slices) {
if (slice.getFileId().equals(fileId1)) {
assertEquals(slice.getBaseCommitTime(), commitTime3);
assertTrue(slice.getDataFile().isPresent());
assertEquals(slice.getLogFiles().count(), 0);
} else if (slice.getFileId().equals(fileId2)) {
assertEquals(slice.getBaseCommitTime(), commitTime4);
assertFalse(slice.getDataFile().isPresent());
assertEquals(slice.getLogFiles().count(), 1);
} else if (slice.getFileId().equals(fileId3)) {
assertEquals(slice.getBaseCommitTime(), commitTime4);
assertTrue(slice.getDataFile().isPresent());
assertEquals(slice.getLogFiles().count(), 0);
}
}
assertEquals(commitTime3, slices.get(0).getBaseCommitTime());
assertEquals(commitTime2, slices.get(1).getBaseCommitTime());
assertEquals(commitTime1, slices.get(2).getBaseCommitTime());
} else if (fileGroup.getId().equals(fileId3)) {
assertEquals(2, slices.size());
assertEquals(commitTime4, slices.get(0).getBaseCommitTime());
assertEquals(commitTime3, slices.get(1).getBaseCommitTime());
}
}
@Test
public void streamLatestVersionsBefore() throws IOException {
// Put some files in the partition
String partitionPath = "2016/05/01/";
String fullPartitionPath = basePath + "/" + partitionPath;
new File(fullPartitionPath).mkdirs();
String commitTime1 = "1";
String commitTime2 = "2";
String commitTime3 = "3";
String commitTime4 = "4";
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(7, statuses.length);
refreshFsView(null);
List<HoodieDataFile> dataFiles =
roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2)
.collect(Collectors.toList());
assertEquals(2, dataFiles.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFiles) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, 1, fileId2)));
}
@Test
public void streamLatestVersions() throws IOException {
// Put some files in the partition
String partitionPath = "2016/05/01/";
String fullPartitionPath = basePath + "/" + partitionPath;
new File(fullPartitionPath).mkdirs();
String commitTime1 = "1";
String commitTime2 = "2";
String commitTime3 = "3";
String commitTime4 = "4";
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(10, statuses.length);
refreshFsView(statuses);
List<HoodieFileGroup> fileGroups = fsView
.getAllFileGroups(partitionPath)
.collect(Collectors.toList());
assertEquals(3, fileGroups.size());
for (HoodieFileGroup fileGroup: fileGroups) {
List<FileSlice> slices = fileGroup.getAllFileSlices().collect(Collectors.toList());
if (fileGroup.getId().equals(fileId1)) {
assertEquals(2, slices.size());
assertEquals(commitTime4, slices.get(0).getBaseCommitTime());
assertEquals(commitTime1, slices.get(1).getBaseCommitTime());
} else if (fileGroup.getId().equals(fileId2)) {
assertEquals(3, slices.size());
assertEquals(commitTime3, slices.get(0).getBaseCommitTime());
assertEquals(commitTime2, slices.get(1).getBaseCommitTime());
assertEquals(commitTime1, slices.get(2).getBaseCommitTime());
} else if (fileGroup.getId().equals(fileId3)) {
assertEquals(2, slices.size());
assertEquals(commitTime4, slices.get(0).getBaseCommitTime());
assertEquals(commitTime3, slices.get(1).getBaseCommitTime());
}
}
List<HoodieDataFile> statuses1 =
roView.getLatestDataFiles().collect(Collectors.toList());
assertEquals(3, statuses1.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : statuses1) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
List<HoodieDataFile> statuses1 =
roView.getLatestDataFiles().collect(Collectors.toList());
assertEquals(3, statuses1.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : statuses1) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
}
}

View File

@@ -19,111 +19,111 @@ package com.uber.hoodie.common.util;
import com.uber.hoodie.avro.MercifulJsonConverter;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.file.FileSystem;
import java.nio.file.FileSystemNotFoundException;
import java.nio.file.FileSystems;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.avro.io.DecoderFactory;
import java.io.IOException;
import java.net.URISyntaxException;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public class SchemaTestUtil {
public static Schema getSimpleSchema() throws IOException {
return new Schema.Parser()
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avro"));
public static Schema getSimpleSchema() throws IOException {
return new Schema.Parser()
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avro"));
}
public static List<IndexedRecord> generateTestRecords(int from, int limit)
throws IOException, URISyntaxException {
return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit);
}
private static List<IndexedRecord> toRecords(Schema writerSchema, Schema readerSchema, int from,
int limit) throws IOException, URISyntaxException {
GenericDatumReader<IndexedRecord> reader =
new GenericDatumReader<>(writerSchema, readerSchema);
// Required to register the necessary JAR:// file system
URI resource = SchemaTestUtil.class.getClass().getResource("/sample.data").toURI();
Path dataPath;
if (resource.toString().contains("!")) {
dataPath = uriToPath(resource);
} else {
dataPath = Paths.get(SchemaTestUtil.class.getClass().getResource("/sample.data").toURI());
}
public static List<IndexedRecord> generateTestRecords(int from, int limit)
throws IOException, URISyntaxException {
return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit);
}
private static List<IndexedRecord> toRecords(Schema writerSchema, Schema readerSchema, int from,
int limit) throws IOException, URISyntaxException {
GenericDatumReader<IndexedRecord> reader =
new GenericDatumReader<>(writerSchema, readerSchema);
// Required to register the necessary JAR:// file system
URI resource = SchemaTestUtil.class.getClass().getResource("/sample.data").toURI();
Path dataPath;
if(resource.toString().contains("!")) {
dataPath = uriToPath(resource);
} else {
dataPath = Paths.get(SchemaTestUtil.class.getClass().getResource("/sample.data").toURI());
}
try (Stream<String> stream = Files.lines(dataPath)) {
return stream.skip(from).limit(limit).map(s -> {
try {
return reader.read(null, DecoderFactory.get().jsonDecoder(writerSchema, s));
} catch (IOException e) {
throw new HoodieIOException("Could not read data from simple_data.json", e);
}
}).collect(Collectors.toList());
} catch (IOException e) {
throw new HoodieIOException("Could not read data from simple_data.json", e);
}
}
static Path uriToPath(URI uri) throws IOException {
final Map<String, String> env = new HashMap<>();
final String[] array = uri.toString().split("!");
FileSystem fs;
try (Stream<String> stream = Files.lines(dataPath)) {
return stream.skip(from).limit(limit).map(s -> {
try {
fs = FileSystems.getFileSystem(URI.create(array[0]));
} catch (FileSystemNotFoundException e) {
fs = FileSystems.newFileSystem(URI.create(array[0]), env);
return reader.read(null, DecoderFactory.get().jsonDecoder(writerSchema, s));
} catch (IOException e) {
throw new HoodieIOException("Could not read data from simple_data.json", e);
}
return fs.getPath(array[1]);
}).collect(Collectors.toList());
} catch (IOException e) {
throw new HoodieIOException("Could not read data from simple_data.json", e);
}
}
public static List<IndexedRecord> generateHoodieTestRecords(int from, int limit)
throws IOException, URISyntaxException {
List<IndexedRecord> records = generateTestRecords(from, limit);
Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
return records.stream()
.map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema))
.map(p -> {
p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString());
p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00");
return p;
}).collect(
Collectors.toList());
static Path uriToPath(URI uri) throws IOException {
final Map<String, String> env = new HashMap<>();
final String[] array = uri.toString().split("!");
FileSystem fs;
try {
fs = FileSystems.getFileSystem(URI.create(array[0]));
} catch (FileSystemNotFoundException e) {
fs = FileSystems.newFileSystem(URI.create(array[0]), env);
}
return fs.getPath(array[1]);
}
public static Schema getEvolvedSchema() throws IOException {
return new Schema.Parser()
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro"));
}
public static List<IndexedRecord> generateHoodieTestRecords(int from, int limit)
throws IOException, URISyntaxException {
List<IndexedRecord> records = generateTestRecords(from, limit);
Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
return records.stream()
.map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema))
.map(p -> {
p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString());
p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00");
return p;
}).collect(
Collectors.toList());
public static List<IndexedRecord> generateEvolvedTestRecords(int from, int limit)
throws IOException, URISyntaxException {
return toRecords(getSimpleSchema(), getEvolvedSchema(), from, limit);
}
}
public static Schema getComplexEvolvedSchema() throws IOException {
return new Schema.Parser()
.parse(SchemaTestUtil.class.getResourceAsStream("/complex-test-evolved.avro"));
}
public static Schema getEvolvedSchema() throws IOException {
return new Schema.Parser()
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro"));
}
public static GenericRecord generateAvroRecordFromJson(Schema schema, int recordNumber,
String commitTime, String fileId) throws IOException {
TestRecord record = new TestRecord(commitTime, recordNumber, fileId);
MercifulJsonConverter converter = new MercifulJsonConverter(schema);
return converter.convert(record.toJsonString());
}
public static List<IndexedRecord> generateEvolvedTestRecords(int from, int limit)
throws IOException, URISyntaxException {
return toRecords(getSimpleSchema(), getEvolvedSchema(), from, limit);
}
public static Schema getComplexEvolvedSchema() throws IOException {
return new Schema.Parser()
.parse(SchemaTestUtil.class.getResourceAsStream("/complex-test-evolved.avro"));
}
public static GenericRecord generateAvroRecordFromJson(Schema schema, int recordNumber,
String commitTime, String fileId) throws IOException {
TestRecord record = new TestRecord(commitTime, recordNumber, fileId);
MercifulJsonConverter converter = new MercifulJsonConverter(schema);
return converter.convert(record.toJsonString());
}
}

View File

@@ -16,48 +16,47 @@
package com.uber.hoodie.common.util;
import org.junit.Test;
import static org.junit.Assert.assertTrue;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.UUID;
import static org.junit.Assert.assertTrue;
import org.junit.Test;
public class TestFSUtils {
@Test
public void testMakeDataFileName() {
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
int taskPartitionId = 2;
String fileName = UUID.randomUUID().toString();
assertTrue(FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName)
.equals(fileName + "_" + taskPartitionId + "_" + commitTime + ".parquet"));
}
@Test
public void testMakeDataFileName() {
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
int taskPartitionId = 2;
String fileName = UUID.randomUUID().toString();
assertTrue(FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName)
.equals(fileName + "_" + taskPartitionId + "_" + commitTime + ".parquet"));
}
@Test
public void testMaskFileName() {
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
int taskPartitionId = 2;
assertTrue(FSUtils.maskWithoutFileId(commitTime, taskPartitionId)
.equals("*_" + taskPartitionId + "_" + commitTime + ".parquet"));
}
@Test
public void testMaskFileName() {
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
int taskPartitionId = 2;
assertTrue(FSUtils.maskWithoutFileId(commitTime, taskPartitionId)
.equals("*_" + taskPartitionId + "_" + commitTime + ".parquet"));
}
@Test
public void testGetCommitTime() {
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
int taskPartitionId = 2;
String fileName = UUID.randomUUID().toString();
String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName);
assertTrue(FSUtils.getCommitTime(fullFileName).equals(commitTime));
}
@Test
public void testGetCommitTime() {
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
int taskPartitionId = 2;
String fileName = UUID.randomUUID().toString();
String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName);
assertTrue(FSUtils.getCommitTime(fullFileName).equals(commitTime));
}
@Test
public void testGetFileNameWithoutMeta() {
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
int taskPartitionId = 2;
String fileName = UUID.randomUUID().toString();
String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName);
assertTrue(FSUtils.getFileId(fullFileName).equals(fileName));
}
@Test
public void testGetFileNameWithoutMeta() {
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
int taskPartitionId = 2;
String fileName = UUID.randomUUID().toString();
String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName);
assertTrue(FSUtils.getFileId(fullFileName).equals(fileName));
}
}

View File

@@ -16,20 +16,22 @@
package com.uber.hoodie.common.util;
import static org.junit.Assert.assertTrue;
import org.junit.Test;
import static org.junit.Assert.*;
public class TestNumericUtils {
@Test
public void testHumanReadableByteCount() {
assertTrue(NumericUtils.humanReadableByteCount(0).equals("0.0 B"));
assertTrue(NumericUtils.humanReadableByteCount(27).equals("27.0 B"));
assertTrue(NumericUtils.humanReadableByteCount(1023).equals("1023.0 B"));
assertTrue(NumericUtils.humanReadableByteCount(1024).equals("1.0 KB"));
assertTrue(NumericUtils.humanReadableByteCount(110592).equals("108.0 KB"));
assertTrue(NumericUtils.humanReadableByteCount(28991029248L).equals("27.0 GB"));
assertTrue(NumericUtils.humanReadableByteCount(1855425871872L).equals("1.7 TB"));
assertTrue(NumericUtils.humanReadableByteCount(9223372036854775807L).equals("8.0 EB"));
}
@Test
public void testHumanReadableByteCount() {
assertTrue(NumericUtils.humanReadableByteCount(0).equals("0.0 B"));
assertTrue(NumericUtils.humanReadableByteCount(27).equals("27.0 B"));
assertTrue(NumericUtils.humanReadableByteCount(1023).equals("1023.0 B"));
assertTrue(NumericUtils.humanReadableByteCount(1024).equals("1.0 KB"));
assertTrue(NumericUtils.humanReadableByteCount(110592).equals("108.0 KB"));
assertTrue(NumericUtils.humanReadableByteCount(28991029248L).equals("27.0 GB"));
assertTrue(NumericUtils.humanReadableByteCount(1855425871872L).equals("1.7 TB"));
assertTrue(NumericUtils.humanReadableByteCount(9223372036854775807L).equals("8.0 EB"));
}
}

View File

@@ -16,10 +16,17 @@
package com.uber.hoodie.common.util;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieRecord;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.UUID;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
@@ -31,62 +38,55 @@ import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.UUID;
import static org.junit.Assert.*;
public class TestParquetUtils {
private String basePath;
private String basePath;
@Before
public void setup() throws IOException {
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
@Before
public void setup() throws IOException {
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
}
@Test
public void testHoodieWriteSupport() throws Exception {
List<String> rowKeys = new ArrayList<>();
for (int i = 0; i < 1000; i++) {
rowKeys.add(UUID.randomUUID().toString());
}
@Test
public void testHoodieWriteSupport() throws Exception {
// Write out a parquet file
Schema schema = HoodieAvroUtils.getRecordKeySchema();
BloomFilter filter = new BloomFilter(1000, 0.0001);
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
new AvroSchemaConverter().convert(schema), schema, filter);
List<String> rowKeys = new ArrayList<>();
for (int i = 0; i < 1000; i++) {
rowKeys.add(UUID.randomUUID().toString());
}
// Write out a parquet file
Schema schema = HoodieAvroUtils.getRecordKeySchema();
BloomFilter filter = new BloomFilter(1000, 0.0001);
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
String filePath = basePath + "/test.parquet";
ParquetWriter writer = new ParquetWriter(new Path(filePath),
writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE);
for (String rowKey : rowKeys) {
GenericRecord rec = new GenericData.Record(schema);
rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey);
writer.write(rec);
filter.add(rowKey);
}
writer.close();
// Read and verify
List<String> rowKeysInFile = new ArrayList<>(ParquetUtils.readRowKeysFromParquet(new Path(filePath)));
Collections.sort(rowKeysInFile);
Collections.sort(rowKeys);
assertEquals("Did not read back the expected list of keys", rowKeys, rowKeysInFile);
BloomFilter filterInFile = ParquetUtils.readBloomFilterFromParquetMetadata(new Path(filePath));
for (String rowKey : rowKeys) {
assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey));
}
String filePath = basePath + "/test.parquet";
ParquetWriter writer = new ParquetWriter(new Path(filePath),
writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024,
ParquetWriter.DEFAULT_PAGE_SIZE);
for (String rowKey : rowKeys) {
GenericRecord rec = new GenericData.Record(schema);
rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey);
writer.write(rec);
filter.add(rowKey);
}
writer.close();
// Read and verify
List<String> rowKeysInFile = new ArrayList<>(
ParquetUtils.readRowKeysFromParquet(new Path(filePath)));
Collections.sort(rowKeysInFile);
Collections.sort(rowKeys);
assertEquals("Did not read back the expected list of keys", rowKeys, rowKeysInFile);
BloomFilter filterInFile = ParquetUtils.readBloomFilterFromParquetMetadata(new Path(filePath));
for (String rowKey : rowKeys) {
assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey));
}
}
}

View File

@@ -17,82 +17,87 @@
package com.uber.hoodie.common.util;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.codehaus.jackson.annotate.JsonAutoDetect;
import org.codehaus.jackson.annotate.JsonMethod;
import org.codehaus.jackson.map.ObjectMapper;
import java.io.IOException;
import java.io.Serializable;
import java.util.HashMap;
import java.util.Map;
import org.codehaus.jackson.annotate.JsonAutoDetect;
import org.codehaus.jackson.annotate.JsonMethod;
import org.codehaus.jackson.map.ObjectMapper;
@JsonIgnoreProperties(ignoreUnknown = true)
@SuppressWarnings({"unused", "FieldCanBeLocal", "MismatchedQueryAndUpdateOfCollection"})
public class TestRecord implements Serializable {
class TestMapItemRecord implements Serializable {
private String item1;
private String item2;
TestMapItemRecord(String item1, String item2) {
this.item1 = item1;
this.item2 = item2;
}
class TestMapItemRecord implements Serializable {
private String item1;
private String item2;
TestMapItemRecord(String item1, String item2) {
this.item1 = item1;
this.item2 = item2;
}
}
class TestNestedRecord implements Serializable {
private boolean isAdmin;
private String userId;
class TestNestedRecord implements Serializable {
TestNestedRecord(boolean isAdmin, String userId) {
this.isAdmin = isAdmin;
this.userId = userId;
}
private boolean isAdmin;
private String userId;
TestNestedRecord(boolean isAdmin, String userId) {
this.isAdmin = isAdmin;
this.userId = userId;
}
}
private String _hoodie_commit_time;
private String _hoodie_record_key;
private String _hoodie_partition_path;
private String _hoodie_file_name;
private String _hoodie_commit_seqno;
private String _hoodie_commit_time;
private String _hoodie_record_key;
private String _hoodie_partition_path;
private String _hoodie_file_name;
private String _hoodie_commit_seqno;
private String field1;
private String field2;
private String name;
private Integer favoriteIntNumber;
private Long favoriteNumber;
private Float favoriteFloatNumber;
private Double favoriteDoubleNumber;
private Map<String, TestMapItemRecord> tags;
private TestNestedRecord testNestedRecord;
private String[] stringArray;
private String field1;
private String field2;
private String name;
private Integer favoriteIntNumber;
private Long favoriteNumber;
private Float favoriteFloatNumber;
private Double favoriteDoubleNumber;
private Map<String, TestMapItemRecord> tags;
private TestNestedRecord testNestedRecord;
private String[] stringArray;
public TestRecord(String commitTime, int recordNumber, String fileId) {
this._hoodie_commit_time = commitTime;
this._hoodie_record_key = "key" + recordNumber;
this._hoodie_partition_path = commitTime;
this._hoodie_file_name = fileId;
this._hoodie_commit_seqno = commitTime + recordNumber;
public TestRecord(String commitTime, int recordNumber, String fileId) {
this._hoodie_commit_time = commitTime;
this._hoodie_record_key = "key" + recordNumber;
this._hoodie_partition_path = commitTime;
this._hoodie_file_name = fileId;
this._hoodie_commit_seqno = commitTime + recordNumber;
String commitTimeSuffix = "@" + commitTime;
int commitHashCode = commitTime.hashCode();
String commitTimeSuffix = "@" + commitTime;
int commitHashCode = commitTime.hashCode();
this.field1 = "field" + recordNumber;
this.field2 = "field" + recordNumber + commitTimeSuffix;
this.name = "name" + recordNumber;
this.favoriteIntNumber = recordNumber + commitHashCode;
this.favoriteNumber = (long)(recordNumber + commitHashCode);
this.favoriteFloatNumber = (float)((recordNumber + commitHashCode) / 1024.0);
this.favoriteDoubleNumber = (recordNumber + commitHashCode) / 1024.0;
this.tags = new HashMap<>();
this.tags.put("mapItem1", new TestMapItemRecord("item" + recordNumber, "item" + recordNumber + commitTimeSuffix));
this.tags.put("mapItem2", new TestMapItemRecord("item2" + recordNumber, "item2" + recordNumber + commitTimeSuffix));
this.testNestedRecord = new TestNestedRecord(false, "UserId" + recordNumber + commitTimeSuffix);
this.stringArray = new String[]{"stringArray0" + commitTimeSuffix, "stringArray1" + commitTimeSuffix};
}
this.field1 = "field" + recordNumber;
this.field2 = "field" + recordNumber + commitTimeSuffix;
this.name = "name" + recordNumber;
this.favoriteIntNumber = recordNumber + commitHashCode;
this.favoriteNumber = (long) (recordNumber + commitHashCode);
this.favoriteFloatNumber = (float) ((recordNumber + commitHashCode) / 1024.0);
this.favoriteDoubleNumber = (recordNumber + commitHashCode) / 1024.0;
this.tags = new HashMap<>();
this.tags.put("mapItem1",
new TestMapItemRecord("item" + recordNumber, "item" + recordNumber + commitTimeSuffix));
this.tags.put("mapItem2",
new TestMapItemRecord("item2" + recordNumber, "item2" + recordNumber + commitTimeSuffix));
this.testNestedRecord = new TestNestedRecord(false, "UserId" + recordNumber + commitTimeSuffix);
this.stringArray = new String[]{"stringArray0" + commitTimeSuffix,
"stringArray1" + commitTimeSuffix};
}
public String toJsonString() throws IOException {
ObjectMapper mapper = new ObjectMapper();
mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY);
return mapper.writerWithDefaultPrettyPrinter().writeValueAsString(this);
}
public String toJsonString() throws IOException {
ObjectMapper mapper = new ObjectMapper();
mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY);
return mapper.writerWithDefaultPrettyPrinter().writeValueAsString(this);
}
}

View File

@@ -13,13 +13,11 @@
# See the License for the specific language governing permissions and
# limitations under the License.
#
log4j.rootLogger=WARN, A1
log4j.category.com.uber=INFO
log4j.category.com.uber.hoodie.table.log=WARN
log4j.category.com.uber.hoodie.common.util=WARN
log4j.category.org.apache.parquet.hadoop=WARN
# A1 is set to be a ConsoleAppender.
log4j.appender.A1=org.apache.log4j.ConsoleAppender
# A1 uses PatternLayout.