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

ASF GitHub Bot commented on KAFKA-5856:
---------------------------------------

GitHub user tombentley opened a pull request:

    https://github.com/apache/kafka/pull/3870

    KAFKA-5856: Add AdminClient.createPartitions()

    See KIP-195.
    
    The contribution is my original work and I license the work to the project 
under the project's open source license.
    
    This patch adds AdminClient.createPartitions() and the network protocol is
    uses. The broker-side algorithm is as follows:
    
    1. KafkaApis makes some initial checks on the request, then delegates to the
       new AdminManager.createPartitions() method.
    2. AdminManager.createPartitions() performs some validation then delegates 
to
       AdminUtils.addPartitions().
    
    Aside: I felt it was safer to add the extra validation in
    AdminManager.createPartitions() than in AdminUtils.addPartitions() since the
    latter is used on other code paths which might fail differently with the
    introduction of extra checks.
    
    3. AdminUtils.addPartitions() does its own checks and adds the partitions.
    4. AdminManager then uses the existing topic purgatory to wait for the
       PartitionInfo available from the metadata cache to become consistent with
       the new total number of partitions.
    
    The messages of exceptions thrown in AdminUtils affecting this new API have
    been made consistent with initial capital letter and terminating period.
    A few have been reworded for clarity. I've also standardized on using
    String.format().
    
    cc @ijuma

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

    $ git pull https://github.com/tombentley/kafka 
KAFKA-5856-AdminClient.createPartitions

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

    https://github.com/apache/kafka/pull/3870.patch

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

    This closes #3870
    
----
commit ab07f15a794c385cbfdecd33a5a44c7725e8d103
Author: Tom Bentley <tbent...@redhat.com>
Date:   2017-09-15T09:50:59Z

    KAFKA-5856: Add AdminClient.createPartitions()
    
    See KIP-195.
    
    This patch adds AdminClient.createPartitions() and the network protocol is
    uses. The broker-side algorithm is as follows:
    
    1. KafkaApis makes some initial checks on the request, then delegates to the
       new AdminManager.createPartitions() method.
    2. AdminManager.createPartitions() performs some validation then delegates 
to
       AdminUtils.addPartitions().
    
    Aside: I felt it was safer to add the extra validation in
    AdminManager.createPartitions() than in AdminUtils.addPartitions() since the
    latter is used on other code paths which might fail differently with the
    introduction of extra checks.
    
    3. AdminUtils.addPartitions() does its own checks and adds the partitions.
    4. AdminManager then uses the existing topic purgatory to wait for the
       PartitionInfo available from the metadata cache to become consistent with
       the new total number of partitions.
    
    The messages of exceptions thrown in AdminUtils affecting this new API have
    been made consistent with initial capital letter and terminating period.
    A few have been reworded for clarity. I've also standardized on using
    String.format().

----


> Add AdminClient.createPartitions()
> ----------------------------------
>
>                 Key: KAFKA-5856
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5856
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Tom Bentley
>            Assignee: Tom Bentley
>              Labels: kip
>
> It should be possible to increase the partition count using the AdminClient. 
> See 
> [KIP-195|https://cwiki.apache.org/confluence/display/KAFKA/KIP-195%3A+AdminClient.increasePartitions]



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

Reply via email to