1
0

[HUDI-912] Refactor and relocate KeyGenerator to support more engines (#2200)

* [HUDI-912] Refactor and relocate KeyGenerator to support more engines

* Rename KeyGenerators
This commit is contained in:
wangxianghu
2020-11-03 05:12:51 +08:00
committed by GitHub
parent 59f995a3f5
commit d160abb437
38 changed files with 1038 additions and 524 deletions

View File

@@ -0,0 +1,33 @@
/*
* 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;
/**
* Exception thrown for any higher level errors when {@link org.apache.hudi.keygen.KeyGeneratorInterface} is generating
* a {@link org.apache.hudi.common.model.HoodieKey}.
*/
public class HoodieKeyGeneratorException extends HoodieException {
public HoodieKeyGeneratorException(String msg, Throwable e) {
super(msg, e);
}
public HoodieKeyGeneratorException(String msg) {
super(msg);
}
}

View File

@@ -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.keygen;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import java.util.List;
import java.util.stream.Collectors;
public abstract class BaseKeyGenerator extends KeyGenerator {
protected List<String> recordKeyFields;
protected List<String> partitionPathFields;
protected final boolean encodePartitionPath;
protected final boolean hiveStylePartitioning;
protected BaseKeyGenerator(TypedProperties config) {
super(config);
this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY,
Boolean.parseBoolean(KeyGeneratorOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL));
this.hiveStylePartitioning = config.getBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY,
Boolean.parseBoolean(KeyGeneratorOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL));
}
/**
* Generate a record Key out of provided generic record.
*/
public abstract String getRecordKey(GenericRecord record);
/**
* Generate a partition path out of provided generic record.
*/
public abstract String getPartitionPath(GenericRecord record);
/**
* Generate a Hoodie Key out of provided generic record.
*/
@Override
public final HoodieKey getKey(GenericRecord record) {
if (getRecordKeyFields() == null || getPartitionPathFields() == null) {
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
}
return new HoodieKey(getRecordKey(record), getPartitionPath(record));
}
@Override
public final List<String> getRecordKeyFieldNames() {
// For nested columns, pick top level column name
return getRecordKeyFields().stream().map(k -> {
int idx = k.indexOf('.');
return idx > 0 ? k.substring(0, idx) : k;
}).collect(Collectors.toList());
}
public List<String> getRecordKeyFields() {
return recordKeyFields;
}
public List<String> getPartitionPathFields() {
return partitionPathFields;
}
}

View File

@@ -0,0 +1,50 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import java.util.Arrays;
import java.util.stream.Collectors;
/**
* Avro complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
*/
public class ComplexAvroKeyGenerator extends BaseKeyGenerator {
public static final String DEFAULT_RECORD_KEY_SEPARATOR = ":";
public ComplexAvroKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
}
@Override
public String getRecordKey(GenericRecord record) {
return KeyGenUtils.getRecordKey(record, getRecordKeyFields());
}
@Override
public String getPartitionPath(GenericRecord record) {
return KeyGenUtils.getRecordPartitionPath(record, getPartitionPathFields(), hiveStylePartitioning, encodePartitionPath);
}
}

View File

