Skip to content

KAFKA-6684: Cast transform bytes#4820

Closed
amitsela wants to merge 2 commits intoapache:trunkfrom
amitsela:cast-transform-bytes
Closed

KAFKA-6684: Cast transform bytes#4820
amitsela wants to merge 2 commits intoapache:trunkfrom
amitsela:cast-transform-bytes

Conversation

@amitsela
Copy link
Copy Markdown
Member

@amitsela amitsela commented Apr 3, 2018

Allow to cast LogicalType to string by calling the serialized (Java) object's toString().

Added tests for BigDecimal and Date as whole record and as fields.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@amitsela
Copy link
Copy Markdown
Member Author

amitsela commented Apr 3, 2018

This helps avoid dealing with bytes in downstream clients such as Python (no Avro) and loading csv/json into downstream databases.

@amitsela
Copy link
Copy Markdown
Member Author

amitsela commented Apr 3, 2018

R: @rhauch @ewencp @hachikuji

@amitsela
Copy link
Copy Markdown
Member Author

amitsela commented Apr 3, 2018

Marked as [WIP] cause Date.toString() doesn't use UTC I wanted to your opinion on that - should I make it print in UTC? like a proper ISO8601: yyyy-mm-ddTHH:MM:SSZ? or with msec yyyy-mm-ddTHH:MM:SS.fZ?

@amitsela amitsela force-pushed the cast-transform-bytes branch from 7c53447 to 5b7166d Compare April 3, 2018 17:09
@amitsela
Copy link
Copy Markdown
Member Author

amitsela commented Apr 3, 2018

fixed .* imports and rebased the commit as a fixup

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Most of this looks fine, but I think just adding BYTES to this isn't quite right. The way this was originally written assumes that all types this transformation supports have a sensible translation to every other type (which is why it stuck to numbers, boolean, and strings). I think you're trying to add support for an additional set of casts that make sense (e.g. most things have a sensible toString()), but now the input and output are not symmetric -- some things make sense as an input, but not as an output (e.g. BYTES). So a single list might not be sufficient to do these checks correctly (e.g. as it is today, if you tried to do something like bytes:string conversion, instead of catching it as a config issue and giving an easy to understand error, it would be caught later and just say it encountered an unexpected type). If the symmetry is no longer going to be there, I think we need to separate out what we support as input and what we support as output (and I'm not even sure that will be sufficient, but it seems like the minimum we'd need).

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Makes sense. Let me check it out and push an update.

@amitsela
Copy link
Copy Markdown
Member Author

@ewencp I think this makes more sense now.
What did you mean by

I'm not even sure that will be sufficient

What do you think is still missing?

On a separate note, WDYT about my note on Date.toString()?

@asfgit
Copy link
Copy Markdown

asfgit commented May 23, 2018

FAILURE
7773 tests run, 1 skipped, 2 failed.
--none--

@amitsela amitsela force-pushed the cast-transform-bytes branch from a71579d to cbfa8f9 Compare May 23, 2018 12:52
@rhauch
Copy link
Copy Markdown
Contributor

rhauch commented May 25, 2018

I don't particular like Date.toString() which outputs strings of the form dow mon dd hh:mm:ss zzz yyyy, or would it be better to use the standard ISO-8601 format? But then this begs the question whether this should know about Connect's Date, Timestamp, and Time logical types.

@amitsela
Copy link
Copy Markdown
Member Author

amitsela commented May 25, 2018

True. To me it makes sense to say that casting to string uses the Object's toString() implementation, whatever that is.
For Date, there might not be a good reason to cast to string, especially when in serialized form it is still useful.
Gonna remove the [WIP] for this now.

@amitsela amitsela changed the title KAFKA-6684 [WIP]: Cast transform bytes KAFKA-6684: Cast transform bytes May 25, 2018
@amitsela
Copy link
Copy Markdown
Member Author

@ewencp @rhauch any thoughts? this PR opened on March 11, it's been a while.

@amitsela amitsela force-pushed the cast-transform-bytes branch from cbfa8f9 to 50783f8 Compare August 23, 2018 15:54
@amitsela
Copy link
Copy Markdown
Member Author

