-
Notifications
You must be signed in to change notification settings - Fork 2.5k
Add support for composite key #862
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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; | ||
|
|
@@ -48,6 +51,7 @@ enum TimestampType implements Serializable { | |
|
|
||
| private final String outputDateFormat; | ||
|
|
||
| private final List<String> fields; | ||
|
|
||
| /** | ||
| * Supported configs | ||
|
|
@@ -79,6 +83,9 @@ public TimestampBasedKeyGenerator(TypedProperties config) { | |
| config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP)); | ||
| this.inputDateFormat.setTimeZone(TimeZone.getTimeZone("GMT")); | ||
| } | ||
|
|
||
| fields = Stream.of(recordKeyField.split(",")) | ||
| .map(String::trim).collect(Collectors.toList()); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -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(".")), | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it can be anything. so I'm just creating a long key: bla.bla1.bla2. Can be bla:bla1:bla2
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @bvaradar any thoughts? lets make a call on this PR?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 ?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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: Our config looks like: What can go wrong with this one?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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( | ||
|
|
||
There was a problem hiding this comment.
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