1
0

HUDI-123 Rename code packages/constants to org.apache.hudi (#830)

- Rename com.uber.hoodie to org.apache.hudi
- Flag to pass com.uber.hoodie Input formats for hoodie-sync
- Works with HUDI demo. 
- Also tested for backwards compatibility with datasets built by com.uber.hoodie packages
- Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
Balaji Varadarajan
2019-08-11 17:48:17 -07:00
committed by vinoth chandar
parent 722b6be04a
commit a4f9d7575f
546 changed files with 3858 additions and 3562 deletions

181
hudi-common/pom.xml Normal file
View File

@@ -0,0 +1,181 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
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>hudi</artifactId>
<groupId>org.apache.hudi</groupId>
<version>0.5.0-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>hudi-common</artifactId>
<properties>
<notice.dir>${project.basedir}/src/main/resources/META-INF</notice.dir>
</properties>
<build>
<resources>
<resource>
<directory>src/main/resources</directory>
</resource>
</resources>
<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>
<phase>test-compile</phase>
</execution>
</executions>
<configuration>
<skip>false</skip>
</configuration>
</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>
<import>${basedir}/src/main/avro/HoodieRestoreMetadata.avsc</import>
</imports>
</configuration>
</plugin>
</plugins>
</build>
<dependencies>
<dependency>
<groupId>org.rocksdb</groupId>
<artifactId>rocksdbjni</artifactId>
</dependency>
<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>
<version>${fasterxml.version}</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</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.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
<version>4.5.4</version>
</dependency>
<dependency>
<groupId>commons-codec</groupId>
<artifactId>commons-codec</artifactId>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>fluent-hc</artifactId>
<version>4.5.4</version>
</dependency>
<dependency>
<groupId>com.esotericsoftware</groupId>
<artifactId>kryo</artifactId>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<exclusions>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.github.stefanbirkner</groupId>
<artifactId>system-rules</artifactId>
<version>1.16.0</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.twitter.common</groupId>
<artifactId>objectsize</artifactId>
<version>0.0.12</version>
</dependency>
<dependency>
<groupId>com.esotericsoftware</groupId>
<artifactId>kryo-shaded</artifactId>
<version>4.0.2</version>
</dependency>
</dependencies>
</project>

View File

@@ -0,0 +1,55 @@
{
"type":"record",
"name":"HoodieArchivedMetaEntry",
"namespace":"org.apache.hudi.avro.model",
"fields":[
{
"name":"hoodieCommitMetadata",
"type":[
"null",
"HoodieCommitMetadata"
],
"default": "null"
},
{
"name":"hoodieCleanMetadata",
"type":[
"null",
"HoodieCleanMetadata"
],
"default": "null"
},
{
"name":"hoodieCompactionMetadata",
"type":[
"null",
"HoodieCompactionMetadata"
],
"default": "null"
},
{
"name":"hoodieRollbackMetadata",
"type":[
"null",
"HoodieRollbackMetadata"
],
"default": "null"
},
{
"name":"hoodieSavePointMetadata",
"type":[
"null",
"HoodieSavepointMetadata"
],
"default": "null"
},
{
"name":"commitTime",
"type":["null","string"]
},
{
"name":"actionType",
"type":["null","string"]
}
]
}

View File

@@ -0,0 +1,24 @@
{"namespace": "org.apache.hudi.avro.model",
"type": "record",
"name": "HoodieCleanMetadata",
"fields": [
{"name": "startCleanTime", "type": "string"},
{"name": "timeTakenInMillis", "type": "long"},
{"name": "totalFilesDeleted", "type": "int"},
{"name": "earliestCommitToRetain", "type": "string"},
{"name": "partitionMetadata", "type": {
"type" : "map", "values" : {
"type": "record",
"name": "HoodieCleanPartitionMetadata",
"fields": [
{"name": "partitionPath", "type": "string"},
{"name": "policy", "type": "string"},
{"name": "deletePathPatterns", "type": {"type": "array", "items": "string"}},
{"name": "successDeleteFiles", "type": {"type": "array", "items": "string"}},
{"name": "failedDeleteFiles", "type": {"type": "array", "items": "string"}}
]
}
}
}
]
}

View File

@@ -0,0 +1,114 @@
{
"namespace":"org.apache.hudi.avro.model",
"type":"record",
"name":"HoodieCommitMetadata",
"fields":[
{
"name":"partitionToWriteStats",
"type":["null", {
"type":"map",
"values":{
"type":"array",
"items":{
"name":"HoodieWriteStat",
"type":"record",
"fields":[
{
"name":"fileId",
"type":["null","string"],
"default" : null
},
{
"name":"path",
"type":["null","string"],
"default" : null
},
{
"name":"prevCommit",
"type":["null","string"],
"default" : null
},
{
"name":"numWrites",
"type":["null","long"],
"default" : null
},
{
"name":"numDeletes",
"type":["null","long"],
"default" : null
},
{
"name":"numUpdateWrites",
"type":["null","long"],
"default" : null
},
{
"name":"totalWriteBytes",
"type":["null","long"],
"default" : null
},
{
"name":"totalWriteErrors",
"type":["null","long"],
"default" : null
},
{
"name":"partitionPath",
"type":["null","string"],
"default" : null
},
{
"name":"totalLogRecords",
"type":["null","long"],
"default" : null
},
{
"name":"totalLogFiles",
"type":["null","long"],
"default" : null
},
{
"name":"totalUpdatedRecordsCompacted",
"type":["null","long"],
"default" : null
},
{
"name":"numInserts",
"type":["null","long"],
"default" : null
},
{
"name":"totalLogBlocks",
"type":["null","long"],
"default" : null
},
{
"name":"totalCorruptLogBlock",
"type":["null","long"],
"default" : null
},
{
"name":"totalRollbackBlocks",
"type":["null","long"],
"default" : null
},
{
"name":"fileSizeInBytes",
"type":["null","long"],
"default" : null
}
]
}
}
}]
},
{
"name":"extraMetadata",
"type":["null", {
"type":"map",
"values":"string"
}]
}
]
}

View File

@@ -0,0 +1,42 @@
{
"namespace":"org.apache.hudi.avro.model",
"type":"record",
"name":"HoodieCompactionMetadata",
"fields":[
{
"name":"partitionToCompactionWriteStats",
"type": ["null", {
"type":"map",
"values":{
"type":"array",
"items":{
"name":"HoodieCompactionWriteStat",
"type":"record",
"fields":[
{
"name":"partitionPath",
"type":["null","string"]
},
{
"name":"totalLogRecords",
"type":["null","long"]
},
{
"name":"totalLogFiles",
"type":["null","long"]
},
{
"name":"totalUpdatedRecordsCompacted",
"type":["null","long"]
},
{
"name":"hoodieWriteStat",
"type":["null","HoodieWriteStat"]
}
]
}
}
}]
}
]
}

View File

@@ -0,0 +1,62 @@
{
"namespace":"org.apache.hudi.avro.model",
"type":"record",
"name":"HoodieCompactionPlan",
"fields":[
{
"name":"operations",
"type":["null", {
"type":"array",
"items":{
"name":"HoodieCompactionOperation",
"type":"record",
"fields":[
{
"name":"baseInstantTime",
"type":["null","string"]
},
{
"name":"deltaFilePaths",
"type":["null", {
"type":"array",
"items":"string"
}],
"default": null
},
{
"name":"dataFilePath",
"type":["null","string"],
"default": null
},
{
"name":"fileId",
"type":["null","string"]
},
{
"name":"partitionPath",
"type":["null","string"],
"default": null
},
{
"name":"metrics",
"type":["null", {
"type":"map",
"values":"double"
}],
"default": null
}
]
}
}],
"default": null
},
{
"name":"extraMetadata",
"type":["null", {
"type":"map",
"values":"string"
}],
"default": null
}
]
}

View File

@@ -0,0 +1,17 @@
{"namespace": "org.apache.hudi.avro.model",
"type": "record",
"name": "HoodieRestoreMetadata",
"fields": [
{"name": "startRestoreTime", "type": "string"},
{"name": "timeTakenInMillis", "type": "long"},
{"name": "instantsToRollback", "type": {"type": "array", "items": "string"}},
{"name": "hoodieRestoreMetadata", "type": {
"type" : "map", "values" : {
"type": "array",
"default": "null",
"items": "HoodieRollbackMetadata",
"name": "hoodieRollbackMetadata"
}
}}
]
}

View File

@@ -0,0 +1,22 @@
{"namespace": "org.apache.hudi.avro.model",
"type": "record",
"name": "HoodieRollbackMetadata",
"fields": [
{"name": "startRollbackTime", "type": "string"},
{"name": "timeTakenInMillis", "type": "long"},
{"name": "totalFilesDeleted", "type": "int"},
{"name": "commitsRollback", "type": {"type": "array", "items": "string"}},
{"name": "partitionMetadata", "type": {
"type" : "map", "values" : {
"type": "record",
"name": "HoodieRollbackPartitionMetadata",
"fields": [
{"name": "partitionPath", "type": "string"},
{"name": "successDeleteFiles", "type": {"type": "array", "items": "string"}},
{"name": "failedDeleteFiles", "type": {"type": "array", "items": "string"}}
]
}
}
}
]
}

View File

@@ -0,0 +1,20 @@
{"namespace": "org.apache.hudi.avro.model",
"type": "record",
"name": "HoodieSavepointMetadata",
"fields": [
{"name": "savepointedBy", "type": "string"},
{"name": "savepointedAt", "type": "long"},
{"name": "comments", "type": "string"},
{"name": "partitionMetadata", "type": {
"type" : "map", "values" : {
"type": "record",
"name": "HoodieSavepointPartitionMetadata",
"fields": [
{"name": "partitionPath", "type": "string"},
{"name": "savepointDataFile", "type": {"type": "array", "items": "string"}}
]
}
}
}
]
}

View File

@@ -0,0 +1,77 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.avro;
import java.util.HashMap;
import org.apache.avro.Schema;
import org.apache.hudi.common.BloomFilter;
import org.apache.parquet.avro.AvroWriteSupport;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.schema.MessageType;
/**
* Wrap AvroWriterSupport for plugging in the bloom filter.
*/
public class HoodieAvroWriteSupport extends AvroWriteSupport {
private BloomFilter bloomFilter;
private String minRecordKey;
private String maxRecordKey;
public static final String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY =
"org.apache.hudi.bloomfilter";
public static final String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key";
public static final String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key";
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;
}
if (maxRecordKey != null) {
maxRecordKey = maxRecordKey.compareTo(recordKey) >= 0 ? maxRecordKey : recordKey;
} else {
maxRecordKey = recordKey;
}
}
}

View File

@@ -0,0 +1,167 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.avro;
import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.IOException;
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;
/**
* 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;
}
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;
}
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 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());
}
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

@@ -0,0 +1,104 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common;
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 org.apache.hudi.exception.HoodieIndexException;
/**
* 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;
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);
}
}
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);
}
}
}

View File

@@ -0,0 +1,128 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common;
import java.io.Serializable;
import java.util.List;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
/**
* 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;
}
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 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(Option<HoodieInstant> earliestCommitToRetain) {
this.earliestCommitToRetain = (earliestCommitToRetain.isPresent())
? earliestCommitToRetain.get().getTimestamp() : "-1";
return this;
}
public HoodieCleanStat build() {
return new HoodieCleanStat(policy, partitionPath, deletePathPatterns,
successDeleteFiles, failedDeleteFiles, earliestCommitToRetain);
}
}
}

View File

@@ -0,0 +1,113 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.StringWriter;
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.apache.hudi.avro.MercifulJsonConverter;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
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();
}
@Override
public HoodieJsonPayload preCombine(HoodieJsonPayload another) {
return this;
}
@Override
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema)
throws IOException {
return getInsertValue(schema);
}
@Override
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return Option.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();
}
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).textValue();
}
public String getRowKey(String keyColumnField) throws IOException {
return getFieldFromJsonOrFail(keyColumnField);
}
public String getPartitionPath(String partitionPathField) throws IOException {
return getFieldFromJsonOrFail(partitionPathField);
}
}

View File

@@ -0,0 +1,99 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common;
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;
}
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 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);
}
}
}

View File

@@ -0,0 +1,60 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import org.apache.hadoop.conf.Configuration;
public class SerializableConfiguration implements Serializable {
private transient Configuration configuration;
public SerializableConfiguration(Configuration configuration) {
this.configuration = new Configuration(configuration);
}
public SerializableConfiguration(SerializableConfiguration configuration) {
this.configuration = configuration.get();
}
public Configuration get() {
return new Configuration(configuration);
}
private void writeObject(ObjectOutputStream out) throws IOException {
out.defaultWriteObject();
configuration.write(out);
}
private void readObject(ObjectInputStream in) throws IOException {
configuration = new Configuration(false);
configuration.readFields(in);
}
@Override
public String toString() {
StringBuilder str = new StringBuilder();
configuration.iterator().forEachRemaining(e ->
str.append(String.format("%s => %s \n", e.getKey(), e.getValue())));
return configuration.toString();
}
}

View File

@@ -0,0 +1,909 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.io.storage;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.FsStatus;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable;
import org.apache.hudi.common.storage.StorageSchemes;
import org.apache.hudi.common.util.ConsistencyGuard;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.NoOpConsistencyGuard;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
/**
* HoodieWrapperFileSystem wraps the default file system. It holds state about the open streams in
* the file system to support getting the written size to each of the open streams.
*/
public class HoodieWrapperFileSystem extends FileSystem {
public static final String HOODIE_SCHEME_PREFIX = "hoodie-";
private ConcurrentMap<String, SizeAwareFSDataOutputStream> openStreams = new
ConcurrentHashMap<>();
private FileSystem fileSystem;
private URI uri;
private ConsistencyGuard consistencyGuard = new NoOpConsistencyGuard();
public HoodieWrapperFileSystem() {
}
public HoodieWrapperFileSystem(FileSystem fileSystem, ConsistencyGuard consistencyGuard) {
this.fileSystem = fileSystem;
this.uri = fileSystem.getUri();
this.consistencyGuard = consistencyGuard;
}
public static Path convertToHoodiePath(Path file, Configuration conf) {
try {
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
return convertPathWithScheme(file, getHoodieScheme(scheme));
} catch (HoodieIOException e) {
throw e;
}
}
private static Path convertPathWithScheme(Path oldPath, String newScheme) {
URI oldURI = oldPath.toUri();
URI newURI;
try {
newURI = new URI(newScheme, oldURI.getUserInfo(), oldURI.getHost(), oldURI.getPort(),
oldURI.getPath(), oldURI.getQuery(), oldURI.getFragment());
return new Path(newURI);
} catch (URISyntaxException e) {
// TODO - Better Exception handling
throw new RuntimeException(e);
}
}
public static String getHoodieScheme(String scheme) {
String newScheme;
if (StorageSchemes.isSchemeSupported(scheme)) {
newScheme = HOODIE_SCHEME_PREFIX + scheme;
} else {
throw new IllegalArgumentException(
"BlockAlignedAvroParquetWriter does not support scheme " + scheme);
}
return newScheme;
}
@Override
public void initialize(URI uri, Configuration conf) throws IOException {
// Get the default filesystem to decorate
Path path = new Path(uri);
// Remove 'hoodie-' prefix from path
if (path.toString().startsWith(HOODIE_SCHEME_PREFIX)) {
path = new Path(path.toString().replace(HOODIE_SCHEME_PREFIX, ""));
this.uri = path.toUri();
} else {
this.uri = uri;
}
this.fileSystem = FSUtils.getFs(path.toString(), conf);
// Do not need to explicitly initialize the default filesystem, its done already in the above
// FileSystem.get
// fileSystem.initialize(FileSystem.getDefaultUri(conf), conf);
// fileSystem.setConf(conf);
}
@Override
public URI getUri() {
return uri;
}
@Override
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
return fileSystem.open(convertToDefaultPath(f), bufferSize);
}
@Override
public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite,
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
final Path translatedPath = convertToDefaultPath(f);
return wrapOutputStream(f, fileSystem
.create(translatedPath, permission, overwrite, bufferSize, replication, blockSize,
progress));
}
private FSDataOutputStream wrapOutputStream(final Path path,
FSDataOutputStream fsDataOutputStream) throws IOException {
if (fsDataOutputStream instanceof SizeAwareFSDataOutputStream) {
return fsDataOutputStream;
}
SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(path,
fsDataOutputStream, consistencyGuard, () -> openStreams.remove(path.getName()));
openStreams.put(path.getName(), os);
return os;
}
@Override
public FSDataOutputStream create(Path f, boolean overwrite) throws IOException {
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite));
}
@Override
public FSDataOutputStream create(Path f) throws IOException {
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f)));
}
@Override
public FSDataOutputStream create(Path f, Progressable progress) throws IOException {
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), progress));
}
@Override
public FSDataOutputStream create(Path f, short replication) throws IOException {
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication));
}
@Override
public FSDataOutputStream create(Path f, short replication, Progressable progress)
throws IOException {
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication, progress));
}
@Override
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException {
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize));
}
@Override
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress)
throws IOException {
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress));
}
@Override
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication,
long blockSize, Progressable progress) throws IOException {
return wrapOutputStream(f, fileSystem
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress));
}
@Override
public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags,
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
return wrapOutputStream(f, fileSystem
.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize,
progress));
}
@Override
public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags,
int bufferSize, short replication, long blockSize, Progressable progress,
Options.ChecksumOpt checksumOpt) throws IOException {
return wrapOutputStream(f, fileSystem
.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize,
progress, checksumOpt));
}
@Override
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication,
long blockSize) throws IOException {
return wrapOutputStream(f, fileSystem
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize));
}
@Override
public FSDataOutputStream append(Path f, int bufferSize, Progressable progress)
throws IOException {
return wrapOutputStream(f, fileSystem.append(convertToDefaultPath(f), bufferSize, progress));
}
@Override
public boolean rename(Path src, Path dst) throws IOException {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(src));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + src + " to appear", e);
}
boolean success = fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst));
if (success) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + dst + " to appear", e);
}
try {
consistencyGuard.waitTillFileDisappears(convertToDefaultPath(src));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + src + " to disappear", e);
}
}
return success;
}
@Override
public boolean delete(Path f, boolean recursive) throws IOException {
boolean success = fileSystem.delete(convertToDefaultPath(f), recursive);
if (success) {
try {
consistencyGuard.waitTillFileDisappears(f);
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + f + " to disappear", e);
}
}
return success;
}
@Override
public FileStatus[] listStatus(Path f) throws IOException {
return fileSystem.listStatus(convertToDefaultPath(f));
}
@Override
public Path getWorkingDirectory() {
return convertToHoodiePath(fileSystem.getWorkingDirectory());
}
@Override
public void setWorkingDirectory(Path newDir) {
fileSystem.setWorkingDirectory(convertToDefaultPath(newDir));
}
@Override
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
boolean success = fileSystem.mkdirs(convertToDefaultPath(f), permission);
if (success) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for directory " + f + " to appear", e);
}
}
return success;
}
@Override
public FileStatus getFileStatus(Path f) throws IOException {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
} catch (TimeoutException e) {
// pass
}
return fileSystem.getFileStatus(convertToDefaultPath(f));
}
@Override
public String getScheme() {
return uri.getScheme();
}
@Override
public String getCanonicalServiceName() {
return fileSystem.getCanonicalServiceName();
}
@Override
public String getName() {
return fileSystem.getName();
}
@Override
public Path makeQualified(Path path) {
return convertToHoodiePath(fileSystem.makeQualified(convertToDefaultPath(path)));
}
@Override
public Token<?> getDelegationToken(String renewer) throws IOException {
return fileSystem.getDelegationToken(renewer);
}
@Override
public Token<?>[] addDelegationTokens(String renewer, Credentials credentials)
throws IOException {
return fileSystem.addDelegationTokens(renewer, credentials);
}
@Override
public FileSystem[] getChildFileSystems() {
return fileSystem.getChildFileSystems();
}
@Override
public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len)
throws IOException {
return fileSystem.getFileBlockLocations(file, start, len);
}
@Override
public BlockLocation[] getFileBlockLocations(Path p, long start, long len) throws IOException {
return fileSystem.getFileBlockLocations(convertToDefaultPath(p), start, len);
}
@Override
public FsServerDefaults getServerDefaults() throws IOException {
return fileSystem.getServerDefaults();
}
@Override
public FsServerDefaults getServerDefaults(Path p) throws IOException {
return fileSystem.getServerDefaults(convertToDefaultPath(p));
}
@Override
public Path resolvePath(Path p) throws IOException {
return convertToHoodiePath(fileSystem.resolvePath(convertToDefaultPath(p)));
}
@Override
public FSDataInputStream open(Path f) throws IOException {
return fileSystem.open(convertToDefaultPath(f));
}
@Override
public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize,
short replication, long blockSize, Progressable progress) throws IOException {
Path p = convertToDefaultPath(f);
return wrapOutputStream(p, fileSystem.createNonRecursive(p, overwrite, bufferSize, replication, blockSize,
progress));
}
@Override
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite,
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
Path p = convertToDefaultPath(f);
return wrapOutputStream(p, fileSystem.createNonRecursive(p, permission, overwrite, bufferSize, replication,
blockSize, progress));
}
@Override
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
Progressable progress) throws IOException {
Path p = convertToDefaultPath(f);
return wrapOutputStream(p, fileSystem.createNonRecursive(p, permission, flags, bufferSize, replication,
blockSize, progress));
}
@Override
public boolean createNewFile(Path f) throws IOException {
boolean newFile = fileSystem.createNewFile(convertToDefaultPath(f));
if (newFile) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + f + " to appear", e);
}
}
return newFile;
}
@Override
public FSDataOutputStream append(Path f) throws IOException {
return wrapOutputStream(f, fileSystem.append(convertToDefaultPath(f)));
}
@Override
public FSDataOutputStream append(Path f, int bufferSize) throws IOException {
return wrapOutputStream(f, fileSystem.append(convertToDefaultPath(f), bufferSize));
}
@Override
public void concat(Path trg, Path[] psrcs) throws IOException {
Path[] psrcsNew = convertDefaults(psrcs);
fileSystem.concat(convertToDefaultPath(trg), psrcsNew);
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(trg));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + trg + " to appear", e);
}
}
@Override
public short getReplication(Path src) throws IOException {
return fileSystem.getReplication(convertToDefaultPath(src));
}
@Override
public boolean setReplication(Path src, short replication) throws IOException {
return fileSystem.setReplication(convertToDefaultPath(src), replication);
}
@Override
public boolean delete(Path f) throws IOException {
return delete(f, true);
}
@Override
public boolean deleteOnExit(Path f) throws IOException {
return fileSystem.deleteOnExit(convertToDefaultPath(f));
}
@Override
public boolean cancelDeleteOnExit(Path f) {
return fileSystem.cancelDeleteOnExit(convertToDefaultPath(f));
}
@Override
public boolean exists(Path f) throws IOException {
return fileSystem.exists(convertToDefaultPath(f));
}
@Override
public boolean isDirectory(Path f) throws IOException {
return fileSystem.isDirectory(convertToDefaultPath(f));
}
@Override
public boolean isFile(Path f) throws IOException {
return fileSystem.isFile(convertToDefaultPath(f));
}
@Override
public long getLength(Path f) throws IOException {
return fileSystem.getLength(convertToDefaultPath(f));
}
@Override
public ContentSummary getContentSummary(Path f) throws IOException {
return fileSystem.getContentSummary(convertToDefaultPath(f));
}
@Override
public RemoteIterator<Path> listCorruptFileBlocks(Path path) throws IOException {
return fileSystem.listCorruptFileBlocks(convertToDefaultPath(path));
}
@Override
public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException {
return fileSystem.listStatus(convertToDefaultPath(f), filter);
}
@Override
public FileStatus[] listStatus(Path[] files) throws IOException {
return fileSystem.listStatus(convertDefaults(files));
}
@Override
public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException {
return fileSystem.listStatus(convertDefaults(files), filter);
}
@Override
public FileStatus[] globStatus(Path pathPattern) throws IOException {
return fileSystem.globStatus(convertToDefaultPath(pathPattern));
}
@Override
public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException {
return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter);
}
@Override
public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f) throws IOException {
return fileSystem.listLocatedStatus(convertToDefaultPath(f));
}
@Override
public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive) throws IOException {
return fileSystem.listFiles(convertToDefaultPath(f), recursive);
}
@Override
public Path getHomeDirectory() {
return convertToHoodiePath(fileSystem.getHomeDirectory());
}
@Override
public boolean mkdirs(Path f) throws IOException {
boolean success = fileSystem.mkdirs(convertToDefaultPath(f));
if (success) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for directory " + f + " to appear", e);
}
}
return success;
}
@Override
public void copyFromLocalFile(Path src, Path dst) throws IOException {
fileSystem.copyFromLocalFile(convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void moveFromLocalFile(Path[] srcs, Path dst) throws IOException {
fileSystem.moveFromLocalFile(convertLocalPaths(srcs), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void moveFromLocalFile(Path src, Path dst) throws IOException {
fileSystem.moveFromLocalFile(convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
fileSystem.copyFromLocalFile(delSrc, convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Path dst)
throws IOException {
fileSystem
.copyFromLocalFile(delSrc, overwrite, convertLocalPaths(srcs), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst)
throws IOException {
fileSystem
.copyFromLocalFile(delSrc, overwrite, convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void copyToLocalFile(Path src, Path dst) throws IOException {
fileSystem.copyToLocalFile(convertToDefaultPath(src), convertToLocalPath(dst));
}
@Override
public void moveToLocalFile(Path src, Path dst) throws IOException {
fileSystem.moveToLocalFile(convertToDefaultPath(src), convertToLocalPath(dst));
}
@Override
public void copyToLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToLocalPath(dst));
}
@Override
public void copyToLocalFile(boolean delSrc, Path src, Path dst, boolean useRawLocalFileSystem)
throws IOException {
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToLocalPath(dst),
useRawLocalFileSystem);
}
@Override
public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException {
return convertToHoodiePath(fileSystem
.startLocalOutput(convertToDefaultPath(fsOutputFile), convertToDefaultPath(tmpLocalFile)));
}
@Override
public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException {
fileSystem.completeLocalOutput(convertToDefaultPath(fsOutputFile),
convertToDefaultPath(tmpLocalFile));
}
@Override
public void close() throws IOException {
// Don't close the wrapped `fileSystem` object. This will end up closing it for every thread since it
// could be cached across jvm. We don't own that object anyway.
super.close();
}
@Override
public long getUsed() throws IOException {
return fileSystem.getUsed();
}
@Override
public long getBlockSize(Path f) throws IOException {
return fileSystem.getBlockSize(convertToDefaultPath(f));
}
@Override
public long getDefaultBlockSize() {
return fileSystem.getDefaultBlockSize();
}
@Override
public long getDefaultBlockSize(Path f) {
return fileSystem.getDefaultBlockSize(convertToDefaultPath(f));
}
@Override
public short getDefaultReplication() {
return fileSystem.getDefaultReplication();
}
@Override
public short getDefaultReplication(Path path) {
return fileSystem.getDefaultReplication(convertToDefaultPath(path));
}
@Override
public void access(Path path, FsAction mode) throws IOException {
fileSystem.access(convertToDefaultPath(path), mode);
}
@Override
public void createSymlink(Path target, Path link, boolean createParent) throws IOException {
fileSystem
.createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent);
}
@Override
public FileStatus getFileLinkStatus(Path f) throws IOException {
return fileSystem.getFileLinkStatus(convertToDefaultPath(f));
}
@Override
public boolean supportsSymlinks() {
return fileSystem.supportsSymlinks();
}
@Override
public Path getLinkTarget(Path f) throws IOException {
return convertToHoodiePath(fileSystem.getLinkTarget(convertToDefaultPath(f)));
}
@Override
public FileChecksum getFileChecksum(Path f) throws IOException {
return fileSystem.getFileChecksum(convertToDefaultPath(f));
}
@Override
public FileChecksum getFileChecksum(Path f, long length) throws IOException {
return fileSystem.getFileChecksum(convertToDefaultPath(f), length);
}
@Override
public void setVerifyChecksum(boolean verifyChecksum) {
fileSystem.setVerifyChecksum(verifyChecksum);
}
@Override
public void setWriteChecksum(boolean writeChecksum) {
fileSystem.setWriteChecksum(writeChecksum);
}
@Override
public FsStatus getStatus() throws IOException {
return fileSystem.getStatus();
}
@Override
public FsStatus getStatus(Path p) throws IOException {
return fileSystem.getStatus(convertToDefaultPath(p));
}
@Override
public void setPermission(Path p, FsPermission permission) throws IOException {
fileSystem.setPermission(convertToDefaultPath(p), permission);
}
@Override
public void setOwner(Path p, String username, String groupname) throws IOException {
fileSystem.setOwner(convertToDefaultPath(p), username, groupname);
}
@Override
public void setTimes(Path p, long mtime, long atime) throws IOException {
fileSystem.setTimes(convertToDefaultPath(p), mtime, atime);
}
@Override
public Path createSnapshot(Path path, String snapshotName) throws IOException {
return convertToHoodiePath(fileSystem.createSnapshot(convertToDefaultPath(path), snapshotName));
}
@Override
public void renameSnapshot(Path path, String snapshotOldName, String snapshotNewName)
throws IOException {
fileSystem.renameSnapshot(convertToDefaultPath(path), snapshotOldName, snapshotNewName);
}
@Override
public void deleteSnapshot(Path path, String snapshotName) throws IOException {
fileSystem.deleteSnapshot(convertToDefaultPath(path), snapshotName);
}
@Override
public void modifyAclEntries(Path path, List<AclEntry> aclSpec) throws IOException {
fileSystem.modifyAclEntries(convertToDefaultPath(path), aclSpec);
}
@Override
public void removeAclEntries(Path path, List<AclEntry> aclSpec) throws IOException {
fileSystem.removeAclEntries(convertToDefaultPath(path), aclSpec);
}
@Override
public void removeDefaultAcl(Path path) throws IOException {
fileSystem.removeDefaultAcl(convertToDefaultPath(path));
}
@Override
public void removeAcl(Path path) throws IOException {
fileSystem.removeAcl(convertToDefaultPath(path));
}
@Override
public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException {
fileSystem.setAcl(convertToDefaultPath(path), aclSpec);
}
@Override
public AclStatus getAclStatus(Path path) throws IOException {
return fileSystem.getAclStatus(convertToDefaultPath(path));
}
@Override
public void setXAttr(Path path, String name, byte[] value) throws IOException {
fileSystem.setXAttr(convertToDefaultPath(path), name, value);
}
@Override
public void setXAttr(Path path, String name, byte[] value, EnumSet<XAttrSetFlag> flag)
throws IOException {
fileSystem.setXAttr(convertToDefaultPath(path), name, value, flag);
}
@Override
public byte[] getXAttr(Path path, String name) throws IOException {
return fileSystem.getXAttr(convertToDefaultPath(path), name);
}
@Override
public Map<String, byte[]> getXAttrs(Path path) throws IOException {
return fileSystem.getXAttrs(convertToDefaultPath(path));
}
@Override
public Map<String, byte[]> getXAttrs(Path path, List<String> names) throws IOException {
return fileSystem.getXAttrs(convertToDefaultPath(path), names);
}
@Override
public List<String> listXAttrs(Path path) throws IOException {
return fileSystem.listXAttrs(convertToDefaultPath(path));
}
@Override
public void removeXAttr(Path path, String name) throws IOException {
fileSystem.removeXAttr(convertToDefaultPath(path), name);
}
@Override
public Configuration getConf() {
return fileSystem.getConf();
}
@Override
public void setConf(Configuration conf) {
// ignore this. we will set conf on init
}
@Override
public int hashCode() {
return fileSystem.hashCode();
}
@Override
public boolean equals(Object obj) {
return fileSystem.equals(obj);
}
@Override
public String toString() {
return fileSystem.toString();
}
public Path convertToHoodiePath(Path oldPath) {
return convertPathWithScheme(oldPath, getHoodieScheme(getScheme()));
}
private Path convertToDefaultPath(Path oldPath) {
return convertPathWithScheme(oldPath, getScheme());
}
private Path convertToLocalPath(Path oldPath) {
try {
return convertPathWithScheme(oldPath, FileSystem.getLocal(getConf()).getScheme());
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
private Path[] convertLocalPaths(Path[] psrcs) {
Path[] psrcsNew = new Path[psrcs.length];
for (int i = 0; i < psrcs.length; i++) {
psrcsNew[i] = convertToLocalPath(psrcs[i]);
}
return psrcsNew;
}
private Path[] convertDefaults(Path[] psrcs) {
Path[] psrcsNew = new Path[psrcs.length];
for (int i = 0; i < psrcs.length; i++) {
psrcsNew[i] = convertToDefaultPath(psrcs[i]);
}
return psrcsNew;
}
public long getBytesWritten(Path file) {
if (openStreams.containsKey(file.getName())) {
return openStreams.get(file.getName()).getBytesWritten();
}
// When the file is first written, we do not have a track of it
throw new IllegalArgumentException(file.toString()
+ " does not have a open stream. Cannot get the bytes written on the stream");
}
public FileSystem getFileSystem() {
return fileSystem;
}
}

View File

@@ -0,0 +1,79 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.io.storage;
import java.io.IOException;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.util.ConsistencyGuard;
import org.apache.hudi.exception.HoodieException;
/**
* Wrapper over <code>FSDataOutputStream</code> to keep track of the size of the written bytes. This
* gives a cheap way to check on the underlying file size.
*/
public class SizeAwareFSDataOutputStream extends FSDataOutputStream {
// A callback to call when the output stream is closed.
private final Runnable closeCallback;
// Keep track of the bytes written
private final AtomicLong bytesWritten = new AtomicLong(0L);
// Path
private final Path path;
// Consistency guard
private final ConsistencyGuard consistencyGuard;
public SizeAwareFSDataOutputStream(Path path, FSDataOutputStream out,
ConsistencyGuard consistencyGuard, Runnable closeCallback) throws IOException {
super(out);
this.path = path;
this.closeCallback = closeCallback;
this.consistencyGuard = consistencyGuard;
}
@Override
public synchronized void write(byte[] b, int off, int len) throws IOException {
bytesWritten.addAndGet(len);
super.write(b, off, len);
}
@Override
public void write(byte[] b) throws IOException {
bytesWritten.addAndGet(b.length);
super.write(b);
}
@Override
public void close() throws IOException {
super.close();
try {
consistencyGuard.waitTillFileAppears(path);
} catch (TimeoutException e) {
throw new HoodieException(e);
}
closeCallback.run();
}
public long getBytesWritten() {
return bytesWritten.get();
}
}

View File

@@ -0,0 +1,23 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
public enum ActionType {
commit, savepoint, compaction, clean, rollback
}

View File

@@ -0,0 +1,169 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
/**
* Encapsulates all the needed information about a compaction and make a decision whether this
* compaction is effective or not
*
*/
public class CompactionOperation implements Serializable {
private String baseInstantTime;
private Option<String> dataFileCommitTime;
private List<String> deltaFilePaths;
private Option<String> dataFilePath;
private HoodieFileGroupId id;
private Map<String, Double> metrics;
//Only for serialization/de-serialization
@Deprecated
public CompactionOperation() {
}
public CompactionOperation(String fileId, String partitionPath, String baseInstantTime,
Option<String> dataFileCommitTime, List<String> deltaFilePaths, Option<String> dataFilePath,
Map<String, Double> metrics) {
this.baseInstantTime = baseInstantTime;
this.dataFileCommitTime = dataFileCommitTime;
this.deltaFilePaths = deltaFilePaths;
this.dataFilePath = dataFilePath;
this.id = new HoodieFileGroupId(partitionPath, fileId);
this.metrics = metrics;
}
public CompactionOperation(Option<HoodieDataFile> dataFile, String partitionPath,
List<HoodieLogFile> logFiles, Map<String, Double> metrics) {
if (dataFile.isPresent()) {
this.baseInstantTime = dataFile.get().getCommitTime();
this.dataFilePath = Option.of(dataFile.get().getPath());
this.id = new HoodieFileGroupId(partitionPath, dataFile.get().getFileId());
this.dataFileCommitTime = Option.of(dataFile.get().getCommitTime());
} else {
assert logFiles.size() > 0;
this.dataFilePath = Option.empty();
this.baseInstantTime = FSUtils.getBaseCommitTimeFromLogPath(logFiles.get(0).getPath());
this.id = new HoodieFileGroupId(partitionPath, FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath()));
this.dataFileCommitTime = Option.empty();
}
this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString())
.collect(Collectors.toList());
this.metrics = metrics;
}
public String getBaseInstantTime() {
return baseInstantTime;
}
public Option<String> getDataFileCommitTime() {
return dataFileCommitTime;
}
public List<String> getDeltaFilePaths() {
return deltaFilePaths;
}
public Option<String> getDataFilePath() {
return dataFilePath;
}
public String getFileId() {
return id.getFileId();
}
public String getPartitionPath() {
return id.getPartitionPath();
}
public Map<String, Double> getMetrics() {
return metrics;
}
public HoodieFileGroupId getFileGroupId() {
return id;
}
public Option<HoodieDataFile> getBaseFile() {
//TODO: HUDI-130 - Paths return in compaction plan needs to be relative to base-path
return dataFilePath.map(df -> new HoodieDataFile(df));
}
/**
* Convert Avro generated Compaction operation to POJO for Spark RDD operation
* @param operation Hoodie Compaction Operation
* @return
*/
public static CompactionOperation convertFromAvroRecordInstance(HoodieCompactionOperation operation) {
CompactionOperation op = new CompactionOperation();
op.baseInstantTime = operation.getBaseInstantTime();
op.dataFilePath = Option.ofNullable(operation.getDataFilePath());
op.dataFileCommitTime =
op.dataFilePath.map(p -> FSUtils.getCommitTime(new Path(p).getName()));
op.deltaFilePaths = new ArrayList<>(operation.getDeltaFilePaths());
op.id = new HoodieFileGroupId(operation.getPartitionPath(), operation.getFileId());
op.metrics = operation.getMetrics() == null ? new HashMap<>() : new HashMap<>(operation.getMetrics());
return op;
}
@Override
public String toString() {
return "CompactionOperation{"
+ "baseInstantTime='" + baseInstantTime + '\''
+ ", dataFileCommitTime=" + dataFileCommitTime
+ ", deltaFilePaths=" + deltaFilePaths
+ ", dataFilePath=" + dataFilePath
+ ", id='" + id + '\''
+ ", metrics=" + metrics
+ '}';
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
CompactionOperation operation = (CompactionOperation) o;
return Objects.equals(baseInstantTime, operation.baseInstantTime)
&& Objects.equals(dataFileCommitTime, operation.dataFileCommitTime)
&& Objects.equals(deltaFilePaths, operation.deltaFilePaths)
&& Objects.equals(dataFilePath, operation.dataFilePath)
&& Objects.equals(id, operation.id);
}
@Override
public int hashCode() {
return Objects.hash(baseInstantTime, id);
}
}

View File

@@ -0,0 +1,139 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import java.io.Serializable;
import java.util.Objects;
import java.util.TreeSet;
import java.util.stream.Stream;
import org.apache.hudi.common.util.Option;
/**
* 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 {
/**
* File Group Id of the Slice
*/
private HoodieFileGroupId fileGroupId;
/**
* Point in the timeline, at which the slice was created
*/
private String baseInstantTime;
/**
* 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;
public FileSlice(String partitionPath, String baseInstantTime, String fileId) {
this(new HoodieFileGroupId(partitionPath, fileId), baseInstantTime);
}
public FileSlice(HoodieFileGroupId fileGroupId, String baseInstantTime) {
this.fileGroupId = fileGroupId;
this.baseInstantTime = baseInstantTime;
this.dataFile = null;
this.logFiles = new TreeSet<>(HoodieLogFile.getReverseLogFileComparator());
}
public void setDataFile(HoodieDataFile dataFile) {
this.dataFile = dataFile;
}
public void addLogFile(HoodieLogFile logFile) {
this.logFiles.add(logFile);
}
public Stream<HoodieLogFile> getLogFiles() {
return logFiles.stream();
}
public String getBaseInstantTime() {
return baseInstantTime;
}
public String getPartitionPath() {
return fileGroupId.getPartitionPath();
}
public String getFileId() {
return fileGroupId.getFileId();
}
public HoodieFileGroupId getFileGroupId() {
return fileGroupId;
}
public Option<HoodieDataFile> getDataFile() {
return Option.ofNullable(dataFile);
}
public Option<HoodieLogFile> getLatestLogFile() {
return Option.fromJavaOptional(logFiles.stream().findFirst());
}
/**
* Returns true if there is no data file and no log files. Happens as part of pending compaction
* @return
*/
public boolean isEmpty() {
return (dataFile == null) && (logFiles.isEmpty());
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("FileSlice {");
sb.append("fileGroupId=").append(fileGroupId);
sb.append(", baseCommitTime=").append(baseInstantTime);
sb.append(", dataFile='").append(dataFile).append('\'');
sb.append(", logFiles='").append(logFiles).append('\'');
sb.append('}');
return sb.toString();
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
FileSlice slice = (FileSlice) o;
return Objects.equals(fileGroupId, slice.fileGroupId)
&& Objects.equals(baseInstantTime, slice.baseInstantTime)
&& Objects.equals(dataFile, slice.dataFile)
&& Objects.equals(logFiles, slice.logFiles);
}
@Override
public int hashCode() {
return Objects.hash(fileGroupId, baseInstantTime);
}
}

View File

@@ -0,0 +1,40 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
public class HoodieArchivedLogFile extends HoodieLogFile {
public static final String ARCHIVE_EXTENSION = ".archive";
public HoodieArchivedLogFile(FileStatus fileStatus) {
super(fileStatus);
}
public HoodieArchivedLogFile(Path logPath) {
super(logPath);
}
@Override
public String toString() {
return "HoodieArchivedLogFile {" + super.getPath() + '}';
}
}

View File

@@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
/**
* 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> {
// Store the GenericRecord converted to bytes - 1) Doesn't store schema hence memory efficient 2) Makes the payload
// java serializable
private final byte [] recordBytes;
public HoodieAvroPayload(Option<GenericRecord> record) {
try {
if (record.isPresent()) {
this.recordBytes = HoodieAvroUtils.avroToBytes(record.get());
} else {
this.recordBytes = new byte[0];
}
} catch (IOException io) {
throw new HoodieIOException("Cannot convert record to bytes", io);
}
}
@Override
public HoodieAvroPayload preCombine(HoodieAvroPayload another) {
return this;
}
@Override
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException {
return getInsertValue(schema);
}
@Override
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
if (recordBytes.length == 0) {
return Option.empty();
}
return Option.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
}
}

View File

@@ -0,0 +1,24 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
public enum HoodieCleaningPolicy {
KEEP_LATEST_FILE_VERSIONS,
KEEP_LATEST_COMMITS
}

View File

@@ -0,0 +1,347 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.PropertyAccessor;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.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;
import org.apache.hudi.common.util.FSUtils;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* 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;
protected Boolean compacted;
private Map<String, String> extraMetadataMap;
// for ser/deser
public HoodieCommitMetadata() {
this(false);
}
public HoodieCommitMetadata(boolean compacted) {
extraMetadataMap = new HashMap<>();
partitionToWriteStats = new HashMap<>();
this.compacted = compacted;
}
public void addWriteStat(String partitionPath, HoodieWriteStat stat) {
if (!partitionToWriteStats.containsKey(partitionPath)) {
partitionToWriteStats.put(partitionPath, new ArrayList<>());
}
partitionToWriteStats.get(partitionPath).add(stat);
}
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 Boolean getCompacted() {
return compacted;
}
public void setCompacted(Boolean compacted) {
this.compacted = compacted;
}
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) ? (FSUtils.getPartitionPath(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);
}
return getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this);
}
public static <T> T fromJsonString(String jsonStr, Class<T> clazz) throws Exception {
if (jsonStr == null || jsonStr.isEmpty()) {
// For empty commit file (no data or somethings bad happen).
return clazz.newInstance();
}
return getObjectMapper().readValue(jsonStr, clazz);
}
// 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().equalsIgnoreCase("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().equalsIgnoreCase("null")) {
totalFilesUpdated++;
}
}
}
return totalFilesUpdated;
}
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().equalsIgnoreCase("null")) {
totalInsertRecordsWritten += stat.getNumInserts();
}
}
}
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;
}
public long getTotalRecordsDeleted() {
long totalDeletes = 0;
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
for (HoodieWriteStat stat : stats) {
totalDeletes += stat.getNumDeletes();
}
}
return totalDeletes;
}
public Long getTotalLogRecordsCompacted() {
Long totalLogRecords = 0L;
for (Map.Entry<String, List<HoodieWriteStat>> entry : partitionToWriteStats.entrySet()) {
for (HoodieWriteStat writeStat : entry.getValue()) {
totalLogRecords += writeStat.getTotalLogRecords();
}
}
return totalLogRecords;
}
public Long getTotalLogFilesCompacted() {
Long totalLogFiles = 0L;
for (Map.Entry<String, List<HoodieWriteStat>> entry : partitionToWriteStats.entrySet()) {
for (HoodieWriteStat writeStat : entry.getValue()) {
totalLogFiles += writeStat.getTotalLogFilesCompacted();
}
}
return totalLogFiles;
}
public Long getTotalCompactedRecordsUpdated() {
Long totalUpdateRecords = 0L;
for (Map.Entry<String, List<HoodieWriteStat>> entry : partitionToWriteStats.entrySet()) {
for (HoodieWriteStat writeStat : entry.getValue()) {
totalUpdateRecords += writeStat.getTotalUpdatedRecordsCompacted();
}
}
return totalUpdateRecords;
}
public Long getTotalLogFilesSize() {
Long totalLogFilesSize = 0L;
for (Map.Entry<String, List<HoodieWriteStat>> entry : partitionToWriteStats.entrySet()) {
for (HoodieWriteStat writeStat : entry.getValue()) {
totalLogFilesSize += writeStat.getTotalLogSizeCompacted();
}
}
return totalLogFilesSize;
}
public Long getTotalScanTime() {
Long totalScanTime = 0L;
for (Map.Entry<String, List<HoodieWriteStat>> entry : partitionToWriteStats.entrySet()) {
for (HoodieWriteStat writeStat : entry.getValue()) {
if (writeStat.getRuntimeStats() != null) {
totalScanTime += writeStat.getRuntimeStats().getTotalScanTime();
}
}
}
return totalScanTime;
}
public Long getTotalCreateTime() {
Long totalCreateTime = 0L;
for (Map.Entry<String, List<HoodieWriteStat>> entry : partitionToWriteStats.entrySet()) {
for (HoodieWriteStat writeStat : entry.getValue()) {
if (writeStat.getRuntimeStats() != null) {
totalCreateTime += writeStat.getRuntimeStats().getTotalCreateTime();
}
}
}
return totalCreateTime;
}
public Long getTotalUpsertTime() {
Long totalUpsertTime = 0L;
for (Map.Entry<String, List<HoodieWriteStat>> entry : partitionToWriteStats.entrySet()) {
for (HoodieWriteStat writeStat : entry.getValue()) {
if (writeStat.getRuntimeStats() != null) {
totalUpsertTime += writeStat.getRuntimeStats().getTotalUpsertTime();
}
}
}
return totalUpsertTime;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieCommitMetadata that = (HoodieCommitMetadata) o;
if (!partitionToWriteStats.equals(that.partitionToWriteStats)) {
return false;
}
return compacted.equals(that.compacted);
}
@Override
public int hashCode() {
int result = partitionToWriteStats.hashCode();
result = 31 * result + compacted.hashCode();
return result;
}
public static <T> T fromBytes(byte[] bytes, Class<T> clazz) throws IOException {
try {
return fromJsonString(new String(bytes, Charset.forName("utf-8")), clazz);
} catch (Exception e) {
throw new IOException("unable to read commit metadata", e);
}
}
protected static ObjectMapper getObjectMapper() {
ObjectMapper mapper = new ObjectMapper();
mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES);
mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY);
return mapper;
}
@Override
public String toString() {
return "HoodieCommitMetadata{"
+ "partitionToWriteStats=" + partitionToWriteStats
+ ", compacted=" + compacted
+ ", extraMetadataMap=" + extraMetadataMap
+ '}';
}
}

View File

@@ -0,0 +1,101 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import java.io.Serializable;
import java.util.Objects;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.util.FSUtils;
public class HoodieDataFile implements Serializable {
private transient FileStatus fileStatus;
private final String fullPath;
private long fileLen;
public HoodieDataFile(FileStatus fileStatus) {
this.fileStatus = fileStatus;
this.fullPath = fileStatus.getPath().toString();
this.fileLen = fileStatus.getLen();
}
public HoodieDataFile(String filePath) {
this.fileStatus = null;
this.fullPath = filePath;
this.fileLen = -1;
}
public String getFileId() {
return FSUtils.getFileId(getFileName());
}
public String getCommitTime() {
return FSUtils.getCommitTime(getFileName());
}
public String getPath() {
return fullPath;
}
public String getFileName() {
return new Path(fullPath).getName();
}
public FileStatus getFileStatus() {
return fileStatus;
}
public long getFileSize() {
return fileLen;
}
public void setFileLen(long fileLen) {
this.fileLen = fileLen;
}
public long getFileLen() {
return fileLen;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieDataFile dataFile = (HoodieDataFile) o;
return Objects.equals(fullPath, dataFile.fullPath);
}
@Override
public int hashCode() {
return Objects.hash(fullPath);
}
@Override
public String toString() {
return "HoodieDataFile{"
+ "fullPath=" + fullPath
+ ", fileLen=" + fileLen
+ '}';
}
}

View File

@@ -0,0 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
/**
* Statistics about a single Hoodie delta log operation.
*/
@JsonIgnoreProperties(ignoreUnknown = true)
public class HoodieDeltaWriteStat extends HoodieWriteStat {
private int logVersion;
private long logOffset;
public void setLogVersion(int logVersion) {
this.logVersion = logVersion;
}
public int getLogVersion() {
return logVersion;
}
public void setLogOffset(long logOffset) {
this.logOffset = logOffset;
}
public long getLogOffset() {
return logOffset;
}
}

View File

@@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
public enum HoodieFileFormat {
PARQUET(".parquet"), HOODIE_LOG(".log");
private final String extension;
HoodieFileFormat(String extension) {
this.extension = extension;
}
public String getFileExtension() {
return extension;
}
}

View File

@@ -0,0 +1,227 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import java.io.Serializable;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.stream.Stream;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
/**
* A set of data/base files + set of log files, that make up an unit for all operations
*/
public class HoodieFileGroup implements Serializable {
public static Comparator<String> getReverseCommitTimeComparator() {
return Comparator.reverseOrder();
}
/**
* file group id
*/
private final HoodieFileGroupId fileGroupId;
/**
* 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 Option<HoodieInstant> lastInstant;
public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) {
this(new HoodieFileGroupId(partitionPath, id), timeline);
}
public HoodieFileGroup(HoodieFileGroupId fileGroupId, HoodieTimeline timeline) {
this.fileGroupId = fileGroupId;
this.fileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator());
this.timeline = timeline;
this.lastInstant = timeline.lastInstant();
}
/**
* Potentially add a new file-slice by adding base-instant time
* A file-slice without any data-file and log-files can exist (if a compaction just got requested)
*/
public void addNewFileSliceAtInstant(String baseInstantTime) {
if (!fileSlices.containsKey(baseInstantTime)) {
fileSlices.put(baseInstantTime, new FileSlice(fileGroupId, baseInstantTime));
}
}
/**
* Add a new datafile into the file group
*/
public void addDataFile(HoodieDataFile dataFile) {
if (!fileSlices.containsKey(dataFile.getCommitTime())) {
fileSlices.put(dataFile.getCommitTime(), new FileSlice(fileGroupId, dataFile.getCommitTime()));
}
fileSlices.get(dataFile.getCommitTime()).setDataFile(dataFile);
}
/**
* Add a new log file into the group
*/
public void addLogFile(HoodieLogFile logFile) {
if (!fileSlices.containsKey(logFile.getBaseCommitTime())) {
fileSlices.put(logFile.getBaseCommitTime(), new FileSlice(fileGroupId, logFile.getBaseCommitTime()));
}
fileSlices.get(logFile.getBaseCommitTime()).addLogFile(logFile);
}
public String getPartitionPath() {
return fileGroupId.getPartitionPath();
}
public HoodieFileGroupId getFileGroupId() {
return fileGroupId;
}
/**
* 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.getBaseInstantTime())
&& HoodieTimeline.compareTimestamps(slice.getBaseInstantTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL);
}
/**
* Get all the the file slices including in-flight ones as seen in underlying file-system
*/
public Stream<FileSlice> getAllFileSlicesIncludingInflight() {
return fileSlices.entrySet().stream().map(Map.Entry::getValue);
}
/**
* Get latest file slices including in-flight ones
*/
public Option<FileSlice> getLatestFileSlicesIncludingInflight() {
return Option.fromJavaOptional(getAllFileSlicesIncludingInflight().findFirst());
}
/**
* Provides a stream of committed file slices, sorted reverse base commit time.
*/
public Stream<FileSlice> getAllFileSlices() {
if (!timeline.empty()) {
return fileSlices.entrySet().stream()
.map(Map.Entry::getValue)
.filter(this::isFileSliceCommitted);
}
return Stream.empty();
}
/**
* Gets the latest slice - this can contain either
* <p>
* - just the log files without data file - (or) data file with 0 or more log files
*/
public Option<FileSlice> getLatestFileSlice() {
// there should always be one
return Option.fromJavaOptional(getAllFileSlices().findFirst());
}
/**
* Gets the latest data file
*/
public Option<HoodieDataFile> getLatestDataFile() {
return Option.fromJavaOptional(getAllDataFiles().findFirst());
}
/**
* Obtain the latest file slice, upto a commitTime i.e <= maxCommitTime
*/
public Option<FileSlice> getLatestFileSliceBeforeOrOn(String maxCommitTime) {
return Option.fromJavaOptional(getAllFileSlices()
.filter(slice ->
HoodieTimeline.compareTimestamps(slice.getBaseInstantTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL))
.findFirst());
}
/**
* Obtain the latest file slice, upto a commitTime i.e < maxInstantTime
* @param maxInstantTime Max Instant Time
* @return
*/
public Option<FileSlice> getLatestFileSliceBefore(String maxInstantTime) {
return Option.fromJavaOptional(getAllFileSlices()
.filter(slice ->
HoodieTimeline.compareTimestamps(slice.getBaseInstantTime(),
maxInstantTime,
HoodieTimeline.LESSER))
.findFirst());
}
public Option<FileSlice> getLatestFileSliceInRange(List<String> commitRange) {
return Option.fromJavaOptional(getAllFileSlices()
.filter(slice -> commitRange.contains(slice.getBaseInstantTime()))
.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());
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieFileGroup {");
sb.append("id=").append(fileGroupId);
sb.append(", fileSlices='").append(fileSlices).append('\'');
sb.append(", lastInstant='").append(lastInstant).append('\'');
sb.append('}');
return sb.toString();
}
public void addFileSlice(FileSlice slice) {
fileSlices.put(slice.getBaseInstantTime(), slice);
}
public Stream<FileSlice> getAllRawFileSlices() {
return fileSlices.values().stream();
}
public HoodieTimeline getTimeline() {
return timeline;
}
}

View File

@@ -0,0 +1,71 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import java.io.Serializable;
import java.util.Objects;
/**
* Unique ID to identify a file-group in a data-set
*/
public class HoodieFileGroupId implements Serializable {
private final String partitionPath;
private final String fileId;
public HoodieFileGroupId(String partitionPath, String fileId) {
this.partitionPath = partitionPath;
this.fileId = fileId;
}
public String getPartitionPath() {
return partitionPath;
}
public String getFileId() {
return fileId;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieFileGroupId that = (HoodieFileGroupId) o;
return Objects.equals(partitionPath, that.partitionPath)
&& Objects.equals(fileId, that.fileId);
}
@Override
public int hashCode() {
return Objects.hash(partitionPath, fileId);
}
@Override
public String toString() {
return "HoodieFileGroupId{"
+ "partitionPath='" + partitionPath + '\''
+ ", fileId='" + fileId + '\''
+ '}';
}
}

View File

@@ -0,0 +1,76 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import com.google.common.base.Objects;
import java.io.Serializable;
/**
* HoodieKey consists of
* <p>
* - 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 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;
}
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();
}
}

View File

@@ -0,0 +1,187 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import java.io.IOException;
import java.io.Serializable;
import java.util.Comparator;
import java.util.Objects;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.util.FSUtils;
/**
* Abstracts a single log file. Contains methods to extract metadata like the fileId, version and extension from the log
* file path.
* <p>
* Also contains logic to roll-over the log file
*/
public class HoodieLogFile implements Serializable {
public static final String DELTA_EXTENSION = ".log";
public static final Integer LOGFILE_BASE_VERSION = 1;
private transient FileStatus fileStatus;
private final String pathStr;
private long fileLen;
public HoodieLogFile(FileStatus fileStatus) {
this.fileStatus = fileStatus;
this.pathStr = fileStatus.getPath().toString();
this.fileLen = fileStatus.getLen();
}
public HoodieLogFile(Path logPath) {
this.fileStatus = null;
this.pathStr = logPath.toString();
this.fileLen = 0;
}
public HoodieLogFile(String logPathStr) {
this.fileStatus = null;
this.pathStr = logPathStr;
this.fileLen = -1;
}
public String getFileId() {
return FSUtils.getFileIdFromLogPath(getPath());
}
public String getBaseCommitTime() {
return FSUtils.getBaseCommitTimeFromLogPath(getPath());
}
public int getLogVersion() {
return FSUtils.getFileVersionFromLog(getPath());
}
public String getLogWriteToken() {
return FSUtils.getWriteTokenFromLogPath(getPath());
}
public String getFileExtension() {
return FSUtils.getFileExtensionFromLog(getPath());
}
public Path getPath() {
return new Path(pathStr);
}
public String getFileName() {
return getPath().getName();
}
public void setFileLen(long fileLen) {
this.fileLen = fileLen;
}
public long getFileSize() {
return fileLen;
}
public FileStatus getFileStatus() {
return fileStatus;
}
public void setFileStatus(FileStatus fileStatus) {
this.fileStatus = fileStatus;
}
public HoodieLogFile rollOver(FileSystem fs, String logWriteToken) throws IOException {
String fileId = getFileId();
String baseCommitTime = getBaseCommitTime();
Path path = getPath();
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, logWriteToken)));
}
public static Comparator<HoodieLogFile> getLogFileComparator() {
return new LogFileComparator();
}
public static Comparator<HoodieLogFile> getReverseLogFileComparator() {
return new LogFileComparator().reversed();
}
/**
* Comparator to order log-files
*/
public static class LogFileComparator implements Comparator<HoodieLogFile>, Serializable {
private transient Comparator<String> writeTokenComparator;
private Comparator<String> getWriteTokenComparator() {
if (null == writeTokenComparator) {
// writeTokenComparator is not serializable. Hence, lazy loading
writeTokenComparator = Comparator.nullsFirst(Comparator.naturalOrder());
}
return writeTokenComparator;
}
@Override
public int compare(HoodieLogFile o1, HoodieLogFile o2) {
String baseInstantTime1 = o1.getBaseCommitTime();
String baseInstantTime2 = o2.getBaseCommitTime();
if (baseInstantTime1.equals(baseInstantTime2)) {
if (o1.getLogVersion() == o2.getLogVersion()) {
// Compare by write token when base-commit and log-version is same
return getWriteTokenComparator().compare(o1.getLogWriteToken(), o2.getLogWriteToken());
}
// compare by log-version when base-commit is same
return Integer.compare(o1.getLogVersion(), o2.getLogVersion());
}
// compare by base-commits
return baseInstantTime1.compareTo(baseInstantTime2);
}
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieLogFile that = (HoodieLogFile) o;
return Objects.equals(pathStr, that.pathStr);
}
@Override
public int hashCode() {
return Objects.hash(pathStr);
}
@Override
public String toString() {
return "HoodieLogFile{"
+ "pathStr='" + pathStr + '\''
+ ", fileLen=" + fileLen
+ '}';
}
}

View File

@@ -0,0 +1,150 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
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;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.exception.HoodieException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* 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";
/**
* Contents of the metadata
*/
private final Properties props;
/**
* Path to the partition, about which we have the metadata
*/
private final Path partitionPath;
private final FileSystem fs;
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 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));
}
/**
* 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;
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 {
// 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);
}
}
}
}
/**
* Read out the metadata for this partition
*/
public void readFromFS() throws IOException {
FSDataInputStream is = null;
try {
Path metaFile = new Path(partitionPath, HOODIE_PARTITION_METAFILE);
is = fs.open(metaFile);
props.load(is);
} catch (IOException ioe) {
throw new HoodieException("Error reading Hoodie partition metadata for " + partitionPath,
ioe);
} finally {
if (is != null) {
is.close();
}
}
}
// 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

