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

@@ -69,14 +69,6 @@ public class DataSourceUtils {
private static final Logger LOG = LogManager.getLogger(DataSourceUtils.class);
/**
* 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 (obj == null) ? null : obj.toString();
}
/**
* Obtain value of the provided field, denoted by dot notation. e.g: a.b.c
*/
@@ -108,8 +100,8 @@ public class DataSourceUtils {
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()));
fieldName + "(Part -" + parts[i] + ") field not found in record. Acceptable fields were :"
+ valueNode.getSchema().getFields().stream().map(Field::name).collect(Collectors.toList()));
}
}
@@ -202,12 +194,12 @@ public class DataSourceUtils {
* @see HoodieWriteConfig#getUserDefinedBulkInsertPartitionerClass()
*/
private static Option<BulkInsertPartitioner> createUserDefinedBulkInsertPartitioner(HoodieWriteConfig config)
throws HoodieException {
throws HoodieException {
String bulkInsertPartitionerClass = config.getUserDefinedBulkInsertPartitionerClass();
try {
return StringUtils.isNullOrEmpty(bulkInsertPartitionerClass)
? Option.empty() :
Option.of((BulkInsertPartitioner) ReflectionUtils.loadClass(bulkInsertPartitionerClass));
? Option.empty() :
Option.of((BulkInsertPartitioner) ReflectionUtils.loadClass(bulkInsertPartitionerClass));
} catch (Throwable e) {
throw new HoodieException("Could not create UserDefinedBulkInsertPartitioner class " + bulkInsertPartitionerClass, e);
}
@@ -343,7 +335,7 @@ public class DataSourceUtils {
props.getString(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
SlashEncodedDayPartitionValueExtractor.class.getName());
hiveSyncConfig.useJdbc = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_USE_JDBC_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_USE_JDBC_OPT_VAL()));
DataSourceWriteOptions.DEFAULT_HIVE_USE_JDBC_OPT_VAL()));
return hiveSyncConfig;
}
}

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.bootstrap;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieFileStatus;
import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider;
import org.apache.hudi.common.bootstrap.FileStatusUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.rdd.RDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.SparkSession;
import java.io.IOException;
import java.util.List;
/**
* Spark Data frame based bootstrap input provider.
*/
public class SparkParquetBootstrapDataProvider extends FullRecordBootstrapDataProvider {
private final transient SparkSession sparkSession;
public SparkParquetBootstrapDataProvider(TypedProperties props,
JavaSparkContext jsc) {
super(props, jsc);
this.sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
}
@Override
public JavaRDD<HoodieRecord> generateInputRecordRDD(String tableName, String sourceBasePath,
List<Pair<String, List<HoodieFileStatus>>> partitionPathsWithFiles) {
String[] filePaths = partitionPathsWithFiles.stream().map(Pair::getValue)
.flatMap(f -> f.stream().map(fs -> FileStatusUtils.toPath(fs.getPath()).toUri().getPath()))
.toArray(String[]::new);
Dataset inputDataset = sparkSession.read().parquet(filePaths);
try {
KeyGenerator keyGenerator = DataSourceUtils.createKeyGenerator(props);
String structName = tableName + "_record";
String namespace = "hoodie." + tableName;
RDD<GenericRecord> genericRecords = AvroConversionUtils.createRdd(inputDataset, structName, namespace);
return genericRecords.toJavaRDD().map(gr -> {
String orderingVal = HoodieAvroUtils.getNestedFieldValAsString(
gr, props.getString("hoodie.datasource.write.precombine.field"), false);
try {
return DataSourceUtils.createHoodieRecord(gr, orderingVal, keyGenerator.getKey(gr),
props.getString("hoodie.datasource.write.payload.class"), "_hoodie_is_deleted");
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
});
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
}

View File

@@ -18,11 +18,8 @@
package org.apache.hudi.keygen;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
@@ -33,20 +30,14 @@ import java.util.stream.Collectors;
/**
* Complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
*/
public class ComplexKeyGenerator extends KeyGenerator {
public class ComplexKeyGenerator extends BuiltinKeyGenerator {
private static final String DEFAULT_PARTITION_PATH = "default";
private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
public static final String DEFAULT_RECORD_KEY_SEPARATOR = ":";
protected static final String NULL_RECORDKEY_PLACEHOLDER = "__null__";
protected static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__";
protected final List<String> recordKeyFields;
protected final List<String> partitionPathFields;
protected final boolean hiveStylePartitioning;
protected final boolean encodePartitionPath;
public ComplexKeyGenerator(TypedProperties props) {
super(props);
@@ -55,59 +46,26 @@ public class ComplexKeyGenerator extends KeyGenerator {
Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
this.hiveStylePartitioning = props.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL()));
this.encodePartitionPath = props.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL()));
}
@Override
public HoodieKey getKey(GenericRecord record) {
String recordKey = getRecordKey(record);
StringBuilder partitionPath = new StringBuilder();
for (String partitionPathField : partitionPathFields) {
partitionPath.append(getPartitionPath(record, partitionPathField));
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
}
partitionPath.deleteCharAt(partitionPath.length() - 1);
return new HoodieKey(recordKey, partitionPath.toString());
public String getRecordKey(GenericRecord record) {
return KeyGenUtils.getRecordKey(record, recordKeyFields);
}
String getPartitionPath(GenericRecord record, String partitionPathField) {
StringBuilder partitionPath = new StringBuilder();
String fieldVal = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField, true);
if (fieldVal == null || fieldVal.isEmpty()) {
partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + DEFAULT_PARTITION_PATH
: DEFAULT_PARTITION_PATH);
} else {
partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + fieldVal : fieldVal);
}
return partitionPath.toString();
}
String getRecordKey(GenericRecord record) {
boolean keyIsNullEmpty = true;
StringBuilder recordKey = new StringBuilder();
for (String recordKeyField : recordKeyFields) {
String recordKeyValue = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField, true);
if (recordKeyValue == null) {
recordKey.append(recordKeyField + ":" + NULL_RECORDKEY_PLACEHOLDER + ",");
} else if (recordKeyValue.isEmpty()) {
recordKey.append(recordKeyField + ":" + EMPTY_RECORDKEY_PLACEHOLDER + ",");
} else {
recordKey.append(recordKeyField + ":" + recordKeyValue + ",");
keyIsNullEmpty = false;
}
}
recordKey.deleteCharAt(recordKey.length() - 1);
if (keyIsNullEmpty) {
throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: "
+ recordKeyFields.toString() + " cannot be entirely null or empty.");
}
return recordKey.toString();
@Override
public String getPartitionPath(GenericRecord record) {
return KeyGenUtils.getRecordPartitionPath(record, partitionPathFields, hiveStylePartitioning, encodePartitionPath);
}
@Override
public List<String> getRecordKeyFields() {
return recordKeyFields;
}
@Override
public List<String> getPartitionPathFields() {
return partitionPathFields;
}

