flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-8544) JSONKeyValueDeserializationSchema throws NPE when message key is null
Date Thu, 01 Mar 2018 09:18:00 GMT

    [ https://issues.apache.org/jira/browse/FLINK-8544?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16381719#comment-16381719

ASF GitHub Bot commented on FLINK-8544:

GitHub user BillLeecn reopened a pull request:


    [FLINK-8544] [Kafka Connector] Handle null message key in JSONKeyValueDeserializationSc…

    ## What is the purpose of the change
    This pull request fix a NPE thrown in JSONKeyValueDeserializationSchema when the message
key is null, allowing JSONKeyValueDeserializationSchema to be used to retrieve message metadata
even if the message key is null.
    The NPE is caused by deserializing the message key without verification.
    ## Brief change log
      - Check nullity before deserializing the message key.
    ## Verifying this change
    This change added tests and can be verified as follows:
      - Added unit tests for deserializing a message with null key.
    ## Does this pull request potentially affect one of the following parts:
      - Dependencies (does it add or upgrade a dependency): no
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
      - The serializers: no
      - The runtime per-record code paths (performance sensitive): yes
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing,
Yarn/Mesos, ZooKeeper: no
      - The S3 file system connector: no
    ## Documentation
      - Does this pull request introduce a new feature? no
      - If yes, how is the feature documented? not applicable

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/BillLeecn/flink flink-8544

Alternatively you can review and apply these changes as the patch at:


To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #5516
commit 2dca64f35b6c27e0620f89697378ceb3a8e5988c
Author: Bill Lee <bill.lee.y@...>
Date:   2018-02-17T09:50:16Z

    [FLINK-8544] Handle null message key in JSONKeyValueDeserializationSchema


> JSONKeyValueDeserializationSchema throws NPE when message key is null
> ---------------------------------------------------------------------
>                 Key: FLINK-8544
>                 URL: https://issues.apache.org/jira/browse/FLINK-8544
>             Project: Flink
>          Issue Type: Bug
>          Components: Kafka Connector
>    Affects Versions: 1.4.0
>            Reporter: Bill Lee
>            Priority: Major
>   Original Estimate: 1h
>  Remaining Estimate: 1h
> JSONKeyValueDeserializationSchema call Jaskon to deserialize the message key without
>  If a message with key == null is read, flink throws an NPE.
> {code:java}
> 	@Override
> 	public ObjectNode deserialize(byte[] messageKey, byte[] message, String topic, int partition,
long offset) throws IOException {
> 		if (mapper == null) {
> 			mapper = new ObjectMapper();
> 		}
> 		ObjectNode node = mapper.createObjectNode();
> 		node.set("key", mapper.readValue(messageKey, JsonNode.class)); // messageKey is not
validate against null.
> 		node.set("value", mapper.readValue(message, JsonNode.class));
> {code}
> The fix is very straightforward.
> {code:java}
> 		if (messageKey == null) {
> 			node.set("key", null)
> 		} else {
> 			node.set("key", mapper.readValue(messageKey, JsonNode.class));
> 		}
> {code}
> If it is appreciated, I would send a pull request.

This message was sent by Atlassian JIRA

View raw message