@@ -0,0 +1,173 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import com.google.common.base.Objects;
import com.google.common.collect.ImmutableList;
import java.io.Serializable;
import java.util.List;
import org.apache.hudi.common.util.Option;
/**
* A Single Record managed by Hoodie TODO - Make this generic
*/
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 final List<String> HOODIE_META_COLUMNS =
new ImmutableList.Builder<String>().add(COMMIT_TIME_METADATA_FIELD)
.add(COMMIT_SEQNO_METADATA_FIELD)
.add(RECORD_KEY_METADATA_FIELD)
.add(PARTITION_PATH_METADATA_FIELD)
.add(FILENAME_METADATA_FIELD)
.build();
/**
* Identifies the record across the table
*/
private HoodieKey key;
/**
* Actual payload of the record
*/
private T data;
/**
* 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;
public HoodieRecord(HoodieKey key, T data) {
this.key = key;
this.data = data;
this.currentLocation = null;
this.newLocation = null;
}
public HoodieRecord(HoodieRecord<T> record) {
this(record.key, record.data);
this.currentLocation = record.currentLocation;
this.newLocation = record.newLocation;
}
public HoodieKey getKey() {
return key;
}
public T getData() {
if (data == null) {
throw new IllegalStateException("Payload already deflated for record.");
}
return data;
}
/**
* 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 Option<HoodieRecordLocation> getNewLocation() {
return Option.ofNullable(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();
}
}

View File

@@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import com.google.common.base.Objects;
import java.io.Serializable;
/**
* Location of a HoodieRecord within the partition it belongs to. Ultimately, this points to an
* actual file on disk
*/
public class HoodieRecordLocation implements Serializable {
private final String instantTime;
private final String fileId;
public HoodieRecordLocation(String instantTime, String fileId) {
this.instantTime = instantTime;
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(instantTime, otherLoc.instantTime)
&& Objects.equal(fileId, otherLoc.fileId);
}
@Override
public int hashCode() {
return Objects.hashCode(instantTime, fileId);
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieRecordLocation {");
sb.append("instantTime=").append(instantTime).append(", ");
sb.append("fileId=").append(fileId);
sb.append('}');
return sb.toString();
}
public String getInstantTime() {
return instantTime;
}
public String getFileId() {
return fileId;
}
}

View File

@@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import java.io.IOException;
import java.io.Serializable;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.common.util.Option;
/**
* 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.
* <p>
* 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.
*/
Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException;
/**
* 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.
*/
Option<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 Option<Map<String, String>> getMetadata() {
return Option.empty();
}
}

View File

@@ -0,0 +1,100 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import java.io.Serializable;
import javax.annotation.Nullable;
@JsonIgnoreProperties(ignoreUnknown = true)
public class HoodieRollingStat implements Serializable {
private String fileId;
private long inserts;
private long upserts;
private long deletes;
// TODO
@Nullable
private long totalInputWriteBytesToDisk;
@Nullable
private long totalInputWriteBytesOnDisk;
public HoodieRollingStat() {
// called by jackson json lib
}
public HoodieRollingStat(String fileId, long inserts, long upserts, long deletes, long totalInputWriteBytesOnDisk) {
this.fileId = fileId;
this.inserts = inserts;
this.upserts = upserts;
this.deletes = deletes;
this.totalInputWriteBytesOnDisk = totalInputWriteBytesOnDisk;
}
public String getFileId() {
return fileId;
}
public void setFileId(String fileId) {
this.fileId = fileId;
}
public long getInserts() {
return inserts;
}
public void setInserts(long inserts) {
this.inserts = inserts;
}
public long getUpserts() {
return upserts;
}
public void setUpserts(long upserts) {
this.upserts = upserts;
}
public long getDeletes() {
return deletes;
}
public void setDeletes(long deletes) {
this.deletes = deletes;
}
public long addInserts(long inserts) {
this.inserts += inserts;
return this.inserts;
}
public long addUpserts(long upserts) {
this.upserts += upserts;
return this.upserts;
}
public long addDeletes(long deletes) {
this.deletes += deletes;
return this.deletes;
}
public long getTotalInputWriteBytesOnDisk() {
return totalInputWriteBytesOnDisk;
}
}

View File

@@ -0,0 +1,103 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import java.io.IOException;
import java.io.Serializable;
import java.util.HashMap;
import java.util.Map;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* This class holds statistics about files belonging to a dataset
*/
public class HoodieRollingStatMetadata implements Serializable {
private static volatile Logger log = LogManager.getLogger(HoodieRollingStatMetadata.class);
protected Map<String, Map<String, HoodieRollingStat>> partitionToRollingStats;
private String actionType = "DUMMY_ACTION";
public static final String ROLLING_STAT_METADATA_KEY = "ROLLING_STAT";
public void addRollingStat(String partitionPath, HoodieRollingStat stat) {
if (!partitionToRollingStats.containsKey(partitionPath)) {
partitionToRollingStats.put(partitionPath, new RollingStatsHashMap<>());
}
partitionToRollingStats.get(partitionPath).put(stat.getFileId(), stat);
}
public HoodieRollingStatMetadata() {
partitionToRollingStats = new HashMap<>();
}
public HoodieRollingStatMetadata(String actionType) {
this();
this.actionType = actionType;
}
class RollingStatsHashMap<K, V> extends HashMap<K, V> {
@Override
public V put(K key, V value) {
V v = this.get(key);
if (v == null) {
super.put(key, value);
} else if (v instanceof HoodieRollingStat) {
long inserts = ((HoodieRollingStat) v).getInserts();
long upserts = ((HoodieRollingStat) v).getUpserts();
long deletes = ((HoodieRollingStat) v).getDeletes();
((HoodieRollingStat) value).addInserts(inserts);
((HoodieRollingStat) value).addUpserts(upserts);
((HoodieRollingStat) value).addDeletes(deletes);
super.put(key, value);
}
return value;
}
}
public static HoodieRollingStatMetadata fromBytes(byte[] bytes) throws IOException {
return HoodieCommitMetadata.fromBytes(bytes, HoodieRollingStatMetadata.class);
}
public String toJsonString() throws IOException {
if (partitionToRollingStats.containsKey(null)) {
log.info("partition path is null for " + partitionToRollingStats.get(null));
partitionToRollingStats.remove(null);
}
return HoodieCommitMetadata.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this);
}
public HoodieRollingStatMetadata merge(HoodieRollingStatMetadata rollingStatMetadata) {
for (Map.Entry<String, Map<String, HoodieRollingStat>> stat : rollingStatMetadata.partitionToRollingStats
.entrySet()) {
for (Map.Entry<String, HoodieRollingStat> innerStat : stat.getValue().entrySet()) {
this.addRollingStat(stat.getKey(), innerStat.getValue());
}
}
return this;
}
public Map<String, Map<String, HoodieRollingStat>> getPartitionToRollingStats() {
return partitionToRollingStats;
}
public String getActionType() {
return actionType;
}
}

View File

@@ -0,0 +1,37 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
/**
* Type of the Hoodie Table.
* <p>
* Currently, 1 type is supported
* <p>
* COPY_ON_WRITE - Performs upserts by versioning entire files, with later versions containing newer
* value of a record.
* <p>
* In the future, following might be added.
* <p>
* MERGE_ON_READ - Speeds up upserts, by delaying merge until enough work piles up.
* <p>
* SIMPLE_LSM - A simple 2 level LSM tree.
*/
public enum HoodieTableType {
COPY_ON_WRITE, MERGE_ON_READ
}

View File

@@ -0,0 +1,412 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import java.io.Serializable;
import javax.annotation.Nullable;
import org.apache.hadoop.fs.Path;
/**
* Statistics about a single Hoodie write operation.
*/
@JsonIgnoreProperties(ignoreUnknown = true)
public class HoodieWriteStat implements Serializable {
public static final String NULL_COMMIT = "null";
/**
* Id of the file being written
*/
private String fileId;
/**
* 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;
/**
* 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 actually changed. (0 for inserts)
*/
private long numUpdateWrites;
/**
* Total number of insert records or converted to updates (for small file handling)
*/
private long numInserts;
/**
* 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;
/**
* Relative path to the temporary file from the base path.
*/
@Nullable
private String tempPath;
/**
* Following properties are associated only with the result of a Compaction Operation
*/
/**
* Partition Path associated with this writeStat
*/
@Nullable
private String partitionPath;
/**
* Total number of log records that were compacted by a compaction operation
*/
@Nullable
private long totalLogRecords;
/**
* Total number of log files compacted for a file slice with this base fileid
*/
@Nullable
private long totalLogFilesCompacted;
/**
* Total size of all log files for a file slice with this base fileid
*/
@Nullable
private long totalLogSizeCompacted;
/**
* Total number of records updated by a compaction operation
*/
@Nullable
private long totalUpdatedRecordsCompacted;
/**
* Total number of log blocks seen in a compaction operation
*/
@Nullable
private long totalLogBlocks;
/**
* Total number of corrupt blocks seen in a compaction operation
*/
@Nullable
private long totalCorruptLogBlock;
/**
* Total number of rollback blocks seen in a compaction operation
*/
private long totalRollbackBlocks;
/**
* File Size as of close
*/
private long fileSizeInBytes;
@Nullable
@JsonIgnore
private RuntimeStats runtimeStats;
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 void setNumInserts(long numInserts) {
this.numInserts = numInserts;
}
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 long getNumInserts() {
return numInserts;
}
public String getFileId() {
return fileId;
}
public String getPath() {
return path;
}
public String getPartitionPath() {
return partitionPath;
}
public void setPartitionPath(String partitionPath) {
this.partitionPath = partitionPath;
}
public long getTotalLogRecords() {
return totalLogRecords;
}
public void setTotalLogRecords(long totalLogRecords) {
this.totalLogRecords = totalLogRecords;
}
public long getTotalLogFilesCompacted() {
return totalLogFilesCompacted;
}
public void setTotalLogFilesCompacted(long totalLogFilesCompacted) {
this.totalLogFilesCompacted = totalLogFilesCompacted;
}
public long getTotalUpdatedRecordsCompacted() {
return totalUpdatedRecordsCompacted;
}
public void setTotalUpdatedRecordsCompacted(long totalUpdatedRecordsCompacted) {
this.totalUpdatedRecordsCompacted = totalUpdatedRecordsCompacted;
}
public void setTempPath(String tempPath) {
this.tempPath = tempPath;
}
public String getTempPath() {
return this.tempPath;
}
public long getTotalLogSizeCompacted() {
return totalLogSizeCompacted;
}
public void setTotalLogSizeCompacted(long totalLogSizeCompacted) {
this.totalLogSizeCompacted = totalLogSizeCompacted;
}
public long getTotalLogBlocks() {
return totalLogBlocks;
}
public void setTotalLogBlocks(long totalLogBlocks) {
this.totalLogBlocks = totalLogBlocks;
}
public long getTotalCorruptLogBlock() {
return totalCorruptLogBlock;
}
public void setTotalCorruptLogBlock(long totalCorruptLogBlock) {
this.totalCorruptLogBlock = totalCorruptLogBlock;
}
public long getTotalRollbackBlocks() {
return totalRollbackBlocks;
}
public void setTotalRollbackBlocks(Long totalRollbackBlocks) {
this.totalRollbackBlocks = totalRollbackBlocks;
}
public long getFileSizeInBytes() {
return fileSizeInBytes;
}
public void setFileSizeInBytes(long fileSizeInBytes) {
this.fileSizeInBytes = fileSizeInBytes;
}
@Nullable
public RuntimeStats getRuntimeStats() {
return runtimeStats;
}
public void setRuntimeStats(@Nullable RuntimeStats runtimeStats) {
this.runtimeStats = runtimeStats;
}
/**
* Set path and tempPath relative to the given basePath.
*/
public void setPath(Path basePath, Path path) {
this.path = path.toString().replace(basePath + "/", "");
}
@Override
public String toString() {
return "HoodieWriteStat{"
+ "fileId='" + fileId + '\''
+ ", path='" + path + '\''
+ ", prevCommit='" + prevCommit + '\''
+ ", numWrites=" + numWrites
+ ", numDeletes=" + numDeletes
+ ", numUpdateWrites=" + numUpdateWrites
+ ", totalWriteBytes=" + totalWriteBytes
+ ", totalWriteErrors=" + totalWriteErrors
+ ", tempPath='" + tempPath + '\''
+ ", partitionPath='" + partitionPath
+ '\'' + ", totalLogRecords=" + totalLogRecords
+ ", totalLogFilesCompacted=" + totalLogFilesCompacted
+ ", totalLogSizeCompacted=" + totalLogSizeCompacted
+ ", totalUpdatedRecordsCompacted=" + totalUpdatedRecordsCompacted
+ ", totalLogBlocks=" + totalLogBlocks
+ ", totalCorruptLogBlock=" + totalCorruptLogBlock
+ ", totalRollbackBlocks=" + totalRollbackBlocks
+ '}';
}
@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;
}
public static class RuntimeStats implements Serializable {
/**
* Total time taken to read and merge logblocks in a log file
*/
@Nullable
private long totalScanTime;
/**
* Total time taken by a Hoodie Merge for an existing file
*/
@Nullable
private long totalUpsertTime;
/**
* Total time taken by a Hoodie Insert to a file
*/
@Nullable
private long totalCreateTime;
@Nullable
public long getTotalScanTime() {
return totalScanTime;
}
public void setTotalScanTime(@Nullable long totalScanTime) {
this.totalScanTime = totalScanTime;
}
@Nullable
public long getTotalUpsertTime() {
return totalUpsertTime;
}
public void setTotalUpsertTime(@Nullable long totalUpsertTime) {
this.totalUpsertTime = totalUpsertTime;
}
@Nullable
public long getTotalCreateTime() {
return totalCreateTime;
}
public void setTotalCreateTime(@Nullable long totalCreateTime) {
this.totalCreateTime = totalCreateTime;
}
}
}

View File

@@ -0,0 +1,65 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.storage;
import java.io.DataInputStream;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Keeps track of how many bytes were read from a DataInputStream
*/
public class SizeAwareDataInputStream {
private final DataInputStream dis;
private final AtomicInteger numberOfBytesRead;
public SizeAwareDataInputStream(DataInputStream dis) {
this.dis = dis;
this.numberOfBytesRead = new AtomicInteger(0);
}
public int readInt() throws IOException {
numberOfBytesRead.addAndGet(Integer.BYTES);
return dis.readInt();
}
public void readFully(byte[] b, int off, int len) throws IOException {
numberOfBytesRead.addAndGet(len);
dis.readFully(b, off, len);
}
public void readFully(byte[] b) throws IOException {
numberOfBytesRead.addAndGet(b.length);
dis.readFully(b);
}
public int skipBytes(int n) throws IOException {
numberOfBytesRead.addAndGet(n);
return dis.skipBytes(n);
}
public void close() throws IOException {
dis.close();
}
public Integer getNumberOfBytesRead() {
return numberOfBytesRead.get();
}
}

View File

@@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.storage;
import java.util.Arrays;
/**
* All the supported storage schemes in Hoodie.
*/
public enum StorageSchemes {
// Local filesystem
FILE("file", false),
// Hadoop File System
HDFS("hdfs", true),
// Mapr File System
MAPRFS("maprfs", true),
// Apache Ignite FS
IGNITE("igfs", true),
// AWS S3
S3A("s3a", false),
S3("s3", false),
// Google Cloud Storage
GCS("gs", false),
// View FS for federated setups. If federating across cloud stores, then append support is false
VIEWFS("viewfs", true);
private String scheme;
private boolean supportsAppend;
StorageSchemes(String scheme, boolean supportsAppend) {
this.scheme = scheme;
this.supportsAppend = supportsAppend;
}
public String getScheme() {
return scheme;
}
public boolean supportsAppend() {
return supportsAppend;
}
public static boolean isSchemeSupported(String scheme) {
return Arrays.stream(values()).anyMatch(s -> s.getScheme().equals(scheme));
}
public static boolean isAppendSupported(String scheme) {
if (!isSchemeSupported(scheme)) {
throw new IllegalArgumentException("Unsupported scheme :" + scheme);
}
return Arrays.stream(StorageSchemes.values()).anyMatch(s -> s.supportsAppend() && s.scheme.equals(scheme));
}
}

View File

@@ -0,0 +1,188 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table;
import java.io.IOException;
import java.io.Serializable;
import java.util.Date;
import java.util.Map;
import java.util.Properties;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* 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.
*
* @see HoodieTableMetaClient
* @since 0.3.0
*/
public class HoodieTableConfig implements Serializable {
private static final transient 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";
public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = "hoodie.archivelog.folder";
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();
public static final String DEFAULT_ARCHIVELOG_FOLDER = "";
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;
}
public HoodieTableConfig(Properties props) {
this.props = props;
}
/**
* 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);
}
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
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);
}
if (!properties.containsKey(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) {
properties.setProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER);
}
properties
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
}
}
/**
* 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
*/
public String getPayloadClass() {
// There could be datasets written with payload class from com.uber.hoodie. Need to transparently
// change to org.apache.hudi
return props.getProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS)
.replace("com.uber.hoodie", "org.apache.hudi");
}
/**
* 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 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 relative path of archive log folder under metafolder, for this dataset
*/
public String getArchivelogFolder() {
return props.getProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER);
}
public Map<String, String> getProps() {
return props.entrySet().stream().collect(
Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())));
}
}

View File