Resolved conflicts, squashed, and rebased.

@rayokota
Copy link
Copy Markdown
Contributor

As @ewencp suggested, for consistency perhaps we can use the same string formats as SimpleHeaderConverter: https://github.com/apache/kafka/blob/trunk/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java#L692-L695

Having them be inconsistent would be confusing IMHO.

@amitsela
Copy link
Copy Markdown
Member Author

Sure thing! I'll update the PR.

@amitsela
Copy link
Copy Markdown
Member Author

@rayokota @ewencp now it uses Values date formatter for consistency.
Also fixed the date pattern for year and day which had a bug.

I assume the goal would be to make the Cast transform use the Values conversion methods? makes sense but feels out-of-scope for this PR anyway.

@amitsela
Copy link
Copy Markdown
Member Author

retest this please

@rayokota
Copy link
Copy Markdown
Contributor

Thanks, @amitsela . Good catch on the broken date pattern. LGTM.

@ewencp , how does this look?

Copy link
Copy Markdown
Contributor

@ewencp ewencp left a comment

Choose a reason for hiding this comment

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

seems very close... just a couple of minor comments

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

hmm, is this just an underlying bug? should this bit be backported?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

yeah that's just a bug I happened to stumble upon, probably should be backported.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

this is a public API change -- everything public in the Connect API is.

do we actually need this here? Scanning through this PR, it looks like it is used in one specific class and then test classes in the same package. Making it package-protected where it is actually used seems sufficient to me, no?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

package-private is not enough since Cast is in package org.apache.kafka.connect.transforms tries to use Values#dateFormatter(java.util.Date) in org.apache.kafka.connect.data.
I think it's fine since we'd want anyone/anything that's manipulating "data values" to use the same formatter, no? so we give access to that method.

infer schema type from schema as fallback

tests

split cast validation to input/output

use Values date fomratter if java.util.Date
@amitsela amitsela force-pushed the cast-transform-bytes branch from 18cbc78 to cfd9cb6 Compare September 10, 2018 12:56
@amitsela
Copy link
Copy Markdown
Member Author

@ewencp I squashed and split the bug fix into it's own commit so you can cherry pick it easily.

@amitsela
Copy link
Copy Markdown
Member Author

amitsela commented Sep 10, 2018

test failure seems unrelated, I'll kick Jenkins to try again

@amitsela
Copy link
Copy Markdown
Member Author

jenkins retest this please

