Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,10 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.TimeZone;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.SimpleKeyGenerator;
Expand All @@ -48,6 +51,7 @@ enum TimestampType implements Serializable {

private final String outputDateFormat;

private final List<String> fields;

/**
* Supported configs
Expand Down Expand Up @@ -79,6 +83,9 @@ public TimestampBasedKeyGenerator(TypedProperties config) {
config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
this.inputDateFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@afilipchik : Can you document some examples of how this is configured and used

fields = Stream.of(recordKeyField.split(","))
.map(String::trim).collect(Collectors.toList());
}

@Override
Expand All @@ -102,7 +109,12 @@ public HoodieKey getKey(GenericRecord record) {
"Unexpected type for partition field: " + partitionVal.getClass().getName());
}

return new HoodieKey(DataSourceUtils.getNestedFieldValAsString(record, recordKeyField),
return new HoodieKey(
fields.stream()
.map(
recordKeyField ->
DataSourceUtils.getNestedFieldValAsString(record, recordKeyField))
.collect(Collectors.joining(".")),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We are joining by "." here. Is there any assumption here ? Wanted to see if this is generic enough to be put it in this class.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it can be anything.
HoodieKey(String recordKey, String partitionPath)

so I'm just creating a long key: bla.bla1.bla2. Can be bla:bla1:bla2

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi I think this kind of key creates ambiguity refer this PR for details #728

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@bvaradar any thoughts? lets make a call on this PR?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jaimin-shah : Wondering if there is a necessity to decode the generated record key. If this is a one-way concatenation of fields to record-key, it should be fine. right ? As we are storing individual fields that constitute the record key separately, there wont be any need to decode this record key and ambiguity during decoding should be ok. Isn't it ?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@bvaradar By ambiguity I meant two different records having same key. For example US, .A => US..A and US. , A => US..A Keeping recordKeyField as part of key resolves this. Although I agree these kind of cases are quite rare.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense @jaimin-shah. Didn't think about this possibility. @afilipchik : Based on this information, this may not be safe in general sense.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hey, not sure I understood the edge case example. Here is how we are using it:
We have an object with id and a version. Version suppose to monotonically increase, and we want to be able to dedub based on key and the version (there should be at most 1 object with the same id and version).

Our config looks like:
hoodie.datasource.write.recordkey.field=object.id,object.version for insert table and

What can go wrong with this one?

Copy link
Contributor

@jaimin-shah jaimin-shah Sep 24, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @afilipchik for your use case I don't think there will be any problem but there can be problem when there no restrictions on recordKeyField ( e.g. Version suppose to monotonically increase ). You can take a look at https://github.com/apache/incubator-hudi/blob/master/hudi-spark/src/main/java/org/apache/hudi/ComplexKeyGenerator.java it has generic implementation of complex key for quite similar requirement.

partitionPathFormat.format(new Date(unixTime * 1000)));
} catch (ParseException pe) {
throw new HoodieDeltaStreamerException(
Expand Down