@@ -0,0 +1,480 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table;
import com.google.common.base.Preconditions;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Objects;
import java.util.Properties;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hudi.common.SerializableConfiguration;
import org.apache.hudi.common.io.storage.HoodieWrapperFileSystem;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.ConsistencyGuardConfig;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.FailSafeConsistencyGuard;
import org.apache.hudi.common.util.NoOpConsistencyGuard;
import org.apache.hudi.exception.DatasetNotFoundException;
import org.apache.hudi.exception.HoodieException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* <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 static final transient Logger log = LogManager.getLogger(HoodieTableMetaClient.class);
public static String METAFOLDER_NAME = ".hoodie";
public static String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp";
public static String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + File.separator + ".aux";
public static final String MARKER_EXTN = ".marker";
private String basePath;
private transient HoodieWrapperFileSystem fs;
private String metaPath;
private SerializableConfiguration hadoopConf;
private HoodieTableType tableType;
private HoodieTableConfig tableConfig;
private HoodieActiveTimeline activeTimeline;
private HoodieArchivedTimeline archivedTimeline;
private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build();
public HoodieTableMetaClient(Configuration conf, String basePath)
throws DatasetNotFoundException {
// Do not load any timeline by default
this(conf, basePath, false);
}
public HoodieTableMetaClient(Configuration conf, String basePath,
boolean loadActiveTimelineOnLoad) {
this(conf, basePath, loadActiveTimelineOnLoad, ConsistencyGuardConfig.newBuilder().build());
}
public HoodieTableMetaClient(Configuration conf, String basePath,
boolean loadActiveTimelineOnLoad, ConsistencyGuardConfig consistencyGuardConfig)
throws DatasetNotFoundException {
log.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath;
this.consistencyGuardConfig = consistencyGuardConfig;
this.hadoopConf = new SerializableConfiguration(conf);
Path basePathDir = new Path(this.basePath);
this.metaPath = new Path(basePath, METAFOLDER_NAME).toString();
Path metaPathDir = new Path(this.metaPath);
this.fs = getFs();
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() {
}
/**
* 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();
fs = null; // will be lazily inited
}
private void writeObject(java.io.ObjectOutputStream out)
throws IOException {
out.defaultWriteObject();
}
/**
* @return Base path
*/
public String getBasePath() {
return basePath;
}
/**
* @return Hoodie Table Type
*/
public HoodieTableType getTableType() {
return tableType;
}
/**
* @return Meta path
*/
public String getMetaPath() {
return metaPath;
}
/**
* @return Temp Folder path
*/
public String getTempFolderPath() {
return basePath + File.separator + TEMPFOLDER_NAME;
}
/**
* Returns Marker folder path
* @param instantTs Instant Timestamp
* @return
*/
public String getMarkerFolderPath(String instantTs) {
return String.format("%s%s%s", getTempFolderPath(), File.separator, instantTs);
}
/**
* @return Auxiliary Meta path
*/
public String getMetaAuxiliaryPath() {
return basePath + File.separator + AUXILIARYFOLDER_NAME;
}
/**
* @return path where archived timeline is stored
*/
public String getArchivePath() {
String archiveFolder = tableConfig.getArchivelogFolder();
if (archiveFolder.equals(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER)) {
return getMetaPath();
} else {
return getMetaPath() + "/" + archiveFolder;
}
}
/**
* @return Table Config
*/
public HoodieTableConfig getTableConfig() {
return tableConfig;
}
/**
* Get the FS implementation for this table
*/
public HoodieWrapperFileSystem getFs() {
if (fs == null) {
FileSystem fileSystem = FSUtils.getFs(metaPath, hadoopConf.get());
Preconditions.checkArgument(!(fileSystem instanceof HoodieWrapperFileSystem),
"File System not expected to be that of HoodieWrapperFileSystem");
fs = new HoodieWrapperFileSystem(fileSystem, consistencyGuardConfig.isConsistencyCheckEnabled()
? new FailSafeConsistencyGuard(fileSystem, consistencyGuardConfig) : new NoOpConsistencyGuard());
}
return fs;
}
/**
* Return raw file-system
* @return
*/
public FileSystem getRawFs() {
return getFs().getFileSystem();
}
public Configuration getHadoopConf() {
return hadoopConf.get();
}
/**
* Get the active instants as a timeline
*
* @return Active instants timeline
*/
public synchronized HoodieActiveTimeline getActiveTimeline() {
if (activeTimeline == null) {
activeTimeline = new HoodieActiveTimeline(this);
}
return activeTimeline;
}
/**
* Reload ActiveTimeline and cache
*
* @return Active instants timeline
*/
public synchronized HoodieActiveTimeline reloadActiveTimeline() {
activeTimeline = new HoodieActiveTimeline(this);
return activeTimeline;
}
public ConsistencyGuardConfig getConsistencyGuardConfig() {
return consistencyGuardConfig;
}
/**
* 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(this);
}
return archivedTimeline;
}
/**
* Helper method to initialize a dataset, with given basePath, tableType, name, archiveFolder
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
String tableType, String tableName, String archiveLogFolder) throws IOException {
HoodieTableType type = HoodieTableType.valueOf(tableType);
Properties properties = new Properties();
properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, type.name());
properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, archiveLogFolder);
return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties);
}
/**
* Helper method to initialize a given path, as a given storage type and table name
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, 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(hadoopConf, 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(Configuration hadoopConf,
String basePath, Properties props) throws IOException {
log.info("Initializing " + basePath + " as hoodie dataset " + basePath);
Path basePathDir = new Path(basePath);
final FileSystem fs = FSUtils.getFs(basePath, hadoopConf);
if (!fs.exists(basePathDir)) {
fs.mkdirs(basePathDir);
}
Path metaPathDir = new Path(basePath, METAFOLDER_NAME);
if (!fs.exists(metaPathDir)) {
fs.mkdirs(metaPathDir);
}
// if anything other than default archive log folder is specified, create that too
String archiveLogPropVal = props
.getProperty(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME,
HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER);
if (!archiveLogPropVal.equals(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER)) {
Path archiveLogDir = new Path(metaPathDir, archiveLogPropVal);
if (!fs.exists(archiveLogDir)) {
fs.mkdirs(archiveLogDir);
}
}
// Always create temporaryFolder which is needed for finalizeWrite for Hoodie tables
final Path temporaryFolder = new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME);
if (!fs.exists(temporaryFolder)) {
fs.mkdirs(temporaryFolder);
}
// Always create auxiliary folder which is needed to track compaction workloads (stats and any metadata in future)
final Path auxiliaryFolder = new Path(basePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME);
if (!fs.exists(auxiliaryFolder)) {
fs.mkdirs(auxiliaryFolder);
}
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props);
// We should not use fs.getConf as this might be different from the original configuration
// used to create the fs in unit tests
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, 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);
}
/**
* Get the commit timeline visible for this table
*/
public HoodieTimeline getCommitsTimeline() {
switch (this.getTableType()) {
case COPY_ON_WRITE:
return getActiveTimeline().getCommitTimeline();
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits
// Include commit action to be able to start doing a MOR over a COW dataset - no
// migration required
return getActiveTimeline().getCommitsTimeline();
default:
throw new HoodieException("Unsupported table type :" + this.getTableType());
}
}
/**
* Get the commit + pending-compaction timeline visible for this table.
* A RT filesystem view is constructed with this timeline so that file-slice after pending compaction-requested
* instant-time is also considered valid. A RT file-system view for reading must then merge the file-slices before
* and after pending compaction instant so that all delta-commits are read.
*/
public HoodieTimeline getCommitsAndCompactionTimeline() {
switch (this.getTableType()) {
case COPY_ON_WRITE:
return getActiveTimeline().getCommitTimeline();
case MERGE_ON_READ:
return getActiveTimeline().getCommitsAndCompactionTimeline();
default:
throw new HoodieException("Unsupported table type :" + this.getTableType());
}
}
/**
* Get the compacted commit timeline visible for this table
*/
public HoodieTimeline getCommitTimeline() {
switch (this.getTableType()) {
case COPY_ON_WRITE:
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits in tagging
return getActiveTimeline().getCommitTimeline();
default:
throw new HoodieException("Unsupported table type :" + this.getTableType());
}
}
/**
* Gets the commit action type
*/
public String getCommitActionType() {
switch (this.getTableType()) {
case COPY_ON_WRITE:
return HoodieActiveTimeline.COMMIT_ACTION;
case MERGE_ON_READ:
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
default:
throw new HoodieException(
"Could not commit on unknown storage type " + this.getTableType());
}
}
/**
* Helper method to scan all hoodie-instant metafiles and construct HoodieInstant objects
*
* @param fs FileSystem
* @param metaPath Meta Path where hoodie instants are present
* @param includedExtensions Included hoodie extensions
* @return List of Hoodie Instants generated
* @throws IOException in case of failure
*/
public static List<HoodieInstant> scanHoodieInstantsFromFileSystem(
FileSystem fs, Path metaPath, Set<String> includedExtensions) throws IOException {
return Arrays.stream(
HoodieTableMetaClient
.scanFiles(fs, metaPath, path -> {
// Include only the meta files with extensions that needs to be included
String extension = FSUtils.getFileExtension(path.getName());
return includedExtensions.contains(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());
}
@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 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();
}
public void setBasePath(String basePath) {
this.basePath = basePath;
}
public void setMetaPath(String metaPath) {
this.metaPath = metaPath;
}
public void setActiveTimeline(HoodieActiveTimeline activeTimeline) {
this.activeTimeline = activeTimeline;
}
public void setTableConfig(HoodieTableConfig tableConfig) {
this.tableConfig = tableConfig;
}
}

View File

@@ -0,0 +1,300 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table;
import java.io.Serializable;
import java.util.function.BiPredicate;
import java.util.function.Predicate;
import java.util.stream.Stream;
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
/**
* 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 HoodieTableMetaClient
* @see HoodieDefaultTimeline
* @see HoodieInstant
* @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 INFLIGHT_EXTENSION = ".inflight";
// With Async Compaction, compaction instant can be in 3 states :
// (compaction-requested), (compaction-inflight), (completed)
String COMPACTION_ACTION = "compaction";
String REQUESTED_EXTENSION = ".requested";
String RESTORE_ACTION = "restore";
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;
//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 REQUESTED_COMPACTION_SUFFIX =
StringUtils.join(COMPACTION_ACTION, REQUESTED_EXTENSION);
String REQUESTED_COMPACTION_EXTENSION =
StringUtils.join(".", REQUESTED_COMPACTION_SUFFIX);
String INFLIGHT_COMPACTION_EXTENSION =
StringUtils.join(".", COMPACTION_ACTION, INFLIGHT_EXTENSION);
String INFLIGHT_RESTORE_EXTENSION = "." + RESTORE_ACTION + INFLIGHT_EXTENSION;
String RESTORE_EXTENSION = "." + RESTORE_ACTION;
String INVALID_INSTANT_TS = "0";
/**
* 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 in-flights excluding compaction instants
*
* @return New instance of HoodieTimeline with just in-flights excluding compaction inflights
*/
HoodieTimeline filterInflightsExcludingCompaction();
/**
* 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 completed + compaction (inflight + requested) instants
* A RT filesystem view is constructed with this timeline so that file-slice after pending compaction-requested
* instant-time is also considered valid. A RT file-system view for reading must then merge the file-slices before
* and after pending compaction instant so that all delta-commits are read.
* @return New instance of HoodieTimeline with just completed instants
*/
HoodieTimeline filterCompletedAndCompactionInstants();
/**
* Timeline to just include commits (commit/deltacommit) and compaction actions
* @return
*/
HoodieTimeline getCommitsAndCompactionTimeline();
/**
* Filter this timeline to just include requested and inflight compaction instants
* @return
*/
HoodieTimeline filterPendingCompactionTimeline();
/**
* 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
*/
HoodieTimeline findInstantsAfter(String commitTime, int numCommits);
/**
* Custom Filter of Instants
*/
HoodieTimeline filter(Predicate<HoodieInstant> filter);
/**
* If the timeline has any instants
*
* @return true if timeline is empty
*/
boolean empty();
/**
* @return total number of completed instants
*/
int countInstants();
/**
* @return first completed instant if available
*/
Option<HoodieInstant> firstInstant();
/**
* @return nth completed instant from the first completed instant
*/
Option<HoodieInstant> nthInstant(int n);
/**
* @return last completed instant if available
*/
Option<HoodieInstant> lastInstant();
/**
* Get hash of timeline
* @return
*/
String getTimelineHash();
/**
* @return nth completed instant going back from the last completed instant
*/
Option<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 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 true if the passed in instant is before the first completed instant in the timeline
*/
boolean isBeforeTimelineStarts(String ts);
/**
* Read the completed instant details
*/
Option<byte[]> getInstantDetails(HoodieInstant instant);
/**
* Helper methods to compare instants
**/
BiPredicate<String, String> EQUAL =
(commit1, commit2) -> commit1.compareTo(commit2) == 0;
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 HoodieInstant getCompletedInstant(final HoodieInstant instant) {
return new HoodieInstant(false, instant.getAction(), instant.getTimestamp());
}
static HoodieInstant getCompactionRequestedInstant(final String timestamp) {
return new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, timestamp);
}
static HoodieInstant getCompactionInflightInstant(final String timestamp) {
return new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, timestamp);
}
static HoodieInstant getInflightInstant(final HoodieInstant instant) {
return new HoodieInstant(true, instant.getAction(), instant.getTimestamp());
}
static String makeCommitFileName(String commitTime) {
return StringUtils.join(commitTime, HoodieTimeline.COMMIT_EXTENSION);
}
static String makeInflightCommitFileName(String commitTime) {
return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_COMMIT_EXTENSION);
}
static String makeCleanerFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.CLEAN_EXTENSION);
}
static String makeInflightCleanerFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.INFLIGHT_CLEAN_EXTENSION);
}
static String makeRollbackFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.ROLLBACK_EXTENSION);
}
static String makeInflightRollbackFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION);
}
static String makeInflightSavePointFileName(String commitTime) {
return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION);
}
static String makeSavePointFileName(String commitTime) {
return StringUtils.join(commitTime, HoodieTimeline.SAVEPOINT_EXTENSION);
}
static String makeInflightDeltaFileName(String commitTime) {
return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION);
}
static String makeInflightCompactionFileName(String commitTime) {
return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION);
}
static String makeRequestedCompactionFileName(String commitTime) {
return StringUtils.join(commitTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION);
}
static String makeRestoreFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.RESTORE_EXTENSION);
}
static String makeInflightRestoreFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.INFLIGHT_RESTORE_EXTENSION);
}
static String makeDeltaFileName(String commitTime) {
return commitTime + HoodieTimeline.DELTA_COMMIT_EXTENSION;
}
static String getCommitFromCommitFile(String commitFileName) {
return commitFileName.split("\\.")[0];
}
static String makeFileNameAsComplete(String fileName) {
return fileName.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
}
static String makeFileNameAsInflight(String fileName) {
return StringUtils.join(fileName, HoodieTimeline.INFLIGHT_EXTENSION);
}
}

View File

@@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table;
/*
* A consolidated file-system view interface exposing both realtime and read-optimized views along with
* update operations.
*/
public interface SyncableFileSystemView extends TableFileSystemView, TableFileSystemView.ReadOptimizedView,
TableFileSystemView.RealtimeView {
/**
* Allow View to release resources and close
*/
void close();
/**
* Reset View so that they can be refreshed
*/
void reset();
/**
* Read the latest timeline and refresh the file-system view to match the current state of the file-system.
* The refresh can either be done incrementally (from reading file-slices in metadata files) or from scratch by
* reseting view storage
*/
void sync();
}

View File

@@ -0,0 +1,166 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table;
import java.util.List;
import java.util.stream.Stream;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
/**
* Interface for viewing the table file system.
*
* @since 0.3.0
*/
public interface TableFileSystemView {
/**
* ReadOptimizedView with methods to only access latest version of file for the instant(s) passed.
*/
interface ReadOptimizedViewWithLatestSlice {
/**
* Stream all the latest data files in the given partition
*/
Stream<HoodieDataFile> getLatestDataFiles(String partitionPath);
/**
* Get Latest data file for a partition and file-Id
*/
Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId);
/**
* 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);
}
/**
* ReadOptimizedView - methods to provide a view of columnar data files only.
*/
interface ReadOptimizedView extends ReadOptimizedViewWithLatestSlice {
/**
* Stream all the data file versions grouped by FileId for a given partition
*/
Stream<HoodieDataFile> getAllDataFiles(String partitionPath);
/**
* Get the version of data file matching the instant time in the given partition
*/
Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId);
}
/**
* RealtimeView with methods to only access latest version of file-slice for the instant(s) passed.
*/
interface RealtimeViewWithLatestSlice {
/**
* Stream all the latest file slices in the given partition
*/
Stream<FileSlice> getLatestFileSlices(String partitionPath);
/**
* Get Latest File Slice for a given fileId in a given partition
*/
Option<FileSlice> getLatestFileSlice(String partitionPath, String fileId);
/**
* Stream all the latest uncompacted file slices in the given partition
*/
Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionPath);
/**
* Stream all latest file slices in given partition with precondition that commitTime(file) before maxCommitTime
*
* @param partitionPath Partition path
* @param maxCommitTime Max Instant Time
* @param includeFileSlicesInPendingCompaction include file-slices that are in pending compaction
*/
Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath,
String maxCommitTime, boolean includeFileSlicesInPendingCompaction);
/**
* Stream all "merged" file-slices before on an instant time
* If a file-group has a pending compaction request, the file-slice before and after compaction request instant
* is merged and returned.
* @param partitionPath Partition Path
* @param maxInstantTime Max Instant Time
* @return
*/
public Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime);
/**
* Stream all the latest file slices, in the given range
*/
Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn);
}
/**
* RealtimeView - methods to access a combination of columnar data files + log files with real time data.
*/
interface RealtimeView extends RealtimeViewWithLatestSlice {
/**
* 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);
/**
* Return Pending Compaction Operations
*
* @return Pair<Pair<InstantTime,CompactionOperation>>
*/
Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations();
/**
* Last Known Instant on which the view is built
*/
Option<HoodieInstant> getLastInstant();
/**
* Timeline corresponding to the view
*/
HoodieTimeline getTimeline();
}

View File

@@ -0,0 +1,356 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log;
import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME;
import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.Arrays;
import java.util.Deque;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
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.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Implements logic to scan log blocks and expose valid and deleted log records to subclass implementation.
* Subclass is free to either apply merging or expose raw data back to the caller.
*
* NOTE: If readBlockLazily is
* turned on, does not merge, instead keeps reading log blocks and merges everything at once This is an optimization to
* avoid seek() back and forth to read new block (forward seek()) and lazily read content of seen block (reverse and
* forward seek()) during merge | | Read Block 1 Metadata | | Read Block 1 Data | | | Read Block 2
* Metadata | | Read Block 2 Data | | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | |
* | Read Block N Metadata | | Read Block N Data | <p> This results in two I/O passes over the log file.
*/
public abstract class AbstractHoodieLogRecordScanner {
private static final Logger log = LogManager.getLogger(AbstractHoodieLogRecordScanner.class);
// Reader schema for the records
protected final Schema readerSchema;
// Latest valid instant time
// Log-Blocks belonging to inflight delta-instants are filtered-out using this high-watermark.
private final String latestInstantTime;
private final HoodieTableMetaClient hoodieTableMetaClient;
// Merge strategy to use when combining records from log
private final String payloadClassFQN;
// Log File Paths
private final List<String> logFilePaths;
// Read Lazily flag
private final boolean readBlocksLazily;
// Reverse reader - Not implemented yet (NA -> Why do we need ?)
// but present here for plumbing for future implementation
private final boolean reverseReader;
// Buffer Size for log file reader
private final int bufferSize;
// FileSystem
private final FileSystem fs;
// Total log files read - for metrics
private AtomicLong totalLogFiles = new AtomicLong(0);
// Total log blocks read - for metrics
private AtomicLong totalLogBlocks = new AtomicLong(0);
// Total log records read - for metrics
private AtomicLong totalLogRecords = new AtomicLong(0);
// Total number of rollbacks written across all log files
private AtomicLong totalRollbacks = new AtomicLong(0);
// Total number of corrupt blocks written across all log files
private AtomicLong totalCorruptBlocks = new AtomicLong(0);
// Store the last instant log blocks (needed to implement rollback)
private Deque<HoodieLogBlock> currentInstantLogBlocks = new ArrayDeque<>();
// Progress
private float progress = 0.0f;
// TODO (NA) - Change this to a builder, this constructor is too long
public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths, Schema readerSchema,
String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize) {
this.readerSchema = readerSchema;
this.latestInstantTime = latestInstantTime;
this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
// load class from the payload fully qualified class name
this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass();
this.totalLogFiles.addAndGet(logFilePaths.size());
this.logFilePaths = logFilePaths;
this.readBlocksLazily = readBlocksLazily;
this.reverseReader = reverseReader;
this.fs = fs;
this.bufferSize = bufferSize;
}
/**
* Scan Log files
*/
public void scan() {
HoodieLogFormatReader logFormatReaderWrapper = null;
try {
// iterate over the paths
logFormatReaderWrapper =
new HoodieLogFormatReader(fs,
logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile)))
.collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader, bufferSize);
Set<HoodieLogFile> scannedLogFiles = new HashSet<>();
while (logFormatReaderWrapper.hasNext()) {
HoodieLogFile logFile = logFormatReaderWrapper.getLogFile();
log.info("Scanning log file " + logFile);
scannedLogFiles.add(logFile);
totalLogFiles.set(scannedLogFiles.size());
// Use the HoodieLogFileReader to iterate through the blocks in the log file
HoodieLogBlock r = logFormatReaderWrapper.next();
totalLogBlocks.incrementAndGet();
if (r.getBlockType() != CORRUPT_BLOCK
&& !HoodieTimeline.compareTimestamps(r.getLogBlockHeader().get(INSTANT_TIME),
this.latestInstantTime,
HoodieTimeline.LESSER_OR_EQUAL)) {
//hit a block with instant time greater than should be processed, stop processing further
break;
}
switch (r.getBlockType()) {
case AVRO_DATA_BLOCK:
log.info("Reading a data block from file " + logFile.getPath());
if (isNewInstantBlock(r) && !readBlocksLazily) {
// If this is an avro data block belonging to a different commit/instant,
// then merge the last blocks and records into the main result
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size());
}
// store the current block
currentInstantLogBlocks.push(r);
break;
case DELETE_BLOCK:
log.info("Reading a delete block from file " + logFile.getPath());
if (isNewInstantBlock(r) && !readBlocksLazily) {
// If this is a delete data block belonging to a different commit/instant,
// then merge the last blocks and records into the main result
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size());
}
// store deletes so can be rolled back
currentInstantLogBlocks.push(r);
break;
case COMMAND_BLOCK:
// Consider the following scenario
// (Time 0, C1, Task T1) -> Running
// (Time 1, C1, Task T1) -> Failed (Wrote either a corrupt block or a correct
// DataBlock (B1) with commitTime C1
// (Time 2, C1, Task T1.2) -> Running (Task T1 was retried and the attempt number is 2)
// (Time 3, C1, Task T1.2) -> Finished (Wrote a correct DataBlock B2)
// Now a logFile L1 can have 2 correct Datablocks (B1 and B2) which are the same.
// Say, commit C1 eventually failed and a rollback is triggered.
// Rollback will write only 1 rollback block (R1) since it assumes one block is
// written per ingestion batch for a file but in reality we need to rollback (B1 & B2)
// The following code ensures the same rollback block (R1) is used to rollback
// both B1 & B2
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.getLogBlockHeader()
.get(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME);
switch (commandBlock.getType()) { // there can be different types of command blocks
case ROLLBACK_PREVIOUS_BLOCK:
// Rollback the last read log block
// Get commit time from last record block, compare with targetCommitTime,
// rollback only if equal, this is required in scenarios of invalid/extra
// rollback blocks written due to failures during the rollback operation itself
// and ensures the same rollback block (R1) is used to rollback both B1 & B2 with
// same instant_time
int numBlocksRolledBack = 0;
totalRollbacks.incrementAndGet();
while (!currentInstantLogBlocks.isEmpty()) {
HoodieLogBlock lastBlock = currentInstantLogBlocks.peek();
// handle corrupt blocks separately since they may not have metadata
if (lastBlock.getBlockType() == CORRUPT_BLOCK) {
log.info(
"Rolling back the last corrupted log block read in " + logFile.getPath());
currentInstantLogBlocks.pop();
numBlocksRolledBack++;
} else if (lastBlock.getBlockType() != CORRUPT_BLOCK
&& targetInstantForCommandBlock
.contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) {
// rollback last data block or delete block
log.info("Rolling back the last log block read in " + logFile.getPath());
currentInstantLogBlocks.pop();
numBlocksRolledBack++;
} else if (!targetInstantForCommandBlock
.contentEquals(
currentInstantLogBlocks.peek().getLogBlockHeader().get(INSTANT_TIME))) {
// invalid or extra rollback block
log.warn("TargetInstantTime " + targetInstantForCommandBlock
+ " invalid or extra rollback command block in " + logFile.getPath());
break;
} else {
// this should not happen ideally
log.warn("Unable to apply rollback command block in " + logFile.getPath());
}
}
log.info("Number of applied rollback blocks " + numBlocksRolledBack);
break;
default:
throw new UnsupportedOperationException("Command type not yet supported.");
}
break;
case CORRUPT_BLOCK:
log.info("Found a corrupt block in " + logFile.getPath());
totalCorruptBlocks.incrementAndGet();
// If there is a corrupt block - we will assume that this was the next data block
currentInstantLogBlocks.push(r);
break;
default:
throw new UnsupportedOperationException("Block type not supported yet");
}
}
// merge the last read block when all the blocks are done reading
if (!currentInstantLogBlocks.isEmpty()) {
log.info("Merging the final data blocks");
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size());
}
// Done
progress = 1.0f;
} catch (Exception e) {
log.error("Got exception when reading log file", e);
throw new HoodieIOException("IOException when reading log file ");
} finally {
try {
if (null != logFormatReaderWrapper) {
logFormatReaderWrapper.close();
}
} catch (IOException ioe) {
// Eat exception as we do not want to mask the original exception that can happen
log.error("Unable to close log format reader", ioe);
}
}
}
/**
* Checks if the current logblock belongs to a later instant
*/
private boolean isNewInstantBlock(HoodieLogBlock logBlock) {
return currentInstantLogBlocks.size() > 0
&& currentInstantLogBlocks.peek().getBlockType() != CORRUPT_BLOCK
&& !logBlock.getLogBlockHeader().get(INSTANT_TIME)
.contentEquals(currentInstantLogBlocks.peek().getLogBlockHeader().get(INSTANT_TIME));
}
/**
* Iterate over the GenericRecord in the block, read the hoodie key and partition path and
* call subclass processors to handle it.
*/
private void processAvroDataBlock(HoodieAvroDataBlock dataBlock) throws Exception {
// TODO (NA) - Implement getRecordItr() in HoodieAvroDataBlock and use that here
List<IndexedRecord> recs = dataBlock.getRecords();
totalLogRecords.addAndGet(recs.size());
for (IndexedRecord rec : recs) {
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord =
SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN);
processNextRecord(hoodieRecord);
}
}
/**
* Process next record
*
* @param hoodieRecord Hoodie Record to process
*/
protected abstract void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord)
throws Exception;
/**
* Process next deleted key
*
* @param key Deleted record key
*/
protected abstract void processNextDeletedKey(HoodieKey key);
/**
* Process the set of log blocks belonging to the last instant which is read fully.
*/
private void processQueuedBlocksForInstant(Deque<HoodieLogBlock> lastBlocks, int numLogFilesSeen)
throws Exception {
while (!lastBlocks.isEmpty()) {
log.info("Number of remaining logblocks to merge " + lastBlocks.size());
// poll the element at the bottom of the stack since that's the order it was inserted
HoodieLogBlock lastBlock = lastBlocks.pollLast();
switch (lastBlock.getBlockType()) {
case AVRO_DATA_BLOCK:
processAvroDataBlock((HoodieAvroDataBlock) lastBlock);
break;
case DELETE_BLOCK:
Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(this::processNextDeletedKey);
break;
case CORRUPT_BLOCK:
log.warn("Found a corrupt block which was not rolled back");
break;
default:
break;
}
}
// At this step the lastBlocks are consumed. We track approximate progress by number of log-files seen
progress = numLogFilesSeen - 1 / logFilePaths.size();
}
/**
* Return progress of scanning as a float between 0.0 to 1.0
*/
public float getProgress() {
return progress;
}
public long getTotalLogFiles() {
return totalLogFiles.get();
}
public long getTotalLogRecords() {
return totalLogRecords.get();
}
public long getTotalLogBlocks() {
return totalLogBlocks.get();
}
protected String getPayloadClassFQN() {
return payloadClassFQN;
}
public long getTotalRollbacks() {
return totalRollbacks.get();
}
public long getTotalCorruptBlocks() {
return totalCorruptBlocks.get();
}
}

View File

@@ -0,0 +1,417 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log;
import com.google.common.base.Preconditions;
import java.io.EOFException;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.BufferedFSInputStream;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
import org.apache.hudi.common.table.log.block.HoodieCorruptBlock;
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.CorruptedLogFileException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieNotSupportedException;
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)
*/
class HoodieLogFileReader implements HoodieLogFormat.Reader {
public static final int DEFAULT_BUFFER_SIZE = 16 * 1024 * 1024; // 16 MB
private static final Logger log = LogManager.getLogger(HoodieLogFileReader.class);
private final FSDataInputStream inputStream;
private final HoodieLogFile logFile;
private static final byte[] magicBuffer = new byte[6];
private final Schema readerSchema;
private HoodieLogFormat.LogFormatVersion nextBlockVersion;
private boolean readBlockLazily;
private long reverseLogFilePosition;
private long lastReverseLogFilePosition;
private boolean reverseReader;
private boolean closed = false;
HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
boolean readBlockLazily, boolean reverseReader) throws IOException {
FSDataInputStream fsDataInputStream = fs.open(logFile.getPath(), bufferSize);
if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) {
this.inputStream = new FSDataInputStream(
new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(),
bufferSize));
} else {
// fsDataInputStream.getWrappedStream() maybe a BufferedFSInputStream
// need to wrap in another BufferedFSInputStream the make bufferSize work?
this.inputStream = fsDataInputStream;
}
this.logFile = logFile;
this.readerSchema = readerSchema;
this.readBlockLazily = readBlockLazily;
this.reverseReader = reverseReader;
if (this.reverseReader) {
this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs
.getFileStatus(logFile.getPath()).getLen();
}
addShutDownHook();
}
HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema,
boolean readBlockLazily, boolean reverseReader) throws IOException {
this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readBlockLazily, reverseReader);
}
HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema)
throws IOException {
this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, false, false);
}
@Override
public HoodieLogFile getLogFile() {
return logFile;
}
/**
* Close the inputstream if not closed when the JVM exits
*/
private void addShutDownHook() {
Runtime.getRuntime().addShutdownHook(new Thread() {
public void run() {
try {
close();
} catch (Exception e) {
log.warn("unable to close input stream for log file " + logFile, e);
// fail silently for any sort of exception
}
}
});
}
// TODO : convert content and block length to long by using ByteBuffer, raw byte [] allows
// for max of Integer size
private HoodieLogBlock readBlock() throws IOException {
int blocksize = -1;
int type = -1;
HoodieLogBlockType blockType = null;
Map<HeaderMetadataType, String> header = null;
try {
// 1 Read the total size of the block
blocksize = (int) inputStream.readLong();
} catch (EOFException | CorruptedLogFileException e) {
// An exception reading any of the above indicates a corrupt block
// Create a corrupt block by finding the next MAGIC marker or EOF
return createCorruptBlock();
}
// We may have had a crash which could have written this block partially
// 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) {
return createCorruptBlock();
}
// 2. Read the version for this log format
this.nextBlockVersion = readVersion();
// 3. Read the block type for a log block
if (nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION) {
type = inputStream.readInt();
Preconditions.checkArgument(type < HoodieLogBlockType.values().length,
"Invalid block byte type found " + type);
blockType = HoodieLogBlockType.values()[type];
}
// 4. Read the header for a log block, if present
if (nextBlockVersion.hasHeader()) {
header = HoodieLogBlock.getLogMetadata(inputStream);
}
int contentLength = blocksize;
// 5. Read the content length for the content
if (nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION) {
contentLength = (int) inputStream.readLong();
}
// 6. Read the content or skip content based on IO vs Memory trade-off by client
// TODO - have a max block size and reuse this buffer in the ByteBuffer
// (hard to guess max block size for now)
long contentPosition = inputStream.getPos();
byte[] content = HoodieLogBlock.readOrSkipContent(inputStream, contentLength, readBlockLazily);
// 7. Read footer if any
Map<HeaderMetadataType, String> footer = null;
if (nextBlockVersion.hasFooter()) {
footer = HoodieLogBlock.getLogMetadata(inputStream);
}
// 8. Read log block length, if present. This acts as a reverse pointer when traversing a
// log file in reverse
long logBlockLength = 0;
if (nextBlockVersion.hasLogBlockLength()) {
logBlockLength = inputStream.readLong();
}
// 9. Read the log block end position in the log file
long blockEndPos = inputStream.getPos();
switch (blockType) {
// based on type read the block
case AVRO_DATA_BLOCK:
if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) {
return HoodieAvroDataBlock.getBlock(content, readerSchema);
} else {
return HoodieAvroDataBlock
.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, readerSchema, header, footer);
}
case DELETE_BLOCK:
return HoodieDeleteBlock
.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, header, footer);
case COMMAND_BLOCK:
return HoodieCommandBlock
.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, header, footer);
default:
throw new HoodieNotSupportedException("Unsupported Block " + blockType);
}
}
private HoodieLogBlock createCorruptBlock() throws IOException {
log.info("Log " + logFile + " has a corrupted block at " + inputStream.getPos());
long currentPos = inputStream.getPos();
long nextBlockOffset = scanForNextAvailableBlockOffset();
// Rewind to the initial start and read corrupted bytes till the nextBlockOffset
inputStream.seek(currentPos);
log.info("Next available block in " + logFile + " starts at " + nextBlockOffset);
int corruptedBlockSize = (int) (nextBlockOffset - currentPos);
long contentPosition = inputStream.getPos();
byte[] corruptedBytes = HoodieLogBlock
.readOrSkipContent(inputStream, corruptedBlockSize, readBlockLazily);
return HoodieCorruptBlock
.getBlock(logFile, inputStream, Option.ofNullable(corruptedBytes), readBlockLazily,
contentPosition, corruptedBlockSize, corruptedBlockSize, new HashMap<>(),
new HashMap<>());
}
private boolean isBlockCorrupt(int blocksize) throws IOException {
long currentPos = inputStream.getPos();
try {
if (FSUtils.isGCSInputStream(inputStream)) {
inputStream.seek(currentPos + blocksize - 1);
} else {
inputStream.seek(currentPos + blocksize);
}
} catch (EOFException e) {
// this is corrupt
// This seek is required because contract of seek() is different for naked DFSInputStream vs BufferedFSInputStream
// release-3.1.0-RC1/DFSInputStream.java#L1455
// release-3.1.0-RC1/BufferedFSInputStream.java#L73
inputStream.seek(currentPos);
return true;
}
try {
readMagic();
// all good - either we found the sync marker or EOF. Reset position and continue
return false;
} catch (CorruptedLogFileException e) {
// This is a corrupted block
return true;
} finally {
inputStream.seek(currentPos);
}
}
private long scanForNextAvailableBlockOffset() throws IOException {
while (true) {
long currentPos = inputStream.getPos();
try {
boolean hasNextMagic = hasNextMagic();
if (hasNextMagic) {
return currentPos;
} else {
// No luck - advance and try again
inputStream.seek(currentPos + 1);
}
} catch (EOFException e) {
return inputStream.getPos();
}
}
}
@Override
public void close() throws IOException {
if (!closed) {
this.inputStream.close();
closed = true;
}
}
@Override
/**
* hasNext is not idempotent. TODO - Fix this. It is okay for now - PR
*/
public boolean hasNext() {
try {
return readMagic();
} catch (IOException e) {
throw new HoodieIOException("IOException when reading logfile " + logFile, e);
}
}
/**
* Read log format version from log file.
*/
private HoodieLogFormat.LogFormatVersion readVersion() throws IOException {
return new HoodieLogFormatVersion(inputStream.readInt());
}
private boolean readMagic() throws IOException {
try {
boolean hasMagic = hasNextMagic();
if (!hasMagic) {
throw new CorruptedLogFileException(
logFile
+ "could not be read. Did not find the magic bytes at the start of the block");
}
return hasMagic;
} catch (EOFException e) {
// We have reached the EOF
return false;
}
}
private boolean hasNextMagic() throws IOException {
long pos = inputStream.getPos();
// 1. Read magic header from the start of the block
inputStream.readFully(magicBuffer, 0, 6);
if (!Arrays.equals(magicBuffer, HoodieLogFormat.MAGIC)) {
return false;
}
return true;
}
@Override
public HoodieLogBlock next() {
try {
// hasNext() must be called before next()
return readBlock();
} catch (IOException io) {
throw new HoodieIOException("IOException when reading logblock from log file " + logFile, io);
}
}
/**
* hasPrev is not idempotent
*/
@Override
public boolean hasPrev() {
try {
if (!this.reverseReader) {
throw new HoodieNotSupportedException("Reverse log reader has not been enabled");
}
reverseLogFilePosition = lastReverseLogFilePosition;
reverseLogFilePosition -= Long.BYTES;
lastReverseLogFilePosition = reverseLogFilePosition;
inputStream.seek(reverseLogFilePosition);
} catch (Exception e) {
// Either reached EOF while reading backwards or an exception
return false;
}
return true;
}
/**
* This is a reverse iterator Note: At any point, an instance of HoodieLogFileReader should either
* iterate reverse (prev) or forward (next). Doing both in the same instance is not supported
* WARNING : Every call to prev() should be preceded with hasPrev()
*/
@Override
public HoodieLogBlock prev() throws IOException {
if (!this.reverseReader) {
throw new HoodieNotSupportedException("Reverse log reader has not been enabled");
}
long blockSize = inputStream.readLong();
long blockEndPos = inputStream.getPos();
// blocksize should read everything about a block including the length as well
try {
inputStream.seek(reverseLogFilePosition - blockSize);
} catch (Exception e) {
// this could be a corrupt block
inputStream.seek(blockEndPos);
throw new CorruptedLogFileException(
"Found possible corrupted block, cannot read log file in reverse, "
+ "fallback to forward reading of logfile");
}
boolean hasNext = hasNext();
reverseLogFilePosition -= blockSize;
lastReverseLogFilePosition = reverseLogFilePosition;
return next();
}
/**
* Reverse pointer, does not read the block. Return the current position of the log file (in
* reverse) If the pointer (inputstream) is moved in any way, it is the job of the client of this
* class to seek/reset it back to the file position returned from the method to expect correct
* results
*/
public long moveToPrev() throws IOException {
if (!this.reverseReader) {
throw new HoodieNotSupportedException("Reverse log reader has not been enabled");
}
inputStream.seek(lastReverseLogFilePosition);
long blockSize = inputStream.readLong();
// blocksize should be everything about a block including the length as well
inputStream.seek(reverseLogFilePosition - blockSize);
reverseLogFilePosition -= blockSize;
lastReverseLogFilePosition = reverseLogFilePosition;
return reverseLogFilePosition;
}
@Override
public void remove() {
throw new UnsupportedOperationException("Remove not supported for HoodieLogFileReader");
}
}

