1
0

[HUDI-114]: added option to overwrite payload implementation in hoodie.properties file

This commit is contained in:
pratyakshsharma
2020-01-09 18:43:36 +05:30
committed by Balaji Varadarajan
parent 5af3dc6aed
commit 3c90d252cc
13 changed files with 129 additions and 39 deletions

View File

@@ -0,0 +1,61 @@
/*
* 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.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.generic.GenericRecord;
import java.io.IOException;
import java.io.Serializable;
/**
* Base class for all AVRO record based payloads, that can be ordered based on a field.
*/
public abstract class BaseAvroPayload implements Serializable {
/**
* Avro data extracted from the source converted to bytes.
*/
public final byte[] recordBytes;
/**
* For purposes of preCombining.
*/
protected final Comparable orderingVal;
/**
* Instantiate {@link BaseAvroPayload}.
*
* @param record Generic record for the payload.
* @param orderingVal {@link Comparable} to be used in pre combine.
*/
public BaseAvroPayload(GenericRecord record, Comparable orderingVal) {
try {
this.recordBytes = record != null ? HoodieAvroUtils.avroToBytes(record) : new byte[0];
} catch (IOException io) {
throw new HoodieIOException("Cannot convert GenericRecord to bytes", io);
}
this.orderingVal = orderingVal;
if (orderingVal == null) {
throw new HoodieException("Ordering value is null for record: " + record);
}
}
}

View File

@@ -0,0 +1,82 @@
/*
* 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.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.common.util.Option;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import java.io.IOException;
/**
* Default payload used for delta streamer.
* <p>
* 1. preCombine - Picks the latest delta record for a key, based on an ordering field 2.
* combineAndGetUpdateValue/getInsertValue - Simply overwrites storage with latest delta record
*/
public class OverwriteWithLatestAvroPayload extends BaseAvroPayload
implements HoodieRecordPayload<OverwriteWithLatestAvroPayload> {
/**
*
*/
public OverwriteWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) {
super(record, orderingVal);
}
public OverwriteWithLatestAvroPayload(Option<GenericRecord> record) {
this(record.isPresent() ? record.get() : null, (record1) -> 0); // natural order
}
@Override
public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload another) {
// pick the payload with greatest ordering value
if (another.orderingVal.compareTo(orderingVal) > 0) {
return another;
} else {
return this;
}
}
@Override
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException {
Option<IndexedRecord> recordOption = getInsertValue(schema);
if (!recordOption.isPresent()) {
return Option.empty();
}
GenericRecord genericRecord = (GenericRecord) recordOption.get();
// combining strategy here trivially ignores currentValue on disk and writes this record
Object deleteMarker = genericRecord.get("_hoodie_is_deleted");
if (deleteMarker instanceof Boolean && (boolean) deleteMarker) {
return Option.empty();
} else {
return Option.of(genericRecord);
}
}
@Override
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return recordBytes.length == 0 ? Option.empty() : Option.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
}
}

View File

@@ -18,9 +18,9 @@
package org.apache.hudi.common.table;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.model.TimelineLayoutVersion;
import org.apache.hudi.exception.HoodieIOException;
@@ -62,13 +62,12 @@ public class HoodieTableConfig implements Serializable {
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET;
public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
public static final String DEFAULT_PAYLOAD_CLASS = OverwriteWithLatestAvroPayload.class.getName();
public static final Integer DEFAULT_TIMELINE_LAYOUT_VERSION = TimelineLayoutVersion.VERSION_0;
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
public static final String DEFAULT_ARCHIVELOG_FOLDER = "";
private Properties props;
public HoodieTableConfig(FileSystem fs, String metaPath) {
public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) {
Properties props = new Properties();
Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
LOG.info("Loading table properties from " + propertyPath);
@@ -76,6 +75,13 @@ public class HoodieTableConfig implements Serializable {
try (FSDataInputStream inputStream = fs.open(propertyPath)) {
props.load(inputStream);
}
if (props.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME) && payloadClassName != null
&& !props.getProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME).equals(payloadClassName)) {
props.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
props.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
}
}
} catch (IOException e) {
throw new HoodieIOException("Could not load Hoodie properties from " + propertyPath, e);
}
@@ -109,7 +115,7 @@ public class HoodieTableConfig implements Serializable {
if (!properties.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
properties.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name());
}
if (properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME) == HoodieTableType.MERGE_ON_READ.name()
if (properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME).equals(HoodieTableType.MERGE_ON_READ.name())
&& !properties.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME)) {
properties.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS);
}

View File

@@ -89,13 +89,22 @@ public class HoodieTableMetaClient implements Serializable {
this(conf, basePath, false);
}
public HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad) {
this(conf, basePath, loadActiveTimelineOnLoad, ConsistencyGuardConfig.newBuilder().build(),
Option.of(TimelineLayoutVersion.CURR_LAYOUT_VERSION));
public HoodieTableMetaClient(Configuration conf, String basePath, String payloadClassName) {
this(conf, basePath, false, ConsistencyGuardConfig.newBuilder().build(), Option.of(TimelineLayoutVersion.CURR_LAYOUT_VERSION),
payloadClassName);
}
public HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad,
ConsistencyGuardConfig consistencyGuardConfig, Option<TimelineLayoutVersion> layoutVersion)
ConsistencyGuardConfig consistencyGuardConfig, Option<TimelineLayoutVersion> layoutVersion) {
this(conf, basePath, loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, null);
}
public HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad) {
this(conf, basePath, loadActiveTimelineOnLoad, ConsistencyGuardConfig.newBuilder().build(), Option.of(TimelineLayoutVersion.CURR_LAYOUT_VERSION), null);
}
public HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad,
ConsistencyGuardConfig consistencyGuardConfig, Option<TimelineLayoutVersion> layoutVersion, String payloadClassName)
throws TableNotFoundException {
LOG.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath;
@@ -106,7 +115,7 @@ public class HoodieTableMetaClient implements Serializable {
Path metaPathDir = new Path(this.metaPath);
this.fs = getFs();
TableNotFoundException.checkTableValidity(fs, basePathDir, metaPathDir);
this.tableConfig = new HoodieTableConfig(fs, metaPath);
this.tableConfig = new HoodieTableConfig(fs, metaPath, payloadClassName);
this.tableType = tableConfig.getTableType();
this.timelineLayoutVersion = layoutVersion.orElse(tableConfig.getTimelineLayoutVersion());
this.loadActiveTimelineOnLoad = loadActiveTimelineOnLoad;
@@ -127,7 +136,7 @@ public class HoodieTableMetaClient implements Serializable {
public static HoodieTableMetaClient reload(HoodieTableMetaClient oldMetaClient) {
return new HoodieTableMetaClient(oldMetaClient.hadoopConf.get(), oldMetaClient.basePath,
oldMetaClient.loadActiveTimelineOnLoad, oldMetaClient.consistencyGuardConfig,
Option.of(oldMetaClient.timelineLayoutVersion));
Option.of(oldMetaClient.timelineLayoutVersion), null);
}
/**
@@ -284,9 +293,9 @@ public class HoodieTableMetaClient implements Serializable {
* Helper method to initialize a table, with given basePath, tableType, name, archiveFolder.
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, String tableType,
String tableName, String archiveLogFolder) throws IOException {
String tableName, String archiveLogFolder, String payloadClassName) throws IOException {
return initTableType(hadoopConf, basePath, HoodieTableType.valueOf(tableType), tableName,
archiveLogFolder, null, null);
archiveLogFolder, payloadClassName, null);
}
/**