1
0

[HUDI-242] Support for RFC-12/Bootstrapping of external datasets to hudi (#1876)

- [HUDI-418] Bootstrap Index Implementation using HFile with unit-test
 - [HUDI-421] FileSystem View Changes to support Bootstrap with unit-tests
 - [HUDI-424] Implement Query Side Integration for querying tables containing bootstrap file slices
 - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices
 - [HUDI-421] Bootstrap Write Client with tests
 - [HUDI-425] Added HoodieDeltaStreamer support
 - [HUDI-899] Add a knob to change partition-path style while performing metadata bootstrap
 - [HUDI-900] Metadata Bootstrap Key Generator needs to handle complex keys correctly
 - [HUDI-424] Simplify Record reader implementation
 - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices
 - [HUDI-420] Hoodie Demo working with hive and sparkSQL. Also, Hoodie CLI working with bootstrap tables

Co-authored-by: Mehrotra <uditme@amazon.com>
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
Co-authored-by: Balaji Varadarajan <varadarb@uber.com>
This commit is contained in:
vinoth chandar
2020-08-03 20:19:21 -07:00
committed by GitHub
parent 266bce12b3
commit 539621bd33
175 changed files with 7540 additions and 779 deletions

View File

@@ -0,0 +1,44 @@
/*
* 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.
*/
{
"namespace":"org.apache.hudi.avro.model",
"type":"record",
"name":"HoodieBootstrapFilePartitionInfo",
"fields":[
{
"name":"version",
"type":["int", "null"],
"default": 1
},
{
"name":"bootstrapPartitionPath",
"type":["null", "string"],
"default" : null
},
{
"name":"bootstrapFileStatus",
"type":["null", "HoodieFileStatus"],
"default" : null
},
{
"name":"partitionPath",
"type":["null", "string"],
"default" : null
}
]
}

View File

@@ -0,0 +1,44 @@
/*
* 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.
*/
{
"namespace":"org.apache.hudi.avro.model",
"type":"record",
"name":"HoodieBootstrapIndexInfo",
"fields":[
{
"name":"version",
"type":["int", "null"],
"default": 1
},
{
"name":"bootstrapBasePath",
"type":["null", "string"],
"default" : null
},
{
"name":"createdTimestamp",
"type":["null", "long"],
"default" : null
},
{
"name":"numKeys",
"type":["null", "int"],
"default" : null
}
]
}

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.
*/
{
"namespace":"org.apache.hudi.avro.model",
"type":"record",
"name":"HoodieBootstrapPartitionMetadata",
"fields":[
{
"name":"version",
"type":["int", "null"],
"default": 1
},
{
"name":"bootstrapPartitionPath",
"type":["null", "string"],
"default" : null
},
{
"name":"partitionPath",
"type":["null", "string"],
"default" : null
},
{
"name":"fileIdToBootstrapFile",
"type":["null", {
"type":"map",
"values": "HoodieFileStatus"
}],
"default": null
}
]
}

View File

@@ -61,6 +61,11 @@
"values":"double"
}],
"default": null
},
{
"name":"bootstrapFilePath",
"type":["null", "string"],
"default": null
}
]
}

View File

@@ -0,0 +1,49 @@
/*
* 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.
*/
{
"namespace":"org.apache.hudi.avro.model",
"type":"record",
"name":"HoodieFSPermission",
"fields":[
{
"name":"version",
"type":["int", "null"],
"default": 1
},
{
"name":"userAction",
"type":[ "null", "string" ],
"default": "null"
},
{
"name":"groupAction",
"type":[ "null", "string" ],
"default": "null"
},
{
"name":"otherAction",
"type":[ "null", "string" ],
"default": "null"
},
{
"name":"stickyBit",
"type":[ "null", "boolean" ],
"default": "null"
}
]
}

View File

@@ -0,0 +1,84 @@
/*
* 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.
*/
{
"namespace":"org.apache.hudi.avro.model",
"type":"record",
"name":"HoodieFileStatus",
"fields":[
{
"name":"version",
"type":["int", "null"],
"default": 1
},
{
"name":"path",
"type":["null", "HoodiePath"],
"default" : null
},
{
"name":"length",
"type":["null", "long"],
"default" : null
},
{
"name":"isDir",
"type":["null", "boolean"],
"default" : null
},
{
"name":"blockReplication",
"type":["null", "int"],
"default" : null
},
{
"name":"blockSize",
"type":["null", "long"],
"default" : null
},
{
"name":"modificationTime",
"type":["null", "long"],
"default" : null
},
{
"name":"accessTime",
"type":["null", "long"],
"default" : null
},
{
"name":"permission",
"type":["null", "HoodieFSPermission"],
"default" : null
},
{
"name":"owner",
"type":["null", "string"],
"default" : null
},
{
"name":"group",
"type":["null", "string"],
"default" : null
},
{
"name":"symlink",
"type":["null", "HoodiePath"],
"default" : null
}
]
}

View File

@@ -0,0 +1,34 @@
/*
* 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.
*/
{
"namespace":"org.apache.hudi.avro.model",
"type":"record",
"name":"HoodiePath",
"fields":[
{
"name":"version",
"type":["int", "null"],
"default": 1
},
{
"name":"uri",
"type":["null", "string"],
"default" : null
}
]
}

View File

@@ -19,7 +19,13 @@
package org.apache.hudi.avro;
import org.apache.avro.JsonProperties;
import java.time.LocalDate;
import org.apache.avro.LogicalTypes;
import org.apache.avro.generic.GenericData.Record;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.StringUtils;
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.SchemaCompatabilityException;
@@ -64,10 +70,10 @@ public class HoodieAvroUtils {
private static ThreadLocal<BinaryDecoder> reuseDecoder = ThreadLocal.withInitial(() -> null);
// All metadata fields are optional strings.
static final Schema METADATA_FIELD_SCHEMA =
public 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();
public static final Schema RECORD_KEY_SCHEMA = initRecordKeySchema();
/**
* Convert a given avro record to bytes.
@@ -251,6 +257,17 @@ public class HoodieAvroUtils {
return record;
}
public static GenericRecord stitchRecords(GenericRecord left, GenericRecord right, Schema stitchedSchema) {
GenericRecord result = new Record(stitchedSchema);
for (Schema.Field f : left.getSchema().getFields()) {
result.put(f.name(), left.get(f.name()));
}
for (Schema.Field f : right.getSchema().getFields()) {
result.put(f.name(), right.get(f.name()));
}
return result;
}
/**
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the old
* schema.
@@ -326,4 +343,105 @@ public class HoodieAvroUtils {
throw new HoodieIOException("IOException while decompressing text", e);
}
}
/**
* Generate a reader schema off the provided writeSchema, to just project out the provided columns.
*/
public static Schema generateProjectionSchema(Schema originalSchema, List<String> fieldNames) {
Map<String, Field> schemaFieldsMap = originalSchema.getFields().stream()
.map(r -> Pair.of(r.name().toLowerCase(), r)).collect(Collectors.toMap(Pair::getLeft, Pair::getRight));
List<Schema.Field> projectedFields = new ArrayList<>();
for (String fn : fieldNames) {
Schema.Field field = schemaFieldsMap.get(fn.toLowerCase());
if (field == null) {
throw new HoodieException("Field " + fn + " not found in log schema. Query cannot proceed! "
+ "Derived Schema Fields: " + new ArrayList<>(schemaFieldsMap.keySet()));
} else {
projectedFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue()));
}
}
Schema projectedSchema = Schema.createRecord(originalSchema.getName(), originalSchema.getDoc(),
originalSchema.getNamespace(), originalSchema.isError());
projectedSchema.setFields(projectedFields);
return projectedSchema;
}
/**
* Obtain value of the provided field as string, denoted by dot notation. e.g: a.b.c
*/
public static String getNestedFieldValAsString(GenericRecord record, String fieldName, boolean returnNullIfNotFound) {
Object obj = getNestedFieldVal(record, fieldName, returnNullIfNotFound);
return StringUtils.objToString(obj);
}
/**
* Obtain value of the provided field, denoted by dot notation. e.g: a.b.c
*/
public static Object getNestedFieldVal(GenericRecord record, String fieldName, boolean returnNullIfNotFound) {
String[] parts = fieldName.split("\\.");
GenericRecord valueNode = record;
int i = 0;
for (; i < parts.length; i++) {
String part = parts[i];
Object val = valueNode.get(part);
if (val == null) {
break;
}
// return, if last part of name
if (i == parts.length - 1) {
Schema fieldSchema = valueNode.getSchema().getField(part).schema();
return convertValueForSpecificDataTypes(fieldSchema, val);
} else {
// VC: Need a test here
if (!(val instanceof GenericRecord)) {
throw new HoodieException("Cannot find a record at part value :" + part);
}
valueNode = (GenericRecord) val;
}
}
if (returnNullIfNotFound) {
return null;
} else {
throw new HoodieException(
fieldName + "(Part -" + parts[i] + ") field not found in record. Acceptable fields were :"
+ valueNode.getSchema().getFields().stream().map(Field::name).collect(Collectors.toList()));
}
}
/**
* This method converts values for fields with certain Avro/Parquet data types that require special handling.
*
* Logical Date Type is converted to actual Date value instead of Epoch Integer which is how it is
* represented/stored in parquet.
*
* @param fieldSchema avro field schema
* @param fieldValue avro field value
* @return field value either converted (for certain data types) or as it is.
*/
private static Object convertValueForSpecificDataTypes(Schema fieldSchema, Object fieldValue) {
if (fieldSchema == null) {
return fieldValue;
}
if (isLogicalTypeDate(fieldSchema)) {
return LocalDate.ofEpochDay(Long.parseLong(fieldValue.toString()));
}
return fieldValue;
}
/**
* Given an Avro field schema checks whether the field is of Logical Date Type or not.
*
* @param fieldSchema avro field schema
* @return boolean indicating whether fieldSchema is of Avro's Date Logical Type
*/
private static boolean isLogicalTypeDate(Schema fieldSchema) {
if (fieldSchema.getType() == Schema.Type.UNION) {
return fieldSchema.getTypes().stream().anyMatch(schema -> schema.getLogicalType() == LogicalTypes.date());
}
return fieldSchema.getLogicalType() == LogicalTypes.date();
}
}

View File