View File

@@ -0,0 +1,299 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log;
import java.io.Closeable;
import java.io.IOException;
import java.util.Iterator;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* File Format for Hoodie Log Files. The File Format consists of blocks each separated 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 6 bytes we put at the start of every block in the log file.
*/
byte[] MAGIC = new byte[] {'#', 'H', 'U', 'D', 'I', '#'};
/**
* The current version of the log format. Anytime the log format changes this version needs to be
* bumped and corresponding changes need to be made to {@link HoodieLogFormatVersion}
*/
int currentVersion = 1;
String UNKNOWN_WRITE_TOKEN = "1-0-1";
/**
* Writer interface to allow appending block to this file format
*/
interface Writer extends Closeable {
/**
* @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;
}
/**
* Reader interface which is an Iterator of HoodieLogBlock
*/
interface Reader extends Closeable, Iterator<HoodieLogBlock> {
/**
* @return the path to this {@link HoodieLogFormat}
*/
HoodieLogFile getLogFile();
/**
* Read log file in reverse order and check if prev block is present
* @return
*/
public boolean hasPrev();
/**
* Read log file in reverse order and return prev block if present
* @return
* @throws IOException
*/
public HoodieLogBlock prev() throws IOException;
}
/**
* Builder class to construct the default log format writer
*/
class WriterBuilder {
private static final Logger log = LogManager.getLogger(WriterBuilder.class);
// Default max log file size 512 MB
public static final long DEFAULT_SIZE_THRESHOLD = 512 * 1024 * 1024L;
// Buffer size
private Integer bufferSize;
// Replication for the log file
private Short replication;
// FileSystem
private FileSystem fs;
// Size threshold for the log file. Useful when used with a rolling log appender
private Long sizeThreshold;
// Log File extension. Could be .avro.delta or .avro.commits etc
private String fileExtension;
// File Id
private String logFileId;
// File Commit Time stamp
private String commitTime;
// version number for this log file. If not specified, then the current version will be
// computed by inspecting the file system
private Integer logVersion;
// Location of the directory containing the log
private Path parentPath;
// Log File Write Token
private String logWriteToken;
// Rollover Log file write token
private String rolloverLogWriteToken;
public WriterBuilder withBufferSize(int bufferSize) {
this.bufferSize = bufferSize;
return this;
}
public WriterBuilder withReplication(short replication) {
this.replication = replication;
return this;
}
public WriterBuilder withLogWriteToken(String writeToken) {
this.logWriteToken = writeToken;
return this;
}
public WriterBuilder withRolloverLogWriteToken(String rolloverLogWriteToken) {
this.rolloverLogWriteToken = rolloverLogWriteToken;
return this;
}
public WriterBuilder withFs(FileSystem fs) {
this.fs = fs;
return this;
}
public WriterBuilder withSizeThreshold(long sizeThreshold) {
this.sizeThreshold = sizeThreshold;
return this;
}
public WriterBuilder withFileExtension(String logFileExtension) {
this.fileExtension = logFileExtension;
return this;
}
public WriterBuilder withFileId(String fileId) {
this.logFileId = fileId;
return this;
}
public WriterBuilder overBaseCommit(String baseCommit) {
this.commitTime = baseCommit;
return this;
}
public WriterBuilder withLogVersion(int version) {
this.logVersion = version;
return this;
}
public WriterBuilder onParentPath(Path parentPath) {
this.parentPath = parentPath;
return this;
}
public Writer build() throws IOException, InterruptedException {
log.info("Building HoodieLogFormat Writer");
if (fs == null) {
throw new IllegalArgumentException("fs is not specified");
}
if (logFileId == null) {
throw new IllegalArgumentException("FileID is not specified");
}
if (commitTime == null) {
throw new IllegalArgumentException("BaseCommitTime is not specified");
}
if (fileExtension == null) {
throw new IllegalArgumentException("File extension is not specified");
}
if (parentPath == null) {
throw new IllegalArgumentException("Log file parent location is not specified");
}
if (rolloverLogWriteToken == null) {
rolloverLogWriteToken = UNKNOWN_WRITE_TOKEN;
}
if (logVersion == null) {
log.info("Computing the next log version for " + logFileId + " in " + parentPath);
Option<Pair<Integer, String>> versionAndWriteToken =
FSUtils.getLatestLogVersion(fs, parentPath, logFileId, fileExtension, commitTime);
if (versionAndWriteToken.isPresent()) {
logVersion = versionAndWriteToken.get().getKey();
logWriteToken = versionAndWriteToken.get().getValue();
} else {
logVersion = HoodieLogFile.LOGFILE_BASE_VERSION;
// this is the case where there is no existing log-file.
// Use rollover write token as write token to create new log file with tokens
logWriteToken = rolloverLogWriteToken;
}
log.info(
"Computed the next log version for " + logFileId + " in " + parentPath + " as "
+ logVersion + " with write-token " + logWriteToken);
}
if (logWriteToken == null) {
// This is the case where we have existing log-file with old format. rollover to avoid any conflicts
logVersion += 1;
logWriteToken = rolloverLogWriteToken;
}
Path logPath = new Path(parentPath,
FSUtils.makeLogFileName(logFileId, fileExtension, commitTime, logVersion, logWriteToken));
log.info("HoodieLogFile on path " + logPath);
HoodieLogFile logFile = new HoodieLogFile(logPath);
if (bufferSize == null) {
bufferSize = FSUtils.getDefaultBufferSize(fs);
}
if (replication == null) {
replication = FSUtils.getDefaultReplication(fs, parentPath);
}
if (sizeThreshold == null) {
sizeThreshold = DEFAULT_SIZE_THRESHOLD;
}
return new HoodieLogFormatWriter(fs, logFile, bufferSize, replication, sizeThreshold, logWriteToken,
rolloverLogWriteToken);
}
}
static WriterBuilder newWriterBuilder() {
return new WriterBuilder();
}
static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema)
throws IOException {
return new HoodieLogFileReader(fs, logFile, readerSchema, HoodieLogFileReader.DEFAULT_BUFFER_SIZE, false, false);
}
static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, boolean
readBlockLazily, boolean reverseReader)
throws IOException {
return new HoodieLogFileReader(fs, logFile, readerSchema, HoodieLogFileReader.DEFAULT_BUFFER_SIZE,
readBlockLazily, reverseReader);
}
/**
* A set of feature flags associated with a log format. Versions are changed when the log format
* changes. TODO(na) - Implement policies around major/minor versions
*/
abstract class LogFormatVersion {
private final int version;
LogFormatVersion(int version) {
this.version = version;
}
public int getVersion() {
return version;
}
public abstract boolean hasMagicHeader();
public abstract boolean hasContent();
public abstract boolean hasContentLength();
public abstract boolean hasOrdinal();
public abstract boolean hasHeader();
public abstract boolean hasFooter();
public abstract boolean hasLogBlockLength();
}
}

View File

@@ -0,0 +1,132 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
private final List<HoodieLogFile> logFiles;
// Readers for previously scanned log-files that are still open
private final List<HoodieLogFileReader> prevReadersInOpenState;
private HoodieLogFileReader currentReader;
private final FileSystem fs;
private final Schema readerSchema;
private final boolean readBlocksLazily;
private final boolean reverseLogReader;
private int bufferSize;
private static final Logger log = LogManager.getLogger(HoodieLogFormatReader.class);
HoodieLogFormatReader(FileSystem fs, List<HoodieLogFile> logFiles,
Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader, int bufferSize) throws IOException {
this.logFiles = logFiles;
this.fs = fs;
this.readerSchema = readerSchema;
this.readBlocksLazily = readBlocksLazily;
this.reverseLogReader = reverseLogReader;
this.bufferSize = bufferSize;
this.prevReadersInOpenState = new ArrayList<>();
if (logFiles.size() > 0) {
HoodieLogFile nextLogFile = logFiles.remove(0);
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false);
}
}
@Override
/**
* Note : In lazy mode, clients must ensure close() should be called only after processing
* all log-blocks as the underlying inputstream will be closed.
* TODO: We can introduce invalidate() API at HoodieLogBlock and this object can call invalidate on
* all returned log-blocks so that we check this scenario specifically in HoodieLogBlock
*/
public void close() throws IOException {
for (HoodieLogFileReader reader : prevReadersInOpenState) {
reader.close();
}
prevReadersInOpenState.clear();
if (currentReader != null) {
currentReader.close();
}
}
@Override
public boolean hasNext() {
if (currentReader == null) {
return false;
} else if (currentReader.hasNext()) {
return true;
} else if (logFiles.size() > 0) {
try {
HoodieLogFile nextLogFile = logFiles.remove(0);
// First close previous reader only if readBlockLazily is true
if (!readBlocksLazily) {
this.currentReader.close();
} else {
this.prevReadersInOpenState.add(currentReader);
}
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily,
false);
} catch (IOException io) {
throw new HoodieIOException("unable to initialize read with log file ", io);
}
log.info("Moving to the next reader for logfile " + currentReader.getLogFile());
return this.currentReader.hasNext();
}
return false;
}
@Override
public HoodieLogBlock next() {
return currentReader.next();
}
@Override
public HoodieLogFile getLogFile() {
return currentReader.getLogFile();
}
@Override
public void remove() {
}
@Override
public boolean hasPrev() {
return this.currentReader.hasPrev();
}
@Override
public HoodieLogBlock prev() throws IOException {
return this.currentReader.prev();
}
}

View File

@@ -0,0 +1,106 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log;
/**
* Implements logic to determine behavior for feature flags for
* {@link HoodieLogFormat.LogFormatVersion}.
*/
final class HoodieLogFormatVersion extends HoodieLogFormat.LogFormatVersion {
public static final int DEFAULT_VERSION = 0;
HoodieLogFormatVersion(int version) {
super(version);
}
@Override
public boolean hasMagicHeader() {
switch (super.getVersion()) {
case DEFAULT_VERSION:
return true;
default:
return true;
}
}
@Override
public boolean hasContent() {
switch (super.getVersion()) {
case DEFAULT_VERSION:
return true;
default:
return true;
}
}
@Override
public boolean hasContentLength() {
switch (super.getVersion()) {
case DEFAULT_VERSION:
return true;
default:
return true;
}
}
@Override
public boolean hasOrdinal() {
switch (super.getVersion()) {
case DEFAULT_VERSION:
return true;
default:
return true;
}
}
@Override
public boolean hasHeader() {
switch (super.getVersion()) {
case DEFAULT_VERSION:
return false;
default:
return true;
}
}
@Override
public boolean hasFooter() {
switch (super.getVersion()) {
case DEFAULT_VERSION:
return false;
case 1:
return true;
default:
return false;
}
}
@Override
public boolean hasLogBlockLength() {
switch (super.getVersion()) {
case DEFAULT_VERSION:
return false;
case 1:
return true;
default:
return false;
}
}
}

View File

@@ -0,0 +1,266 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log;
import java.io.IOException;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.storage.StorageSchemes;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.HoodieLogFormat.WriterBuilder;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
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
*/
public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
private static final Logger log = LogManager.getLogger(HoodieLogFormatWriter.class);
private HoodieLogFile logFile;
private final FileSystem fs;
private final long sizeThreshold;
private final Integer bufferSize;
private final Short replication;
private final String logWriteToken;
private final String rolloverLogWriteToken;
private FSDataOutputStream output;
private static final String APPEND_UNAVAILABLE_EXCEPTION_MESSAGE = "not sufficiently replicated yet";
/**
* @param fs
* @param logFile
* @param bufferSize
* @param replication
* @param sizeThreshold
*/
HoodieLogFormatWriter(FileSystem fs, HoodieLogFile logFile, Integer bufferSize,
Short replication, Long sizeThreshold, String logWriteToken, String rolloverLogWriteToken)
throws IOException, InterruptedException {
this.fs = fs;
this.logFile = logFile;
this.sizeThreshold = sizeThreshold;
this.bufferSize = bufferSize;
this.replication = replication;
this.logWriteToken = logWriteToken;
this.rolloverLogWriteToken = rolloverLogWriteToken;
Path path = logFile.getPath();
if (fs.exists(path)) {
boolean isAppendSupported = StorageSchemes.isAppendSupported(fs.getScheme());
if (isAppendSupported) {
log.info(logFile + " exists. Appending to existing file");
try {
this.output = fs.append(path, bufferSize);
} catch (RemoteException e) {
log.warn("Remote Exception, attempting to handle or recover lease", e);
handleAppendExceptionOrRecoverLease(path, e);
} catch (IOException ioe) {
if (ioe.getMessage().toLowerCase().contains("not supported")) {
// may still happen if scheme is viewfs.
isAppendSupported = false;
} else {
throw ioe;
}
}
}
if (!isAppendSupported) {
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
log.info("Append not supported.. Rolling over to " + logFile);
createNewFile();
}
} else {
log.info(logFile + " does not exist. Create a new file");
// Block size does not matter as we will always manually autoflush
createNewFile();
}
}
public FileSystem getFs() {
return fs;
}
public HoodieLogFile getLogFile() {
return logFile;
}
public long getSizeThreshold() {
return sizeThreshold;
}
@Override
public Writer appendBlock(HoodieLogBlock block)
throws IOException, InterruptedException {
// Find current version
HoodieLogFormat.LogFormatVersion currentLogFormatVersion = new HoodieLogFormatVersion(
HoodieLogFormat.currentVersion);
long currentSize = this.output.size();
// 1. Write the magic header for the start of the block
this.output.write(HoodieLogFormat.MAGIC);
// bytes for header
byte[] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader());
// content bytes
byte[] content = block.getContentBytes();
// bytes for footer
byte[] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter());
// 2. Write the total size of the block (excluding Magic)
this.output
.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length));
// 3. Write the version of this log block
this.output.writeInt(currentLogFormatVersion.getVersion());
// 4. Write the block type
this.output.writeInt(block.getBlockType().ordinal());
// 5. Write the headers for the log block
this.output.write(headerBytes);
// 6. Write the size of the content block
this.output.writeLong(content.length);
// 7. Write the contents of the data block
this.output.write(content);
// 8. Write the footers for the log block
this.output.write(footerBytes);
// 9. Write the total size of the log block (including magic) which is everything written
// until now (for reverse pointer)
this.output.writeLong(this.output.size() - currentSize);
// Flush every block to disk
flush();
// roll over if size is past the threshold
return rolloverIfNeeded();
}
/**
* This method returns the total LogBlock Length which is the sum of 1. Number of bytes to write
* version 2. Number of bytes to write ordinal 3. Length of the headers 4. Number of bytes used to
* write content length 5. Length of the content 6. Length of the footers 7. Number of bytes to
* write totalLogBlockLength
*/
private int getLogBlockLength(int contentLength, int headerLength, int footerLength) {
return
Integer.BYTES + // Number of bytes to write version
Integer.BYTES + // Number of bytes to write ordinal
headerLength + // Length of the headers
Long.BYTES + // Number of bytes used to write content length
contentLength + // Length of the content
footerLength + // Length of the footers
Long.BYTES; // bytes to write totalLogBlockLength at end of block (for reverse ptr)
}
private Writer rolloverIfNeeded() throws IOException, InterruptedException {
// Roll over if the size is past the threshold
if (getCurrentSize() > sizeThreshold) {
//TODO - make an end marker which seals the old log file (no more appends possible to that
// file).
log.info("CurrentSize " + getCurrentSize() + " has reached threshold " + sizeThreshold
+ ". Rolling over to the next version");
HoodieLogFile newLogFile = logFile.rollOver(fs, rolloverLogWriteToken);
// close this writer and return the new writer
close();
return new HoodieLogFormatWriter(fs, newLogFile, bufferSize, replication, sizeThreshold, logWriteToken,
rolloverLogWriteToken);
}
return this;
}
private void createNewFile() throws IOException {
this.output = fs.create(this.logFile.getPath(), false, bufferSize, replication,
WriterBuilder.DEFAULT_SIZE_THRESHOLD, null);
}
@Override
public void close() throws IOException {
flush();
output.close();
output = null;
}
private void flush() throws IOException {
if (output == null) {
return; // Presume closed
}
output.flush();
// NOTE : the following API call makes sure that the data is flushed to disk on DataNodes (akin to POSIX fsync())
// See more details here : https://issues.apache.org/jira/browse/HDFS-744
output.hsync();
}
public long getCurrentSize() throws IOException {
if (output == null) {
throw new IllegalStateException(
"Cannot get current size as the underlying stream has been closed already");
}
return output.getPos();
}
private void handleAppendExceptionOrRecoverLease(Path path, RemoteException e) throws IOException,
InterruptedException {
if (e.getMessage().contains(APPEND_UNAVAILABLE_EXCEPTION_MESSAGE)) {
// This issue happens when all replicas for a file are down and/or being decommissioned.
// The fs.append() API could append to the last block for a file. If the last block is full, a new block is
// appended to. In a scenario when a lot of DN's are decommissioned, it can happen that DN's holding all
// replicas for a block/file are decommissioned together. During this process, all these blocks will start to
// get replicated to other active DataNodes but this process might take time (can be of the order of few
// hours). During this time, if a fs.append() API is invoked for a file whose last block is eligible to be
// appended to, then the NN will throw an exception saying that it couldn't find any active replica with the
// last block. Find more information here : https://issues.apache.org/jira/browse/HDFS-6325
log.warn("Failed to open an append stream to the log file. Opening a new log file..", e);
// Rollover the current log file (since cannot get a stream handle) and create new one
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
createNewFile();
} else if (e.getClassName().contentEquals(AlreadyBeingCreatedException.class.getName())) {
log.warn("Another task executor writing to the same log file(" + logFile + ". Rolling over");
// Rollover the current log file (since cannot get a stream handle) and create new one
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
createNewFile();
} else if (e.getClassName().contentEquals(RecoveryInProgressException.class.getName())
&& (fs instanceof DistributedFileSystem)) {
// this happens when either another task executor writing to this file died or
// data node is going down. Note that we can only try to recover lease for a DistributedFileSystem.
// ViewFileSystem unfortunately does not support this operation
log.warn("Trying to recover log on path " + path);
if (FSUtils.recoverDFSFileLease((DistributedFileSystem) fs, path)) {
log.warn("Recovered lease on path " + path);
// try again
this.output = fs.append(path, bufferSize);
} else {
log.warn("Failed to recover lease on path " + path);
throw new HoodieException(e);
}
} else {
throw new HoodieIOException("Failed to open an append stream ", e);
}
}
}

View File

@@ -0,0 +1,131 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.HoodieRecordSizeEstimator;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.exception.HoodieIOException;
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.
*
* NOTE: If readBlockLazily is
* turned on, does not merge, instead keeps reading log blocks and merges everything at once This is an optimization to
* avoid seek() back and forth to read new block (forward seek()) and lazily read content of seen block (reverse and
* forward seek()) during merge | | Read Block 1 Metadata | | Read Block 1 Data | | | Read Block 2
* Metadata | | Read Block 2 Data | | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | |
* | Read Block N Metadata | | Read Block N Data | <p> This results in two I/O passes over the log file.
*/
public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner
implements Iterable<HoodieRecord<? extends HoodieRecordPayload>> {
private static final Logger log = LogManager.getLogger(HoodieMergedLogRecordScanner.class);
// Final map of compacted/merged records
private final ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records;
// count of merged records in log
private long numMergedRecordsInLog;
// Stores the total time taken to perform reading and merging of log blocks
private final long totalTimeTakenToReadAndMergeBlocks;
// A timer for calculating elapsed time in millis
public final HoodieTimer timer = new HoodieTimer();
@SuppressWarnings("unchecked")
public HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths,
Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes,
boolean readBlocksLazily, boolean reverseReader, int bufferSize, String spillableMapBasePath) {
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize);
try {
// Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize
this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath,
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(readerSchema));
// Do the scan and merge
timer.startTimer();
scan();
this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer();
this.numMergedRecordsInLog = records.size();
log.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes);
log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records
.getInMemoryMapNumEntries());
log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records
.getCurrentInMemoryMapSize());
log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records
.getDiskBasedMapNumEntries());
log.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes());
} catch (IOException e) {
throw new HoodieIOException("IOException when reading log file ");
}
}
@Override
public Iterator<HoodieRecord<? extends HoodieRecordPayload>> iterator() {
return records.iterator();
}
public Map<String, HoodieRecord<? extends HoodieRecordPayload>> getRecords() {
return records;
}
public long getNumMergedRecordsInLog() {
return numMergedRecordsInLog;
}
@Override
protected void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) throws IOException {
String key = hoodieRecord.getRecordKey();
if (records.containsKey(key)) {
// Merge and store the merged record. The HoodieRecordPayload implementation is free to decide what should be
// done when a delete (empty payload) is encountered before or after an insert/update.
HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(records.get(key).getData());
records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue));
} else {
// Put the record as is
records.put(key, hoodieRecord);
}
}
@Override
protected void processNextDeletedKey(HoodieKey hoodieKey) {
records.put(hoodieKey.getRecordKey(), SpillableMapUtils.generateEmptyPayload(hoodieKey.getRecordKey(),
hoodieKey.getPartitionPath(), getPayloadClassFQN()));
}
public long getTotalTimeTakenToReadAndMergeBlocks() {
return totalTimeTakenToReadAndMergeBlocks;
}
}

View File

@@ -0,0 +1,56 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordScanner {
private final LogRecordScannerCallback callback;
public HoodieUnMergedLogRecordScanner(FileSystem fs, String basePath,
List<String> logFilePaths, Schema readerSchema, String latestInstantTime,
boolean readBlocksLazily, boolean reverseReader, int bufferSize,
LogRecordScannerCallback callback) {
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize);
this.callback = callback;
}
@Override
protected void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) throws Exception {
// Just call callback without merging
callback.apply(hoodieRecord);
}
@Override
protected void processNextDeletedKey(HoodieKey key) {
throw new IllegalStateException("Not expected to see delete records in this log-scan mode. Check Job Config");
}
@FunctionalInterface
public static interface LogRecordScannerCallback {
public void apply(HoodieRecord<? extends HoodieRecordPayload> record) throws Exception;
}
}

View File

@@ -0,0 +1,319 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log.block;
import com.google.common.annotations.VisibleForTesting;
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.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import javax.annotation.Nonnull;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.IndexedRecord;
import org.apache.avro.io.BinaryDecoder;
import org.apache.avro.io.BinaryEncoder;
import org.apache.avro.io.Decoder;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.Encoder;
import org.apache.avro.io.EncoderFactory;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.storage.SizeAwareDataInputStream;
import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
/**
* DataBlock contains a list of records serialized using Avro. The Datablock contains 1. Data Block
* version 2. Total number of records in the block 3. Size of a record 4. Actual avro serialized
* content of the record
*/
public class HoodieAvroDataBlock extends HoodieLogBlock {
private List<IndexedRecord> records;
private Schema schema;
private ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
private ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
public HoodieAvroDataBlock(@Nonnull List<IndexedRecord> records,
@Nonnull Map<HeaderMetadataType, String> header,
@Nonnull Map<HeaderMetadataType, String> footer) {
super(header, footer, Option.empty(), Option.empty(), null, false);
this.records = records;
this.schema = Schema.parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
}
public HoodieAvroDataBlock(@Nonnull List<IndexedRecord> records,
@Nonnull Map<HeaderMetadataType, String> header) {
this(records, header, new HashMap<>());
}
private HoodieAvroDataBlock(Option<byte[]> content, @Nonnull FSDataInputStream inputStream,
boolean readBlockLazily, Option<HoodieLogBlockContentLocation> blockContentLocation,
Schema readerSchema, @Nonnull Map<HeaderMetadataType, String> headers,
@Nonnull Map<HeaderMetadataType, String> footer) {
super(headers, footer, blockContentLocation, content, inputStream, readBlockLazily);
this.schema = readerSchema;
}
public static HoodieLogBlock getBlock(HoodieLogFile logFile,
FSDataInputStream inputStream,
Option<byte[]> content,
boolean readBlockLazily,
long position,
long blockSize,
long blockEndpos,
Schema readerSchema,
Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer) {
return new HoodieAvroDataBlock(content, inputStream, readBlockLazily,
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)),
readerSchema, header, footer);
}
@Override
public byte[] getContentBytes() throws IOException {
// In case this method is called before realizing records from content
if (getContent().isPresent()) {
return getContent().get();
} else if (readBlockLazily && !getContent().isPresent() && records == null) {
// read block lazily
createRecordsFromContentBytes();
}
Schema schema = Schema.parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
GenericDatumWriter<IndexedRecord> writer = new GenericDatumWriter<>(schema);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
// 1. Write out the log block version
output.writeInt(HoodieLogBlock.version);
// 2. Write total number of records
output.writeInt(records.size());
// 3. Write the records
Iterator<IndexedRecord> itr = records.iterator();
while (itr.hasNext()) {
IndexedRecord s = itr.next();
ByteArrayOutputStream temp = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(temp, encoderCache.get());
encoderCache.set(encoder);
try {
// Encode the record into bytes
writer.write(s, encoder);
encoder.flush();
// Get the size of the bytes
int size = temp.toByteArray().length;
// Write the record size
output.writeInt(size);
// Write the content
output.write(temp.toByteArray());
itr.remove();
} catch (IOException e) {
throw new HoodieIOException("IOException converting HoodieAvroDataBlock to bytes", e);
}
}
output.close();
return baos.toByteArray();
}
@Override
public HoodieLogBlockType getBlockType() {
return HoodieLogBlockType.AVRO_DATA_BLOCK;
}
public List<IndexedRecord> getRecords() {
if (records == null) {
try {
// in case records are absent, read content lazily and then convert to IndexedRecords
createRecordsFromContentBytes();
} catch (IOException io) {
throw new HoodieIOException("Unable to convert content bytes to records", io);
}
}
return records;
}
public Schema getSchema() {
// if getSchema was invoked before converting byte [] to records
if (records == null) {
getRecords();
}
return schema;
}
//TODO (na) - Break down content into smaller chunks of byte [] to be GC as they are used
//TODO (na) - Implement a recordItr instead of recordList
private void createRecordsFromContentBytes() throws IOException {
if (readBlockLazily && !getContent().isPresent()) {
// read log block contents from disk
inflate();
}
SizeAwareDataInputStream dis =
new SizeAwareDataInputStream(
new DataInputStream(new ByteArrayInputStream(getContent().get())));
// 1. Read version for this data block
int version = dis.readInt();
HoodieAvroDataBlockVersion logBlockVersion = new HoodieAvroDataBlockVersion(version);
// Get schema from the header
Schema writerSchema = new Schema.Parser()
.parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
// If readerSchema was not present, use writerSchema
if (schema == null) {
schema = writerSchema;
}
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, schema);
// 2. Get the total records
int totalRecords = 0;
if (logBlockVersion.hasRecordCount()) {
totalRecords = dis.readInt();
}
List<IndexedRecord> records = new ArrayList<>(totalRecords);
// 3. Read the content
for (int i = 0; i < totalRecords; i++) {
int recordLength = dis.readInt();
BinaryDecoder decoder = DecoderFactory.get()
.binaryDecoder(getContent().get(), dis.getNumberOfBytesRead(), recordLength, decoderCache.get());
decoderCache.set(decoder);
IndexedRecord record = reader.read(null, decoder);
records.add(record);
dis.skipBytes(recordLength);
}
dis.close();
this.records = records;
// Free up content to be GC'd, deflate
deflate();
}
/*********************************DEPRECATED METHODS***********************************/
@Deprecated
@VisibleForTesting
/**
* This constructor is retained to provide backwards compatibility to HoodieArchivedLogs
* which were written using HoodieLogFormat V1
*/
public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema) {
super(new HashMap<>(), new HashMap<>(), Option.empty(), Option.empty(), null, false);
this.records = records;
this.schema = schema;
}
@Deprecated
/**
* This method is retained to provide backwards compatibility to HoodieArchivedLogs which
* were written using HoodieLogFormat V1
*/
public static HoodieLogBlock getBlock(byte[] content, Schema readerSchema) throws IOException {
SizeAwareDataInputStream dis = new SizeAwareDataInputStream(
new DataInputStream(new ByteArrayInputStream(content)));
// 1. Read the schema written out
int schemaLength = dis.readInt();
byte[] compressedSchema = new byte[schemaLength];
dis.readFully(compressedSchema, 0, schemaLength);
Schema writerSchema = new Schema.Parser().parse(HoodieAvroUtils.decompress(compressedSchema));
if (readerSchema == null) {
readerSchema = writerSchema;
}
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
// 2. Get the total records
int totalRecords = dis.readInt();
List<IndexedRecord> records = new ArrayList<>(totalRecords);
// 3. Read the content
for (int i = 0; i < totalRecords; i++) {
int recordLength = dis.readInt();
Decoder decoder = DecoderFactory.get()
.binaryDecoder(content, dis.getNumberOfBytesRead(), recordLength, null);
IndexedRecord record = reader.read(null, decoder);
records.add(record);
dis.skipBytes(recordLength);
}
dis.close();
return new HoodieAvroDataBlock(records, readerSchema);
}
@Deprecated
@VisibleForTesting
public byte[] getBytes(Schema schema) throws IOException {
GenericDatumWriter<IndexedRecord> writer = new GenericDatumWriter<>(schema);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
// 2. Compress and Write schema out
byte[] schemaContent = HoodieAvroUtils.compress(schema.toString());
output.writeInt(schemaContent.length);
output.write(schemaContent);
// 3. Write total number of records
output.writeInt(records.size());
// 4. Write the records
Iterator<IndexedRecord> itr = records.iterator();
while (itr.hasNext()) {
IndexedRecord s = itr.next();
ByteArrayOutputStream temp = new ByteArrayOutputStream();
Encoder encoder = EncoderFactory.get().binaryEncoder(temp, null);
try {
// Encode the record into bytes
writer.write(s, encoder);
encoder.flush();
// Get the size of the bytes
int size = temp.toByteArray().length;
// Write the record size
output.writeInt(size);
// Write the content
output.write(temp.toByteArray());
itr.remove();
} catch (IOException e) {
throw new HoodieIOException("IOException converting HoodieAvroDataBlock to bytes", e);
}
}
output.close();
return baos.toByteArray();
}
}

View File

@@ -0,0 +1,39 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log.block;
/**
* A set of feature flags associated with a data log block format. Versions are changed when the log
* block format changes. TODO(na) - Implement policies around major/minor versions
*/
final class HoodieAvroDataBlockVersion extends HoodieLogBlockVersion {
HoodieAvroDataBlockVersion(int version) {
super(version);
}
public boolean hasRecordCount() {
switch (super.getVersion()) {
case DEFAULT_VERSION:
return true;
default:
return true;
}
}
}

View File

@@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log.block;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.util.Option;
/**
* Command block issues a specific command to the scanner
*/
public class HoodieCommandBlock extends HoodieLogBlock {
private final HoodieCommandBlockTypeEnum type;
public enum HoodieCommandBlockTypeEnum {
ROLLBACK_PREVIOUS_BLOCK
}
public HoodieCommandBlock(Map<HeaderMetadataType, String> header) {
this(Option.empty(), null, false, Option.empty(), header, new HashMap<>());
}
private HoodieCommandBlock(Option<byte[]> content, FSDataInputStream inputStream,
boolean readBlockLazily, Option<HoodieLogBlockContentLocation> blockContentLocation,
Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> footer) {
super(header, footer, blockContentLocation, content, inputStream, readBlockLazily);
this.type = HoodieCommandBlockTypeEnum.values()[Integer
.parseInt(header.get(HeaderMetadataType.COMMAND_BLOCK_TYPE))];
}
public HoodieCommandBlockTypeEnum getType() {
return type;
}
@Override
public HoodieLogBlockType getBlockType() {
return HoodieLogBlockType.COMMAND_BLOCK;
}
@Override
public byte[] getContentBytes() {
return new byte[0];
}
public static HoodieLogBlock getBlock(HoodieLogFile logFile,
FSDataInputStream inputStream,
Option<byte[]> content,
boolean readBlockLazily,
long position,
long blockSize,
long blockEndpos,
Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer) {
return new HoodieCommandBlock(content, inputStream, readBlockLazily,
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)),
header, footer);
}
}

View File

@@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log.block;
/**
* A set of feature flags associated with a command log block format. Versions are changed when the
* log block format changes. TODO(na) - Implement policies around major/minor versions
*/
final class HoodieCommandBlockVersion extends HoodieLogBlockVersion {
HoodieCommandBlockVersion(int version) {
super(version);
}
}

View File

@@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log.block;
import java.io.IOException;
import java.util.Map;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.util.Option;
/**
* Corrupt block is emitted whenever the scanner finds the length of the block written at the
* beginning does not match (did not find a EOF or a sync marker after the length)
*/
public class HoodieCorruptBlock extends HoodieLogBlock {
private HoodieCorruptBlock(Option<byte[]> corruptedBytes, FSDataInputStream inputStream,
boolean readBlockLazily, Option<HoodieLogBlockContentLocation> blockContentLocation,
Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> footer) {
super(header, footer, blockContentLocation, corruptedBytes, inputStream, readBlockLazily);
}
@Override
public byte[] getContentBytes() throws IOException {
if (!getContent().isPresent() && readBlockLazily) {
// read content from disk
inflate();
}
return getContent().get();
}
@Override
public HoodieLogBlockType getBlockType() {
return HoodieLogBlockType.CORRUPT_BLOCK;
}
public static HoodieLogBlock getBlock(HoodieLogFile logFile,
FSDataInputStream inputStream,
Option<byte[]> corruptedBytes,
boolean readBlockLazily,
long position,
long blockSize,
long blockEndPos,
Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer) {
return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily,
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)),
header, footer);
}
}

View File

@@ -0,0 +1,118 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log.block;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.storage.SizeAwareDataInputStream;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SerializationUtils;
import org.apache.hudi.exception.HoodieIOException;
/**
* Delete block contains a list of keys to be deleted from scanning the blocks so far
*/
public class HoodieDeleteBlock extends HoodieLogBlock {
private HoodieKey[] keysToDelete;
public HoodieDeleteBlock(HoodieKey[] keysToDelete,
Map<HeaderMetadataType, String> header) {
this(Option.empty(), null, false, Option.empty(), header, new HashMap<>());
this.keysToDelete = keysToDelete;
}
private HoodieDeleteBlock(Option<byte[]> content, FSDataInputStream inputStream,
boolean readBlockLazily, Option<HoodieLogBlockContentLocation> blockContentLocation,
Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> footer) {
super(header, footer, blockContentLocation, content, inputStream, readBlockLazily);
}
@Override
public byte[] getContentBytes() throws IOException {
// In case this method is called before realizing keys from content
if (getContent().isPresent()) {
return getContent().get();
} else if (readBlockLazily && !getContent().isPresent() && keysToDelete == null) {
// read block lazily
getKeysToDelete();
}
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
byte[] bytesToWrite = SerializationUtils.serialize(getKeysToDelete());
output.writeInt(version);
output.writeInt(bytesToWrite.length);
output.write(bytesToWrite);
return baos.toByteArray();
}
public HoodieKey[] getKeysToDelete() {
try {
if (keysToDelete == null) {
if (!getContent().isPresent() && readBlockLazily) {
// read content from disk
inflate();
}
SizeAwareDataInputStream dis =
new SizeAwareDataInputStream(
new DataInputStream(new ByteArrayInputStream(getContent().get())));
int version = dis.readInt();
int dataLength = dis.readInt();
byte[] data = new byte[dataLength];
dis.readFully(data);
this.keysToDelete = SerializationUtils.<HoodieKey[]>deserialize(data);
deflate();
}
return keysToDelete;
} catch (IOException io) {
throw new HoodieIOException("Unable to generate keys to delete from block content", io);
}
}
@Override
public HoodieLogBlockType getBlockType() {
return HoodieLogBlockType.DELETE_BLOCK;
}
public static HoodieLogBlock getBlock(HoodieLogFile logFile,
FSDataInputStream inputStream,
Option<byte[]> content,
boolean readBlockLazily,
long position,
long blockSize,
long blockEndPos,
Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer) throws IOException {
return new HoodieDeleteBlock(content, inputStream, readBlockLazily,
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)),
header, footer);
}
}

View File

@@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log.block;
/**
* A set of feature flags associated with a delete log block format. Versions are changed when the
* log block format changes. TODO(na) - Implement policies around major/minor versions
*/
final class HoodieDeleteBlockVersion extends HoodieLogBlockVersion {
HoodieDeleteBlockVersion(int version) {
super(version);
}
}

View File