@@ -0,0 +1,124 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.hudi.exception.HoodieKeyGeneratorException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import java.io.IOException;
import java.util.Arrays;
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:
* <p>
* properties.put("hoodie.datasource.write.partitionpath.field", "field1:PartitionKeyType1,field2:PartitionKeyType2").
* <p>
* The complete partition path is created as <value for field1 basis PartitionKeyType1>/<value for field2 basis PartitionKeyType2> and so on.
* <p>
* 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.
* <p>
* RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator.
*/
public class CustomAvroKeyGenerator extends BaseKeyGenerator {
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 CustomAvroKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
}
@Override
public String getPartitionPath(GenericRecord record) {
if (getPartitionPathFields() == 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 (getPartitionPathFields().size() == 1 && getPartitionPathFields().get(0).isEmpty()) {
return "";
}
for (String field : getPartitionPathFields()) {
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 SimpleAvroKeyGenerator(config, partitionPathField).getPartitionPath(record));
break;
case TIMESTAMP:
try {
partitionPath.append(new TimestampBasedAvroKeyGenerator(config, partitionPathField).getPartitionPath(record));
} catch (IOException e) {
throw new HoodieKeyGeneratorException("Unable to initialise TimestampBasedKeyGenerator class");
}
break;
default:
throw new HoodieKeyGeneratorException("Please provide valid PartitionKeyType with fields! You provided: " + keyType);
}
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
}
partitionPath.deleteCharAt(partitionPath.length() - 1);
return partitionPath.toString();
}
@Override
public String getRecordKey(GenericRecord record) {
validateRecordKeyFields();
return getRecordKeyFields().size() == 1
? new SimpleAvroKeyGenerator(config).getRecordKey(record)
: new ComplexAvroKeyGenerator(config).getRecordKey(record);
}
private void validateRecordKeyFields() {
if (getRecordKeyFields() == null || getRecordKeyFields().isEmpty()) {
throw new HoodieKeyException("Unable to find field names for record key in cfg");
}
}
public String getDefaultPartitionPathSeparator() {
return DEFAULT_PARTITION_PATH_SEPARATOR;
}
public String getSplitRegex() {
return SPLIT_REGEX;
}
}

View File

@@ -0,0 +1,59 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
/**
* Avro 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 GlobalAvroDeleteKeyGenerator extends BaseKeyGenerator {
private static final String EMPTY_PARTITION = "";
public GlobalAvroDeleteKeyGenerator(TypedProperties config) {
super(config);
this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(","));
}
@Override
public String getRecordKey(GenericRecord record) {
return KeyGenUtils.getRecordKey(record, getRecordKeyFields());
}
@Override
public String getPartitionPath(GenericRecord record) {
return EMPTY_PARTITION;
}
@Override
public List<String> getPartitionPathFields() {
return new ArrayList<>();
}
public String getEmptyPartition() {
return EMPTY_PARTITION;
}
}

View File

@@ -0,0 +1,139 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.keygen.parser.AbstractHoodieDateTimeParser;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.util.List;
public class KeyGenUtils {
protected static final String NULL_RECORDKEY_PLACEHOLDER = "__null__";
protected static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__";
protected static final String DEFAULT_PARTITION_PATH = "default";
protected static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
public static String getRecordKey(GenericRecord record, List<String> recordKeyFields) {
boolean keyIsNullEmpty = true;
StringBuilder recordKey = new StringBuilder();
for (String recordKeyField : recordKeyFields) {
String recordKeyValue = HoodieAvroUtils.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();
}
public static String getRecordPartitionPath(GenericRecord record, List<String> partitionPathFields,
boolean hiveStylePartitioning, boolean encodePartitionPath) {
if (partitionPathFields.isEmpty()) {
return "";
}
StringBuilder partitionPath = new StringBuilder();
for (String partitionPathField : partitionPathFields) {
String fieldVal = HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathField, true);
if (fieldVal == null || fieldVal.isEmpty()) {
partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + DEFAULT_PARTITION_PATH
: DEFAULT_PARTITION_PATH);
} else {
if (encodePartitionPath) {
try {
fieldVal = URLEncoder.encode(fieldVal, StandardCharsets.UTF_8.toString());
} catch (UnsupportedEncodingException uoe) {
throw new HoodieException(uoe.getMessage(), uoe);
}
}
partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + fieldVal : fieldVal);
}
partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
}
partitionPath.deleteCharAt(partitionPath.length() - 1);
return partitionPath.toString();
}
public static String getRecordKey(GenericRecord record, String recordKeyField) {
String recordKey = HoodieAvroUtils.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;
}
public static String getPartitionPath(GenericRecord record, String partitionPathField,
boolean hiveStylePartitioning, boolean encodePartitionPath) {
String partitionPath = HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathField, true);
if (partitionPath == null || partitionPath.isEmpty()) {
partitionPath = DEFAULT_PARTITION_PATH;
}
if (encodePartitionPath) {
try {
partitionPath = URLEncoder.encode(partitionPath, StandardCharsets.UTF_8.toString());
} catch (UnsupportedEncodingException uoe) {
throw new HoodieException(uoe.getMessage(), uoe);
}
}
if (hiveStylePartitioning) {
partitionPath = partitionPathField + "=" + partitionPath;
}
return partitionPath;
}
/**
* Create a date time parser class for TimestampBasedKeyGenerator, passing in any configs needed.
*/
public static AbstractHoodieDateTimeParser createDateTimeParser(TypedProperties props, String parserClass) throws IOException {
try {
return (AbstractHoodieDateTimeParser) ReflectionUtils.loadClass(parserClass, props);
} catch (Throwable e) {
throw new IOException("Could not load date time parser class " + parserClass, e);
}
}
public static void checkRequiredProperties(TypedProperties props, List<String> checkPropNames) {
checkPropNames.forEach(prop -> {
if (!props.containsKey(prop)) {
throw new HoodieNotSupportedException("Required property " + prop + " is missing");
}
});
}
}