@@ -0,0 +1,123 @@
/*
* 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.bootstrap;
import org.apache.hudi.avro.model.HoodieFSPermission;
import org.apache.hudi.avro.model.HoodieFileStatus;
import org.apache.hudi.avro.model.HoodiePath;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import java.io.IOException;
/**
* Helper functions around FileStatus and HoodieFileStatus.
*/
public class FileStatusUtils {
public static Path toPath(HoodiePath path) {
if (null == path) {
return null;
}
return new Path(path.getUri());
}
public static HoodiePath fromPath(Path path) {
if (null == path) {
return null;
}
return HoodiePath.newBuilder().setUri(path.toString()).build();
}
public static FsPermission toFSPermission(HoodieFSPermission fsPermission) {
if (null == fsPermission) {
return null;
}
FsAction userAction = fsPermission.getUserAction() != null ? FsAction.valueOf(fsPermission.getUserAction()) : null;
FsAction grpAction = fsPermission.getGroupAction() != null ? FsAction.valueOf(fsPermission.getGroupAction()) : null;
FsAction otherAction =
fsPermission.getOtherAction() != null ? FsAction.valueOf(fsPermission.getOtherAction()) : null;
boolean stickyBit = fsPermission.getStickyBit() != null ? fsPermission.getStickyBit() : false;
return new FsPermission(userAction, grpAction, otherAction, stickyBit);
}
public static HoodieFSPermission fromFSPermission(FsPermission fsPermission) {
if (null == fsPermission) {
return null;
}
String userAction = fsPermission.getUserAction() != null ? fsPermission.getUserAction().name() : null;
String grpAction = fsPermission.getGroupAction() != null ? fsPermission.getGroupAction().name() : null;
String otherAction = fsPermission.getOtherAction() != null ? fsPermission.getOtherAction().name() : null;
return HoodieFSPermission.newBuilder().setUserAction(userAction).setGroupAction(grpAction)
.setOtherAction(otherAction).setStickyBit(fsPermission.getStickyBit()).build();
}
public static FileStatus toFileStatus(HoodieFileStatus fileStatus) {
if (null == fileStatus) {
return null;
}
return new FileStatus(fileStatus.getLength(), fileStatus.getIsDir() == null ? false : fileStatus.getIsDir(),
fileStatus.getBlockReplication(), fileStatus.getBlockSize(), fileStatus.getModificationTime(),
fileStatus.getAccessTime(), toFSPermission(fileStatus.getPermission()), fileStatus.getOwner(),
fileStatus.getGroup(), toPath(fileStatus.getSymlink()), toPath(fileStatus.getPath()));
}
public static HoodieFileStatus fromFileStatus(FileStatus fileStatus) {
if (null == fileStatus) {
return null;
}
HoodieFileStatus fStatus = new HoodieFileStatus();
try {
fStatus.setPath(fromPath(fileStatus.getPath()));
fStatus.setLength(fileStatus.getLen());
fStatus.setIsDir(fileStatus.isDirectory());
fStatus.setBlockReplication((int) fileStatus.getReplication());
fStatus.setBlockSize(fileStatus.getBlockSize());
fStatus.setModificationTime(fileStatus.getModificationTime());
fStatus.setAccessTime(fileStatus.getModificationTime());
fStatus.setSymlink(fileStatus.isSymlink() ? fromPath(fileStatus.getSymlink()) : null);
safeReadAndSetMetadata(fStatus, fileStatus);
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
return fStatus;
}
/**
* Used to safely handle FileStatus calls which might fail on some FileSystem implementation.
* (DeprecatedLocalFileSystem)
*/
private static void safeReadAndSetMetadata(HoodieFileStatus fStatus, FileStatus fileStatus) {
try {
fStatus.setOwner(fileStatus.getOwner());
fStatus.setGroup(fileStatus.getGroup());
fStatus.setPermission(fromFSPermission(fileStatus.getPermission()));
} catch (IllegalArgumentException ie) {
// Deprecated File System (testing) does not work well with this call
// skipping
}
}
}

View File

@@ -0,0 +1,161 @@
/*
* 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.bootstrap.index;
import org.apache.hudi.common.model.BootstrapFileMapping;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import java.io.Serializable;
import java.util.List;
import java.util.Map;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.ReflectionUtils;
/**
* Bootstrap Index Interface.
*/
public abstract class BootstrapIndex implements Serializable {
protected static final long serialVersionUID = 1L;
protected final HoodieTableMetaClient metaClient;
public BootstrapIndex(HoodieTableMetaClient metaClient) {
this.metaClient = metaClient;
}
/**
* Create Bootstrap Index Reader.
* @return Index Reader
*/
public abstract IndexReader createReader();
/**
* Create Bootstrap Index Writer.
* @param sourceBasePath Source Base Path
* @return Index Writer
*/
public abstract IndexWriter createWriter(String sourceBasePath);
/**
* Drop bootstrap index.
*/
public abstract void dropIndex();
/**
* Returns true if valid metadata bootstrap is present.
* @return
*/
public final boolean useIndex() {
boolean validInstantTime = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant()
.map(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS,
HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS)).orElse(false);
return validInstantTime && metaClient.getTableConfig().getBootstrapBasePath().isPresent() && isPresent();
}
/**
* Check if bootstrap Index is present and ensures readable.
*/
protected abstract boolean isPresent();
/**
* Bootstrap Index Reader Interface.
*/
public abstract static class IndexReader implements Serializable, AutoCloseable {
protected final HoodieTableMetaClient metaClient;
public IndexReader(HoodieTableMetaClient metaClient) {
this.metaClient = metaClient;
}
/**
* Return Source base path.
* @return
*/
public abstract String getBootstrapBasePath();
/**
* Return list of partitions indexed.
* @return
*/
public abstract List<String> getIndexedPartitionPaths();
/**
* Return list file-ids indexed.
* @return
*/
public abstract List<String> getIndexedFileIds();
/**
* Lookup bootstrap index by partition.
* @param partition Partition to lookup
* @return
*/
public abstract List<BootstrapFileMapping> getSourceFileMappingForPartition(String partition);
/**
* Lookup Bootstrap index by file group ids.
* @param ids File Group Ids
* @return
*/
public abstract Map<HoodieFileGroupId, BootstrapFileMapping> getSourceFileMappingForFileIds(
List<HoodieFileGroupId> ids);
public abstract void close();
}
/**
* Bootstrap Index Writer Interface.
*/
public abstract static class IndexWriter implements AutoCloseable {
protected final HoodieTableMetaClient metaClient;
public IndexWriter(HoodieTableMetaClient metaClient) {
this.metaClient = metaClient;
}
/**
* Writer calls this method before beginning indexing partitions.
*/
public abstract void begin();
/**
* Append bootstrap index entries for next partitions in sorted order.
* @param partitionPath Partition Path
* @param bootstrapFileMappings Bootstrap Source File to File Id mapping
*/
public abstract void appendNextPartition(String partitionPath,
List<BootstrapFileMapping> bootstrapFileMappings);
/**
* Writer calls this method after appending all partitions to be indexed.
*/
public abstract void finish();
public abstract void close();
}
public static BootstrapIndex getBootstrapIndex(HoodieTableMetaClient metaClient) {
return ((BootstrapIndex)(ReflectionUtils.loadClass(
metaClient.getTableConfig().getBootstrapIndexClass(), metaClient)));
}
}

View File

