Skip to content

KAFKA-6290: Support casting from logical types in cast transform#7371

Merged
rhauch merged 1 commit intoapache:trunkfrom
ncliang:CC-1345
Oct 3, 2019
Merged

KAFKA-6290: Support casting from logical types in cast transform#7371
rhauch merged 1 commit intoapache:trunkfrom
ncliang:CC-1345

Conversation

@ncliang
Copy link
Copy Markdown
Contributor

@ncliang ncliang commented Sep 19, 2019

This PR adds support for cast transforms to cast from logical types Date, Time, and Timestamp to
their internal int32 or int64 representations. Any valid cast on their internal representations
would also be applicable. For instance, Time has internal int32 representation, but can be cast to
int64 if desired.

Committer Checklist (excluded from commit message)

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

Copy link
Copy Markdown
Contributor

@rhauch rhauch left a comment

Choose a reason for hiding this comment

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

Thanks, @ncliang! Looks really good, but I have a question/suggestion below.

Comment thread connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java Outdated
Comment thread connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java Outdated
Copy link
Copy Markdown
Contributor

@rhauch rhauch left a comment

Choose a reason for hiding this comment

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

Looks great. A few more suggestions below.

Comment thread connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java Outdated
Copy link
Copy Markdown
Contributor

@rhauch rhauch left a comment

Choose a reason for hiding this comment

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

Thanks for the adjustments, @ncliang. A few more suggestions below, but otherwise looks good.

Finally, we need some indication of in which versions this should be fixed, since that tells us to which branches this PR, once merged, should be cherry-picked. Since the bug was first reported against 0.11.0.0, Ideally we should probably go back as far as we can, though it's not immediately apparent how far this PR will be easily cherry-picked.

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.

I think it'd be useful to verify the behavior of casting all of the logical types to strings, just so that we verify the formats (e.g., timestamps, times, and dates should use the ISO 8601 representation) and have some regression tests for the future to help ensure we maintain backward compatibility.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Done! I moved the castToString tests to their own testcase.

Looking back at release branches, it seems schema was not piped into castValueToType() until 2.1. Before that, we can not easily determine if we are casting from a logical type. So, I'd say easiest is to go back to 2.1. Otherwise, we will have to backport KAFKA-6684 as well.

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.

If that's the case, then we may need a separate PR to go back further than 2.1. I'd say we'd want to correct this issue (which has irked quite a few people) as far back as 1.0.

Copy link
Copy Markdown
Contributor Author

@ncliang ncliang Sep 23, 2019

Choose a reason for hiding this comment

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

#4820 cherry-picks pretty cleanly onto 1.0. Then, these changes apply cleanly. My vote would be to take both fixes for KAFKA-6684 and this fix as far back as 1.0. WDYT?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Ping @rhauch . WDYT about taking both KAFKA-6684 and this PR as far back as 1.0?

Copy link
Copy Markdown
Contributor

@rhauch rhauch left a comment

Choose a reason for hiding this comment

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

Thanks for responding so quickly, @ncliang. One more suggestion for the test code, but otherwise looks good.

This PR adds support for cast transforms to cast from logical types. Casting to numeric types will
produce the underlying numeric value represented in the desired type. For logical types represented
by underlying Java Date class, this means the milliseconds since EPOCH. For Decimal, this means the
underlying value. If the value does not fit in the desired target type, it may overflow.

Casting to String from Date, Time, and Timestamp types will produce their ISO 8601 representation.
Casting to String from Decimal will result in the value represented as a string. e.g. 1234 -> "1234"
@rhauch
Copy link
Copy Markdown
Contributor

rhauch commented Oct 3, 2019

Retest this, please.

Copy link
Copy Markdown
Contributor

@rhauch rhauch left a comment

Choose a reason for hiding this comment

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

LGTM. Thanks for the fix, @ncliang!