View File

@@ -0,0 +1,59 @@
/*
* 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.ApiMaturityLevel;
import org.apache.hudi.PublicAPIClass;
import org.apache.hudi.PublicAPIMethod;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.avro.generic.GenericRecord;
import java.util.List;
/**
* Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
*/
@PublicAPIClass(maturity = ApiMaturityLevel.STABLE)
public abstract class KeyGenerator implements KeyGeneratorInterface {
protected TypedProperties config;
protected KeyGenerator(TypedProperties config) {
this.config = config;
}
/**
* Generate a Hoodie Key out of provided generic record.
*/
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
public abstract HoodieKey getKey(GenericRecord record);
/**
* Used during bootstrap, to project out only the record key fields from bootstrap source dataset.
*
* @return list of field names, when concatenated make up the record key.
*/
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public List<String> getRecordKeyFieldNames() {
throw new UnsupportedOperationException("Bootstrap not supported for key generator. "
+ "Please override this method in your custom key generator.");
}
}

View File

@@ -0,0 +1,51 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import java.util.ArrayList;
import java.util.List;
/**
* Avro simple Key generator for unpartitioned Hive Tables.
*/
public class NonpartitionedAvroKeyGenerator extends SimpleAvroKeyGenerator {
private static final String EMPTY_PARTITION = "";
private static final List<String> EMPTY_PARTITION_FIELD_LIST = new ArrayList<>();
public NonpartitionedAvroKeyGenerator(TypedProperties props) {
super(props);
}
@Override
public String getPartitionPath(GenericRecord record) {
return EMPTY_PARTITION;
}
@Override
public List<String> getPartitionPathFields() {
return EMPTY_PARTITION_FIELD_LIST;
}
public String getEmptyPartition() {
return EMPTY_PARTITION;
}
}

View File

@@ -0,0 +1,57 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import java.util.Collections;
/**
* Avro simple key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
*/
public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
public SimpleAvroKeyGenerator(TypedProperties props) {
this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
}
SimpleAvroKeyGenerator(TypedProperties props, String partitionPathField) {
this(props, null, partitionPathField);
}
SimpleAvroKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) {
super(props);
this.recordKeyFields = recordKeyField == null
? Collections.emptyList()
: Collections.singletonList(recordKeyField);
this.partitionPathFields = Collections.singletonList(partitionPathField);
}
@Override
public String getRecordKey(GenericRecord record) {
return KeyGenUtils.getRecordKey(record, getRecordKeyFields().get(0));
}
@Override
public String getPartitionPath(GenericRecord record) {
return KeyGenUtils.getPartitionPath(record, getPartitionPathFields().get(0), hiveStylePartitioning, encodePartitionPath);
}
}

View File