@@ -0,0 +1,534 @@
/*
* 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.bootstrap.index;
import org.apache.hudi.avro.model.HoodieBootstrapFilePartitionInfo;
import org.apache.hudi.avro.model.HoodieBootstrapIndexInfo;
import org.apache.hudi.avro.model.HoodieBootstrapPartitionMetadata;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.BootstrapFileMapping;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* Maintains mapping from skeleton file id to external bootstrap file.
* It maintains 2 physical indices.
* (a) At partition granularity to lookup all indices for each partition.
* (b) At file-group granularity to lookup bootstrap mapping for an individual file-group.
*
* This implementation uses HFile as physical storage of index. FOr the initial run, bootstrap
* mapping for the entire dataset resides in a single file but care has been taken in naming
* the index files in the same way as Hudi data files so that we can reuse file-system abstraction
* on these index files to manage multiple file-groups.
*/
public class HFileBootstrapIndex extends BootstrapIndex {
protected static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(HFileBootstrapIndex.class);
public static final String BOOTSTRAP_INDEX_FILE_ID = "00000000-0000-0000-0000-000000000000-0";
// Additional Metadata written to HFiles.
public static final byte[] INDEX_INFO_KEY = Bytes.toBytes("INDEX_INFO");
private final boolean isPresent;
public HFileBootstrapIndex(HoodieTableMetaClient metaClient) {
super(metaClient);
Path indexByPartitionPath = partitionIndexPath(metaClient);
Path indexByFilePath = fileIdIndexPath(metaClient);
try {
FileSystem fs = metaClient.getFs();
isPresent = fs.exists(indexByPartitionPath) && fs.exists(indexByFilePath);
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
private static String getPartitionKey(String partition) {
return "part=" + partition;
}
private static String getFileGroupKey(HoodieFileGroupId fileGroupId) {
return "part=" + fileGroupId.getPartitionPath() + ";fileid=" + fileGroupId.getFileId();
}
private static Path partitionIndexPath(HoodieTableMetaClient metaClient) {
return new Path(metaClient.getBootstrapIndexByPartitionFolderPath(),
FSUtils.makeBootstrapIndexFileName(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, BOOTSTRAP_INDEX_FILE_ID,
HoodieFileFormat.HFILE.getFileExtension()));
}
private static Path fileIdIndexPath(HoodieTableMetaClient metaClient) {
return new Path(metaClient.getBootstrapIndexByFileIdFolderNameFolderPath(),
FSUtils.makeBootstrapIndexFileName(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, BOOTSTRAP_INDEX_FILE_ID,
HoodieFileFormat.HFILE.getFileExtension()));
}
/**
* Helper method to create HFile Reader.
*
* @param hFilePath File Path
* @param conf Configuration
* @param fileSystem File System
*/
private static HFile.Reader createReader(String hFilePath, Configuration conf, FileSystem fileSystem) {
try {
LOG.info("Opening HFile for reading :" + hFilePath);
HFile.Reader reader = HFile.createReader(fileSystem, new HFilePathForReader(hFilePath),
new CacheConfig(conf), conf);
return reader;
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
@Override
public BootstrapIndex.IndexReader createReader() {
return new HFileBootstrapIndexReader(metaClient);
}
@Override
public BootstrapIndex.IndexWriter createWriter(String bootstrapBasePath) {
return new HFileBootstrapIndexWriter(bootstrapBasePath, metaClient);
}
@Override
public void dropIndex() {
try {
Path[] indexPaths = new Path[]{partitionIndexPath(metaClient), fileIdIndexPath(metaClient)};
for (Path indexPath : indexPaths) {
if (metaClient.getFs().exists(indexPath)) {
LOG.info("Dropping bootstrap index. Deleting file : " + indexPath);
metaClient.getFs().delete(indexPath);
}
}
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
@Override
protected boolean isPresent() {
return isPresent;
}
/**
* HFile Based Index Reader.
*/
public static class HFileBootstrapIndexReader extends BootstrapIndex.IndexReader {
// Base Path of external files.
private final String bootstrapBasePath;
// Well Known Paths for indices
private final String indexByPartitionPath;
private final String indexByFileIdPath;
// Index Readers
private transient HFile.Reader indexByPartitionReader;
private transient HFile.Reader indexByFileIdReader;
// Bootstrap Index Info
private transient HoodieBootstrapIndexInfo bootstrapIndexInfo;
public HFileBootstrapIndexReader(HoodieTableMetaClient metaClient) {
super(metaClient);
Path indexByPartitionPath = partitionIndexPath(metaClient);
Path indexByFilePath = fileIdIndexPath(metaClient);
this.indexByPartitionPath = indexByPartitionPath.toString();
this.indexByFileIdPath = indexByFilePath.toString();
initIndexInfo();
this.bootstrapBasePath = bootstrapIndexInfo.getBootstrapBasePath();
LOG.info("Loaded HFileBasedBootstrapIndex with source base path :" + bootstrapBasePath);
}
private void initIndexInfo() {
synchronized (this) {
if (null == bootstrapIndexInfo) {
try {
bootstrapIndexInfo = fetchBootstrapIndexInfo();
} catch (IOException ioe) {
throw new HoodieException(ioe.getMessage(), ioe);
}
}
}
}
private HoodieBootstrapIndexInfo fetchBootstrapIndexInfo() throws IOException {
return TimelineMetadataUtils.deserializeAvroMetadata(
partitionIndexReader().loadFileInfo().get(INDEX_INFO_KEY),
HoodieBootstrapIndexInfo.class);
}
private HFile.Reader partitionIndexReader() {
if (null == indexByPartitionReader) {
synchronized (this) {
if (null == indexByPartitionReader) {
LOG.info("Opening partition index :" + indexByPartitionPath);
this.indexByPartitionReader =
createReader(indexByPartitionPath, metaClient.getHadoopConf(), metaClient.getFs());
}
}
}
return indexByPartitionReader;
}
private HFile.Reader fileIdIndexReader() {
if (null == indexByFileIdReader) {
synchronized (this) {
if (null == indexByFileIdReader) {
LOG.info("Opening fileId index :" + indexByFileIdPath);
this.indexByFileIdReader =
createReader(indexByFileIdPath, metaClient.getHadoopConf(), metaClient.getFs());
}
}
}
return indexByFileIdReader;
}
@Override
public List<String> getIndexedPartitionPaths() {
HFileScanner scanner = partitionIndexReader().getScanner(true, true);
return getAllKeys(scanner);
}
@Override
public List<String> getIndexedFileIds() {
HFileScanner scanner = fileIdIndexReader().getScanner(true, true);
return getAllKeys(scanner);
}
private List<String> getAllKeys(HFileScanner scanner) {
List<String> keys = new ArrayList<>();
try {
boolean available = scanner.seekTo();
while (available) {
keys.add(CellUtil.getCellKeyAsString(scanner.getKeyValue()));
available = scanner.next();
}
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
return keys;
}
@Override
public List<BootstrapFileMapping> getSourceFileMappingForPartition(String partition) {
try {
HFileScanner scanner = partitionIndexReader().getScanner(true, true);
KeyValue keyValue = new KeyValue(Bytes.toBytes(getPartitionKey(partition)), new byte[0], new byte[0],
HConstants.LATEST_TIMESTAMP, KeyValue.Type.Put, new byte[0]);
if (scanner.seekTo(keyValue) == 0) {
ByteBuffer readValue = scanner.getValue();
byte[] valBytes = Bytes.toBytes(readValue);
HoodieBootstrapPartitionMetadata metadata =
TimelineMetadataUtils.deserializeAvroMetadata(valBytes, HoodieBootstrapPartitionMetadata.class);
return metadata.getFileIdToBootstrapFile().entrySet().stream()
.map(e -> new BootstrapFileMapping(bootstrapBasePath, metadata.getBootstrapPartitionPath(),
partition, e.getValue(), e.getKey())).collect(Collectors.toList());
} else {
LOG.warn("No value found for partition key (" + partition + ")");
return new ArrayList<>();
}
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
@Override
public String getBootstrapBasePath() {
return bootstrapBasePath;
}
@Override
public Map<HoodieFileGroupId, BootstrapFileMapping> getSourceFileMappingForFileIds(
List<HoodieFileGroupId> ids) {
Map<HoodieFileGroupId, BootstrapFileMapping> result = new HashMap<>();
// Arrange input Keys in sorted order for 1 pass scan
List<HoodieFileGroupId> fileGroupIds = new ArrayList<>(ids);
Collections.sort(fileGroupIds);
try {
HFileScanner scanner = fileIdIndexReader().getScanner(true, true);
for (HoodieFileGroupId fileGroupId : fileGroupIds) {
KeyValue keyValue = new KeyValue(Bytes.toBytes(getFileGroupKey(fileGroupId)), new byte[0], new byte[0],
HConstants.LATEST_TIMESTAMP, KeyValue.Type.Put, new byte[0]);
if (scanner.seekTo(keyValue) == 0) {
ByteBuffer readValue = scanner.getValue();
byte[] valBytes = Bytes.toBytes(readValue);
HoodieBootstrapFilePartitionInfo fileInfo = TimelineMetadataUtils.deserializeAvroMetadata(valBytes,
HoodieBootstrapFilePartitionInfo.class);
BootstrapFileMapping mapping = new BootstrapFileMapping(bootstrapBasePath,
fileInfo.getBootstrapPartitionPath(), fileInfo.getPartitionPath(), fileInfo.getBootstrapFileStatus(),
fileGroupId.getFileId());
result.put(fileGroupId, mapping);
}
}
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
return result;
}
@Override
public void close() {
try {
if (indexByPartitionReader != null) {
indexByPartitionReader.close(true);
indexByPartitionReader = null;
}
if (indexByFileIdReader != null) {
indexByFileIdReader.close(true);
indexByFileIdReader = null;
}
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
}
/**
* Boostrap Index Writer to build bootstrap index.
*/
public static class HFileBootstrapIndexWriter extends BootstrapIndex.IndexWriter {
private final String bootstrapBasePath;
private final Path indexByPartitionPath;
private final Path indexByFileIdPath;
private HFile.Writer indexByPartitionWriter;
private HFile.Writer indexByFileIdWriter;
private boolean closed = false;
private int numPartitionKeysAdded = 0;
private int numFileIdKeysAdded = 0;
private final Map<String, List<BootstrapFileMapping>> sourceFileMappings = new HashMap<>();
private HFileBootstrapIndexWriter(String bootstrapBasePath, HoodieTableMetaClient metaClient) {
super(metaClient);
try {
metaClient.initializeBootstrapDirsIfNotExists();
this.bootstrapBasePath = bootstrapBasePath;
this.indexByPartitionPath = partitionIndexPath(metaClient);
this.indexByFileIdPath = fileIdIndexPath(metaClient);
if (metaClient.getFs().exists(indexByPartitionPath) || metaClient.getFs().exists(indexByFileIdPath)) {
String errMsg = "Previous version of bootstrap index exists. Partition Index Path :" + indexByPartitionPath
+ ", FileId index Path :" + indexByFileIdPath;
LOG.info(errMsg);
throw new HoodieException(errMsg);
}
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
/**
* Append bootstrap index entries for next partitions in sorted order.
* @param partitionPath Hudi Partition Path
* @param bootstrapPartitionPath Source Partition Path
* @param bootstrapFileMappings Bootstrap Source File to Hudi File Id mapping
*/
private void writeNextPartition(String partitionPath, String bootstrapPartitionPath,
List<BootstrapFileMapping> bootstrapFileMappings) {
try {
LOG.info("Adding bootstrap partition Index entry for partition :" + partitionPath
+ ", bootstrap Partition :" + bootstrapPartitionPath + ", Num Entries :" + bootstrapFileMappings.size());
LOG.info("ADDING entries :" + bootstrapFileMappings);
HoodieBootstrapPartitionMetadata bootstrapPartitionMetadata = new HoodieBootstrapPartitionMetadata();
bootstrapPartitionMetadata.setBootstrapPartitionPath(bootstrapPartitionPath);
bootstrapPartitionMetadata.setPartitionPath(partitionPath);
bootstrapPartitionMetadata.setFileIdToBootstrapFile(
bootstrapFileMappings.stream().map(m -> Pair.of(m.getFileId(),
m.getBoostrapFileStatus())).collect(Collectors.toMap(Pair::getKey, Pair::getValue)));
Option<byte[]> bytes = TimelineMetadataUtils.serializeAvroMetadata(bootstrapPartitionMetadata, HoodieBootstrapPartitionMetadata.class);
if (bytes.isPresent()) {
indexByPartitionWriter
.append(new KeyValue(Bytes.toBytes(getPartitionKey(partitionPath)), new byte[0], new byte[0],
HConstants.LATEST_TIMESTAMP, KeyValue.Type.Put, bytes.get()));
numPartitionKeysAdded++;
}
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
/**
* Write next source file to hudi file-id. Entries are expected to be appended in hudi file-group id
* order.
* @param mapping boostrap source file mapping.
*/
private void writeNextSourceFileMapping(BootstrapFileMapping mapping) {
try {
HoodieBootstrapFilePartitionInfo srcFilePartitionInfo = new HoodieBootstrapFilePartitionInfo();
srcFilePartitionInfo.setPartitionPath(mapping.getPartitionPath());
srcFilePartitionInfo.setBootstrapPartitionPath(mapping.getBootstrapPartitionPath());
srcFilePartitionInfo.setBootstrapFileStatus(mapping.getBoostrapFileStatus());
KeyValue kv = new KeyValue(getFileGroupKey(mapping.getFileGroupId()).getBytes(), new byte[0], new byte[0],
HConstants.LATEST_TIMESTAMP, KeyValue.Type.Put,
TimelineMetadataUtils.serializeAvroMetadata(srcFilePartitionInfo,
HoodieBootstrapFilePartitionInfo.class).get());
indexByFileIdWriter.append(kv);
numFileIdKeysAdded++;
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
/**
* Commit bootstrap index entries. Appends Metadata and closes write handles.
*/
private void commit() {
try {
if (!closed) {
HoodieBootstrapIndexInfo partitionIndexInfo = HoodieBootstrapIndexInfo.newBuilder()
.setCreatedTimestamp(new Date().getTime())
.setNumKeys(numPartitionKeysAdded)
.setBootstrapBasePath(bootstrapBasePath)
.build();
LOG.info("Adding Partition FileInfo :" + partitionIndexInfo);
HoodieBootstrapIndexInfo fileIdIndexInfo = HoodieBootstrapIndexInfo.newBuilder()
.setCreatedTimestamp(new Date().getTime())
.setNumKeys(numFileIdKeysAdded)
.setBootstrapBasePath(bootstrapBasePath)
.build();
LOG.info("Appending FileId FileInfo :" + fileIdIndexInfo);
indexByPartitionWriter.appendFileInfo(INDEX_INFO_KEY,
TimelineMetadataUtils.serializeAvroMetadata(partitionIndexInfo, HoodieBootstrapIndexInfo.class).get());
indexByFileIdWriter.appendFileInfo(INDEX_INFO_KEY,
TimelineMetadataUtils.serializeAvroMetadata(fileIdIndexInfo, HoodieBootstrapIndexInfo.class).get());
close();
}
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
/**
* Close Writer Handles.
*/
public void close() {
try {
if (!closed) {
indexByPartitionWriter.close();
indexByFileIdWriter.close();
closed = true;
}
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
@Override
public void begin() {
try {
HFileContext meta = new HFileContextBuilder().build();
this.indexByPartitionWriter = HFile.getWriterFactory(metaClient.getHadoopConf(),
new CacheConfig(metaClient.getHadoopConf())).withPath(metaClient.getFs(), indexByPartitionPath)
.withFileContext(meta).withComparator(new HoodieKVComparator()).create();
this.indexByFileIdWriter = HFile.getWriterFactory(metaClient.getHadoopConf(),
new CacheConfig(metaClient.getHadoopConf())).withPath(metaClient.getFs(), indexByFileIdPath)
.withFileContext(meta).withComparator(new HoodieKVComparator()).create();
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
@Override
public void appendNextPartition(String partitionPath, List<BootstrapFileMapping> bootstrapFileMappings) {
sourceFileMappings.put(partitionPath, bootstrapFileMappings);
}
@Override
public void finish() {
// Sort and write
List<String> partitions = sourceFileMappings.keySet().stream().sorted().collect(Collectors.toList());
partitions.forEach(p -> writeNextPartition(p, sourceFileMappings.get(p).get(0).getBootstrapPartitionPath(),
sourceFileMappings.get(p)));
sourceFileMappings.values().stream().flatMap(Collection::stream).sorted()
.forEach(this::writeNextSourceFileMapping);
commit();
}
}
/**
* IMPORTANT :
* HFile Readers use HFile name (instead of path) as cache key. This could be fine as long
* as file names are UUIDs. For bootstrap, we are using well-known index names.
* Hence, this hacky workaround to return full path string from Path subclass and pass it to reader.
* The other option is to disable block cache for Bootstrap which again involves some custom code
* as there is no API to disable cache.
*/
private static class HFilePathForReader extends Path {
public HFilePathForReader(String pathString) throws IllegalArgumentException {
super(pathString);
}
@Override
public String getName() {
return toString();
}
}
/**
* This class is explicitly used as Key Comparator to workaround hard coded
* legacy format class names inside HBase. Otherwise we will face issues with shading.
*/
public static class HoodieKVComparator extends KeyValue.KVComparator {
}
}

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.common.fs;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
@@ -45,6 +46,7 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map.Entry;
import java.util.Objects;
@@ -52,6 +54,7 @@ import java.util.UUID;
import java.util.function.Function;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
@@ -114,6 +117,10 @@ public class FSUtils {
return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, fileExtension);
}
public static String makeBootstrapIndexFileName(String instantTime, String fileId, String ext) {
return String.format("%s_%s_%s%s", fileId, "1-0-1", instantTime, ext);
}
public static String maskWithoutFileId(String instantTime, int taskPartitionId) {
return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieFileFormat.PARQUET.getFileExtension());
}
@@ -225,8 +232,8 @@ public class FSUtils {
public static String getFileExtension(String fullName) {
Objects.requireNonNull(fullName);
String fileName = (new File(fullName)).getName();
int dotIndex = fileName.indexOf('.');
String fileName = new File(fullName).getName();
int dotIndex = fileName.lastIndexOf('.');
return dotIndex == -1 ? "" : fileName.substring(dotIndex);
}
@@ -235,10 +242,6 @@ public class FSUtils {
return (path) -> !path.toString().contains(HoodieTableMetaClient.METAFOLDER_NAME);
}
public static String getInstantTime(String name) {
return name.replace(getFileExtension(name), "");
}
/**
* Returns a new unique prefix for creating a file group.
*/
@@ -516,4 +519,33 @@ public class FSUtils {
return returnConf;
}
/**
* Get the FS implementation for this table.
* @param path Path String
* @param hadoopConf Serializable Hadoop Configuration
* @param consistencyGuardConfig Consistency Guard Config
* @return HoodieWrapperFileSystem
*/
public static HoodieWrapperFileSystem getFs(String path, SerializableConfiguration hadoopConf,
ConsistencyGuardConfig consistencyGuardConfig) {
FileSystem fileSystem = FSUtils.getFs(path, hadoopConf.newCopy());
return new HoodieWrapperFileSystem(fileSystem,
consistencyGuardConfig.isConsistencyCheckEnabled()
? new FailSafeConsistencyGuard(fileSystem, consistencyGuardConfig)
: new NoOpConsistencyGuard());
}
/**
* Helper to filter out paths under metadata folder when running fs.globStatus.
* @param fs File System
* @param globPath Glob Path
* @return
* @throws IOException
*/
public static List<FileStatus> getGlobStatusExcludingMetaFolder(FileSystem fs, Path globPath) throws IOException {
FileStatus[] statuses = fs.globStatus(globPath);
return Arrays.stream(statuses)
.filter(fileStatus -> !fileStatus.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME))
.collect(Collectors.toList());
}
}

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 org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import java.io.Serializable;
import java.util.Objects;
/**
* Represents common metadata about base-file.
* A base file can be Hudi base file or even an external (non-hudi) base file too.
*/
public class BaseFile implements Serializable {
private static final long serialVersionUID = 1L;
private transient FileStatus fileStatus;
private final String fullPath;
private long fileLen;
public BaseFile(BaseFile dataFile) {
this.fileStatus = dataFile.fileStatus;
this.fullPath = dataFile.fullPath;
this.fileLen = dataFile.fileLen;
}
public BaseFile(FileStatus fileStatus) {
this.fileStatus = fileStatus;
this.fullPath = fileStatus.getPath().toString();
this.fileLen = fileStatus.getLen();
}
public BaseFile(String filePath) {
this.fileStatus = null;
this.fullPath = filePath;
this.fileLen = -1;
}
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;
}
BaseFile dataFile = (BaseFile) o;
return Objects.equals(fullPath, dataFile.fullPath);
}
@Override
public int hashCode() {
return Objects.hash(fullPath);
}
@Override
public String toString() {
return "BaseFile{fullPath=" + fullPath + ", fileLen=" + fileLen + '}';
}
}

View File

@@ -0,0 +1,54 @@
/*
* 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 org.apache.hudi.avro.model.HoodieFileStatus;
import org.apache.hudi.common.bootstrap.FileStatusUtils;
/**
* POJO storing (partitionPath, hoodieFileId) -> external base file path.
*/
public class BootstrapBaseFileMapping implements Serializable {
private final HoodieFileGroupId fileGroupId;
private final HoodieFileStatus bootstrapFileStatus;
public BootstrapBaseFileMapping(HoodieFileGroupId fileGroupId, HoodieFileStatus bootstrapFileStatus) {
this.fileGroupId = fileGroupId;
this.bootstrapFileStatus = bootstrapFileStatus;
}
public HoodieFileGroupId getFileGroupId() {
return fileGroupId;
}
public BaseFile getBootstrapBaseFile() {
return new BaseFile(FileStatusUtils.toFileStatus(bootstrapFileStatus));
}
@Override
public String toString() {
return "BootstrapBaseFileMapping{"
+ "fileGroupId=" + fileGroupId
+ ", bootstrapFileStatus=" + bootstrapFileStatus
+ '}';
}
}

View File

@@ -0,0 +1,110 @@
/*
* 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.hudi.avro.model.HoodieFileStatus;
/**
* Value stored in the bootstrap index.
*/
public class BootstrapFileMapping implements Serializable, Comparable<BootstrapFileMapping> {
private final String bootstrapBasePath;
private final String bootstrapPartitionPath;
private final HoodieFileStatus boostrapFileStatus;
private final String partitionPath;
private final String fileId;
public BootstrapFileMapping(String bootstrapBasePath, String bootstrapPartitionPath, String partitionPath,
HoodieFileStatus boostrapFileStatus, String fileId) {
this.bootstrapBasePath = bootstrapBasePath;
this.bootstrapPartitionPath = bootstrapPartitionPath;
this.partitionPath = partitionPath;
this.boostrapFileStatus = boostrapFileStatus;
this.fileId = fileId;
}
@Override
public String toString() {
return "BootstrapFileMapping{"
+ "bootstrapBasePath='" + bootstrapBasePath + '\''
+ ", bootstrapPartitionPath='" + bootstrapPartitionPath + '\''
+ ", boostrapFileStatus=" + boostrapFileStatus
+ ", partitionPath='" + partitionPath + '\''
+ ", fileId='" + fileId + '\''
+ '}';
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
BootstrapFileMapping mapping = (BootstrapFileMapping) o;
return Objects.equals(bootstrapBasePath, mapping.bootstrapBasePath)
&& Objects.equals(bootstrapPartitionPath, mapping.bootstrapPartitionPath)
&& Objects.equals(partitionPath, mapping.partitionPath)
&& Objects.equals(boostrapFileStatus, mapping.boostrapFileStatus)
&& Objects.equals(fileId, mapping.fileId);
}
@Override
public int hashCode() {
return Objects.hash(bootstrapBasePath, bootstrapPartitionPath, partitionPath, boostrapFileStatus, fileId);
}
public String getBootstrapBasePath() {
return bootstrapBasePath;
}
public String getBootstrapPartitionPath() {
return bootstrapPartitionPath;
}
public String getPartitionPath() {
return partitionPath;
}
public HoodieFileStatus getBoostrapFileStatus() {
return boostrapFileStatus;
}
public String getFileId() {
return fileId;
}
public HoodieFileGroupId getFileGroupId() {
return new HoodieFileGroupId(partitionPath, fileId);
}
@Override
public int compareTo(BootstrapFileMapping o) {
int ret = partitionPath.compareTo(o.partitionPath);
if (ret == 0) {
ret = fileId.compareTo(o.fileId);
}
return ret;
}
}

View File

@@ -44,18 +44,20 @@ public class CompactionOperation implements Serializable {
private Option<String> dataFileName;
private HoodieFileGroupId id;
private Map<String, Double> metrics;
private Option<String> bootstrapFilePath;
// Only for serialization/de-serialization
@Deprecated
public CompactionOperation() {}
public CompactionOperation(String fileId, String partitionPath, String baseInstantTime,
Option<String> dataFileCommitTime, List<String> deltaFileNames, Option<String> dataFileName,
Map<String, Double> metrics) {
Option<String> dataFileCommitTime, List<String> deltaFileNames, Option<String> dataFileName,
Option<String> bootstrapFilePath, Map<String, Double> metrics) {
this.baseInstantTime = baseInstantTime;
this.dataFileCommitTime = dataFileCommitTime;
this.deltaFileNames = deltaFileNames;
this.dataFileName = dataFileName;
this.bootstrapFilePath = bootstrapFilePath;
this.id = new HoodieFileGroupId(partitionPath, fileId);
this.metrics = metrics;
}
@@ -67,14 +69,15 @@ public class CompactionOperation implements Serializable {
this.dataFileName = Option.of(dataFile.get().getFileName());
this.id = new HoodieFileGroupId(partitionPath, dataFile.get().getFileId());
this.dataFileCommitTime = Option.of(dataFile.get().getCommitTime());
this.bootstrapFilePath = dataFile.get().getBootstrapBaseFile().map(BaseFile::getPath);
} else {
assert logFiles.size() > 0;
this.dataFileName = 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.bootstrapFilePath = Option.empty();
}
this.deltaFileNames = logFiles.stream().map(s -> s.getPath().getName()).collect(Collectors.toList());
this.metrics = metrics;
}
@@ -111,9 +114,17 @@ public class CompactionOperation implements Serializable {
return id;
}
public Option<String> getBootstrapFilePath() {
return bootstrapFilePath;
}
public Option<HoodieBaseFile> getBaseFile(String basePath, String partitionPath) {
Option<BaseFile> externalBaseFile = bootstrapFilePath.map(BaseFile::new);
Path dirPath = FSUtils.getPartitionPath(basePath, partitionPath);
return dataFileName.map(df -> new HoodieBaseFile(new Path(dirPath, df).toString()));
return dataFileName.map(df -> {
return externalBaseFile.map(ext -> new HoodieBaseFile(new Path(dirPath, df).toString(), ext))
.orElseGet(() -> new HoodieBaseFile(new Path(dirPath, df).toString()));
});
}
/**
@@ -130,6 +141,7 @@ public class CompactionOperation implements Serializable {
op.deltaFileNames = new ArrayList<>(operation.getDeltaFilePaths());
op.id = new HoodieFileGroupId(operation.getPartitionPath(), operation.getFileId());
op.metrics = operation.getMetrics() == null ? new HashMap<>() : new HashMap<>(operation.getMetrics());
op.bootstrapFilePath = Option.ofNullable(operation.getBootstrapFilePath());
return op;
}
@@ -137,7 +149,7 @@ public class CompactionOperation implements Serializable {
public String toString() {
return "CompactionOperation{baseInstantTime='" + baseInstantTime + '\'' + ", dataFileCommitTime="
+ dataFileCommitTime + ", deltaFileNames=" + deltaFileNames + ", dataFileName=" + dataFileName + ", id='" + id
+ '\'' + ", metrics=" + metrics + '}';
+ '\'' + ", metrics=" + metrics + ", bootstrapFilePath=" + bootstrapFilePath + '}';
}
@Override

View File

@@ -52,6 +52,14 @@ public class FileSlice implements Serializable {
*/
private final TreeSet<HoodieLogFile> logFiles;
public FileSlice(FileSlice fileSlice) {
this.baseInstantTime = fileSlice.baseInstantTime;
this.baseFile = fileSlice.baseFile != null ? new HoodieBaseFile(fileSlice.baseFile) : null;
this.fileGroupId = fileSlice.fileGroupId;
this.logFiles = new TreeSet<>(HoodieLogFile.getReverseLogFileComparator());
fileSlice.logFiles.forEach(lf -> this.logFiles.add(new HoodieLogFile(lf)));
}
public FileSlice(String partitionPath, String baseInstantTime, String fileId) {
this(new HoodieFileGroupId(partitionPath, fileId), baseInstantTime);
}

View File

@@ -19,33 +19,39 @@
package org.apache.hudi.common.model;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import java.io.Serializable;
import java.util.Objects;
/**
* Hoodie base file.
* Hoodie base file - Represents metadata about Hudi file in DFS.
* Supports APIs to get Hudi FileId, Commit Time and bootstrap file (if any).
*/
public class HoodieBaseFile implements Serializable {
public class HoodieBaseFile extends BaseFile {
private static final long serialVersionUID = 1L;
private transient FileStatus fileStatus;
private final String fullPath;
private long fileLen;
private Option<BaseFile> bootstrapBaseFile;
public HoodieBaseFile(HoodieBaseFile dataFile) {
super(dataFile);
this.bootstrapBaseFile = dataFile.bootstrapBaseFile;
}
public HoodieBaseFile(FileStatus fileStatus) {
this.fileStatus = fileStatus;
this.fullPath = fileStatus.getPath().toString();
this.fileLen = fileStatus.getLen();
this(fileStatus, null);
}
public HoodieBaseFile(FileStatus fileStatus, BaseFile bootstrapBaseFile) {
super(fileStatus);
this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile);
}
public HoodieBaseFile(String filePath) {
this.fileStatus = null;
this.fullPath = filePath;
this.fileLen = -1;
this(filePath, null);
}
public HoodieBaseFile(String filePath, BaseFile bootstrapBaseFile) {
super(filePath);
this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile);
}
public String getFileId() {
@@ -56,49 +62,17 @@ public class HoodieBaseFile implements Serializable {
return FSUtils.getCommitTime(getFileName());
}
public String getPath() {
return fullPath;
public Option<BaseFile> getBootstrapBaseFile() {
return bootstrapBaseFile;
}
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;
}
HoodieBaseFile dataFile = (HoodieBaseFile) o;
return Objects.equals(fullPath, dataFile.fullPath);
}
@Override
public int hashCode() {
return Objects.hash(fullPath);
public void setBootstrapBaseFile(BaseFile bootstrapBaseFile) {
this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile);
}
@Override
public String toString() {
return "HoodieDataFile{fullPath=" + fullPath + ", fileLen=" + fileLen + '}';
return "HoodieBaseFile{fullPath=" + getPath() + ", fileLen=" + getFileLen()
+ ", BootstrapBaseFile=" + bootstrapBaseFile.orElse(null) + '}';
}
}

View File

@@ -22,7 +22,9 @@ package org.apache.hudi.common.model;
* Hoodie file format.
*/
public enum HoodieFileFormat {
PARQUET(".parquet"), HOODIE_LOG(".log");
PARQUET(".parquet"),
HOODIE_LOG(".log"),
HFILE(".hfile");
private final String extension;

View File

@@ -21,12 +21,14 @@ package org.apache.hudi.common.model;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import java.io.Serializable;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
@@ -58,6 +60,15 @@ public class HoodieFileGroup implements Serializable {
*/
private final Option<HoodieInstant> lastInstant;
public HoodieFileGroup(HoodieFileGroup fileGroup) {
this.timeline = fileGroup.timeline;
this.fileGroupId = fileGroup.fileGroupId;
this.fileSlices = new TreeMap<>(fileGroup.fileSlices.entrySet().stream()
.map(e -> Pair.of(e.getKey(), new FileSlice(e.getValue())))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue)));
this.lastInstant = fileGroup.lastInstant;
}
public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) {
this(new HoodieFileGroupId(partitionPath, id), timeline);
}

View File

@@ -24,7 +24,7 @@ import java.util.Objects;
/**
* Unique ID to identify a file-group in a data-set.
*/
public class HoodieFileGroupId implements Serializable {
public class HoodieFileGroupId implements Serializable, Comparable<HoodieFileGroupId> {
private final String partitionPath;
@@ -64,4 +64,13 @@ public class HoodieFileGroupId implements Serializable {
public String toString() {
return "HoodieFileGroupId{partitionPath='" + partitionPath + '\'' + ", fileId='" + fileId + '\'' + '}';
}
@Override
public int compareTo(HoodieFileGroupId o) {
int ret = partitionPath.compareTo(o.partitionPath);
if (ret == 0) {
ret = fileId.compareTo(fileId);
}
return ret;
}
}

View File

@@ -45,6 +45,12 @@ public class HoodieLogFile implements Serializable {
private final String pathStr;
private long fileLen;
public HoodieLogFile(HoodieLogFile logFile) {
this.fileStatus = logFile.fileStatus;
this.pathStr = logFile.pathStr;
this.fileLen = logFile.fileLen;
}
public HoodieLogFile(FileStatus fileStatus) {
this.fileStatus = fileStatus;
this.pathStr = fileStatus.getPath().toString();

View File

@@ -37,6 +37,7 @@ public enum WriteOperationType {
BULK_INSERT_PREPPED("bulk_insert_prepped"),
// delete
DELETE("delete"),
BOOTSTRAP("bootstrap"),
// used for old version
UNKNOWN("unknown");

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.common.table;
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
@@ -63,11 +64,15 @@ public class HoodieTableConfig implements Serializable {
public static final String HOODIE_TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version";
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 String HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME = "hoodie.bootstrap.index.class";
public static final String HOODIE_BOOTSTRAP_BASE_PATH = "hoodie.bootstrap.base.path";
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
public static final HoodieFileFormat DEFAULT_BASE_FILE_FORMAT = HoodieFileFormat.PARQUET;
public static final HoodieFileFormat DEFAULT_LOG_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
public static final String DEFAULT_PAYLOAD_CLASS = OverwriteWithLatestAvroPayload.class.getName();
public static final String DEFAULT_BOOTSTRAP_INDEX_CLASS = HFileBootstrapIndex.class.getName();
public static final Integer DEFAULT_TIMELINE_LAYOUT_VERSION = TimelineLayoutVersion.VERSION_0;
public static final String DEFAULT_ARCHIVELOG_FOLDER = "";
private Properties props;
@@ -131,6 +136,10 @@ public class HoodieTableConfig implements Serializable {
// Use latest Version as default unless forced by client
properties.setProperty(HOODIE_TIMELINE_LAYOUT_VERSION, TimelineLayoutVersion.CURR_VERSION.toString());
}
if (properties.containsKey(HOODIE_BOOTSTRAP_BASE_PATH) && !properties.containsKey(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME)) {
// Use the default bootstrap index class.
properties.setProperty(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, DEFAULT_BOOTSTRAP_INDEX_CLASS);
}
properties.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
}
}
@@ -161,6 +170,19 @@ public class HoodieTableConfig implements Serializable {
"org.apache.hudi");
}
/**
* Read the payload class for HoodieRecords from the table properties.
*/
public String getBootstrapIndexClass() {
// There could be tables written with payload class from com.uber.hoodie. Need to transparently
// change to org.apache.hudi
return props.getProperty(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, DEFAULT_BOOTSTRAP_INDEX_CLASS);
}
public Option<String> getBootstrapBasePath() {
return Option.ofNullable(props.getProperty(HOODIE_BOOTSTRAP_BASE_PATH));
}
/**
* Read the table name.
*/

View File

@@ -74,6 +74,12 @@ public class HoodieTableMetaClient implements Serializable {
public static final String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp";
public static final String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + File.separator + ".aux";
public static final String BOOTSTRAP_INDEX_ROOT_FOLDER_PATH = AUXILIARYFOLDER_NAME + File.separator + ".bootstrap";
public static final String BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH
+ File.separator + ".partitions";
public static final String BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + File.separator
+ ".fileids";
public static final String MARKER_EXTN = ".marker";
private String basePath;
@@ -209,6 +215,20 @@ public class HoodieTableMetaClient implements Serializable {
return basePath + File.separator + AUXILIARYFOLDER_NAME;
}
/**
* @return Bootstrap Index By Partition Folder
*/
public String getBootstrapIndexByPartitionFolderPath() {
return basePath + File.separator + BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH;
}
/**
* @return Bootstrap Index By Hudi File Id Folder
*/
public String getBootstrapIndexByFileIdFolderNameFolderPath() {
return basePath + File.separator + BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH;
}
/**
* @return path where archived timeline is stored
*/
@@ -300,44 +320,45 @@ public class HoodieTableMetaClient implements Serializable {
return archivedTimeline;
}
/**
* Helper method to initialize a table, with given basePath, tableType, name, archiveFolder, payloadClass.
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, String tableType,
String tableName, String archiveLogFolder, String payloadClassName) throws IOException {
return initTableType(hadoopConf, basePath, HoodieTableType.valueOf(tableType), tableName,
archiveLogFolder, payloadClassName, null, null);
}
/**
* Helper method to initialize a table, with given basePath, tableType, name, archiveFolder, payloadClass and
* base file format.
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, String tableType,
String tableName, String archiveLogFolder, String payloadClassName, String baseFileFormat) throws IOException {
return initTableType(hadoopConf, basePath, HoodieTableType.valueOf(tableType), tableName,
archiveLogFolder, payloadClassName, null, baseFileFormat);
public static HoodieTableMetaClient initTableTypeWithBootstrap(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String archiveLogFolder, String payloadClassName,
String baseFileFormat, String bootstrapIndexClass,
String bootstrapBasePath) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName,
archiveLogFolder, payloadClassName, null, baseFileFormat, bootstrapIndexClass, bootstrapBasePath);
}
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String archiveLogFolder, String payloadClassName,
String baseFileFormat) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName,
archiveLogFolder, payloadClassName, null, baseFileFormat, null, null);
}
/**
* Helper method to initialize a given path, as a given type and table name.
* Used primarily by tests, examples.
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
HoodieTableType tableType, String tableName, String payloadClassName) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName, null, null);
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String payloadClassName) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName,
null, null, null, null);
}
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
HoodieTableType tableType, String tableName,
String archiveLogFolder, String payloadClassName,
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String archiveLogFolder, String payloadClassName,
Integer timelineLayoutVersion) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName, archiveLogFolder, payloadClassName,
timelineLayoutVersion, null);
timelineLayoutVersion, null, null, null);
}
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
HoodieTableType tableType, String tableName, String archiveLogFolder, String payloadClassName,
Integer timelineLayoutVersion, String baseFileFormat) throws IOException {
private static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String archiveLogFolder, String payloadClassName,
Integer timelineLayoutVersion, String baseFileFormat,
String bootstrapIndexClass, String bootstrapBasePath) 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());
@@ -356,6 +377,15 @@ public class HoodieTableMetaClient implements Serializable {
if (null != baseFileFormat) {
properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, baseFileFormat.toUpperCase());
}
if (null != bootstrapIndexClass) {
properties.put(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, bootstrapIndexClass);
}
if (null != bootstrapBasePath) {
properties.put(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH, bootstrapBasePath);
}
return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties);
}
@@ -399,6 +429,7 @@ public class HoodieTableMetaClient implements Serializable {
fs.mkdirs(auxiliaryFolder);
}
initializeBootstrapDirsIfNotExists(hadoopConf, basePath, fs);
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
@@ -407,6 +438,25 @@ public class HoodieTableMetaClient implements Serializable {
return metaClient;
}
public static void initializeBootstrapDirsIfNotExists(Configuration hadoopConf,
String basePath, FileSystem fs) throws IOException {
// Create bootstrap index by partition folder if it does not exist
final Path bootstrap_index_folder_by_partition =
new Path(basePath, HoodieTableMetaClient.BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH);
if (!fs.exists(bootstrap_index_folder_by_partition)) {
fs.mkdirs(bootstrap_index_folder_by_partition);
}
// Create bootstrap index by partition folder if it does not exist
final Path bootstrap_index_folder_by_fileids =
new Path(basePath, HoodieTableMetaClient.BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH);
if (!fs.exists(bootstrap_index_folder_by_fileids)) {
fs.mkdirs(bootstrap_index_folder_by_fileids);
}
}
/**
* Helper method to scan all hoodie-instant metafiles.
*
@@ -512,7 +562,7 @@ public class HoodieTableMetaClient implements Serializable {
HoodieTableMetaClient
.scanFiles(getFs(), timelinePath, path -> {
// Include only the meta files with extensions that needs to be included
String extension = FSUtils.getFileExtension(path.getName());
String extension = HoodieInstant.getTimelineFileExtension(path.getName());
return includedExtensions.contains(extension);
})).map(HoodieInstant::new);
@@ -549,6 +599,10 @@ public class HoodieTableMetaClient implements Serializable {
return sb.toString();
}
public void initializeBootstrapDirsIfNotExists() throws IOException {
initializeBootstrapDirsIfNotExists(getHadoopConf(), basePath, getFs());
}
public void setBasePath(String basePath) {
this.basePath = basePath;
}

View File

@@ -18,9 +18,7 @@
package org.apache.hudi.common.table.timeline;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hadoop.fs.FileStatus;
import java.io.Serializable;
@@ -53,6 +51,12 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
return COMPARABLE_ACTIONS.getOrDefault(action, action);
}
public static String getTimelineFileExtension(String fileName) {
Objects.requireNonNull(fileName);
int dotIndex = fileName.indexOf('.');
return dotIndex == -1 ? "" : fileName.substring(dotIndex);
}
/**
* Instant State.
*/
@@ -77,7 +81,7 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
public HoodieInstant(FileStatus fileStatus) {
// First read the instant timestamp. [==>20170101193025<==].commit
String fileName = fileStatus.getPath().getName();
String fileExtension = FSUtils.getFileExtension(fileName);
String fileExtension = getTimelineFileExtension(fileName);
timestamp = fileName.replace(fileExtension, "");
// Next read the action for this marker

View File

@@ -81,6 +81,13 @@ public interface HoodieTimeline extends Serializable {
String INVALID_INSTANT_TS = "0";
// Instant corresponding to pristine state of the table after its creation
String INIT_INSTANT_TS = "00000000000000";
// Instant corresponding to METADATA bootstrapping of table/partitions
String METADATA_BOOTSTRAP_INSTANT_TS = "00000000000001";
// Instant corresponding to full bootstrapping of table/partitions
String FULL_BOOTSTRAP_INSTANT_TS = "00000000000002";
/**
* Filter this timeline to just include the in-flights.
*

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.common.table.timeline.dto;
import org.apache.hudi.common.model.BaseFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
@@ -35,6 +36,8 @@ public class BaseFileDTO {
private String fullPath;
@JsonProperty("fileLen")
private long fileLen;
@JsonProperty("bootstrapBaseFile")
private BaseFileDTO bootstrapBaseFile;
public static HoodieBaseFile toHoodieBaseFile(BaseFileDTO dto) {
if (null == dto) {
@@ -48,18 +51,39 @@ public class BaseFileDTO {
baseFile = new HoodieBaseFile(dto.fullPath);
baseFile.setFileLen(dto.fileLen);
}
baseFile.setBootstrapBaseFile(toBaseFile(dto.bootstrapBaseFile));
return baseFile;
}
public static BaseFileDTO fromHoodieBaseFile(HoodieBaseFile dataFile) {
if (null == dataFile) {
private static BaseFile toBaseFile(BaseFileDTO dto) {
if (null == dto) {
return null;
}
BaseFile baseFile;
if (null != dto.fileStatus) {
baseFile = new BaseFile(FileStatusDTO.toFileStatus(dto.fileStatus));
} else {
baseFile = new BaseFile(dto.fullPath);
baseFile.setFileLen(dto.fileLen);
}
return baseFile;
}
public static BaseFileDTO fromHoodieBaseFile(BaseFile baseFile) {
if (null == baseFile) {
return null;
}
BaseFileDTO dto = new BaseFileDTO();
dto.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus());
dto.fullPath = dataFile.getPath();
dto.fileLen = dataFile.getFileLen();
dto.fileStatus = FileStatusDTO.fromFileStatus(baseFile.getFileStatus());
dto.fullPath = baseFile.getPath();
dto.fileLen = baseFile.getFileLen();
if (baseFile instanceof HoodieBaseFile) {
dto.bootstrapBaseFile = ((HoodieBaseFile)baseFile).getBootstrapBaseFile()
.map(BaseFileDTO::fromHoodieBaseFile).orElse(null);
}
return dto;
}

View File

@@ -60,6 +60,9 @@ public class CompactionOpDTO {
@JsonProperty("metrics")
private Map<String, Double> metrics;
@JsonProperty("bootstrapBaseFile")
private String bootstrapBaseFile;
public static CompactionOpDTO fromCompactionOperation(String compactionInstantTime, CompactionOperation op) {
CompactionOpDTO dto = new CompactionOpDTO();
dto.fileId = op.getFileId();
@@ -70,13 +73,14 @@ public class CompactionOpDTO {
dto.deltaFilePaths = new ArrayList<>(op.getDeltaFileNames());
dto.partitionPath = op.getPartitionPath();
dto.metrics = op.getMetrics() == null ? new HashMap<>() : new HashMap<>(op.getMetrics());
dto.bootstrapBaseFile = op.getBootstrapFilePath().orElse(null);
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));
Option.ofNullable(dto.dataFileCommitTime), dto.deltaFilePaths,
Option.ofNullable(dto.dataFilePath), Option.ofNullable(dto.bootstrapBaseFile), dto.metrics));
}
}

View File

@@ -68,7 +68,7 @@ public class FileStatusDTO {
dto.blockReplication = fileStatus.getReplication();
dto.blocksize = fileStatus.getBlockSize();
dto.modificationTime = fileStatus.getModificationTime();
dto.accessTime = fileStatus.getModificationTime();
dto.accessTime = fileStatus.getAccessTime();
dto.symlink = fileStatus.isSymlink() ? FilePathDTO.fromPath(fileStatus.getSymlink()) : null;
safeReadAndSetMetadata(dto, fileStatus);
} catch (IOException ioe) {

View File

@@ -18,8 +18,11 @@
package org.apache.hudi.common.table.view;
import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.BootstrapFileMapping;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.BootstrapBaseFileMapping;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroup;
@@ -34,7 +37,6 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
@@ -56,6 +58,8 @@ import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS;
/**
* 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
@@ -73,7 +77,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
private HoodieTimeline visibleCommitsAndCompactionTimeline;
// Used to concurrently load and populate partition views
private ConcurrentHashMap<String, Boolean> addedPartitions = new ConcurrentHashMap<>(4096);
private final 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
@@ -81,6 +85,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
private final ReadLock readLock = globalLock.readLock();
private final WriteLock writeLock = globalLock.writeLock();
private BootstrapIndex bootstrapIndex;
private String getPartitionPathFromFilePath(String fullPath) {
return FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), new Path(fullPath).getParent());
}
@@ -91,10 +97,11 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
this.metaClient = metaClient;
refreshTimeline(visibleActiveTimeline);
this.bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient);
// Load Pending Compaction Operations
resetPendingCompactionOperations(CompactionUtils.getAllPendingCompactionOperations(metaClient).values().stream()
.map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue()))));
resetBootstrapBaseFileMapping(Stream.empty());
}
/**
@@ -117,6 +124,16 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
// Group by partition for efficient updates for both InMemory and DiskBased stuctures.
fileGroups.stream().collect(Collectors.groupingBy(HoodieFileGroup::getPartitionPath)).forEach((partition, value) -> {
if (!isPartitionAvailableInStore(partition)) {
if (bootstrapIndex.useIndex()) {
try (BootstrapIndex.IndexReader reader = bootstrapIndex.createReader()) {
LOG.info("Boostrap Index available for partition " + partition);
List<BootstrapFileMapping> sourceFileMappings =
reader.getSourceFileMappingForPartition(partition);
addBootstrapBaseFileMapping(sourceFileMappings.stream()
.map(s -> new BootstrapBaseFileMapping(new HoodieFileGroupId(s.getPartitionPath(),
s.getFileId()), s.getBoostrapFileStatus())));
}
}
storePartitionView(partition, value);
}
});
@@ -163,6 +180,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
if (logFiles.containsKey(pair)) {
logFiles.get(pair).forEach(group::addLogFile);
}
if (addPendingCompactionFileSlice) {
Option<Pair<String, CompactionOperation>> pendingCompaction =
getPendingCompactionOperationWithInstant(group.getFileGroupId());
@@ -189,6 +207,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
addedPartitions.clear();
resetViewState();
bootstrapIndex = null;
// Initialize with new Hoodie timeline.
init(metaClient, getTimeline());
} finally {
@@ -313,6 +333,41 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
return fileSlice;
}
protected HoodieFileGroup addBootstrapBaseFileIfPresent(HoodieFileGroup fileGroup) {
boolean hasBootstrapBaseFile = fileGroup.getAllFileSlices()
.anyMatch(fs -> fs.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS));
if (hasBootstrapBaseFile) {
HoodieFileGroup newFileGroup = new HoodieFileGroup(fileGroup);
newFileGroup.getAllFileSlices().filter(fs -> fs.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS))
.forEach(fs -> fs.setBaseFile(
addBootstrapBaseFileIfPresent(fs.getFileGroupId(), fs.getBaseFile().get())));
return newFileGroup;
}
return fileGroup;
}
protected FileSlice addBootstrapBaseFileIfPresent(FileSlice fileSlice) {
if (fileSlice.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)) {
FileSlice copy = new FileSlice(fileSlice);
copy.getBaseFile().ifPresent(dataFile -> {
Option<BootstrapBaseFileMapping> edf = getBootstrapBaseFile(copy.getFileGroupId());
edf.ifPresent(e -> dataFile.setBootstrapBaseFile(e.getBootstrapBaseFile()));
});
return copy;
}
return fileSlice;
}
protected HoodieBaseFile addBootstrapBaseFileIfPresent(HoodieFileGroupId fileGroupId, HoodieBaseFile baseFile) {
if (baseFile.getCommitTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)) {
HoodieBaseFile copy = new HoodieBaseFile(baseFile);
Option<BootstrapBaseFileMapping> edf = getBootstrapBaseFile(fileGroupId);
edf.ifPresent(e -> copy.setBootstrapBaseFile(e.getBootstrapBaseFile()));
return copy;
}
return baseFile;
}
@Override
public final Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations() {
try {
@@ -329,7 +384,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestBaseFiles(partitionPath);
return fetchLatestBaseFiles(partitionPath)
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df));
} finally {
readLock.unlock();
}
@@ -356,7 +412,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
.filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, maxCommitTime
))
.filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst()))
.filter(Option::isPresent).map(Option::get);
.filter(Option::isPresent).map(Option::get)
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df));
} finally {
readLock.unlock();
}
@@ -369,9 +426,9 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles()
.filter(
baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS, instantTime))
.filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst().orElse(null));
.filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS,
instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst().orElse(null))
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df));
} finally {
readLock.unlock();
}
@@ -386,7 +443,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestBaseFile(partitionPath, fileId);
return fetchLatestBaseFile(partitionPath, fileId)
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df));
} finally {
readLock.unlock();
}
@@ -396,9 +454,10 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
public final Stream<HoodieBaseFile> getLatestBaseFilesInRange(List<String> commitsToReturn) {
try {
readLock.lock();
return fetchAllStoredFileGroups().map(fileGroup -> Option.fromJavaOptional(
return fetchAllStoredFileGroups().map(fileGroup -> Pair.of(fileGroup.getFileGroupId(), Option.fromJavaOptional(
fileGroup.getAllBaseFiles().filter(baseFile -> commitsToReturn.contains(baseFile.getCommitTime())
&& !isBaseFileDueToPendingCompaction(baseFile)).findFirst())).filter(Option::isPresent).map(Option::get);
&& !isBaseFileDueToPendingCompaction(baseFile)).findFirst()))).filter(p -> p.getValue().isPresent())
.map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get()));
} finally {
readLock.unlock();
}
@@ -412,7 +471,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllBaseFiles(partitionPath)
.filter(df -> visibleCommitsAndCompactionTimeline.containsOrBeforeTimelineStarts(df.getCommitTime()))
.filter(df -> !isBaseFileDueToPendingCompaction(df));
.filter(df -> !isBaseFileDueToPendingCompaction(df))
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df));
} finally {
readLock.unlock();
}
@@ -424,7 +484,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestFileSlices(partitionPath).map(this::filterBaseFileAfterPendingCompaction);
return fetchLatestFileSlices(partitionPath).map(this::filterBaseFileAfterPendingCompaction)
.map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
@@ -440,7 +501,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
Option<FileSlice> fs = fetchLatestFileSlice(partitionPath, fileId);
return fs.map(this::filterBaseFileAfterPendingCompaction);
return fs.map(this::filterBaseFileAfterPendingCompaction).map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
@@ -462,7 +523,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
return fileGroup.getLatestFileSliceBefore(compactionInstantTime);
}
return Option.of(fileSlice);
}).map(Option::get);
}).map(Option::get).map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
@@ -477,9 +538,10 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
ensurePartitionLoadedCorrectly(partitionPath);
Stream<FileSlice> fileSliceStream = fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime);
if (includeFileSlicesInPendingCompaction) {
return fileSliceStream.map(this::filterBaseFileAfterPendingCompaction);
return fileSliceStream.map(this::filterBaseFileAfterPendingCompaction).map(this::addBootstrapBaseFileIfPresent);
} else {
return fileSliceStream.filter(fs -> !isPendingCompactionScheduledForFileId(fs.getFileGroupId()));
return fileSliceStream.filter(fs -> !isPendingCompactionScheduledForFileId(fs.getFileGroupId()))
.map(this::addBootstrapBaseFileIfPresent);
}
} finally {
readLock.unlock();
@@ -499,7 +561,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
fileSlice = Option.of(fetchMergedFileSlice(fileGroup, fileSlice.get()));
}
return fileSlice;
}).filter(Option::isPresent).map(Option::get);
}).filter(Option::isPresent).map(Option::get).map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
@@ -509,7 +571,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
public final Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
try {
readLock.lock();
return fetchLatestFileSliceInRange(commitsToReturn);
return fetchLatestFileSliceInRange(commitsToReturn).map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
@@ -521,7 +583,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partition = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partition);
return fetchAllFileSlices(partition);
return fetchAllFileSlices(partition).map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
@@ -543,7 +605,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
// in other places.
String partition = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partition);
return fetchAllStoredFileGroups(partition);
return fetchAllStoredFileGroups(partition).map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
@@ -593,6 +655,48 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
*/
abstract Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations();
/**
* Check if there is an bootstrap base file present for this file.
*
* @param fgId File-Group Id
* @return true if there is associated bootstrap base-file, false otherwise
*/
protected abstract boolean isBootstrapBaseFilePresentForFileId(HoodieFileGroupId fgId);
/**
* Resets the bootstrap base file stream and overwrite with the new list.
*
* @param bootstrapBaseFileStream bootstrap Base File Stream
*/
abstract void resetBootstrapBaseFileMapping(Stream<BootstrapBaseFileMapping> bootstrapBaseFileStream);
/**
* Add bootstrap base file stream to store.
*
* @param bootstrapBaseFileStream bootstrap Base File Stream to be added
*/
abstract void addBootstrapBaseFileMapping(Stream<BootstrapBaseFileMapping> bootstrapBaseFileStream);
/**
* Remove bootstrap base file stream from store.
*
* @param bootstrapBaseFileStream bootstrap Base File Stream to be removed
*/
abstract void removeBootstrapBaseFileMapping(Stream<BootstrapBaseFileMapping> bootstrapBaseFileStream);
/**
* Return pending compaction operation for a file-group.
*
* @param fileGroupId File-Group Id
*/
protected abstract Option<BootstrapBaseFileMapping> getBootstrapBaseFile(HoodieFileGroupId fileGroupId);
/**
* Fetch all bootstrap data files.
*/
abstract Stream<BootstrapBaseFileMapping> fetchBootstrapBaseFiles();
/**
* Checks if partition is pre-loaded and available in store.
*
@@ -635,7 +739,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
*/
Stream<FileSlice> fetchLatestFileSliceInRange(List<String> commitsToReturn) {
return fetchAllStoredFileGroups().map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
.map(Option::get);
.map(Option::get).map(this::addBootstrapBaseFileIfPresent);
}
/**
@@ -645,16 +749,18 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
* @return file-slice stream
*/
Stream<FileSlice> fetchAllFileSlices(String partitionPath) {
return fetchAllStoredFileGroups(partitionPath).map(HoodieFileGroup::getAllFileSlices)
.flatMap(sliceList -> sliceList);
return fetchAllStoredFileGroups(partitionPath).map(this::addBootstrapBaseFileIfPresent)
.map(HoodieFileGroup::getAllFileSlices).flatMap(sliceList -> sliceList);
}
/**
* Default implementation for fetching latest base-files for the partition-path.
*/
Stream<HoodieBaseFile> fetchLatestBaseFiles(final String partitionPath) {
return fetchAllStoredFileGroups(partitionPath).map(this::getLatestBaseFile).filter(Option::isPresent)
.map(Option::get);
return fetchAllStoredFileGroups(partitionPath)
.map(fg -> Pair.of(fg.getFileGroupId(), getLatestBaseFile(fg)))
.filter(p -> p.getValue().isPresent())
.map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get()));
}
protected Option<HoodieBaseFile> getLatestBaseFile(HoodieFileGroup fileGroup) {
@@ -663,10 +769,12 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
/**
* Default implementation for fetching latest base-files across all partitions.
* Fetch latest base-files across all partitions.
*/
Stream<HoodieBaseFile> fetchLatestBaseFiles() {
return fetchAllStoredFileGroups().map(this::getLatestBaseFile).filter(Option::isPresent).map(Option::get);
private Stream<HoodieBaseFile> fetchLatestBaseFiles() {
return fetchAllStoredFileGroups().map(fg -> Pair.of(fg.getFileGroupId(), getLatestBaseFile(fg)))
.filter(p -> p.getValue().isPresent())
.map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get()));
}
/**

View File

@@ -41,6 +41,8 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
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";
public static final String FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION =
"hoodie.filesystem.view.spillable.bootstrap.base.file.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;
@@ -53,6 +55,7 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
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 Double DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE = 0.05;
private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB
/**
@@ -90,7 +93,7 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
public long getMaxMemoryForFileGroupMap() {
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
return totalMemory - getMaxMemoryForPendingCompaction();
return totalMemory - getMaxMemoryForPendingCompaction() - getMaxMemoryForBootstrapBaseFile();
}
public long getMaxMemoryForPendingCompaction() {
@@ -99,6 +102,14 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
.longValue();
}
public long getMaxMemoryForBootstrapBaseFile() {
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
long reservedForExternalDataFile =
new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION)))
.longValue();
return reservedForExternalDataFile;
}
public String getBaseStoreDir() {
return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR);
}
@@ -169,6 +180,11 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
return this;
}
public Builder withMemFractionForExternalDataFile(Double memFractionForExternalDataFile) {
props.setProperty(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, memFractionForExternalDataFile.toString());
return this;
}
public Builder withBaseStoreDir(String baseStorePath) {
props.setProperty(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath);
return this;
@@ -202,6 +218,8 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
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(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION),
FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE.toString());
setDefaultOnCondition(props, !props.containsKey(ROCKSDB_BASE_PATH_PROP), ROCKSDB_BASE_PATH_PROP,
DEFAULT_ROCKSDB_BASE_PATH);

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.common.table.view;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.BootstrapBaseFileMapping;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -58,6 +59,11 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
*/
protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingCompaction;
/**
* PartitionPath + File-Id to bootstrap base File (Index Only bootstrapped).
*/
protected Map<HoodieFileGroupId, BootstrapBaseFileMapping> fgIdToBootstrapBaseFile;
/**
* Flag to determine if closed.
*/
@@ -99,6 +105,7 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
protected void resetViewState() {
this.fgIdToPendingCompaction = null;
this.partitionToFileGroupsMap = null;
this.fgIdToBootstrapBaseFile = null;
}
protected Map<String, List<HoodieFileGroup>> createPartitionToFileGroups() {
@@ -110,6 +117,11 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
return fileIdToPendingCompaction;
}
protected Map<HoodieFileGroupId, BootstrapBaseFileMapping> createFileIdToBootstrapBaseFileMap(
Map<HoodieFileGroupId, BootstrapBaseFileMapping> fileGroupIdBootstrapBaseFileMap) {
return fileGroupIdBootstrapBaseFileMap;
}
/**
* Create a file system view, as of the given timeline, with the provided file statuses.
*/
@@ -185,6 +197,48 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
}
@Override
protected boolean isBootstrapBaseFilePresentForFileId(HoodieFileGroupId fgId) {
return fgIdToBootstrapBaseFile.containsKey(fgId);
}
@Override
void resetBootstrapBaseFileMapping(Stream<BootstrapBaseFileMapping> bootstrapBaseFileStream) {
// Build fileId to bootstrap Data File
this.fgIdToBootstrapBaseFile = createFileIdToBootstrapBaseFileMap(bootstrapBaseFileStream
.collect(Collectors.toMap(BootstrapBaseFileMapping::getFileGroupId, x -> x)));
}
@Override
void addBootstrapBaseFileMapping(Stream<BootstrapBaseFileMapping> bootstrapBaseFileStream) {
bootstrapBaseFileStream.forEach(bootstrapBaseFile -> {
ValidationUtils.checkArgument(!fgIdToBootstrapBaseFile.containsKey(bootstrapBaseFile.getFileGroupId()),
"Duplicate FileGroupId found in bootstrap base file mapping. FgId :"
+ bootstrapBaseFile.getFileGroupId());
fgIdToBootstrapBaseFile.put(bootstrapBaseFile.getFileGroupId(), bootstrapBaseFile);
});
}
@Override
void removeBootstrapBaseFileMapping(Stream<BootstrapBaseFileMapping> bootstrapBaseFileStream) {
bootstrapBaseFileStream.forEach(bootstrapBaseFile -> {
ValidationUtils.checkArgument(fgIdToBootstrapBaseFile.containsKey(bootstrapBaseFile.getFileGroupId()),
"Trying to remove a FileGroupId which is not found in bootstrap base file mapping. FgId :"
+ bootstrapBaseFile.getFileGroupId());
fgIdToBootstrapBaseFile.remove(bootstrapBaseFile.getFileGroupId());
});
}
@Override
protected Option<BootstrapBaseFileMapping> getBootstrapBaseFile(HoodieFileGroupId fileGroupId) {
return Option.ofNullable(fgIdToBootstrapBaseFile.get(fileGroupId));
}
@Override
Stream<BootstrapBaseFileMapping> fetchBootstrapBaseFiles() {
return fgIdToBootstrapBaseFile.values().stream();
}
@Override
protected Option<Pair<String, CompactionOperation>> getPendingCompactionOperationWithInstant(HoodieFileGroupId fgId) {
return Option.ofNullable(fgIdToPendingCompaction.get(fgId));
@@ -213,6 +267,7 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
super.reset();
partitionToFileGroupsMap = null;
fgIdToPendingCompaction = null;
fgIdToBootstrapBaseFile = null;
}
@Override

View File

@@ -91,7 +91,6 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
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);
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.common.table.view;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.BootstrapBaseFileMapping;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroup;
@@ -267,6 +268,62 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
schemaHelper.getPrefixForDataFileViewByPartition(partitionPath)).map(Pair::getValue);
}
@Override
protected boolean isBootstrapBaseFilePresentForFileId(HoodieFileGroupId fgId) {
return getBootstrapBaseFile(fgId).isPresent();
}
@Override
void resetBootstrapBaseFileMapping(Stream<BootstrapBaseFileMapping> bootstrapBaseFileStream) {
rocksDB.writeBatch(batch -> {
bootstrapBaseFileStream.forEach(externalBaseFile -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForBootstrapBaseFile(),
schemaHelper.getKeyForBootstrapBaseFile(externalBaseFile.getFileGroupId()), externalBaseFile);
});
LOG.info("Initializing external data file mapping. Count=" + batch.count());
});
}
@Override
void addBootstrapBaseFileMapping(Stream<BootstrapBaseFileMapping> bootstrapBaseFileStream) {
rocksDB.writeBatch(batch -> {
bootstrapBaseFileStream.forEach(externalBaseFile -> {
ValidationUtils.checkArgument(!isBootstrapBaseFilePresentForFileId(externalBaseFile.getFileGroupId()),
"Duplicate FileGroupId found in external data file. FgId :" + externalBaseFile.getFileGroupId());
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForBootstrapBaseFile(),
schemaHelper.getKeyForBootstrapBaseFile(externalBaseFile.getFileGroupId()), externalBaseFile);
});
});
}
@Override
void removeBootstrapBaseFileMapping(Stream<BootstrapBaseFileMapping> bootstrapBaseFileStream) {
rocksDB.writeBatch(batch -> {
bootstrapBaseFileStream.forEach(externalBaseFile -> {
ValidationUtils.checkArgument(
getBootstrapBaseFile(externalBaseFile.getFileGroupId()) != null,
"Trying to remove a FileGroupId which is not found in external data file mapping. FgId :"
+ externalBaseFile.getFileGroupId());
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForBootstrapBaseFile(),
schemaHelper.getKeyForBootstrapBaseFile(externalBaseFile.getFileGroupId()));
});
});
}
@Override
protected Option<BootstrapBaseFileMapping> getBootstrapBaseFile(HoodieFileGroupId fileGroupId) {
String lookupKey = schemaHelper.getKeyForBootstrapBaseFile(fileGroupId);
BootstrapBaseFileMapping externalBaseFile =
rocksDB.get(schemaHelper.getColFamilyForBootstrapBaseFile(), lookupKey);
return Option.ofNullable(externalBaseFile);
}
@Override
Stream<BootstrapBaseFileMapping> fetchBootstrapBaseFiles() {
return rocksDB.<BootstrapBaseFileMapping>prefixSearch(schemaHelper.getColFamilyForBootstrapBaseFile(), "")
.map(Pair::getValue);
}
@Override
Stream<HoodieFileGroup> fetchAllStoredFileGroups(String partitionPath) {
return getFileGroups(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),

View File

@@ -18,7 +18,14 @@
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.BootstrapBaseFileMapping;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -26,17 +33,9 @@ import org.apache.hudi.common.table.timeline.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.hadoop.fs.FileStatus;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Stream;
/**
* Table FileSystemView implementation where view is stored in spillable disk using fixed memory.
*/
@@ -46,6 +45,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
private final long maxMemoryForFileGroupMap;
private final long maxMemoryForPendingCompaction;
private final long maxMemoryForBootstrapBaseFile;
private final String baseStoreDir;
public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline,
@@ -53,6 +53,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
super(config.isIncrementalTimelineSyncEnabled());
this.maxMemoryForFileGroupMap = config.getMaxMemoryForFileGroupMap();
this.maxMemoryForPendingCompaction = config.getMaxMemoryForPendingCompaction();
this.maxMemoryForBootstrapBaseFile = config.getMaxMemoryForBootstrapBaseFile();
this.baseStoreDir = config.getBaseStoreDir();
init(metaClient, visibleActiveTimeline);
}
@@ -92,6 +93,22 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
}
}
@Override
protected Map<HoodieFileGroupId, BootstrapBaseFileMapping> createFileIdToBootstrapBaseFileMap(
Map<HoodieFileGroupId, BootstrapBaseFileMapping> fileGroupIdBootstrapBaseFileMap) {
try {
LOG.info("Creating bootstrap base File Map using external spillable Map. Max Mem=" + maxMemoryForBootstrapBaseFile
+ ", BaseDir=" + baseStoreDir);
new File(baseStoreDir).mkdirs();
Map<HoodieFileGroupId, BootstrapBaseFileMapping> pendingMap = new ExternalSpillableMap<>(
maxMemoryForBootstrapBaseFile, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>());
pendingMap.putAll(fileGroupIdBootstrapBaseFileMap);
return pendingMap;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public Stream<HoodieFileGroup> getAllFileGroups() {
return ((ExternalSpillableMap) partitionToFileGroupsMap).valueStream()
@@ -101,7 +118,11 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
@Override
Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations() {
return ((ExternalSpillableMap) fgIdToPendingCompaction).valueStream();
}
@Override
Stream<BootstrapBaseFileMapping> fetchBootstrapBaseFiles() {
return ((ExternalSpillableMap) fgIdToBootstrapBaseFile).valueStream();
}
@Override

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.common.util;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.model.BaseFile;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieFileGroupId;
@@ -72,6 +73,8 @@ public class CompactionUtils {
builder.setDeltaFilePaths(fileSlice.getLogFiles().map(lf -> lf.getPath().getName()).collect(Collectors.toList()));
if (fileSlice.getBaseFile().isPresent()) {
builder.setDataFilePath(fileSlice.getBaseFile().get().getFileName());
builder.setBootstrapFilePath(fileSlice.getBaseFile().get().getBootstrapBaseFile().map(BaseFile::getPath)
.orElse(null));
}
if (metricsCaptureFunction.isPresent()) {
@@ -106,6 +109,7 @@ public class CompactionUtils {
public static HoodieCompactionOperation buildHoodieCompactionOperation(CompactionOperation op) {
return HoodieCompactionOperation.newBuilder().setFileId(op.getFileId()).setBaseInstantTime(op.getBaseInstantTime())
.setPartitionPath(op.getPartitionPath())
.setBootstrapFilePath(op.getBootstrapFilePath().orElse(null))
.setDataFilePath(op.getDataFileName().isPresent() ? op.getDataFileName().get() : null)
.setDeltaFilePaths(op.getDeltaFileNames()).setMetrics(op.getMetrics()).build();
}

View File

@@ -190,7 +190,7 @@ public class ParquetUtils {
}
public static Schema readAvroSchema(Configuration configuration, Path parquetFilePath) {
return new AvroSchemaConverter().convert(readSchema(configuration, parquetFilePath));
return new AvroSchemaConverter(configuration).convert(readSchema(configuration, parquetFilePath));
}
/**

View File

@@ -45,16 +45,19 @@ public class RocksDBSchemaHelper {
private final String colFamilyForView;
private final String colFamilyForPendingCompaction;
private final String colFamilyForBootstrapBaseFile;
private final String colFamilyForStoredPartitions;
public RocksDBSchemaHelper(HoodieTableMetaClient metaClient) {
this.colFamilyForBootstrapBaseFile = "hudi_bootstrap_basefile_" + metaClient.getBasePath().replace("/", "_");
this.colFamilyForPendingCompaction = "hudi_pending_compaction_" + metaClient.getBasePath().replace("/", "_");
this.colFamilyForStoredPartitions = "hudi_partitions_" + metaClient.getBasePath().replace("/", "_");
this.colFamilyForView = "hudi_view_" + metaClient.getBasePath().replace("/", "_");
}
public List<String> getAllColumnFamilies() {
return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForStoredPartitions());
return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForBootstrapBaseFile(),
getColFamilyForStoredPartitions());
}
public String getKeyForPartitionLookup(String partition) {
@@ -65,6 +68,10 @@ public class RocksDBSchemaHelper {
return getPartitionFileIdBasedLookup(fgId);
}
public String getKeyForBootstrapBaseFile(HoodieFileGroupId fgId) {
return getPartitionFileIdBasedLookup(fgId);
}
public String getKeyForSliceView(HoodieFileGroup fileGroup, FileSlice slice) {
return getKeyForSliceView(fileGroup.getPartitionPath(), fileGroup.getFileGroupId().getFileId(),
slice.getBaseInstantTime());
@@ -111,6 +118,10 @@ public class RocksDBSchemaHelper {
return colFamilyForPendingCompaction;
}
public String getColFamilyForBootstrapBaseFile() {
return colFamilyForBootstrapBaseFile;
}
public String getColFamilyForStoredPartitions() {
return colFamilyForStoredPartitions;
}

View File

@@ -81,6 +81,10 @@ public class StringUtils {
return string == null ? "" : string;
}
public static String objToString(@Nullable Object obj) {
return obj == null ? null : obj.toString();
}
/**
* Returns the given string if it is nonempty; {@code null} otherwise.
*