public class CastTest {
private final Cast<SourceRecord> xformKey = new Cast.Key<>();
private final Cast<SourceRecord> xformValue = new Cast.Value<>();
private static final long MILLIS_PER_DAY = 24 * 60 * 60 * 1000;
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

minor: consider using TimeUnit.DAYS.toMillis(1)

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Could you do a static import for that Values class, then?

Copy link
Copy Markdown
Member Author

@amitsela amitsela Sep 17, 2018

Choose a reason for hiding this comment

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

If a the committer wants, he can. This PR had a ridiculous back and forth and had been stretched for ages compared to what it covers.

@lindong28
Copy link
Copy Markdown
Member

Hey @ewencp, if this PR is close to be merged, would you have time to review it for 2.1.0 release?

ewencp pushed a commit that referenced this pull request Oct 1, 2018
Switches to normal year format instead of week date years and day of month instead of day of year.

This is directly from #4820, but separated into a different JIRA/PR to keep the fixes independent. Original authorship should be maintained in the commit.

Author: Amit Sela <amitsela33@gmail.com>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #5718 from ewencp/fix-header-converter-date-format

(cherry picked from commit c1457be)
Signed-off-by: Ewen Cheslack-Postava <me@ewencp.org>
ewencp pushed a commit that referenced this pull request Oct 1, 2018
Switches to normal year format instead of week date years and day of month instead of day of year.

This is directly from #4820, but separated into a different JIRA/PR to keep the fixes independent. Original authorship should be maintained in the commit.

Author: Amit Sela <amitsela33@gmail.com>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #5718 from ewencp/fix-header-converter-date-format

(cherry picked from commit c1457be)
Signed-off-by: Ewen Cheslack-Postava <me@ewencp.org>
ewencp pushed a commit that referenced this pull request Oct 1, 2018
Switches to normal year format instead of week date years and day of month instead of day of year.

This is directly from #4820, but separated into a different JIRA/PR to keep the fixes independent. Original authorship should be maintained in the commit.

Author: Amit Sela <amitsela33@gmail.com>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #5718 from ewencp/fix-header-converter-date-format
Copy link
Copy Markdown
Contributor

@ewencp ewencp left a comment

Choose a reason for hiding this comment

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

@amitsela Thank you! Merging to trunk for 2.1.0.

Sorry for the (slow) back and forth. We prefer to be very certain of patches than to merge them too quickly, and in this case review throughput was limited.

I think there's still substantial follow-up to be done here re: handling of types and test coverage, but this is a significant improvement regardless.


private static String castToString(Object value) {
return value.toString();
if (value instanceof java.util.Date) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Given we're doing this in the core api/transform code now, this probably warrants a more generalizable approach. On first iteration, the logical type classes (in o.a.k.connect.data) were sufficient, but it seems they may not quite expose enough info. In particular, other than the fromLogical return type and toLogical parameter type, they may not make it clear enough what the conversion to primitive type is. Also, they probably aren't sufficiently isolated.

We shouldn't block getting this in since Values currently tries to do its best to determine the correct type, though even that is subject to failure modes (e.g. @rhauch just because a timestamp falls on a day or within the first day after 0AD doesn't mean they actually are that type -- we should really be passing through schema info to determine that). Given lack of that info provided to currently public APIs, I don't think there are real compatibility issues -- in the future we'd want a more correct alternative we'd provide a separate multi-argument version that includes schema info and fallback to the single-argument version.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

That makes sense.
One why to do this would be to create an abstraction o.a.k.connect.data.Logical which would avoid the whole "if isinstaceof.." code in Cast or any other place you're looking to convert to/from logical.

@ewencp ewencp closed this in fd44dc7 Oct 1, 2018
@amitsela
Copy link
Copy Markdown
Member Author

amitsela commented Oct 1, 2018

I agree both with being thorough, and with the fact that there's a lot that could be done to further improve this area of the code.
Having said that, this PR was very focused and reviews (which weren't a lot of code at any point) sometimes took 1-2 months, but I guess that's due to limited committer resources.
Anyway, thanks!

@amitsela amitsela deleted the cast-transform-bytes branch October 1, 2018 16:17
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
Switches to normal year format instead of week date years and day of month instead of day of year.

This is directly from apache#4820, but separated into a different JIRA/PR to keep the fixes independent. Original authorship should be maintained in the commit.

Author: Amit Sela <amitsela33@gmail.com>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes apache#5718 from ewencp/fix-header-converter-date-format
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
Allow to cast LogicalType to string by calling the serialized (Java) object's toString().

Added tests for `BigDecimal` and `Date` as whole record and as fields.

Author: Amit Sela <amitsela33@gmail.com>

Reviewers: Randall Hauch <rhauch@gmail.com>, Robert Yokota <rayokota@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes apache#4820 from amitsela/cast-transform-bytes
rhauch pushed a commit that referenced this pull request Oct 3, 2019
Allow to cast LogicalType to string by calling the serialized (Java) object's toString().

Added tests for `BigDecimal` and `Date` as whole record and as fields.

Author: Amit Sela <amitsela33@gmail.com>

Reviewers: Randall Hauch <rhauch@gmail.com>, Robert Yokota <rayokota@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4820 from amitsela/cast-transform-bytes
rhauch pushed a commit that referenced this pull request Oct 3, 2019
Allow to cast LogicalType to string by calling the serialized (Java) object's toString().

Added tests for `BigDecimal` and `Date` as whole record and as fields.

Author: Amit Sela <amitsela33@gmail.com>

Reviewers: Randall Hauch <rhauch@gmail.com>, Robert Yokota <rayokota@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4820 from amitsela/cast-transform-bytes
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

8 participants