[FLINK-7440] [kinesis] Add various eager serializability checks on to Kinesis connector#4537
Closed
tzulitai wants to merge 2 commits intoapache:masterfrom
Closed
[FLINK-7440] [kinesis] Add various eager serializability checks on to Kinesis connector#4537tzulitai wants to merge 2 commits intoapache:masterfrom
tzulitai wants to merge 2 commits intoapache:masterfrom
Conversation
…on schema in FlinkKinesisConsumer This commit also adds tests for verifying that the FlinkKinesisConsumer itself is serializable.
…oner is serializable in FlinkKinesisProducer This commit also adds a test to verify that the FlinkKinesisProducer is serializable.
Contributor
Author
|
R: @bowenli86 would you like to take a look at the changes here? |
zentol
reviewed
Aug 14, 2017
| // check the configuration properties for any conflicting settings | ||
| KinesisConfigUtil.validateConsumerConfiguration(this.configProps); | ||
|
|
||
| checkArgument( |
Contributor
There was a problem hiding this comment.
How about first checking for null, and then for serializability? (Just seem more intuitive to me)
bowenli86
reviewed
Aug 14, 2017
| return fakeRestoredState; | ||
| } | ||
|
|
||
| private final class NonSerializableDeserializationSchema implements KinesisDeserializationSchema<String> { |
Member
There was a problem hiding this comment.
Can you please add a comment explaining why this is not serializable and why the following one is serializable? Basically pointing out this class is not static.
Member
|
@tzulitai LGTM! |
Member
|
Once this is checked in, we'll be more confident that #4473 is all good and it doesn't break anything |
Contributor
Author
|
@bowenli86 @zentol Thanks a lot for the reviews! I will address your comments and merge this. |
tzulitai
added a commit
to tzulitai/flink
that referenced
this pull request
Aug 15, 2017
…oner is serializable in FlinkKinesisProducer This commit also adds a test to verify that the FlinkKinesisProducer is serializable. This closes apache#4537.
tzulitai
added a commit
to tzulitai/flink
that referenced
this pull request
Aug 16, 2017
…oner is serializable in FlinkKinesisProducer This commit also adds a test to verify that the FlinkKinesisProducer is serializable. This closes apache#4537.
tzulitai
added a commit
to tzulitai/flink
that referenced
this pull request
Aug 18, 2017
…oner is serializable in FlinkKinesisProducer This commit also adds a test to verify that the FlinkKinesisProducer is serializable. This closes apache#4537.
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
What is the purpose of the change
This is a minor improvement for user experience for the Kinesis consumer and producer.
If the provided
KinesisDeserializationSchema,KinesisSerializationSchema,KinesisPartitioneris not serializable, a more clear error message is shown instead of some vague "the implementation of RichSunkFunction is not serializable".Brief change log
Changes are broken up into 2 commits:
KinesisDeserializationSchema, with better error msgsKinesisSerializationSchemaandKinesisPartitioner, with better error msgs.Verifying this change
Added new tests to both
FlinkKinesisConsumerTestandFlinkKinesisProducertest, to verify that the expected error message is thrown (and also not thrown if the provided arg instances are serializable). Also included tests to check thatFlinkKinesisConsumerandFlinkKinesisProducerthemselves are serializable.Does this pull request potentially affect one of the following parts:
@Public(Evolving): noDocumentation