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-8181) Kafka producer's FlinkFixedPartitioner returns different partitions when a target topic is rescaled
Date Fri, 01 Dec 2017 15:14:00 GMT

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

ASF GitHub Bot commented on FLINK-8181:
---------------------------------------

Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5108#discussion_r154368266
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java
---
    @@ -68,6 +78,13 @@ public int partition(T record, byte[] key, byte[] value, String targetTopic,
int
     			partitions != null && partitions.length > 0,
     			"Partitions of the target topic is empty.");
     
    -		return partitions[parallelInstanceId % partitions.length];
    +		if (topicToFixedPartition.containsKey(targetTopic)) {
    --- End diff --
    
    @aljoscha yes, the semantics is a bit odd / needs some clarification before we move on.
I've been having a go at implementing state checkpointing for the `FlinkFixedPartitioner`
today, and for example one unclear case I bumped into was the following:
    Subtask 1 writes to partition X for "some-topic"
    Subtask 2 writes to partition Y for "some-topic"
    On restore and say the sink is rescaled to DOP of 1, should the single subtask continue
writing to partition X or Y for "some-topic"?
    
    Regarding the default Kafka behaviour:
    It's hash partitioning on the attached key for the records. I've also thought about using
that as the default instead of the fixed partitioner; see the relevant discussion here: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/FlinkKafkaProducerXX-td16951.html


> Kafka producer's FlinkFixedPartitioner returns different partitions when a target topic
is rescaled
> ---------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-8181
>                 URL: https://issues.apache.org/jira/browse/FLINK-8181
>             Project: Flink
>          Issue Type: Bug
>          Components: Kafka Connector
>    Affects Versions: 1.4.0, 1.5.0
>            Reporter: Tzu-Li (Gordon) Tai
>            Assignee: Tzu-Li (Gordon) Tai
>            Priority: Blocker
>             Fix For: 1.4.0, 1.5.0
>
>
> On fixing FLINK-6288 and migrating the original {{FlinkFixedPartitioner}} to the new
partitioning API (commit 9ed9b68397b51bfd2b0f6e532212a82f771641bd), the {{FlinkFixedPartitioner}}
no longer returns identical target partitions once a target topic is rescaled.
> This results in a behavioral regression when the {{FlinkFixedPartitioner}} is used.
> The {{FlinkFixedPartitionerTest}} should also be strengthened to cover the target topic
rescaling case.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message