View File

@@ -19,7 +19,6 @@
package org.apache.hudi.keygen;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.avro.generic.GenericRecord;
@@ -47,7 +46,7 @@ import java.util.stream.Collectors;
*
* RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator.
*/
public class CustomKeyGenerator extends KeyGenerator {
public class CustomKeyGenerator extends BuiltinKeyGenerator {
protected final List<String> recordKeyFields;
protected final List<String> partitionPathFields;
@@ -71,15 +70,7 @@ public class CustomKeyGenerator extends KeyGenerator {
}
@Override
public HoodieKey getKey(GenericRecord record) {
//call function to get the record key
String recordKey = getRecordKey(record);
//call function to get the partition key based on the type for that partition path field
String partitionPath = getPartitionPath(record);
return new HoodieKey(recordKey, partitionPath);
}
private String getPartitionPath(GenericRecord record) {
public String getPartitionPath(GenericRecord record) {
if (partitionPathFields == null) {
throw new HoodieKeyException("Unable to find field names for partition path in cfg");
}
@@ -101,11 +92,11 @@ public class CustomKeyGenerator extends KeyGenerator {
PartitionKeyType keyType = PartitionKeyType.valueOf(fieldWithType[1].toUpperCase());
switch (keyType) {
case SIMPLE:
partitionPath.append(new SimpleKeyGenerator(properties).getPartitionPath(record, partitionPathField));
partitionPath.append(new SimpleKeyGenerator(properties, partitionPathField).getPartitionPath(record));
break;
case TIMESTAMP:
try {
partitionPath.append(new TimestampBasedKeyGenerator(properties).getPartitionPath(record, partitionPathField));
partitionPath.append(new TimestampBasedKeyGenerator(properties, partitionPathField).getPartitionPath(record));
} catch (IOException ioe) {
throw new HoodieDeltaStreamerException("Unable to initialise TimestampBasedKeyGenerator class");
}
@@ -121,11 +112,22 @@ public class CustomKeyGenerator extends KeyGenerator {
return partitionPath.toString();
}
private String getRecordKey(GenericRecord record) {
@Override
public String getRecordKey(GenericRecord record) {
if (recordKeyFields == null || recordKeyFields.isEmpty()) {
throw new HoodieKeyException("Unable to find field names for record key in cfg");
}
return recordKeyFields.size() == 1 ? new SimpleKeyGenerator(properties).getRecordKey(record) : new ComplexKeyGenerator(properties).getRecordKey(record);
}
@Override
public List<String> getRecordKeyFields() {
return recordKeyFields;
}
@Override
public List<String> getPartitionPathFields() {
return partitionPathFields;
}
}

View File

@@ -18,14 +18,12 @@
package org.apache.hudi.keygen;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
@@ -34,11 +32,9 @@ import java.util.stream.Collectors;
* Key generator for deletes using global indices. Global index deletes do not require partition value
* so this key generator avoids using partition value for generating HoodieKey.
*/
public class GlobalDeleteKeyGenerator extends KeyGenerator {
public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator {
private static final String EMPTY_PARTITION = "";
private static final String NULL_RECORDKEY_PLACEHOLDER = "__null__";
private static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__";
protected final List<String> recordKeyFields;
@@ -48,30 +44,22 @@ public class GlobalDeleteKeyGenerator extends KeyGenerator {
}
@Override
public HoodieKey getKey(GenericRecord record) {
return new HoodieKey(getRecordKey(record), EMPTY_PARTITION);
public String getRecordKey(GenericRecord record) {
return KeyGenUtils.getRecordKey(record, recordKeyFields);
}
String getRecordKey(GenericRecord record) {
boolean keyIsNullEmpty = true;
StringBuilder recordKey = new StringBuilder();
for (String recordKeyField : recordKeyFields) {
String recordKeyValue = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField, true);
if (recordKeyValue == null) {
recordKey.append(recordKeyField + ":" + NULL_RECORDKEY_PLACEHOLDER + ",");
} else if (recordKeyValue.isEmpty()) {
recordKey.append(recordKeyField + ":" + EMPTY_RECORDKEY_PLACEHOLDER + ",");
} else {
recordKey.append(recordKeyField + ":" + recordKeyValue + ",");
keyIsNullEmpty = false;
}
}
recordKey.deleteCharAt(recordKey.length() - 1);
if (keyIsNullEmpty) {
throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: "
+ recordKeyFields.toString() + " cannot be entirely null or empty.");
}
return recordKey.toString();
@Override
public String getPartitionPath(GenericRecord record) {
return EMPTY_PARTITION;
}
}
@Override
public List<String> getRecordKeyFields() {
return recordKeyFields;
}
@Override
public List<String> getPartitionPathFields() {
return new ArrayList<>();
}
}

View File

@@ -1,43 +0,0 @@
/*
* 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.keygen;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.avro.generic.GenericRecord;
import java.io.Serializable;
/**
* 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);
}

View File

@@ -19,17 +19,17 @@
package org.apache.hudi.keygen;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import java.util.ArrayList;
import java.util.List;
/**
* Simple Key generator for unpartitioned Hive Tables.
*/
public class NonpartitionedKeyGenerator extends KeyGenerator {
public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
private static final String EMPTY_PARTITION = "";
@@ -41,11 +41,12 @@ public class NonpartitionedKeyGenerator extends KeyGenerator {
}
@Override
public HoodieKey getKey(GenericRecord record) {
String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField, true);
if (recordKey == null || recordKey.isEmpty()) {
throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty.");
}
return new HoodieKey(recordKey, EMPTY_PARTITION);
public String getPartitionPath(GenericRecord record) {
return EMPTY_PARTITION;
}
@Override
public List<String> getPartitionPathFields() {
return new ArrayList<>();
}
}

View File

@@ -18,20 +18,18 @@
package org.apache.hudi.keygen;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import java.util.Arrays;
import java.util.List;
/**
* 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";
public class SimpleKeyGenerator extends BuiltinKeyGenerator {
protected final String recordKeyField;
@@ -39,46 +37,39 @@ public class SimpleKeyGenerator extends KeyGenerator {
protected final boolean hiveStylePartitioning;
protected final boolean encodePartitionPath;
public SimpleKeyGenerator(TypedProperties props) {
this(props, props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()));
}
public SimpleKeyGenerator(TypedProperties props, String partitionPathField) {
super(props);
this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
this.partitionPathField = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY());
this.partitionPathField = partitionPathField;
this.hiveStylePartitioning = props.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL()));
this.encodePartitionPath = props.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL()));
}
@Override
public HoodieKey getKey(GenericRecord record) {
String recordKey = getRecordKey(record);
String partitionPath = getPartitionPath(record, partitionPathField);
return new HoodieKey(recordKey, partitionPath);
public String getRecordKey(GenericRecord record) {
return KeyGenUtils.getRecordKey(record, recordKeyField);
}
String getPartitionPath(GenericRecord record, String partitionPathField) {
String partitionPath = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField, true);
if (partitionPath == null || partitionPath.isEmpty()) {
partitionPath = DEFAULT_PARTITION_PATH;
}
if (hiveStylePartitioning) {
partitionPath = partitionPathField + "=" + partitionPath;
}
return partitionPath;
@Override
public String getPartitionPath(GenericRecord record) {
return KeyGenUtils.getPartitionPath(record, partitionPathField, hiveStylePartitioning, encodePartitionPath);
}
String getRecordKey(GenericRecord record) {
String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField, true);
if (recordKey == null || recordKey.isEmpty()) {
throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty.");
}
return recordKey;
@Override
public List<String> getRecordKeyFields() {
return Arrays.asList(recordKeyField);
}
public String getRecordKeyField() {
return recordKeyField;
}
public String getPartitionPathField() {
return partitionPathField;
@Override
public List<String> getPartitionPathFields() {
return Arrays.asList(partitionPathField);
}
}

View File

@@ -19,9 +19,11 @@
package org.apache.hudi.keygen;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieDeltaStreamerException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.avro.generic.GenericRecord;
@@ -34,6 +36,9 @@ import org.joda.time.format.DateTimeFormatter;
import java.io.IOException;
import java.io.Serializable;
import java.io.UnsupportedEncodingException;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.util.concurrent.TimeUnit;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -58,6 +63,8 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
// https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html
private final DateTimeZone outputDateTimeZone;
protected final boolean encodePartitionPath;
/**
* Supported configs.
*/
@@ -82,7 +89,11 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
}
public TimestampBasedKeyGenerator(TypedProperties config) throws IOException {
super(config);
this(config, config.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()));
}
public TimestampBasedKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {
super(config, partitionPathField);
String dateTimeParserClass = config.getString(Config.DATE_TIME_PARSER_PROP, HoodieDateTimeParserImpl.class.getName());
this.parser = DataSourceUtils.createDateTimeParser(config, dateTimeParserClass);
this.outputDateTimeZone = parser.getOutputDateTimeZone();
@@ -108,17 +119,13 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
default:
timeUnit = null;
}
this.encodePartitionPath = config.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL()));
}
@Override
public HoodieKey getKey(GenericRecord record) {
String recordKey = getRecordKey(record);
String partitionPath = getPartitionPath(record, partitionPathField);
return new HoodieKey(recordKey, partitionPath);
}
String getPartitionPath(GenericRecord record, String partitionPathField) {
Object partitionVal = DataSourceUtils.getNestedFieldVal(record, partitionPathField, true);
public String getPartitionPath(GenericRecord record) {
Object partitionVal = HoodieAvroUtils.getNestedFieldVal(record, partitionPathField, true);
if (partitionVal == null) {
partitionVal = 1L;
}
@@ -146,11 +153,18 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
timeMs = inputFormatter.parseDateTime(partitionVal.toString()).getMillis();
} else {
throw new HoodieNotSupportedException(
"Unexpected type for partition field: " + partitionVal.getClass().getName());
"Unexpected type for partition field: " + partitionVal.getClass().getName());
}
DateTime timestamp = new DateTime(timeMs, outputDateTimeZone);
return hiveStylePartitioning ? partitionPathField + "=" + timestamp.toString(partitionFormatter)
: timestamp.toString(partitionFormatter);
String partitionPath = timestamp.toString(partitionFormatter);
if (encodePartitionPath) {
try {
partitionPath = URLEncoder.encode(partitionPath, StandardCharsets.UTF_8.toString());
} catch (UnsupportedEncodingException uoe) {
throw new HoodieException(uoe.getMessage(), uoe);
}
}
return hiveStylePartitioning ? partitionPathField + "=" + partitionPath : partitionPath;
} catch (Exception e) {
throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + partitionVal, e);
}