@@ -0,0 +1,289 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log.block;
import com.google.common.collect.Maps;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.util.Map;
import javax.annotation.Nonnull;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
/**
* Abstract class defining a block in HoodieLogFile
*/
public abstract class HoodieLogBlock {
/**
* The current version of the log block. Anytime the logBlock format changes this version needs to
* be bumped and corresponding changes need to be made to {@link HoodieLogBlockVersion} TODO :
* Change this to a class, something like HoodieLogBlockVersionV1/V2 and implement/override
* operations there
*/
public static int version = 1;
// Header for each log block
private final Map<HeaderMetadataType, String> logBlockHeader;
// Footer for each log block
private final Map<HeaderMetadataType, String> logBlockFooter;
// Location of a log block on disk
private final Option<HoodieLogBlockContentLocation> blockContentLocation;
// data for a specific block
private Option<byte[]> content;
// TODO : change this to just InputStream so this works for any FileSystem
// create handlers to return specific type of inputstream based on FS
// input stream corresponding to the log file where this logBlock belongs
protected FSDataInputStream inputStream;
// Toggle flag, whether to read blocks lazily (I/O intensive) or not (Memory intensive)
protected boolean readBlockLazily;
public HoodieLogBlock(@Nonnull Map<HeaderMetadataType, String> logBlockHeader,
@Nonnull Map<HeaderMetadataType, String> logBlockFooter,
@Nonnull Option<HoodieLogBlockContentLocation> blockContentLocation,
@Nonnull Option<byte[]> content,
FSDataInputStream inputStream,
boolean readBlockLazily) {
this.logBlockHeader = logBlockHeader;
this.logBlockFooter = logBlockFooter;
this.blockContentLocation = blockContentLocation;
this.content = content;
this.inputStream = inputStream;
this.readBlockLazily = readBlockLazily;
}
// Return the bytes representation of the data belonging to a LogBlock
public byte[] getContentBytes() throws IOException {
throw new HoodieException("No implementation was provided");
}
public byte[] getMagic() {
throw new HoodieException("No implementation was provided");
}
public HoodieLogBlockType getBlockType() {
throw new HoodieException("No implementation was provided");
}
public long getLogBlockLength() {
throw new HoodieException("No implementation was provided");
}
public Option<HoodieLogBlockContentLocation> getBlockContentLocation() {
return this.blockContentLocation;
}
public Map<HeaderMetadataType, String> getLogBlockHeader() {
return logBlockHeader;
}
public Map<HeaderMetadataType, String> getLogBlockFooter() {
return logBlockFooter;
}
public Option<byte[]> getContent() {
return content;
}
/**
* 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,
DELETE_BLOCK,
CORRUPT_BLOCK,
AVRO_DATA_BLOCK
}
/**
* Log Metadata headers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the
* ordinal. Only add new enums at the end.
*/
public enum HeaderMetadataType {
INSTANT_TIME,
TARGET_INSTANT_TIME,
SCHEMA,
COMMAND_BLOCK_TYPE
}
/**
* Log Metadata footers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the
* ordinal. Only add new enums at the end.
*/
public enum FooterMetadataType {
}
/**
* This class is used to store the Location of the Content of a Log Block. It's used when a client
* chooses for a IO intensive CompactedScanner, the location helps to lazily read contents from
* the log file
*/
public static final class HoodieLogBlockContentLocation {
// The logFile that contains this block
private final HoodieLogFile logFile;
// The filePosition in the logFile for the contents of this block
private final long contentPositionInLogFile;
// The number of bytes / size of the contents of this block
private final long blockSize;
// The final position where the complete block ends
private final long blockEndPos;
HoodieLogBlockContentLocation(HoodieLogFile logFile, long contentPositionInLogFile,
long blockSize, long blockEndPos) {
this.logFile = logFile;
this.contentPositionInLogFile = contentPositionInLogFile;
this.blockSize = blockSize;
this.blockEndPos = blockEndPos;
}
public HoodieLogFile getLogFile() {
return logFile;
}
public long getContentPositionInLogFile() {
return contentPositionInLogFile;
}
public long getBlockSize() {
return blockSize;
}
public long getBlockEndPos() {
return blockEndPos;
}
}
/**
* Convert log metadata to bytes 1. Write size of metadata 2. Write enum ordinal 3. Write actual
* bytes
*/
public static byte[] getLogMetadataBytes(Map<HeaderMetadataType, String> metadata)
throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream output = new DataOutputStream(baos);
output.writeInt(metadata.size());
for (Map.Entry<HeaderMetadataType, String> entry : metadata.entrySet()) {
output.writeInt(entry.getKey().ordinal());
byte[] bytes = entry.getValue().getBytes();
output.writeInt(bytes.length);
output.write(bytes);
}
return baos.toByteArray();
}
/**
* Convert bytes to LogMetadata, follow the same order as
* {@link HoodieLogBlock#getLogMetadataBytes}
*/
public static Map<HeaderMetadataType, String> getLogMetadata(DataInputStream dis)
throws IOException {
Map<HeaderMetadataType, String> metadata = Maps.newHashMap();
// 1. Read the metadata written out
int metadataCount = dis.readInt();
try {
while (metadataCount > 0) {
int metadataEntryIndex = dis.readInt();
int metadataEntrySize = dis.readInt();
byte[] metadataEntry = new byte[metadataEntrySize];
dis.readFully(metadataEntry, 0, metadataEntrySize);
metadata.put(HeaderMetadataType.values()[metadataEntryIndex], new String(metadataEntry));
metadataCount--;
}
return metadata;
} catch (EOFException eof) {
throw new IOException("Could not read metadata fields ", eof);
}
}
/**
* Read or Skip block content of a log block in the log file. Depends on lazy reading enabled in
* {@link HoodieMergedLogRecordScanner}
*/
public static byte[] readOrSkipContent(FSDataInputStream inputStream,
Integer contentLength, boolean readBlockLazily) throws IOException {
byte[] content = null;
if (!readBlockLazily) {
// Read the contents in memory
content = new byte[contentLength];
inputStream.readFully(content, 0, contentLength);
} else {
// Seek to the end of the content block
safeSeek(inputStream, inputStream.getPos() + contentLength);
}
return content;
}
/**
* When lazyReading of blocks is turned on, inflate the content of a log block from disk
*/
protected void inflate() throws IOException {
try {
content = Option.of(new byte[(int) this.getBlockContentLocation().get().getBlockSize()]);
safeSeek(inputStream, this.getBlockContentLocation().get().getContentPositionInLogFile());
inputStream.readFully(content.get(), 0, content.get().length);
safeSeek(inputStream, this.getBlockContentLocation().get().getBlockEndPos());
} catch (IOException e) {
try {
// TODO : fs.open() and return inputstream again, need to pass FS configuration
// because the inputstream might close/timeout for large number of log blocks to be merged
inflate();
} catch (IOException io) {
throw new HoodieIOException("unable to lazily read log block from disk", io);
}
}
}
/**
* After the content bytes is converted into the required DataStructure by a logBlock, deflate the
* content to release byte [] and relieve memory pressure when GC kicks in. NOTE: This still
* leaves the heap fragmented
*/
protected void deflate() {
content = Option.empty();
}
/**
* Handles difference in seek behavior for GCS and non-GCS input stream
* @param inputStream Input Stream
* @param pos Position to seek
* @throws IOException
*/
private static void safeSeek(FSDataInputStream inputStream, long pos) throws IOException {
try {
inputStream.seek(pos);
} catch (EOFException e) {
if (FSUtils.isGCSInputStream(inputStream)) {
inputStream.seek(pos - 1);
} else {
throw e;
}
}
}
}

View File

@@ -0,0 +1,35 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.log.block;
abstract class HoodieLogBlockVersion {
private final int currentVersion;
public static final int DEFAULT_VERSION = 0;
HoodieLogBlockVersion(int version) {
this.currentVersion = version;
}
int getVersion() {
return currentVersion;
}
}

View File

@@ -0,0 +1,406 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import java.io.IOException;
import java.io.Serializable;
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Date;
import java.util.HashSet;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Stream;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* 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 Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList(
new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION,
CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION,
INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
private static final transient Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
protected HoodieTableMetaClient metaClient;
/**
* Returns next commit time in the {@link #COMMIT_FORMATTER} format.
*/
public static String createNewCommitTime() {
return HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
}
protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, Set<String> includedExtensions) {
// Filter all the filter in the metapath and include only the extensions passed and
// convert them into HoodieInstant
try {
this.setInstants(HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
new Path(metaClient.getMetaPath()), includedExtensions));
log.info("Loaded instants " + getInstants());
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.metaClient = metaClient;
// 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, Option<byte[]>> & Serializable) this::getInstantDetails;
}
public HoodieActiveTimeline(HoodieTableMetaClient metaClient) {
this(metaClient,
new ImmutableSet.Builder<String>()
.addAll(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE).build());
}
/**
* 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();
}
/**
* Get all instants (commits, delta commits) that produce new data, in the active timeline *
*
*/
public HoodieTimeline getCommitsTimeline() {
return getTimelineOfActions(
Sets.newHashSet(COMMIT_ACTION, DELTA_COMMIT_ACTION));
}
/**
* Get all instants (commits, delta commits, in-flight/request compaction) that produce new data, in the active
* timeline *
* With Async compaction a requested/inflight compaction-instant is a valid baseInstant for a file-slice as there
* could be delta-commits with that baseInstant.
*/
public HoodieTimeline getCommitsAndCompactionTimeline() {
return getTimelineOfActions(
Sets.newHashSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION));
}
/**
* Get all instants (commits, delta commits, clean, savepoint, rollback) that result in actions,
* in the active timeline *
*/
public HoodieTimeline getAllCommitsTimeline() {
return getTimelineOfActions(
Sets.newHashSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, COMPACTION_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, Option<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(getInstants().filter(s -> actions.contains(s.getAction())),
(Function<HoodieInstant, Option<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, Option<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, Option<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, Option<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the restore action (inflight and completed) in the active timeline
*/
public HoodieTimeline getRestoreTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(RESTORE_ACTION),
(Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails);
}
protected Stream<HoodieInstant> filterInstantsByAction(String action) {
return getInstants().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(), Option.empty());
}
public void saveAsComplete(HoodieInstant instant, Option<byte[]> data) {
log.info("Marking instant complete " + instant);
Preconditions.checkArgument(instant.isInflight(),
"Could not mark an already completed instant as complete again " + instant);
transitionState(instant, HoodieTimeline.getCompletedInstant(instant), data);
log.info("Completed " + instant);
}
public void revertToInflight(HoodieInstant instant) {
log.info("Reverting instant to inflight " + instant);
revertCompleteToInflight(instant, HoodieTimeline.getInflightInstant(instant));
log.info("Reverted " + instant + " to inflight");
}
public void deleteInflight(HoodieInstant instant) {
Preconditions.checkArgument(instant.isInflight());
deleteInstantFile(instant);
}
public void deleteCompactionRequested(HoodieInstant instant) {
Preconditions.checkArgument(instant.isRequested());
Preconditions.checkArgument(instant.getAction() == HoodieTimeline.COMPACTION_ACTION);
deleteInstantFile(instant);
}
private void deleteInstantFile(HoodieInstant instant) {
log.info("Deleting instant " + instant);
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName());
try {
boolean result = metaClient.getFs().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 Option<byte[]> getInstantDetails(HoodieInstant instant) {
Path detailPath = new Path(metaClient.getMetaPath(), instant.getFileName());
return readDataFromPath(detailPath);
}
/** BEGIN - COMPACTION RELATED META-DATA MANAGEMENT **/
public Option<byte[]> getInstantAuxiliaryDetails(HoodieInstant instant) {
Path detailPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
return readDataFromPath(detailPath);
}
/**
* Revert compaction State from inflight to requested
*
* @param inflightInstant Inflight Instant
* @return requested instant
*/
public HoodieInstant revertCompactionInflightToRequested(HoodieInstant inflightInstant) {
Preconditions.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
Preconditions.checkArgument(inflightInstant.isInflight());
HoodieInstant requestedInstant =
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, inflightInstant.getTimestamp());
// Pass empty data since it is read from the corresponding .aux/.compaction instant file
transitionState(inflightInstant, requestedInstant, Option.empty());
return requestedInstant;
}
/**
* Transition Compaction State from requested to inflight
*
* @param requestedInstant Requested instant
* @return inflight instant
*/
public HoodieInstant transitionCompactionRequestedToInflight(HoodieInstant requestedInstant) {
Preconditions.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
Preconditions.checkArgument(requestedInstant.isRequested());
HoodieInstant inflightInstant =
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, requestedInstant.getTimestamp());
transitionState(requestedInstant, inflightInstant, Option.empty());
return inflightInstant;
}
/**
* Transition Compaction State from inflight to Committed
*
* @param inflightInstant Inflight instant
* @param data Extra Metadata
* @return commit instant
*/
public HoodieInstant transitionCompactionInflightToComplete(HoodieInstant inflightInstant, Option<byte[]> data) {
Preconditions.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
Preconditions.checkArgument(inflightInstant.isInflight());
HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, COMMIT_ACTION, inflightInstant.getTimestamp());
transitionState(inflightInstant, commitInstant, data);
return commitInstant;
}
private void createFileInAuxiliaryFolder(HoodieInstant instant, Option<byte[]> data) {
Path fullPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
createFileInPath(fullPath, data);
}
/**
* END - COMPACTION RELATED META-DATA MANAGEMENT
**/
private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant,
Option<byte[]> data) {
Preconditions.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp()));
Path commitFilePath = new Path(metaClient.getMetaPath(), toInstant.getFileName());
try {
// Re-create the .inflight file by opening a new file and write the commit metadata in
Path inflightCommitFile = new Path(metaClient.getMetaPath(), fromInstant.getFileName());
createFileInMetaPath(fromInstant.getFileName(), data);
boolean success = metaClient.getFs().rename(inflightCommitFile, commitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + inflightCommitFile + " to " + commitFilePath);
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete " + fromInstant, e);
}
}
private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
Preconditions.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp()));
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName());
try {
if (!metaClient.getFs().exists(inFlightCommitFilePath)) {
Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName());
boolean success = metaClient.getFs().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, Option<byte[]> content) {
Preconditions.checkArgument(instant.isInflight());
createFileInMetaPath(instant.getFileName(), content);
}
public void saveToCompactionRequested(HoodieInstant instant, Option<byte[]> content) {
Preconditions.checkArgument(instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
// Write workload to auxiliary folder
createFileInAuxiliaryFolder(instant, content);
createFileInMetaPath(instant.getFileName(), content);
}
private void createFileInMetaPath(String filename, Option<byte[]> content) {
Path fullPath = new Path(metaClient.getMetaPath(), filename);
createFileInPath(fullPath, content);
}
private void createFileInPath(Path fullPath, Option<byte[]> content) {
try {
// If the path does not exist, create it first
if (!metaClient.getFs().exists(fullPath)) {
if (metaClient.getFs().createNewFile(fullPath)) {
log.info("Created a new file in meta path: " + fullPath);
} else {
throw new HoodieIOException("Failed to create file " + fullPath);
}
}
if (content.isPresent()) {
FSDataOutputStream fsout = metaClient.getFs().create(fullPath, true);
fsout.write(content.get());
fsout.close();
}
} catch (IOException e) {
throw new HoodieIOException("Failed to create file " + fullPath, e);
}
}
private Option<byte[]> readDataFromPath(Path detailPath) {
try (FSDataInputStream is = metaClient.getFs().open(detailPath)) {
return Option.of(IOUtils.toByteArray(is));
} catch (IOException e) {
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
}
}
public HoodieActiveTimeline reload() {
return new HoodieActiveTimeline(metaClient);
}
}

View File

@@ -0,0 +1,112 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline;
import java.io.IOException;
import java.io.Serializable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
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.
*/
public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits";
private HoodieTableMetaClient metaClient;
private Map<String, byte[]> readCommits = new HashMap<>();
private static final transient Logger log = LogManager.getLogger(HoodieArchivedTimeline.class);
public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) {
// Read back the commits to make sure
Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath());
try (SequenceFile.Reader reader =
new SequenceFile.Reader(metaClient.getHadoopConf(),
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.setInstants(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, Option<byte[]>> & Serializable) this::getInstantDetails;
this.metaClient = metaClient;
}
/**
* 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();
}
public static Path getArchiveLogPath(String archiveFolder) {
return new Path(archiveFolder, HOODIE_COMMIT_ARCHIVE_LOG_FILE);
}
@Override
public Option<byte[]> getInstantDetails(HoodieInstant instant) {
return Option.ofNullable(readCommits.get(instant.getTimestamp()));
}
public HoodieArchivedTimeline reload() {
return new HoodieArchivedTimeline(metaClient);
}
}

View File

@@ -0,0 +1,216 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline;
import com.google.common.collect.Sets;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.List;
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.codec.binary.Hex;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.exception.HoodieException;
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.
*
* @see HoodieTimeline
*/
public class HoodieDefaultTimeline implements HoodieTimeline {
private static final transient Logger log = LogManager.getLogger(HoodieDefaultTimeline.class);
private static final String HASHING_ALGORITHM = "SHA-256";
protected transient Function<HoodieInstant, Option<byte[]>> details;
private List<HoodieInstant> instants;
private String timelineHash;
public HoodieDefaultTimeline(Stream<HoodieInstant> instants,
Function<HoodieInstant, Option<byte[]>> details) {
this.details = details;
setInstants(instants.collect(Collectors.toList()));
}
public void setInstants(List<HoodieInstant> instants) {
this.instants = instants;
final MessageDigest md;
try {
md = MessageDigest.getInstance(HASHING_ALGORITHM);
this.instants.stream().forEach(i -> md.update(
StringUtils.joinUsingDelim("_", i.getTimestamp(), i.getAction(), i.getState().name()).getBytes()));
} catch (NoSuchAlgorithmException nse) {
throw new HoodieException(nse);
}
this.timelineHash = new String(Hex.encodeHex(md.digest()));
}
/**
* For serailizing and de-serializing
*
* @deprecated
*/
public HoodieDefaultTimeline() {
}
@Override
public HoodieTimeline filterInflights() {
return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isInflight),
details);
}
@Override
public HoodieTimeline filterInflightsExcludingCompaction() {
return new HoodieDefaultTimeline(instants.stream().filter(instant -> {
return instant.isInflight() && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
}), details);
}
@Override
public HoodieTimeline filterCompletedInstants() {
return new HoodieDefaultTimeline(instants.stream().filter(s -> !s.isInflight()), details);
}
@Override
public HoodieTimeline filterCompletedAndCompactionInstants() {
return new HoodieDefaultTimeline(instants.stream().filter(s -> {
return !s.isInflight() || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION);
}), details);
}
@Override
public HoodieTimeline getCommitsAndCompactionTimeline() {
Set<String> validActions = Sets.newHashSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION);
return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details);
}
@Override
public HoodieTimeline filterPendingCompactionTimeline() {
return new HoodieDefaultTimeline(
instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION)),
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 HoodieTimeline filter(Predicate<HoodieInstant> filter) {
return new HoodieDefaultTimeline(instants.stream().filter(filter), details);
}
@Override
public boolean empty() {
return !instants.stream().findFirst().isPresent();
}
@Override
public int countInstants() {
return new Long(instants.stream().count()).intValue();
}
@Override
public Option<HoodieInstant> firstInstant() {
return Option.fromJavaOptional(instants.stream().findFirst());
}
@Override
public Option<HoodieInstant> nthInstant(int n) {
if (empty() || n >= countInstants()) {
return Option.empty();
}
return Option.of(instants.get(n));
}
@Override
public Option<HoodieInstant> lastInstant() {
return empty() ? Option.empty() : nthInstant(countInstants() - 1);
}
@Override
public Option<HoodieInstant> nthFromLastInstant(int n) {
if (countInstants() < n + 1) {
return Option.empty();
}
return nthInstant(countInstants() - 1 - n);
}
@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 String getTimelineHash() {
return timelineHash;
}
@Override
public Stream<HoodieInstant> getInstants() {
return instants.stream();
}
@Override
public boolean isBeforeTimelineStarts(String instant) {
Option<HoodieInstant> firstCommit = firstInstant();
return firstCommit.isPresent()
&& HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER);
}
@Override
public Option<byte[]> getInstantDetails(HoodieInstant instant) {
return details.apply(instant);
}
@Override
public String toString() {
return this.getClass().getName() + ": " + instants.stream().map(Object::toString)
.collect(Collectors.joining(","));
}
}

View File

@@ -0,0 +1,172 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline;
import java.io.Serializable;
import java.util.Objects;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.util.FSUtils;
/**
* 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 {
/**
* Instant State
*/
public enum State {
// Requested State (valid state for Compaction)
REQUESTED,
// Inflight instant
INFLIGHT,
// Committed instant
COMPLETED,
// Invalid instant
INVALID
}
private State state = State.COMPLETED;
private String action;
private String timestamp;
/**
* 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";
state = State.INFLIGHT;
} else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
state = State.INFLIGHT;
action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
} else if (action.equals(HoodieTimeline.REQUESTED_COMPACTION_SUFFIX)) {
state = State.REQUESTED;
action = action.replace(HoodieTimeline.REQUESTED_EXTENSION, "");
}
}
public HoodieInstant(boolean isInflight, String action, String timestamp) {
//TODO: vb - Preserving for avoiding cascading changes. This constructor will be updated in subsequent PR
this.state = isInflight ? State.INFLIGHT : State.COMPLETED;
this.action = action;
this.timestamp = timestamp;
}
public HoodieInstant(State state, String action, String timestamp) {
this.state = state;
this.action = action;
this.timestamp = timestamp;
}
public boolean isCompleted() {
return state == State.COMPLETED;
}
public boolean isInflight() {
return state == State.INFLIGHT;
}
public boolean isRequested() {
return state == State.REQUESTED;
}
public String getAction() {
return action;
}
public String getTimestamp() {
return timestamp;
}
/**
* 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.DELTA_COMMIT_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightDeltaFileName(timestamp)
: HoodieTimeline.makeDeltaFileName(timestamp);
} else if (HoodieTimeline.COMPACTION_ACTION.equals(action)) {
if (isInflight()) {
return HoodieTimeline.makeInflightCompactionFileName(timestamp);
} else if (isRequested()) {
return HoodieTimeline.makeRequestedCompactionFileName(timestamp);
} else {
return HoodieTimeline.makeCommitFileName(timestamp);
}
} else if (HoodieTimeline.RESTORE_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightRestoreFileName(timestamp)
: HoodieTimeline.makeRestoreFileName(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 state == that.state
&& Objects.equals(action, that.action)
&& Objects.equals(timestamp, that.timestamp);
}
public State getState() {
return state;
}
@Override
public int hashCode() {
return Objects.hash(state, action, timestamp);
}
@Override
public String toString() {
return "[" + ((isInflight() || isRequested()) ? "==>" : "") + timestamp + "__" + action + "__" + state + "]";
}
}

View File

@@ -0,0 +1,77 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline.dto;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
@JsonIgnoreProperties(ignoreUnknown = true)
public class CompactionOpDTO {
@JsonProperty("baseInstant")
String baseInstantTime;
@JsonProperty("compactionInstant")
String compactionInstantTime;
@JsonProperty("dataFileInstant")
private String dataFileCommitTime;
@JsonProperty("deltaFiles")
private List<String> deltaFilePaths;
@JsonProperty("baseFile")
private String dataFilePath;
@JsonProperty("id")
private String fileId;
@JsonProperty("partition")
private String partitionPath;
@JsonProperty("metrics")
private Map<String, Double> metrics;
public static CompactionOpDTO fromCompactionOperation(String compactionInstantTime,
CompactionOperation op) {
CompactionOpDTO dto = new CompactionOpDTO();
dto.fileId = op.getFileId();
dto.compactionInstantTime = compactionInstantTime;
dto.baseInstantTime = op.getBaseInstantTime();
dto.dataFileCommitTime = op.getDataFileCommitTime().orElse(null);
dto.dataFilePath = op.getDataFilePath().orElse(null);
dto.deltaFilePaths = new ArrayList<>(op.getDeltaFilePaths());
dto.partitionPath = op.getPartitionPath();
dto.metrics = op.getMetrics() == null ? new HashMap<>() : new HashMap<>(op.getMetrics());
return dto;
}
public static Pair<String, CompactionOperation> toCompactionOperation(CompactionOpDTO dto) {
return Pair.of(dto.compactionInstantTime, new CompactionOperation(dto.fileId, dto.partitionPath,
dto.baseInstantTime, Option.ofNullable(dto.dataFileCommitTime), dto.deltaFilePaths,
Option.ofNullable(dto.dataFilePath), dto.metrics));
}
}

View File

@@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline.dto;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.hudi.common.model.HoodieDataFile;
@JsonIgnoreProperties(ignoreUnknown = true)
public class DataFileDTO {
@JsonProperty("fileStatus")
private FileStatusDTO fileStatus;
@JsonProperty("fullPath")
private String fullPath;
@JsonProperty("fileLen")
private long fileLen;
public static HoodieDataFile toHoodieDataFile(DataFileDTO dto) {
if (null == dto) {
return null;
}
HoodieDataFile dataFile = null;
if (null != dto.fileStatus) {
dataFile = new HoodieDataFile(FileStatusDTO.toFileStatus(dto.fileStatus));
} else {
dataFile = new HoodieDataFile(dto.fullPath);
dataFile.setFileLen(dto.fileLen);
}
return dataFile;
}
public static DataFileDTO fromHoodieDataFile(HoodieDataFile dataFile) {
if (null == dataFile) {
return null;
}
DataFileDTO dto = new DataFileDTO();
dto.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus());
dto.fullPath = dataFile.getPath();
dto.fileLen = dataFile.getFileLen();
return dto;
}
}

View File

@@ -0,0 +1,63 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline.dto;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.io.Serializable;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
/**
* A serializable FS Permission
*/
@JsonIgnoreProperties(ignoreUnknown = true)
public class FSPermissionDTO implements Serializable {
@JsonProperty("useraction")
FsAction useraction;
@JsonProperty("groupaction")
FsAction groupaction;
@JsonProperty("otheraction")
FsAction otheraction;
@JsonProperty("stickyBit")
boolean stickyBit;
public static FSPermissionDTO fromFsPermission(FsPermission permission) {
if (null == permission) {
return null;
}
FSPermissionDTO dto = new FSPermissionDTO();
dto.useraction = permission.getUserAction();
dto.groupaction = permission.getGroupAction();
dto.otheraction = permission.getOtherAction();
dto.stickyBit = permission.getStickyBit();
return dto;
}
public static FsPermission fromFsPermissionDTO(FSPermissionDTO dto) {
if (null == dto) {
return null;
}
return new FsPermission(dto.useraction, dto.groupaction, dto.otheraction, dto.stickyBit);
}
}

View File

@@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline.dto;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@JsonIgnoreProperties(ignoreUnknown = true)
public class FileGroupDTO {
@JsonProperty("partition")
String partition;
@JsonProperty("fileId")
String id;
@JsonProperty("slices")
List<FileSliceDTO> slices;
@JsonProperty("timeline")
TimelineDTO timeline;
public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) {
FileGroupDTO dto = new FileGroupDTO();
dto.partition = fileGroup.getPartitionPath();
dto.id = fileGroup.getFileGroupId().getFileId();
dto.slices = fileGroup.getAllRawFileSlices().map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
dto.timeline = TimelineDTO.fromTimeline(fileGroup.getTimeline());
return dto;
}
public static HoodieFileGroup toFileGroup(FileGroupDTO dto, HoodieTableMetaClient metaClient) {
HoodieFileGroup fileGroup = new HoodieFileGroup(dto.partition, dto.id,
TimelineDTO.toTimeline(dto.timeline, metaClient));
dto.slices.stream().map(FileSliceDTO::toFileSlice).forEach(fileSlice -> fileGroup.addFileSlice(fileSlice));
return fileGroup;
}
}

View File

@@ -0,0 +1,53 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline.dto;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.net.URI;
import java.net.URISyntaxException;
import org.apache.hadoop.fs.Path;
@JsonIgnoreProperties(ignoreUnknown = true)
public class FilePathDTO {
@JsonProperty("uri")
private String uri;
public static FilePathDTO fromPath(Path path) {
if (null == path) {
return null;
}
FilePathDTO dto = new FilePathDTO();
dto.uri = path.toUri().toString();
return dto;
}
public static Path toPath(FilePathDTO dto) {
if (null == dto) {
return null;
}
try {
return new Path(new URI(dto.uri));
} catch (URISyntaxException e) {
throw new RuntimeException(e);
}
}
}

View File

@@ -0,0 +1,57 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline.dto;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hudi.common.model.FileSlice;
@JsonIgnoreProperties(ignoreUnknown = true)
public class FileSliceDTO {
@JsonProperty("dataFile")
DataFileDTO dataFile;
@JsonProperty("logFiles")
List<LogFileDTO> logFiles;
@JsonProperty("partition")
private String partitionPath;
@JsonProperty("fileId")
private String fileId;
@JsonProperty("baseInstant")
private String baseInstantTime;
public static FileSliceDTO fromFileSlice(FileSlice slice) {
FileSliceDTO dto = new FileSliceDTO();
dto.partitionPath = slice.getPartitionPath();
dto.baseInstantTime = slice.getBaseInstantTime();
dto.fileId = slice.getFileId();
dto.dataFile = slice.getDataFile().map(DataFileDTO::fromHoodieDataFile).orElse(null);
dto.logFiles = slice.getLogFiles().map(LogFileDTO::fromHoodieLogFile).collect(Collectors.toList());
return dto;
}
public static FileSlice toFileSlice(FileSliceDTO dto) {
FileSlice slice = new FileSlice(dto.partitionPath, dto.baseInstantTime, dto.fileId);
slice.setDataFile(DataFileDTO.toHoodieDataFile(dto.dataFile));
dto.logFiles.stream().forEach(lf -> slice.addLogFile(LogFileDTO.toHoodieLogFile(lf)));
return slice;
}
}

View File

@@ -0,0 +1,99 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline.dto;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.io.IOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hudi.exception.HoodieException;
@JsonIgnoreProperties(ignoreUnknown = true)
public class FileStatusDTO {
@JsonProperty("path")
FilePathDTO path;
@JsonProperty("length")
long length;
@JsonProperty("isdir")
boolean isdir;
@JsonProperty("blockReplication")
short blockReplication;
@JsonProperty("blocksize")
long blocksize;
@JsonProperty("modificationTime")
long modificationTime;
@JsonProperty("accessTime")
long accessTime;
@JsonProperty("permission")
FSPermissionDTO permission;
@JsonProperty("owner")
String owner;
@JsonProperty("group")
String group;
@JsonProperty("symlink")
FilePathDTO symlink;
public static FileStatusDTO fromFileStatus(FileStatus fileStatus) {
if (null == fileStatus) {
return null;
}
FileStatusDTO dto = new FileStatusDTO();
try {
dto.path = FilePathDTO.fromPath(fileStatus.getPath());
dto.length = fileStatus.getLen();
dto.isdir = fileStatus.isDirectory();
dto.blockReplication = fileStatus.getReplication();
dto.blocksize = fileStatus.getBlockSize();
dto.modificationTime = fileStatus.getModificationTime();
dto.accessTime = fileStatus.getModificationTime();
dto.symlink = fileStatus.isSymlink() ? FilePathDTO.fromPath(fileStatus.getSymlink()) : null;
safeReadAndSetMetadata(dto, fileStatus);
} catch (IOException ioe) {
throw new HoodieException(ioe);
}
return dto;
}
/**
* Used to safely handle FileStatus calls which might fail on some FileSystem implementation.
* (DeprecatedLocalFileSystem)
*/
private static void safeReadAndSetMetadata(FileStatusDTO dto, FileStatus fileStatus) {
try {
dto.owner = fileStatus.getOwner();
dto.group = fileStatus.getGroup();
dto.permission = FSPermissionDTO.fromFsPermission(fileStatus.getPermission());
} catch (IllegalArgumentException ie) {
// Deprecated File System (testing) does not work well with this call
// skipping
}
}
public static FileStatus toFileStatus(FileStatusDTO dto) {
if (null == dto) {
return null;
}
return new FileStatus(dto.length, dto.isdir, dto.blockReplication, dto.blocksize, dto.modificationTime,
dto.accessTime, FSPermissionDTO.fromFsPermissionDTO(dto.permission), dto.owner, dto.group,
FilePathDTO.toPath(dto.symlink), FilePathDTO.toPath(dto.path));
}
}

View File

@@ -0,0 +1,55 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline.dto;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.hudi.common.table.timeline.HoodieInstant;
@JsonIgnoreProperties(ignoreUnknown = true)
public class InstantDTO {
@JsonProperty("action")
String action;
@JsonProperty("ts")
String timestamp;
@JsonProperty("state")
String state;
public static InstantDTO fromInstant(HoodieInstant instant) {
if (null == instant) {
return null;
}
InstantDTO dto = new InstantDTO();
dto.action = instant.getAction();
dto.timestamp = instant.getTimestamp();
dto.state = instant.getState().toString();
return dto;
}
public static HoodieInstant toInstant(InstantDTO dto) {
if (null == dto) {
return null;
}
return new HoodieInstant(HoodieInstant.State.valueOf(dto.state), dto.action, dto.timestamp);
}
}

View File

@@ -0,0 +1,50 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline.dto;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hudi.common.model.HoodieLogFile;
@JsonIgnoreProperties(ignoreUnknown = true)
public class LogFileDTO {
@JsonProperty("fileStatus")
private FileStatusDTO fileStatus;
@JsonProperty("path")
private String pathStr;
@JsonProperty("len")
private long fileLen;
public static HoodieLogFile toHoodieLogFile(LogFileDTO dto) {
FileStatus status = FileStatusDTO.toFileStatus(dto.fileStatus);
HoodieLogFile logFile = (status == null) ? new HoodieLogFile(dto.pathStr) : new HoodieLogFile(status);
logFile.setFileLen(dto.fileLen);
return logFile;
}
public static LogFileDTO fromHoodieLogFile(HoodieLogFile dataFile) {
LogFileDTO logFile = new LogFileDTO();
logFile.fileLen = dataFile.getFileSize();
logFile.pathStr = dataFile.getPath().toString();
logFile.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus());
return logFile;
}
}

View File

@@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.timeline.dto;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
@JsonIgnoreProperties(ignoreUnknown = true)
public class TimelineDTO {
@JsonProperty("instants")
List<InstantDTO> instants;
public static TimelineDTO fromTimeline(HoodieTimeline timeline) {
TimelineDTO dto = new TimelineDTO();
dto.instants = timeline.getInstants().map(InstantDTO::fromInstant).collect(Collectors.toList());
return dto;
}
public static HoodieTimeline toTimeline(TimelineDTO dto, HoodieTableMetaClient metaClient) {
//TODO: For Now, we will assume, only active-timeline will be transferred.
return new HoodieDefaultTimeline(dto.instants.stream().map(InstantDTO::toInstant),
metaClient.getActiveTimeline()::getInstantDetails);
}
}

View File