@@ -0,0 +1,230 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieKeyGeneratorException;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.keygen.parser.AbstractHoodieDateTimeParser;
import org.apache.hudi.keygen.parser.HoodieDateTimeParserImpl;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormat;
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.TimeZone;
import java.util.concurrent.TimeUnit;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
/**
* Avro Key generator, that relies on timestamps for partitioning field. Still picks record key by name.
*/
public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator {
public enum TimestampType implements Serializable {
UNIX_TIMESTAMP, DATE_STRING, MIXED, EPOCHMILLISECONDS, SCALAR
}
private final TimeUnit timeUnit;
private final TimestampType timestampType;
private final String outputDateFormat;
private transient Option<DateTimeFormatter> inputFormatter;
private transient DateTimeFormatter partitionFormatter;
private final AbstractHoodieDateTimeParser parser;
// TimeZone detailed settings reference
// https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html
private final DateTimeZone inputDateTimeZone;
private final DateTimeZone outputDateTimeZone;
protected final boolean encodePartitionPath;
/**
* Supported configs.
*/
public static class Config {
// One value from TimestampType above
public static final String TIMESTAMP_TYPE_FIELD_PROP = "hoodie.deltastreamer.keygen.timebased.timestamp.type";
public static final String INPUT_TIME_UNIT =
"hoodie.deltastreamer.keygen.timebased.timestamp.scalar.time.unit";
//This prop can now accept list of input date formats.
public static final String TIMESTAMP_INPUT_DATE_FORMAT_PROP =
"hoodie.deltastreamer.keygen.timebased.input.dateformat";
public static final String TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMITER_REGEX_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformat.list.delimiter.regex";
public static final String TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.input.timezone";
public static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP =
"hoodie.deltastreamer.keygen.timebased.output.dateformat";
//still keeping this prop for backward compatibility so that functionality for existing users does not break.
public static final String TIMESTAMP_TIMEZONE_FORMAT_PROP =
"hoodie.deltastreamer.keygen.timebased.timezone";
public static final String TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.timezone";
static final String DATE_TIME_PARSER_PROP = "hoodie.deltastreamer.keygen.datetime.parser.class";
}
public TimestampBasedAvroKeyGenerator(TypedProperties config) throws IOException {
this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
}
TimestampBasedAvroKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {
this(config, null, partitionPathField);
}
TimestampBasedAvroKeyGenerator(TypedProperties config, String recordKeyField, String partitionPathField) throws IOException {
super(config, recordKeyField, partitionPathField);
String dateTimeParserClass = config.getString(Config.DATE_TIME_PARSER_PROP, HoodieDateTimeParserImpl.class.getName());
this.parser = KeyGenUtils.createDateTimeParser(config, dateTimeParserClass);
this.inputDateTimeZone = parser.getInputDateTimeZone();
this.outputDateTimeZone = parser.getOutputDateTimeZone();
this.outputDateFormat = parser.getOutputDateFormat();
this.timestampType = TimestampType.valueOf(config.getString(Config.TIMESTAMP_TYPE_FIELD_PROP));
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;
}
this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY,
Boolean.parseBoolean(KeyGeneratorOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL));
}
@Override
public String getPartitionPath(GenericRecord record) {
Object partitionVal = HoodieAvroUtils.getNestedFieldVal(record, getPartitionPathFields().get(0), true);
if (partitionVal == null) {
partitionVal = getDefaultPartitionVal();
}
try {
return getPartitionPath(partitionVal);
} catch (Exception e) {
throw new HoodieKeyGeneratorException("Unable to parse input partition field :" + partitionVal, e);
}
}
/**
* Set default value to partitionVal if the input value of partitionPathField is null.
*/
public Object getDefaultPartitionVal() {
Object result = 1L;
if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) {
// since partitionVal is null, we can set a default value of any format as TIMESTAMP_INPUT_DATE_FORMAT_PROP
// configured, here we take the first.
// {Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP} won't be null, it has been checked in the initialization process of
// inputFormatter
String delimiter = parser.getConfigInputDateFormatDelimiter();
String format = config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP, "").split(delimiter)[0];
// if both input and output timeZone are not configured, use GMT.
if (null != inputDateTimeZone) {
return new DateTime(result, inputDateTimeZone).toString(format);
} else if (null != outputDateTimeZone) {
return new DateTime(result, outputDateTimeZone).toString(format);
} else {
return new DateTime(result, DateTimeZone.forTimeZone(TimeZone.getTimeZone("GMT"))).toString(format);
}
}
return result;
}
/**
* The function takes care of lazily initialising dateTimeFormatter variables only once.
*/
private void initIfNeeded() {
if (this.inputFormatter == null) {
this.inputFormatter = parser.getInputFormatter();
}
if (this.partitionFormatter == null) {
this.partitionFormatter = DateTimeFormat.forPattern(outputDateFormat);
if (this.outputDateTimeZone != null) {
partitionFormatter = partitionFormatter.withZone(outputDateTimeZone);
}
}
}
/**
* Parse and fetch partition path based on data type.
*
* @param partitionVal partition path object value fetched from record/row
* @return the parsed partition path based on data type
*/
public String getPartitionPath(Object partitionVal) {
initIfNeeded();
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) {
if (!inputFormatter.isPresent()) {
throw new HoodieException("Missing inputformatter. Ensure " + Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP + " config is set when timestampType is DATE_STRING or MIXED!");
}
DateTime parsedDateTime = inputFormatter.get().parseDateTime(partitionVal.toString());
if (this.outputDateTimeZone == null) {
// Use the timezone that came off the date that was passed in, if it had one
partitionFormatter = partitionFormatter.withZone(parsedDateTime.getZone());
}
timeMs = inputFormatter.get().parseDateTime(partitionVal.toString()).getMillis();
} else {
throw new HoodieNotSupportedException(
"Unexpected type for partition field: " + partitionVal.getClass().getName());
}
DateTime timestamp = new DateTime(timeMs, outputDateTimeZone);
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 ? getPartitionPathFields().get(0) + "=" + partitionPath : partitionPath;
}
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);
}
}