View File

@@ -214,7 +214,8 @@ object DataSourceWriteOptions {
*/
val HIVE_STYLE_PARTITIONING_OPT_KEY = "hoodie.datasource.write.hive_style_partitioning"
val DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL = "false"
val URL_ENCODE_PARTITIONING_OPT_KEY = "hoodie.datasource.write.partitionpath.urlencode"
val DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL = "false"
/**
* Key generator class, that implements will extract the key out of incoming record
*

View File

@@ -24,10 +24,11 @@ import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.hudi.client.{HoodieWriteClient, WriteStatus}
import org.apache.hudi.common.config.TypedProperties
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model.HoodieRecordPayload
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
import org.apache.hudi.config.HoodieWriteConfig
@@ -107,7 +108,7 @@ private[hudi] object HoodieSparkSqlWriter {
val keyGenerator = DataSourceUtils.createKeyGenerator(toProperties(parameters))
val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace)
val hoodieAllIncomingRecords = genericRecords.map(gr => {
val orderingVal = DataSourceUtils.getNestedFieldVal(gr, parameters(PRECOMBINE_FIELD_OPT_KEY), false)
val orderingVal = HoodieAvroUtils.getNestedFieldVal(gr, parameters(PRECOMBINE_FIELD_OPT_KEY), false)
.asInstanceOf[Comparable[_]]
DataSourceUtils.createHoodieRecord(gr,
orderingVal, keyGenerator.getKey(gr),
@@ -131,8 +132,9 @@ private[hudi] object HoodieSparkSqlWriter {
// Create the table if not present
if (!exists) {
HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, tableType,
tblName, "archived", parameters(PAYLOAD_CLASS_OPT_KEY))
//FIXME(bootstrap): bootstrapIndexClass needs to be set when bootstrap index class is integrated.
HoodieTableMetaClient.initTableTypeWithBootstrap(sparkContext.hadoopConfiguration, path.get, HoodieTableType.valueOf(tableType),
tblName, "archived", parameters(PAYLOAD_CLASS_OPT_KEY), null, null, null)
}
// Create a HoodieWriteClient & issue the write.