vinothchandar commented on a change in pull request #1597: URL: https://github.com/apache/incubator-hudi/pull/1597#discussion_r421263630
########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/keygen/MultiFormatTimestampBasedKeyGenerator.java ########## @@ -0,0 +1,183 @@ +/* + * 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.utilities.keygen; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.util.TypedProperties; +import org.apache.hudi.exception.HoodieKeyException; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.TimeZone; + +import org.joda.time.DateTime; +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; + +/** + * Key generator, that relies on timestamps for partitioning field. Still picks record key by name. + */ +public class MultiFormatTimestampBasedKeyGenerator extends SimpleKeyGenerator { + + enum TimestampType implements Serializable { + UNIX_TIMESTAMP, DATE_STRING, MIXED, EPOCHMILLISECONDS + } + + private final TimestampType timestampType; + private final String outputDateFormat; + private final String configInputDateFormatList; + private final String configInputDateFormatDelimiter; + + + // TimeZone detailed settings reference + // https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html + private final DateTimeZone inputDateTimeZone; + private final DateTimeZone outputDateTimeZone; + + /** + * 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 TIMESTAMP_INPUT_DATE_FORMAT_LIST_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformatlist"; + private static final String TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMETER_REGEX_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformatlistdelimiterregex"; + private static final String TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.input.timezone"; + + private static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.dateformat"; + private static final String TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.timezone"; + } + + public MultiFormatTimestampBasedKeyGenerator(TypedProperties config) { + super(config); + DataSourceUtils.checkRequiredProperties(config, + Arrays.asList(Config.TIMESTAMP_TYPE_FIELD_PROP, Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP)); + + String inputTimeZone = config.getString(Config.TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP, ""); + String outputTimeZone = config.getString(Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP, ""); + + this.timestampType = TimestampType.valueOf(config.getString(Config.TIMESTAMP_TYPE_FIELD_PROP)); + this.outputDateFormat = config.getString(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP); + this.configInputDateFormatList = config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_PROP, ""); + + String inputDateFormatDelimiter = this.config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMETER_REGEX_PROP, ",").trim(); + inputDateFormatDelimiter = inputDateFormatDelimiter.isEmpty() ? "," : inputDateFormatDelimiter; + this.configInputDateFormatDelimiter = inputDateFormatDelimiter; + + if (inputTimeZone != null && !inputTimeZone.trim().isEmpty()) { + this.inputDateTimeZone = DateTimeZone.forTimeZone(TimeZone.getTimeZone(config.getString(Config.TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP, ""))); + } else { + this.inputDateTimeZone = null; + } + if (outputTimeZone != null && !outputTimeZone.trim().isEmpty()) { + this.outputDateTimeZone = DateTimeZone.forTimeZone(TimeZone.getTimeZone(config.getString(Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP, ""))); + } else { + this.outputDateTimeZone = null; + } + + if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) { + DataSourceUtils.checkRequiredProperties(config, + Collections.singletonList(Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_PROP)); + } + } + + private DateTimeFormatter getInputDateFormatter() { + if (this.configInputDateFormatList.isEmpty()) { + throw new IllegalArgumentException(Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_PROP + " configuration is required"); + } + + DateTimeFormatter formatter = new DateTimeFormatterBuilder() + .append( + null, + Arrays.asList( + this.configInputDateFormatList.split(this.configInputDateFormatDelimiter)) + .stream() + .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 HoodieKey getKey(GenericRecord record) { + Object partitionVal = DataSourceUtils.getNestedFieldVal(record, partitionPathField, true); + if (partitionVal == null) { + partitionVal = 1L; + } + + DateTimeFormatter inputFormatter = this.getInputDateFormatter(); + DateTimeFormatter partitionFormatter = DateTimeFormat.forPattern(outputDateFormat); + if (this.outputDateTimeZone != null) { + partitionFormatter = partitionFormatter.withZone(outputDateTimeZone); + } + + try { + long unixTime; Review comment: wonder if we can reuse some code by having this class extend the TimestampBasedKeyGenerator.. ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/keygen/MultiFormatTimestampBasedKeyGenerator.java ########## @@ -0,0 +1,183 @@ +/* + * 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.utilities.keygen; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.util.TypedProperties; +import org.apache.hudi.exception.HoodieKeyException; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.TimeZone; + +import org.joda.time.DateTime; +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; + +/** + * Key generator, that relies on timestamps for partitioning field. Still picks record key by name. + */ +public class MultiFormatTimestampBasedKeyGenerator extends SimpleKeyGenerator { + + enum TimestampType implements Serializable { + UNIX_TIMESTAMP, DATE_STRING, MIXED, EPOCHMILLISECONDS + } + + private final TimestampType timestampType; + private final String outputDateFormat; + private final String configInputDateFormatList; + private final String configInputDateFormatDelimiter; + + + // TimeZone detailed settings reference + // https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html + private final DateTimeZone inputDateTimeZone; + private final DateTimeZone outputDateTimeZone; + + /** + * 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 TIMESTAMP_INPUT_DATE_FORMAT_LIST_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformatlist"; + private static final String TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMETER_REGEX_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformatlistdelimiterregex"; + private static final String TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.input.timezone"; + + private static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.dateformat"; + private static final String TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.timezone"; + } + + public MultiFormatTimestampBasedKeyGenerator(TypedProperties config) { + super(config); + DataSourceUtils.checkRequiredProperties(config, + Arrays.asList(Config.TIMESTAMP_TYPE_FIELD_PROP, Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP)); + + String inputTimeZone = config.getString(Config.TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP, ""); + String outputTimeZone = config.getString(Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP, ""); + + this.timestampType = TimestampType.valueOf(config.getString(Config.TIMESTAMP_TYPE_FIELD_PROP)); + this.outputDateFormat = config.getString(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP); + this.configInputDateFormatList = config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_PROP, ""); + + String inputDateFormatDelimiter = this.config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMETER_REGEX_PROP, ",").trim(); + inputDateFormatDelimiter = inputDateFormatDelimiter.isEmpty() ? "," : inputDateFormatDelimiter; + this.configInputDateFormatDelimiter = inputDateFormatDelimiter; + + if (inputTimeZone != null && !inputTimeZone.trim().isEmpty()) { + this.inputDateTimeZone = DateTimeZone.forTimeZone(TimeZone.getTimeZone(config.getString(Config.TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP, ""))); + } else { + this.inputDateTimeZone = null; + } + if (outputTimeZone != null && !outputTimeZone.trim().isEmpty()) { + this.outputDateTimeZone = DateTimeZone.forTimeZone(TimeZone.getTimeZone(config.getString(Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP, ""))); + } else { + this.outputDateTimeZone = null; + } + + if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) { + DataSourceUtils.checkRequiredProperties(config, + Collections.singletonList(Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_PROP)); + } + } + + private DateTimeFormatter getInputDateFormatter() { + if (this.configInputDateFormatList.isEmpty()) { + throw new IllegalArgumentException(Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_PROP + " configuration is required"); + } + + DateTimeFormatter formatter = new DateTimeFormatterBuilder() + .append( + null, + Arrays.asList( + this.configInputDateFormatList.split(this.configInputDateFormatDelimiter)) + .stream() + .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 HoodieKey getKey(GenericRecord record) { + Object partitionVal = DataSourceUtils.getNestedFieldVal(record, partitionPathField, true); + if (partitionVal == null) { + partitionVal = 1L; + } + + DateTimeFormatter inputFormatter = this.getInputDateFormatter(); + DateTimeFormatter partitionFormatter = DateTimeFormat.forPattern(outputDateFormat); + if (this.outputDateTimeZone != null) { + partitionFormatter = partitionFormatter.withZone(outputDateTimeZone); + } + + try { + long unixTime; + if (partitionVal instanceof Double) { + unixTime = ((Double) partitionVal).longValue(); + } else if (partitionVal instanceof Float) { + unixTime = ((Float) partitionVal).longValue(); + } else if (partitionVal instanceof Long) { + unixTime = (Long) partitionVal; + } else if (partitionVal instanceof CharSequence) { + DateTime parsedDateTime = inputFormatter.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()); + } + + unixTime = inputFormatter.parseDateTime(partitionVal.toString()).getMillis() / 1000; + } else { + throw new HoodieNotSupportedException( + "Unexpected type for partition field: " + partitionVal.getClass().getName()); + } + DateTime timestamp = this.timestampType == TimestampType.EPOCHMILLISECONDS ? new DateTime(unixTime, outputDateTimeZone) : new DateTime(unixTime * 1000, outputDateTimeZone); Review comment: There is another PR which changes the millisecond handling more gracefully.. ########## File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/TestMultiFormatTimestampBasedKeyGenerator.java ########## @@ -0,0 +1,205 @@ +/* + * 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.utilities; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.util.SchemaTestUtil; +import org.apache.hudi.common.util.TypedProperties; +import org.apache.hudi.utilities.keygen.MultiFormatTimestampBasedKeyGenerator; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + +public class TestMultiFormatTimestampBasedKeyGenerator { + private GenericRecord baseRecord; + private TypedProperties properties = new TypedProperties(); + + @Before + public void initialize() throws IOException { + Schema schema = SchemaTestUtil.getTimestampEvolvedSchema(); + baseRecord = SchemaTestUtil + .generateAvroRecordFromJson(schema, 1, "001", "f1"); + + properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "field1"); + properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "createTime"); + properties.setProperty(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(), "false"); + } + + private TypedProperties getBaseKeyConfig(String timestampType, String inputFormatList, String inputFormatDelimiterRegex, String inputTimezone, String outputFormat, String outputTimezone) { + if (timestampType != null) { + properties.setProperty("hoodie.deltastreamer.keygen.timebased.timestamp.type", timestampType); + } + if (inputFormatList != null) { + properties.setProperty("hoodie.deltastreamer.keygen.timebased.input.dateformatlist", inputFormatList); + } + if (inputFormatDelimiterRegex != null) { + properties.setProperty("hoodie.deltastreamer.keygen.timebased.input.dateformatlistdelimiterregex", inputFormatDelimiterRegex); + } + if (inputTimezone != null) { + properties.setProperty("hoodie.deltastreamer.keygen.timebased.input.timezone", inputTimezone); + } + if (outputFormat != null) { + properties.setProperty("hoodie.deltastreamer.keygen.timebased.output.dateformat", outputFormat); + } + if (outputTimezone != null) { + properties.setProperty("hoodie.deltastreamer.keygen.timebased.output.timezone", outputTimezone); + } + + return properties; + } + + @Test + public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsUTC() { + baseRecord.put("createTime", "2020-04-01T13:01:33.428Z"); + properties = this.getBaseKeyConfig( + "DATE_STRING", + "yyyy-MM-dd'T'HH:mm:ss.SSSZ", + "", + "", + "yyyyMMddHH", + "UTC"); + HoodieKey hk1 = new MultiFormatTimestampBasedKeyGenerator(properties).getKey(baseRecord); + + assertEquals("2020040113", hk1.getPartitionPath()); + } + + @Test + public void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAsInputDateTimeZone() { + baseRecord.put("createTime", "2020-04-01T13:01:33.428Z"); + properties = this.getBaseKeyConfig( + "DATE_STRING", + "yyyy-MM-dd'T'HH:mm:ss.SSSZ", + "", + "", + "yyyyMMddHH", + ""); + HoodieKey hk1 = new MultiFormatTimestampBasedKeyGenerator(properties).getKey(baseRecord); + + assertEquals("2020040113", hk1.getPartitionPath()); + } + + @Test + public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsUTC() { + baseRecord.put("createTime", "2020-04-01T13:01:33.428Z"); + properties = this.getBaseKeyConfig( + "DATE_STRING", + "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ", + "", + "", + "yyyyMMddHH", + "UTC"); + HoodieKey hk1 = new MultiFormatTimestampBasedKeyGenerator(properties).getKey(baseRecord); + + assertEquals("2020040113", hk1.getPartitionPath()); + } + + @Test + public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAsUTC() { + baseRecord.put("createTime", "2020-04-01T13:01:33Z"); + properties = this.getBaseKeyConfig( + "DATE_STRING", + "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ", + "", + "", + "yyyyMMddHH", + "UTC"); + HoodieKey hk1 = new MultiFormatTimestampBasedKeyGenerator(properties).getKey(baseRecord); + + assertEquals("2020040113", hk1.getPartitionPath()); + } + + @Test + public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputTimezoneAsUTC() { + baseRecord.put("createTime", "2020-04-01T13:01:33-05:00"); + properties = this.getBaseKeyConfig( + "DATE_STRING", + "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ", + "", + "", + "yyyyMMddHH", + "UTC"); + HoodieKey hk1 = new MultiFormatTimestampBasedKeyGenerator(properties).getKey(baseRecord); + + assertEquals("2020040118", hk1.getPartitionPath()); + } + + @Test + public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_OutputTimezoneAsUTC() { + baseRecord.put("createTime", "2020-04-01T13:01:33.123-05:00"); + properties = this.getBaseKeyConfig( + "DATE_STRING", + "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ", + "", + "", + "yyyyMMddHH", + "UTC"); + HoodieKey hk1 = new MultiFormatTimestampBasedKeyGenerator(properties).getKey(baseRecord); + + assertEquals("2020040118", hk1.getPartitionPath()); + } + + @Test + public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsEST() { + baseRecord.put("createTime", "2020-04-01T13:01:33.123Z"); + properties = this.getBaseKeyConfig( + "DATE_STRING", + "yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ", + "", + "", + "yyyyMMddHH", + "EST"); + HoodieKey hk1 = new MultiFormatTimestampBasedKeyGenerator(properties).getKey(baseRecord); + + assertEquals("2020040109", hk1.getPartitionPath()); + } + + @Test(expected = Exception.class) Review comment: test for a specific exception class? ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/keygen/MultiFormatTimestampBasedKeyGenerator.java ########## @@ -0,0 +1,183 @@ +/* + * 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.utilities.keygen; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.util.TypedProperties; +import org.apache.hudi.exception.HoodieKeyException; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.TimeZone; + +import org.joda.time.DateTime; +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; + +/** + * Key generator, that relies on timestamps for partitioning field. Still picks record key by name. + */ +public class MultiFormatTimestampBasedKeyGenerator extends SimpleKeyGenerator { + + enum TimestampType implements Serializable { + UNIX_TIMESTAMP, DATE_STRING, MIXED, EPOCHMILLISECONDS + } + + private final TimestampType timestampType; + private final String outputDateFormat; + private final String configInputDateFormatList; + private final String configInputDateFormatDelimiter; + + + // TimeZone detailed settings reference + // https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html + private final DateTimeZone inputDateTimeZone; + private final DateTimeZone outputDateTimeZone; + + /** + * 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 TIMESTAMP_INPUT_DATE_FORMAT_LIST_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformatlist"; + private static final String TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMETER_REGEX_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformatlistdelimiterregex"; + private static final String TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.input.timezone"; + + private static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.dateformat"; + private static final String TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.timezone"; + } + + public MultiFormatTimestampBasedKeyGenerator(TypedProperties config) { + super(config); + DataSourceUtils.checkRequiredProperties(config, + Arrays.asList(Config.TIMESTAMP_TYPE_FIELD_PROP, Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP)); + + String inputTimeZone = config.getString(Config.TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP, ""); + String outputTimeZone = config.getString(Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP, ""); + + this.timestampType = TimestampType.valueOf(config.getString(Config.TIMESTAMP_TYPE_FIELD_PROP)); + this.outputDateFormat = config.getString(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP); + this.configInputDateFormatList = config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_PROP, ""); + + String inputDateFormatDelimiter = this.config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMETER_REGEX_PROP, ",").trim(); + inputDateFormatDelimiter = inputDateFormatDelimiter.isEmpty() ? "," : inputDateFormatDelimiter; + this.configInputDateFormatDelimiter = inputDateFormatDelimiter; + + if (inputTimeZone != null && !inputTimeZone.trim().isEmpty()) { Review comment: `inputTimeZone` wont be null at all since `""` is the default value? ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/keygen/MultiFormatTimestampBasedKeyGenerator.java ########## @@ -0,0 +1,183 @@ +/* + * 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.utilities.keygen; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.util.TypedProperties; +import org.apache.hudi.exception.HoodieKeyException; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.TimeZone; + +import org.joda.time.DateTime; +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; + +/** + * Key generator, that relies on timestamps for partitioning field. Still picks record key by name. + */ +public class MultiFormatTimestampBasedKeyGenerator extends SimpleKeyGenerator { + + enum TimestampType implements Serializable { + UNIX_TIMESTAMP, DATE_STRING, MIXED, EPOCHMILLISECONDS + } + + private final TimestampType timestampType; + private final String outputDateFormat; + private final String configInputDateFormatList; + private final String configInputDateFormatDelimiter; + + + // TimeZone detailed settings reference + // https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html + private final DateTimeZone inputDateTimeZone; + private final DateTimeZone outputDateTimeZone; + + /** + * 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 TIMESTAMP_INPUT_DATE_FORMAT_LIST_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformatlist"; + private static final String TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMETER_REGEX_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformatlistdelimiterregex"; Review comment: typo: DELIMITER ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
