[HUDI-728]: Implemented custom key generator (#1433)
This commit is contained in:
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.exception;
|
||||
|
||||
public class HoodieDeltaStreamerException extends HoodieException {
|
||||
|
||||
public HoodieDeltaStreamerException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieDeltaStreamerException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -49,21 +49,39 @@ public class ComplexKeyGenerator extends KeyGenerator {
|
||||
|
||||
public ComplexKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.recordKeyFields = Arrays.asList(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(","))
|
||||
.stream().map(String::trim).collect(Collectors.toList());
|
||||
this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
this.partitionPathFields =
|
||||
Arrays.asList(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(","))
|
||||
.stream().map(String::trim).collect(Collectors.toList());
|
||||
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()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
if (recordKeyFields == null || partitionPathFields == null) {
|
||||
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
|
||||
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());
|
||||
}
|
||||
|
||||
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) {
|
||||
@@ -80,30 +98,8 @@ public class ComplexKeyGenerator extends KeyGenerator {
|
||||
recordKey.deleteCharAt(recordKey.length() - 1);
|
||||
if (keyIsNullEmpty) {
|
||||
throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: "
|
||||
+ recordKeyFields.toString() + " cannot be entirely null or empty.");
|
||||
+ recordKeyFields.toString() + " cannot be entirely null or empty.");
|
||||
}
|
||||
|
||||
StringBuilder partitionPath = new StringBuilder();
|
||||
for (String partitionPathField : partitionPathFields) {
|
||||
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);
|
||||
}
|
||||
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
|
||||
}
|
||||
partitionPath.deleteCharAt(partitionPath.length() - 1);
|
||||
|
||||
return new HoodieKey(recordKey.toString(), partitionPath.toString());
|
||||
}
|
||||
|
||||
public List<String> getRecordKeyFields() {
|
||||
return recordKeyFields;
|
||||
}
|
||||
|
||||
public List<String> getPartitionPathFields() {
|
||||
return partitionPathFields;
|
||||
return recordKey.toString();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,126 @@
|
||||
/*
|
||||
* 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.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.exception.HoodieDeltaStreamerException;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields.
|
||||
* Similarly partition path can be configured to have multiple fields or only one field. This class expects value for prop
|
||||
* "hoodie.datasource.write.partitionpath.field" in a specific format. For example:
|
||||
*
|
||||
* properties.put("hoodie.datasource.write.partitionpath.field", "field1:PartitionKeyType1,field2:PartitionKeyType2").
|
||||
*
|
||||
* The complete partition path is created as <value for field1 basis PartitionKeyType1>/<value for field2 basis PartitionKeyType2> and so on.
|
||||
*
|
||||
* Few points to consider:
|
||||
* 1. If you want to customize some partition path field on a timestamp basis, you can use field1:timestampBased
|
||||
* 2. If you simply want to have the value of your configured field in the partition path, use field1:simple
|
||||
* 3. If you want your table to be non partitioned, simply leave it as blank.
|
||||
*
|
||||
* RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator.
|
||||
*/
|
||||
public class CustomKeyGenerator extends KeyGenerator {
|
||||
|
||||
protected final List<String> recordKeyFields;
|
||||
protected final List<String> partitionPathFields;
|
||||
protected final TypedProperties properties;
|
||||
private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
|
||||
private static final String SPLIT_REGEX = ":";
|
||||
|
||||
/**
|
||||
* Used as a part of config in CustomKeyGenerator.java.
|
||||
*/
|
||||
public enum PartitionKeyType {
|
||||
SIMPLE, TIMESTAMP
|
||||
}
|
||||
|
||||
public CustomKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.properties = props;
|
||||
this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
this.partitionPathFields =
|
||||
Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@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) {
|
||||
if (partitionPathFields == null) {
|
||||
throw new HoodieKeyException("Unable to find field names for partition path in cfg");
|
||||
}
|
||||
|
||||
String partitionPathField;
|
||||
StringBuilder partitionPath = new StringBuilder();
|
||||
|
||||
//Corresponds to no partition case
|
||||
if (partitionPathFields.size() == 1 && partitionPathFields.get(0).isEmpty()) {
|
||||
return "";
|
||||
}
|
||||
for (String field : partitionPathFields) {
|
||||
String[] fieldWithType = field.split(SPLIT_REGEX);
|
||||
if (fieldWithType.length != 2) {
|
||||
throw new HoodieKeyException("Unable to find field names for partition path in proper format");
|
||||
}
|
||||
|
||||
partitionPathField = fieldWithType[0];
|
||||
PartitionKeyType keyType = PartitionKeyType.valueOf(fieldWithType[1].toUpperCase());
|
||||
switch (keyType) {
|
||||
case SIMPLE:
|
||||
partitionPath.append(new SimpleKeyGenerator(properties).getPartitionPath(record, partitionPathField));
|
||||
break;
|
||||
case TIMESTAMP:
|
||||
partitionPath.append(new TimestampBasedKeyGenerator(properties).getPartitionPath(record, partitionPathField));
|
||||
break;
|
||||
default:
|
||||
throw new HoodieDeltaStreamerException("Please provide valid PartitionKeyType with fields! You provided: " + keyType);
|
||||
}
|
||||
|
||||
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
|
||||
}
|
||||
partitionPath.deleteCharAt(partitionPath.length() - 1);
|
||||
|
||||
return partitionPath.toString();
|
||||
}
|
||||
|
||||
private 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);
|
||||
}
|
||||
}
|
||||
@@ -28,6 +28,7 @@ import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Key generator for deletes using global indices. Global index deletes do not require partition value
|
||||
@@ -43,15 +44,15 @@ public class GlobalDeleteKeyGenerator extends KeyGenerator {
|
||||
|
||||
public GlobalDeleteKeyGenerator(TypedProperties config) {
|
||||
super(config);
|
||||
this.recordKeyFields = Arrays.asList(config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(","));
|
||||
this.recordKeyFields = Arrays.stream(config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
if (recordKeyFields == null) {
|
||||
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
|
||||
}
|
||||
return new HoodieKey(getRecordKey(record), EMPTY_PARTITION);
|
||||
}
|
||||
|
||||
String getRecordKey(GenericRecord record) {
|
||||
boolean keyIsNullEmpty = true;
|
||||
StringBuilder recordKey = new StringBuilder();
|
||||
for (String recordKeyField : recordKeyFields) {
|
||||
@@ -68,9 +69,9 @@ public class GlobalDeleteKeyGenerator extends KeyGenerator {
|
||||
recordKey.deleteCharAt(recordKey.length() - 1);
|
||||
if (keyIsNullEmpty) {
|
||||
throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: "
|
||||
+ recordKeyFields.toString() + " cannot be entirely null or empty.");
|
||||
+ recordKeyFields.toString() + " cannot be entirely null or empty.");
|
||||
}
|
||||
|
||||
return new HoodieKey(recordKey.toString(), EMPTY_PARTITION);
|
||||
return recordKey.toString();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
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;
|
||||
@@ -28,12 +29,15 @@ import org.apache.avro.generic.GenericRecord;
|
||||
/**
|
||||
* Simple Key generator for unpartitioned Hive Tables.
|
||||
*/
|
||||
public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
|
||||
public class NonpartitionedKeyGenerator extends KeyGenerator {
|
||||
|
||||
private static final String EMPTY_PARTITION = "";
|
||||
|
||||
protected final String recordKeyField;
|
||||
|
||||
public NonpartitionedKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -49,15 +49,12 @@ public class SimpleKeyGenerator extends KeyGenerator {
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
if (recordKeyField == null || partitionPathField == null) {
|
||||
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
|
||||
}
|
||||
|
||||
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.");
|
||||
}
|
||||
String recordKey = getRecordKey(record);
|
||||
String partitionPath = getPartitionPath(record, partitionPathField);
|
||||
return new HoodieKey(recordKey, partitionPath);
|
||||
}
|
||||
|
||||
String getPartitionPath(GenericRecord record, String partitionPathField) {
|
||||
String partitionPath = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField, true);
|
||||
if (partitionPath == null || partitionPath.isEmpty()) {
|
||||
partitionPath = DEFAULT_PARTITION_PATH;
|
||||
@@ -66,6 +63,14 @@ public class SimpleKeyGenerator extends KeyGenerator {
|
||||
partitionPath = partitionPathField + "=" + partitionPath;
|
||||
}
|
||||
|
||||
return new HoodieKey(recordKey, partitionPath);
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,156 @@
|
||||
/*
|
||||
* 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.DataSourceUtils;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.exception.HoodieDeltaStreamerException;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.TimeZone;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static java.util.concurrent.TimeUnit.MILLISECONDS;
|
||||
import static java.util.concurrent.TimeUnit.SECONDS;
|
||||
|
||||
/**
|
||||
* Key generator, that relies on timestamps for partitioning field. Still picks record key by name.
|
||||
*/
|
||||
public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
|
||||
|
||||
enum TimestampType implements Serializable {
|
||||
UNIX_TIMESTAMP, DATE_STRING, MIXED, EPOCHMILLISECONDS, SCALAR
|
||||
}
|
||||
|
||||
private final TimeUnit timeUnit;
|
||||
|
||||
private final TimestampType timestampType;
|
||||
|
||||
private SimpleDateFormat inputDateFormat;
|
||||
|
||||
private final String outputDateFormat;
|
||||
|
||||
// TimeZone detailed settings reference
|
||||
// https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html
|
||||
private final TimeZone timeZone;
|
||||
|
||||
/**
|
||||
* Supported configs.
|
||||
*/
|
||||
static class Config {
|
||||
|
||||
// One value from TimestampType above
|
||||
private static final String TIMESTAMP_TYPE_FIELD_PROP = "hoodie.deltastreamer.keygen.timebased.timestamp.type";
|
||||
private static final String INPUT_TIME_UNIT =
|
||||
"hoodie.deltastreamer.keygen.timebased.timestamp.scalar.time.unit";
|
||||
private static final String TIMESTAMP_INPUT_DATE_FORMAT_PROP =
|
||||
"hoodie.deltastreamer.keygen.timebased.input.dateformat";
|
||||
private static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP =
|
||||
"hoodie.deltastreamer.keygen.timebased.output.dateformat";
|
||||
private static final String TIMESTAMP_TIMEZONE_FORMAT_PROP =
|
||||
"hoodie.deltastreamer.keygen.timebased.timezone";
|
||||
}
|
||||
|
||||
public TimestampBasedKeyGenerator(TypedProperties config) {
|
||||
super(config);
|
||||
DataSourceUtils.checkRequiredProperties(config,
|
||||
Arrays.asList(Config.TIMESTAMP_TYPE_FIELD_PROP, Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP));
|
||||
this.timestampType = TimestampType.valueOf(config.getString(Config.TIMESTAMP_TYPE_FIELD_PROP));
|
||||
this.outputDateFormat = config.getString(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP);
|
||||
this.timeZone = TimeZone.getTimeZone(config.getString(Config.TIMESTAMP_TIMEZONE_FORMAT_PROP, "GMT"));
|
||||
|
||||
if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) {
|
||||
DataSourceUtils.checkRequiredProperties(config,
|
||||
Collections.singletonList(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
||||
this.inputDateFormat = new SimpleDateFormat(config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
||||
this.inputDateFormat.setTimeZone(timeZone);
|
||||
}
|
||||
|
||||
switch (this.timestampType) {
|
||||
case EPOCHMILLISECONDS:
|
||||
timeUnit = MILLISECONDS;
|
||||
break;
|
||||
case UNIX_TIMESTAMP:
|
||||
timeUnit = SECONDS;
|
||||
break;
|
||||
case SCALAR:
|
||||
String timeUnitStr = config.getString(Config.INPUT_TIME_UNIT, TimeUnit.SECONDS.toString());
|
||||
timeUnit = TimeUnit.valueOf(timeUnitStr.toUpperCase());
|
||||
break;
|
||||
default:
|
||||
timeUnit = null;
|
||||
}
|
||||
}
|
||||
|
||||
@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);
|
||||
if (partitionVal == null) {
|
||||
partitionVal = 1L;
|
||||
}
|
||||
SimpleDateFormat partitionPathFormat = new SimpleDateFormat(outputDateFormat);
|
||||
partitionPathFormat.setTimeZone(timeZone);
|
||||
|
||||
try {
|
||||
long timeMs;
|
||||
if (partitionVal instanceof Double) {
|
||||
timeMs = convertLongTimeToMillis(((Double) partitionVal).longValue());
|
||||
} else if (partitionVal instanceof Float) {
|
||||
timeMs = convertLongTimeToMillis(((Float) partitionVal).longValue());
|
||||
} else if (partitionVal instanceof Long) {
|
||||
timeMs = convertLongTimeToMillis((Long) partitionVal);
|
||||
} else if (partitionVal instanceof CharSequence) {
|
||||
timeMs = inputDateFormat.parse(partitionVal.toString()).getTime();
|
||||
} else {
|
||||
throw new HoodieNotSupportedException(
|
||||
"Unexpected type for partition field: " + partitionVal.getClass().getName());
|
||||
}
|
||||
Date timestamp = new Date(timeMs);
|
||||
|
||||
return hiveStylePartitioning ? partitionPathField + "=" + partitionPathFormat.format(timestamp)
|
||||
: partitionPathFormat.format(timestamp);
|
||||
} catch (ParseException pe) {
|
||||
throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + partitionVal, pe);
|
||||
}
|
||||
}
|
||||
|
||||
private long convertLongTimeToMillis(Long partitionVal) {
|
||||
if (timeUnit == null) {
|
||||
// should not be possible
|
||||
throw new RuntimeException(Config.INPUT_TIME_UNIT + " is not specified but scalar it supplied as time value");
|
||||
}
|
||||
|
||||
return MILLISECONDS.convert(partitionVal, timeUnit);
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user