@@ -0,0 +1,849 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.view;
import com.google.common.base.Preconditions;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.SyncableFileSystemView;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Common thread-safe implementation for multiple TableFileSystemView Implementations.
* Provides uniform handling of
* (a) Loading file-system views from underlying file-system
* (b) Pending compaction operations and changing file-system views based on that
* (c) Thread-safety in loading and managing file system views for this dataset.
* (d) resetting file-system views
* The actual mechanism of fetching file slices from different view storages is delegated to sub-classes.
*/
public abstract class AbstractTableFileSystemView implements SyncableFileSystemView, Serializable {
private static Logger log = LogManager.getLogger(AbstractTableFileSystemView.class);
protected HoodieTableMetaClient metaClient;
// This is the commits timeline that will be visible for all views extending this view
private HoodieTimeline visibleCommitsAndCompactionTimeline;
// Used to concurrently load and populate partition views
private ConcurrentHashMap<String, Boolean> addedPartitions = new ConcurrentHashMap<>(4096);
// Locks to control concurrency. Sync operations use write-lock blocking all fetch operations.
// For the common-case, we allow concurrent read of single or multiple partitions
private final ReentrantReadWriteLock globalLock = new ReentrantReadWriteLock();
private final ReadLock readLock = globalLock.readLock();
private final WriteLock writeLock = globalLock.writeLock();
private String getPartitionPathFromFilePath(String fullPath) {
return FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), new Path(fullPath).getParent());
}
/**
* Initialize the view.
*/
protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
this.metaClient = metaClient;
refreshTimeline(visibleActiveTimeline);
// Load Pending Compaction Operations
resetPendingCompactionOperations(
CompactionUtils.getAllPendingCompactionOperations(metaClient).values()
.stream().map(e -> Pair.of(e.getKey(),
CompactionOperation.convertFromAvroRecordInstance(e.getValue()))));
}
/**
* Refresh commits timeline
* @param visibleActiveTimeline Visible Active Timeline
*/
protected void refreshTimeline(HoodieTimeline visibleActiveTimeline) {
this.visibleCommitsAndCompactionTimeline = visibleActiveTimeline.getCommitsAndCompactionTimeline();
}
/**
* Adds the provided statuses into the file system view, and also caches it inside this object.
*/
protected List<HoodieFileGroup> addFilesToView(FileStatus[] statuses) {
HoodieTimer timer = new HoodieTimer().startTimer();
List<HoodieFileGroup> fileGroups = buildFileGroups(statuses, visibleCommitsAndCompactionTimeline, true);
long fgBuildTimeTakenMs = timer.endTimer();
timer.startTimer();
// Group by partition for efficient updates for both InMemory and DiskBased stuctures.
fileGroups.stream().collect(Collectors.groupingBy(HoodieFileGroup::getPartitionPath)).entrySet()
.forEach(entry -> {
String partition = entry.getKey();
if (!isPartitionAvailableInStore(partition)) {
storePartitionView(partition, entry.getValue());
}
});
long storePartitionsTs = timer.endTimer();
log.info("addFilesToView: NumFiles=" + statuses.length + ", FileGroupsCreationTime=" + fgBuildTimeTakenMs
+ ", StoreTimeTaken=" + storePartitionsTs);
return fileGroups;
}
/**
* Build FileGroups from passed in file-status
*/
protected List<HoodieFileGroup> buildFileGroups(FileStatus[] statuses, HoodieTimeline timeline,
boolean addPendingCompactionFileSlice) {
return buildFileGroups(convertFileStatusesToDataFiles(statuses), convertFileStatusesToLogFiles(statuses), timeline,
addPendingCompactionFileSlice);
}
protected List<HoodieFileGroup> buildFileGroups(Stream<HoodieDataFile> dataFileStream,
Stream<HoodieLogFile> logFileStream, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) {
Map<Pair<String, String>, List<HoodieDataFile>> dataFiles = dataFileStream
.collect(Collectors.groupingBy((dataFile) -> {
String partitionPathStr = getPartitionPathFromFilePath(dataFile.getPath());
return Pair.of(partitionPathStr, dataFile.getFileId());
}));
Map<Pair<String, String>, List<HoodieLogFile>> logFiles = logFileStream
.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 -> {
String fileId = pair.getValue();
HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, timeline);
if (dataFiles.containsKey(pair)) {
dataFiles.get(pair).forEach(group::addDataFile);
}
if (logFiles.containsKey(pair)) {
logFiles.get(pair).forEach(group::addLogFile);
}
if (addPendingCompactionFileSlice) {
Option<Pair<String, CompactionOperation>> pendingCompaction =
getPendingCompactionOperationWithInstant(group.getFileGroupId());
if (pendingCompaction.isPresent()) {
// If there is no delta-commit after compaction request, this step would ensure a new file-slice appears
// so that any new ingestion uses the correct base-instant
group.addNewFileSliceAtInstant(pendingCompaction.get().getKey());
}
}
fileGroups.add(group);
});
return fileGroups;
}
/**
* Clears the partition Map and reset view states
*/
public final void reset() {
try {
writeLock.lock();
addedPartitions.clear();
resetViewState();
// Initialize with new Hoodie timeline.
init(metaClient, getTimeline());
} finally {
writeLock.unlock();
}
}
/**
* Allows all view metadata in file system view storage to be reset by subclasses
*/
protected abstract void resetViewState();
/**
* Allows lazily loading the partitions if needed
*
* @param partition partition to be loaded if not present
*/
private void ensurePartitionLoadedCorrectly(String partition) {
Preconditions.checkArgument(!isClosed(), "View is already closed");
// ensure we list files only once even in the face of concurrency
addedPartitions.computeIfAbsent(partition, (partitionPathStr) -> {
long beginTs = System.currentTimeMillis();
if (!isPartitionAvailableInStore(partitionPathStr)) {
// Not loaded yet
try {
log.info("Building file system view for partition (" + partitionPathStr + ")");
// Create the path if it does not exist already
Path partitionPath = FSUtils.getPartitionPath(metaClient.getBasePath(), partitionPathStr);
FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath);
long beginLsTs = System.currentTimeMillis();
FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath);
long endLsTs = System.currentTimeMillis();
log.info("#files found in partition (" + partitionPathStr + ") =" + statuses.length
+ ", Time taken =" + (endLsTs - beginLsTs));
List<HoodieFileGroup> groups = addFilesToView(statuses);
if (groups.isEmpty()) {
storePartitionView(partitionPathStr, new ArrayList<>());
}
} catch (IOException e) {
throw new HoodieIOException("Failed to list data files in partition " + partitionPathStr, e);
}
} else {
log.debug("View already built for Partition :" + partitionPathStr + ", FOUND is ");
}
long endTs = System.currentTimeMillis();
log.info("Time to load partition (" + partitionPathStr + ") =" + (endTs - beginTs));
return true;
});
}
/**
* Helper to convert file-status to data-files
*
* @param statuses List of Fole-Status
*/
private Stream<HoodieDataFile> convertFileStatusesToDataFiles(FileStatus[] statuses) {
Predicate<FileStatus> roFilePredicate = fileStatus ->
fileStatus.getPath().getName()
.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension());
return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new);
}
/**
* Helper to convert file-status to log-files
*
* @param statuses List of FIle-Status
*/
private Stream<HoodieLogFile> convertFileStatusesToLogFiles(FileStatus[] statuses) {
Predicate<FileStatus> rtFilePredicate = fileStatus ->
fileStatus.getPath().getName()
.contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension());
return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new);
}
/**
* With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore those
* data-files
*
* @param dataFile Data File
*/
protected boolean isDataFileDueToPendingCompaction(HoodieDataFile dataFile) {
final String partitionPath = getPartitionPathFromFilePath(dataFile.getPath());
Option<Pair<String, CompactionOperation>> compactionWithInstantTime =
getPendingCompactionOperationWithInstant(new HoodieFileGroupId(partitionPath, dataFile.getFileId()));
return (compactionWithInstantTime.isPresent()) && (null != compactionWithInstantTime.get().getKey())
&& dataFile.getCommitTime().equals(compactionWithInstantTime.get().getKey());
}
/**
* Returns true if the file-group is under pending-compaction and the file-slice' baseInstant matches compaction
* Instant
*
* @param fileSlice File Slice
*/
protected boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) {
Option<Pair<String, CompactionOperation>> compactionWithInstantTime =
getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId());
log.info("Pending Compaction instant for (" + fileSlice + ") is :" + compactionWithInstantTime);
return (compactionWithInstantTime.isPresent())
&& fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.get().getKey());
}
/**
* With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore those
* data-files
*
* @param fileSlice File Slice
*/
protected FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) {
if (isFileSliceAfterPendingCompaction(fileSlice)) {
log.info("File Slice (" + fileSlice + ") is in pending compaction");
// Data file is filtered out of the file-slice as the corresponding compaction
// instant not completed yet.
FileSlice transformed = new FileSlice(fileSlice.getPartitionPath(),
fileSlice.getBaseInstantTime(), fileSlice.getFileId());
fileSlice.getLogFiles().forEach(transformed::addLogFile);
return transformed;
}
return fileSlice;
}
@Override
public final Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations() {
try {
readLock.lock();
return fetchPendingCompactionOperations();
} finally {
readLock.unlock();
}
}
@Override
public final Stream<HoodieDataFile> getLatestDataFiles(String partitionStr) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestDataFiles(partitionPath);
} finally {
readLock.unlock();
}
}
@Override
public final Stream<HoodieDataFile> getLatestDataFiles() {
try {
readLock.lock();
return fetchLatestDataFiles();
} finally {
readLock.unlock();
}
}
@Override
public final Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionStr, String maxCommitTime) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllStoredFileGroups(partitionPath)
.map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllDataFiles()
.filter(dataFile ->
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL))
.filter(df -> !isDataFileDueToPendingCompaction(df))
.findFirst()))
.filter(Option::isPresent)
.map(Option::get);
} finally {
readLock.unlock();
}
}
@Override
public final Option<HoodieDataFile> getDataFileOn(String partitionStr, String instantTime, String fileId) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchHoodieFileGroup(partitionPath, fileId)
.map(fileGroup -> fileGroup.getAllDataFiles()
.filter(dataFile ->
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
instantTime, HoodieTimeline.EQUAL))
.filter(df -> !isDataFileDueToPendingCompaction(df))
.findFirst().orElse(null));
} finally {
readLock.unlock();
}
}
/**
* Get Latest data file for a partition and file-Id
*/
public final Option<HoodieDataFile> getLatestDataFile(String partitionStr, String fileId) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestDataFile(partitionPath, fileId);
} finally {
readLock.unlock();
}
}
@Override
public final Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
try {
readLock.lock();
return fetchAllStoredFileGroups().map(fileGroup -> {
return Option.fromJavaOptional(fileGroup.getAllDataFiles()
.filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime())
&& !isDataFileDueToPendingCompaction(dataFile))
.findFirst());
}).filter(Option::isPresent).map(Option::get);
} finally {
readLock.unlock();
}
}
@Override
public final Stream<HoodieDataFile> getAllDataFiles(String partitionStr) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllDataFiles(partitionPath)
.filter(df -> visibleCommitsAndCompactionTimeline.containsOrBeforeTimelineStarts(df.getCommitTime()))
.filter(df -> !isDataFileDueToPendingCompaction(df));
} finally {
readLock.unlock();
}
}
@Override
public final Stream<FileSlice> getLatestFileSlices(String partitionStr) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestFileSlices(partitionPath).map(fs -> filterDataFileAfterPendingCompaction(fs));
} finally {
readLock.unlock();
}
}
/**
* Get Latest File Slice for a given fileId in a given partition
*/
public final Option<FileSlice> getLatestFileSlice(String partitionStr, String fileId) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
Option<FileSlice> fs = fetchLatestFileSlice(partitionPath, fileId);
return fs.map(f -> filterDataFileAfterPendingCompaction(f));
} finally {
readLock.unlock();
}
}
@Override
public final Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionStr) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllStoredFileGroups(partitionPath)
.map(fileGroup -> {
FileSlice fileSlice = fileGroup.getLatestFileSlice().get();
// if the file-group is under compaction, pick the latest before compaction instant time.
Option<Pair<String, CompactionOperation>> compactionWithInstantPair =
getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId());
if (compactionWithInstantPair.isPresent()) {
String compactionInstantTime = compactionWithInstantPair.get().getLeft();
return fileGroup.getLatestFileSliceBefore(compactionInstantTime);
}
return Option.of(fileSlice);
})
.map(Option::get);
} finally {
readLock.unlock();
}
}
@Override
public final Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionStr, String maxCommitTime,
boolean includeFileSlicesInPendingCompaction) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
Stream<FileSlice> fileSliceStream =
fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime);
if (includeFileSlicesInPendingCompaction) {
return fileSliceStream.map(fs -> filterDataFileAfterPendingCompaction(fs));
} else {
return fileSliceStream.filter(fs -> !isPendingCompactionScheduledForFileId(fs.getFileGroupId()));
}
} finally {
readLock.unlock();
}
}
@Override
public final Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionStr, String maxInstantTime) {
try {
readLock.lock();
String partition = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partition);
return fetchAllStoredFileGroups(partition)
.map(fileGroup -> {
Option<FileSlice> fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime);
// if the file-group is under construction, pick the latest before compaction instant time.
if (fileSlice.isPresent()) {
fileSlice = Option.of(fetchMergedFileSlice(fileGroup, fileSlice.get()));
}
return fileSlice;
})
.filter(Option::isPresent)
.map(Option::get);
} finally {
readLock.unlock();
}
}
@Override
public final Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
try {
readLock.lock();
return fetchLatestFileSliceInRange(commitsToReturn);
} finally {
readLock.unlock();
}
}
@Override
public final Stream<FileSlice> getAllFileSlices(String partitionStr) {
try {
readLock.lock();
String partition = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partition);
return fetchAllFileSlices(partition);
} finally {
readLock.unlock();
}
}
/**
* Ensure there is consistency in handling trailing slash in partition-path. Always trim it which is what is done in
* other places.
*/
private String formatPartitionKey(String partitionStr) {
return partitionStr.endsWith("/") ? partitionStr.substring(0, partitionStr.length() - 1) : partitionStr;
}
@Override
public final Stream<HoodieFileGroup> getAllFileGroups(String partitionStr) {
try {
readLock.lock();
// Ensure there is consistency in handling trailing slash in partition-path. Always trim it which is what is done
// in other places.
String partition = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partition);
return fetchAllStoredFileGroups(partition);
} finally {
readLock.unlock();
}
}
// Fetch APIs to be implemented by concrete sub-classes
/**
* Check if there is an outstanding compaction scheduled for this file
*
* @param fgId File-Group Id
* @return true if there is a pending compaction, false otherwise
*/
protected abstract boolean isPendingCompactionScheduledForFileId(HoodieFileGroupId fgId);
/**
* resets the pending compaction operation and overwrite with the new list
*
* @param operations Pending Compaction Operations
*/
abstract void resetPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations);
/**
* Add pending compaction operations to store
*
* @param operations Pending compaction operations to be added
*/
abstract void addPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations);
/**
* Remove pending compaction operations from store
*
* @param operations Pending compaction operations to be removed
*/
abstract void removePendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations);
/**
* Return pending compaction operation for a file-group
*
* @param fileGroupId File-Group Id
*/
protected abstract Option<Pair<String, CompactionOperation>> getPendingCompactionOperationWithInstant(
HoodieFileGroupId fileGroupId);
/**
* Fetch all pending compaction operations
*/
abstract Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations();
/**
* Checks if partition is pre-loaded and available in store
*
* @param partitionPath Partition Path
*/
abstract boolean isPartitionAvailableInStore(String partitionPath);
/**
* Add a complete partition view to store
*
* @param partitionPath Partition Path
* @param fileGroups File Groups for the partition path
*/
abstract void storePartitionView(String partitionPath, List<HoodieFileGroup> fileGroups);
/**
* Fetch all file-groups stored for a partition-path
*
* @param partitionPath Partition path for which the file-groups needs to be retrieved.
* @return file-group stream
*/
abstract Stream<HoodieFileGroup> fetchAllStoredFileGroups(String partitionPath);
/**
* Fetch all Stored file-groups across all partitions loaded
*
* @return file-group stream
*/
abstract Stream<HoodieFileGroup> fetchAllStoredFileGroups();
/**
* Check if the view is already closed
*/
abstract boolean isClosed();
/**
* Default implementation for fetching latest file-slice in commit range
*
* @param commitsToReturn Commits
*/
Stream<FileSlice> fetchLatestFileSliceInRange(List<String> commitsToReturn) {
return fetchAllStoredFileGroups().map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
.map(Option::get);
}
/**
* Default implementation for fetching all file-slices for a partition-path
*
* @param partitionPath Partition path
* @return file-slice stream
*/
Stream<FileSlice> fetchAllFileSlices(String partitionPath) {
return fetchAllStoredFileGroups(partitionPath)
.map(HoodieFileGroup::getAllFileSlices)
.flatMap(sliceList -> sliceList);
}
/**
* Default implementation for fetching latest data-files for the partition-path
*/
Stream<HoodieDataFile> fetchLatestDataFiles(final String partitionPath) {
return fetchAllStoredFileGroups(partitionPath)
.map(this::getLatestDataFile)
.filter(Option::isPresent)
.map(Option::get);
}
protected Option<HoodieDataFile> getLatestDataFile(HoodieFileGroup fileGroup) {
return Option.fromJavaOptional(
fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst());
}
/**
* Default implementation for fetching latest data-files across all partitions
*/
Stream<HoodieDataFile> fetchLatestDataFiles() {
return fetchAllStoredFileGroups()
.map(this::getLatestDataFile)
.filter(Option::isPresent)
.map(Option::get);
}
/**
* Default implementation for fetching all data-files for a partition
*
* @param partitionPath partition-path
*/
Stream<HoodieDataFile> fetchAllDataFiles(String partitionPath) {
return fetchAllStoredFileGroups(partitionPath)
.map(HoodieFileGroup::getAllDataFiles)
.flatMap(dataFileList -> dataFileList);
}
/**
* Default implementation for fetching file-group
*/
Option<HoodieFileGroup> fetchHoodieFileGroup(String partitionPath, String fileId) {
return Option.fromJavaOptional(fetchAllStoredFileGroups(partitionPath)
.filter(fileGroup -> fileGroup.getFileGroupId().getFileId().equals(fileId)).findFirst());
}
/**
* Default implementation for fetching latest file-slices for a partition path
*/
Stream<FileSlice> fetchLatestFileSlices(String partitionPath) {
return fetchAllStoredFileGroups(partitionPath)
.map(HoodieFileGroup::getLatestFileSlice)
.filter(Option::isPresent)
.map(Option::get);
}
/**
* Default implementation for fetching latest file-slices for a partition path as of instant
*
* @param partitionPath Partition Path
* @param maxCommitTime Instant Time
*/
Stream<FileSlice> fetchLatestFileSlicesBeforeOrOn(String partitionPath,
String maxCommitTime) {
return fetchAllStoredFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime))
.filter(Option::isPresent)
.map(Option::get);
}
/**
* Helper to merge last 2 file-slices. These 2 file-slices do not have compaction done yet.
*
* @param lastSlice Latest File slice for a file-group
* @param penultimateSlice Penultimate file slice for a file-group in commit timeline order
*/
private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, FileSlice penultimateSlice) {
FileSlice merged = new FileSlice(penultimateSlice.getPartitionPath(),
penultimateSlice.getBaseInstantTime(), penultimateSlice.getFileId());
if (penultimateSlice.getDataFile().isPresent()) {
merged.setDataFile(penultimateSlice.getDataFile().get());
}
// Add Log files from penultimate and last slices
penultimateSlice.getLogFiles().forEach(merged::addLogFile);
lastSlice.getLogFiles().forEach(merged::addLogFile);
return merged;
}
/**
* If the file-slice is because of pending compaction instant, this method merges the file-slice with the one before
* the compaction instant time
*
* @param fileGroup File Group for which the file slice belongs to
* @param fileSlice File Slice which needs to be merged
*/
private FileSlice fetchMergedFileSlice(HoodieFileGroup fileGroup, FileSlice fileSlice) {
// if the file-group is under construction, pick the latest before compaction instant time.
Option<Pair<String, CompactionOperation>> compactionOpWithInstant =
getPendingCompactionOperationWithInstant(fileGroup.getFileGroupId());
if (compactionOpWithInstant.isPresent()) {
String compactionInstantTime = compactionOpWithInstant.get().getKey();
if (fileSlice.getBaseInstantTime().equals(compactionInstantTime)) {
Option<FileSlice> prevFileSlice = fileGroup.getLatestFileSliceBefore(compactionInstantTime);
if (prevFileSlice.isPresent()) {
return mergeCompactionPendingFileSlices(fileSlice, prevFileSlice.get());
}
}
}
return fileSlice;
}
/**
* Default implementation for fetching latest data-file
* @param partitionPath Partition path
* @param fileId File Id
* @return Data File if present
*/
protected Option<HoodieDataFile> fetchLatestDataFile(String partitionPath, String fileId) {
return Option.fromJavaOptional(fetchLatestDataFiles(partitionPath)
.filter(fs -> fs.getFileId().equals(fileId)).findFirst());
}
/**
* Default implementation for fetching file-slice
* @param partitionPath Partition path
* @param fileId File Id
* @return File Slice if present
*/
protected Option<FileSlice> fetchLatestFileSlice(String partitionPath, String fileId) {
return Option.fromJavaOptional(fetchLatestFileSlices(partitionPath)
.filter(fs -> fs.getFileId().equals(fileId)).findFirst());
}
@Override
public Option<HoodieInstant> getLastInstant() {
return getTimeline().lastInstant();
}
@Override
public HoodieTimeline getTimeline() {
return visibleCommitsAndCompactionTimeline;
}
@Override
public void sync() {
HoodieTimeline oldTimeline = getTimeline();
HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants();
try {
writeLock.lock();
runSync(oldTimeline, newTimeline);
} finally {
writeLock.unlock();
}
}
/**
* Performs complete reset of file-system view. Subsequent partition view calls will load file slices against latest
* timeline
*
* @param oldTimeline Old Hoodie Timeline
* @param newTimeline New Hoodie Timeline
*/
protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) {
refreshTimeline(newTimeline);
addedPartitions.clear();
resetViewState();
// Initialize with new Hoodie timeline.
init(metaClient, newTimeline);
}
/**
* Return Only Commits and Compaction timeline for building file-groups
* @return
*/
public HoodieTimeline getVisibleCommitsAndCompactionTimeline() {
return visibleCommitsAndCompactionTimeline;
}
}

View File

@@ -0,0 +1,218 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.view;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hudi.common.SerializableConfiguration;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.SyncableFileSystemView;
import org.apache.hudi.common.util.Functions.Function2;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* A container that can potentially hold one or more dataset's
* file-system views. There is one view for each dataset. This is a view built against a timeline containing completed
* actions. In an embedded timeline-server mode, this typically holds only one dataset's view.
* In a stand-alone server mode, this can hold more than one dataset's views.
*
* FileSystemView can be stored "locally" using the following storage mechanisms:
* a. In Memory
* b. Spillable Map
* c. RocksDB
*
* But there can be cases where the file-system view is managed remoted. For example : Embedded Timeline Server). In
* this case, the clients will configure a remote filesystem view client (RemoteHoodieTableFileSystemView) for the
* dataset which can connect to the remote file system view and fetch views. THere are 2 modes here : REMOTE_FIRST and
* REMOTE_ONLY
* REMOTE_FIRST : The file-system view implementation on client side will act as a remote proxy. In case, if there
* is problem (or exceptions) querying remote file-system view, a backup local file-system view(using
* either one of in-memory, spillable, rocksDB) is used to server file-system view queries
* REMOTE_ONLY : In this case, there is no backup local file-system view. If there is problem (or exceptions)
* querying remote file-system view, then the exceptions are percolated back to client.
*
* FileSystemViewManager is designed to encapsulate the file-system view storage from clients using the file-system
* view. FileSystemViewManager uses a factory to construct specific implementation of file-system view and passes it to
* clients for querying.
*/
public class FileSystemViewManager {
private static Logger logger = LogManager.getLogger(FileSystemViewManager.class);
private final SerializableConfiguration conf;
// The View Storage config used to store file-system views
private final FileSystemViewStorageConfig viewStorageConfig;
// Map from Base-Path to View
private final ConcurrentHashMap<String, SyncableFileSystemView> globalViewMap;
// Factory Map to create file-system views
private final Function2<String, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator;
public FileSystemViewManager(SerializableConfiguration conf, FileSystemViewStorageConfig viewStorageConfig,
Function2<String, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator) {
this.conf = new SerializableConfiguration(conf);
this.viewStorageConfig = viewStorageConfig;
this.globalViewMap = new ConcurrentHashMap<>();
this.viewCreator = viewCreator;
}
/**
* Drops reference to File-System Views. Future calls to view results in creating a new view
* @param basePath
*/
public void clearFileSystemView(String basePath) {
SyncableFileSystemView view = globalViewMap.remove(basePath);
if (view != null) {
view.close();
}
}
/**
* Main API to get the file-system view for the base-path
* @param basePath
* @return
*/
public SyncableFileSystemView getFileSystemView(String basePath) {
return globalViewMap.computeIfAbsent(basePath,
(path) -> viewCreator.apply(path, viewStorageConfig));
}
/**
* Closes all views opened
*/
public void close() {
this.globalViewMap.values().stream().forEach(v -> v.close());
this.globalViewMap.clear();
}
// FACTORY METHODS FOR CREATING FILE-SYSTEM VIEWS
/**
* Create RocksDB based file System view for a dataset
* @param conf Hadoop Configuration
* @param viewConf View Storage Configuration
* @param basePath Base Path of dataset
* @return
*/
private static RocksDbBasedFileSystemView createRocksDBBasedFileSystemView(SerializableConfiguration conf,
FileSystemViewStorageConfig viewConf, String basePath) {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.get(), basePath, true);
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
return new RocksDbBasedFileSystemView(metaClient, timeline, viewConf);
}
/**
* Create a spillable Map based file System view for a dataset
* @param conf Hadoop Configuration
* @param viewConf View Storage Configuration
* @param basePath Base Path of dataset
* @return
*/
private static SpillableMapBasedFileSystemView createSpillableMapBasedFileSystemView(SerializableConfiguration conf,
FileSystemViewStorageConfig viewConf, String basePath) {
logger.info("Creating SpillableMap based view for basePath " + basePath);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.get(), basePath, true);
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
return new SpillableMapBasedFileSystemView(metaClient, timeline, viewConf);
}
/**
* Create an in-memory file System view for a dataset
* @param conf Hadoop Configuration
* @param viewConf View Storage Configuration
* @param basePath Base Path of dataset
* @return
*/
private static HoodieTableFileSystemView createInMemoryFileSystemView(SerializableConfiguration conf,
FileSystemViewStorageConfig viewConf, String basePath) {
logger.info("Creating InMemory based view for basePath " + basePath);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.get(), basePath, true);
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
return new HoodieTableFileSystemView(metaClient, timeline, viewConf.isIncrementalTimelineSyncEnabled());
}
/**
* Create a remote file System view for a dataset
* @param conf Hadoop Configuration
* @param viewConf View Storage Configuration
* @param metaClient Hoodie Table MetaClient for the dataset.
* @return
*/
private static RemoteHoodieTableFileSystemView createRemoteFileSystemView(SerializableConfiguration conf,
FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
logger.info("Creating remote view for basePath " + metaClient.getBasePath() + ". Server="
+ viewConf.getRemoteViewServerHost() + ":" + viewConf.getRemoteViewServerPort());
return new RemoteHoodieTableFileSystemView(viewConf.getRemoteViewServerHost(),
viewConf.getRemoteViewServerPort(), metaClient);
}
/**
* Main Factory method for building file-system views
* @param conf Hadoop Configuration
* @param config View Storage Configuration
* @return
*/
public static FileSystemViewManager createViewManager(
final SerializableConfiguration conf, final FileSystemViewStorageConfig config) {
logger.info("Creating View Manager with storage type :" + config.getStorageType());
switch (config.getStorageType()) {
case EMBEDDED_KV_STORE:
logger.info("Creating embedded rocks-db based Table View");
return new FileSystemViewManager(conf, config,
(basePath, viewConf) -> createRocksDBBasedFileSystemView(conf, viewConf, basePath));
case SPILLABLE_DISK:
logger.info("Creating Spillable Disk based Table View");
return new FileSystemViewManager(conf, config,
(basePath, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, basePath));
case MEMORY:
logger.info("Creating in-memory based Table View");
return new FileSystemViewManager(conf, config,
(basePath, viewConfig) -> createInMemoryFileSystemView(conf, viewConfig, basePath));
case REMOTE_ONLY:
logger.info("Creating remote only table view");
return new FileSystemViewManager(conf, config,
(basePath, viewConfig) -> createRemoteFileSystemView(conf, viewConfig,
new HoodieTableMetaClient(conf.get(), basePath)));
case REMOTE_FIRST:
logger.info("Creating remote first table view");
return new FileSystemViewManager(conf, config, (basePath, viewConfig) -> {
RemoteHoodieTableFileSystemView remoteFileSystemView =
createRemoteFileSystemView(conf, viewConfig, new HoodieTableMetaClient(conf.get(), basePath));
SyncableFileSystemView secondaryView = null;
switch (viewConfig.getSecondaryStorageType()) {
case MEMORY:
secondaryView = createInMemoryFileSystemView(conf, viewConfig, basePath);
break;
case EMBEDDED_KV_STORE:
secondaryView = createRocksDBBasedFileSystemView(conf, viewConfig, basePath);
break;
case SPILLABLE_DISK:
secondaryView = createSpillableMapBasedFileSystemView(conf, viewConfig, basePath);
break;
default:
throw new IllegalArgumentException("Secondary Storage type can only be in-memory or spillable. Was :"
+ viewConfig.getSecondaryStorageType());
}
return new PriorityBasedFileSystemView(remoteFileSystemView, secondaryView);
});
default:
throw new IllegalArgumentException("Unknown file system view type :" + config.getStorageType());
}
}
}

View File

@@ -0,0 +1,199 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.view;
import com.google.common.base.Preconditions;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
import org.apache.hudi.config.DefaultHoodieConfig;
/**
* File System View Storage Configurations
*/
public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
//Property Names
public static final String FILESYSTEM_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.type";
public static final String FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "hoodie.filesystem.view.incr.timeline.sync.enable";
public static final String FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.secondary.type";
public static final String FILESYSTEM_VIEW_REMOTE_HOST = "hoodie.filesystem.view.remote.host";
public static final String FILESYSTEM_VIEW_REMOTE_PORT = "hoodie.filesystem.view.remote.port";
public static final String FILESYSTEM_VIEW_SPILLABLE_DIR = "hoodie.filesystem.view.spillable.dir";
public static final String FILESYSTEM_VIEW_SPILLABLE_MEM = "hoodie.filesystem.view.spillable.mem";
public static final String FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION =
"hoodie.filesystem.view.spillable.compaction.mem.fraction";
private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path";
public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
public static final String DEFAULT_ROCKSDB_BASE_PATH = "/tmp/hoodie_timeline_rocksdb";
public static final String DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "false";
public static final String DEFUALT_REMOTE_VIEW_SERVER_HOST = "localhost";
public static final Integer DEFAULT_REMOTE_VIEW_SERVER_PORT = 26754;
public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/";
private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01;
private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB
public static FileSystemViewStorageConfig.Builder newBuilder() {
return new Builder();
}
private FileSystemViewStorageConfig(Properties props) {
super(props);
}
public FileSystemViewStorageType getStorageType() {
return FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_VIEW_STORAGE_TYPE));
}
public boolean isIncrementalTimelineSyncEnabled() {
return Boolean.valueOf(props.getProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE));
}
public String getRemoteViewServerHost() {
return props.getProperty(FILESYSTEM_VIEW_REMOTE_HOST);
}
public Integer getRemoteViewServerPort() {
return Integer.parseInt(props.getProperty(FILESYSTEM_VIEW_REMOTE_PORT));
}
public long getMaxMemoryForFileGroupMap() {
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
return totalMemory - getMaxMemoryForPendingCompaction();
}
public long getMaxMemoryForPendingCompaction() {
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
long reservedForPendingComaction = new Double(totalMemory * Double.parseDouble(
props.getProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION))).longValue();
return reservedForPendingComaction;
}
public String getBaseStoreDir() {
return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR);
}
public FileSystemViewStorageType getSecondaryStorageType() {
return FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
}
public String getRocksdbBasePath() {
return props.getProperty(ROCKSDB_BASE_PATH_PROP);
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
props.load(reader);
return this;
} finally {
reader.close();
}
}
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public Builder withStorageType(FileSystemViewStorageType storageType) {
props.setProperty(FILESYSTEM_VIEW_STORAGE_TYPE, storageType.name());
return this;
}
public Builder withSecondaryStorageType(FileSystemViewStorageType storageType) {
props.setProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, storageType.name());
return this;
}
public Builder withIncrementalTimelineSync(boolean enableIncrTimelineSync) {
props.setProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, Boolean.toString(enableIncrTimelineSync));
return this;
}
public Builder withRemoteServerHost(String remoteServerHost) {
props.setProperty(FILESYSTEM_VIEW_REMOTE_HOST, remoteServerHost);
return this;
}
public Builder withRemoteServerPort(Integer remoteServerPort) {
props.setProperty(FILESYSTEM_VIEW_REMOTE_PORT, remoteServerPort.toString());
return this;
}
public Builder withMaxMemoryForView(Long maxMemoryForView) {
props.setProperty(FILESYSTEM_VIEW_SPILLABLE_MEM, maxMemoryForView.toString());
return this;
}
public Builder withMemFractionForPendingCompaction(Double memFractionForPendingCompaction) {
props.setProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
return this;
}
public Builder withBaseStoreDir(String baseStorePath) {
props.setProperty(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath);
return this;
}
public Builder withRocksDBPath(String basePath) {
props.setProperty(ROCKSDB_BASE_PATH_PROP, basePath);
return this;
}
public FileSystemViewStorageConfig build() {
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_STORAGE_TYPE),
FILESYSTEM_VIEW_STORAGE_TYPE, DEFAULT_VIEW_STORAGE_TYPE.name());
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE),
FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE);
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE),
FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, DEFAULT_SECONDARY_VIEW_STORAGE_TYPE.name());
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REMOTE_HOST),
FILESYSTEM_VIEW_REMOTE_HOST, DEFUALT_REMOTE_VIEW_SERVER_HOST);
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REMOTE_PORT),
FILESYSTEM_VIEW_REMOTE_PORT, DEFAULT_REMOTE_VIEW_SERVER_PORT.toString());
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_SPILLABLE_DIR),
FILESYSTEM_VIEW_SPILLABLE_DIR, DEFAULT_VIEW_SPILLABLE_DIR);
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_SPILLABLE_MEM),
FILESYSTEM_VIEW_SPILLABLE_MEM, DEFAULT_MAX_MEMORY_FOR_VIEW.toString());
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION),
FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION.toString());
setDefaultOnCondition(props, !props.containsKey(ROCKSDB_BASE_PATH_PROP),
ROCKSDB_BASE_PATH_PROP, DEFAULT_ROCKSDB_BASE_PATH);
// Validations
FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_VIEW_STORAGE_TYPE));
FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
Preconditions.checkArgument(Integer.parseInt(props.getProperty(FILESYSTEM_VIEW_REMOTE_PORT)) > 0);
return new FileSystemViewStorageConfig(props);
}
}
}

View File

@@ -0,0 +1,36 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.view;
/**
* Storage Type used to store/retrieve File system view of a table
*/
public enum FileSystemViewStorageType {
// In-memory storage of file-system view
MEMORY,
// Constrained Memory storage for file-system view with overflow data spilled to disk
SPILLABLE_DISK,
// EMBEDDED Key Value Storage for file-system view
EMBEDDED_KV_STORE,
// Delegate file-system view to remote server
REMOTE_ONLY,
// A composite storage where file-system view calls are first delegated to Remote server ( REMOTE_ONLY )
// In case of failures, switches subsequent calls to secondary local storage type
REMOTE_FIRST
}

View File

@@ -0,0 +1,218 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.view;
import com.google.common.base.Preconditions;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.TableFileSystemView;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* TableFileSystemView Implementations based on in-memory storage.
* @see TableFileSystemView
* @since 0.3.0
*/
public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystemView {
private static Logger log = LogManager.getLogger(HoodieTableFileSystemView.class);
// mapping from partition paths to file groups contained within them
protected Map<String, List<HoodieFileGroup>> partitionToFileGroupsMap;
/**
* PartitionPath + File-Id to pending compaction instant time
*/
protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingCompaction;
/**
* Flag to determine if closed
*/
private boolean closed = false;
HoodieTableFileSystemView(boolean enableIncrementalTimelineSync) {
super(enableIncrementalTimelineSync);
}
/**
* Create a file system view, as of the given timeline
*/
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
this(metaClient, visibleActiveTimeline, false);
}
/**
* Create a file system view, as of the given timeline
*/
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline,
boolean enableIncrementalTimelineSync) {
super(enableIncrementalTimelineSync);
init(metaClient, visibleActiveTimeline);
}
@Override
public void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
this.partitionToFileGroupsMap = createPartitionToFileGroups();
super.init(metaClient, visibleActiveTimeline);
}
@Override
protected void resetViewState() {
this.fgIdToPendingCompaction = null;
this.partitionToFileGroupsMap = null;
}
protected Map<String, List<HoodieFileGroup>> createPartitionToFileGroups() {
return new ConcurrentHashMap<>();
}
protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> createFileIdToPendingCompactionMap(
Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fileIdToPendingCompaction) {
return fileIdToPendingCompaction;
}
/**
* Create a file system view, as of the given timeline, with the provided file statuses.
*/
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline,
FileStatus[] fileStatuses) {
this(metaClient, visibleActiveTimeline);
addFilesToView(fileStatuses);
}
/**
* 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();
}
private void writeObject(java.io.ObjectOutputStream out)
throws IOException {
out.defaultWriteObject();
}
@Override
protected boolean isPendingCompactionScheduledForFileId(HoodieFileGroupId fgId) {
return fgIdToPendingCompaction.containsKey(fgId);
}
@Override
protected void resetPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
// Build fileId to Pending Compaction Instants
this.fgIdToPendingCompaction = createFileIdToPendingCompactionMap(
operations.map(entry -> {
return Pair.of(entry.getValue().getFileGroupId(), Pair.of(entry.getKey(),entry.getValue()));
}).collect(Collectors.toMap(Pair::getKey, Pair::getValue)));
}
@Override
protected void addPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
operations.forEach(opInstantPair -> {
Preconditions.checkArgument(!fgIdToPendingCompaction.containsKey(opInstantPair.getValue().getFileGroupId()),
"Duplicate FileGroupId found in pending compaction operations. FgId :"
+ opInstantPair.getValue().getFileGroupId());
fgIdToPendingCompaction.put(opInstantPair.getValue().getFileGroupId(),
Pair.of(opInstantPair.getKey(), opInstantPair.getValue()));
});
}
@Override
protected void removePendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
operations.forEach(opInstantPair -> {
Preconditions.checkArgument(fgIdToPendingCompaction.containsKey(opInstantPair.getValue().getFileGroupId()),
"Trying to remove a FileGroupId which is not found in pending compaction operations. FgId :"
+ opInstantPair.getValue().getFileGroupId());
fgIdToPendingCompaction.remove(opInstantPair.getValue().getFileGroupId());
});
}
/**
* Given a partition path, obtain all filegroups within that. All methods, that work at the
* partition level go through this.
*/
@Override
Stream<HoodieFileGroup> fetchAllStoredFileGroups(String partition) {
final List<HoodieFileGroup> fileGroups = new ArrayList<>();
fileGroups.addAll(partitionToFileGroupsMap.get(partition));
return fileGroups.stream();
}
public Stream<HoodieFileGroup> getAllFileGroups() {
return fetchAllStoredFileGroups();
}
@Override
Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations() {
return fgIdToPendingCompaction.values().stream();
}
@Override
protected Option<Pair<String, CompactionOperation>> getPendingCompactionOperationWithInstant(HoodieFileGroupId fgId) {
return Option.ofNullable(fgIdToPendingCompaction.get(fgId));
}
@Override
protected boolean isPartitionAvailableInStore(String partitionPath) {
return partitionToFileGroupsMap.containsKey(partitionPath);
}
@Override
protected void storePartitionView(String partitionPath, List<HoodieFileGroup> fileGroups) {
log.info("Adding file-groups for partition :" + partitionPath + ", #FileGroups=" + fileGroups.size());
List<HoodieFileGroup> newList = new ArrayList<>(fileGroups);
partitionToFileGroupsMap.put(partitionPath, newList);
}
@Override
public Stream<HoodieFileGroup> fetchAllStoredFileGroups() {
return partitionToFileGroupsMap.values().stream().flatMap(fg -> {
return fg.stream();
});
}
public void close() {
closed = true;
super.reset();
partitionToFileGroupsMap = null;
fgIdToPendingCompaction = null;
}
public boolean isClosed() {
return closed;
}
}

View File