View File

@@ -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.keygen.constant;
public class KeyGeneratorOptions {
/**
* Flag to indicate whether to use Hive style partitioning.
* If set true, the names of partition folders follow <partition_column_name>=<partition_value> format.
* By default false (the names of partition folders are only partition values)
*/
public static final String URL_ENCODE_PARTITIONING_OPT_KEY = "hoodie.datasource.write.partitionpath.urlencode";
public static final String DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL = "false";
public static final String HIVE_STYLE_PARTITIONING_OPT_KEY = "hoodie.datasource.write.hive_style_partitioning";
public static final String DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL = "false";
/**
* Record key field. Value to be used as the `recordKey` component of `HoodieKey`. Actual value
* will be obtained by invoking .toString() on the field value. Nested fields can be specified using
* the dot notation eg: `a.b.c`
*/
public static final String RECORDKEY_FIELD_OPT_KEY = "hoodie.datasource.write.recordkey.field";
public static final String PARTITIONPATH_FIELD_OPT_KEY = "hoodie.datasource.write.partitionpath.field";
}

View File

@@ -0,0 +1,73 @@
/*
* 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.parser;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.Config;
import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormatter;
import java.io.Serializable;
public abstract class AbstractHoodieDateTimeParser implements Serializable {
protected final TypedProperties config;
protected final String configInputDateFormatDelimiter;
public AbstractHoodieDateTimeParser(TypedProperties config) {
this.config = config;
this.configInputDateFormatDelimiter = initInputDateFormatDelimiter();
}
private String initInputDateFormatDelimiter() {
String inputDateFormatDelimiter = config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMITER_REGEX_PROP, ",").trim();
inputDateFormatDelimiter = inputDateFormatDelimiter.isEmpty() ? "," : inputDateFormatDelimiter;
return inputDateFormatDelimiter;
}
/**
* Returns the output date format in which the partition paths will be created for the hudi dataset.
*/
public String getOutputDateFormat() {
return config.getString(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP);
}
/**
* Returns input formats in which datetime based values might be coming in incoming records.
*/
public abstract Option<DateTimeFormatter> getInputFormatter();
/**
* Returns the datetime zone one should expect the incoming values into.
*/
public abstract DateTimeZone getInputDateTimeZone();
/**
* Returns the datetime zone using which the final partition paths for hudi dataset are created.
*/
public abstract DateTimeZone getOutputDateTimeZone();
/**
* Returns the input date format delimiter, comma by default.
*/
public String getConfigInputDateFormatDelimiter() {
return this.configInputDateFormatDelimiter;
}
}