@rhauch rhauch merged commit ded1fb8 into apache:trunk Oct 3, 2019
rhauch pushed a commit that referenced this pull request Oct 3, 2019
Adds support for the Connect Cast transforms to cast from Connect logical types, such as DATE, TIME, TIMESTAMP, and DECIMAL. Casting to numeric types will produce the underlying numeric value represented in the desired type. For logical types represented by underlying Java Date class, this means the milliseconds since EPOCH. For Decimal, this means the underlying value. If the value does not fit in the desired target type, it may overflow.

Casting to String from Date, Time, and Timestamp types will produce their ISO 8601 representation. Casting to String from Decimal will result in the value represented as a string. e.g. 1234 -> "1234".

Author: Nigel Liang <nigel@nigelliang.com>
Reviewer: Randall Hauch <rhauch@gmail.com>
@ncliang ncliang deleted the CC-1345 branch October 3, 2019 20:08
rhauch pushed a commit that referenced this pull request Oct 3, 2019
Adds support for the Connect Cast transforms to cast from Connect logical types, such as DATE, TIME, TIMESTAMP, and DECIMAL. Casting to numeric types will produce the underlying numeric value represented in the desired type. For logical types represented by underlying Java Date class, this means the milliseconds since EPOCH. For Decimal, this means the underlying value. If the value does not fit in the desired target type, it may overflow.

Casting to String from Date, Time, and Timestamp types will produce their ISO 8601 representation. Casting to String from Decimal will result in the value represented as a string. e.g. 1234 -> "1234".

Author: Nigel Liang <nigel@nigelliang.com>
Reviewer: Randall Hauch <rhauch@gmail.com>
rhauch pushed a commit that referenced this pull request Oct 3, 2019
Adds support for the Connect Cast transforms to cast from Connect logical types, such as DATE, TIME, TIMESTAMP, and DECIMAL. Casting to numeric types will produce the underlying numeric value represented in the desired type. For logical types represented by underlying Java Date class, this means the milliseconds since EPOCH. For Decimal, this means the underlying value. If the value does not fit in the desired target type, it may overflow.

Casting to String from Date, Time, and Timestamp types will produce their ISO 8601 representation. Casting to String from Decimal will result in the value represented as a string. e.g. 1234 -> "1234".

Author: Nigel Liang <nigel@nigelliang.com>
Reviewer: Randall Hauch <rhauch@gmail.com>
rhauch pushed a commit that referenced this pull request Oct 3, 2019
Adds support for the Connect Cast transforms to cast from Connect logical types, such as DATE, TIME, TIMESTAMP, and DECIMAL. Casting to numeric types will produce the underlying numeric value represented in the desired type. For logical types represented by underlying Java Date class, this means the milliseconds since EPOCH. For Decimal, this means the underlying value. If the value does not fit in the desired target type, it may overflow.

Casting to String from Date, Time, and Timestamp types will produce their ISO 8601 representation. Casting to String from Decimal will result in the value represented as a string. e.g. 1234 -> "1234".

Author: Nigel Liang <nigel@nigelliang.com>
Reviewer: Randall Hauch <rhauch@gmail.com>
rhauch pushed a commit that referenced this pull request Oct 3, 2019
Adds support for the Connect Cast transforms to cast from Connect logical types, such as DATE, TIME, TIMESTAMP, and DECIMAL. Casting to numeric types will produce the underlying numeric value represented in the desired type. For logical types represented by underlying Java Date class, this means the milliseconds since EPOCH. For Decimal, this means the underlying value. If the value does not fit in the desired target type, it may overflow.

Casting to String from Date, Time, and Timestamp types will produce their ISO 8601 representation. Casting to String from Decimal will result in the value represented as a string. e.g. 1234 -> "1234".

Author: Nigel Liang <nigel@nigelliang.com>
Reviewer: Randall Hauch <rhauch@gmail.com>
@rhauch
Copy link
Copy Markdown
Contributor

rhauch commented Oct 3, 2019

Merged back to the 1.1 branch.

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.

3 participants