@@ -0,0 +1,354 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.view;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.TimelineDiffHelper;
import org.apache.hudi.common.util.TimelineDiffHelper.TimelineDiffResult;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Adds the capability to incrementally sync the changes to file-system view as and when new instants gets completed.
*/
public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTableFileSystemView {
private static Logger log = LogManager.getLogger(IncrementalTimelineSyncFileSystemView.class);
// Allows incremental Timeline syncing
private final boolean incrementalTimelineSyncEnabled;
// This is the visible active timeline used only for incremental view syncing
private HoodieTimeline visibleActiveTimeline;
protected IncrementalTimelineSyncFileSystemView(boolean enableIncrementalTimelineSync) {
this.incrementalTimelineSyncEnabled = enableIncrementalTimelineSync;
}
@Override
protected void refreshTimeline(HoodieTimeline visibleActiveTimeline) {
this.visibleActiveTimeline = visibleActiveTimeline;
super.refreshTimeline(visibleActiveTimeline);
}
@Override
protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) {
try {
if (incrementalTimelineSyncEnabled) {
TimelineDiffResult diffResult = TimelineDiffHelper.getNewInstantsForIncrementalSync(oldTimeline, newTimeline);
if (diffResult.canSyncIncrementally()) {
log.info("Doing incremental sync");
runIncrementalSync(newTimeline, diffResult);
log.info("Finished incremental sync");
// Reset timeline to latest
refreshTimeline(newTimeline);
return;
}
}
} catch (Exception ioe) {
log.error("Got exception trying to perform incremental sync. Reverting to complete sync", ioe);
}
log.warn("Incremental Sync of timeline is turned off or deemed unsafe. Will revert to full syncing");
super.runSync(oldTimeline, newTimeline);
}
/**
* Run incremental sync based on the diff result produced.
*
* @param timeline New Timeline
* @param diffResult Timeline Diff Result
*/
private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diffResult) {
log.info("Timeline Diff Result is :" + diffResult);
// First remove pending compaction instants which were completed
diffResult.getFinishedCompactionInstants().stream().forEach(instant -> {
try {
removePendingCompactionInstant(timeline, instant);
} catch (IOException e) {
throw new HoodieException(e);
}
});
// Add new completed instants found in the latest timeline
diffResult.getNewlySeenInstants().stream()
.filter(instant -> instant.isCompleted() || instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION))
.forEach(instant -> {
try {
if (instant.getAction().equals(HoodieTimeline.COMMIT_ACTION)
|| instant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)) {
addCommitInstant(timeline, instant);
} else if (instant.getAction().equals(HoodieTimeline.RESTORE_ACTION)) {
addRestoreInstant(timeline, instant);
} else if (instant.getAction().equals(HoodieTimeline.CLEAN_ACTION)) {
addCleanInstant(timeline, instant);
} else if (instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
addPendingCompactionInstant(timeline, instant);
} else if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) {
addRollbackInstant(timeline, instant);
}
} catch (IOException ioe) {
throw new HoodieException(ioe);
}
});
}
/**
* Remove Pending compaction instant
*
* @param timeline New Hoodie Timeline
* @param instant Compaction Instant to be removed
*/
private void removePendingCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Removing completed compaction instant (" + instant + ")");
HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp());
removePendingCompactionOperations(CompactionUtils.getPendingCompactionOperations(instant, plan)
.map(instantPair -> Pair.of(instantPair.getValue().getKey(),
CompactionOperation.convertFromAvroRecordInstance(instantPair.getValue().getValue()))));
}
/**
* Add newly found compaction instant
*
* @param timeline Hoodie Timeline
* @param instant Compaction Instant
*/
private void addPendingCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Syncing pending compaction instant (" + instant + ")");
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp());
List<Pair<String, CompactionOperation>> pendingOps =
CompactionUtils.getPendingCompactionOperations(instant, compactionPlan).map(p -> Pair.of(p.getValue().getKey(),
CompactionOperation.convertFromAvroRecordInstance(p.getValue().getValue()))).collect(Collectors.toList());
// First, update Pending compaction instants
addPendingCompactionOperations(pendingOps.stream());
Map<String, List<Pair<String, HoodieFileGroup>>> partitionToFileGroups =
pendingOps.stream().map(opPair -> {
String compactionInstantTime = opPair.getKey();
HoodieFileGroup fileGroup = new HoodieFileGroup(opPair.getValue().getFileGroupId(), timeline);
fileGroup.addNewFileSliceAtInstant(compactionInstantTime);
return Pair.of(compactionInstantTime, fileGroup);
}).collect(Collectors.groupingBy(x -> x.getValue().getPartitionPath()));
partitionToFileGroups.entrySet().forEach(entry -> {
if (isPartitionAvailableInStore(entry.getKey())) {
applyDeltaFileSlicesToPartitionView(entry.getKey(),
entry.getValue().stream().map(Pair::getValue).collect(Collectors.toList()), DeltaApplyMode.ADD);
}
});
}
/**
* Add newly found commit/delta-commit instant
*
* @param timeline Hoodie Timeline
* @param instant Instant
*/
private void addCommitInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Syncing committed instant (" + instant + ")");
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(instant).get(),
HoodieCommitMetadata.class);
commitMetadata.getPartitionToWriteStats().entrySet().stream().forEach(entry -> {
String partition = entry.getKey();
if (isPartitionAvailableInStore(partition)) {
log.info("Syncing partition (" + partition + ") of instant (" + instant + ")");
FileStatus[] statuses = entry.getValue().stream().map(p -> {
FileStatus status = new FileStatus(p.getFileSizeInBytes(), false, 0, 0, 0, 0, null, null, null,
new Path(String.format("%s/%s", metaClient.getBasePath(), p.getPath())));
return status;
}).toArray(FileStatus[]::new);
List<HoodieFileGroup> fileGroups = buildFileGroups(statuses, timeline.filterCompletedAndCompactionInstants(),
false);
applyDeltaFileSlicesToPartitionView(partition, fileGroups, DeltaApplyMode.ADD);
} else {
log.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
}
});
log.info("Done Syncing committed instant (" + instant + ")");
}
/**
* Add newly found restore instant
*
* @param timeline Hoodie Timeline
* @param instant Restore Instant
*/
private void addRestoreInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Syncing restore instant (" + instant + ")");
HoodieRestoreMetadata metadata = AvroUtils.deserializeAvroMetadata(
timeline.getInstantDetails(instant).get(), HoodieRestoreMetadata.class);
Map<String, List<Pair<String, String>>> partitionFiles =
metadata.getHoodieRestoreMetadata().entrySet().stream().flatMap(entry -> {
return entry.getValue().stream().flatMap(e -> e.getPartitionMetadata().entrySet().stream().flatMap(e2 -> {
return e2.getValue().getSuccessDeleteFiles().stream().map(x -> Pair.of(e2.getKey(), x));
}));
}).collect(Collectors.groupingBy(Pair::getKey));
partitionFiles.entrySet().stream().forEach(e -> {
removeFileSlicesForPartition(timeline, instant, e.getKey(),
e.getValue().stream().map(x -> x.getValue()).collect(Collectors.toList()));
});
log.info("Done Syncing restore instant (" + instant + ")");
}
/**
* Add newly found rollback instant
*
* @param timeline Hoodie Timeline
* @param instant Rollback Instant
*/
private void addRollbackInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Syncing rollback instant (" + instant + ")");
HoodieRollbackMetadata metadata = AvroUtils.deserializeAvroMetadata(
timeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class);
metadata.getPartitionMetadata().entrySet().stream().forEach(e -> {
removeFileSlicesForPartition(timeline, instant, e.getKey(), e.getValue().getSuccessDeleteFiles());
});
log.info("Done Syncing rollback instant (" + instant + ")");
}
/**
* Add newly found clean instant
*
* @param timeline Timeline
* @param instant Clean instant
*/
private void addCleanInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Syncing cleaner instant (" + instant + ")");
HoodieCleanMetadata cleanMetadata = AvroUtils
.deserializeHoodieCleanMetadata(timeline.getInstantDetails(instant).get());
cleanMetadata.getPartitionMetadata().entrySet().stream().forEach(entry -> {
removeFileSlicesForPartition(timeline, instant, entry.getKey(), entry.getValue().getSuccessDeleteFiles());
});
log.info("Done Syncing cleaner instant (" + instant + ")");
}
private void removeFileSlicesForPartition(HoodieTimeline timeline, HoodieInstant instant,
String partition, List<String> paths) {
if (isPartitionAvailableInStore(partition)) {
log.info("Removing file slices for partition (" + partition + ") for instant (" + instant + ")");
FileStatus[] statuses = paths.stream().map(p -> {
FileStatus status = new FileStatus();
status.setPath(new Path(p));
return status;
}).toArray(FileStatus[]::new);
List<HoodieFileGroup> fileGroups = buildFileGroups(statuses,
timeline.filterCompletedAndCompactionInstants(), false);
applyDeltaFileSlicesToPartitionView(partition, fileGroups, DeltaApplyMode.REMOVE);
} else {
log.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
}
}
/**
* Apply mode whether to add or remove the delta view
*/
enum DeltaApplyMode {
ADD,
REMOVE
}
/**
* Apply changes to partition file-system view. Base Implementation overwrites the entire partitions view assuming
* some sort of map (in-mem/disk-based) is used. For View implementation which supports fine-granular updates (e:g
* RocksDB), override this method.
*
* @param partition PartitionPath
* @param deltaFileGroups Changed file-slices aggregated as file-groups
* @param mode Delta Apply mode
*/
protected void applyDeltaFileSlicesToPartitionView(String partition, List<HoodieFileGroup> deltaFileGroups,
DeltaApplyMode mode) {
if (deltaFileGroups.isEmpty()) {
log.info("No delta file groups for partition :" + partition);
return;
}
List<HoodieFileGroup> fileGroups = fetchAllStoredFileGroups(partition).collect(Collectors.toList());
/**
* Note that while finding the new data/log files added/removed, the path stored in metadata will be missing
* the base-path,scheme and authority. Ensure the matching process takes care of this discrepancy.
*/
Map<String, HoodieDataFile> viewDataFiles = fileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
.map(FileSlice::getDataFile).filter(Option::isPresent).map(Option::get)
.map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
//Note: Delta Log Files and Data FIles can be empty when adding/removing pending compactions
Map<String, HoodieDataFile> deltaDataFiles = deltaFileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
.map(FileSlice::getDataFile).filter(Option::isPresent).map(Option::get)
.map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
Map<String, HoodieLogFile> viewLogFiles = fileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
.flatMap(FileSlice::getLogFiles)
.map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
Map<String, HoodieLogFile> deltaLogFiles = deltaFileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
.flatMap(FileSlice::getLogFiles)
.map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
switch (mode) {
case ADD:
viewDataFiles.putAll(deltaDataFiles);
viewLogFiles.putAll(deltaLogFiles);
break;
case REMOVE:
deltaDataFiles.keySet().stream().forEach(p -> viewDataFiles.remove(p));
deltaLogFiles.keySet().stream().forEach(p -> viewLogFiles.remove(p));
break;
default:
throw new IllegalStateException("Unknown diff apply mode=" + mode);
}
HoodieTimeline timeline = deltaFileGroups.stream().map(df -> df.getTimeline()).findAny().get();
List<HoodieFileGroup> fgs =
buildFileGroups(viewDataFiles.values().stream(), viewLogFiles.values().stream(), timeline, true);
storePartitionView(partition, fgs);
}
@Override
public HoodieTimeline getTimeline() {
return visibleActiveTimeline;
}
}

View File

@@ -0,0 +1,241 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.view;
import java.io.Serializable;
import java.util.List;
import java.util.stream.Stream;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.SyncableFileSystemView;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Functions.Function0;
import org.apache.hudi.common.util.Functions.Function1;
import org.apache.hudi.common.util.Functions.Function2;
import org.apache.hudi.common.util.Functions.Function3;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* A file system view which proxies request to a preferred File System View implementation. In case of error,
* flip all subsequent calls to a backup file-system view implementation.
*/
public class PriorityBasedFileSystemView implements SyncableFileSystemView, Serializable {
private static Logger log = LogManager.getLogger(PriorityBasedFileSystemView.class);
private final SyncableFileSystemView preferredView;
private final SyncableFileSystemView secondaryView;
private boolean errorOnPreferredView;
public PriorityBasedFileSystemView(SyncableFileSystemView preferredView, SyncableFileSystemView secondaryView) {
this.preferredView = preferredView;
this.secondaryView = secondaryView;
this.errorOnPreferredView = false;
}
private <R> R execute(Function0<R> preferredFunction, Function0<R> secondaryFunction) {
if (errorOnPreferredView) {
log.warn("Routing request to secondary file-system view");
return secondaryFunction.apply();
} else {
try {
return preferredFunction.apply();
} catch (RuntimeException re) {
log.error("Got error running preferred function. Trying secondary", re);
errorOnPreferredView = true;
return secondaryFunction.apply();
}
}
}
private <T1, R> R execute(T1 val, Function1<T1, R> preferredFunction, Function1<T1, R> secondaryFunction) {
if (errorOnPreferredView) {
log.warn("Routing request to secondary file-system view");
return secondaryFunction.apply(val);
} else {
try {
return preferredFunction.apply(val);
} catch (RuntimeException re) {
log.error("Got error running preferred function. Trying secondary", re);
errorOnPreferredView = true;
return secondaryFunction.apply(val);
}
}
}
private <T1, T2, R> R execute(T1 val, T2 val2, Function2<T1, T2, R> preferredFunction,
Function2<T1, T2, R> secondaryFunction) {
if (errorOnPreferredView) {
log.warn("Routing request to secondary file-system view");
return secondaryFunction.apply(val, val2);
} else {
try {
return preferredFunction.apply(val, val2);
} catch (RuntimeException re) {
log.error("Got error running preferred function. Trying secondary", re);
errorOnPreferredView = true;
return secondaryFunction.apply(val, val2);
}
}
}
private <T1, T2, T3, R> R execute(T1 val, T2 val2, T3 val3, Function3<T1, T2, T3, R> preferredFunction,
Function3<T1, T2, T3, R> secondaryFunction) {
if (errorOnPreferredView) {
log.warn("Routing request to secondary file-system view");
return secondaryFunction.apply(val, val2, val3);
} else {
try {
return preferredFunction.apply(val, val2, val3);
} catch (RuntimeException re) {
log.error("Got error running preferred function. Trying secondary", re);
errorOnPreferredView = true;
return secondaryFunction.apply(val, val2, val3);
}
}
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles(String partitionPath) {
return execute(partitionPath, preferredView::getLatestDataFiles, secondaryView::getLatestDataFiles);
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles() {
return execute(preferredView::getLatestDataFiles, secondaryView::getLatestDataFiles);
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) {
return execute(partitionPath, maxCommitTime, preferredView::getLatestDataFilesBeforeOrOn,
secondaryView::getLatestDataFilesBeforeOrOn);
}
@Override
public Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId) {
return execute(partitionPath, fileId, preferredView::getLatestDataFile, secondaryView::getLatestDataFile);
}
@Override
public Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId) {
return execute(partitionPath, instantTime, fileId, preferredView::getDataFileOn,
secondaryView::getDataFileOn);
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
return execute(commitsToReturn, preferredView::getLatestDataFilesInRange, secondaryView::getLatestDataFilesInRange);
}
@Override
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
return execute(partitionPath, preferredView::getAllDataFiles, secondaryView::getAllDataFiles);
}
@Override
public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
return execute(partitionPath, preferredView::getLatestFileSlices, secondaryView::getLatestFileSlices);
}
@Override
public Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionPath) {
return execute(partitionPath, preferredView::getLatestUnCompactedFileSlices,
secondaryView::getLatestUnCompactedFileSlices);
}
@Override
public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath, String maxCommitTime,
boolean includeFileSlicesInPendingCompaction) {
return execute(partitionPath, maxCommitTime, includeFileSlicesInPendingCompaction,
preferredView::getLatestFileSlicesBeforeOrOn, secondaryView::getLatestFileSlicesBeforeOrOn);
}
@Override
public Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime) {
return execute(partitionPath, maxInstantTime, preferredView::getLatestMergedFileSlicesBeforeOrOn,
secondaryView::getLatestMergedFileSlicesBeforeOrOn);
}
@Override
public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
return execute(commitsToReturn, preferredView::getLatestFileSliceInRange, secondaryView::getLatestFileSliceInRange);
}
@Override
public Stream<FileSlice> getAllFileSlices(String partitionPath) {
return execute(partitionPath, preferredView::getAllFileSlices, secondaryView::getAllFileSlices);
}
@Override
public Stream<HoodieFileGroup> getAllFileGroups(String partitionPath) {
return execute(partitionPath, preferredView::getAllFileGroups, secondaryView::getAllFileGroups);
}
@Override
public Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations() {
return execute(preferredView::getPendingCompactionOperations, secondaryView::getPendingCompactionOperations);
}
@Override
public void close() {
preferredView.close();
secondaryView.close();
}
@Override
public void reset() {
preferredView.reset();
secondaryView.reset();
}
@Override
public Option<HoodieInstant> getLastInstant() {
return execute(preferredView::getLastInstant, secondaryView::getLastInstant);
}
@Override
public HoodieTimeline getTimeline() {
return execute(preferredView::getTimeline, secondaryView::getTimeline);
}
@Override
public void sync() {
preferredView.reset();
secondaryView.reset();
}
@Override
public Option<FileSlice> getLatestFileSlice(String partitionPath, String fileId) {
return execute(partitionPath, fileId, preferredView::getLatestFileSlice, secondaryView::getLatestFileSlice);
}
public SyncableFileSystemView getPreferredView() {
return preferredView;
}
public SyncableFileSystemView getSecondaryView() {
return secondaryView;
}
}

View File

@@ -0,0 +1,476 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.view;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import java.io.IOException;
import java.io.Serializable;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Stream;
import org.apache.http.client.fluent.Request;
import org.apache.http.client.fluent.Response;
import org.apache.http.client.utils.URIBuilder;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.SyncableFileSystemView;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO;
import org.apache.hudi.common.table.timeline.dto.DataFileDTO;
import org.apache.hudi.common.table.timeline.dto.FileGroupDTO;
import org.apache.hudi.common.table.timeline.dto.FileSliceDTO;
import org.apache.hudi.common.table.timeline.dto.InstantDTO;
import org.apache.hudi.common.table.timeline.dto.TimelineDTO;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieRemoteException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* A proxy for table file-system view which translates local View API calls to REST calls to remote timeline service
*/
public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, Serializable {
private static final String BASE_URL = "/v1/hoodie/view";
public static final String LATEST_PARTITION_SLICES_URL = String.format("%s/%s", BASE_URL,
"slices/partition/latest/");
public static final String LATEST_PARTITION_SLICE_URL = String.format("%s/%s", BASE_URL,
"slices/file/latest/");
public static final String LATEST_PARTITION_UNCOMPACTED_SLICES_URL = String.format("%s/%s", BASE_URL,
"slices/uncompacted/partition/latest/");
public static final String ALL_SLICES_URL = String.format("%s/%s", BASE_URL, "slices/all");
public static final String LATEST_SLICES_MERGED_BEFORE_ON_INSTANT_URL =
String.format("%s/%s", BASE_URL, "slices/merged/beforeoron/latest/");
public static final String LATEST_SLICES_RANGE_INSTANT_URL =
String.format("%s/%s", BASE_URL, "slices/range/latest/");
public static final String LATEST_SLICES_BEFORE_ON_INSTANT_URL =
String.format("%s/%s", BASE_URL, "slices/beforeoron/latest/");
public static final String PENDING_COMPACTION_OPS =
String.format("%s/%s", BASE_URL, "compactions/pending/");
public static final String LATEST_PARTITION_DATA_FILES_URL = String.format("%s/%s", BASE_URL,
"datafiles/latest/partition");
public static final String LATEST_PARTITION_DATA_FILE_URL = String.format("%s/%s", BASE_URL,
"datafile/latest/partition");
public static final String ALL_DATA_FILES = String.format("%s/%s", BASE_URL, "datafiles/all");
public static final String LATEST_ALL_DATA_FILES = String.format("%s/%s", BASE_URL, "datafiles/all/latest/");
public static final String LATEST_DATA_FILE_ON_INSTANT_URL =
String.format("%s/%s", BASE_URL, "datafile/on/latest/");
public static final String LATEST_DATA_FILES_RANGE_INSTANT_URL =
String.format("%s/%s", BASE_URL, "datafiles/range/latest/");
public static final String LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL =
String.format("%s/%s", BASE_URL, "datafiles/beforeoron/latest/");
public static final String ALL_FILEGROUPS_FOR_PARTITION_URL =
String.format("%s/%s", BASE_URL, "filegroups/all/partition/");
public static final String LAST_INSTANT = String.format("%s/%s", BASE_URL, "timeline/instant/last");
public static final String LAST_INSTANTS = String.format("%s/%s", BASE_URL, "timeline/instants/last");
public static final String TIMELINE = String.format("%s/%s", BASE_URL, "timeline/instants/all");
// POST Requests
public static final String REFRESH_DATASET = String.format("%s/%s", BASE_URL, "refresh/");
public static final String PARTITION_PARAM = "partition";
public static final String BASEPATH_PARAM = "basepath";
public static final String INSTANT_PARAM = "instant";
public static final String MAX_INSTANT_PARAM = "maxinstant";
public static final String INSTANTS_PARAM = "instants";
public static final String FILEID_PARAM = "fileid";
public static final String LAST_INSTANT_TS = "lastinstantts";
public static final String TIMELINE_HASH = "timelinehash";
public static final String REFRESH_OFF = "refreshoff";
public static final String INCLUDE_FILES_IN_PENDING_COMPACTION_PARAM = "includependingcompaction";
private static Logger log = LogManager.getLogger(RemoteHoodieTableFileSystemView.class);
private final String serverHost;
private final int serverPort;
private final String basePath;
private final HoodieTableMetaClient metaClient;
private final HoodieTimeline timeline;
private final ObjectMapper mapper;
private boolean closed = false;
private enum RequestMethod {
GET,
POST
}
public RemoteHoodieTableFileSystemView(String server, int port, HoodieTableMetaClient metaClient) {
this.basePath = metaClient.getBasePath();
this.serverHost = server;
this.serverPort = port;
this.mapper = new ObjectMapper();
this.metaClient = metaClient;
this.timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
}
private <T> T executeRequest(String requestPath, Map<String, String> queryParameters, TypeReference reference,
RequestMethod method) throws IOException {
Preconditions.checkArgument(!closed, "View already closed");
URIBuilder builder = new URIBuilder().setHost(serverHost).setPort(serverPort).setPath(requestPath)
.setScheme("http");
queryParameters.entrySet().stream().forEach(entry -> {
builder.addParameter(entry.getKey(), entry.getValue());
});
// Adding mandatory parameters - Last instants affecting file-slice
timeline.lastInstant().ifPresent(instant -> builder.addParameter(LAST_INSTANT_TS, instant.getTimestamp()));
builder.addParameter(TIMELINE_HASH, timeline.getTimelineHash());
String url = builder.toString();
log.info("Sending request : (" + url + ")");
Response response = null;
int timeout = 1000 * 300; // 5 min timeout
switch (method) {
case GET:
response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
break;
case POST:
default:
response = Request.Post(url).connectTimeout(timeout).socketTimeout(timeout).execute();
break;
}
String content = response.returnContent().asString();
return mapper.readValue(content, reference);
}
private Map<String, String> getParamsWithPartitionPath(String partitionPath) {
Map<String, String> paramsMap = new HashMap<>();
paramsMap.put(BASEPATH_PARAM, basePath);
paramsMap.put(PARTITION_PARAM, partitionPath);
return paramsMap;
}
private Map<String, String> getParams() {
Map<String, String> paramsMap = new HashMap<>();
paramsMap.put(BASEPATH_PARAM, basePath);
return paramsMap;
}
private Map<String, String> getParams(String paramName, String instant) {
Map<String, String> paramsMap = new HashMap<>();
paramsMap.put(BASEPATH_PARAM, basePath);
paramsMap.put(paramName, instant);
return paramsMap;
}
private Map<String, String> getParamsWithAdditionalParam(String partitionPath, String paramName, String paramVal) {
Map<String, String> paramsMap = new HashMap<>();
paramsMap.put(BASEPATH_PARAM, basePath);
paramsMap.put(PARTITION_PARAM, partitionPath);
paramsMap.put(paramName, paramVal);
return paramsMap;
}
private Map<String, String> getParamsWithAdditionalParams(String partitionPath, String[] paramNames,
String[] paramVals) {
Map<String, String> paramsMap = new HashMap<>();
paramsMap.put(BASEPATH_PARAM, basePath);
paramsMap.put(PARTITION_PARAM, partitionPath);
Preconditions.checkArgument(paramNames.length == paramVals.length);
for (int i = 0; i < paramNames.length; i++) {
paramsMap.put(paramNames[i], paramVals[i]);
}
return paramsMap;
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles(String partitionPath) {
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_PARTITION_DATA_FILES_URL, paramsMap,
new TypeReference<List<DataFileDTO>>() {
}, RequestMethod.GET);
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles() {
Map<String, String> paramsMap = getParams();
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_ALL_DATA_FILES, paramsMap,
new TypeReference<List<DataFileDTO>>() {
}, RequestMethod.GET);
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) {
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime);
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL, paramsMap,
new TypeReference<List<DataFileDTO>>() {
}, RequestMethod.GET);
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId) {
Map<String, String> paramsMap = getParamsWithAdditionalParams(partitionPath,
new String[]{INSTANT_PARAM, FILEID_PARAM},
new String[]{instantTime, fileId});
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILE_ON_INSTANT_URL, paramsMap,
new TypeReference<List<DataFileDTO>>() {
}, RequestMethod.GET);
return Option.fromJavaOptional(dataFiles.stream().map(DataFileDTO::toHoodieDataFile).findFirst());
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
Map<String, String> paramsMap = getParams(INSTANTS_PARAM,
StringUtils.join(commitsToReturn.toArray(new String[0]), ","));
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILES_RANGE_INSTANT_URL, paramsMap,
new TypeReference<List<DataFileDTO>>() {
}, RequestMethod.GET);
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
try {
List<DataFileDTO> dataFiles = executeRequest(ALL_DATA_FILES, paramsMap,
new TypeReference<List<DataFileDTO>>() {
}, RequestMethod.GET);
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
try {
List<FileSliceDTO> dataFiles = executeRequest(LATEST_PARTITION_SLICES_URL, paramsMap,
new TypeReference<List<FileSliceDTO>>() {
}, RequestMethod.GET);
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Option<FileSlice> getLatestFileSlice(String partitionPath, String fileId) {
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId);
try {
List<FileSliceDTO> dataFiles = executeRequest(LATEST_PARTITION_SLICE_URL, paramsMap,
new TypeReference<List<FileSliceDTO>>() {
}, RequestMethod.GET);
return Option.fromJavaOptional(dataFiles.stream().map(FileSliceDTO::toFileSlice).findFirst());
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionPath) {
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
try {
List<FileSliceDTO> dataFiles = executeRequest(LATEST_PARTITION_UNCOMPACTED_SLICES_URL, paramsMap,
new TypeReference<List<FileSliceDTO>>() {
}, RequestMethod.GET);
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath, String maxCommitTime,
boolean includeFileSlicesInPendingCompaction) {
Map<String, String> paramsMap = getParamsWithAdditionalParams(partitionPath,
new String[]{MAX_INSTANT_PARAM, INCLUDE_FILES_IN_PENDING_COMPACTION_PARAM},
new String[]{maxCommitTime, String.valueOf(includeFileSlicesInPendingCompaction)});
try {
List<FileSliceDTO> dataFiles = executeRequest(LATEST_SLICES_BEFORE_ON_INSTANT_URL, paramsMap,
new TypeReference<List<FileSliceDTO>>() {
}, RequestMethod.GET);
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime) {
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxInstantTime);
try {
List<FileSliceDTO> dataFiles = executeRequest(LATEST_SLICES_MERGED_BEFORE_ON_INSTANT_URL, paramsMap,
new TypeReference<List<FileSliceDTO>>() {
}, RequestMethod.GET);
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
Map<String, String> paramsMap = getParams(INSTANTS_PARAM,
StringUtils.join(commitsToReturn.toArray(new String[0]), ","));
try {
List<FileSliceDTO> dataFiles = executeRequest(LATEST_SLICES_RANGE_INSTANT_URL, paramsMap,
new TypeReference<List<FileSliceDTO>>() {
}, RequestMethod.GET);
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<FileSlice> getAllFileSlices(String partitionPath) {
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
try {
List<FileSliceDTO> dataFiles = executeRequest(ALL_SLICES_URL, paramsMap,
new TypeReference<List<FileSliceDTO>>() {
}, RequestMethod.GET);
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<HoodieFileGroup> getAllFileGroups(String partitionPath) {
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
try {
List<FileGroupDTO> fileGroups = executeRequest(ALL_FILEGROUPS_FOR_PARTITION_URL, paramsMap,
new TypeReference<List<FileGroupDTO>>() {
}, RequestMethod.GET);
return fileGroups.stream().map(dto -> FileGroupDTO.toFileGroup(dto, metaClient));
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
public boolean refresh() {
Map<String, String> paramsMap = getParams();
try {
return executeRequest(REFRESH_DATASET, paramsMap, new TypeReference<Boolean>() {
}, RequestMethod.POST);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations() {
Map<String, String> paramsMap = getParams();
try {
List<CompactionOpDTO> dtos = executeRequest(PENDING_COMPACTION_OPS, paramsMap,
new TypeReference<List<CompactionOpDTO>>() {
}, RequestMethod.GET);
return dtos.stream().map(CompactionOpDTO::toCompactionOperation);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public void close() {
closed = true;
}
@Override
public void reset() {
refresh();
}
@Override
public Option<HoodieInstant> getLastInstant() {
Map<String, String> paramsMap = getParams();
try {
List<InstantDTO> instants = executeRequest(LAST_INSTANT, paramsMap,
new TypeReference<List<InstantDTO>>() {
}, RequestMethod.GET);
return Option.fromJavaOptional(instants.stream().map(InstantDTO::toInstant).findFirst());
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public HoodieTimeline getTimeline() {
Map<String, String> paramsMap = getParams();
try {
TimelineDTO timeline = executeRequest(TIMELINE, paramsMap,
new TypeReference<TimelineDTO>() {
}, RequestMethod.GET);
return TimelineDTO.toTimeline(timeline, metaClient);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public void sync() {
//noop
}
@Override
public Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId) {
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId);
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_PARTITION_DATA_FILE_URL, paramsMap,
new TypeReference<List<DataFileDTO>>() {
}, RequestMethod.GET);
return Option.fromJavaOptional(dataFiles.stream().map(DataFileDTO::toHoodieDataFile).findFirst());
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
}

View File

@@ -0,0 +1,344 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.view;
import com.google.common.base.Preconditions;
import java.io.Serializable;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.RocksDBDAO;
import org.apache.hudi.common.util.RocksDBSchemaHelper;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* A file-system view implementation on top of embedded Rocks DB store.
* For each DataSet : 3 column Family is added for storing
* (1) File-Slices and Data Files for View lookups
* (2) Pending compaction operations
* (3) Partitions tracked
*
* Fine-grained retrieval API to fetch latest file-slice and data-file which are common operations
* for ingestion/compaction are supported.
*
* TODO: vb The current implementation works in embedded server mode where each restarts blows away the view stores.
* To support view-state preservation across restarts, Hoodie timeline also needs to be stored
* inorder to detect changes to timeline across restarts.
*/
public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSystemView {
private static Logger log = LogManager.getLogger(RocksDbBasedFileSystemView.class);
private final FileSystemViewStorageConfig config;
private final RocksDBSchemaHelper schemaHelper;
private RocksDBDAO rocksDB;
private boolean closed = false;
public RocksDbBasedFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline, FileSystemViewStorageConfig config) {
super(config.isIncrementalTimelineSyncEnabled());
this.config = config;
this.schemaHelper = new RocksDBSchemaHelper(metaClient);
this.rocksDB = new RocksDBDAO(metaClient.getBasePath(), config.getRocksdbBasePath());
init(metaClient, visibleActiveTimeline);
}
public RocksDbBasedFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline, FileStatus[] fileStatuses, FileSystemViewStorageConfig config) {
this(metaClient, visibleActiveTimeline, config);
addFilesToView(fileStatuses);
}
@Override
protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
schemaHelper.getAllColumnFamilies().stream().forEach(rocksDB::addColumnFamily);
super.init(metaClient, visibleActiveTimeline);
log.info("Created ROCKSDB based file-system view at " + config.getRocksdbBasePath());
}
@Override
protected boolean isPendingCompactionScheduledForFileId(HoodieFileGroupId fgId) {
return getPendingCompactionOperationWithInstant(fgId).isPresent();
}
@Override
protected void resetPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
rocksDB.writeBatch(batch -> {
operations.forEach(opPair -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
schemaHelper.getKeyForPendingCompactionLookup(opPair.getValue().getFileGroupId()), opPair);
});
log.info("Initializing pending compaction operations. Count=" + batch.count());
});
}
@Override
protected void addPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
rocksDB.writeBatch(batch -> {
operations.forEach(opInstantPair -> {
Preconditions.checkArgument(!isPendingCompactionScheduledForFileId(opInstantPair.getValue().getFileGroupId()),
"Duplicate FileGroupId found in pending compaction operations. FgId :"
+ opInstantPair.getValue().getFileGroupId());
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
schemaHelper.getKeyForPendingCompactionLookup(opInstantPair.getValue().getFileGroupId()), opInstantPair);
});
});
}
@Override
void removePendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
rocksDB.writeBatch(batch -> {
operations.forEach(opInstantPair -> {
Preconditions.checkArgument(
getPendingCompactionOperationWithInstant(opInstantPair.getValue().getFileGroupId()) != null,
"Trying to remove a FileGroupId which is not found in pending compaction operations. FgId :"
+ opInstantPair.getValue().getFileGroupId());
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
schemaHelper.getKeyForPendingCompactionLookup(opInstantPair.getValue().getFileGroupId()));
});
});
}
@Override
protected void resetViewState() {
log.info("Deleting all rocksdb data associated with dataset filesystem view");
rocksDB.close();
rocksDB = new RocksDBDAO(metaClient.getBasePath(), config.getRocksdbBasePath());
}
@Override
protected Option<Pair<String, CompactionOperation>> getPendingCompactionOperationWithInstant(HoodieFileGroupId fgId) {
String lookupKey = schemaHelper.getKeyForPendingCompactionLookup(fgId);
Pair<String, CompactionOperation> instantOperationPair =
rocksDB.get(schemaHelper.getColFamilyForPendingCompaction(), lookupKey);
return Option.ofNullable(instantOperationPair);
}
@Override
protected boolean isPartitionAvailableInStore(String partitionPath) {
String lookupKey = schemaHelper.getKeyForPartitionLookup(partitionPath);
Serializable obj = rocksDB.get(schemaHelper.getColFamilyForStoredPartitions(), lookupKey);
return obj != null;
}
@Override
protected void storePartitionView(String partitionPath, List<HoodieFileGroup> fileGroups) {
log.info("Resetting and adding new partition (" + partitionPath + ") to ROCKSDB based file-system view at "
+ config.getRocksdbBasePath() + ", Total file-groups=" + fileGroups.size());
String lookupKey = schemaHelper.getKeyForPartitionLookup(partitionPath);
rocksDB.delete(schemaHelper.getColFamilyForStoredPartitions(), lookupKey);
// First delete partition views
rocksDB.prefixDelete(schemaHelper.getColFamilyForView(),
schemaHelper.getPrefixForSliceViewByPartition(partitionPath));
rocksDB.prefixDelete(schemaHelper.getColFamilyForView(),
schemaHelper.getPrefixForDataFileViewByPartition(partitionPath));
// Now add them
fileGroups.stream().forEach(fg -> {
rocksDB.writeBatch(batch -> {
fg.getAllFileSlicesIncludingInflight().forEach(fs -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs);
fs.getDataFile().ifPresent(df -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs),
df);
});
});
});
});
// record that partition is loaded.
rocksDB.put(schemaHelper.getColFamilyForStoredPartitions(), lookupKey, Boolean.TRUE);
log.info("Finished adding new partition (" + partitionPath + ") to ROCKSDB based file-system view at "
+ config.getRocksdbBasePath() + ", Total file-groups=" + fileGroups.size());
}
@Override
/**
* This is overridden to incrementally apply file-slices to rocks DB
*/
protected void applyDeltaFileSlicesToPartitionView(String partition, List<HoodieFileGroup> deltaFileGroups,
DeltaApplyMode mode) {
rocksDB.writeBatch(batch -> {
deltaFileGroups.stream().forEach(fg -> {
fg.getAllRawFileSlices().map(fs -> {
FileSlice oldSlice = getFileSlice(partition, fs.getFileId(), fs.getBaseInstantTime());
if (null == oldSlice) {
return fs;
} else {
// First remove the file-slice
log.info("Removing old Slice in DB. FS=" + oldSlice);
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(),
schemaHelper.getKeyForSliceView(fg, oldSlice));
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(),
schemaHelper.getKeyForDataFileView(fg, oldSlice));
Map<String, HoodieLogFile> logFiles = oldSlice.getLogFiles()
.map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
Map<String, HoodieLogFile> deltaLogFiles = fs.getLogFiles()
.map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
switch (mode) {
case ADD: {
FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
oldSlice.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
fs.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
Map<String, HoodieLogFile> newLogFiles = new HashMap<>(logFiles);
deltaLogFiles.entrySet().stream().filter(e -> !logFiles.containsKey(e.getKey()))
.forEach(p -> newLogFiles.put(p.getKey(), p.getValue()));
newLogFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf));
log.info("Adding back new File Slice after add FS=" + newFileSlice);
return newFileSlice;
}
case REMOVE: {
log.info("Removing old File Slice =" + fs);
FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
fs.getDataFile().orElseGet(() -> {
oldSlice.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
return null;
});
deltaLogFiles.keySet().stream().forEach(p -> logFiles.remove(p));
//Add remaining log files back
logFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf));
if (newFileSlice.getDataFile().isPresent() || (newFileSlice.getLogFiles().count() > 0)) {
log.info("Adding back new file-slice after remove FS=" + newFileSlice);
return newFileSlice;
}
return null;
}
default:
throw new IllegalStateException("Unknown diff apply mode=" + mode);
}
}
}).filter(Objects::nonNull).forEach(fs -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs);
fs.getDataFile().ifPresent(df -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs),
df);
});
});
});
});
}
@Override
Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations() {
return rocksDB.<Pair<String, CompactionOperation>>prefixSearch(
schemaHelper.getColFamilyForPendingCompaction(), "").map(Pair::getValue);
}
@Override
Stream<HoodieDataFile> fetchAllDataFiles(String partitionPath) {
return rocksDB.<HoodieDataFile>prefixSearch(schemaHelper.getColFamilyForView(),
schemaHelper.getPrefixForDataFileViewByPartition(partitionPath))
.map(Pair::getValue);
}
@Override
Stream<HoodieFileGroup> fetchAllStoredFileGroups(String partitionPath) {
return getFileGroups(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
schemaHelper.getPrefixForSliceViewByPartition(partitionPath)).map(Pair::getValue));
}
@Override
Stream<HoodieFileGroup> fetchAllStoredFileGroups() {
return getFileGroups(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
schemaHelper.getPrefixForSliceView()).map(Pair::getValue));
}
@Override
protected Option<FileSlice> fetchLatestFileSlice(String partitionPath, String fileId) {
// Retries only file-slices of the file and filters for the latest
return Option.ofNullable(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
schemaHelper.getPrefixForSliceViewByPartitionFile(partitionPath, fileId))
.map(Pair::getValue)
.reduce(null, (x, y) ->
((x == null) ? y : (y == null) ? null : HoodieTimeline.compareTimestamps(x.getBaseInstantTime(),
y.getBaseInstantTime(), HoodieTimeline.GREATER) ? x : y)));
}
@Override
protected Option<HoodieDataFile> fetchLatestDataFile(String partitionPath, String fileId) {
// Retries only file-slices of the file and filters for the latest
return Option.ofNullable(rocksDB.<HoodieDataFile>prefixSearch(schemaHelper.getColFamilyForView(),
schemaHelper.getPrefixForDataFileViewByPartitionFile(partitionPath, fileId))
.map(Pair::getValue)
.reduce(null, (x, y) ->
((x == null) ? y : (y == null) ? null : HoodieTimeline.compareTimestamps(x.getCommitTime(),
y.getCommitTime(), HoodieTimeline.GREATER) ? x : y)));
}
@Override
Option<HoodieFileGroup> fetchHoodieFileGroup(String partitionPath, String fileId) {
return Option.fromJavaOptional(
getFileGroups(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
schemaHelper.getPrefixForSliceViewByPartitionFile(partitionPath, fileId))
.map(Pair::getValue)).findFirst());
}
private Stream<HoodieFileGroup> getFileGroups(Stream<FileSlice> sliceStream) {
return sliceStream.map(s -> Pair.of(Pair.of(s.getPartitionPath(), s.getFileId()), s))
.collect(Collectors.groupingBy(Pair::getKey)).entrySet().stream().map(slicePair -> {
HoodieFileGroup fg =
new HoodieFileGroup(slicePair.getKey().getKey(), slicePair.getKey().getValue(),
getVisibleCommitsAndCompactionTimeline());
slicePair.getValue().forEach(e -> fg.addFileSlice(e.getValue()));
return fg;
});
}
private FileSlice getFileSlice(String partitionPath, String fileId, String instantTime) {
String key = schemaHelper.getKeyForSliceView(partitionPath, fileId, instantTime);
return rocksDB.<FileSlice>get(schemaHelper.getColFamilyForView(), key);
}
@Override
public void close() {
closed = true;
rocksDB.close();
}
@Override
boolean isClosed() {
return closed;
}
}

View File