View File

@@ -0,0 +1,113 @@
/*
* 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.parser;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.TimestampType;
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.Config;
import org.apache.hudi.keygen.KeyGenUtils;
import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.DateTimeFormatterBuilder;
import org.joda.time.format.DateTimeParser;
import java.util.Arrays;
import java.util.Collections;
import java.util.TimeZone;
public class HoodieDateTimeParserImpl extends AbstractHoodieDateTimeParser {
private String configInputDateFormatList;
// TimeZone detailed settings reference
// https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html
private final DateTimeZone inputDateTimeZone;
public HoodieDateTimeParserImpl(TypedProperties config) {
super(config);
KeyGenUtils.checkRequiredProperties(config, Arrays.asList(Config.TIMESTAMP_TYPE_FIELD_PROP, Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP));
this.inputDateTimeZone = getInputDateTimeZone();
}
private DateTimeFormatter getInputDateFormatter() {
if (this.configInputDateFormatList.isEmpty()) {
throw new IllegalArgumentException(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP + " configuration is required");
}
DateTimeFormatter formatter = new DateTimeFormatterBuilder()
.append(
null,
Arrays.stream(
this.configInputDateFormatList.split(super.configInputDateFormatDelimiter))
.map(String::trim)
.map(DateTimeFormat::forPattern)
.map(DateTimeFormatter::getParser)
.toArray(DateTimeParser[]::new))
.toFormatter();
if (this.inputDateTimeZone != null) {
formatter = formatter.withZone(this.inputDateTimeZone);
} else {
formatter = formatter.withOffsetParsed();
}
return formatter;
}
@Override
public String getOutputDateFormat() {
return config.getString(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP);
}
@Override
public Option<DateTimeFormatter> getInputFormatter() {
TimestampType timestampType = TimestampType.valueOf(config.getString(Config.TIMESTAMP_TYPE_FIELD_PROP));
if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) {
KeyGenUtils.checkRequiredProperties(config,
Collections.singletonList(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
this.configInputDateFormatList = config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP, "");
return Option.of(getInputDateFormatter());
}
return Option.empty();
}
@Override
public DateTimeZone getInputDateTimeZone() {
String inputTimeZone;
if (config.containsKey(Config.TIMESTAMP_TIMEZONE_FORMAT_PROP)) {
inputTimeZone = config.getString(Config.TIMESTAMP_TIMEZONE_FORMAT_PROP, "GMT");
} else {
inputTimeZone = config.getString(Config.TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP, "");
}
return !inputTimeZone.trim().isEmpty() ? DateTimeZone.forTimeZone(TimeZone.getTimeZone(inputTimeZone)) : null;
}
@Override
public DateTimeZone getOutputDateTimeZone() {
String outputTimeZone;
if (config.containsKey(Config.TIMESTAMP_TIMEZONE_FORMAT_PROP)) {
outputTimeZone = config.getString(Config.TIMESTAMP_TIMEZONE_FORMAT_PROP, "GMT");
} else {
outputTimeZone = config.getString(Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP, "");
}
return !outputTimeZone.trim().isEmpty() ? DateTimeZone.forTimeZone(TimeZone.getTimeZone(outputTimeZone)) : null;
}
}