HUDI-123 Rename code packages/constants to org.apache.hudi (#830)
- Rename com.uber.hoodie to org.apache.hudi - Flag to pass com.uber.hoodie Input formats for hoodie-sync - Works with HUDI demo. - Also tested for backwards compatibility with datasets built by com.uber.hoodie packages - Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
committed by
vinoth chandar
parent
722b6be04a
commit
a4f9d7575f
@@ -0,0 +1,58 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
/**
|
||||
* 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
|
||||
*/
|
||||
protected final byte [] recordBytes;
|
||||
|
||||
/**
|
||||
* For purposes of preCombining
|
||||
*/
|
||||
protected final Comparable orderingVal;
|
||||
|
||||
/**
|
||||
* @param record
|
||||
* @param orderingVal
|
||||
*/
|
||||
public BaseAvroPayload(GenericRecord record, Comparable orderingVal) {
|
||||
try {
|
||||
this.recordBytes = HoodieAvroUtils.avroToBytes(record);
|
||||
} 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,81 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
/**
|
||||
* Complex key generator, which takes names of fields to be used for recordKey and partitionPath as
|
||||
* configs.
|
||||
*/
|
||||
public class ComplexKeyGenerator extends KeyGenerator {
|
||||
|
||||
private static final String DEFAULT_PARTITION_PATH = "default";
|
||||
|
||||
private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
|
||||
|
||||
protected final List<String> recordKeyFields;
|
||||
|
||||
protected final List<String> partitionPathFields;
|
||||
|
||||
public ComplexKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.recordKeyFields = Arrays.asList(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(","));
|
||||
this.partitionPathFields = Arrays.asList(props
|
||||
.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(","));
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
if (recordKeyFields == null || partitionPathFields == null) {
|
||||
throw new HoodieException(
|
||||
"Unable to find field names for record key or partition path in cfg");
|
||||
}
|
||||
StringBuilder recordKey = new StringBuilder();
|
||||
for (String recordKeyField : recordKeyFields) {
|
||||
recordKey.append(recordKeyField + ":" + DataSourceUtils.getNestedFieldValAsString(record, recordKeyField) + ",");
|
||||
}
|
||||
recordKey.deleteCharAt(recordKey.length() - 1);
|
||||
StringBuilder partitionPath = new StringBuilder();
|
||||
try {
|
||||
for (String partitionPathField : partitionPathFields) {
|
||||
partitionPath.append(DataSourceUtils.getNestedFieldValAsString(record, partitionPathField));
|
||||
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
|
||||
}
|
||||
partitionPath.deleteCharAt(partitionPath.length() - 1);
|
||||
} catch (HoodieException e) {
|
||||
partitionPath = partitionPath.append(DEFAULT_PARTITION_PATH);
|
||||
}
|
||||
|
||||
return new HoodieKey(recordKey.toString(), partitionPath.toString());
|
||||
}
|
||||
|
||||
public List<String> getRecordKeyFields() {
|
||||
return recordKeyFields;
|
||||
}
|
||||
|
||||
public List<String> getPartitionPathFields() {
|
||||
return partitionPathFields;
|
||||
}
|
||||
}
|
||||
250
hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java
Normal file
250
hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java
Normal file
@@ -0,0 +1,250 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.DatasetNotFoundException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.PartitionValueExtractor;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Utilities used throughout the data source
|
||||
*/
|
||||
public class DataSourceUtils {
|
||||
|
||||
/**
|
||||
* 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) {
|
||||
Object obj = getNestedFieldVal(record, fieldName);
|
||||
return (obj == null) ? null : obj.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain value of the provided field, denoted by dot notation. e.g: a.b.c
|
||||
*/
|
||||
public static Object getNestedFieldVal(GenericRecord record, String fieldName) {
|
||||
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) {
|
||||
return 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;
|
||||
}
|
||||
}
|
||||
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()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a key generator class via reflection, passing in any configs needed.
|
||||
*
|
||||
* If the class name of key generator is configured through the properties file, i.e., {@code
|
||||
* props}, use the corresponding key generator class; otherwise, use the default key generator
|
||||
* class specified in {@code DataSourceWriteOptions}.
|
||||
*/
|
||||
public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOException {
|
||||
String keyGeneratorClass = props.getString(
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
|
||||
DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL()
|
||||
);
|
||||
try {
|
||||
return (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, props);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not load key generator class " + keyGeneratorClass, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a partition value extractor class via reflection, passing in any configs needed
|
||||
*/
|
||||
public static PartitionValueExtractor createPartitionExtractor(String partitionExtractorClass) {
|
||||
try {
|
||||
return (PartitionValueExtractor) ReflectionUtils.loadClass(partitionExtractorClass);
|
||||
} catch (Throwable e) {
|
||||
throw new HoodieException("Could not load partition extractor class " + partitionExtractorClass, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a payload class via reflection, passing in an ordering/precombine value.
|
||||
*/
|
||||
public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record,
|
||||
Comparable orderingVal) throws IOException {
|
||||
try {
|
||||
return (HoodieRecordPayload) ReflectionUtils
|
||||
.loadClass(payloadClass, new Class<?>[]{GenericRecord.class, Comparable.class}, record, orderingVal);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not create payload for class: " + payloadClass, e);
|
||||
}
|
||||
}
|
||||
|
||||
public static void checkRequiredProperties(TypedProperties props,
|
||||
List<String> checkPropNames) {
|
||||
checkPropNames.stream().forEach(prop -> {
|
||||
if (!props.containsKey(prop)) {
|
||||
throw new HoodieNotSupportedException("Required property " + prop + " is missing");
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr,
|
||||
String basePath, String tblName, Map<String, String> parameters) throws Exception {
|
||||
|
||||
// inline compaction is on by default for MOR
|
||||
boolean inlineCompact = parameters.get(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY())
|
||||
.equals(DataSourceWriteOptions.MOR_STORAGE_TYPE_OPT_VAL());
|
||||
|
||||
// insert/bulk-insert combining to be true, if filtering for duplicates
|
||||
boolean combineInserts = Boolean.parseBoolean(parameters.get(
|
||||
DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY()));
|
||||
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder()
|
||||
.withPath(basePath).withAutoCommit(false)
|
||||
.combineInput(combineInserts, true)
|
||||
.withSchema(schemaStr).forTable(tblName).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withPayloadClass(parameters.get(
|
||||
DataSourceWriteOptions
|
||||
.PAYLOAD_CLASS_OPT_KEY()))
|
||||
.withInlineCompaction(inlineCompact)
|
||||
.build())
|
||||
// override above with Hoodie configs specified as options.
|
||||
.withProps(parameters).build();
|
||||
|
||||
return new HoodieWriteClient<>(jssc, writeConfig, true);
|
||||
}
|
||||
|
||||
|
||||
public static JavaRDD<WriteStatus> doWriteOperation(HoodieWriteClient client,
|
||||
JavaRDD<HoodieRecord> hoodieRecords, String commitTime, String operation) {
|
||||
if (operation.equals(DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL())) {
|
||||
return client.bulkInsert(hoodieRecords, commitTime);
|
||||
} else if (operation.equals(DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())) {
|
||||
return client.insert(hoodieRecords, commitTime);
|
||||
} else {
|
||||
//default is upsert
|
||||
return client.upsert(hoodieRecords, commitTime);
|
||||
}
|
||||
}
|
||||
|
||||
public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal,
|
||||
HoodieKey hKey, String payloadClass) throws IOException {
|
||||
HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal);
|
||||
return new HoodieRecord<>(hKey, payload);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
|
||||
JavaRDD<HoodieRecord> incomingHoodieRecords,
|
||||
HoodieWriteConfig writeConfig, Option<EmbeddedTimelineService> timelineService) throws Exception {
|
||||
HoodieReadClient client = null;
|
||||
try {
|
||||
client = new HoodieReadClient<>(jssc, writeConfig, timelineService);
|
||||
return client.tagLocation(incomingHoodieRecords)
|
||||
.filter(r -> !((HoodieRecord<HoodieRecordPayload>) r).isCurrentLocationKnown());
|
||||
} catch (DatasetNotFoundException e) {
|
||||
// this will be executed when there is no hoodie dataset yet
|
||||
// so no dups to drop
|
||||
return incomingHoodieRecords;
|
||||
} finally {
|
||||
if (null != client) {
|
||||
client.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
|
||||
JavaRDD<HoodieRecord> incomingHoodieRecords,
|
||||
Map<String, String> parameters,
|
||||
Option<EmbeddedTimelineService> timelineService)
|
||||
throws Exception {
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig
|
||||
.newBuilder()
|
||||
.withPath(parameters.get("path"))
|
||||
.withProps(parameters).build();
|
||||
return dropDuplicates(jssc, incomingHoodieRecords, writeConfig, timelineService);
|
||||
}
|
||||
|
||||
public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath) {
|
||||
checkRequiredProperties(props, Arrays.asList(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY()));
|
||||
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
|
||||
hiveSyncConfig.basePath = basePath;
|
||||
hiveSyncConfig.usePreApacheInputFormat =
|
||||
props.getBoolean(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY(),
|
||||
Boolean.valueOf(DataSourceWriteOptions.DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL()));
|
||||
hiveSyncConfig.assumeDatePartitioning =
|
||||
props.getBoolean(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION_OPT_KEY(),
|
||||
Boolean.valueOf(DataSourceWriteOptions.DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL()));
|
||||
hiveSyncConfig.databaseName = props.getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(),
|
||||
DataSourceWriteOptions.DEFAULT_HIVE_DATABASE_OPT_VAL());
|
||||
hiveSyncConfig.tableName = props.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY());
|
||||
hiveSyncConfig.hiveUser = props.getString(DataSourceWriteOptions.HIVE_USER_OPT_KEY(),
|
||||
DataSourceWriteOptions.DEFAULT_HIVE_USER_OPT_VAL());
|
||||
hiveSyncConfig.hivePass = props.getString(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(),
|
||||
DataSourceWriteOptions.DEFAULT_HIVE_PASS_OPT_VAL());
|
||||
hiveSyncConfig.jdbcUrl = props.getString(DataSourceWriteOptions.HIVE_URL_OPT_KEY(),
|
||||
DataSourceWriteOptions.DEFAULT_HIVE_URL_OPT_VAL());
|
||||
hiveSyncConfig.partitionFields =
|
||||
props.getStringList(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), ",", new ArrayList<>());
|
||||
hiveSyncConfig.partitionValueExtractorClass =
|
||||
props.getString(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
|
||||
SlashEncodedDayPartitionValueExtractor.class.getName());
|
||||
return hiveSyncConfig;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
* Empty payload used for deletions
|
||||
*/
|
||||
public class EmptyHoodieRecordPayload implements HoodieRecordPayload<EmptyHoodieRecordPayload> {
|
||||
|
||||
public EmptyHoodieRecordPayload(GenericRecord record, Comparable orderingVal) { }
|
||||
|
||||
@Override
|
||||
public EmptyHoodieRecordPayload preCombine(EmptyHoodieRecordPayload another) {
|
||||
return another;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> getInsertValue(Schema schema) {
|
||||
return Option.empty();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
|
||||
/**
|
||||
* List of helpers to aid, construction of instanttime for read and write operations using
|
||||
* datasource
|
||||
*/
|
||||
public class HoodieDataSourceHelpers {
|
||||
|
||||
/**
|
||||
* Checks if the Hoodie dataset has new data since given timestamp. This can be subsequently fed
|
||||
* to an incremental view read, to perform incremental processing.
|
||||
*/
|
||||
public static boolean hasNewCommits(FileSystem fs, String basePath, String commitTimestamp) {
|
||||
return listCommitsSince(fs, basePath, commitTimestamp).size() > 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a list of instant times that have occurred, from the given instant timestamp.
|
||||
*/
|
||||
public static List<String> listCommitsSince(FileSystem fs, String basePath,
|
||||
String instantTimestamp) {
|
||||
HoodieTimeline timeline = allCompletedCommitsCompactions(fs, basePath);
|
||||
return timeline.findInstantsAfter(instantTimestamp, Integer.MAX_VALUE).getInstants()
|
||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the last successful write operation's instant time
|
||||
*/
|
||||
public static String latestCommit(FileSystem fs, String basePath) {
|
||||
HoodieTimeline timeline = allCompletedCommitsCompactions(fs, basePath);
|
||||
return timeline.lastInstant().get().getTimestamp();
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain all the commits, compactions that have occurred on the timeline, whose instant times
|
||||
* could be fed into the datasource options.
|
||||
*/
|
||||
public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true);
|
||||
if (metaClient.getTableType().equals(HoodieTableType.MERGE_ON_READ)) {
|
||||
return metaClient.getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION));
|
||||
} else {
|
||||
return metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
}
|
||||
}
|
||||
}
|
||||
43
hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
Normal file
43
hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
Normal file
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import java.io.Serializable;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
|
||||
/**
|
||||
* Abstract class to extend for plugging in extraction of
|
||||
* {@link HoodieKey}
|
||||
* from an Avro record
|
||||
*/
|
||||
public abstract class KeyGenerator implements Serializable {
|
||||
|
||||
protected transient TypedProperties config;
|
||||
|
||||
protected KeyGenerator(TypedProperties config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a Hoodie Key out of provided generic record.
|
||||
*/
|
||||
public abstract HoodieKey getKey(GenericRecord record);
|
||||
}
|
||||
@@ -0,0 +1,41 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
|
||||
/**
|
||||
* Simple Key generator for unpartitioned Hive Tables
|
||||
*/
|
||||
public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
|
||||
|
||||
private static final String EMPTY_PARTITION = "";
|
||||
|
||||
public NonpartitionedKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField);
|
||||
return new HoodieKey(recordKey, EMPTY_PARTITION);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,71 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
* 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> {
|
||||
|
||||
/**
|
||||
* @param record
|
||||
* @param orderingVal
|
||||
*/
|
||||
public OverwriteWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) {
|
||||
super(record, orderingVal);
|
||||
}
|
||||
|
||||
public OverwriteWithLatestAvroPayload(Option<GenericRecord> record) {
|
||||
this(record.get(), (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 {
|
||||
// combining strategy here trivially ignores currentValue on disk and writes this record
|
||||
return getInsertValue(schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
|
||||
return Option.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,63 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
/**
|
||||
* Simple key generator, which takes names of fields to be used for recordKey and partitionPath as
|
||||
* configs.
|
||||
*/
|
||||
public class SimpleKeyGenerator extends KeyGenerator {
|
||||
|
||||
private static final String DEFAULT_PARTITION_PATH = "default";
|
||||
|
||||
protected final String recordKeyField;
|
||||
|
||||
protected final String partitionPathField;
|
||||
|
||||
public SimpleKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
|
||||
this.partitionPathField = props
|
||||
.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
if (recordKeyField == null || partitionPathField == null) {
|
||||
throw new HoodieException(
|
||||
"Unable to find field names for record key or partition path in cfg");
|
||||
}
|
||||
|
||||
String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField);
|
||||
String partitionPath;
|
||||
try {
|
||||
partitionPath = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField);
|
||||
} catch (HoodieException e) {
|
||||
// if field is not found, lump it into default partition
|
||||
partitionPath = DEFAULT_PARTITION_PATH;
|
||||
}
|
||||
|
||||
return new HoodieKey(recordKey, partitionPath);
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user