@@ -0,0 +1,111 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.view;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Stream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Table FileSystemView implementation where view is stored in spillable disk using fixed memory
*/
public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
private static Logger log = LogManager.getLogger(SpillableMapBasedFileSystemView.class);
private final long maxMemoryForFileGroupMap;
private final long maxMemoryForPendingCompaction;
private final String baseStoreDir;
public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline, FileSystemViewStorageConfig config) {
super(config.isIncrementalTimelineSyncEnabled());
this.maxMemoryForFileGroupMap = config.getMaxMemoryForFileGroupMap();
this.maxMemoryForPendingCompaction = config.getMaxMemoryForPendingCompaction();
this.baseStoreDir = config.getBaseStoreDir();
init(metaClient, visibleActiveTimeline);
}
public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline, FileStatus[] fileStatuses, FileSystemViewStorageConfig config) {
this(metaClient, visibleActiveTimeline, config);
addFilesToView(fileStatuses);
}
@Override
protected Map<String, List<HoodieFileGroup>> createPartitionToFileGroups() {
try {
log.info("Creating Partition To File groups map using external spillable Map. Max Mem="
+ maxMemoryForFileGroupMap + ", BaseDir=" + baseStoreDir);
new File(baseStoreDir).mkdirs();
return (Map<String, List<HoodieFileGroup>>)
(new ExternalSpillableMap<>(maxMemoryForFileGroupMap, baseStoreDir, new DefaultSizeEstimator(),
new DefaultSizeEstimator<>()));
} catch (IOException e) {
throw new RuntimeException(e);
}
}
protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> createFileIdToPendingCompactionMap(
Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingCompaction) {
try {
log.info("Creating Pending Compaction map using external spillable Map. Max Mem="
+ maxMemoryForPendingCompaction + ", BaseDir=" + baseStoreDir);
new File(baseStoreDir).mkdirs();
Map<HoodieFileGroupId, Pair<String, CompactionOperation>> pendingMap =
new ExternalSpillableMap<>(maxMemoryForPendingCompaction, baseStoreDir, new DefaultSizeEstimator(),
new DefaultSizeEstimator<>());
pendingMap.putAll(fgIdToPendingCompaction);
return pendingMap;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
public Stream<HoodieFileGroup> getAllFileGroups() {
return ((ExternalSpillableMap)partitionToFileGroupsMap).valueStream()
.flatMap(fg -> ((List<HoodieFileGroup>)fg).stream());
}
@Override
Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations() {
return ((ExternalSpillableMap)fgIdToPendingCompaction).valueStream();
}
@Override
public Stream<HoodieFileGroup> fetchAllStoredFileGroups() {
return ((ExternalSpillableMap)partitionToFileGroupsMap).valueStream().flatMap(fg -> {
return ((List<HoodieFileGroup>)fg).stream();
});
}
}

View File

@@ -0,0 +1,221 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.file.FileReader;
import org.apache.avro.file.SeekableByteArrayInput;
import org.apache.avro.file.SeekableInput;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumReader;
import org.apache.avro.io.DatumWriter;
import org.apache.avro.mapred.FsInput;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.avro.specific.SpecificDatumWriter;
import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata;
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
import org.apache.hudi.avro.model.HoodieSavepointPartitionMetadata;
import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.exception.HoodieIOException;
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>> 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(Option.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,
Option<Long> durationInMs, List<HoodieCleanStat> cleanStats) {
ImmutableMap.Builder<String, HoodieCleanPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.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 HoodieRestoreMetadata convertRestoreMetadata(String startRestoreTime,
Option<Long> durationInMs, List<String> commits, Map<String, List<HoodieRollbackStat>> commitToStats) {
ImmutableMap.Builder<String, List<HoodieRollbackMetadata>> commitToStatBuilder = ImmutableMap.builder();
for (Map.Entry<String, List<HoodieRollbackStat>> commitToStat : commitToStats.entrySet()) {
commitToStatBuilder.put(commitToStat.getKey(), Arrays.asList(convertRollbackMetadata(startRestoreTime,
durationInMs, commits, commitToStat.getValue())));
}
return new HoodieRestoreMetadata(startRestoreTime, durationInMs.orElseGet(() -> -1L), commits,
commitToStatBuilder.build());
}
public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime,
Option<Long> durationInMs, List<String> commits, List<HoodieRollbackStat> rollbackStats) {
ImmutableMap.Builder<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.builder();
int totalDeleted = 0;
for (HoodieRollbackStat stat : rollbackStats) {
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.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 Option<byte[]> serializeCompactionPlan(HoodieCompactionPlan compactionWorkload)
throws IOException {
return serializeAvroMetadata(compactionWorkload, HoodieCompactionPlan.class);
}
public static Option<byte[]> serializeCleanMetadata(HoodieCleanMetadata metadata)
throws IOException {
return serializeAvroMetadata(metadata, HoodieCleanMetadata.class);
}
public static Option<byte[]> serializeSavepointMetadata(HoodieSavepointMetadata metadata)
throws IOException {
return serializeAvroMetadata(metadata, HoodieSavepointMetadata.class);
}
public static Option<byte[]> serializeRollbackMetadata(
HoodieRollbackMetadata rollbackMetadata) throws IOException {
return serializeAvroMetadata(rollbackMetadata, HoodieRollbackMetadata.class);
}
public static Option<byte[]> serializeRestoreMetadata(
HoodieRestoreMetadata restoreMetadata) throws IOException {
return serializeAvroMetadata(restoreMetadata, HoodieRestoreMetadata.class);
}
public static <T extends SpecificRecordBase> Option<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 Option.of(baos.toByteArray());
}
public static HoodieCompactionPlan deserializeCompactionPlan(byte[] bytes)
throws IOException {
return deserializeAvroMetadata(bytes, HoodieCompactionPlan.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 <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

@@ -0,0 +1,184 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Helper class to generate compaction plan from FileGroup/FileSlice abstraction
*/
public class CompactionUtils {
private static final Logger LOG = LogManager.getLogger(CompactionUtils.class);
/**
* Generate compaction operation from file-slice
*
* @param partitionPath Partition path
* @param fileSlice File Slice
* @param metricsCaptureFunction Metrics Capture function
* @return Compaction Operation
*/
public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, FileSlice fileSlice,
Option<Function<Pair<String, FileSlice>, Map<String, Double>>> metricsCaptureFunction) {
HoodieCompactionOperation.Builder builder = HoodieCompactionOperation.newBuilder();
builder.setPartitionPath(partitionPath);
builder.setFileId(fileSlice.getFileId());
builder.setBaseInstantTime(fileSlice.getBaseInstantTime());
builder.setDeltaFilePaths(fileSlice.getLogFiles().map(lf -> lf.getPath().toString()).collect(Collectors.toList()));
if (fileSlice.getDataFile().isPresent()) {
builder.setDataFilePath(fileSlice.getDataFile().get().getPath());
}
if (metricsCaptureFunction.isPresent()) {
builder.setMetrics(metricsCaptureFunction.get().apply(Pair.of(partitionPath, fileSlice)));
}
return builder.build();
}
/**
* Generate compaction plan from file-slices
*
* @param partitionFileSlicePairs list of partition file-slice pairs
* @param extraMetadata Extra Metadata
* @param metricsCaptureFunction Metrics Capture function
*/
public static HoodieCompactionPlan buildFromFileSlices(
List<Pair<String, FileSlice>> partitionFileSlicePairs,
Option<Map<String, String>> extraMetadata,
Option<Function<Pair<String, FileSlice>, Map<String, Double>>> metricsCaptureFunction) {
HoodieCompactionPlan.Builder builder = HoodieCompactionPlan.newBuilder();
extraMetadata.ifPresent(m -> builder.setExtraMetadata(m));
builder.setOperations(partitionFileSlicePairs.stream().map(pfPair ->
buildFromFileSlice(pfPair.getKey(), pfPair.getValue(), metricsCaptureFunction)).collect(Collectors.toList()));
return builder.build();
}
/**
* Build Avro generated Compaction operation payload from compaction operation POJO for serialization
*/
public static HoodieCompactionOperation buildHoodieCompactionOperation(CompactionOperation op) {
return HoodieCompactionOperation.newBuilder().setFileId(op.getFileId())
.setBaseInstantTime(op.getBaseInstantTime())
.setPartitionPath(op.getPartitionPath())
.setDataFilePath(op.getDataFilePath().isPresent() ? op.getDataFilePath().get() : null)
.setDeltaFilePaths(op.getDeltaFilePaths())
.setMetrics(op.getMetrics()).build();
}
/**
* Build Compaction operation payload from Avro version for using in Spark executors
*
* @param hc HoodieCompactionOperation
*/
public static CompactionOperation buildCompactionOperation(HoodieCompactionOperation hc) {
return CompactionOperation.convertFromAvroRecordInstance(hc);
}
/**
* Get all pending compaction plans along with their instants
*
* @param metaClient Hoodie Meta Client
*/
public static List<Pair<HoodieInstant, HoodieCompactionPlan>> getAllPendingCompactionPlans(
HoodieTableMetaClient metaClient) {
List<HoodieInstant> pendingCompactionInstants =
metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList());
return pendingCompactionInstants.stream().map(instant -> {
try {
return Pair.of(instant, getCompactionPlan(metaClient, instant.getTimestamp()));
} catch (IOException e) {
throw new HoodieException(e);
}
}).collect(Collectors.toList());
}
public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient,
String compactionInstant) throws IOException {
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
metaClient.getActiveTimeline().getInstantAuxiliaryDetails(
HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
return compactionPlan;
}
/**
* Get all PartitionPath + file-ids with pending Compaction operations and their target compaction instant time
*
* @param metaClient Hoodie Table Meta Client
*/
public static Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> getAllPendingCompactionOperations(
HoodieTableMetaClient metaClient) {
List<Pair<HoodieInstant, HoodieCompactionPlan>> pendingCompactionPlanWithInstants =
getAllPendingCompactionPlans(metaClient);
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToPendingCompactionWithInstantMap =
new HashMap<>();
pendingCompactionPlanWithInstants.stream().flatMap(instantPlanPair -> {
return getPendingCompactionOperations(instantPlanPair.getKey(), instantPlanPair.getValue());
}).forEach(pair -> {
// Defensive check to ensure a single-fileId does not have more than one pending compaction
if (fgIdToPendingCompactionWithInstantMap.containsKey(pair.getKey())) {
String msg = "Hoodie File Id (" + pair.getKey() + ") has more thant 1 pending compactions. Instants: "
+ pair.getValue() + ", " + fgIdToPendingCompactionWithInstantMap.get(pair.getKey());
throw new IllegalStateException(msg);
}
fgIdToPendingCompactionWithInstantMap.put(pair.getKey(), pair.getValue());
});
return fgIdToPendingCompactionWithInstantMap;
}
public static Stream<Pair<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>>> getPendingCompactionOperations(
HoodieInstant instant, HoodieCompactionPlan compactionPlan) {
List<HoodieCompactionOperation> ops = compactionPlan.getOperations();
if (null != ops) {
return ops.stream().map(op -> {
return Pair.of(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()),
Pair.of(instant.getTimestamp(), op));
});
} else {
return Stream.empty();
}
}
/**
* Return all pending compaction instant times
* @return
*/
public static List<HoodieInstant> getPendingCompactionInstantTimes(HoodieTableMetaClient metaClient) {
return metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList());
}
}

View File

@@ -0,0 +1,89 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.fs.Path;
/**
* Ensures file create/delete operation is visible
*/
public interface ConsistencyGuard {
/**
* File Visibility
*/
enum FileVisibility {
APPEAR,
DISAPPEAR,
}
/**
* Wait for file to be listable based on configurable timeout
* @param filePath
* @throws IOException when having trouble listing the path
* @throws TimeoutException when retries exhausted
*/
void waitTillFileAppears(Path filePath) throws IOException, TimeoutException;
/**
* Wait for file to be listable based on configurable timeout
* @param filePath
* @throws IOException when having trouble listing the path
* @throws TimeoutException when retries exhausted
*/
void waitTillFileDisappears(Path filePath) throws IOException, TimeoutException;
/**
* Wait till all passed files belonging to a directory shows up in the listing
*/
void waitTillAllFilesAppear(String dirPath, List<String> files) throws IOException, TimeoutException;
/**
* Wait till all passed files belonging to a directory disappears from listing
*/
void waitTillAllFilesDisappear(String dirPath, List<String> files) throws IOException, TimeoutException;
/**
* Wait Till target visibility is reached
* @param dirPath Directory Path
* @param files Files
* @param targetVisibility Target Visibitlity
* @throws IOException
* @throws TimeoutException
*/
default void waitTill(String dirPath, List<String> files, FileVisibility targetVisibility)
throws IOException, TimeoutException {
switch (targetVisibility) {
case APPEAR: {
waitTillAllFilesAppear(dirPath, files);
break;
}
case DISAPPEAR: {
waitTillAllFilesDisappear(dirPath, files);
break;
}
default:
throw new IllegalStateException("Unknown File Visibility");
}
}
}

View File

@@ -0,0 +1,121 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
import org.apache.hudi.config.DefaultHoodieConfig;
public class ConsistencyGuardConfig extends DefaultHoodieConfig {
private static final String CONSISTENCY_CHECK_ENABLED_PROP = "hoodie.consistency.check.enabled";
private static final String DEFAULT_CONSISTENCY_CHECK_ENABLED = "false";
// time between successive attempts to ensure written data's metadata is consistent on storage
private static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP =
"hoodie.consistency.check.initial_interval_ms";
private static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = 2000L;
// max interval time
private static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = "hoodie.consistency.check.max_interval_ms";
private static long DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS = 300000L;
// maximum number of checks, for consistency of written data. Will wait upto 256 Secs
private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7;
public ConsistencyGuardConfig(Properties props) {
super(props);
}
public static ConsistencyGuardConfig.Builder newBuilder() {
return new Builder();
}
public boolean isConsistencyCheckEnabled() {
return Boolean.parseBoolean(props.getProperty(CONSISTENCY_CHECK_ENABLED_PROP));
}
public int getMaxConsistencyChecks() {
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECKS_PROP));
}
public int getInitialConsistencyCheckIntervalMs() {
return Integer.parseInt(props.getProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
}
public int getMaxConsistencyCheckIntervalMs() {
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
props.load(reader);
return this;
} finally {
reader.close();
}
}
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public Builder withConsistencyCheckEnabled(boolean enabled) {
props.setProperty(CONSISTENCY_CHECK_ENABLED_PROP, String.valueOf(enabled));
return this;
}
public Builder withInitialConsistencyCheckIntervalMs(int initialIntevalMs) {
props.setProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(initialIntevalMs));
return this;
}
public Builder withMaxConsistencyCheckIntervalMs(int maxIntervalMs) {
props.setProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(maxIntervalMs));
return this;
}
public Builder withMaxConsistencyChecks(int maxConsistencyChecks) {
props.setProperty(MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(maxConsistencyChecks));
return this;
}
public ConsistencyGuardConfig build() {
setDefaultOnCondition(props, !props.containsKey(CONSISTENCY_CHECK_ENABLED_PROP),
CONSISTENCY_CHECK_ENABLED_PROP, DEFAULT_CONSISTENCY_CHECK_ENABLED);
setDefaultOnCondition(props, !props.containsKey(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS));
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS));
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECKS_PROP),
MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECKS));
return new ConsistencyGuardConfig(props);
}
}
}

View File

@@ -0,0 +1,114 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.util.HashSet;
import java.util.Set;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* A simplified versions of Apache commons - PropertiesConfiguration, that supports limited field types and hierarchical
* configurations within the same folder as the root file.
*
* Includes denoted by the same include=filename.properties syntax, with relative path from root file's folder. Lines
* beginning with '#' are ignored as comments. Final values for properties are resolved by the order in which they are
* specified in the files, with included files treated as if they are inline.
*
* Note: Not reusing commons-configuration since it has too many conflicting runtime deps.
*/
public class DFSPropertiesConfiguration {
private static volatile Logger log = LogManager.getLogger(DFSPropertiesConfiguration.class);
private final FileSystem fs;
private final Path rootFile;
private final TypedProperties props;
// Keep track of files visited, to detect loops
private final Set<String> visitedFiles;
public DFSPropertiesConfiguration(FileSystem fs, Path rootFile, TypedProperties defaults) {
this.fs = fs;
this.rootFile = rootFile;
this.props = defaults;
this.visitedFiles = new HashSet<>();
visitFile(rootFile);
}
private String[] splitProperty(String line) {
int ind = line.indexOf('=');
String k = line.substring(0, ind).trim();
String v = line.substring(ind + 1).trim();
return new String[]{k, v};
}
private void visitFile(Path file) {
try {
if (visitedFiles.contains(file.getName())) {
throw new IllegalStateException("Loop detected; file " + file + " already referenced");
}
visitedFiles.add(file.getName());
BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(file)));
addProperties(reader);
} catch (IOException ioe) {
log.error("Error reading in properies from dfs", ioe);
throw new IllegalArgumentException("Cannot read properties from dfs", ioe);
}
}
/**
* Add properties from input stream
* @param reader Buffered Reader
* @throws IOException
*/
public void addProperties(BufferedReader reader) throws IOException {
try {
String line;
while ((line = reader.readLine()) != null) {
if (line.startsWith("#") || line.equals("") || !line.contains("=")) {
continue;
}
String[] split = splitProperty(line);
if (line.startsWith("include=") || line.startsWith("include =")) {
visitFile(new Path(rootFile.getParent(), split[1]));
} else {
props.setProperty(split[0], split[1]);
}
}
} finally {
reader.close();
}
}
public DFSPropertiesConfiguration(FileSystem fs, Path rootFile) {
this(fs, rootFile, new TypedProperties());
}
public TypedProperties getConfig() {
return props;
}
}

View File

@@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import com.twitter.common.objectsize.ObjectSizeCalculator;
/**
* Default implementation of size-estimator that uses Twitter's ObjectSizeCalculator
* @param <T>
*/
public class DefaultSizeEstimator<T> implements SizeEstimator<T> {
@Override
public long sizeEstimate(T t) {
return ObjectSizeCalculator.getObjectSize(t);
}
}

View File

@@ -0,0 +1,564 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
import java.util.Map.Entry;
import java.util.UUID;
import java.util.function.Function;
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.FSDataInputStream;
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.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.InvalidHoodiePathException;
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]*)(_(([0-9]*)-([0-9]*)-([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 final String HOODIE_ENV_PROPS_PREFIX = "HOODIE_ENV_";
private static final PathFilter ALLOW_ALL_FILTER = new PathFilter() {
@Override
public boolean accept(Path file) {
return true;
}
};
public static Configuration prepareHadoopConf(Configuration conf) {
conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
// look for all properties, prefixed to be picked up
for (Entry<String, String> prop : System.getenv().entrySet()) {
if (prop.getKey().startsWith(HOODIE_ENV_PROPS_PREFIX)) {
LOG.info("Picking up value for hoodie env var :" + prop.getKey());
conf.set(prop.getKey()
.replace(HOODIE_ENV_PROPS_PREFIX, "")
.replaceAll("_DOT_", "."),
prop.getValue());
}
}
return conf;
}
public static FileSystem getFs(String path, Configuration conf) {
FileSystem fs;
conf = prepareHadoopConf(conf);
try {
fs = new Path(path).getFileSystem(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;
}
/**
* A write token uniquely identifies an attempt at one of the IOHandle operations (Merge/Create/Append)
*/
public static String makeWriteToken(int taskPartitionId, int stageId, long taskAttemptId) {
return String.format("%d-%d-%d", taskPartitionId, stageId, taskAttemptId);
}
public static String makeDataFileName(String commitTime, String writeToken, String fileId) {
return String.format("%s_%s_%s.parquet", fileId, writeToken, commitTime);
}
public static String makeMarkerFile(String commitTime, String writeToken, String fileId) {
return String.format("%s_%s_%s%s", fileId, writeToken, commitTime, HoodieTableMetaClient.MARKER_EXTN);
}
public static String translateMarkerToDataPath(String basePath, String markerPath, String instantTs) {
Preconditions.checkArgument(markerPath.endsWith(HoodieTableMetaClient.MARKER_EXTN));
String markerRootPath = Path.getPathWithoutSchemeAndAuthority(new Path(
String.format("%s/%s/%s", basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, instantTs))).toString();
int begin = markerPath.indexOf(markerRootPath);
Preconditions.checkArgument(begin >= 0, "Not in marker dir. Marker Path=" + markerPath
+ ", Expected Marker Root=" + markerRootPath);
String rPath = markerPath.substring(begin + markerRootPath.length() + 1);
return String.format("%s/%s%s", basePath, rPath.replace(HoodieTableMetaClient.MARKER_EXTN, ""),
HoodieFileFormat.PARQUET.getFileExtension());
}
public static String maskWithoutFileId(String commitTime, int taskPartitionId) {
return String.format("*_%s_%s%s", taskPartitionId, commitTime, HoodieFileFormat.PARQUET.getFileExtension());
}
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> getAllPartitionFoldersThreeLevelsDown(FileSystem fs, String basePath)
throws IOException {
List<String> datePartitions = new ArrayList<>();
// Avoid listing and including any folders under the metafolder
PathFilter filter = getExcludeMetaPathFilter();
FileStatus[] folders = fs.globStatus(new Path(basePath + "/*/*/*"), filter);
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;
}
/**
* Given a base partition and a partition path, return
* relative path of partition path to the base path
*/
public static String getRelativePartitionPath(Path basePath, Path partitionPath) {
basePath = Path.getPathWithoutSchemeAndAuthority(basePath);
partitionPath = Path.getPathWithoutSchemeAndAuthority(partitionPath);
String partitionFullPath = partitionPath.toString();
int partitionStartIndex = partitionFullPath.indexOf(
basePath.getName(),
basePath.getParent() == null ? 0 : basePath.getParent().toString().length());
// Partition-Path could be empty for non-partitioned tables
return partitionStartIndex + basePath.getName().length() == partitionFullPath.length() ? "" :
partitionFullPath.substring(partitionStartIndex + basePath.getName().length() + 1);
}
/**
* Obtain all the partition paths, that are present in this table, denoted by presence of {@link
* HoodiePartitionMetadata#HOODIE_PARTITION_METAFILE}
*/
public static List<String> getAllFoldersWithPartitionMetaFile(FileSystem fs, String basePathStr)
throws IOException {
final Path basePath = new Path(basePathStr);
final List<String> partitions = new ArrayList<>();
processFiles(fs, basePathStr, (locatedFileStatus) -> {
Path filePath = locatedFileStatus.getPath();
if (filePath.getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)) {
partitions.add(getRelativePartitionPath(basePath, filePath.getParent()));
}
return true;
}, true);
return partitions;
}
public static final List<String> getAllDataFilesForMarkers(FileSystem fs, String basePath, String instantTs,
String markerDir) throws IOException {
List<String> dataFiles = new LinkedList<>();
processFiles(fs, markerDir, (status) -> {
String pathStr = status.getPath().toString();
if (pathStr.endsWith(HoodieTableMetaClient.MARKER_EXTN)) {
dataFiles.add(FSUtils.translateMarkerToDataPath(basePath, pathStr, instantTs));
}
return true;
}, false);
return dataFiles;
}
/**
* Recursively processes all files in the base-path. If excludeMetaFolder is set, the meta-folder and all its
* subdirs are skipped
* @param fs File System
* @param basePathStr Base-Path
* @param consumer Callback for processing
* @param excludeMetaFolder Exclude .hoodie folder
* @throws IOException
*/
@VisibleForTesting
static void processFiles(FileSystem fs, String basePathStr,
Function<FileStatus, Boolean> consumer, boolean excludeMetaFolder) throws IOException {
PathFilter pathFilter = excludeMetaFolder ? getExcludeMetaPathFilter() : ALLOW_ALL_FILTER;
FileStatus[] topLevelStatuses = fs.listStatus(new Path(basePathStr));
for (int i = 0; i < topLevelStatuses.length; i++) {
FileStatus child = topLevelStatuses[i];
if (child.isFile()) {
boolean success = consumer.apply(child);
if (!success) {
throw new HoodieException("Failed to process file-status=" + child);
}
} else if (pathFilter.accept(child.getPath())) {
RemoteIterator<LocatedFileStatus> itr = fs.listFiles(child.getPath(), true);
while (itr.hasNext()) {
FileStatus status = itr.next();
boolean success = consumer.apply(status);
if (!success) {
throw new HoodieException("Failed to process file-status=" + status);
}
}
}
}
}
public static List<String> getAllPartitionPaths(FileSystem fs, String basePathStr,
boolean assumeDatePartitioning)
throws IOException {
if (assumeDatePartitioning) {
return getAllPartitionFoldersThreeLevelsDown(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);
}
private static PathFilter getExcludeMetaPathFilter() {
// Avoid listing and including any folders under the metafolder
return (path) -> {
if (path.toString().contains(HoodieTableMetaClient.METAFOLDER_NAME)) {
return false;
}
return true;
};
}
public static String getInstantTime(String name) {
return name.replace(getFileExtension(name), "");
}
/**
* Returns a new unique prefix for creating a file group.
*/
public static String createNewFileIdPfx() {
return UUID.randomUUID().toString();
}
/**
* 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);
}
/**
* Check if the file is a parquet file of a log file. Then get the fileId appropriately.
*/
public static String getFileIdFromFilePath(Path filePath) {
if (FSUtils.isLogFile(filePath)) {
return FSUtils.getFileIdFromLogPath(filePath);
}
return FSUtils.getFileId(filePath.getName());
}
/**
* 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 TaskId used in log-path
*/
public static Integer getTaskPartitionIdFromLogPath(Path path) {
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(path, "LogFile");
}
String val = matcher.group(7);
return val == null ? null : Integer.parseInt(val);
}
/**
* Get Write-Token used in log-path
*/
public static String getWriteTokenFromLogPath(Path path) {
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(path, "LogFile");
}
return matcher.group(6);
}
/**
* Get StageId used in log-path
*/
public static Integer getStageIdFromLogPath(Path path) {
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(path, "LogFile");
}
String val = matcher.group(8);
return val == null ? null : Integer.parseInt(val);
}
/**
* Get Task Attempt Id used in log-path
*/
public static Integer getTaskAttemptIdFromLogPath(Path path) {
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
if (!matcher.find()) {
throw new InvalidHoodiePathException(path, "LogFile");
}
String val = matcher.group(9);
return val == null ? null : Integer.parseInt(val);
}
/**
* 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, String writeToken) {
String suffix = (writeToken == null) ? String.format("%s_%s%s.%d",fileId, baseCommitTime, logFileExtension, version)
: String.format("%s_%s%s.%d_%s", fileId, baseCommitTime, logFileExtension, version, writeToken);
return LOG_FILE_PREFIX + suffix;
}
public static boolean isLogFile(Path logPath) {
Matcher matcher = LOG_FILE_PATTERN.matcher(logPath.getName());
if (!matcher.find()) {
return false;
}
return true;
}
/**
* Get the latest log file written from the list of log files passed in
*/
public static Option<HoodieLogFile> getLatestLogFile(Stream<HoodieLogFile> logFiles) {
return Option.fromJavaOptional(logFiles.sorted(HoodieLogFile.getReverseLogFileComparator()).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 Option<Pair<Integer, String>> getLatestLogVersion(FileSystem fs, Path partitionPath,
final String fileId, final String logFileExtension, final String baseCommitTime)
throws IOException {
Option<HoodieLogFile> latestLogFile =
getLatestLogFile(
getAllLogFiles(fs, partitionPath, fileId, logFileExtension, baseCommitTime));
if (latestLogFile.isPresent()) {
return Option.of(Pair.of(latestLogFile.get().getLogVersion(),
getWriteTokenFromLogPath(latestLogFile.get().getPath())));
}
return Option.empty();
}
/**
* 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 {
Option<Pair<Integer, String>> currentVersionWithWriteToken =
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
// handle potential overflow
return (currentVersionWithWriteToken.isPresent()) ? currentVersionWithWriteToken.get().getKey() + 1
: HoodieLogFile.LOGFILE_BASE_VERSION;
}
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);
}
/**
* 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 deleteOlderRestoreMetaFiles(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 restore 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 Path getPartitionPath(String basePath, String partitionPath) {
return getPartitionPath(new Path(basePath), partitionPath);
}
public static Path getPartitionPath(Path basePath, String partitionPath) {
// FOr non-partitioned table, return only base-path
return ((partitionPath == null) || (partitionPath.isEmpty())) ? basePath :
new Path(basePath, partitionPath);
}
/**
* This is due to HUDI-140 GCS has a different behavior for detecting EOF during seek().
* @param inputStream FSDataInputStream
* @return true if the inputstream or the wrapped one is of type GoogleHadoopFSInputStream
*/
public static boolean isGCSInputStream(FSDataInputStream inputStream) {
return inputStream.getClass().getCanonicalName().equals("com.google.cloud.hadoop.fs.gcs.GoogleHadoopFSInputStream")
|| inputStream.getWrappedStream().getClass().getCanonicalName()
.equals("com.google.cloud.hadoop.fs.gcs.GoogleHadoopFSInputStream");
}
}

View File

@@ -0,0 +1,198 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import com.google.common.base.Preconditions;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeoutException;
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* A consistency checker that fails if it is unable to meet the required condition within a specified timeout
*/
public class FailSafeConsistencyGuard implements ConsistencyGuard {
private static final transient Logger log = LogManager.getLogger(FailSafeConsistencyGuard.class);
private final FileSystem fs;
private final ConsistencyGuardConfig consistencyGuardConfig;
public FailSafeConsistencyGuard(FileSystem fs, ConsistencyGuardConfig consistencyGuardConfig) {
this.fs = fs;
this.consistencyGuardConfig = consistencyGuardConfig;
Preconditions.checkArgument(consistencyGuardConfig.isConsistencyCheckEnabled());
}
@Override
public void waitTillFileAppears(Path filePath) throws TimeoutException {
waitForFileVisibility(filePath, FileVisibility.APPEAR);
}
@Override
public void waitTillFileDisappears(Path filePath)
throws TimeoutException {
waitForFileVisibility(filePath, FileVisibility.DISAPPEAR);
}
@Override
public void waitTillAllFilesAppear(String dirPath, List<String> files) throws TimeoutException {
waitForFilesVisibility(dirPath, files, FileVisibility.APPEAR);
}
@Override
public void waitTillAllFilesDisappear(String dirPath, List<String> files) throws TimeoutException {
waitForFilesVisibility(dirPath, files, FileVisibility.DISAPPEAR);
}
/**
* Helper function to wait for all files belonging to single directory to appear
* @param dirPath Dir Path
* @param files Files to appear/disappear
* @param event Appear/Disappear
* @throws TimeoutException
*/
public void waitForFilesVisibility(String dirPath, List<String> files, FileVisibility event)
throws TimeoutException {
Path dir = new Path(dirPath);
List<String> filesWithoutSchemeAndAuthority =
files.stream().map(f -> Path.getPathWithoutSchemeAndAuthority(new Path(f))).map(p -> p.toString())
.collect(Collectors.toList());
retryTillSuccess((retryNum) -> {
try {
log.info("Trying " + retryNum);
FileStatus[] entries = fs.listStatus(dir);
List<String> gotFiles = Arrays.stream(entries).map(e -> Path.getPathWithoutSchemeAndAuthority(e.getPath()))
.map(p -> p.toString()).collect(Collectors.toList());
List<String> candidateFiles = new ArrayList<>(filesWithoutSchemeAndAuthority);
boolean altered = candidateFiles.removeAll(gotFiles);
switch (event) {
case DISAPPEAR:
log.info("Following files are visible" + candidateFiles);
// If no candidate files gets removed, it means all of them have disappeared
return !altered;
case APPEAR:
default:
// if all files appear, the list is empty
return candidateFiles.isEmpty();
}
} catch (IOException ioe) {
log.warn("Got IOException waiting for file event. Have tried " + retryNum + " time(s)", ioe);
}
return false;
}, "Timed out waiting for filles to become visible");
}
/**
* Helper to check of file visibility
* @param filePath File Path
* @param visibility Visibility
* @return
* @throws IOException
*/
private boolean checkFileVisibility(Path filePath, FileVisibility visibility) throws IOException {
try {
FileStatus status = fs.getFileStatus(filePath);
switch (visibility) {
case APPEAR:
return status != null;
case DISAPPEAR:
default:
return status == null;
}
} catch (FileNotFoundException nfe) {
switch (visibility) {
case APPEAR:
return false;
case DISAPPEAR:
default:
return true;
}
}
}
/**
* Helper function to wait till file either appears/disappears
* @param filePath File Path
* @param visibility
* @throws TimeoutException
*/
private void waitForFileVisibility(Path filePath, FileVisibility visibility) throws TimeoutException {
long waitMs = consistencyGuardConfig.getInitialConsistencyCheckIntervalMs();
int attempt = 0;
while (attempt < consistencyGuardConfig.getMaxConsistencyChecks()) {
try {
if (checkFileVisibility(filePath, visibility)) {
return;
}
} catch (IOException ioe) {
log.warn("Got IOException waiting for file visibility. Retrying", ioe);
}
sleepSafe(waitMs);
waitMs = waitMs * 2; // double check interval every attempt
waitMs = Math.min(waitMs, consistencyGuardConfig.getMaxConsistencyCheckIntervalMs());
attempt++;
}
throw new TimeoutException("Timed-out waiting for the file to " + visibility.name());
}
/**
* Retries the predicate for condfigurable number of times till we the predicate returns success
* @param predicate Predicate Function
* @param timedOutMessage Timed-Out message for logging
* @throws TimeoutException when retries are exhausted
*/
private void retryTillSuccess(Function<Integer, Boolean> predicate, String timedOutMessage) throws TimeoutException {
long waitMs = consistencyGuardConfig.getInitialConsistencyCheckIntervalMs();
int attempt = 0;
log.info("Max Attempts=" + consistencyGuardConfig.getMaxConsistencyChecks());
while (attempt < consistencyGuardConfig.getMaxConsistencyChecks()) {
boolean success = predicate.apply(attempt);
if (success) {
return;
}
sleepSafe(waitMs);
waitMs = waitMs * 2; // double check interval every attempt
waitMs = Math.min(waitMs, consistencyGuardConfig.getMaxConsistencyCheckIntervalMs());
attempt++;
}
throw new TimeoutException(timedOutMessage);
}
void sleepSafe(long waitMs) {
try {
Thread.sleep(waitMs);
} catch (InterruptedException e) {
// ignore & continue next attempt
}
}
}

View File

@@ -0,0 +1,40 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import java.io.Serializable;
public interface Functions {
public interface Function0<R> extends Serializable {
R apply();
}
public interface Function1<T1, R> extends Serializable {
R apply(T1 val1);
}
public interface Function2<T1, T2, R> extends Serializable {
R apply(T1 val1, T2 val2);
}
public interface Function3<T1, T2, T3, R> extends Serializable {
R apply(T1 val1, T2 val2, T3 val3);
}
}

View File

@@ -0,0 +1,242 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
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.Map;
import java.util.stream.Collectors;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
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.BinaryDecoder;
import org.apache.avro.io.BinaryEncoder;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.EncoderFactory;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.SchemaCompatabilityException;
import org.codehaus.jackson.JsonNode;
import org.codehaus.jackson.node.NullNode;
/**
* Helper class to do common stuff across Avro.
*/
public class HoodieAvroUtils {
private static ThreadLocal<BinaryEncoder> reuseEncoder = ThreadLocal.withInitial(() -> null);
private static ThreadLocal<BinaryDecoder> reuseDecoder = ThreadLocal.withInitial(() -> null);
// All metadata fields are optional strings.
private static final Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList(
Schema.create(Schema.Type.NULL),
Schema.create(Schema.Type.STRING)));
private static final 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, reuseEncoder.get());
reuseEncoder.set(encoder);
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 {
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(bytes, reuseDecoder.get());
reuseDecoder.set(decoder);
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<GenericRecord>(schema);
return reader.read(null, decoder);
}
public static boolean isMetadataField(String fieldName) {
return HoodieRecord.COMMIT_TIME_METADATA_FIELD.equals(fieldName)
|| HoodieRecord.COMMIT_SEQNO_METADATA_FIELD.equals(fieldName)
|| HoodieRecord.RECORD_KEY_METADATA_FIELD.equals(fieldName)
|| HoodieRecord.PARTITION_PATH_METADATA_FIELD.equals(fieldName)
|| HoodieRecord.FILENAME_METADATA_FIELD.equals(fieldName);
}
/**
* 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, "", NullNode.getInstance());
Schema.Field commitSeqnoField = new Schema.Field(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD,
METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
Schema.Field recordKeyField = new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD,
METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
Schema.Field partitionPathField = new Schema.Field(HoodieRecord.PARTITION_PATH_METADATA_FIELD,
METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
Schema.Field fileNameField = new Schema.Field(HoodieRecord.FILENAME_METADATA_FIELD,
METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
parentFields.add(commitTimeField);
parentFields.add(commitSeqnoField);
parentFields.add(recordKeyField);
parentFields.add(partitionPathField);
parentFields.add(fileNameField);
for (Schema.Field field : schema.getFields()) {
if (!isMetadataField(field.name())) {
Schema.Field newField = new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue());
for (Map.Entry<String, JsonNode> prop : field.getJsonProps().entrySet()) {
newField.addProp(prop.getKey(), prop.getValue());
}
parentFields.add(newField);
}
}
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, "", NullNode.getInstance());
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;
}
/**
* Add null fields to passed in schema. Caller is responsible for ensuring there is no duplicates.
* As different query engines have varying constraints regarding treating the case-sensitivity of fields, its best
* to let caller determine that.
*
* @param schema Passed in schema
* @param newFieldNames Null Field names to be added
*/
public static Schema appendNullSchemaFields(Schema schema, List<String> newFieldNames) {
List<Field> newFields = schema.getFields().stream().map(field -> {
return new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue());
}).collect(Collectors.toList());
for (String newField : newFieldNames) {
newFields.add(new Schema.Field(newField, METADATA_FIELD_SCHEMA, "", NullNode.getInstance()));
}
Schema newSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), schema.isError());
newSchema.setFields(newFields);
return newSchema;
}
/**
* 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 while setting fields only from the old
* schema
*/
public static GenericRecord rewriteRecord(GenericRecord record, Schema newSchema) {
return rewrite(record, record.getSchema(), newSchema);
}
/**
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the new
* schema
*/
public static GenericRecord rewriteRecordWithOnlyNewSchemaFields(GenericRecord record, Schema newSchema) {
return rewrite(record, newSchema, newSchema);
}
private static GenericRecord rewrite(GenericRecord record, Schema schemaWithFields, Schema newSchema) {
GenericRecord newRecord = new GenericData.Record(newSchema);
for (Schema.Field f : schemaWithFields.getFields()) {
newRecord.put(f.name(), record.get(f.name()));
}
if (!GenericData.get().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);
}
}
}

View File

@@ -0,0 +1,55 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import com.twitter.common.objectsize.ObjectSizeCalculator;
import org.apache.avro.Schema;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Size Estimator for Hoodie record payload
* @param <T>
*/
public class HoodieRecordSizeEstimator<T extends HoodieRecordPayload> implements SizeEstimator<HoodieRecord<T>> {
private static Logger log = LogManager.getLogger(HoodieRecordSizeEstimator.class);
// Schema used to get GenericRecord from HoodieRecordPayload then convert to bytes and vice-versa
private final Schema schema;
public HoodieRecordSizeEstimator(Schema schema) {
this.schema = schema;
}
@Override
public long sizeEstimate(HoodieRecord<T> hoodieRecord) {
// Most HoodieRecords are bound to have data + schema. Although, the same schema object is shared amongst
// all records in the JVM. Calculate and print the size of the Schema and of the Record to
// note the sizes and differences. A correct estimation in such cases is handled in
/** {@link ExternalSpillableMap} **/
long sizeOfRecord = ObjectSizeCalculator.getObjectSize(hoodieRecord);
long sizeOfSchema = ObjectSizeCalculator.getObjectSize(schema);
log.info("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema);
return sizeOfRecord;
}
}

View File

@@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import java.util.ArrayDeque;
import java.util.Deque;
import org.apache.hudi.exception.HoodieException;
/**
* Timing utility to help keep track of execution times of code blocks. This class helps to allow multiple
* timers started at the same time and automatically returns the execution time in the order in which the
* timers are stopped.
*/
public class HoodieTimer {
// Ordered stack of TimeInfo's to make sure stopping the timer returns the correct elapsed time
Deque<TimeInfo> timeInfoDeque = new ArrayDeque<>();
class TimeInfo {
// captures the startTime of the code block
long startTime;
// is the timing still running for the last started timer
boolean isRunning;
public TimeInfo(long startTime) {
this.startTime = startTime;
this.isRunning = true;
}
public long getStartTime() {
return startTime;
}
public boolean isRunning() {
return isRunning;
}
public long stop() {
this.isRunning = false;
return System.currentTimeMillis() - startTime;
}
}
public HoodieTimer startTimer() {
timeInfoDeque.push(new TimeInfo(System.currentTimeMillis()));
return this;
}
public long endTimer() {
if (timeInfoDeque.isEmpty()) {
throw new HoodieException("Timer was not started");
}
return timeInfoDeque.pop().stop();
}
}

Some files were not shown because too many files have changed in this diff Show More